Merge branch 'master' of github.com:ClickHouse/ClickHouse into enable-coverage-for-debug-build

This commit is contained in:
Alexey Milovidov 2024-02-09 22:52:49 +01:00
commit a7c30098b6
98 changed files with 2314 additions and 321 deletions

View File

@ -327,15 +327,19 @@ jobs:
run_command: |
python3 build_report_check.py "$CHECK_NAME"
MarkReleaseReady:
needs: [RunConfig, BuilderBinDarwin, BuilderBinDarwinAarch64, BuilderDebRelease, BuilderDebAarch64]
if: ${{ !failure() && !cancelled() }}
uses: ./.github/workflows/reusable_test.yml
with:
test_name: Mark Commit Release Ready
runner_type: style-checker
data: ${{ needs.RunConfig.outputs.data }}
run_command: |
python3 mark_release_ready.py
needs:
- BuilderBinDarwin
- BuilderBinDarwinAarch64
- BuilderDebRelease
- BuilderDebAarch64
runs-on: [self-hosted, style-checker]
steps:
- name: Check out repository code
uses: ClickHouse/checkout@v1
- name: Mark Commit Release Ready
run: |
cd "$GITHUB_WORKSPACE/tests/ci"
python3 mark_release_ready.py
############################################################################################
#################################### INSTALL PACKAGES ######################################
############################################################################################

View File

@ -91,6 +91,8 @@ jobs:
build_name: package_release
checkout_depth: 0
data: ${{ needs.RunConfig.outputs.data }}
# always rebuild on release branches to be able to publish from any commit
force: true
BuilderDebAarch64:
needs: [RunConfig, BuildDockers]
if: ${{ !failure() && !cancelled() }}
@ -99,6 +101,8 @@ jobs:
build_name: package_aarch64
checkout_depth: 0
data: ${{ needs.RunConfig.outputs.data }}
# always rebuild on release branches to be able to publish from any commit
force: true
BuilderDebAsan:
needs: [RunConfig, BuildDockers]
if: ${{ !failure() && !cancelled() }}
@ -142,6 +146,8 @@ jobs:
build_name: binary_darwin
checkout_depth: 0
data: ${{ needs.RunConfig.outputs.data }}
# always rebuild on release branches to be able to publish from any commit
force: true
BuilderBinDarwinAarch64:
needs: [RunConfig, BuildDockers]
if: ${{ !failure() && !cancelled() }}
@ -150,6 +156,8 @@ jobs:
build_name: binary_darwin_aarch64
checkout_depth: 0
data: ${{ needs.RunConfig.outputs.data }}
# always rebuild on release branches to be able to publish from any commit
force: true
############################################################################################
##################################### Docker images #######################################
############################################################################################
@ -206,13 +214,8 @@ jobs:
if: ${{ !cancelled() }}
needs:
- RunConfig
- BuilderDebRelease
- BuilderDebAarch64
- BuilderDebAsan
- BuilderDebTsan
- BuilderDebUBsan
- BuilderDebMsan
- BuilderDebDebug
- BuilderBinDarwin
- BuilderBinDarwinAarch64
uses: ./.github/workflows/reusable_test.yml
with:
test_name: ClickHouse special build check
@ -225,7 +228,6 @@ jobs:
run_command: |
python3 build_report_check.py "$CHECK_NAME"
MarkReleaseReady:
if: ${{ !failure() && !cancelled() }}
needs:
- BuilderBinDarwin
- BuilderBinDarwinAarch64
@ -235,8 +237,6 @@ jobs:
steps:
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
clear-repository: true
- name: Mark Commit Release Ready
run: |
cd "$GITHUB_WORKSPACE/tests/ci"

View File

@ -26,6 +26,10 @@ name: Build ClickHouse
description: json ci data
type: string
required: true
force:
description: disallow job skipping
type: boolean
default: false
additional_envs:
description: additional ENV variables to setup the job
type: string
@ -33,7 +37,7 @@ name: Build ClickHouse
jobs:
Build:
name: Build-${{inputs.build_name}}
if: contains(fromJson(inputs.data).jobs_data.jobs_to_do, inputs.build_name)
if: ${{ contains(fromJson(inputs.data).jobs_data.jobs_to_do, inputs.build_name) || inputs.force }}
env:
GITHUB_JOB_OVERRIDDEN: Build-${{inputs.build_name}}
runs-on: [self-hosted, '${{inputs.runner_type}}']
@ -78,7 +82,8 @@ jobs:
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" \
--infile ${{ toJson(inputs.data) }} \
--job-name "$BUILD_NAME" \
--run
--run \
${{ inputs.force && '--force' || '' }}
- name: Post
# it still be build report to upload for failed build job
if: ${{ !cancelled() }}

View File

@ -254,10 +254,17 @@ endif()
include(cmake/cpu_features.cmake)
# Asynchronous unwind tables are needed for Query Profiler.
# They are already by default on some platforms but possibly not on all platforms.
# Enable it explicitly.
set (COMPILER_FLAGS "${COMPILER_FLAGS} -fasynchronous-unwind-tables")
# Query Profiler doesn't work on MacOS for several reasons
# - PHDR cache is not available
# - We use native functionality to get stacktraces which is not async signal safe
# and thus we don't need to generate asynchronous unwind tables
if (NOT OS_DARWIN)
# Asynchronous unwind tables are needed for Query Profiler.
# They are already by default on some platforms but possibly not on all platforms.
# Enable it explicitly.
set (COMPILER_FLAGS "${COMPILER_FLAGS} -fasynchronous-unwind-tables")
endif()
# Reproducible builds.
if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")

View File

@ -1,6 +1,10 @@
#include <libunwind.h>
/// On MacOS this function will be replaced with a dynamic symbol
/// from the system library.
#if !defined(OS_DARWIN)
int backtrace(void ** buffer, int size)
{
return unw_backtrace(buffer, size);
}
#endif

@ -1 +1 @@
Subproject commit 2568a7cd1297c7c3044b0f3cc0c23a6f6444d856
Subproject commit d2142eed98046a47ff7112e3cc1e197c8a5cd80f

View File

@ -451,3 +451,24 @@ To disallow concurrent backup/restore, you can use these settings respectively.
The default value for both is true, so by default concurrent backup/restores are allowed.
When these settings are false on a cluster, only 1 backup/restore is allowed to run on a cluster at a time.
## Configuring BACKUP/RESTORE to use an AzureBlobStorage Endpoint
To write backups to an AzureBlobStorage container you need the following pieces of information:
- AzureBlobStorage endpoint connection string / url,
- Container,
- Path,
- Account name (if url is specified)
- Account Key (if url is specified)
The destination for a backup will be specified like this:
```
AzureBlobStorage('<connection string>/<url>', '<container>', '<path>', '<account name>', '<account key>')
```
```sql
BACKUP TABLE data TO AzureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/;',
'test_container', 'data_backup');
RESTORE TABLE data AS data_restored FROM AzureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/;',
'test_container', 'data_backup');
```

View File

@ -28,6 +28,8 @@ The maximum amount of RAM to use for running a query on a single server.
The default setting is unlimited (set to `0`).
Cloud default value: depends on the amount of RAM on the replica.
The setting does not consider the volume of available memory or the total volume of memory on the machine.
The restriction applies to a single query within a single server.
You can use `SHOW PROCESSLIST` to see the current memory consumption for each query.
@ -104,7 +106,9 @@ Possible values:
- Maximum volume of RAM (in bytes) that can be used by the single [GROUP BY](../../sql-reference/statements/select/group-by.md#select-group-by-clause) operation.
- 0 — `GROUP BY` in external memory disabled.
Default value: 0.
Default value: `0`.
Cloud default value: half the memory amount per replica.
## max_bytes_before_external_sort {#settings-max_bytes_before_external_sort}
@ -115,6 +119,8 @@ Enables or disables execution of `ORDER BY` clauses in external memory. See [ORD
Default value: 0.
Cloud default value: half the memory amount per replica.
## max_rows_to_sort {#max-rows-to-sort}
A maximum number of rows before sorting. This allows you to limit memory consumption when sorting.
@ -129,7 +135,11 @@ What to do if the number of rows received before sorting exceeds one of the limi
## max_result_rows {#setting-max_result_rows}
Limit on the number of rows in the result. Also checked for subqueries, and on remote servers when running parts of a distributed query.
Limit on the number of rows in the result. Also checked for subqueries, and on remote servers when running parts of a distributed query. No limit is applied when value is `0`.
Default value: `0`.
Cloud default value: `0`.
## max_result_bytes {#max-result-bytes}
@ -137,10 +147,14 @@ Limit on the number of bytes in the result. The same as the previous setting.
## result_overflow_mode {#result-overflow-mode}
What to do if the volume of the result exceeds one of the limits: throw or break. By default, throw.
What to do if the volume of the result exceeds one of the limits: throw or break.
Using break is similar to using LIMIT. `Break` interrupts execution only at the block level. This means that amount of returned rows is greater than [max_result_rows](#setting-max_result_rows), multiple of [max_block_size](../../operations/settings/settings.md#setting-max_block_size) and depends on [max_threads](../../operations/settings/settings.md#max_threads).
Default value: `throw`.
Cloud default value: `throw`.
Example:
``` sql

View File

@ -212,6 +212,8 @@ Possible values:
Default value: `'basic'`.
Cloud default value: `'best_effort'`.
See also:
- [DateTime data type.](../../sql-reference/data-types/datetime.md)

View File

@ -508,7 +508,9 @@ Possible values:
- Any positive integer number of hops.
- 0 — No hops allowed.
Default value: 0.
Default value: `0`.
Cloud default value: `10`.
## insert_null_as_default {#insert_null_as_default}
@ -1126,7 +1128,9 @@ Possible values:
- 0 (or 1) — `INSERT SELECT` no parallel execution.
- Positive integer. Bigger than 1.
Default value: 0.
Default value: `0`.
Cloud default value: from `2` to `4`, depending on the service size.
Parallel `INSERT SELECT` has effect only if the `SELECT` part is executed in parallel, see [max_threads](#max_threads) setting.
Higher values will lead to higher memory usage.
@ -1207,7 +1211,9 @@ Default value: 10000.
Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connection without waiting for the response.
Default value: 0
Default value: `0`.
Cloud default value: `1`.
## poll_interval {#poll-interval}
@ -1946,6 +1952,8 @@ Possible values:
Default value: `200`.
Cloud default value: `1000`.
### async_insert_poll_timeout_ms {#async-insert-poll-timeout-ms}
Timeout in milliseconds for polling data from asynchronous insert queue.
@ -2130,7 +2138,9 @@ Possible values:
- Positive integer.
- 0 — Retries are disabled
Default value: 0
Default value: 20
Cloud default value: `20`.
Keeper request retries are done after some timeout. The timeout is controlled by the following settings: `insert_keeper_retry_initial_backoff_ms`, `insert_keeper_retry_max_backoff_ms`.
The first retry is done after `insert_keeper_retry_initial_backoff_ms` timeout. The consequent timeouts will be calculated as follows:
@ -2660,6 +2670,8 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md).
Default value: 1000000000 nanoseconds (once a second).
**Temporarily disabled in ClickHouse Cloud.**
See also:
- System table [trace_log](../../operations/system-tables/trace_log.md/#system_tables-trace_log)
@ -2683,6 +2695,8 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md).
Default value: 1000000000 nanoseconds.
**Temporarily disabled in ClickHouse Cloud.**
See also:
- System table [trace_log](../../operations/system-tables/trace_log.md/#system_tables-trace_log)
@ -2804,6 +2818,8 @@ Possible values:
Default value: `0`.
Cloud default value: `1`.
**See Also**
- [Distributed Table Engine](../../engines/table-engines/special/distributed.md/#distributed)
@ -3319,7 +3335,9 @@ Possible values:
- a string representing any valid table engine name
Default value: `None`
Default value: `MergeTree`.
Cloud default value: `SharedMergeTree`.
**Example**
@ -3895,6 +3913,8 @@ Possible values:
Default value: `0`.
Cloud default value: `1`.
## database_replicated_initial_query_timeout_sec {#database_replicated_initial_query_timeout_sec}
Sets how long initial DDL query should wait for Replicated database to process previous DDL queue entries in seconds.
@ -3933,6 +3953,8 @@ Possible values:
Default value: `throw`.
Cloud default value: `none`.
## flatten_nested {#flatten-nested}
Sets the data format of a [nested](../../sql-reference/data-types/nested-data-structures/index.md) columns.
@ -4068,6 +4090,8 @@ Possible values:
Default value: `1`.
Cloud default value: `0`.
:::note
`alter_sync` is applicable to `Replicated` tables only, it does nothing to alters of not `Replicated` tables.
:::
@ -4723,6 +4747,8 @@ other connections are cancelled. Queries with `max_parallel_replicas > 1` are su
Enabled by default.
Disabled by default on Cloud.
## hedged_connection_timeout {#hedged_connection_timeout}
If we can't establish connection with replica after this timeout in hedged requests, we start working with the next replica without cancelling connection to the previous.
@ -5348,10 +5374,11 @@ Default value: `false`.
## max_partition_size_to_drop
Restriction on dropping partitions in query time.
Restriction on dropping partitions in query time. The value 0 means that you can drop partitions without any restrictions.
Default value: 50 GB.
The value 0 means that you can drop partitions without any restrictions.
Cloud default value: 1 TB.
:::note
This query setting overwrites its server setting equivalent, see [max_partition_size_to_drop](/docs/en/operations/server-configuration-parameters/settings.md/#max-partition-size-to-drop)
@ -5359,10 +5386,11 @@ This query setting overwrites its server setting equivalent, see [max_partition_
## max_table_size_to_drop
Restriction on deleting tables in query time.
Restriction on deleting tables in query time. The value 0 means that you can delete all tables without any restrictions.
Default value: 50 GB.
The value 0 means that you can delete all tables without any restrictions.
Cloud default value: 1 TB.
:::note
This query setting overwrites its server setting equivalent, see [max_table_size_to_drop](/docs/en/operations/server-configuration-parameters/settings.md/#max-table-size-to-drop)

View File

@ -206,7 +206,7 @@ Some of these settings will disable cache features per query/profile that are en
- `read_from_filesystem_cache_if_exists_otherwise_bypass_cache` - allows to use cache in query only if it already exists, otherwise query data will not be written to local cache storage. Default: `false`.
- `enable_filesystem_cache_on_write_operations` - turn on `write-through` cache. This setting works only if setting `cache_on_write_operations` in cache configuration is turned on. Default: `false`.
- `enable_filesystem_cache_on_write_operations` - turn on `write-through` cache. This setting works only if setting `cache_on_write_operations` in cache configuration is turned on. Default: `false`. Cloud default value: `true`.
- `enable_filesystem_cache_log` - turn on logging to `system.filesystem_cache_log` table. Gives a detailed view of cache usage per query. It can be turn on for specific queries or enabled in a profile. Default: `false`.

View File

@ -19,7 +19,7 @@ slug: /ru/operations/system-tables/grants
- `column` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — Имя столбца, к которому предоставляется доступ.
- `is_partial_revoke` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Логическое значение. Показывает, были ли отменены некоторые привилегии. Возможные значения:
- `0` — Строка описывает частичный отзыв.
- `1` — Строка описывает грант.
- `0` — Строка описывает грант.
- `1` — Строка описывает частичный отзыв.
- `grant_option` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Разрешение предоставлено с опцией `WITH GRANT OPTION`, подробнее см. [GRANT](../../sql-reference/statements/grant.md#grant-privigele-syntax).

View File

@ -527,10 +527,11 @@ let queries = [];
/// Query parameters with predefined default values.
/// All other parameters will be automatically found in the queries.
let params = {
let default_params = {
'rounding': '60',
'seconds': '86400'
};
let params = default_params;
/// Palette generation for charts
function generatePalette(baseColor, numColors) {
@ -594,13 +595,19 @@ let plots = [];
let charts = document.getElementById('charts');
/// This is not quite correct (we cannot really parse SQL with regexp) but tolerable.
const query_param_regexp = /\{(\w+):[^}]+\}/g;
const query_param_regexp = /\{(\w+):([^}]+)\}/g;
/// Automatically parse more parameters from the queries.
function findParamsInQuery(query, new_params) {
const typeDefault = (type) => type.includes('Int') ? '0'
: (type.includes('Float') ? '0.0'
: (type.includes('Bool') ? 'false'
: (type.includes('Date') ? new Date().toISOString().slice(0, 10)
: (type.includes('UUID') ? '00000000-0000-0000-0000-000000000000'
: ''))));
for (let match of query.matchAll(query_param_regexp)) {
const name = match[1];
new_params[name] = params[name] || '';
new_params[name] = params[name] || default_params[name] || typeDefault(match[2]);
}
}

View File

@ -404,12 +404,12 @@ private:
auto operand_type = and_operands[0]->getResultType();
auto function_type = function_node.getResultType();
assert(!function_type->isNullable());
chassert(!function_type->isNullable());
if (!function_type->equals(*operand_type))
{
/// Result of equality operator can be low cardinality, while AND always returns UInt8.
/// In that case we replace `(lc = 1) AND (lc = 1)` with `(lc = 1) AS UInt8`
assert(function_type->equals(*removeLowCardinality(operand_type)));
chassert(function_type->equals(*removeLowCardinality(operand_type)));
node = createCastFunction(std::move(and_operands[0]), function_type, getContext());
}
else
@ -427,7 +427,7 @@ private:
void tryReplaceOrEqualsChainWithIn(QueryTreeNodePtr & node)
{
auto & function_node = node->as<FunctionNode &>();
assert(function_node.getFunctionName() == "or");
chassert(function_node.getFunctionName() == "or");
QueryTreeNodes or_operands;
@ -486,7 +486,7 @@ private:
/// first we create tuple from RHS of equals functions
for (const auto & equals : equals_functions)
{
is_any_nullable |= equals->getResultType()->isNullable();
is_any_nullable |= removeLowCardinality(equals->getResultType())->isNullable();
const auto * equals_function = equals->as<FunctionNode>();
assert(equals_function && equals_function->getFunctionName() == "equals");

View File

@ -33,11 +33,13 @@ void BackupFactory::registerBackupEngine(const String & engine_name, const Creat
void registerBackupEnginesFileAndDisk(BackupFactory &);
void registerBackupEngineS3(BackupFactory &);
void registerBackupEngineAzureBlobStorage(BackupFactory &);
void registerBackupEngines(BackupFactory & factory)
{
registerBackupEnginesFileAndDisk(factory);
registerBackupEngineS3(factory);
registerBackupEngineAzureBlobStorage(factory);
}
BackupFactory::BackupFactory()

View File

@ -0,0 +1,320 @@
#include <Backups/BackupIO_AzureBlobStorage.h>
#if USE_AZURE_BLOB_STORAGE
#include <Common/quoteString.h>
#include <Interpreters/threadPoolCallbackRunner.h>
#include <Interpreters/Context.h>
#include <IO/SharedThreadPools.h>
#include <IO/HTTPHeaderEntries.h>
#include <Storages/StorageAzureBlobCluster.h>
#include <Disks/IO/ReadBufferFromAzureBlobStorage.h>
#include <Disks/IO/WriteBufferFromAzureBlobStorage.h>
#include <IO/AzureBlobStorage/copyAzureBlobStorageFile.h>
#include <Disks/IDisk.h>
#include <Disks/DiskType.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
namespace ErrorCodes
{
extern const int AZURE_BLOB_STORAGE_ERROR;
extern const int LOGICAL_ERROR;
}
BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage(
StorageAzureBlob::Configuration configuration_,
const ReadSettings & read_settings_,
const WriteSettings & write_settings_,
const ContextPtr & context_)
: BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderAzureBlobStorage"))
, data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false}
, configuration(configuration_)
{
auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false);
object_storage = std::make_unique<AzureObjectStorage>("BackupReaderAzureBlobStorage",
std::move(client_ptr),
StorageAzureBlob::createSettings(context_),
configuration_.container);
client = object_storage->getAzureBlobStorageClient();
settings = object_storage->getSettings();
}
BackupReaderAzureBlobStorage::~BackupReaderAzureBlobStorage() = default;
bool BackupReaderAzureBlobStorage::fileExists(const String & file_name)
{
String key;
if (startsWith(file_name, "."))
{
key= configuration.blob_path + file_name;
}
else
{
key = file_name;
}
return object_storage->exists(StoredObject(key));
}
UInt64 BackupReaderAzureBlobStorage::getFileSize(const String & file_name)
{
String key;
if (startsWith(file_name, "."))
{
key= configuration.blob_path + file_name;
}
else
{
key = file_name;
}
ObjectMetadata object_metadata = object_storage->getObjectMetadata(key);
return object_metadata.size_bytes;
}
std::unique_ptr<SeekableReadBuffer> BackupReaderAzureBlobStorage::readFile(const String & file_name)
{
String key;
if (startsWith(file_name, "."))
{
key= configuration.blob_path + file_name;
}
else
{
key = file_name;
}
return std::make_unique<ReadBufferFromAzureBlobStorage>(
client, key, read_settings, settings->max_single_read_retries,
settings->max_single_download_retries);
}
void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup,
DiskPtr destination_disk, const String & destination_path, WriteMode write_mode)
{
auto destination_data_source_description = destination_disk->getDataSourceDescription();
if ((destination_data_source_description.type == DataSourceType::ObjectStorage)
&& (destination_data_source_description.object_storage_type == ObjectStorageType::Azure)
&& (destination_data_source_description.is_encrypted == encrypted_in_backup))
{
LOG_TRACE(log, "Copying {} from AzureBlobStorage to disk {}", path_in_backup, destination_disk->getName());
auto write_blob_function = [&](const Strings & blob_path, WriteMode mode, const std::optional<ObjectAttributes> &) -> size_t
{
/// Object storage always uses mode `Rewrite` because it simulates append using metadata and different files.
if (blob_path.size() != 2 || mode != WriteMode::Rewrite)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Blob writing function called with unexpected blob_path.size={} or mode={}",
blob_path.size(), mode);
copyAzureBlobStorageFile(
client,
destination_disk->getObjectStorage()->getAzureBlobStorageClient(),
configuration.container,
fs::path(configuration.blob_path) / path_in_backup,
0,
file_size,
/* dest_container */ blob_path[1],
/* dest_path */ blob_path[0],
settings,
read_settings,
threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupRDAzure"),
/* for_disk_azure_blob_storage= */ true);
return file_size;
};
destination_disk->writeFileUsingBlobWritingFunction(destination_path, write_mode, write_blob_function);
return; /// copied!
}
/// Fallback to copy through buffers.
BackupReaderDefault::copyFileToDisk(path_in_backup, file_size, encrypted_in_backup, destination_disk, destination_path, write_mode);
}
BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage(
StorageAzureBlob::Configuration configuration_,
const ReadSettings & read_settings_,
const WriteSettings & write_settings_,
const ContextPtr & context_)
: BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterAzureBlobStorage"))
, data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false}
, configuration(configuration_)
{
auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false);
object_storage = std::make_unique<AzureObjectStorage>("BackupWriterAzureBlobStorage",
std::move(client_ptr),
StorageAzureBlob::createSettings(context_),
configuration_.container);
client = object_storage->getAzureBlobStorageClient();
settings = object_storage->getSettings();
}
void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path,
bool copy_encrypted, UInt64 start_pos, UInt64 length)
{
/// Use the native copy as a more optimal way to copy a file from AzureBlobStorage to AzureBlobStorage if it's possible.
auto source_data_source_description = src_disk->getDataSourceDescription();
if (source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted))
{
/// getBlobPath() can return more than 3 elements if the file is stored as multiple objects in AzureBlobStorage container.
/// In this case we can't use the native copy.
if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2)
{
LOG_TRACE(log, "Copying file {} from disk {} to AzureBlobStorag", src_path, src_disk->getName());
copyAzureBlobStorageFile(
src_disk->getObjectStorage()->getAzureBlobStorageClient(),
client,
/* src_container */ blob_path[1],
/* src_path */ blob_path[0],
start_pos,
length,
configuration.container,
fs::path(configuration.blob_path) / path_in_backup,
settings,
read_settings,
threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupWRAzure"));
return; /// copied!
}
}
/// Fallback to copy through buffers.
BackupWriterDefault::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length);
}
void BackupWriterAzureBlobStorage::copyFile(const String & destination, const String & source, size_t size)
{
LOG_TRACE(log, "Copying file inside backup from {} to {} ", source, destination);
copyAzureBlobStorageFile(
client,
client,
configuration.container,
fs::path(source),
0,
size,
/* dest_container */ configuration.container,
/* dest_path */ destination,
settings,
read_settings,
threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupWRAzure"),
/* for_disk_azure_blob_storage= */ true);
}
void BackupWriterAzureBlobStorage::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length)
{
copyDataToAzureBlobStorageFile(create_read_buffer, start_pos, length, client, configuration.container, path_in_backup, settings,
threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupWRAzure"));
}
BackupWriterAzureBlobStorage::~BackupWriterAzureBlobStorage() = default;
bool BackupWriterAzureBlobStorage::fileExists(const String & file_name)
{
String key;
if (startsWith(file_name, "."))
{
key= configuration.blob_path + file_name;
}
else
{
key = file_name;
}
return object_storage->exists(StoredObject(key));
}
UInt64 BackupWriterAzureBlobStorage::getFileSize(const String & file_name)
{
String key;
if (startsWith(file_name, "."))
{
key= configuration.blob_path + file_name;
}
else
{
key = file_name;
}
RelativePathsWithMetadata children;
object_storage->listObjects(key,children,/*max_keys*/0);
if (children.empty())
throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Object must exist");
return children[0].metadata.size_bytes;
}
std::unique_ptr<ReadBuffer> BackupWriterAzureBlobStorage::readFile(const String & file_name, size_t /*expected_file_size*/)
{
String key;
if (startsWith(file_name, "."))
{
key= configuration.blob_path + file_name;
}
else
{
key = file_name;
}
return std::make_unique<ReadBufferFromAzureBlobStorage>(
client, key, read_settings, settings->max_single_read_retries,
settings->max_single_download_retries);
}
std::unique_ptr<WriteBuffer> BackupWriterAzureBlobStorage::writeFile(const String & file_name)
{
String key;
if (startsWith(file_name, "."))
{
key= configuration.blob_path + file_name;
}
else
{
key = file_name;
}
return std::make_unique<WriteBufferFromAzureBlobStorage>(
client,
key,
settings->max_single_part_upload_size,
settings->max_unexpected_write_error_retries,
DBMS_DEFAULT_BUFFER_SIZE,
write_settings);
}
void BackupWriterAzureBlobStorage::removeFile(const String & file_name)
{
String key;
if (startsWith(file_name, "."))
{
key= configuration.blob_path + file_name;
}
else
{
key = file_name;
}
StoredObject object(key);
object_storage->removeObjectIfExists(object);
}
void BackupWriterAzureBlobStorage::removeFiles(const Strings & file_names)
{
StoredObjects objects;
for (const auto & file_name : file_names)
objects.emplace_back(file_name);
object_storage->removeObjectsIfExist(objects);
}
void BackupWriterAzureBlobStorage::removeFilesBatch(const Strings & file_names)
{
StoredObjects objects;
for (const auto & file_name : file_names)
objects.emplace_back(file_name);
object_storage->removeObjectsIfExist(objects);
}
}
#endif

View File

@ -0,0 +1,68 @@
#pragma once
#include "config.h"
#if USE_AZURE_BLOB_STORAGE
#include <Backups/BackupIO_Default.h>
#include <Disks/DiskType.h>
#include <Storages/StorageAzureBlobCluster.h>
#include <Interpreters/Context_fwd.h>
namespace DB
{
/// Represents a backup stored to Azure
class BackupReaderAzureBlobStorage : public BackupReaderDefault
{
public:
BackupReaderAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_);
~BackupReaderAzureBlobStorage() override;
bool fileExists(const String & file_name) override;
UInt64 getFileSize(const String & file_name) override;
std::unique_ptr<SeekableReadBuffer> readFile(const String & file_name) override;
void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup,
DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) override;
private:
const DataSourceDescription data_source_description;
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> client;
StorageAzureBlob::Configuration configuration;
std::unique_ptr<AzureObjectStorage> object_storage;
std::shared_ptr<const AzureObjectStorageSettings> settings;
};
class BackupWriterAzureBlobStorage : public BackupWriterDefault
{
public:
BackupWriterAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_);
~BackupWriterAzureBlobStorage() override;
bool fileExists(const String & file_name) override;
UInt64 getFileSize(const String & file_name) override;
std::unique_ptr<WriteBuffer> writeFile(const String & file_name) override;
void copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) override;
void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path,
bool copy_encrypted, UInt64 start_pos, UInt64 length) override;
void copyFile(const String & destination, const String & source, size_t size) override;
void removeFile(const String & file_name) override;
void removeFiles(const Strings & file_names) override;
private:
std::unique_ptr<ReadBuffer> readFile(const String & file_name, size_t expected_file_size) override;
void removeFilesBatch(const Strings & file_names);
const DataSourceDescription data_source_description;
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> client;
StorageAzureBlob::Configuration configuration;
std::unique_ptr<AzureObjectStorage> object_storage;
std::shared_ptr<const AzureObjectStorageSettings> settings;
};
}
#endif

View File

@ -939,12 +939,12 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry)
}
else if (src_disk && from_immutable_file)
{
LOG_TRACE(log, "Writing backup for file {} from {} (disk {}): data file #{}", info.data_file_name, src_file_desc, src_disk->getName(), info.data_file_index);
LOG_INFO(log, "Writing backup for file {} from {} (disk {}): data file #{}", info.data_file_name, src_file_desc, src_disk->getName(), info.data_file_index);
writer->copyFileFromDisk(info.data_file_name, src_disk, src_file_path, info.encrypted_by_disk, info.base_size, info.size - info.base_size);
}
else
{
LOG_TRACE(log, "Writing backup for file {} from {}: data file #{}", info.data_file_name, src_file_desc, info.data_file_index);
LOG_INFO(log, "Writing backup for file {} from {}: data file #{}", info.data_file_name, src_file_desc, info.data_file_index);
auto create_read_buffer = [entry, read_settings = writer->getReadSettings()] { return entry->getReadBuffer(read_settings); };
writer->copyDataToFile(info.data_file_name, create_read_buffer, info.base_size, info.size - info.base_size);
}

View File

@ -0,0 +1,172 @@
#include "config.h"
#include <Backups/BackupFactory.h>
#include <Common/Exception.h>
#if USE_AZURE_BLOB_STORAGE
#include <Backups/BackupIO_AzureBlobStorage.h>
#include <Storages/StorageAzureBlob.h>
#include <Backups/BackupImpl.h>
#include <IO/Archives/hasRegisteredArchiveFileExtension.h>
#include <Interpreters/Context.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <filesystem>
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int SUPPORT_IS_DISABLED;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
#if USE_AZURE_BLOB_STORAGE
namespace
{
String removeFileNameFromURL(String & url)
{
Poco::URI url2{url};
String path = url2.getPath();
size_t slash_pos = path.find_last_of('/');
String file_name = path.substr(slash_pos + 1);
path.resize(slash_pos + 1);
url2.setPath(path);
url = url2.toString();
return file_name;
}
}
#endif
void registerBackupEngineAzureBlobStorage(BackupFactory & factory)
{
auto creator_fn = []([[maybe_unused]] const BackupFactory::CreateParams & params) -> std::unique_ptr<IBackup>
{
#if USE_AZURE_BLOB_STORAGE
const String & id_arg = params.backup_info.id_arg;
const auto & args = params.backup_info.args;
StorageAzureBlob::Configuration configuration;
if (!id_arg.empty())
{
const auto & config = params.context->getConfigRef();
auto config_prefix = "named_collections." + id_arg;
if (!config.has(config_prefix))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", id_arg);
if (config.has(config_prefix + ".connection_string"))
{
configuration.connection_url = config.getString(config_prefix + ".connection_string");
configuration.is_connection_string = true;
configuration.container = config.getString(config_prefix + ".container");
}
else
{
configuration.connection_url = config.getString(config_prefix + ".storage_account_url");
configuration.is_connection_string = false;
configuration.container = config.getString(config_prefix + ".container");
configuration.account_name = config.getString(config_prefix + ".account_name");
configuration.account_key = config.getString(config_prefix + ".account_key");
}
if (args.size() > 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Backup AzureBlobStorage requires 1 or 2 arguments: named_collection, [filename]");
if (args.size() == 1)
configuration.blob_path = args[0].safeGet<String>();
}
else
{
if (args.size() == 3)
{
configuration.connection_url = args[0].safeGet<String>();
configuration.is_connection_string = true;
configuration.container = args[1].safeGet<String>();
configuration.blob_path = args[2].safeGet<String>();
}
else if (args.size() == 5)
{
configuration.connection_url = args[0].safeGet<String>();
configuration.is_connection_string = false;
configuration.container = args[1].safeGet<String>();
configuration.blob_path = args[2].safeGet<String>();
configuration.account_name = args[3].safeGet<String>();
configuration.account_key = args[4].safeGet<String>();
}
else
{
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Backup AzureBlobStorage requires 3 or 5 arguments: connection string>/<url, container, path, [account name], [account key]");
}
}
BackupImpl::ArchiveParams archive_params;
if (hasRegisteredArchiveFileExtension(configuration.blob_path))
{
if (params.is_internal_backup)
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Using archives with backups on clusters is disabled");
archive_params.archive_name = removeFileNameFromURL(configuration.blob_path);
archive_params.compression_method = params.compression_method;
archive_params.compression_level = params.compression_level;
archive_params.password = params.password;
}
else
{
if (!params.password.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Password is not applicable, backup cannot be encrypted");
}
if (params.open_mode == IBackup::OpenMode::READ)
{
auto reader = std::make_shared<BackupReaderAzureBlobStorage>(configuration,
params.read_settings,
params.write_settings,
params.context);
return std::make_unique<BackupImpl>(
params.backup_info,
archive_params,
params.base_backup_info,
reader,
params.context,
/* use_same_s3_credentials_for_base_backup*/ false);
}
else
{
auto writer = std::make_shared<BackupWriterAzureBlobStorage>(configuration,
params.read_settings,
params.write_settings,
params.context);
return std::make_unique<BackupImpl>(
params.backup_info,
archive_params,
params.base_backup_info,
writer,
params.context,
params.is_internal_backup,
params.backup_coordination,
params.backup_uuid,
params.deduplicate_files,
/* use_same_s3_credentials_for_base_backup */ false);
}
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "AzureBlobStorage support is disabled");
#endif
};
factory.registerBackupEngine("AzureBlobStorage", creator_fn);
}
}

View File

@ -89,6 +89,7 @@ add_headers_and_sources(clickhouse_common_io Common/SSH)
add_headers_and_sources(clickhouse_common_io IO)
add_headers_and_sources(clickhouse_common_io IO/Archives)
add_headers_and_sources(clickhouse_common_io IO/S3)
add_headers_and_sources(clickhouse_common_io IO/AzureBlobStorage)
list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp)
@ -141,6 +142,7 @@ endif()
if (TARGET ch_contrib::azure_sdk)
add_headers_and_sources(dbms Disks/ObjectStorages/AzureBlobStorage)
add_headers_and_sources(dbms IO/AzureBlobStorage)
endif()
if (TARGET ch_contrib::hdfs)
@ -496,6 +498,7 @@ if (TARGET ch_contrib::aws_s3)
endif()
if (TARGET ch_contrib::azure_sdk)
target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::azure_sdk)
dbms_target_link_libraries (PRIVATE ch_contrib::azure_sdk)
endif()

View File

@ -19,7 +19,6 @@
#include <Storages/MergeTree/RequestResponse.h>
#include <atomic>
#include <optional>
#include "config.h"

View File

@ -27,6 +27,9 @@ class IConnectionPool : private boost::noncopyable
public:
using Entry = PoolBase<Connection>::Entry;
IConnectionPool() = default;
IConnectionPool(String host_, UInt16 port_) : host(host_), port(port_), address(host + ":" + toString(port_)) {}
virtual ~IConnectionPool() = default;
/// Selects the connection to work.
@ -36,7 +39,15 @@ public:
const Settings & settings,
bool force_connected = true) = 0;
const std::string & getHost() const { return host; }
UInt16 getPort() const { return port; }
const String & getAddress() const { return address; }
virtual Priority getPriority() const { return Priority{1}; }
protected:
const String host;
const UInt16 port = 0;
const String address;
};
using ConnectionPoolPtr = std::shared_ptr<IConnectionPool>;
@ -63,10 +74,9 @@ public:
Protocol::Compression compression_,
Protocol::Secure secure_,
Priority priority_ = Priority{1})
: Base(max_connections_,
: IConnectionPool(host_, port_),
Base(max_connections_,
getLogger("ConnectionPool (" + host_ + ":" + toString(port_) + ")")),
host(host_),
port(port_),
default_database(default_database_),
user(user_),
password(password_),
@ -99,10 +109,6 @@ public:
return entry;
}
const std::string & getHost() const
{
return host;
}
std::string getDescription() const
{
return host + ":" + toString(port);
@ -125,8 +131,6 @@ protected:
}
private:
String host;
UInt16 port;
String default_database;
String user;
String password;

View File

@ -1,7 +1,5 @@
#pragma once
#include <compare>
#include <Client/Connection.h>
#include <Storages/MergeTree/RequestResponse.h>

View File

@ -593,6 +593,7 @@
M(711, FILECACHE_ACCESS_DENIED) \
M(712, TOO_MANY_MATERIALIZED_VIEWS) \
M(713, BROKEN_PROJECTION) \
M(714, UNEXPECTED_CLUSTER) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

View File

@ -384,6 +384,10 @@ The server successfully detected this situation and will download merged part fr
M(S3PutObject, "Number of S3 API PutObject calls.") \
M(S3GetObject, "Number of S3 API GetObject calls.") \
\
M(AzureUploadPart, "Number of Azure blob storage API UploadPart calls") \
M(DiskAzureUploadPart, "Number of Disk Azure blob storage API UploadPart calls") \
M(AzureCopyObject, "Number of Azure blob storage API CopyObject calls") \
M(DiskAzureCopyObject, "Number of Disk Azure blob storage API CopyObject calls") \
M(AzureDeleteObjects, "Number of Azure blob storage API DeleteObject(s) calls.") \
M(AzureListObjects, "Number of Azure blob storage API ListObjects calls.") \
\
@ -625,6 +629,8 @@ The server successfully detected this situation and will download merged part fr
M(InterfacePostgreSQLReceiveBytes, "Number of bytes received through PostgreSQL interfaces") \
\
M(ParallelReplicasUsedCount, "Number of replicas used to execute a query with task-based parallel replicas") \
M(ParallelReplicasAvailableCount, "Number of replicas available to execute a query with task-based parallel replicas") \
M(ParallelReplicasUnavailableCount, "Number of replicas which was chosen, but found to be unavailable during query execution with task-based parallel replicas") \
#ifdef APPLY_FOR_EXTERNAL_EVENTS
#define APPLY_FOR_EVENTS(M) APPLY_FOR_BUILTIN_EVENTS(M) APPLY_FOR_EXTERNAL_EVENTS(M)

View File

@ -4,6 +4,7 @@
#include <base/constexpr_helpers.h>
#include <base/demangle.h>
#include <Common/scope_guard_safe.h>
#include <Common/Dwarf.h>
#include <Common/Elf.h>
#include <Common/MemorySanitizer.h>
@ -24,6 +25,15 @@
#include "config.h"
#include <boost/algorithm/string/split.hpp>
#if defined(OS_DARWIN)
/// This header contains functions like `backtrace` and `backtrace_symbols`
/// Which will be used for stack unwinding on Mac.
/// Read: https://developer.apple.com/library/archive/documentation/System/Conceptual/ManPages_iPhoneOS/man3/backtrace.3.html
#include "execinfo.h"
#endif
namespace
{
/// Currently this variable is set up once on server startup.
@ -262,6 +272,33 @@ void StackTrace::forEachFrame(
callback(current_inline_frame);
}
callback(current_frame);
}
#elif defined(OS_DARWIN)
UNUSED(fatal);
/// This function returns an array of string in a special (a little bit weird format)
/// The frame number, library name, address in hex, mangled symbol name, `+` sign, the offset.
char** strs = ::backtrace_symbols(frame_pointers.data(), static_cast<int>(size));
SCOPE_EXIT_SAFE({free(strs);});
for (size_t i = offset; i < size; ++i)
{
StackTrace::Frame current_frame;
std::vector<std::string> split;
boost::split(split, strs[i], isWhitespaceASCII);
split.erase(
std::remove_if(
split.begin(), split.end(),
[](const std::string & x) { return x.empty(); }),
split.end());
assert(split.size() == 6);
current_frame.virtual_addr = frame_pointers[i];
current_frame.physical_addr = frame_pointers[i];
current_frame.object = split[1];
current_frame.symbol = split[3];
callback(current_frame);
}
#else
@ -306,7 +343,11 @@ StackTrace::StackTrace(const ucontext_t & signal_context)
void StackTrace::tryCapture()
{
#if defined(OS_DARWIN)
size = backtrace(frame_pointers.data(), capacity);
#else
size = unw_backtrace(frame_pointers.data(), capacity);
#endif
__msan_unpoison(frame_pointers.data(), size * sizeof(frame_pointers[0]));
}
@ -376,7 +417,7 @@ toStringEveryLineImpl([[maybe_unused]] bool fatal, const StackTraceRefTriple & s
return callback("<Empty trace>");
size_t frame_index = stack_trace.offset;
#if defined(__ELF__) && !defined(OS_FREEBSD)
#if (defined(__ELF__) && !defined(OS_FREEBSD)) || defined(OS_DARWIN)
size_t inline_frame_index = 0;
auto callback_wrapper = [&](const StackTrace::Frame & frame)
{

View File

@ -82,7 +82,8 @@ class IColumn;
M(UInt64, s3_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to S3, s3_min_upload_part_size is multiplied by s3_upload_part_size_multiply_factor.", 0) \
M(UInt64, s3_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited. You ", 0) \
M(UInt64, s3_max_single_part_upload_size, 32*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \
M(UInt64, azure_max_single_part_upload_size, 100*1024*1024, "The maximum size of object to upload using singlepart upload to Azure blob storage.", 0) \
M(UInt64, azure_max_single_part_upload_size, 100*1024*1024, "The maximum size of object to upload using singlepart upload to Azure blob storage.", 0) \
M(UInt64, azure_max_single_part_copy_size, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage.", 0) \
M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \
M(UInt64, azure_max_single_read_retries, 4, "The maximum number of retries during single Azure blob storage read.", 0) \
M(UInt64, azure_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write", 0) \

View File

@ -166,6 +166,9 @@ std::unique_ptr<AzureObjectStorageSettings> getAzureBlobStorageSettings(const Po
config.getInt(config_prefix + ".max_single_read_retries", 3),
config.getInt(config_prefix + ".max_single_download_retries", 3),
config.getInt(config_prefix + ".list_object_keys_size", 1000),
config.getUInt64(config_prefix + ".max_upload_part_size", 5ULL * 1024 * 1024 * 1024),
config.getUInt64(config_prefix + ".max_single_part_copy_size", context->getSettings().azure_max_single_part_copy_size),
config.getBool(config_prefix + ".use_native_copy", false),
config.getUInt64(config_prefix + ".max_unexpected_write_error_retries", context->getSettings().azure_max_unexpected_write_error_retries)
);
}

View File

@ -92,10 +92,12 @@ private:
AzureObjectStorage::AzureObjectStorage(
const String & name_,
AzureClientPtr && client_,
SettingsPtr && settings_)
SettingsPtr && settings_,
const String & container_)
: name(name_)
, client(std::move(client_))
, settings(std::move(settings_))
, container(container_)
, log(getLogger("AzureObjectStorage"))
{
}
@ -376,7 +378,8 @@ std::unique_ptr<IObjectStorage> AzureObjectStorage::cloneObjectStorage(const std
return std::make_unique<AzureObjectStorage>(
name,
getAzureBlobContainerClient(config, config_prefix),
getAzureBlobStorageSettings(config, config_prefix, context)
getAzureBlobStorageSettings(config, config_prefix, context),
container
);
}

View File

@ -24,12 +24,18 @@ struct AzureObjectStorageSettings
int max_single_read_retries_,
int max_single_download_retries_,
int list_object_keys_size_,
size_t max_upload_part_size_,
size_t max_single_part_copy_size_,
bool use_native_copy_,
size_t max_unexpected_write_error_retries_)
: max_single_part_upload_size(max_single_part_upload_size_)
, min_bytes_for_seek(min_bytes_for_seek_)
, max_single_read_retries(max_single_read_retries_)
, max_single_download_retries(max_single_download_retries_)
, list_object_keys_size(list_object_keys_size_)
, max_upload_part_size(max_upload_part_size_)
, max_single_part_copy_size(max_single_part_copy_size_)
, use_native_copy(use_native_copy_)
, max_unexpected_write_error_retries (max_unexpected_write_error_retries_)
{
}
@ -41,6 +47,10 @@ struct AzureObjectStorageSettings
size_t max_single_read_retries = 3;
size_t max_single_download_retries = 3;
int list_object_keys_size = 1000;
size_t min_upload_part_size = 16 * 1024 * 1024;
size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024;
size_t max_single_part_copy_size = 256 * 1024 * 1024;
bool use_native_copy = false;
size_t max_unexpected_write_error_retries = 4;
};
@ -56,7 +66,8 @@ public:
AzureObjectStorage(
const String & name_,
AzureClientPtr && client_,
SettingsPtr && settings_);
SettingsPtr && settings_,
const String & container_);
void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override;
@ -119,7 +130,7 @@ public:
const std::string & config_prefix,
ContextPtr context) override;
String getObjectsNamespace() const override { return ""; }
String getObjectsNamespace() const override { return container ; }
std::unique_ptr<IObjectStorage> cloneObjectStorage(
const std::string & new_namespace,
@ -131,11 +142,19 @@ public:
bool isRemote() const override { return true; }
std::shared_ptr<const AzureObjectStorageSettings> getSettings() { return settings.get(); }
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> getAzureBlobStorageClient() override
{
return client.get();
}
private:
const String name;
/// client used to access the files in the Blob Storage cloud
MultiVersion<Azure::Storage::Blobs::BlobContainerClient> client;
MultiVersion<AzureObjectStorageSettings> settings;
const String container;
LoggerPtr log;
};

View File

@ -3,6 +3,7 @@
#include <Disks/ObjectStorages/IObjectStorage.h>
#include <Interpreters/Cache/FileCacheKey.h>
#include <Interpreters/Cache/FileCacheSettings.h>
#include "config.h"
namespace Poco
{
@ -120,6 +121,13 @@ public:
static bool canUseReadThroughCache(const ReadSettings & settings);
#if USE_AZURE_BLOB_STORAGE
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> getAzureBlobStorageClient() override
{
return object_storage->getAzureBlobStorageClient();
}
#endif
private:
FileCacheKey getCacheKey(const std::string & path) const;

View File

@ -23,11 +23,22 @@
#include <Disks/DirectoryIterator.h>
#include <Common/ThreadPool.h>
#include <Interpreters/threadPoolCallbackRunner.h>
#include <Common/Exception.h>
#include "config.h"
#if USE_AZURE_BLOB_STORAGE
#include <Common/MultiVersion.h>
#include <azure/storage/blobs.hpp>
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
class ReadBufferFromFileBase;
class WriteBufferFromFileBase;
@ -214,6 +225,14 @@ public:
virtual WriteSettings patchSettings(const WriteSettings & write_settings) const;
#if USE_AZURE_BLOB_STORAGE
virtual std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> getAzureBlobStorageClient()
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "This function is only implemented for AzureBlobStorage");
}
#endif
private:
mutable std::mutex throttlers_mutex;
ThrottlerPtr remote_read_throttler;

View File

@ -213,10 +213,12 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory)
const ContextPtr & context,
bool /* skip_access_check */) -> ObjectStoragePtr
{
String container_name = config.getString(config_prefix + ".container_name", "default-container");
return std::make_unique<AzureObjectStorage>(
name,
getAzureBlobContainerClient(config, config_prefix),
getAzureBlobStorageSettings(config, config_prefix, context));
getAzureBlobStorageSettings(config, config_prefix, context),
container_name);
});
}

View File

@ -1,6 +1,7 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/IColumn.h>
#include <Common/TargetSpecific.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/IDataType.h>
@ -9,6 +10,10 @@
#include <Functions/FunctionHelpers.h>
#include <base/range.h>
#if USE_MULTITARGET_CODE
#include <immintrin.h>
#endif
namespace DB
{
namespace ErrorCodes
@ -75,6 +80,49 @@ struct L2Distance
state.sum += other_state.sum;
}
#if USE_MULTITARGET_CODE
template <typename ResultType>
AVX512_FUNCTION_SPECIFIC_ATTRIBUTE static void accumulateCombine(
const ResultType * __restrict data_x,
const ResultType * __restrict data_y,
size_t i_max,
size_t & i_x,
size_t & i_y,
State<ResultType> & state)
{
__m512 sums;
if constexpr (std::is_same_v<ResultType, Float32>)
sums = _mm512_setzero_ps();
else
sums = _mm512_setzero_pd();
const size_t n = (std::is_same_v<ResultType, Float32>) ? 16 : 8;
for (; i_x + n < i_max; i_x += n, i_y += n)
{
if constexpr (std::is_same_v<ResultType, Float32>)
{
__m512 x = _mm512_loadu_ps(data_x + i_x);
__m512 y = _mm512_loadu_ps(data_y + i_y);
__m512 differences = _mm512_sub_ps(x, y);
sums = _mm512_fmadd_ps(differences, differences, sums);
}
else
{
__m512 x = _mm512_loadu_pd(data_x + i_x);
__m512 y = _mm512_loadu_pd(data_y + i_y);
__m512 differences = _mm512_sub_pd(x, y);
sums = _mm512_fmadd_pd(differences, differences, sums);
}
}
if constexpr (std::is_same_v<ResultType, Float32>)
state.sum = _mm512_reduce_add_ps(sums);
else
state.sum = _mm512_reduce_add_pd(sums);
}
#endif
template <typename ResultType>
static ResultType finalize(const State<ResultType> & state, const ConstParams &)
{
@ -189,6 +237,70 @@ struct CosineDistance
state.y_squared += other_state.y_squared;
}
#if USE_MULTITARGET_CODE
template <typename ResultType>
AVX512_FUNCTION_SPECIFIC_ATTRIBUTE static void accumulateCombine(
const ResultType * __restrict data_x,
const ResultType * __restrict data_y,
size_t i_max,
size_t & i_x,
size_t & i_y,
State<ResultType> & state)
{
__m512 dot_products;
__m512 x_squareds;
__m512 y_squareds;
if constexpr (std::is_same_v<ResultType, Float32>)
{
dot_products = _mm512_setzero_ps();
x_squareds = _mm512_setzero_ps();
y_squareds = _mm512_setzero_ps();
}
else
{
dot_products = _mm512_setzero_pd();
x_squareds = _mm512_setzero_pd();
y_squareds = _mm512_setzero_pd();
}
const size_t n = (std::is_same_v<ResultType, Float32>) ? 16 : 8;
for (; i_x + n < i_max; i_x += n, i_y += n)
{
if constexpr (std::is_same_v<ResultType, Float32>)
{
__m512 x = _mm512_loadu_ps(data_x + i_x);
__m512 y = _mm512_loadu_ps(data_y + i_y);
dot_products = _mm512_fmadd_ps(x, y, dot_products);
x_squareds = _mm512_fmadd_ps(x, x, x_squareds);
y_squareds = _mm512_fmadd_ps(y, y, y_squareds);
}
else
{
__m512 x = _mm512_loadu_pd(data_x + i_x);
__m512 y = _mm512_loadu_pd(data_y + i_y);
dot_products = _mm512_fmadd_pd(x, y, dot_products);
x_squareds = _mm512_fmadd_pd(x, x, x_squareds);
y_squareds = _mm512_fmadd_pd(y, y, y_squareds);
}
}
if constexpr (std::is_same_v<ResultType, Float32>)
{
state.dot_prod = _mm512_reduce_add_ps(dot_products);
state.x_squared = _mm512_reduce_add_ps(x_squareds);
state.y_squared = _mm512_reduce_add_ps(y_squareds);
}
else
{
state.dot_prod = _mm512_reduce_add_pd(dot_products);
state.x_squared = _mm512_reduce_add_pd(x_squareds);
state.y_squared = _mm512_reduce_add_pd(y_squareds);
}
}
#endif
template <typename ResultType>
static ResultType finalize(const State<ResultType> & state, const ConstParams &)
{
@ -352,7 +464,7 @@ private:
/// Check that arrays in both columns are the sames size
for (size_t row = 0; row < offsets_x.size(); ++row)
{
if (unlikely(offsets_x[row] != offsets_y[row]))
if (offsets_x[row] != offsets_y[row]) [[unlikely]]
{
ColumnArray::Offset prev_offset = row > 0 ? offsets_x[row] : 0;
throw Exception(
@ -420,7 +532,7 @@ private:
ColumnArray::Offset prev_offset = 0;
for (size_t row : collections::range(0, offsets_y.size()))
{
if (unlikely(offsets_x[0] != offsets_y[row] - prev_offset))
if (offsets_x[0] != offsets_y[row] - prev_offset) [[unlikely]]
{
throw Exception(
ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH,
@ -438,14 +550,35 @@ private:
auto & result_data = result->getData();
/// Do the actual computation
ColumnArray::Offset prev = 0;
size_t prev = 0;
size_t row = 0;
for (auto off : offsets_y)
{
size_t i = 0;
typename Kernel::template State<ResultType> state;
/// SIMD optimization: process multiple elements in both input arrays at once.
/// To avoid combinatorial explosion of SIMD kernels, focus on
/// - the two most common input/output types (Float32 x Float32) --> Float32 and (Float64 x Float64) --> Float64 instead of 10 x
/// 10 input types x 2 output types,
/// - const/non-const inputs instead of non-const/non-const inputs
/// - the two most common metrics L2 and cosine distance,
/// - the most powerful SIMD instruction set (AVX-512F).
#if USE_MULTITARGET_CODE
if constexpr (std::is_same_v<ResultType, FirstArgType> && std::is_same_v<ResultType, SecondArgType>) /// ResultType is Float32 or Float64
{
if constexpr (std::is_same_v<Kernel, L2Distance>
|| std::is_same_v<Kernel, CosineDistance>)
{
if (isArchSupported(TargetArch::AVX512F))
Kernel::template accumulateCombine<ResultType>(data_x.data(), data_y.data(), i + offsets_x[0], i, prev, state);
}
}
#else
/// Process chunks in vectorized manner
static constexpr size_t VEC_SIZE = 4;
typename Kernel::template State<ResultType> states[VEC_SIZE];
size_t i = 0;
for (; prev + VEC_SIZE < off; i += VEC_SIZE, prev += VEC_SIZE)
{
for (size_t s = 0; s < VEC_SIZE; ++s)
@ -453,10 +586,9 @@ private:
states[s], static_cast<ResultType>(data_x[i + s]), static_cast<ResultType>(data_y[prev + s]), kernel_params);
}
typename Kernel::template State<ResultType> state;
for (const auto & other_state : states)
Kernel::template combine<ResultType>(state, other_state, kernel_params);
#endif
/// Process the tail
for (; prev < off; ++i, ++prev)
{
@ -466,6 +598,7 @@ private:
result_data[row] = Kernel::finalize(state, kernel_params);
row++;
}
return result;
}

View File

@ -0,0 +1,340 @@
#include <IO/AzureBlobStorage/copyAzureBlobStorageFile.h>
#if USE_AZURE_BLOB_STORAGE
#include <Common/ProfileEvents.h>
#include <Common/typeid_cast.h>
#include <Interpreters/Context.h>
#include <IO/LimitSeekableReadBuffer.h>
#include <IO/SeekableReadBuffer.h>
#include <IO/StdStreamFromReadBuffer.h>
#include <Disks/IO/ReadBufferFromAzureBlobStorage.h>
#include <Disks/IO/WriteBufferFromAzureBlobStorage.h>
#include <Common/getRandomASCIIString.h>
#include <IO/SharedThreadPools.h>
namespace ProfileEvents
{
extern const Event AzureCopyObject;
extern const Event AzureUploadPart;
extern const Event DiskAzureCopyObject;
extern const Event DiskAzureUploadPart;
}
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_CONFIG_PARAMETER;
extern const int AZURE_BLOB_STORAGE_ERROR;
}
namespace
{
class UploadHelper
{
public:
UploadHelper(
const CreateReadBuffer & create_read_buffer_,
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> client_,
size_t offset_,
size_t total_size_,
const String & dest_container_for_logging_,
const String & dest_blob_,
std::shared_ptr<const AzureObjectStorageSettings> settings_,
ThreadPoolCallbackRunner<void> schedule_,
bool for_disk_azure_blob_storage_,
const Poco::Logger * log_)
: create_read_buffer(create_read_buffer_)
, client(client_)
, offset (offset_)
, total_size (total_size_)
, dest_container_for_logging(dest_container_for_logging_)
, dest_blob(dest_blob_)
, settings(settings_)
, schedule(schedule_)
, for_disk_azure_blob_storage(for_disk_azure_blob_storage_)
, log(log_)
, max_single_part_upload_size(settings_->max_single_part_upload_size)
{
}
virtual ~UploadHelper() = default;
protected:
std::function<std::unique_ptr<SeekableReadBuffer>()> create_read_buffer;
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> client;
size_t offset;
size_t total_size;
const String & dest_container_for_logging;
const String & dest_blob;
std::shared_ptr<const AzureObjectStorageSettings> settings;
ThreadPoolCallbackRunner<void> schedule;
bool for_disk_azure_blob_storage;
const Poco::Logger * log;
size_t max_single_part_upload_size;
struct UploadPartTask
{
size_t part_offset;
size_t part_size;
std::vector<std::string> block_ids;
bool is_finished = false;
std::exception_ptr exception;
};
size_t normal_part_size;
std::vector<std::string> block_ids;
std::list<UploadPartTask> TSA_GUARDED_BY(bg_tasks_mutex) bg_tasks;
int num_added_bg_tasks TSA_GUARDED_BY(bg_tasks_mutex) = 0;
int num_finished_bg_tasks TSA_GUARDED_BY(bg_tasks_mutex) = 0;
std::mutex bg_tasks_mutex;
std::condition_variable bg_tasks_condvar;
void calculatePartSize()
{
auto max_upload_part_size = settings->max_upload_part_size;
if (!max_upload_part_size)
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "max_upload_part_size must not be 0");
/// We've calculated the size of a normal part (the final part can be smaller).
normal_part_size = max_upload_part_size;
}
public:
void performCopy()
{
performMultipartUpload();
}
void completeMultipartUpload()
{
auto block_blob_client = client->GetBlockBlobClient(dest_blob);
block_blob_client.CommitBlockList(block_ids);
}
void performMultipartUpload()
{
calculatePartSize();
size_t position = offset;
size_t end_position = offset + total_size;
try
{
while (position < end_position)
{
size_t next_position = std::min(position + normal_part_size, end_position);
size_t part_size = next_position - position; /// `part_size` is either `normal_part_size` or smaller if it's the final part.
uploadPart(position, part_size);
position = next_position;
}
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
waitForAllBackgroundTasks();
throw;
}
waitForAllBackgroundTasks();
completeMultipartUpload();
}
void uploadPart(size_t part_offset, size_t part_size)
{
LOG_TRACE(log, "Writing part. Container: {}, Blob: {}, Size: {}", dest_container_for_logging, dest_blob, part_size);
if (!part_size)
{
LOG_TRACE(log, "Skipping writing an empty part.");
return;
}
if (schedule)
{
UploadPartTask * task = nullptr;
{
std::lock_guard lock(bg_tasks_mutex);
task = &bg_tasks.emplace_back();
++num_added_bg_tasks;
}
/// Notify waiting thread when task finished
auto task_finish_notify = [this, task]()
{
std::lock_guard lock(bg_tasks_mutex);
task->is_finished = true;
++num_finished_bg_tasks;
/// Notification under mutex is important here.
/// Otherwise, WriteBuffer could be destroyed in between
/// Releasing lock and condvar notification.
bg_tasks_condvar.notify_one();
};
try
{
task->part_offset = part_offset;
task->part_size = part_size;
schedule([this, task, task_finish_notify]()
{
try
{
processUploadPartRequest(*task);
}
catch (...)
{
task->exception = std::current_exception();
}
task_finish_notify();
}, Priority{});
}
catch (...)
{
task_finish_notify();
throw;
}
}
else
{
UploadPartTask task;
task.part_offset = part_offset;
task.part_size = part_size;
processUploadPartRequest(task);
block_ids.insert(block_ids.end(),task.block_ids.begin(), task.block_ids.end());
}
}
void processUploadPartRequest(UploadPartTask & task)
{
ProfileEvents::increment(ProfileEvents::AzureUploadPart);
if (for_disk_azure_blob_storage)
ProfileEvents::increment(ProfileEvents::DiskAzureUploadPart);
auto block_blob_client = client->GetBlockBlobClient(dest_blob);
auto read_buffer = std::make_unique<LimitSeekableReadBuffer>(create_read_buffer(), task.part_offset, task.part_size);
while (!read_buffer->eof())
{
auto size = read_buffer->available();
if (size > 0)
{
auto block_id = getRandomASCIIString(64);
Azure::Core::IO::MemoryBodyStream memory(reinterpret_cast<const uint8_t *>(read_buffer->position()), size);
block_blob_client.StageBlock(block_id, memory);
task.block_ids.emplace_back(block_id);
read_buffer->ignore(size);
LOG_TRACE(log, "Writing part. Container: {}, Blob: {}, block_id: {}", dest_container_for_logging, dest_blob, block_id);
}
}
std::lock_guard lock(bg_tasks_mutex); /// Protect bg_tasks from race
LOG_TRACE(log, "Writing part finished. Container: {}, Blob: {}, Parts: {}", dest_container_for_logging, dest_blob, bg_tasks.size());
}
void waitForAllBackgroundTasks()
{
if (!schedule)
return;
std::unique_lock lock(bg_tasks_mutex);
/// Suppress warnings because bg_tasks_mutex is actually hold, but tsa annotations do not understand std::unique_lock
bg_tasks_condvar.wait(lock, [this]() {return TSA_SUPPRESS_WARNING_FOR_READ(num_added_bg_tasks) == TSA_SUPPRESS_WARNING_FOR_READ(num_finished_bg_tasks); });
auto & tasks = TSA_SUPPRESS_WARNING_FOR_WRITE(bg_tasks);
for (auto & task : tasks)
{
if (task.exception)
std::rethrow_exception(task.exception);
block_ids.insert(block_ids.end(),task.block_ids.begin(), task.block_ids.end());
}
}
};
}
void copyDataToAzureBlobStorageFile(
const std::function<std::unique_ptr<SeekableReadBuffer>()> & create_read_buffer,
size_t offset,
size_t size,
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> dest_client,
const String & dest_container_for_logging,
const String & dest_blob,
std::shared_ptr<const AzureObjectStorageSettings> settings,
ThreadPoolCallbackRunner<void> schedule,
bool for_disk_azure_blob_storage)
{
UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container_for_logging, dest_blob, settings, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyDataToAzureBlobStorageFile")};
helper.performCopy();
}
void copyAzureBlobStorageFile(
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> src_client,
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> dest_client,
const String & src_container_for_logging,
const String & src_blob,
size_t offset,
size_t size,
const String & dest_container_for_logging,
const String & dest_blob,
std::shared_ptr<const AzureObjectStorageSettings> settings,
const ReadSettings & read_settings,
ThreadPoolCallbackRunner<void> schedule,
bool for_disk_azure_blob_storage)
{
if (settings->use_native_copy)
{
ProfileEvents::increment(ProfileEvents::AzureCopyObject);
if (for_disk_azure_blob_storage)
ProfileEvents::increment(ProfileEvents::DiskAzureCopyObject);
auto block_blob_client_src = src_client->GetBlockBlobClient(src_blob);
auto block_blob_client_dest = dest_client->GetBlockBlobClient(dest_blob);
auto source_uri = block_blob_client_src.GetUrl();
if (size < settings->max_single_part_copy_size)
{
block_blob_client_dest.CopyFromUri(source_uri);
}
else
{
Azure::Storage::Blobs::StartBlobCopyOperation operation = block_blob_client_dest.StartCopyFromUri(source_uri);
// Wait for the operation to finish, checking for status every 100 second.
auto copy_response = operation.PollUntilDone(std::chrono::milliseconds(100));
auto properties_model = copy_response.Value;
if (properties_model.CopySource.HasValue())
{
throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Copy failed");
}
}
}
else
{
LOG_TRACE(&Poco::Logger::get("copyAzureBlobStorageFile"), "Reading from Container: {}, Blob: {}", src_container_for_logging, src_blob);
auto create_read_buffer = [&]
{
return std::make_unique<ReadBufferFromAzureBlobStorage>(src_client, src_blob, read_settings, settings->max_single_read_retries,
settings->max_single_download_retries);
};
UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container_for_logging, dest_blob, settings, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyAzureBlobStorageFile")};
helper.performCopy();
}
}
}
#endif

View File

@ -0,0 +1,56 @@
#pragma once
#include "config.h"
#if USE_AZURE_BLOB_STORAGE
#include <Storages/StorageAzureBlobCluster.h>
#include <Storages/StorageAzureBlob.h>
#include <Interpreters/threadPoolCallbackRunner.h>
#include <base/types.h>
#include <functional>
#include <memory>
namespace DB
{
class SeekableReadBuffer;
using CreateReadBuffer = std::function<std::unique_ptr<SeekableReadBuffer>()>;
/// Copies a file from AzureBlobStorage to AzureBlobStorage.
/// The parameters `src_offset` and `src_size` specify a part in the source to copy.
void copyAzureBlobStorageFile(
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> src_client,
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> dest_client,
const String & src_container_for_logging,
const String & src_blob,
size_t src_offset,
size_t src_size,
const String & dest_container_for_logging,
const String & dest_blob,
std::shared_ptr<const AzureObjectStorageSettings> settings,
const ReadSettings & read_settings,
ThreadPoolCallbackRunner<void> schedule_ = {},
bool for_disk_azure_blob_storage = false);
/// Copies data from any seekable source to AzureBlobStorage.
/// The same functionality can be done by using the function copyData() and the class WriteBufferFromS3
/// however copyDataToS3File() is faster and spends less memory.
/// The callback `create_read_buffer` can be called from multiple threads in parallel, so that should be thread-safe.
/// The parameters `offset` and `size` specify a part in the source to copy.
void copyDataToAzureBlobStorageFile(
const std::function<std::unique_ptr<SeekableReadBuffer>()> & create_read_buffer,
size_t offset,
size_t size,
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> client,
const String & dest_container_for_logging,
const String & dest_blob,
std::shared_ptr<const AzureObjectStorageSettings> settings,
ThreadPoolCallbackRunner<void> schedule_ = {},
bool for_disk_azure_blob_storage = false);
}
#endif

View File

@ -353,7 +353,18 @@ AsynchronousInsertQueue::pushDataChunk(ASTPtr query, DataChunk chunk, ContextPtr
auto [it, inserted] = shard.iterators.try_emplace(key.hash);
auto now = std::chrono::steady_clock::now();
auto timeout_ms = getBusyWaitTimeoutMs(settings, shard, shard_num, flush_time_points, now);
auto timeout_ms = getBusyWaitTimeoutMs(settings, shard, flush_time_points, now);
if (timeout_ms != shard.busy_timeout_ms)
{
LOG_TRACE(
log,
"Asynchronous timeout {} from {} to {} for queue shard {}.",
timeout_ms < shard.busy_timeout_ms ? "decreased" : "increased",
shard.busy_timeout_ms.count(),
timeout_ms.count(),
size_t(shard_num));
}
if (inserted)
it->second = shard.queue.emplace(now + timeout_ms, Container{key, std::make_unique<InsertData>(timeout_ms)}).first;
@ -431,7 +442,6 @@ AsynchronousInsertQueue::pushDataChunk(ASTPtr query, DataChunk chunk, ContextPtr
AsynchronousInsertQueue::Milliseconds AsynchronousInsertQueue::getBusyWaitTimeoutMs(
const Settings & settings,
const QueueShard & shard,
size_t shard_num,
const QueueShardFlushTimeHistory::TimePoints & flush_time_points,
std::chrono::steady_clock::time_point now) const
{
@ -460,13 +470,6 @@ AsynchronousInsertQueue::Milliseconds AsynchronousInsertQueue::getBusyWaitTimeou
auto timeout_ms = std::max(
std::chrono::duration_cast<Milliseconds>(shard.busy_timeout_ms * (1.0 + increase_rate)),
shard.busy_timeout_ms + Milliseconds(1));
if (timeout_ms != shard.busy_timeout_ms)
LOG_TRACE(
log,
"Async timeout increased from {} to {} for queue shard {}.",
shard.busy_timeout_ms.count(),
timeout_ms.count(),
shard_num);
return normalize(timeout_ms);
}
@ -475,18 +478,7 @@ AsynchronousInsertQueue::Milliseconds AsynchronousInsertQueue::getBusyWaitTimeou
/// long enough (exceeding the adjusted timeout).
/// This ensures the timeout value converges to the minimum over time for non-frequent inserts.
else if (last_insert_time + decreased_timeout_ms < now && t1 + decreased_timeout_ms < t2)
{
auto timeout_ms = decreased_timeout_ms;
if (timeout_ms != shard.busy_timeout_ms)
LOG_TRACE(
log,
"Async timeout decreased from {} to {} for queue shard {}.",
shard.busy_timeout_ms.count(),
timeout_ms.count(),
shard_num);
return normalize(timeout_ms);
}
return normalize(decreased_timeout_ms);
return normalize(shard.busy_timeout_ms);
}

View File

@ -248,7 +248,6 @@ private:
Milliseconds getBusyWaitTimeoutMs(
const Settings & settings,
const QueueShard & shard,
size_t shard_num,
const QueueShardFlushTimeHistory::TimePoints & flush_time_points,
std::chrono::steady_clock::time_point now) const;

View File

@ -32,6 +32,7 @@ namespace ErrorCodes
extern const int TOO_LARGE_DISTRIBUTED_DEPTH;
extern const int LOGICAL_ERROR;
extern const int CLUSTER_DOESNT_EXIST;
extern const int UNEXPECTED_CLUSTER;
}
namespace ClusterProxy
@ -374,12 +375,12 @@ void executeQueryWithParallelReplicas(
shard_num = column->getUInt(0);
}
ClusterPtr new_cluster;
const auto shard_count = not_optimized_cluster->getShardCount();
ClusterPtr new_cluster = not_optimized_cluster;
/// if got valid shard_num from query initiator, then parallel replicas scope is the specified shard
/// shards are numbered in order of appearance in the cluster config
if (shard_num > 0)
{
const auto shard_count = not_optimized_cluster->getShardCount();
if (shard_num > shard_count)
throw Exception(
ErrorCodes::LOGICAL_ERROR,
@ -395,17 +396,18 @@ void executeQueryWithParallelReplicas(
// get cluster for shard specified by shard_num
// shard_num is 1-based, but getClusterWithSingleShard expects 0-based index
auto single_shard_cluster = not_optimized_cluster->getClusterWithSingleShard(shard_num - 1);
// convert cluster to representation expected by parallel replicas
new_cluster = single_shard_cluster->getClusterWithReplicasAsShards(settings, settings.max_parallel_replicas);
new_cluster = not_optimized_cluster->getClusterWithSingleShard(shard_num - 1);
}
else
{
new_cluster = not_optimized_cluster->getClusterWithReplicasAsShards(settings, settings.max_parallel_replicas);
if (not_optimized_cluster->getShardCount() > 1)
throw DB::Exception(
ErrorCodes::UNEXPECTED_CLUSTER,
"`cluster_for_parallel_replicas` setting refers to cluster with several shards. Expected a cluster with one shard");
}
auto coordinator
= std::make_shared<ParallelReplicasReadingCoordinator>(new_cluster->getShardCount(), settings.parallel_replicas_mark_segment_size);
auto coordinator = std::make_shared<ParallelReplicasReadingCoordinator>(
new_cluster->getShardsInfo().begin()->getAllNodeCount(), settings.parallel_replicas_mark_segment_size);
auto external_tables = new_context->getExternalTables();
auto read_from_remote = std::make_unique<ReadFromParallelRemoteReplicasStep>(
query_ast,

View File

@ -1076,15 +1076,22 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data
{
const auto * kind = create.is_dictionary ? "Dictionary" : "Table";
const auto * kind_upper = create.is_dictionary ? "DICTIONARY" : "TABLE";
bool is_replicated_database_internal = database->getEngineName() == "Replicated" && getContext()->getClientInfo().is_replicated_database_internal;
bool from_path = create.attach_from_path.has_value();
bool is_on_cluster = getContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
if (database->getEngineName() == "Replicated" && getContext()->getClientInfo().is_replicated_database_internal
&& !internal)
if (is_replicated_database_internal && !internal)
{
if (create.uuid == UUIDHelpers::Nil)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table UUID is not specified in DDL log");
}
bool from_path = create.attach_from_path.has_value();
if (create.refresh_strategy && database->getEngineName() != "Atomic")
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Refreshable materialized view requires Atomic database engine, but database {} has engine {}", create.getDatabase(), database->getEngineName());
/// TODO: Support Replicated databases, only with Shared/ReplicatedMergeTree.
/// Figure out how to make the refreshed data appear all at once on other
/// replicas; maybe a replicated SYSTEM SYNC REPLICA query before the rename?
if (database->getUUID() != UUIDHelpers::Nil)
{
@ -1108,7 +1115,6 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data
}
else
{
bool is_on_cluster = getContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
bool has_uuid = create.uuid != UUIDHelpers::Nil || create.to_inner_uuid != UUIDHelpers::Nil;
if (has_uuid && !is_on_cluster && !internal)
{
@ -1121,13 +1127,6 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data
"{} UUID specified, but engine of database {} is not Atomic", kind, create.getDatabase());
}
if (create.refresh_strategy && database->getEngineName() != "Atomic")
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Refreshable materialized view requires Atomic database engine, but database {} has engine {}", create.getDatabase(), database->getEngineName());
/// TODO: Support Replicated databases, only with Shared/ReplicatedMergeTree.
/// Figure out how to make the refreshed data appear all at once on other
/// replicas; maybe a replicated SYSTEM SYNC REPLICA query before the rename?
/// The database doesn't support UUID so we'll ignore it. The UUID could be set here because of either
/// a) the initiator of `ON CLUSTER` query generated it to ensure the same UUIDs are used on different hosts; or
/// b) `RESTORE from backup` query generated it to ensure the same UUIDs are used on different hosts.

View File

@ -77,6 +77,7 @@ public:
const DataTypes & getElementsTypes() const { return set_elements_types; }
bool hasExplicitSetElements() const { return fill_set_elements || (!set_elements.empty() && set_elements.front()->size() == data.getTotalRowCount()); }
bool hasSetElements() const { return !set_elements.empty(); }
Columns getSetElements() const { checkIsCreated(); return { set_elements.begin(), set_elements.end() }; }
void checkColumnsNumber(size_t num_key_columns) const;

View File

@ -933,6 +933,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
reason = "asynchronous insert queue is not configured";
else if (insert_query->select)
reason = "insert query has select";
else if (settings.deduplicate_blocks_in_dependent_materialized_views)
reason = "dependent materialized views block deduplication is enabled";
else if (insert_query->hasInlinedData())
async_insert = true;

View File

@ -100,7 +100,7 @@ public:
merged_rows = 0;
sum_blocks_granularity = 0;
++total_chunks;
total_allocated_bytes += chunk.allocatedBytes();
total_allocated_bytes += chunk.bytes();
need_flush = false;
return chunk;
@ -122,7 +122,7 @@ public:
{
size_t merged_bytes = 0;
for (const auto & column : columns)
merged_bytes += column->allocatedBytes();
merged_bytes += column->byteSize();
if (merged_bytes >= max_block_size_bytes)
return true;
}

View File

@ -157,6 +157,34 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::Subqueries subque
query_plan.unitePlans(std::move(creating_sets), std::move(plans));
}
QueryPipelineBuilderPtr addCreatingSetsTransform(QueryPipelineBuilderPtr pipeline, PreparedSets::Subqueries subqueries, ContextPtr context)
{
DataStreams input_streams;
input_streams.emplace_back(DataStream{pipeline->getHeader()});
QueryPipelineBuilders pipelines;
pipelines.reserve(1 + subqueries.size());
pipelines.push_back(std::move(pipeline));
auto plan_settings = QueryPlanOptimizationSettings::fromContext(context);
auto pipeline_settings = BuildQueryPipelineSettings::fromContext(context);
for (auto & future_set : subqueries)
{
if (future_set->get())
continue;
auto plan = future_set->build(context);
if (!plan)
continue;
input_streams.emplace_back(plan->getCurrentDataStream());
pipelines.emplace_back(plan->buildQueryPipeline(plan_settings, pipeline_settings));
}
return CreatingSetsStep(input_streams).updatePipeline(std::move(pipelines), pipeline_settings);
}
std::vector<std::unique_ptr<QueryPlan>> DelayedCreatingSetsStep::makePlansForSets(DelayedCreatingSetsStep && step)
{
std::vector<std::unique_ptr<QueryPlan>> plans;

View File

@ -72,4 +72,6 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::Subqueries subque
void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context);
QueryPipelineBuilderPtr addCreatingSetsTransform(QueryPipelineBuilderPtr pipeline, PreparedSets::Subqueries subqueries, ContextPtr context);
}

View File

@ -12,7 +12,7 @@
#include <Processors/Sources/DelayedSource.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Interpreters/ActionsDAG.h>
#include "Common/logger_useful.h"
#include <Common/logger_useful.h>
#include <Common/checkStackSize.h>
#include <Core/QueryProcessingStage.h>
#include <Client/ConnectionPool.h>
@ -375,10 +375,11 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
, storage_limits(std::move(storage_limits_))
, log(log_)
{
std::vector<String> description;
chassert(cluster->getShardCount() == 1);
for (const auto & address : cluster->getShardsAddresses())
description.push_back(fmt::format("Replica: {}", address[0].host_name));
std::vector<String> description;
for (const auto & pool : cluster->getShardsInfo().front().per_replica_pools)
description.push_back(fmt::format("Replica: {}", pool->getHost()));
setStepDescription(boost::algorithm::join(description, ", "));
}
@ -399,51 +400,44 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder
const Settings & current_settings = context->getSettingsRef();
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
const auto & shard = cluster->getShardsInfo().at(0);
size_t all_replicas_count = current_settings.max_parallel_replicas;
if (all_replicas_count > cluster->getShardsInfo().size())
if (all_replicas_count > shard.getAllNodeCount())
{
LOG_INFO(getLogger("ReadFromParallelRemoteReplicasStep"),
"The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). "\
"Will use the latter number to execute the query.", current_settings.max_parallel_replicas, cluster->getShardsInfo().size());
all_replicas_count = cluster->getShardsInfo().size();
LOG_INFO(
getLogger("ReadFromParallelRemoteReplicasStep"),
"The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). "
"Will use the latter number to execute the query.",
current_settings.max_parallel_replicas,
shard.getAllNodeCount());
all_replicas_count = shard.getAllNodeCount();
}
/// Find local shard. It might happen that there is no local shard, but that's fine
for (const auto & shard: cluster->getShardsInfo())
{
if (shard.isLocal())
{
IConnections::ReplicaInfo replica_info
{
.all_replicas_count = all_replicas_count,
/// `shard_num` will be equal to the number of the given replica in the cluster (set by `Cluster::getClusterWithReplicasAsShards`).
/// we should use this number specifically because efficiency of data distribution by consistent hash depends on it.
.number_of_current_replica = shard.shard_num - 1,
};
addPipeForSingeReplica(pipes, shard.pool, replica_info);
}
std::vector<ConnectionPoolWithFailover::Base::ShuffledPool> shuffled_pool;
if (all_replicas_count < shard.getAllNodeCount())
{
shuffled_pool = shard.pool->getShuffledPools(current_settings);
shuffled_pool.resize(all_replicas_count);
}
else
{
/// try to preserve replicas order if all replicas in cluster are used for query execution
/// it's important for data locality during query execution
auto priority_func = [](size_t i) { return Priority{static_cast<Int64>(i)}; };
shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func);
}
auto current_shard = cluster->getShardsInfo().begin();
while (pipes.size() != all_replicas_count)
for (size_t i=0; i < all_replicas_count; ++i)
{
if (current_shard->isLocal())
{
++current_shard;
continue;
}
IConnections::ReplicaInfo replica_info
{
.all_replicas_count = all_replicas_count,
/// `shard_num` will be equal to the number of the given replica in the cluster (set by `Cluster::getClusterWithReplicasAsShards`).
/// we should use this number specifically because efficiency of data distribution by consistent hash depends on it.
.number_of_current_replica = current_shard->shard_num - 1,
.number_of_current_replica = i,
};
addPipeForSingeReplica(pipes, current_shard->pool, replica_info);
++current_shard;
addPipeForSingeReplica(pipes, shuffled_pool[i].pool, replica_info);
}
auto pipe = Pipe::unitePipes(std::move(pipes));
@ -456,7 +450,8 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder
}
void ReadFromParallelRemoteReplicasStep::addPipeForSingeReplica(Pipes & pipes, std::shared_ptr<ConnectionPoolWithFailover> pool, IConnections::ReplicaInfo replica_info)
void ReadFromParallelRemoteReplicasStep::addPipeForSingeReplica(
Pipes & pipes, const ConnectionPoolPtr & pool, IConnections::ReplicaInfo replica_info)
{
bool add_agg_info = stage == QueryProcessingStage::WithMergeableState;
bool add_totals = false;
@ -476,7 +471,14 @@ void ReadFromParallelRemoteReplicasStep::addPipeForSingeReplica(Pipes & pipes, s
assert(output_stream);
auto remote_query_executor = std::make_shared<RemoteQueryExecutor>(
pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage,
pool,
query_string,
output_stream->header,
context,
throttler,
scalars,
external_tables,
stage,
RemoteQueryExecutor::Extension{.parallel_reading_coordinator = coordinator, .replica_info = std::move(replica_info)});
remote_query_executor->setLogger(log);

View File

@ -9,10 +9,6 @@
namespace DB
{
class ConnectionPoolWithFailover;
using ConnectionPoolWithFailoverPtr = std::shared_ptr<ConnectionPoolWithFailover>;
class Throttler;
using ThrottlerPtr = std::shared_ptr<Throttler>;
@ -91,8 +87,7 @@ public:
void enforceAggregationInOrder();
private:
void addPipeForSingeReplica(Pipes & pipes, std::shared_ptr<ConnectionPoolWithFailover> pool, IConnections::ReplicaInfo replica_info);
void addPipeForSingeReplica(Pipes & pipes, const ConnectionPoolPtr & pool, IConnections::ReplicaInfo replica_info);
ClusterPtr cluster;
ASTPtr query_ast;

View File

@ -11,8 +11,9 @@ namespace ErrorCodes
}
ITTLAlgorithm::ITTLAlgorithm(
const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_)
: description(description_)
const TTLExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_)
: ttl_expressions(ttl_expressions_)
, description(description_)
, old_ttl_info(old_ttl_info_)
, current_time(current_time_)
, force(force_)

View File

@ -8,6 +8,12 @@
namespace DB
{
struct TTLExpressions
{
ExpressionActionsPtr expression;
ExpressionActionsPtr where_expression;
};
/**
* Represents the actions, which are required to do
* with data, when TTL is expired: delete, aggregate, etc.
@ -18,7 +24,7 @@ public:
using TTLInfo = IMergeTreeDataPart::TTLInfo;
using MutableDataPartPtr = MergeTreeMutableDataPartPtr;
ITTLAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_);
ITTLAlgorithm(const TTLExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_);
virtual ~ITTLAlgorithm() = default;
virtual void execute(Block & block) = 0;
@ -39,6 +45,7 @@ protected:
bool isTTLExpired(time_t ttl) const;
UInt32 getTimestampByIndex(const IColumn * column, size_t index) const;
const TTLExpressions ttl_expressions;
const TTLDescription description;
const TTLInfo old_ttl_info;
const time_t current_time;

View File

@ -5,13 +5,14 @@ namespace DB
{
TTLAggregationAlgorithm::TTLAggregationAlgorithm(
const TTLExpressions & ttl_expressions_,
const TTLDescription & description_,
const TTLInfo & old_ttl_info_,
time_t current_time_,
bool force_,
const Block & header_,
const MergeTreeData & storage_)
: ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_)
: ITTLAlgorithm(ttl_expressions_, description_, old_ttl_info_, current_time_, force_)
, header(header_)
{
current_key_value.resize(description.group_by_keys.size());
@ -75,8 +76,8 @@ void TTLAggregationAlgorithm::execute(Block & block)
const auto & column_names = header.getNames();
MutableColumns aggregate_columns = header.cloneEmptyColumns();
auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column);
auto where_column = executeExpressionAndGetColumn(description.where_expression, block, description.where_result_column);
auto ttl_column = executeExpressionAndGetColumn(ttl_expressions.expression, block, description.result_column);
auto where_column = executeExpressionAndGetColumn(ttl_expressions.where_expression, block, description.where_result_column);
size_t rows_aggregated = 0;
size_t current_key_start = 0;
@ -157,8 +158,8 @@ void TTLAggregationAlgorithm::execute(Block & block)
/// If some rows were aggregated we have to recalculate ttl info's
if (some_rows_were_aggregated)
{
auto ttl_column_after_aggregation = executeExpressionAndGetColumn(description.expression, block, description.result_column);
auto where_column_after_aggregation = executeExpressionAndGetColumn(description.where_expression, block, description.where_result_column);
auto ttl_column_after_aggregation = executeExpressionAndGetColumn(ttl_expressions.expression, block, description.result_column);
auto where_column_after_aggregation = executeExpressionAndGetColumn(ttl_expressions.where_expression, block, description.where_result_column);
for (size_t i = 0; i < block.rows(); ++i)
{
bool where_filter_passed = !where_column_after_aggregation || where_column_after_aggregation->getBool(i);

View File

@ -13,6 +13,7 @@ class TTLAggregationAlgorithm final : public ITTLAlgorithm
{
public:
TTLAggregationAlgorithm(
const TTLExpressions & ttl_expressions_,
const TTLDescription & description_,
const TTLInfo & old_ttl_info_,
time_t current_time_,

View File

@ -4,6 +4,7 @@ namespace DB
{
TTLColumnAlgorithm::TTLColumnAlgorithm(
const TTLExpressions & ttl_expressions_,
const TTLDescription & description_,
const TTLInfo & old_ttl_info_,
time_t current_time_,
@ -12,7 +13,7 @@ TTLColumnAlgorithm::TTLColumnAlgorithm(
const ExpressionActionsPtr & default_expression_,
const String & default_column_name_,
bool is_compact_part_)
: ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_)
: ITTLAlgorithm(ttl_expressions_, description_, old_ttl_info_, current_time_, force_)
, column_name(column_name_)
, default_expression(default_expression_)
, default_column_name(default_column_name_)
@ -49,7 +50,7 @@ void TTLColumnAlgorithm::execute(Block & block)
if (default_column)
default_column = default_column->convertToFullColumnIfConst();
auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column);
auto ttl_column = executeExpressionAndGetColumn(ttl_expressions.expression, block, description.result_column);
auto & column_with_type = block.getByName(column_name);
const IColumn * values_column = column_with_type.column.get();

View File

@ -11,6 +11,7 @@ class TTLColumnAlgorithm final : public ITTLAlgorithm
{
public:
TTLColumnAlgorithm(
const TTLExpressions & ttl_expressions_,
const TTLDescription & description_,
const TTLInfo & old_ttl_info_,
time_t current_time_,

View File

@ -4,8 +4,8 @@ namespace DB
{
TTLDeleteAlgorithm::TTLDeleteAlgorithm(
const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_)
: ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_)
const TTLExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_)
: ITTLAlgorithm(ttl_expressions_, description_, old_ttl_info_, current_time_, force_)
{
if (!isMinTTLExpired())
new_ttl_info = old_ttl_info;
@ -19,8 +19,8 @@ void TTLDeleteAlgorithm::execute(Block & block)
if (!block || !isMinTTLExpired())
return;
auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column);
auto where_column = executeExpressionAndGetColumn(description.where_expression, block, description.where_result_column);
auto ttl_column = executeExpressionAndGetColumn(ttl_expressions.expression, block, description.result_column);
auto where_column = executeExpressionAndGetColumn(ttl_expressions.where_expression, block, description.where_result_column);
MutableColumns result_columns;
const auto & column_names = block.getNames();
@ -54,7 +54,7 @@ void TTLDeleteAlgorithm::execute(Block & block)
void TTLDeleteAlgorithm::finalize(const MutableDataPartPtr & data_part) const
{
if (description.where_expression)
if (ttl_expressions.where_expression)
data_part->ttl_infos.rows_where_ttl[description.result_column] = new_ttl_info;
else
data_part->ttl_infos.table_ttl = new_ttl_info;

View File

@ -10,7 +10,7 @@ namespace DB
class TTLDeleteAlgorithm final : public ITTLAlgorithm
{
public:
TTLDeleteAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_);
TTLDeleteAlgorithm(const TTLExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_);
void execute(Block & block) override;
void finalize(const MutableDataPartPtr & data_part) const override;

View File

@ -4,13 +4,14 @@ namespace DB
{
TTLUpdateInfoAlgorithm::TTLUpdateInfoAlgorithm(
const TTLExpressions & ttl_expressions_,
const TTLDescription & description_,
const TTLUpdateField ttl_update_field_,
const String ttl_update_key_,
const TTLInfo & old_ttl_info_,
time_t current_time_,
bool force_)
: ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_)
: ITTLAlgorithm(ttl_expressions_, description_, old_ttl_info_, current_time_, force_)
, ttl_update_field(ttl_update_field_)
, ttl_update_key(ttl_update_key_)
{
@ -21,7 +22,7 @@ void TTLUpdateInfoAlgorithm::execute(Block & block)
if (!block)
return;
auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column);
auto ttl_column = executeExpressionAndGetColumn(ttl_expressions.expression, block, description.result_column);
for (size_t i = 0; i < block.rows(); ++i)
{
UInt32 cur_ttl = ITTLAlgorithm::getTimestampByIndex(ttl_column.get(), i);

View File

@ -20,6 +20,7 @@ class TTLUpdateInfoAlgorithm : public ITTLAlgorithm
{
public:
TTLUpdateInfoAlgorithm(
const TTLExpressions & ttl_expressions_,
const TTLDescription & description_,
const TTLUpdateField ttl_update_field_,
const String ttl_update_key_,

View File

@ -145,10 +145,14 @@ void ColumnGathererStream::gather(Column & column_res)
next_required_source = -1;
while (row_source_pos < row_sources_end
&& column_res.size() < block_preferred_size_rows
&& column_res.allocatedBytes() < block_preferred_size_bytes)
/// We use do ... while here to ensure there will be at least one iteration of this loop.
/// Because the column_res.byteSize() could be bigger than block_preferred_size_bytes already at this point.
do
{
if (row_source_pos >= row_sources_end)
break;
RowSourcePart row_source = *row_source_pos;
size_t source_num = row_source.getSourceNum();
Source & source = sources[source_num];
@ -191,7 +195,7 @@ void ColumnGathererStream::gather(Column & column_res)
}
source.pos += len;
}
} while (column_res.size() < block_preferred_size_rows && column_res.byteSize() < block_preferred_size_bytes);
}
}

View File

@ -4,7 +4,24 @@
namespace DB
{
static TTLExpressions getExpressions(const TTLDescription & ttl_descr, PreparedSets::Subqueries & subqueries_for_sets, const ContextPtr & context)
{
auto expr = ttl_descr.buildExpression(context);
auto expr_queries = expr.sets->getSubqueries();
subqueries_for_sets.insert(subqueries_for_sets.end(), expr_queries.begin(), expr_queries.end());
auto where_expr = ttl_descr.buildWhereExpression(context);
if (where_expr.sets)
{
auto where_expr_queries = where_expr.sets->getSubqueries();
subqueries_for_sets.insert(subqueries_for_sets.end(), where_expr_queries.begin(), where_expr_queries.end());
}
return {expr.expression, where_expr.expression};
}
TTLCalcTransform::TTLCalcTransform(
const ContextPtr & context,
const Block & header_,
const MergeTreeData & storage_,
const StorageMetadataPtr & metadata_snapshot_,
@ -21,33 +38,39 @@ TTLCalcTransform::TTLCalcTransform(
{
const auto & rows_ttl = metadata_snapshot_->getRowsTTL();
algorithms.emplace_back(std::make_unique<TTLUpdateInfoAlgorithm>(
rows_ttl, TTLUpdateField::TABLE_TTL, rows_ttl.result_column, old_ttl_infos.table_ttl, current_time_, force_));
getExpressions(rows_ttl, subqueries_for_sets, context), rows_ttl,
TTLUpdateField::TABLE_TTL, rows_ttl.result_column, old_ttl_infos.table_ttl, current_time_, force_));
}
for (const auto & where_ttl : metadata_snapshot_->getRowsWhereTTLs())
algorithms.emplace_back(std::make_unique<TTLUpdateInfoAlgorithm>(
where_ttl, TTLUpdateField::ROWS_WHERE_TTL, where_ttl.result_column, old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_));
getExpressions(where_ttl, subqueries_for_sets, context), where_ttl,
TTLUpdateField::ROWS_WHERE_TTL, where_ttl.result_column, old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_));
for (const auto & group_by_ttl : metadata_snapshot_->getGroupByTTLs())
algorithms.emplace_back(std::make_unique<TTLUpdateInfoAlgorithm>(
group_by_ttl, TTLUpdateField::GROUP_BY_TTL, group_by_ttl.result_column, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_));
getExpressions(group_by_ttl, subqueries_for_sets, context), group_by_ttl,
TTLUpdateField::GROUP_BY_TTL, group_by_ttl.result_column, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_));
if (metadata_snapshot_->hasAnyColumnTTL())
{
for (const auto & [name, description] : metadata_snapshot_->getColumnTTLs())
{
algorithms.emplace_back(std::make_unique<TTLUpdateInfoAlgorithm>(
description, TTLUpdateField::COLUMNS_TTL, name, old_ttl_infos.columns_ttl[name], current_time_, force_));
getExpressions(description, subqueries_for_sets, context), description,
TTLUpdateField::COLUMNS_TTL, name, old_ttl_infos.columns_ttl[name], current_time_, force_));
}
}
for (const auto & move_ttl : metadata_snapshot_->getMoveTTLs())
algorithms.emplace_back(std::make_unique<TTLUpdateInfoAlgorithm>(
move_ttl, TTLUpdateField::MOVES_TTL, move_ttl.result_column, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_));
getExpressions(move_ttl, subqueries_for_sets, context), move_ttl,
TTLUpdateField::MOVES_TTL, move_ttl.result_column, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_));
for (const auto & recompression_ttl : metadata_snapshot_->getRecompressionTTLs())
algorithms.emplace_back(std::make_unique<TTLUpdateInfoAlgorithm>(
recompression_ttl, TTLUpdateField::RECOMPRESSION_TTL, recompression_ttl.result_column, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_));
getExpressions(recompression_ttl, subqueries_for_sets, context), recompression_ttl,
TTLUpdateField::RECOMPRESSION_TTL, recompression_ttl.result_column, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_));
}
void TTLCalcTransform::consume(Chunk chunk)

View File

@ -15,6 +15,7 @@ class TTLCalcTransform : public IAccumulatingTransform
{
public:
TTLCalcTransform(
const ContextPtr & context,
const Block & header_,
const MergeTreeData & storage_,
const StorageMetadataPtr & metadata_snapshot_,
@ -23,6 +24,8 @@ public:
bool force_
);
PreparedSets::Subqueries getSubqueries() { return std::move(subqueries_for_sets); }
String getName() const override { return "TTL_CALC"; }
Status prepare() override;
@ -35,6 +38,7 @@ protected:
private:
std::vector<TTLAlgorithmPtr> algorithms;
PreparedSets::Subqueries subqueries_for_sets;
/// ttl_infos and empty_columns are updating while reading
const MergeTreeData::MutableDataPartPtr & data_part;

View File

@ -16,7 +16,24 @@
namespace DB
{
static TTLExpressions getExpressions(const TTLDescription & ttl_descr, PreparedSets::Subqueries & subqueries_for_sets, const ContextPtr & context)
{
auto expr = ttl_descr.buildExpression(context);
auto expr_queries = expr.sets->getSubqueries();
subqueries_for_sets.insert(subqueries_for_sets.end(), expr_queries.begin(), expr_queries.end());
auto where_expr = ttl_descr.buildWhereExpression(context);
if (where_expr.sets)
{
auto where_expr_queries = where_expr.sets->getSubqueries();
subqueries_for_sets.insert(subqueries_for_sets.end(), where_expr_queries.begin(), where_expr_queries.end());
}
return {expr.expression, where_expr.expression};
}
TTLTransform::TTLTransform(
const ContextPtr & context,
const Block & header_,
const MergeTreeData & storage_,
const StorageMetadataPtr & metadata_snapshot_,
@ -33,10 +50,11 @@ TTLTransform::TTLTransform(
{
const auto & rows_ttl = metadata_snapshot_->getRowsTTL();
auto algorithm = std::make_unique<TTLDeleteAlgorithm>(
rows_ttl, old_ttl_infos.table_ttl, current_time_, force_);
getExpressions(rows_ttl, subqueries_for_sets, context), rows_ttl,
old_ttl_infos.table_ttl, current_time_, force_);
/// Skip all data if table ttl is expired for part
if (algorithm->isMaxTTLExpired() && !rows_ttl.where_expression)
if (algorithm->isMaxTTLExpired() && !rows_ttl.where_expression_ast)
all_data_dropped = true;
delete_algorithm = algorithm.get();
@ -45,11 +63,13 @@ TTLTransform::TTLTransform(
for (const auto & where_ttl : metadata_snapshot_->getRowsWhereTTLs())
algorithms.emplace_back(std::make_unique<TTLDeleteAlgorithm>(
where_ttl, old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_));
getExpressions(where_ttl, subqueries_for_sets, context), where_ttl,
old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_));
for (const auto & group_by_ttl : metadata_snapshot_->getGroupByTTLs())
algorithms.emplace_back(std::make_unique<TTLAggregationAlgorithm>(
group_by_ttl, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_,
getExpressions(group_by_ttl, subqueries_for_sets, context), group_by_ttl,
old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_,
getInputPort().getHeader(), storage_));
if (metadata_snapshot_->hasAnyColumnTTL())
@ -75,18 +95,21 @@ TTLTransform::TTLTransform(
}
algorithms.emplace_back(std::make_unique<TTLColumnAlgorithm>(
description, old_ttl_infos.columns_ttl[name], current_time_,
getExpressions(description, subqueries_for_sets, context), description,
old_ttl_infos.columns_ttl[name], current_time_,
force_, name, default_expression, default_column_name, isCompactPart(data_part)));
}
}
for (const auto & move_ttl : metadata_snapshot_->getMoveTTLs())
algorithms.emplace_back(std::make_unique<TTLUpdateInfoAlgorithm>(
move_ttl, TTLUpdateField::MOVES_TTL, move_ttl.result_column, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_));
getExpressions(move_ttl, subqueries_for_sets, context), move_ttl,
TTLUpdateField::MOVES_TTL, move_ttl.result_column, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_));
for (const auto & recompression_ttl : metadata_snapshot_->getRecompressionTTLs())
algorithms.emplace_back(std::make_unique<TTLUpdateInfoAlgorithm>(
recompression_ttl, TTLUpdateField::RECOMPRESSION_TTL, recompression_ttl.result_column, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_));
getExpressions(recompression_ttl, subqueries_for_sets, context), recompression_ttl,
TTLUpdateField::RECOMPRESSION_TTL, recompression_ttl.result_column, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_));
}
Block reorderColumns(Block block, const Block & header)

View File

@ -16,6 +16,7 @@ class TTLTransform : public IAccumulatingTransform
{
public:
TTLTransform(
const ContextPtr & context,
const Block & header_,
const MergeTreeData & storage_,
const StorageMetadataPtr & metadata_snapshot_,
@ -28,6 +29,8 @@ public:
Status prepare() override;
PreparedSets::Subqueries getSubqueries() { return std::move(subqueries_for_sets); }
protected:
void consume(Chunk chunk) override;
Chunk generate() override;
@ -40,6 +43,8 @@ private:
const TTLDeleteAlgorithm * delete_algorithm = nullptr;
bool all_data_dropped = false;
PreparedSets::Subqueries subqueries_for_sets;
/// ttl_infos and empty_columns are updating while reading
const MergeTreeData::MutableDataPartPtr & data_part;
LoggerPtr log;

View File

@ -4,7 +4,7 @@
#include <Columns/ColumnConst.h>
#include <Common/CurrentThread.h>
#include "Core/Protocol.h"
#include <Core/Protocol.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
@ -17,6 +17,7 @@
#include <Interpreters/Context.h>
#include <Interpreters/InternalTextLogsQueue.h>
#include <IO/ConnectionTimeouts.h>
#include <Client/ConnectionEstablisher.h>
#include <Client/MultiplexedConnections.h>
#include <Client/HedgedConnections.h>
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
@ -29,6 +30,7 @@ namespace ProfileEvents
extern const Event SuspendSendingQueryToShard;
extern const Event ReadTaskRequestsReceived;
extern const Event MergeTreeReadTaskRequestsReceived;
extern const Event ParallelReplicasAvailableCount;
}
namespace DB
@ -62,6 +64,55 @@ RemoteQueryExecutor::RemoteQueryExecutor(
{
}
RemoteQueryExecutor::RemoteQueryExecutor(
ConnectionPoolPtr pool,
const String & query_,
const Block & header_,
ContextPtr context_,
ThrottlerPtr throttler,
const Scalars & scalars_,
const Tables & external_tables_,
QueryProcessingStage::Enum stage_,
std::optional<Extension> extension_)
: RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_)
{
create_connections = [this, pool, throttler, extension_](AsyncCallback)
{
const Settings & current_settings = context->getSettingsRef();
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
ConnectionPoolWithFailover::TryResult result;
std::string fail_message;
if (main_table)
{
auto table_name = main_table.getQualifiedName();
ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, log, &table_name);
connection_establisher.run(result, fail_message);
}
else
{
ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, log, nullptr);
connection_establisher.run(result, fail_message);
}
std::vector<IConnectionPool::Entry> connection_entries;
if (!result.entry.isNull() && result.is_usable)
{
if (extension_ && extension_->parallel_reading_coordinator)
ProfileEvents::increment(ProfileEvents::ParallelReplicasAvailableCount);
connection_entries.emplace_back(std::move(result.entry));
}
auto res = std::make_unique<MultiplexedConnections>(std::move(connection_entries), current_settings, throttler);
if (extension_ && extension_->replica_info)
res->setReplicaInfo(*extension_->replica_info);
return res;
};
}
RemoteQueryExecutor::RemoteQueryExecutor(
Connection & connection,
const String & query_,

View File

@ -52,6 +52,18 @@ public:
std::optional<IConnections::ReplicaInfo> replica_info = {};
};
/// Takes a connection pool for a node (not cluster)
RemoteQueryExecutor(
ConnectionPoolPtr pool,
const String & query_,
const Block & header_,
ContextPtr context_,
ThrottlerPtr throttler = nullptr,
const Scalars & scalars_ = Scalars(),
const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
std::optional<Extension> extension_ = std::nullopt);
/// Takes already set connection.
RemoteQueryExecutor(
Connection & connection,

View File

@ -933,7 +933,7 @@ void TCPHandler::processInsertQuery()
if (auto table = DatabaseCatalog::instance().tryGetTable(insert_query.table_id, query_context))
async_insert_enabled |= table->areAsynchronousInsertsEnabled();
if (insert_queue && async_insert_enabled && !insert_query.select)
if (insert_queue && async_insert_enabled && !insert_query.select && !settings.deduplicate_blocks_in_dependent_materialized_views)
{
auto result = processAsyncInsertQuery(*insert_queue);
if (result.status == AsynchronousInsertQueue::PushResult::OK)

View File

@ -33,6 +33,9 @@
#include <Processors/Transforms/TTLCalcTransform.h>
#include <Processors/Transforms/DistinctSortedTransform.h>
#include <Processors/Transforms/DistinctTransform.h>
#include <Processors/QueryPlan/CreatingSetsStep.h>
#include <Interpreters/PreparedSets.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
namespace DB
{
@ -1056,13 +1059,14 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
break;
}
auto res_pipe = Pipe::unitePipes(std::move(pipes));
res_pipe.addTransform(std::move(merged_transform));
auto builder = std::make_unique<QueryPipelineBuilder>();
builder->init(Pipe::unitePipes(std::move(pipes)));
builder->addTransform(std::move(merged_transform));
#ifndef NDEBUG
if (!sort_description.empty())
{
res_pipe.addSimpleTransform([&](const Block & header_)
builder->addSimpleTransform([&](const Block & header_)
{
auto transform = std::make_shared<CheckSortedTransform>(header_, sort_description);
return transform;
@ -1084,26 +1088,34 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
}
if (DistinctSortedTransform::isApplicable(header, sort_description, global_ctx->deduplicate_by_columns))
res_pipe.addTransform(std::make_shared<DistinctSortedTransform>(
res_pipe.getHeader(), sort_description, SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns));
builder->addTransform(std::make_shared<DistinctSortedTransform>(
builder->getHeader(), sort_description, SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns));
else
res_pipe.addTransform(std::make_shared<DistinctTransform>(
res_pipe.getHeader(), SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns));
builder->addTransform(std::make_shared<DistinctTransform>(
builder->getHeader(), SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns));
}
PreparedSets::Subqueries subqueries;
if (ctx->need_remove_expired_values)
res_pipe.addTransform(std::make_shared<TTLTransform>(
res_pipe.getHeader(), *global_ctx->data, global_ctx->metadata_snapshot, global_ctx->new_data_part, global_ctx->time_of_merge, ctx->force_ttl));
{
auto transform = std::make_shared<TTLTransform>(global_ctx->context, builder->getHeader(), *global_ctx->data, global_ctx->metadata_snapshot, global_ctx->new_data_part, global_ctx->time_of_merge, ctx->force_ttl);
subqueries = transform->getSubqueries();
builder->addTransform(std::move(transform));
}
if (global_ctx->metadata_snapshot->hasSecondaryIndices())
{
const auto & indices = global_ctx->metadata_snapshot->getSecondaryIndices();
res_pipe.addTransform(std::make_shared<ExpressionTransform>(
res_pipe.getHeader(), indices.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext())));
res_pipe.addTransform(std::make_shared<MaterializingTransform>(res_pipe.getHeader()));
builder->addTransform(std::make_shared<ExpressionTransform>(
builder->getHeader(), indices.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext())));
builder->addTransform(std::make_shared<MaterializingTransform>(builder->getHeader()));
}
global_ctx->merged_pipeline = QueryPipeline(std::move(res_pipe));
if (!subqueries.empty())
builder = addCreatingSetsTransform(std::move(builder), std::move(subqueries), global_ctx->context);
global_ctx->merged_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
/// Dereference unique_ptr and pass horizontal_stage_progress by reference
global_ctx->merged_pipeline.setProgressCallback(MergeProgressCallback(global_ctx->merge_list_element_ptr, global_ctx->watch_prev_elapsed, *global_ctx->horizontal_stage_progress));
/// Is calculated inside MergeProgressCallback.

View File

@ -132,13 +132,18 @@ void buildScatterSelector(
/// Computes ttls and updates ttl infos
void updateTTL(
const ContextPtr context,
const TTLDescription & ttl_entry,
IMergeTreeDataPart::TTLInfos & ttl_infos,
DB::MergeTreeDataPartTTLInfo & ttl_info,
const Block & block,
bool update_part_min_max_ttls)
{
auto ttl_column = ITTLAlgorithm::executeExpressionAndGetColumn(ttl_entry.expression, block, ttl_entry.result_column);
auto expr_and_set = ttl_entry.buildExpression(context);
for (auto & subquery : expr_and_set.sets->getSubqueries())
subquery->buildSetInplace(context);
auto ttl_column = ITTLAlgorithm::executeExpressionAndGetColumn(expr_and_set.expression, block, ttl_entry.result_column);
if (const ColumnUInt16 * column_date = typeid_cast<const ColumnUInt16 *>(ttl_column.get()))
{
@ -507,7 +512,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl(
DB::IMergeTreeDataPart::TTLInfos move_ttl_infos;
const auto & move_ttl_entries = metadata_snapshot->getMoveTTLs();
for (const auto & ttl_entry : move_ttl_entries)
updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false);
updateTTL(context, ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false);
ReservationPtr reservation = data.reserveSpacePreferringTTLRules(metadata_snapshot, expected_size, move_ttl_infos, time(nullptr), 0, true);
VolumePtr volume = data.getStoragePolicy()->getVolume(0);
@ -562,20 +567,20 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl(
}
if (metadata_snapshot->hasRowsTTL())
updateTTL(metadata_snapshot->getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true);
updateTTL(context, metadata_snapshot->getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true);
for (const auto & ttl_entry : metadata_snapshot->getGroupByTTLs())
updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.group_by_ttl[ttl_entry.result_column], block, true);
updateTTL(context, ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.group_by_ttl[ttl_entry.result_column], block, true);
for (const auto & ttl_entry : metadata_snapshot->getRowsWhereTTLs())
updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.rows_where_ttl[ttl_entry.result_column], block, true);
updateTTL(context, ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.rows_where_ttl[ttl_entry.result_column], block, true);
for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs())
updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true);
updateTTL(context, ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true);
const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs();
for (const auto & ttl_entry : recompression_ttl_entries)
updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.recompression_ttl[ttl_entry.result_column], block, false);
updateTTL(context, ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.recompression_ttl[ttl_entry.result_column], block, false);
new_data_part->ttl_infos.update(move_ttl_infos);

View File

@ -15,6 +15,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
@ -217,7 +218,20 @@ MergeTreeIndexPtr minmaxIndexCreator(
return std::make_shared<MergeTreeIndexMinMax>(index);
}
void minmaxIndexValidator(const IndexDescription & /* index */, bool /* attach */)
void minmaxIndexValidator(const IndexDescription & index, bool attach)
{
if (attach)
return;
for (const auto & column : index.sample_block)
{
if (!column.type->isComparable())
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Data type of argument for minmax index must be comparable, got {} type for column {} instead",
column.type->getName(), column.name);
}
}
}
}

View File

@ -8,6 +8,7 @@
#include <Parsers/queryToString.h>
#include <Interpreters/SquashingTransform.h>
#include <Interpreters/MergeTreeTransaction.h>
#include <Interpreters/PreparedSets.h>
#include <Processors/Transforms/TTLTransform.h>
#include <Processors/Transforms/TTLCalcTransform.h>
#include <Processors/Transforms/DistinctSortedTransform.h>
@ -16,6 +17,7 @@
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/MaterializingTransform.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Processors/QueryPlan/CreatingSetsStep.h>
#include <Storages/MergeTree/StorageFromMergeTreeDataPart.h>
#include <Storages/MergeTree/MergeTreeDataWriter.h>
#include <Storages/MutationCommands.h>
@ -1552,21 +1554,34 @@ private:
if (!ctx->mutating_pipeline_builder.initialized())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot mutate part columns with uninitialized mutations stream. It's a bug");
QueryPipelineBuilder builder(std::move(ctx->mutating_pipeline_builder));
auto builder = std::make_unique<QueryPipelineBuilder>(std::move(ctx->mutating_pipeline_builder));
if (ctx->metadata_snapshot->hasPrimaryKey() || ctx->metadata_snapshot->hasSecondaryIndices())
{
builder.addTransform(std::make_shared<ExpressionTransform>(
builder.getHeader(), ctx->data->getPrimaryKeyAndSkipIndicesExpression(ctx->metadata_snapshot, skip_indices)));
builder->addTransform(std::make_shared<ExpressionTransform>(
builder->getHeader(), ctx->data->getPrimaryKeyAndSkipIndicesExpression(ctx->metadata_snapshot, skip_indices)));
builder.addTransform(std::make_shared<MaterializingTransform>(builder.getHeader()));
builder->addTransform(std::make_shared<MaterializingTransform>(builder->getHeader()));
}
PreparedSets::Subqueries subqueries;
if (ctx->execute_ttl_type == ExecuteTTLType::NORMAL)
builder.addTransform(std::make_shared<TTLTransform>(builder.getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true));
{
auto transform = std::make_shared<TTLTransform>(ctx->context, builder->getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true);
subqueries = transform->getSubqueries();
builder->addTransform(std::move(transform));
}
if (ctx->execute_ttl_type == ExecuteTTLType::RECALCULATE)
builder.addTransform(std::make_shared<TTLCalcTransform>(builder.getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true));
{
auto transform = std::make_shared<TTLCalcTransform>(ctx->context, builder->getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true);
subqueries = transform->getSubqueries();
builder->addTransform(std::move(transform));
}
if (!subqueries.empty())
builder = addCreatingSetsTransform(std::move(builder), std::move(subqueries), ctx->context);
ctx->minmax_idx = std::make_shared<IMergeTreeDataPart::MinMaxIndex>();
@ -1600,7 +1615,7 @@ private:
ctx->context->getWriteSettings(),
computed_granularity);
ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(builder));
ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
ctx->mutating_pipeline.setProgressCallback(ctx->progress_callback);
/// Is calculated inside MergeProgressCallback.
ctx->mutating_pipeline.disableProfileEventUpdate();
@ -1796,13 +1811,25 @@ private:
if (ctx->mutating_pipeline_builder.initialized())
{
QueryPipelineBuilder builder(std::move(ctx->mutating_pipeline_builder));
auto builder = std::make_unique<QueryPipelineBuilder>(std::move(ctx->mutating_pipeline_builder));
PreparedSets::Subqueries subqueries;
if (ctx->execute_ttl_type == ExecuteTTLType::NORMAL)
builder.addTransform(std::make_shared<TTLTransform>(builder.getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true));
{
auto transform = std::make_shared<TTLTransform>(ctx->context, builder->getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true);
subqueries = transform->getSubqueries();
builder->addTransform(std::move(transform));
}
if (ctx->execute_ttl_type == ExecuteTTLType::RECALCULATE)
builder.addTransform(std::make_shared<TTLCalcTransform>(builder.getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true));
{
auto transform = std::make_shared<TTLCalcTransform>(ctx->context, builder->getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true);
subqueries = transform->getSubqueries();
builder->addTransform(std::move(transform));
}
if (!subqueries.empty())
builder = addCreatingSetsTransform(std::move(builder), std::move(subqueries), ctx->context);
ctx->out = std::make_shared<MergedColumnOnlyOutputStream>(
ctx->new_data_part,
@ -1816,7 +1843,7 @@ private:
&ctx->source_part->index_granularity_info
);
ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(builder));
ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
ctx->mutating_pipeline.setProgressCallback(ctx->progress_callback);
/// Is calculated inside MergeProgressCallback.
ctx->mutating_pipeline.disableProfileEventUpdate();

View File

@ -97,11 +97,9 @@ extern const Event ParallelReplicasCollectingOwnedSegmentsMicroseconds;
extern const Event ParallelReplicasReadAssignedMarks;
extern const Event ParallelReplicasReadUnassignedMarks;
extern const Event ParallelReplicasReadAssignedForStealingMarks;
}
namespace ProfileEvents
{
extern const Event ParallelReplicasUsedCount;
extern const Event ParallelReplicasUsedCount;
extern const Event ParallelReplicasUnavailableCount;
}
namespace DB
@ -1025,6 +1023,8 @@ ParallelReadResponse ParallelReplicasReadingCoordinator::handleRequest(ParallelR
void ParallelReplicasReadingCoordinator::markReplicaAsUnavailable(size_t replica_number)
{
ProfileEvents::increment(ProfileEvents::ParallelReplicasUnavailableCount);
std::lock_guard lock(mutex);
if (!pimpl)

View File

@ -297,7 +297,7 @@ void registerStorageAzureBlob(StorageFactory & factory)
return std::make_shared<StorageAzureBlob>(
std::move(configuration),
std::make_unique<AzureObjectStorage>("AzureBlobStorage", std::move(client), std::move(settings)),
std::make_unique<AzureObjectStorage>("AzureBlobStorage", std::move(client), std::move(settings),configuration.container),
args.getContext(),
args.table_id,
args.columns,

View File

@ -200,7 +200,7 @@ TTLDescription StorageInMemoryMetadata::getRowsTTL() const
bool StorageInMemoryMetadata::hasRowsTTL() const
{
return table_ttl.rows_ttl.expression != nullptr;
return table_ttl.rows_ttl.expression_ast != nullptr;
}
TTLDescriptions StorageInMemoryMetadata::getRowsWhereTTLs() const
@ -258,9 +258,8 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(
NameSet required_ttl_columns;
NameSet updated_ttl_columns;
auto add_dependent_columns = [&updated_columns](const auto & expression, auto & to_set)
auto add_dependent_columns = [&updated_columns](const Names & required_columns, auto & to_set)
{
auto required_columns = expression->getRequiredColumns();
for (const auto & dependency : required_columns)
{
if (updated_columns.contains(dependency))
@ -276,18 +275,18 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(
for (const auto & index : getSecondaryIndices())
{
if (has_dependency(index.name, ColumnDependency::SKIP_INDEX))
add_dependent_columns(index.expression, indices_columns);
add_dependent_columns(index.expression->getRequiredColumns(), indices_columns);
}
for (const auto & projection : getProjections())
{
if (has_dependency(projection.name, ColumnDependency::PROJECTION))
add_dependent_columns(&projection, projections_columns);
add_dependent_columns(projection.getRequiredColumns(), projections_columns);
}
auto add_for_rows_ttl = [&](const auto & expression, auto & to_set)
{
if (add_dependent_columns(expression, to_set) && include_ttl_target)
if (add_dependent_columns(expression.getNames(), to_set) && include_ttl_target)
{
/// Filter all columns, if rows TTL expression have to be recalculated.
for (const auto & column : getColumns().getAllPhysical())
@ -296,25 +295,25 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(
};
if (hasRowsTTL())
add_for_rows_ttl(getRowsTTL().expression, required_ttl_columns);
add_for_rows_ttl(getRowsTTL().expression_columns, required_ttl_columns);
for (const auto & entry : getRowsWhereTTLs())
add_for_rows_ttl(entry.expression, required_ttl_columns);
add_for_rows_ttl(entry.expression_columns, required_ttl_columns);
for (const auto & entry : getGroupByTTLs())
add_for_rows_ttl(entry.expression, required_ttl_columns);
add_for_rows_ttl(entry.expression_columns, required_ttl_columns);
for (const auto & entry : getRecompressionTTLs())
add_dependent_columns(entry.expression, required_ttl_columns);
add_dependent_columns(entry.expression_columns.getNames(), required_ttl_columns);
for (const auto & [name, entry] : getColumnTTLs())
{
if (add_dependent_columns(entry.expression, required_ttl_columns) && include_ttl_target)
if (add_dependent_columns(entry.expression_columns.getNames(), required_ttl_columns) && include_ttl_target)
updated_ttl_columns.insert(name);
}
for (const auto & entry : getMoveTTLs())
add_dependent_columns(entry.expression, required_ttl_columns);
add_dependent_columns(entry.expression_columns.getNames(), required_ttl_columns);
//TODO what about rows_where_ttl and group_by_ttl ??

View File

@ -756,16 +756,23 @@ QueryTreeNodePtr replaceTableExpressionAndRemoveJoin(
auto join_tree_type = query_node->getJoinTree()->getNodeType();
auto modified_query = query_node->cloneAndReplace(original_table_expression, replacement_table_expression);
// For the case when join tree is just a table or a table function we don't need to do anything more.
if (join_tree_type == QueryTreeNodeType::TABLE || join_tree_type == QueryTreeNodeType::TABLE_FUNCTION)
return modified_query;
// JOIN needs to be removed because StorageMerge should produce not joined data.
// GROUP BY should be removed as well.
auto * modified_query_node = modified_query->as<QueryNode>();
// Remove the JOIN statement. As a result query will have a form like: SELECT * FROM <table> ...
modified_query = modified_query->cloneAndReplace(modified_query_node->getJoinTree(), replacement_table_expression);
modified_query_node = modified_query->as<QueryNode>();
query_node = modified_query->as<QueryNode>();
// For backward compatibility we need to leave all filters related to this table.
// It may lead to some incorrect result.
if (query_node->hasPrewhere())
replaceFilterExpression(query_node->getPrewhere(), replacement_table_expression, context);
if (query_node->hasWhere())
@ -779,6 +786,9 @@ QueryTreeNodePtr replaceTableExpressionAndRemoveJoin(
projection.clear();
NamesAndTypes projection_columns;
// Select only required columns from the table, because projection list may contain:
// 1. aggregate functions
// 2. expressions referencing other tables of JOIN
for (auto const & column_name : required_column_names)
{
QueryTreeNodePtr fake_node = std::make_shared<IdentifierNode>(Identifier{column_name});
@ -791,6 +801,8 @@ QueryTreeNodePtr replaceTableExpressionAndRemoveJoin(
throw Exception(ErrorCodes::LOGICAL_ERROR, "Required column '{}' is not resolved", column_name);
auto fake_column = resolved_column->getColumn();
// Identifier is resolved to ColumnNode, but we need to get rid of ALIAS columns
// and also fix references to source expression (now column is referencing original table expression).
ApplyAliasColumnExpressionsVisitor visitor(replacement_table_expression);
visitor.visit(fake_node);
@ -860,7 +872,7 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextPtr & modified_
QueryTreeNodePtr column_node;
// Replace all references to ALIAS columns in the query by expressions.
if (is_alias)
{
QueryTreeNodePtr fake_node = std::make_shared<IdentifierNode>(Identifier{column});

View File

@ -18,6 +18,7 @@
#include <Interpreters/FunctionNameNormalizer.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
namespace DB
@ -110,7 +111,10 @@ using FindAggregateFunctionVisitor = InDepthNodeVisitor<FindAggregateFunctionFin
TTLDescription::TTLDescription(const TTLDescription & other)
: mode(other.mode)
, expression_ast(other.expression_ast ? other.expression_ast->clone() : nullptr)
, expression_columns(other.expression_columns)
, result_column(other.result_column)
, where_expression_ast(other.where_expression_ast ? other.where_expression_ast->clone() : nullptr)
, where_expression_columns(other.where_expression_columns)
, where_result_column(other.where_result_column)
, group_by_keys(other.group_by_keys)
, set_parts(other.set_parts)
@ -120,11 +124,6 @@ TTLDescription::TTLDescription(const TTLDescription & other)
, if_exists(other.if_exists)
, recompression_codec(other.recompression_codec)
{
if (other.expression)
expression = other.expression->clone();
if (other.where_expression)
where_expression = other.where_expression->clone();
}
TTLDescription & TTLDescription::operator=(const TTLDescription & other)
@ -138,17 +137,15 @@ TTLDescription & TTLDescription::operator=(const TTLDescription & other)
else
expression_ast.reset();
if (other.expression)
expression = other.expression->clone();
else
expression.reset();
expression_columns = other.expression_columns;
result_column = other.result_column;
if (other.where_expression)
where_expression = other.where_expression->clone();
else
where_expression.reset();
if (other.where_expression_ast)
where_expression_ast = other.where_expression_ast->clone();
else
where_expression_ast.reset();
where_expression_columns = other.where_expression_columns;
where_result_column = other.where_result_column;
group_by_keys = other.group_by_keys;
set_parts = other.set_parts;
@ -165,6 +162,44 @@ TTLDescription & TTLDescription::operator=(const TTLDescription & other)
return * this;
}
static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndTypesList & columns, const ContextPtr & context)
{
ExpressionAndSets result;
auto ttl_string = queryToString(ast);
auto syntax_analyzer_result = TreeRewriter(context).analyze(ast, columns);
ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context);
auto dag = analyzer.getActionsDAG(false);
const auto * col = &dag->findInOutputs(ast->getColumnName());
if (col->result_name != ttl_string)
col = &dag->addAlias(*col, ttl_string);
dag->getOutputs() = {col};
dag->removeUnusedActions();
result.expression = std::make_shared<ExpressionActions>(dag, ExpressionActionsSettings::fromContext(context));
result.sets = analyzer.getPreparedSets();
return result;
}
ExpressionAndSets TTLDescription::buildExpression(const ContextPtr & context) const
{
auto ast = expression_ast->clone();
return buildExpressionAndSets(ast, expression_columns, context);
}
ExpressionAndSets TTLDescription::buildWhereExpression(const ContextPtr & context) const
{
if (where_expression_ast)
{
auto ast = where_expression_ast->clone();
return buildExpressionAndSets(ast, where_expression_columns, context);
}
return {};
}
TTLDescription TTLDescription::getTTLFromAST(
const ASTPtr & definition_ast,
const ColumnsDescription & columns,
@ -182,9 +217,12 @@ TTLDescription TTLDescription::getTTLFromAST(
result.expression_ast = definition_ast->clone();
auto ttl_ast = result.expression_ast->clone();
auto syntax_analyzer_result = TreeRewriter(context).analyze(ttl_ast, columns.getAllPhysical());
result.expression = ExpressionAnalyzer(ttl_ast, syntax_analyzer_result, context).getActions(false);
result.result_column = ttl_ast->getColumnName();
auto expression = buildExpressionAndSets(ttl_ast, columns.getAllPhysical(), context).expression;
result.expression_columns = expression->getRequiredColumnsWithTypes();
result.result_column = expression->getSampleBlock().safeGetByPosition(0).name;
ExpressionActionsPtr where_expression;
if (ttl_element == nullptr) /// columns TTL
{
@ -202,9 +240,10 @@ TTLDescription TTLDescription::getTTLFromAST(
{
if (ASTPtr where_expr_ast = ttl_element->where())
{
auto where_syntax_result = TreeRewriter(context).analyze(where_expr_ast, columns.getAllPhysical());
result.where_expression = ExpressionAnalyzer(where_expr_ast, where_syntax_result, context).getActions(false);
result.where_result_column = where_expr_ast->getColumnName();
result.where_expression_ast = where_expr_ast->clone();
where_expression = buildExpressionAndSets(where_expr_ast, columns.getAllPhysical(), context).expression;
result.where_expression_columns = where_expression->getRequiredColumnsWithTypes();
result.where_result_column = where_expression->getSampleBlock().safeGetByPosition(0).name;
}
}
else if (ttl_element->mode == TTLMode::GROUP_BY)
@ -229,17 +268,17 @@ TTLDescription TTLDescription::getTTLFromAST(
for (const auto & ast : ttl_element->group_by_assignments)
{
const auto assignment = ast->as<const ASTAssignment &>();
auto expression = assignment.expression();
auto ass_expression = assignment.expression();
FindAggregateFunctionVisitor::Data data{false};
FindAggregateFunctionVisitor(data).visit(expression);
FindAggregateFunctionVisitor(data).visit(ass_expression);
if (!data.has_aggregate_function)
throw Exception(ErrorCodes::BAD_TTL_EXPRESSION,
"Invalid expression for assignment of column {}. Should contain an aggregate function", assignment.column_name);
expression = addTypeConversionToAST(std::move(expression), columns.getPhysical(assignment.column_name).type->getName());
aggregations.emplace_back(assignment.column_name, std::move(expression));
ass_expression = addTypeConversionToAST(std::move(ass_expression), columns.getPhysical(assignment.column_name).type->getName());
aggregations.emplace_back(assignment.column_name, std::move(ass_expression));
aggregation_columns_set.insert(assignment.column_name);
}
@ -297,7 +336,7 @@ TTLDescription TTLDescription::getTTLFromAST(
}
}
checkTTLExpression(result.expression, result.result_column, is_attach || context->getSettingsRef().allow_suspicious_ttl_expressions);
checkTTLExpression(expression, result.result_column, is_attach || context->getSettingsRef().allow_suspicious_ttl_expressions);
return result;
}
@ -350,7 +389,7 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST(
auto ttl = TTLDescription::getTTLFromAST(ttl_element_ptr, columns, context, primary_key, is_attach);
if (ttl.mode == TTLMode::DELETE)
{
if (!ttl.where_expression)
if (!ttl.where_expression_ast)
{
if (have_unconditional_delete_ttl)
throw Exception(ErrorCodes::BAD_TTL_EXPRESSION, "More than one DELETE TTL expression without WHERE expression is not allowed");

View File

@ -35,6 +35,15 @@ struct TTLAggregateDescription
using TTLAggregateDescriptions = std::vector<TTLAggregateDescription>;
class PreparedSets;
using PreparedSetsPtr = std::shared_ptr<PreparedSets>;
struct ExpressionAndSets
{
ExpressionActionsPtr expression;
PreparedSetsPtr sets;
};
/// Common struct for TTL record in storage
struct TTLDescription
{
@ -44,9 +53,10 @@ struct TTLDescription
/// TTL d + INTERVAL 1 DAY
/// ^~~~~~~~~~~~~~~~~~~^
ASTPtr expression_ast;
NamesAndTypesList expression_columns;
/// Expression actions evaluated from AST
ExpressionActionsPtr expression;
ExpressionAndSets buildExpression(const ContextPtr & context) const;
/// Result column of this TTL expression
String result_column;
@ -54,7 +64,9 @@ struct TTLDescription
/// WHERE part in TTL expression
/// TTL ... WHERE x % 10 == 0 and y > 5
/// ^~~~~~~~~~~~~~~~~~~~~~^
ExpressionActionsPtr where_expression;
ASTPtr where_expression_ast;
NamesAndTypesList where_expression_columns;
ExpressionAndSets buildWhereExpression(const ContextPtr & context) const;
/// Name of result column from WHERE expression
String where_result_column;

View File

@ -262,7 +262,7 @@ ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(Contex
auto client = StorageAzureBlob::createClient(configuration, !is_insert_query);
auto settings = StorageAzureBlob::createSettings(context);
auto object_storage = std::make_unique<AzureObjectStorage>("AzureBlobStorageTableFunction", std::move(client), std::move(settings));
auto object_storage = std::make_unique<AzureObjectStorage>("AzureBlobStorageTableFunction", std::move(client), std::move(settings), configuration.container);
return StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context, false);
}
@ -293,7 +293,7 @@ StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_funct
StoragePtr storage = std::make_shared<StorageAzureBlob>(
configuration,
std::make_unique<AzureObjectStorage>(table_name, std::move(client), std::move(settings)),
std::make_unique<AzureObjectStorage>(table_name, std::move(client), std::move(settings), configuration.container),
context,
StorageID(getDatabaseName(), table_name),
columns,

View File

@ -40,7 +40,7 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl(
/// On worker node this filename won't contains globs
storage = std::make_shared<StorageAzureBlob>(
configuration,
std::make_unique<AzureObjectStorage>(table_name, std::move(client), std::move(settings)),
std::make_unique<AzureObjectStorage>(table_name, std::move(client), std::move(settings), configuration.container),
context,
StorageID(getDatabaseName(), table_name),
columns,
@ -55,7 +55,7 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl(
storage = std::make_shared<StorageAzureBlobCluster>(
cluster_name,
configuration,
std::make_unique<AzureObjectStorage>(table_name, std::move(client), std::move(settings)),
std::make_unique<AzureObjectStorage>(table_name, std::move(client), std::move(settings), configuration.container),
StorageID(getDatabaseName(), table_name),
columns,
ConstraintsDescription{},

View File

@ -646,14 +646,14 @@ class CiCache:
TIMEOUT = 3600
await_finished: Dict[str, List[int]] = {}
round_cnt = 0
while len(jobs_with_params) > 5 and round_cnt < 3:
while len(jobs_with_params) > 4 and round_cnt < 5:
round_cnt += 1
GHActions.print_in_group(
f"Wait pending jobs, round [{round_cnt}]:", list(jobs_with_params)
)
# this is initial approach to wait pending jobs:
# start waiting for the next TIMEOUT seconds if there are more than X(=5) jobs to wait
# wait TIMEOUT seconds in rounds. Y(=3) is the max number of rounds
# start waiting for the next TIMEOUT seconds if there are more than X(=4) jobs to wait
# wait TIMEOUT seconds in rounds. Y(=5) is the max number of rounds
expired_sec = 0
start_at = int(time.time())
while expired_sec < TIMEOUT and jobs_with_params:
@ -701,13 +701,11 @@ class CiCache:
print(
f"...awaiting continues... seconds left [{TIMEOUT - expired_sec}]"
)
if await_finished:
GHActions.print_in_group(
"Finished jobs:",
[f"{job}:{batches}" for job, batches in await_finished.items()],
)
else:
print("Awaiting FAILED. No job has finished successfully.")
if await_finished:
GHActions.print_in_group(
f"Finished jobs, round [{round_cnt}]:",
[f"{job}:{batches}" for job, batches in await_finished.items()],
)
GHActions.print_in_group(
"Remaining jobs:",
[f"{job}:{params['batches']}" for job, params in jobs_with_params.items()],
@ -816,6 +814,12 @@ def parse_args(parser: argparse.ArgumentParser) -> argparse.Namespace:
default=False,
help="skip fetching data about job runs, used in --configure action (for debugging and nigthly ci)",
)
parser.add_argument(
"--force",
action="store_true",
default=False,
help="Used with --run, force the job to run, omitting the ci cache",
)
# FIXME: remove, not used
parser.add_argument(
"--rebuild-all-binaries",
@ -1762,7 +1766,7 @@ def main() -> int:
previous_status = job_status.status
GHActions.print_in_group("Commit Status Data", job_status)
if previous_status:
if previous_status and not args.force:
print(
f"Commit status or Build Report is already present - job will be skipped with status: [{previous_status}]"
)

View File

@ -143,8 +143,6 @@ class JobNames(metaclass=WithIter):
DOCS_CHECK = "Docs check"
BUGFIX_VALIDATE = "tests bugfix validate check"
MARK_RELEASE_READY = "Mark Commit Release Ready"
# dynamically update JobName with Build jobs
for attr_name in dir(Build):
@ -840,9 +838,6 @@ CI_CONFIG = CIConfig(
),
},
other_jobs_configs={
JobNames.MARK_RELEASE_READY: TestConfig(
"", job_config=JobConfig(release_only=True)
),
JobNames.DOCKER_SERVER: TestConfig(
"",
job_config=JobConfig(

View File

@ -22,18 +22,29 @@ mkdir "$PACKAGE"
cp app.py "$PACKAGE"
if [ -f requirements.txt ]; then
VENV=lambda-venv
rm -rf "$VENV" lambda-package.zip
rm -rf "$VENV"
docker run --net=host --rm --user="${UID}" -e HOME=/tmp --entrypoint=/bin/bash \
--volume="${WORKDIR}/..:/ci" --workdir="/ci/${DIR_NAME}" "${DOCKER_IMAGE}" \
-exc "
'$PY_EXEC' -m venv '$VENV' &&
source '$VENV/bin/activate' &&
pip install -r requirements.txt
pip install -r requirements.txt &&
# To have consistent pyc files
find '$VENV/lib' -name '*.pyc' -delete
find '$VENV/lib' ! -type d -exec touch -t 201212121212 {} +
python -m compileall
"
cp -rT "$VENV/lib/$PY_EXEC/site-packages/" "$PACKAGE"
rm -r "$PACKAGE"/{pip,pip-*,setuptools,setuptools-*}
# zip stores metadata about timestamps
find "$PACKAGE" ! -type d -exec touch -t 201212121212 {} +
fi
( cd "$PACKAGE" && zip -9 -r ../"$PACKAGE".zip . )
(
export LC_ALL=c
cd "$PACKAGE"
# zip uses random files order by default, so we sort the files alphabetically
find . ! -type d -print0 | sort -z | tr '\0' '\n' | zip -XD -0 ../"$PACKAGE".zip --names-stdin
)
ECHO=()
if [ -n "$DRY_RUN" ]; then

View File

@ -0,0 +1 @@
#!/usr/bin/env python3

View File

@ -0,0 +1,270 @@
#!/usr/bin/env python3
import gzip
import json
import logging
import os
import io
import random
import threading
import time
from azure.storage.blob import BlobServiceClient
import helpers.client
import pytest
from helpers.cluster import ClickHouseCluster, ClickHouseInstance
from helpers.network import PartitionManager
from helpers.mock_servers import start_mock_servers
from helpers.test_tools import exec_query_with_retry
def generate_cluster_def(port):
path = os.path.join(
os.path.dirname(os.path.realpath(__file__)),
"./_gen/named_collections.xml",
)
os.makedirs(os.path.dirname(path), exist_ok=True)
with open(path, "w") as f:
f.write(
f"""<clickhouse>
<named_collections>
<azure_conf1>
<connection_string>DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:{port}/devstoreaccount1;</connection_string>
<container>cont</container>
<format>CSV</format>
</azure_conf1>
<azure_conf2>
<storage_account_url>http://azurite1:{port}/devstoreaccount1</storage_account_url>
<container>cont</container>
<format>CSV</format>
<account_name>devstoreaccount1</account_name>
<account_key>Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==</account_key>
</azure_conf2>
</named_collections>
</clickhouse>
"""
)
return path
@pytest.fixture(scope="module")
def cluster():
try:
cluster = ClickHouseCluster(__file__)
port = cluster.azurite_port
path = generate_cluster_def(port)
cluster.add_instance(
"node",
main_configs=[path],
with_azurite=True,
)
cluster.start()
yield cluster
finally:
cluster.shutdown()
def azure_query(
node, query, expect_error="false", try_num=10, settings={}, query_on_retry=None
):
for i in range(try_num):
try:
if expect_error == "true":
return node.query_and_get_error(query, settings=settings)
else:
return node.query(query, settings=settings)
except Exception as ex:
retriable_errors = [
"DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response",
"DB::Exception: Azure::Core::Http::TransportException: Connection closed before getting full response or response is less than expected",
"DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response",
"DB::Exception: Azure::Core::Http::TransportException: Error while polling for socket ready read",
"Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response",
"Azure::Core::Http::TransportException, e.what() = Connection closed before getting full response or response is less than expected",
"Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response",
"Azure::Core::Http::TransportException, e.what() = Error while polling for socket ready read",
]
retry = False
for error in retriable_errors:
if error in str(ex):
retry = True
print(f"Try num: {i}. Having retriable error: {ex}")
time.sleep(i)
break
if not retry or i == try_num - 1:
raise Exception(ex)
if query_on_retry is not None:
node.query(query_on_retry)
continue
def get_azure_file_content(filename, port):
container_name = "cont"
connection_string = (
f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;"
f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;"
f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;"
)
blob_service_client = BlobServiceClient.from_connection_string(
str(connection_string)
)
container_client = blob_service_client.get_container_client(container_name)
blob_client = container_client.get_blob_client(filename)
download_stream = blob_client.download_blob()
return download_stream.readall().decode("utf-8")
def put_azure_file_content(filename, port, data):
container_name = "cont"
connection_string = (
f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;"
f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;"
f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;"
)
blob_service_client = BlobServiceClient.from_connection_string(connection_string)
try:
container_client = blob_service_client.create_container(container_name)
except:
container_client = blob_service_client.get_container_client(container_name)
blob_client = container_client.get_blob_client(filename)
buf = io.BytesIO(data)
blob_client.upload_blob(buf)
@pytest.fixture(autouse=True, scope="function")
def delete_all_files(cluster):
port = cluster.env_variables["AZURITE_PORT"]
connection_string = (
f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;"
f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;"
f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;"
)
blob_service_client = BlobServiceClient.from_connection_string(connection_string)
containers = blob_service_client.list_containers()
for container in containers:
container_client = blob_service_client.get_container_client(container)
blob_list = container_client.list_blobs()
for blob in blob_list:
print(blob)
blob_client = container_client.get_blob_client(blob)
blob_client.delete_blob()
assert len(list(container_client.list_blobs())) == 0
yield
def test_backup_restore(cluster):
node = cluster.instances["node"]
port = cluster.env_variables["AZURITE_PORT"]
azure_query(
node,
f"CREATE TABLE test_simple_write_connection_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_write_c.csv', 'CSV')",
)
azure_query(
node, f"INSERT INTO test_simple_write_connection_string VALUES (1, 'a')"
)
print(get_azure_file_content("test_simple_write_c.csv", port))
assert get_azure_file_content("test_simple_write_c.csv", port) == '1,"a"\n'
backup_destination = f"AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_write_c_backup.csv')"
azure_query(
node,
f"BACKUP TABLE test_simple_write_connection_string TO {backup_destination}",
)
print(get_azure_file_content("test_simple_write_c_backup.csv.backup", port))
azure_query(
node,
f"RESTORE TABLE test_simple_write_connection_string AS test_simple_write_connection_string_restored FROM {backup_destination};",
)
assert (
azure_query(node, f"SELECT * from test_simple_write_connection_string_restored")
== "1\ta\n"
)
def test_backup_restore_diff_container(cluster):
node = cluster.instances["node"]
port = cluster.env_variables["AZURITE_PORT"]
azure_query(
node,
f"CREATE TABLE test_simple_write_connection_string_cont1 (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_write_c_cont1.csv', 'CSV')",
)
azure_query(
node, f"INSERT INTO test_simple_write_connection_string_cont1 VALUES (1, 'a')"
)
backup_destination = f"AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont1', 'test_simple_write_c_backup_cont1.csv')"
azure_query(
node,
f"BACKUP TABLE test_simple_write_connection_string_cont1 TO {backup_destination}",
)
azure_query(
node,
f"RESTORE TABLE test_simple_write_connection_string_cont1 AS test_simple_write_connection_string_restored_cont1 FROM {backup_destination};",
)
assert (
azure_query(
node, f"SELECT * from test_simple_write_connection_string_restored_cont1"
)
== "1\ta\n"
)
def test_backup_restore_with_named_collection_azure_conf1(cluster):
node = cluster.instances["node"]
port = cluster.env_variables["AZURITE_PORT"]
azure_query(
node,
f"CREATE TABLE test_write_connection_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_write.csv', 'CSV')",
)
azure_query(node, f"INSERT INTO test_write_connection_string VALUES (1, 'a')")
print(get_azure_file_content("test_simple_write.csv", port))
assert get_azure_file_content("test_simple_write.csv", port) == '1,"a"\n'
backup_destination = (
f"AzureBlobStorage(azure_conf1, 'test_simple_write_nc_backup.csv')"
)
azure_query(
node,
f"BACKUP TABLE test_write_connection_string TO {backup_destination}",
)
print(get_azure_file_content("test_simple_write_nc_backup.csv.backup", port))
azure_query(
node,
f"RESTORE TABLE test_write_connection_string AS test_write_connection_string_restored FROM {backup_destination};",
)
assert (
azure_query(node, f"SELECT * from test_write_connection_string_restored")
== "1\ta\n"
)
def test_backup_restore_with_named_collection_azure_conf2(cluster):
node = cluster.instances["node"]
port = cluster.env_variables["AZURITE_PORT"]
azure_query(
node,
f"CREATE TABLE test_write_connection_string_2 (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_write_2.csv', 'CSV')",
)
azure_query(node, f"INSERT INTO test_write_connection_string_2 VALUES (1, 'a')")
print(get_azure_file_content("test_simple_write_2.csv", port))
assert get_azure_file_content("test_simple_write_2.csv", port) == '1,"a"\n'
backup_destination = (
f"AzureBlobStorage(azure_conf2, 'test_simple_write_nc_backup_2.csv')"
)
azure_query(
node,
f"BACKUP TABLE test_write_connection_string_2 TO {backup_destination}",
)
print(get_azure_file_content("test_simple_write_nc_backup_2.csv.backup", port))
azure_query(
node,
f"RESTORE TABLE test_write_connection_string_2 AS test_write_connection_string_restored_2 FROM {backup_destination};",
)
assert (
azure_query(node, f"SELECT * from test_write_connection_string_restored_2")
== "1\ta\n"
)

View File

@ -155,7 +155,7 @@ def test_recompression_multiple_ttls(started_cluster):
node2.query(
"SELECT recompression_ttl_info.expression FROM system.parts where name = 'all_1_1_4'"
)
== "['plus(d, toIntervalSecond(10))','plus(d, toIntervalSecond(15))','plus(d, toIntervalSecond(5))']\n"
== "['d + toIntervalSecond(10)','d + toIntervalSecond(15)','d + toIntervalSecond(5)']\n"
)

View File

@ -13,9 +13,9 @@ CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt6
1_1_1 LZ4
2_2_2 ZSTD(12)
3_3_3 ZSTD(12)
1_1_1 ['plus(dt, toIntervalDay(1))']
2_2_2 ['plus(dt, toIntervalDay(1))']
3_3_3 ['plus(dt, toIntervalDay(1))']
1_1_1 ['dt + toIntervalDay(1)']
2_2_2 ['dt + toIntervalDay(1)']
3_3_3 ['dt + toIntervalDay(1)']
1_1_1 LZ4
2_2_2 LZ4
3_3_3 LZ4

View File

@ -80,3 +80,7 @@ nan
5 6 268 2 10.234459893824097 23.15167380558045 536 0.00007815428961455151
6 5 268 2 10.234459893824097 23.15167380558045 536 0.00007815428961455151
6 6 0 0 0 0 0 0
5.8309517
0.0003244877
5.830951894845301
0.0003245172890904424

View File

@ -12,10 +12,10 @@ SELECT cosineDistance([1, 2, 3], [0, 0, 0]);
-- Overflows
WITH CAST([-547274980, 1790553898, 1981517754, 1908431500, 1352428565, -573412550, -552499284, 2096941042], 'Array(Int32)') AS a
SELECT
L1Distance(a,a),
L2Distance(a,a),
L2SquaredDistance(a,a),
LinfDistance(a,a),
L1Distance(a, a),
L2Distance(a, a),
L2SquaredDistance(a, a),
LinfDistance(a, a),
cosineDistance(a, a);
DROP TABLE IF EXISTS vec1;
@ -88,15 +88,33 @@ SELECT
FROM vec2f v1, vec2d v2
WHERE length(v1.v) == length(v2.v);
SELECT L1Distance([0, 0], [1]); -- { serverError 190 }
SELECT L2Distance([1, 2], (3,4)); -- { serverError 43 }
SELECT L2SquaredDistance([1, 2], (3,4)); -- { serverError 43 }
SELECT LpDistance([1, 2], [3,4]); -- { serverError 42 }
SELECT LpDistance([1, 2], [3,4], -1.); -- { serverError 69 }
SELECT LpDistance([1, 2], [3,4], 'aaa'); -- { serverError 43 }
SELECT LpDistance([1, 2], [3,4], materialize(2.7)); -- { serverError 44 }
SELECT L1Distance([0, 0], [1]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH }
SELECT L2Distance([1, 2], (3,4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT L2SquaredDistance([1, 2], (3,4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT LpDistance([1, 2], [3,4]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT LpDistance([1, 2], [3,4], -1.); -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT LpDistance([1, 2], [3,4], 'aaa'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT LpDistance([1, 2], [3,4], materialize(2.7)); -- { serverError ILLEGAL_COLUMN }
DROP TABLE vec1;
DROP TABLE vec2;
DROP TABLE vec2f;
DROP TABLE vec2d;
-- Queries which trigger manually vectorized implementation
SELECT L2Distance(
[toFloat32(0.0), toFloat32(1.0), toFloat32(2.0), toFloat32(3.0), toFloat32(4.0), toFloat32(5.0), toFloat32(6.0), toFloat32(7.0), toFloat32(8.0), toFloat32(9.0), toFloat32(10.0), toFloat32(11.0), toFloat32(12.0), toFloat32(13.0), toFloat32(14.0), toFloat32(15.0), toFloat32(16.0), toFloat32(17.0), toFloat32(18.0), toFloat32(19.0), toFloat32(20.0), toFloat32(21.0), toFloat32(22.0), toFloat32(23.0), toFloat32(24.0), toFloat32(25.0), toFloat32(26.0), toFloat32(27.0), toFloat32(28.0), toFloat32(29.0), toFloat32(30.0), toFloat32(31.0), toFloat32(32.0), toFloat32(33.0)],
materialize([toFloat32(1.0), toFloat32(2.0), toFloat32(3.0), toFloat32(4.0), toFloat32(5.0), toFloat32(6.0), toFloat32(7.0), toFloat32(8.0), toFloat32(9.0), toFloat32(10.0), toFloat32(11.0), toFloat32(12.0), toFloat32(13.0), toFloat32(14.0), toFloat32(15.0), toFloat32(16.0), toFloat32(17.0), toFloat32(18.0), toFloat32(19.0), toFloat32(20.0), toFloat32(21.0), toFloat32(22.0), toFloat32(23.0), toFloat32(24.0), toFloat32(25.0), toFloat32(26.0), toFloat32(27.0), toFloat32(28.0), toFloat32(29.0), toFloat32(30.0), toFloat32(31.0), toFloat32(32.0), toFloat32(33.0), toFloat32(34.0)]));
SELECT cosineDistance(
[toFloat32(0.0), toFloat32(1.0), toFloat32(2.0), toFloat32(3.0), toFloat32(4.0), toFloat32(5.0), toFloat32(6.0), toFloat32(7.0), toFloat32(8.0), toFloat32(9.0), toFloat32(10.0), toFloat32(11.0), toFloat32(12.0), toFloat32(13.0), toFloat32(14.0), toFloat32(15.0), toFloat32(16.0), toFloat32(17.0), toFloat32(18.0), toFloat32(19.0), toFloat32(20.0), toFloat32(21.0), toFloat32(22.0), toFloat32(23.0), toFloat32(24.0), toFloat32(25.0), toFloat32(26.0), toFloat32(27.0), toFloat32(28.0), toFloat32(29.0), toFloat32(30.0), toFloat32(31.0), toFloat32(32.0), toFloat32(33.0)],
materialize([toFloat32(1.0), toFloat32(2.0), toFloat32(3.0), toFloat32(4.0), toFloat32(5.0), toFloat32(6.0), toFloat32(7.0), toFloat32(8.0), toFloat32(9.0), toFloat32(10.0), toFloat32(11.0), toFloat32(12.0), toFloat32(13.0), toFloat32(14.0), toFloat32(15.0), toFloat32(16.0), toFloat32(17.0), toFloat32(18.0), toFloat32(19.0), toFloat32(20.0), toFloat32(21.0), toFloat32(22.0), toFloat32(23.0), toFloat32(24.0), toFloat32(25.0), toFloat32(26.0), toFloat32(27.0), toFloat32(28.0), toFloat32(29.0), toFloat32(30.0), toFloat32(31.0), toFloat32(32.0), toFloat32(33.0), toFloat32(34.0)]));
SELECT L2Distance(
[toFloat64(0.0), toFloat64(1.0), toFloat64(2.0), toFloat64(3.0), toFloat64(4.0), toFloat64(5.0), toFloat64(6.0), toFloat64(7.0), toFloat64(8.0), toFloat64(9.0), toFloat64(10.0), toFloat64(11.0), toFloat64(12.0), toFloat64(13.0), toFloat64(14.0), toFloat64(15.0), toFloat64(16.0), toFloat64(17.0), toFloat64(18.0), toFloat64(19.0), toFloat64(20.0), toFloat64(21.0), toFloat64(22.0), toFloat64(23.0), toFloat64(24.0), toFloat64(25.0), toFloat64(26.0), toFloat64(27.0), toFloat64(28.0), toFloat64(29.0), toFloat64(30.0), toFloat64(31.0), toFloat64(32.0), toFloat64(33.0)],
materialize([toFloat64(1.0), toFloat64(2.0), toFloat64(3.0), toFloat64(4.0), toFloat64(5.0), toFloat64(6.0), toFloat64(7.0), toFloat64(8.0), toFloat64(9.0), toFloat64(10.0), toFloat64(11.0), toFloat64(12.0), toFloat64(13.0), toFloat64(14.0), toFloat64(15.0), toFloat64(16.0), toFloat64(17.0), toFloat64(18.0), toFloat64(19.0), toFloat64(20.0), toFloat64(21.0), toFloat64(22.0), toFloat64(23.0), toFloat64(24.0), toFloat64(25.0), toFloat64(26.0), toFloat64(27.0), toFloat64(28.0), toFloat64(29.0), toFloat64(30.0), toFloat64(31.0), toFloat64(32.0), toFloat64(33.0), toFloat64(34.0)]));
SELECT cosineDistance(
[toFloat64(0.0), toFloat64(1.0), toFloat64(2.0), toFloat64(3.0), toFloat64(4.0), toFloat64(5.0), toFloat64(6.0), toFloat64(7.0), toFloat64(8.0), toFloat64(9.0), toFloat64(10.0), toFloat64(11.0), toFloat64(12.0), toFloat64(13.0), toFloat64(14.0), toFloat64(15.0), toFloat64(16.0), toFloat64(17.0), toFloat64(18.0), toFloat64(19.0), toFloat64(20.0), toFloat64(21.0), toFloat64(22.0), toFloat64(23.0), toFloat64(24.0), toFloat64(25.0), toFloat64(26.0), toFloat64(27.0), toFloat64(28.0), toFloat64(29.0), toFloat64(30.0), toFloat64(31.0), toFloat64(32.0), toFloat64(33.0)],
materialize([toFloat64(1.0), toFloat64(2.0), toFloat64(3.0), toFloat64(4.0), toFloat64(5.0), toFloat64(6.0), toFloat64(7.0), toFloat64(8.0), toFloat64(9.0), toFloat64(10.0), toFloat64(11.0), toFloat64(12.0), toFloat64(13.0), toFloat64(14.0), toFloat64(15.0), toFloat64(16.0), toFloat64(17.0), toFloat64(18.0), toFloat64(19.0), toFloat64(20.0), toFloat64(21.0), toFloat64(22.0), toFloat64(23.0), toFloat64(24.0), toFloat64(25.0), toFloat64(26.0), toFloat64(27.0), toFloat64(28.0), toFloat64(29.0), toFloat64(30.0), toFloat64(31.0), toFloat64(32.0), toFloat64(33.0), toFloat64(34.0)]));

View File

@ -2,14 +2,13 @@ DROP TABLE IF EXISTS test_parallel_replicas_unavailable_shards;
CREATE TABLE test_parallel_replicas_unavailable_shards (n UInt64) ENGINE=MergeTree() ORDER BY tuple();
INSERT INTO test_parallel_replicas_unavailable_shards SELECT * FROM numbers(10);
SYSTEM FLUSH LOGS;
SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=11, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1;
SET send_logs_level='error';
SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*);
SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79';
SYSTEM FLUSH LOGS;
SELECT count() > 0 FROM system.text_log WHERE yesterday() <= event_date AND message LIKE '%Replica number 10 is unavailable%';
SET allow_experimental_parallel_reading_from_replicas=0;
SELECT ProfileEvents['ParallelReplicasUnavailableCount'] FROM system.query_log WHERE yesterday() <= event_date AND query_id in (select query_id from system.query_log where log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79' and current_database = currentDatabase()) and type = 'QueryFinish' and query_id == initial_query_id;
DROP TABLE test_parallel_replicas_unavailable_shards;

View File

@ -0,0 +1,3 @@
0
0
0

View File

@ -0,0 +1,18 @@
-- Tags: no-ordinary-database
create or replace table t_temp (
a UInt32,
timestamp DateTime
)
engine = MergeTree
order by a
TTL timestamp + INTERVAL 2 SECOND WHERE a in (select number from system.numbers limit 100_000);
select sleep(1);
insert into t_temp select rand(), now() from system.numbers limit 1_000_000;
select sleep(1);
insert into t_temp select rand(), now() from system.numbers limit 1_000_000;
select sleep(1);
optimize table t_temp final;
DROP TABLE t_temp;

View File

@ -12,16 +12,16 @@ SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.
WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 10000)
SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a
SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_two_shards', max_parallel_replicas = 3;
SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3;
-- Testing that it is disabled for allow_experimental_analyzer=0. With analyzer it will be supported (with correct result)
WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 10000)
SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a
SETTINGS allow_experimental_analyzer = 0, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_two_shards', max_parallel_replicas = 3; -- { serverError SUPPORT_IS_DISABLED }
SETTINGS allow_experimental_analyzer = 0, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- { serverError SUPPORT_IS_DISABLED }
-- Sanitizer
SELECT count() FROM pr_2 JOIN numbers(10) as pr_1 ON pr_2.a = pr_1.number
SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_two_shards', max_parallel_replicas = 3;
SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3;
DROP TABLE IF EXISTS pr_1;
DROP TABLE IF EXISTS pr_2;

View File

@ -0,0 +1,8 @@
DROP TABLE IF EXISTS test_unexpected_cluster;
CREATE TABLE test_unexpected_cluster (n UInt64) ENGINE=MergeTree() ORDER BY tuple();
INSERT INTO test_unexpected_cluster SELECT * FROM numbers(10);
SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=2, cluster_for_parallel_replicas='test_cluster_two_shards', parallel_replicas_for_non_replicated_merge_tree=1;
SELECT count() FROM test_unexpected_cluster WHERE NOT ignore(*); -- { serverError UNEXPECTED_CLUSTER }
DROP TABLE test_unexpected_cluster;

View File

@ -0,0 +1 @@
Values Ok 4 Parsed

View File

@ -0,0 +1,46 @@
-- Tags: no-parallel
SET async_insert = 1;
SET insert_deduplicate = 1;
SET deduplicate_blocks_in_dependent_materialized_views = 1;
DROP TABLE IF EXISTS 02985_test;
CREATE TABLE 02985_test
(
d Date,
value UInt64
) ENGINE = MergeTree ORDER BY tuple() SETTINGS non_replicated_deduplication_window = 1000;
DROP VIEW IF EXISTS 02985_mv;
CREATE MATERIALIZED VIEW 02985_mv
ENGINE = SummingMergeTree ORDER BY d AS
SELECT
d, sum(value) s
FROM 02985_test GROUP BY d;
-- Inserts are synchronous.
INSERT INTO 02985_test (*)
VALUES ('2024-01-01', 1), ('2024-01-01', 2), ('2024-01-02', 1);
SYSTEM FLUSH LOGS;
SELECT format, status, rows, data_kind FROM system.asynchronous_insert_log
WHERE database = currentDatabase() AND table = '02985_test';
SET deduplicate_blocks_in_dependent_materialized_views = 0;
-- Set a large value for async_insert_busy_timeout_max_ms to avoid flushing the entry synchronously.
INSERT INTO 02985_test (*)
SETTINGS
async_insert_busy_timeout_min_ms=200,
async_insert_busy_timeout_max_ms=100000
VALUES ('2024-01-01', 1), ('2024-01-01', 2), ('2024-01-02', 1), ('2024-01-02', 4);
SYSTEM FLUSH LOGS;
SELECT format, status, rows, data_kind
FROM system.asynchronous_insert_log
WHERE database = currentDatabase() AND table = '02985_test';
DROP VIEW IF EXISTS 02985_mv;
DROP TABLE IF EXISTS 02985_test;

View File

@ -0,0 +1,6 @@
1
5 10
6 11
7 12
8 13
9 14

View File

@ -0,0 +1,36 @@
DROP TABLE IF EXISTS t_index_agg_func;
CREATE TABLE t_index_agg_func
(
id UInt64,
v AggregateFunction(avg, UInt64),
INDEX idx_v v TYPE minmax GRANULARITY 1
)
ENGINE = AggregatingMergeTree ORDER BY id
SETTINGS index_granularity = 4; -- { serverError BAD_ARGUMENTS }
CREATE TABLE t_index_agg_func
(
id UInt64,
v AggregateFunction(avg, UInt64),
)
ENGINE = AggregatingMergeTree ORDER BY id
SETTINGS index_granularity = 4;
ALTER TABLE t_index_agg_func ADD INDEX idx_v v TYPE minmax GRANULARITY 1; -- { serverError BAD_ARGUMENTS }
ALTER TABLE t_index_agg_func ADD INDEX idx_v finalizeAggregation(v) TYPE minmax GRANULARITY 1;
INSERT INTO t_index_agg_func SELECT number % 10, initializeAggregation('avgState', toUInt64(number % 20)) FROM numbers(1000);
INSERT INTO t_index_agg_func SELECT number % 10, initializeAggregation('avgState', toUInt64(number % 20)) FROM numbers(1000, 1000);
OPTIMIZE TABLE t_index_agg_func FINAL;
SELECT count() FROM system.parts WHERE table = 't_index_agg_func' AND database = currentDatabase() AND active;
SET force_data_skipping_indices = 'idx_v';
SET use_skip_indexes_if_final = 1;
SELECT id, finalizeAggregation(v) AS vv FROM t_index_agg_func FINAL WHERE vv >= 10 ORDER BY id;
DROP TABLE t_index_agg_func;

View File

@ -0,0 +1,5 @@
CREATE TABLE 02987_logical_optimizer_table (key Int, value Int) ENGINE=Memory();
CREATE VIEW v1 AS SELECT * FROM 02987_logical_optimizer_table;
CREATE TABLE 02987_logical_optimizer_merge AS v1 ENGINE=Merge(currentDatabase(), 'v1');
SELECT _table, key FROM 02987_logical_optimizer_merge WHERE (_table = toFixedString(toFixedString(toFixedString('v1', toNullable(2)), 2), 2)) OR ((value = toLowCardinality(toNullable(10))) AND (_table = toFixedString(toNullable('v3'), 2))) OR ((value = 20) AND (_table = toFixedString(toFixedString(toFixedString('v1', 2), 2), 2)) AND (_table = toFixedString(toLowCardinality(toFixedString('v3', 2)), 2))) SETTINGS allow_experimental_analyzer = true, join_use_nulls = true, convert_query_to_cnf = true;