mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge branch 'master' into pretty-type-names-default
This commit is contained in:
commit
f89bd40575
@ -3847,6 +3847,8 @@ Possible values:
|
||||
- `none` — Is similar to throw, but distributed DDL query returns no result set.
|
||||
- `null_status_on_timeout` — Returns `NULL` as execution status in some rows of result set instead of throwing `TIMEOUT_EXCEEDED` if query is not finished on the corresponding hosts.
|
||||
- `never_throw` — Do not throw `TIMEOUT_EXCEEDED` and do not rethrow exceptions if query has failed on some hosts.
|
||||
- `null_status_on_timeout_only_active` — similar to `null_status_on_timeout`, but doesn't wait for inactive replicas of the `Replicated` database
|
||||
- `throw_only_active` — similar to `throw`, but doesn't wait for inactive replicas of the `Replicated` database
|
||||
|
||||
Default value: `throw`.
|
||||
|
||||
|
@ -1483,7 +1483,9 @@ For mode values with a meaning of “with 4 or more days this year,” weeks are
|
||||
|
||||
- Otherwise, it is the last week of the previous year, and the next week is week 1.
|
||||
|
||||
For mode values with a meaning of “contains January 1”, the week contains January 1 is week 1. It does not matter how many days in the new year the week contained, even if it contained only one day.
|
||||
For mode values with a meaning of “contains January 1”, the week contains January 1 is week 1.
|
||||
It does not matter how many days in the new year the week contained, even if it contained only one day.
|
||||
I.e. if the last week of December contains January 1 of the next year, it will be week 1 of the next year.
|
||||
|
||||
**Syntax**
|
||||
|
||||
|
@ -1779,7 +1779,9 @@ Result:
|
||||
|
||||
## sqid
|
||||
|
||||
Transforms numbers into YouTube-like short URL hash called [Sqid](https://sqids.org/).
|
||||
Transforms numbers into a [Sqid](https://sqids.org/) which is a YouTube-like ID string.
|
||||
The output alphabet is `abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789`.
|
||||
Do not use this function for hashing - the generated IDs can be decoded back into numbers.
|
||||
|
||||
**Syntax**
|
||||
|
||||
|
@ -53,7 +53,7 @@ The rounded number of the same type as the input number.
|
||||
**Example of use with Float**
|
||||
|
||||
``` sql
|
||||
SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3
|
||||
SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3;
|
||||
```
|
||||
|
||||
``` text
|
||||
@ -67,7 +67,22 @@ SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3
|
||||
**Example of use with Decimal**
|
||||
|
||||
``` sql
|
||||
SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3
|
||||
SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3;
|
||||
```
|
||||
|
||||
``` text
|
||||
┌───x─┬─round(CAST(divide(number, 2), 'Decimal(10, 4)'))─┐
|
||||
│ 0 │ 0 │
|
||||
│ 0.5 │ 1 │
|
||||
│ 1 │ 1 │
|
||||
└─────┴──────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
If you want to keep the trailing zeros, you need to enable `output_format_decimal_trailing_zeros`
|
||||
|
||||
``` sql
|
||||
SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3 settings output_format_decimal_trailing_zeros=1;
|
||||
|
||||
```
|
||||
|
||||
``` text
|
||||
|
@ -578,7 +578,9 @@ SELECT
|
||||
|
||||
- В противном случае это последняя неделя предыдущего года, а следующая неделя - неделя 1.
|
||||
|
||||
Для режимов со значением «содержит 1 января», неделя 1 – это неделя содержащая 1 января. Не имеет значения, сколько дней в новом году содержала неделя, даже если она содержала только один день.
|
||||
Для режимов со значением «содержит 1 января», неделя 1 – это неделя, содержащая 1 января.
|
||||
Не имеет значения, сколько дней нового года содержит эта неделя, даже если она содержит только один день.
|
||||
Так, если последняя неделя декабря содержит 1 января следующего года, то она считается неделей 1 следующего года.
|
||||
|
||||
**Пример**
|
||||
|
||||
|
@ -1260,11 +1260,11 @@ try
|
||||
{
|
||||
Settings::checkNoSettingNamesAtTopLevel(*config, config_path);
|
||||
|
||||
ServerSettings server_settings_;
|
||||
server_settings_.loadSettingsFromConfig(*config);
|
||||
ServerSettings new_server_settings;
|
||||
new_server_settings.loadSettingsFromConfig(*config);
|
||||
|
||||
size_t max_server_memory_usage = server_settings_.max_server_memory_usage;
|
||||
double max_server_memory_usage_to_ram_ratio = server_settings_.max_server_memory_usage_to_ram_ratio;
|
||||
size_t max_server_memory_usage = new_server_settings.max_server_memory_usage;
|
||||
double max_server_memory_usage_to_ram_ratio = new_server_settings.max_server_memory_usage_to_ram_ratio;
|
||||
|
||||
size_t current_physical_server_memory = getMemoryAmount(); /// With cgroups, the amount of memory available to the server can be changed dynamically.
|
||||
size_t default_max_server_memory_usage = static_cast<size_t>(current_physical_server_memory * max_server_memory_usage_to_ram_ratio);
|
||||
@ -1294,9 +1294,9 @@ try
|
||||
total_memory_tracker.setDescription("(total)");
|
||||
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
|
||||
|
||||
size_t merges_mutations_memory_usage_soft_limit = server_settings_.merges_mutations_memory_usage_soft_limit;
|
||||
size_t merges_mutations_memory_usage_soft_limit = new_server_settings.merges_mutations_memory_usage_soft_limit;
|
||||
|
||||
size_t default_merges_mutations_server_memory_usage = static_cast<size_t>(current_physical_server_memory * server_settings_.merges_mutations_memory_usage_to_ram_ratio);
|
||||
size_t default_merges_mutations_server_memory_usage = static_cast<size_t>(current_physical_server_memory * new_server_settings.merges_mutations_memory_usage_to_ram_ratio);
|
||||
if (merges_mutations_memory_usage_soft_limit == 0)
|
||||
{
|
||||
merges_mutations_memory_usage_soft_limit = default_merges_mutations_server_memory_usage;
|
||||
@ -1304,7 +1304,7 @@ try
|
||||
" ({} available * {:.2f} merges_mutations_memory_usage_to_ram_ratio)",
|
||||
formatReadableSizeWithBinarySuffix(merges_mutations_memory_usage_soft_limit),
|
||||
formatReadableSizeWithBinarySuffix(current_physical_server_memory),
|
||||
server_settings_.merges_mutations_memory_usage_to_ram_ratio);
|
||||
new_server_settings.merges_mutations_memory_usage_to_ram_ratio);
|
||||
}
|
||||
else if (merges_mutations_memory_usage_soft_limit > default_merges_mutations_server_memory_usage)
|
||||
{
|
||||
@ -1313,7 +1313,7 @@ try
|
||||
" ({} available * {:.2f} merges_mutations_memory_usage_to_ram_ratio)",
|
||||
formatReadableSizeWithBinarySuffix(merges_mutations_memory_usage_soft_limit),
|
||||
formatReadableSizeWithBinarySuffix(current_physical_server_memory),
|
||||
server_settings_.merges_mutations_memory_usage_to_ram_ratio);
|
||||
new_server_settings.merges_mutations_memory_usage_to_ram_ratio);
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Merges and mutations memory limit is set to {}",
|
||||
@ -1322,7 +1322,7 @@ try
|
||||
background_memory_tracker.setDescription("(background)");
|
||||
background_memory_tracker.setMetric(CurrentMetrics::MergesMutationsMemoryTracking);
|
||||
|
||||
total_memory_tracker.setAllowUseJemallocMemory(server_settings_.allow_use_jemalloc_memory);
|
||||
total_memory_tracker.setAllowUseJemallocMemory(new_server_settings.allow_use_jemalloc_memory);
|
||||
|
||||
auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker();
|
||||
total_memory_tracker.setOvercommitTracker(global_overcommit_tracker);
|
||||
@ -1346,26 +1346,26 @@ try
|
||||
global_context->setRemoteHostFilter(*config);
|
||||
global_context->setHTTPHeaderFilter(*config);
|
||||
|
||||
global_context->setMaxTableSizeToDrop(server_settings_.max_table_size_to_drop);
|
||||
global_context->setMaxPartitionSizeToDrop(server_settings_.max_partition_size_to_drop);
|
||||
global_context->setMaxTableNumToWarn(server_settings_.max_table_num_to_warn);
|
||||
global_context->setMaxDatabaseNumToWarn(server_settings_.max_database_num_to_warn);
|
||||
global_context->setMaxPartNumToWarn(server_settings_.max_part_num_to_warn);
|
||||
global_context->setMaxTableSizeToDrop(new_server_settings.max_table_size_to_drop);
|
||||
global_context->setMaxPartitionSizeToDrop(new_server_settings.max_partition_size_to_drop);
|
||||
global_context->setMaxTableNumToWarn(new_server_settings.max_table_num_to_warn);
|
||||
global_context->setMaxDatabaseNumToWarn(new_server_settings.max_database_num_to_warn);
|
||||
global_context->setMaxPartNumToWarn(new_server_settings.max_part_num_to_warn);
|
||||
|
||||
ConcurrencyControl::SlotCount concurrent_threads_soft_limit = ConcurrencyControl::Unlimited;
|
||||
if (server_settings_.concurrent_threads_soft_limit_num > 0 && server_settings_.concurrent_threads_soft_limit_num < concurrent_threads_soft_limit)
|
||||
concurrent_threads_soft_limit = server_settings_.concurrent_threads_soft_limit_num;
|
||||
if (server_settings_.concurrent_threads_soft_limit_ratio_to_cores > 0)
|
||||
if (new_server_settings.concurrent_threads_soft_limit_num > 0 && new_server_settings.concurrent_threads_soft_limit_num < concurrent_threads_soft_limit)
|
||||
concurrent_threads_soft_limit = new_server_settings.concurrent_threads_soft_limit_num;
|
||||
if (new_server_settings.concurrent_threads_soft_limit_ratio_to_cores > 0)
|
||||
{
|
||||
auto value = server_settings_.concurrent_threads_soft_limit_ratio_to_cores * std::thread::hardware_concurrency();
|
||||
auto value = new_server_settings.concurrent_threads_soft_limit_ratio_to_cores * std::thread::hardware_concurrency();
|
||||
if (value > 0 && value < concurrent_threads_soft_limit)
|
||||
concurrent_threads_soft_limit = value;
|
||||
}
|
||||
ConcurrencyControl::instance().setMaxConcurrency(concurrent_threads_soft_limit);
|
||||
|
||||
global_context->getProcessList().setMaxSize(server_settings_.max_concurrent_queries);
|
||||
global_context->getProcessList().setMaxInsertQueriesAmount(server_settings_.max_concurrent_insert_queries);
|
||||
global_context->getProcessList().setMaxSelectQueriesAmount(server_settings_.max_concurrent_select_queries);
|
||||
global_context->getProcessList().setMaxSize(new_server_settings.max_concurrent_queries);
|
||||
global_context->getProcessList().setMaxInsertQueriesAmount(new_server_settings.max_concurrent_insert_queries);
|
||||
global_context->getProcessList().setMaxSelectQueriesAmount(new_server_settings.max_concurrent_select_queries);
|
||||
|
||||
if (config->has("keeper_server"))
|
||||
global_context->updateKeeperConfiguration(*config);
|
||||
@ -1376,68 +1376,68 @@ try
|
||||
/// This is done for backward compatibility.
|
||||
if (global_context->areBackgroundExecutorsInitialized())
|
||||
{
|
||||
auto new_pool_size = server_settings_.background_pool_size;
|
||||
auto new_ratio = server_settings_.background_merges_mutations_concurrency_ratio;
|
||||
auto new_pool_size = new_server_settings.background_pool_size;
|
||||
auto new_ratio = new_server_settings.background_merges_mutations_concurrency_ratio;
|
||||
global_context->getMergeMutateExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, static_cast<size_t>(new_pool_size * new_ratio));
|
||||
global_context->getMergeMutateExecutor()->updateSchedulingPolicy(server_settings_.background_merges_mutations_scheduling_policy.toString());
|
||||
global_context->getMergeMutateExecutor()->updateSchedulingPolicy(new_server_settings.background_merges_mutations_scheduling_policy.toString());
|
||||
}
|
||||
|
||||
if (global_context->areBackgroundExecutorsInitialized())
|
||||
{
|
||||
auto new_pool_size = server_settings_.background_move_pool_size;
|
||||
auto new_pool_size = new_server_settings.background_move_pool_size;
|
||||
global_context->getMovesExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size);
|
||||
}
|
||||
|
||||
if (global_context->areBackgroundExecutorsInitialized())
|
||||
{
|
||||
auto new_pool_size = server_settings_.background_fetches_pool_size;
|
||||
auto new_pool_size = new_server_settings.background_fetches_pool_size;
|
||||
global_context->getFetchesExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size);
|
||||
}
|
||||
|
||||
if (global_context->areBackgroundExecutorsInitialized())
|
||||
{
|
||||
auto new_pool_size = server_settings_.background_common_pool_size;
|
||||
auto new_pool_size = new_server_settings.background_common_pool_size;
|
||||
global_context->getCommonExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size);
|
||||
}
|
||||
|
||||
global_context->getBufferFlushSchedulePool().increaseThreadsCount(server_settings_.background_buffer_flush_schedule_pool_size);
|
||||
global_context->getSchedulePool().increaseThreadsCount(server_settings_.background_schedule_pool_size);
|
||||
global_context->getMessageBrokerSchedulePool().increaseThreadsCount(server_settings_.background_message_broker_schedule_pool_size);
|
||||
global_context->getDistributedSchedulePool().increaseThreadsCount(server_settings_.background_distributed_schedule_pool_size);
|
||||
global_context->getBufferFlushSchedulePool().increaseThreadsCount(new_server_settings.background_buffer_flush_schedule_pool_size);
|
||||
global_context->getSchedulePool().increaseThreadsCount(new_server_settings.background_schedule_pool_size);
|
||||
global_context->getMessageBrokerSchedulePool().increaseThreadsCount(new_server_settings.background_message_broker_schedule_pool_size);
|
||||
global_context->getDistributedSchedulePool().increaseThreadsCount(new_server_settings.background_distributed_schedule_pool_size);
|
||||
|
||||
global_context->getAsyncLoader().setMaxThreads(TablesLoaderForegroundPoolId, server_settings_.tables_loader_foreground_pool_size);
|
||||
global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundLoadPoolId, server_settings_.tables_loader_background_pool_size);
|
||||
global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundStartupPoolId, server_settings_.tables_loader_background_pool_size);
|
||||
global_context->getAsyncLoader().setMaxThreads(TablesLoaderForegroundPoolId, new_server_settings.tables_loader_foreground_pool_size);
|
||||
global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundLoadPoolId, new_server_settings.tables_loader_background_pool_size);
|
||||
global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundStartupPoolId, new_server_settings.tables_loader_background_pool_size);
|
||||
|
||||
getIOThreadPool().reloadConfiguration(
|
||||
server_settings.max_io_thread_pool_size,
|
||||
server_settings.max_io_thread_pool_free_size,
|
||||
server_settings.io_thread_pool_queue_size);
|
||||
new_server_settings.max_io_thread_pool_size,
|
||||
new_server_settings.max_io_thread_pool_free_size,
|
||||
new_server_settings.io_thread_pool_queue_size);
|
||||
|
||||
getBackupsIOThreadPool().reloadConfiguration(
|
||||
server_settings.max_backups_io_thread_pool_size,
|
||||
server_settings.max_backups_io_thread_pool_free_size,
|
||||
server_settings.backups_io_thread_pool_queue_size);
|
||||
new_server_settings.max_backups_io_thread_pool_size,
|
||||
new_server_settings.max_backups_io_thread_pool_free_size,
|
||||
new_server_settings.backups_io_thread_pool_queue_size);
|
||||
|
||||
getActivePartsLoadingThreadPool().reloadConfiguration(
|
||||
server_settings.max_active_parts_loading_thread_pool_size,
|
||||
new_server_settings.max_active_parts_loading_thread_pool_size,
|
||||
0, // We don't need any threads once all the parts will be loaded
|
||||
server_settings.max_active_parts_loading_thread_pool_size);
|
||||
new_server_settings.max_active_parts_loading_thread_pool_size);
|
||||
|
||||
getOutdatedPartsLoadingThreadPool().reloadConfiguration(
|
||||
server_settings.max_outdated_parts_loading_thread_pool_size,
|
||||
new_server_settings.max_outdated_parts_loading_thread_pool_size,
|
||||
0, // We don't need any threads once all the parts will be loaded
|
||||
server_settings.max_outdated_parts_loading_thread_pool_size);
|
||||
new_server_settings.max_outdated_parts_loading_thread_pool_size);
|
||||
|
||||
/// It could grow if we need to synchronously wait until all the data parts will be loaded.
|
||||
getOutdatedPartsLoadingThreadPool().setMaxTurboThreads(
|
||||
server_settings.max_active_parts_loading_thread_pool_size
|
||||
new_server_settings.max_active_parts_loading_thread_pool_size
|
||||
);
|
||||
|
||||
getPartsCleaningThreadPool().reloadConfiguration(
|
||||
server_settings.max_parts_cleaning_thread_pool_size,
|
||||
new_server_settings.max_parts_cleaning_thread_pool_size,
|
||||
0, // We don't need any threads one all the parts will be deleted
|
||||
server_settings.max_parts_cleaning_thread_pool_size);
|
||||
new_server_settings.max_parts_cleaning_thread_pool_size);
|
||||
|
||||
if (config->has("resources"))
|
||||
{
|
||||
|
@ -1379,6 +1379,9 @@
|
||||
|
||||
<!-- Controls how many tasks could be in the queue -->
|
||||
<!-- <max_tasks_in_queue>1000</max_tasks_in_queue> -->
|
||||
|
||||
<!-- Host name of the current node. If specified, will only compare and not resolve hostnames inside the DDL tasks -->
|
||||
<!-- <host_name>replica</host_name> -->
|
||||
</distributed_ddl>
|
||||
|
||||
<!-- Settings to fine tune MergeTree tables. See documentation in source code, in MergeTreeSettings.h -->
|
||||
|
@ -14,8 +14,9 @@
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/iota.h>
|
||||
#include <base/types.h>
|
||||
|
||||
#include <boost/math/distributions/normal.hpp>
|
||||
@ -48,7 +49,7 @@ struct LargestTriangleThreeBucketsData : public StatisticalSample<Float64, Float
|
||||
// sort the this->x and this->y in ascending order of this->x using index
|
||||
std::vector<size_t> index(this->x.size());
|
||||
|
||||
std::iota(index.begin(), index.end(), 0);
|
||||
iota(index.data(), index.size(), size_t(0));
|
||||
::sort(index.begin(), index.end(), [&](size_t i1, size_t i2) { return this->x[i1] < this->x[i2]; });
|
||||
|
||||
SampleX temp_x{};
|
||||
|
@ -1,7 +1,8 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include <AggregateFunctions/HelpersMinMaxAny.h>
|
||||
#include <AggregateFunctions/findNumeric.h>
|
||||
#include <Common/Concepts.h>
|
||||
#include <Common/findExtreme.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -19,7 +20,7 @@ public:
|
||||
explicit AggregateFunctionsSingleValueMax(const DataTypePtr & type) : Parent(type) { }
|
||||
|
||||
/// Specializations for native numeric types
|
||||
ALWAYS_INLINE inline void addBatchSinglePlace(
|
||||
void addBatchSinglePlace(
|
||||
size_t row_begin,
|
||||
size_t row_end,
|
||||
AggregateDataPtr __restrict place,
|
||||
@ -27,7 +28,7 @@ public:
|
||||
Arena * arena,
|
||||
ssize_t if_argument_pos) const override;
|
||||
|
||||
ALWAYS_INLINE inline void addBatchSinglePlaceNotNull(
|
||||
void addBatchSinglePlaceNotNull(
|
||||
size_t row_begin,
|
||||
size_t row_end,
|
||||
AggregateDataPtr __restrict place,
|
||||
@ -53,10 +54,10 @@ void AggregateFunctionsSingleValueMax<typename DB::AggregateFunctionMaxData<Sing
|
||||
if (if_argument_pos >= 0) \
|
||||
{ \
|
||||
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData(); \
|
||||
opt = findNumericMaxIf(column.getData().data(), flags.data(), row_begin, row_end); \
|
||||
opt = findExtremeMaxIf(column.getData().data(), flags.data(), row_begin, row_end); \
|
||||
} \
|
||||
else \
|
||||
opt = findNumericMax(column.getData().data(), row_begin, row_end); \
|
||||
opt = findExtremeMax(column.getData().data(), row_begin, row_end); \
|
||||
if (opt.has_value()) \
|
||||
this->data(place).changeIfGreater(opt.value()); \
|
||||
}
|
||||
@ -74,7 +75,57 @@ void AggregateFunctionsSingleValueMax<Data>::addBatchSinglePlace(
|
||||
Arena * arena,
|
||||
ssize_t if_argument_pos) const
|
||||
{
|
||||
return Parent::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos);
|
||||
if constexpr (!is_any_of<typename Data::Impl, SingleValueDataString, SingleValueDataGeneric>)
|
||||
{
|
||||
/// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's
|
||||
/// faster than doing a permutation
|
||||
return Parent::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos);
|
||||
}
|
||||
|
||||
constexpr int nan_direction_hint = 1;
|
||||
auto const & column = *columns[0];
|
||||
if (if_argument_pos >= 0)
|
||||
{
|
||||
size_t index = row_begin;
|
||||
const auto & if_flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
|
||||
while (if_flags[index] == 0 && index < row_end)
|
||||
index++;
|
||||
if (index >= row_end)
|
||||
return;
|
||||
|
||||
for (size_t i = index + 1; i < row_end; i++)
|
||||
{
|
||||
if ((if_flags[i] != 0) && (column.compareAt(i, index, column, nan_direction_hint) > 0))
|
||||
index = i;
|
||||
}
|
||||
this->data(place).changeIfGreater(column, index, arena);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (row_begin >= row_end)
|
||||
return;
|
||||
|
||||
/// TODO: Introduce row_begin and row_end to getPermutation
|
||||
if (row_begin != 0 || row_end != column.size())
|
||||
{
|
||||
size_t index = row_begin;
|
||||
for (size_t i = index + 1; i < row_end; i++)
|
||||
{
|
||||
if (column.compareAt(i, index, column, nan_direction_hint) > 0)
|
||||
index = i;
|
||||
}
|
||||
this->data(place).changeIfGreater(column, index, arena);
|
||||
}
|
||||
else
|
||||
{
|
||||
constexpr IColumn::PermutationSortDirection direction = IColumn::PermutationSortDirection::Descending;
|
||||
constexpr IColumn::PermutationSortStability stability = IColumn::PermutationSortStability::Unstable;
|
||||
IColumn::Permutation permutation;
|
||||
constexpr UInt64 limit = 1;
|
||||
column.getPermutation(direction, stability, limit, nan_direction_hint, permutation);
|
||||
this->data(place).changeIfGreater(column, permutation[0], arena);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTBEGIN(bugprone-macro-parentheses)
|
||||
@ -97,10 +148,10 @@ void AggregateFunctionsSingleValueMax<typename DB::AggregateFunctionMaxData<Sing
|
||||
auto final_flags = std::make_unique<UInt8[]>(row_end); \
|
||||
for (size_t i = row_begin; i < row_end; ++i) \
|
||||
final_flags[i] = (!null_map[i]) & !!if_flags[i]; \
|
||||
opt = findNumericMaxIf(column.getData().data(), final_flags.get(), row_begin, row_end); \
|
||||
opt = findExtremeMaxIf(column.getData().data(), final_flags.get(), row_begin, row_end); \
|
||||
} \
|
||||
else \
|
||||
opt = findNumericMaxNotNull(column.getData().data(), null_map, row_begin, row_end); \
|
||||
opt = findExtremeMaxNotNull(column.getData().data(), null_map, row_begin, row_end); \
|
||||
if (opt.has_value()) \
|
||||
this->data(place).changeIfGreater(opt.value()); \
|
||||
}
|
||||
@ -119,7 +170,46 @@ void AggregateFunctionsSingleValueMax<Data>::addBatchSinglePlaceNotNull(
|
||||
Arena * arena,
|
||||
ssize_t if_argument_pos) const
|
||||
{
|
||||
return Parent::addBatchSinglePlaceNotNull(row_begin, row_end, place, columns, null_map, arena, if_argument_pos);
|
||||
if constexpr (!is_any_of<typename Data::Impl, SingleValueDataString, SingleValueDataGeneric>)
|
||||
{
|
||||
/// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's
|
||||
/// faster than doing a permutation
|
||||
return Parent::addBatchSinglePlaceNotNull(row_begin, row_end, place, columns, null_map, arena, if_argument_pos);
|
||||
}
|
||||
|
||||
constexpr int nan_direction_hint = 1;
|
||||
auto const & column = *columns[0];
|
||||
if (if_argument_pos >= 0)
|
||||
{
|
||||
size_t index = row_begin;
|
||||
const auto & if_flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
|
||||
while ((if_flags[index] == 0 || null_map[index] != 0) && (index < row_end))
|
||||
index++;
|
||||
if (index >= row_end)
|
||||
return;
|
||||
|
||||
for (size_t i = index + 1; i < row_end; i++)
|
||||
{
|
||||
if ((if_flags[i] != 0) && (null_map[i] == 0) && (column.compareAt(i, index, column, nan_direction_hint) > 0))
|
||||
index = i;
|
||||
}
|
||||
this->data(place).changeIfGreater(column, index, arena);
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t index = row_begin;
|
||||
while ((null_map[index] != 0) && (index < row_end))
|
||||
index++;
|
||||
if (index >= row_end)
|
||||
return;
|
||||
|
||||
for (size_t i = index + 1; i < row_end; i++)
|
||||
{
|
||||
if ((null_map[i] == 0) && (column.compareAt(i, index, column, nan_direction_hint) > 0))
|
||||
index = i;
|
||||
}
|
||||
this->data(place).changeIfGreater(column, index, arena);
|
||||
}
|
||||
}
|
||||
|
||||
AggregateFunctionPtr createAggregateFunctionMax(
|
||||
|
@ -1,7 +1,8 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include <AggregateFunctions/HelpersMinMaxAny.h>
|
||||
#include <AggregateFunctions/findNumeric.h>
|
||||
#include <Common/Concepts.h>
|
||||
#include <Common/findExtreme.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -20,7 +21,7 @@ public:
|
||||
explicit AggregateFunctionsSingleValueMin(const DataTypePtr & type) : Parent(type) { }
|
||||
|
||||
/// Specializations for native numeric types
|
||||
ALWAYS_INLINE inline void addBatchSinglePlace(
|
||||
void addBatchSinglePlace(
|
||||
size_t row_begin,
|
||||
size_t row_end,
|
||||
AggregateDataPtr __restrict place,
|
||||
@ -28,7 +29,7 @@ public:
|
||||
Arena * arena,
|
||||
ssize_t if_argument_pos) const override;
|
||||
|
||||
ALWAYS_INLINE inline void addBatchSinglePlaceNotNull(
|
||||
void addBatchSinglePlaceNotNull(
|
||||
size_t row_begin,
|
||||
size_t row_end,
|
||||
AggregateDataPtr __restrict place,
|
||||
@ -54,10 +55,10 @@ public:
|
||||
if (if_argument_pos >= 0) \
|
||||
{ \
|
||||
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData(); \
|
||||
opt = findNumericMinIf(column.getData().data(), flags.data(), row_begin, row_end); \
|
||||
opt = findExtremeMinIf(column.getData().data(), flags.data(), row_begin, row_end); \
|
||||
} \
|
||||
else \
|
||||
opt = findNumericMin(column.getData().data(), row_begin, row_end); \
|
||||
opt = findExtremeMin(column.getData().data(), row_begin, row_end); \
|
||||
if (opt.has_value()) \
|
||||
this->data(place).changeIfLess(opt.value()); \
|
||||
}
|
||||
@ -75,7 +76,57 @@ void AggregateFunctionsSingleValueMin<Data>::addBatchSinglePlace(
|
||||
Arena * arena,
|
||||
ssize_t if_argument_pos) const
|
||||
{
|
||||
return Parent::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos);
|
||||
if constexpr (!is_any_of<typename Data::Impl, SingleValueDataString, SingleValueDataGeneric>)
|
||||
{
|
||||
/// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's
|
||||
/// faster than doing a permutation
|
||||
return Parent::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos);
|
||||
}
|
||||
|
||||
constexpr int nan_direction_hint = 1;
|
||||
auto const & column = *columns[0];
|
||||
if (if_argument_pos >= 0)
|
||||
{
|
||||
size_t index = row_begin;
|
||||
const auto & if_flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
|
||||
while (if_flags[index] == 0 && index < row_end)
|
||||
index++;
|
||||
if (index >= row_end)
|
||||
return;
|
||||
|
||||
for (size_t i = index + 1; i < row_end; i++)
|
||||
{
|
||||
if ((if_flags[i] != 0) && (column.compareAt(i, index, column, nan_direction_hint) < 0))
|
||||
index = i;
|
||||
}
|
||||
this->data(place).changeIfLess(column, index, arena);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (row_begin >= row_end)
|
||||
return;
|
||||
|
||||
/// TODO: Introduce row_begin and row_end to getPermutation
|
||||
if (row_begin != 0 || row_end != column.size())
|
||||
{
|
||||
size_t index = row_begin;
|
||||
for (size_t i = index + 1; i < row_end; i++)
|
||||
{
|
||||
if (column.compareAt(i, index, column, nan_direction_hint) < 0)
|
||||
index = i;
|
||||
}
|
||||
this->data(place).changeIfLess(column, index, arena);
|
||||
}
|
||||
else
|
||||
{
|
||||
constexpr IColumn::PermutationSortDirection direction = IColumn::PermutationSortDirection::Ascending;
|
||||
constexpr IColumn::PermutationSortStability stability = IColumn::PermutationSortStability::Unstable;
|
||||
IColumn::Permutation permutation;
|
||||
constexpr UInt64 limit = 1;
|
||||
column.getPermutation(direction, stability, limit, nan_direction_hint, permutation);
|
||||
this->data(place).changeIfLess(column, permutation[0], arena);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTBEGIN(bugprone-macro-parentheses)
|
||||
@ -98,10 +149,10 @@ void AggregateFunctionsSingleValueMin<Data>::addBatchSinglePlace(
|
||||
auto final_flags = std::make_unique<UInt8[]>(row_end); \
|
||||
for (size_t i = row_begin; i < row_end; ++i) \
|
||||
final_flags[i] = (!null_map[i]) & !!if_flags[i]; \
|
||||
opt = findNumericMinIf(column.getData().data(), final_flags.get(), row_begin, row_end); \
|
||||
opt = findExtremeMinIf(column.getData().data(), final_flags.get(), row_begin, row_end); \
|
||||
} \
|
||||
else \
|
||||
opt = findNumericMinNotNull(column.getData().data(), null_map, row_begin, row_end); \
|
||||
opt = findExtremeMinNotNull(column.getData().data(), null_map, row_begin, row_end); \
|
||||
if (opt.has_value()) \
|
||||
this->data(place).changeIfLess(opt.value()); \
|
||||
}
|
||||
@ -120,7 +171,46 @@ void AggregateFunctionsSingleValueMin<Data>::addBatchSinglePlaceNotNull(
|
||||
Arena * arena,
|
||||
ssize_t if_argument_pos) const
|
||||
{
|
||||
return Parent::addBatchSinglePlaceNotNull(row_begin, row_end, place, columns, null_map, arena, if_argument_pos);
|
||||
if constexpr (!is_any_of<typename Data::Impl, SingleValueDataString, SingleValueDataGeneric>)
|
||||
{
|
||||
/// Leave other numeric types (large integers, decimals, etc) to keep doing the comparison as it's
|
||||
/// faster than doing a permutation
|
||||
return Parent::addBatchSinglePlaceNotNull(row_begin, row_end, place, columns, null_map, arena, if_argument_pos);
|
||||
}
|
||||
|
||||
constexpr int nan_direction_hint = 1;
|
||||
auto const & column = *columns[0];
|
||||
if (if_argument_pos >= 0)
|
||||
{
|
||||
size_t index = row_begin;
|
||||
const auto & if_flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
|
||||
while ((if_flags[index] == 0 || null_map[index] != 0) && (index < row_end))
|
||||
index++;
|
||||
if (index >= row_end)
|
||||
return;
|
||||
|
||||
for (size_t i = index + 1; i < row_end; i++)
|
||||
{
|
||||
if ((if_flags[i] != 0) && (null_map[index] == 0) && (column.compareAt(i, index, column, nan_direction_hint) < 0))
|
||||
index = i;
|
||||
}
|
||||
this->data(place).changeIfLess(column, index, arena);
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t index = row_begin;
|
||||
while ((null_map[index] != 0) && (index < row_end))
|
||||
index++;
|
||||
if (index >= row_end)
|
||||
return;
|
||||
|
||||
for (size_t i = index + 1; i < row_end; i++)
|
||||
{
|
||||
if ((null_map[i] == 0) && (column.compareAt(i, index, column, nan_direction_hint) < 0))
|
||||
index = i;
|
||||
}
|
||||
this->data(place).changeIfLess(column, index, arena);
|
||||
}
|
||||
}
|
||||
|
||||
AggregateFunctionPtr createAggregateFunctionMin(
|
||||
|
@ -965,6 +965,7 @@ template <typename Data>
|
||||
struct AggregateFunctionMinData : Data
|
||||
{
|
||||
using Self = AggregateFunctionMinData;
|
||||
using Impl = Data;
|
||||
|
||||
bool changeIfBetter(const IColumn & column, size_t row_num, Arena * arena) { return this->changeIfLess(column, row_num, arena); }
|
||||
bool changeIfBetter(const Self & to, Arena * arena) { return this->changeIfLess(to, arena); }
|
||||
@ -993,6 +994,7 @@ template <typename Data>
|
||||
struct AggregateFunctionMaxData : Data
|
||||
{
|
||||
using Self = AggregateFunctionMaxData;
|
||||
using Impl = Data;
|
||||
|
||||
bool changeIfBetter(const IColumn & column, size_t row_num, Arena * arena) { return this->changeIfGreater(column, row_num, arena); }
|
||||
bool changeIfBetter(const Self & to, Arena * arena) { return this->changeIfGreater(to, arena); }
|
||||
|
@ -6,6 +6,7 @@
|
||||
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -63,10 +64,9 @@ struct QuantileLevels
|
||||
|
||||
if (isNaN(levels[i]) || levels[i] < 0 || levels[i] > 1)
|
||||
throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Quantile level is out of range [0..1]");
|
||||
|
||||
permutation[i] = i;
|
||||
}
|
||||
|
||||
iota(permutation.data(), size, Permutation::value_type(0));
|
||||
::sort(permutation.begin(), permutation.end(), [this] (size_t a, size_t b) { return levels[a] < levels[b]; });
|
||||
}
|
||||
};
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <base/sort.h>
|
||||
|
||||
#include <Common/ArenaAllocator.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
@ -30,7 +31,7 @@ std::pair<RanksArray, Float64> computeRanksAndTieCorrection(const Values & value
|
||||
const size_t size = values.size();
|
||||
/// Save initial positions, than sort indices according to the values.
|
||||
std::vector<size_t> indexes(size);
|
||||
std::iota(indexes.begin(), indexes.end(), 0);
|
||||
iota(indexes.data(), indexes.size(), size_t(0));
|
||||
std::sort(indexes.begin(), indexes.end(),
|
||||
[&] (size_t lhs, size_t rhs) { return values[lhs] < values[rhs]; });
|
||||
|
||||
|
@ -1,15 +0,0 @@
|
||||
#include <AggregateFunctions/findNumeric.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
#define INSTANTIATION(T) \
|
||||
template std::optional<T> findNumericMin(const T * __restrict ptr, size_t start, size_t end); \
|
||||
template std::optional<T> findNumericMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \
|
||||
template std::optional<T> findNumericMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \
|
||||
template std::optional<T> findNumericMax(const T * __restrict ptr, size_t start, size_t end); \
|
||||
template std::optional<T> findNumericMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \
|
||||
template std::optional<T> findNumericMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end);
|
||||
|
||||
FOR_BASIC_NUMERIC_TYPES(INSTANTIATION)
|
||||
#undef INSTANTIATION
|
||||
}
|
@ -1,5 +1,6 @@
|
||||
#include <Analyzer/Passes/FuseFunctionsPass.h>
|
||||
|
||||
#include <Common/iota.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
@ -184,7 +185,7 @@ FunctionNodePtr createFusedQuantilesNode(std::vector<QueryTreeNodePtr *> & nodes
|
||||
{
|
||||
/// Sort nodes and parameters in ascending order of quantile level
|
||||
std::vector<size_t> permutation(nodes.size());
|
||||
std::iota(permutation.begin(), permutation.end(), 0);
|
||||
iota(permutation.data(), permutation.size(), size_t(0));
|
||||
std::sort(permutation.begin(), permutation.end(), [&](size_t i, size_t j) { return parameters[i].get<Float64>() < parameters[j].get<Float64>(); });
|
||||
|
||||
std::vector<QueryTreeNodePtr *> new_nodes;
|
||||
|
@ -1,18 +1,19 @@
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Columns/MaskOperations.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Processors/Transforms/ColumnGathererTransform.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBufferFromArena.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Processors/Transforms/ColumnGathererTransform.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -626,8 +627,7 @@ void ColumnAggregateFunction::getPermutation(PermutationSortDirection /*directio
|
||||
{
|
||||
size_t s = data.size();
|
||||
res.resize(s);
|
||||
for (size_t i = 0; i < s; ++i)
|
||||
res[i] = i;
|
||||
iota(res.data(), s, IColumn::Permutation::value_type(0));
|
||||
}
|
||||
|
||||
void ColumnAggregateFunction::updatePermutation(PermutationSortDirection, PermutationSortStability,
|
||||
|
@ -2,9 +2,10 @@
|
||||
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <base/defines.h>
|
||||
|
||||
@ -128,8 +129,7 @@ void ColumnConst::getPermutation(PermutationSortDirection /*direction*/, Permuta
|
||||
size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const
|
||||
{
|
||||
res.resize(s);
|
||||
for (size_t i = 0; i < s; ++i)
|
||||
res[i] = i;
|
||||
iota(res.data(), s, IColumn::Permutation::value_type(0));
|
||||
}
|
||||
|
||||
void ColumnConst::updatePermutation(PermutationSortDirection /*direction*/, PermutationSortStability /*stability*/,
|
||||
|
@ -1,10 +1,11 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Common/RadixSort.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
#include <base/sort.h>
|
||||
|
||||
@ -163,8 +164,7 @@ void ColumnDecimal<T>::getPermutation(IColumn::PermutationSortDirection directio
|
||||
if (limit >= data_size)
|
||||
limit = 0;
|
||||
|
||||
for (size_t i = 0; i < data_size; ++i)
|
||||
res[i] = i;
|
||||
iota(res.data(), data_size, IColumn::Permutation::value_type(0));
|
||||
|
||||
if constexpr (is_arithmetic_v<NativeT> && !is_big_int_v<NativeT>)
|
||||
{
|
||||
@ -183,8 +183,7 @@ void ColumnDecimal<T>::getPermutation(IColumn::PermutationSortDirection directio
|
||||
/// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters.
|
||||
if (data_size >= 256 && data_size <= std::numeric_limits<UInt32>::max() && use_radix_sort)
|
||||
{
|
||||
for (size_t i = 0; i < data_size; ++i)
|
||||
res[i] = i;
|
||||
iota(res.data(), data_size, IColumn::Permutation::value_type(0));
|
||||
|
||||
bool try_sort = false;
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Columns/ColumnObject.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Common/iota.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
@ -838,7 +839,7 @@ MutableColumnPtr ColumnObject::cloneResized(size_t new_size) const
|
||||
void ColumnObject::getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation & res) const
|
||||
{
|
||||
res.resize(num_rows);
|
||||
std::iota(res.begin(), res.end(), 0);
|
||||
iota(res.data(), res.size(), size_t(0));
|
||||
}
|
||||
|
||||
void ColumnObject::compareColumn(const IColumn & rhs, size_t rhs_row_num,
|
||||
|
@ -1,11 +1,12 @@
|
||||
#include <Columns/ColumnSparse.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Columns/ColumnCompressed.h>
|
||||
#include <Columns/ColumnSparse.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Processors/Transforms/ColumnGathererTransform.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <bit>
|
||||
@ -499,8 +500,7 @@ void ColumnSparse::getPermutationImpl(IColumn::PermutationSortDirection directio
|
||||
res.resize(_size);
|
||||
if (offsets->empty())
|
||||
{
|
||||
for (size_t i = 0; i < _size; ++i)
|
||||
res[i] = i;
|
||||
iota(res.data(), _size, IColumn::Permutation::value_type(0));
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -1,16 +1,17 @@
|
||||
#include <Columns/ColumnTuple.h>
|
||||
|
||||
#include <base/sort.h>
|
||||
#include <Columns/IColumnImpl.h>
|
||||
#include <Columns/ColumnCompressed.h>
|
||||
#include <Columns/IColumnImpl.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Processors/Transforms/ColumnGathererTransform.h>
|
||||
#include <DataTypes/Serializations/SerializationInfoTuple.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Processors/Transforms/ColumnGathererTransform.h>
|
||||
#include <base/sort.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataTypes/Serializations/SerializationInfoTuple.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -378,8 +379,7 @@ void ColumnTuple::getPermutationImpl(IColumn::PermutationSortDirection direction
|
||||
{
|
||||
size_t rows = size();
|
||||
res.resize(rows);
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
res[i] = i;
|
||||
iota(res.data(), rows, IColumn::Permutation::value_type(0));
|
||||
|
||||
if (limit >= rows)
|
||||
limit = 0;
|
||||
|
@ -1,24 +1,25 @@
|
||||
#include "ColumnVector.h"
|
||||
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Columns/ColumnCompressed.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Columns/MaskOperations.h>
|
||||
#include <Columns/RadixSortHelper.h>
|
||||
#include <Processors/Transforms/ColumnGathererTransform.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Processors/Transforms/ColumnGathererTransform.h>
|
||||
#include <base/bit_cast.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <base/sort.h>
|
||||
#include <base/unaligned.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
#include <Common/RadixSort.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/TargetSpecific.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <base/sort.h>
|
||||
#include <base/unaligned.h>
|
||||
#include <base/bit_cast.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
#include <bit>
|
||||
#include <cmath>
|
||||
@ -244,8 +245,7 @@ void ColumnVector<T>::getPermutation(IColumn::PermutationSortDirection direction
|
||||
if (limit >= data_size)
|
||||
limit = 0;
|
||||
|
||||
for (size_t i = 0; i < data_size; ++i)
|
||||
res[i] = i;
|
||||
iota(res.data(), data_size, IColumn::Permutation::value_type(0));
|
||||
|
||||
if constexpr (is_arithmetic_v<T> && !is_big_int_v<T>)
|
||||
{
|
||||
|
@ -1,7 +1,8 @@
|
||||
#include <Common/Arena.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Columns/IColumnDummy.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Columns/IColumnDummy.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -87,8 +88,7 @@ void IColumnDummy::getPermutation(IColumn::PermutationSortDirection /*direction*
|
||||
size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const
|
||||
{
|
||||
res.resize(s);
|
||||
for (size_t i = 0; i < s; ++i)
|
||||
res[i] = i;
|
||||
iota(res.data(), s, IColumn::Permutation::value_type(0));
|
||||
}
|
||||
|
||||
ColumnPtr IColumnDummy::replicate(const Offsets & offsets) const
|
||||
|
@ -6,10 +6,11 @@
|
||||
* implementation.
|
||||
*/
|
||||
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <base/sort.h>
|
||||
#include <algorithm>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <base/sort.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -299,8 +300,7 @@ void IColumn::getPermutationImpl(
|
||||
if (limit >= data_size)
|
||||
limit = 0;
|
||||
|
||||
for (size_t i = 0; i < data_size; ++i)
|
||||
res[i] = i;
|
||||
iota(res.data(), data_size, Permutation::value_type(0));
|
||||
|
||||
if (limit)
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Columns/ColumnSparse.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
#include <Common/iota.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <pcg_random.hpp>
|
||||
#include <gtest/gtest.h>
|
||||
@ -191,7 +192,7 @@ TEST(ColumnSparse, Permute)
|
||||
auto [sparse_src, full_src] = createColumns(n, k);
|
||||
|
||||
IColumn::Permutation perm(n);
|
||||
std::iota(perm.begin(), perm.end(), 0);
|
||||
iota(perm.data(), perm.size(), size_t(0));
|
||||
std::shuffle(perm.begin(), perm.end(), rng);
|
||||
|
||||
auto sparse_dst = sparse_src->permute(perm, limit);
|
||||
|
@ -9,7 +9,6 @@
|
||||
#include <Columns/ColumnUnique.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
@ -17,6 +16,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
|
||||
using namespace DB;
|
||||
@ -32,8 +32,7 @@ void stableGetColumnPermutation(
|
||||
|
||||
size_t size = column.size();
|
||||
out_permutation.resize(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
out_permutation[i] = i;
|
||||
iota(out_permutation.data(), size, IColumn::Permutation::value_type(0));
|
||||
|
||||
std::stable_sort(
|
||||
out_permutation.begin(),
|
||||
@ -146,10 +145,7 @@ void assertColumnPermutations(ColumnCreateFunc column_create_func, ValueTransfor
|
||||
|
||||
std::vector<std::vector<Field>> ranges(ranges_size);
|
||||
std::vector<size_t> ranges_permutations(ranges_size);
|
||||
for (size_t i = 0; i < ranges_size; ++i)
|
||||
{
|
||||
ranges_permutations[i] = i;
|
||||
}
|
||||
iota(ranges_permutations.data(), ranges_size, IColumn::Permutation::value_type(0));
|
||||
|
||||
IColumn::Permutation actual_permutation;
|
||||
IColumn::Permutation expected_permutation;
|
||||
|
@ -288,6 +288,18 @@ The server successfully detected this situation and will download merged part fr
|
||||
M(OSReadChars, "Number of bytes read from filesystem, including page cache.") \
|
||||
M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \
|
||||
\
|
||||
M(ParallelReplicasHandleRequestMicroseconds, "Time spent processing requests for marks from replicas") \
|
||||
M(ParallelReplicasHandleAnnouncementMicroseconds, "Time spent processing replicas announcements") \
|
||||
\
|
||||
M(ParallelReplicasReadAssignedMarks, "Sum across all replicas of how many of scheduled marks were assigned by consistent hash") \
|
||||
M(ParallelReplicasReadUnassignedMarks, "Sum across all replicas of how many unassigned marks were scheduled") \
|
||||
M(ParallelReplicasReadAssignedForStealingMarks, "Sum across all replicas of how many of scheduled marks were assigned for stealing by consistent hash") \
|
||||
\
|
||||
M(ParallelReplicasStealingByHashMicroseconds, "Time spent collecting segments meant for stealing by hash") \
|
||||
M(ParallelReplicasProcessingPartsMicroseconds, "Time spent processing data parts") \
|
||||
M(ParallelReplicasStealingLeftoversMicroseconds, "Time spent collecting orphaned segments") \
|
||||
M(ParallelReplicasCollectingOwnedSegmentsMicroseconds, "Time spent collecting segments meant by hash") \
|
||||
\
|
||||
M(PerfCpuCycles, "Total cycles. Be wary of what happens during CPU frequency scaling.") \
|
||||
M(PerfInstructions, "Retired instructions. Be careful, these can be affected by various issues, most notably hardware interrupt counts.") \
|
||||
M(PerfCacheReferences, "Cache accesses. Usually, this indicates Last Level Cache accesses, but this may vary depending on your CPU. This may include prefetches and coherency messages; again this depends on the design of your CPU.") \
|
||||
|
@ -365,7 +365,7 @@ DECLARE_AVX512VBMI2_SPECIFIC_CODE(
|
||||
FUNCTION_HEADER \
|
||||
\
|
||||
name \
|
||||
FUNCTION_BODY \
|
||||
FUNCTION_BODY \
|
||||
|
||||
/// NOLINTNEXTLINE
|
||||
#define MULTITARGET_FUNCTION_AVX512BW_AVX512F_AVX2_SSE42(FUNCTION_HEADER, name, FUNCTION_BODY) \
|
||||
|
@ -1,18 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <base/defines.h>
|
||||
#include <base/types.h>
|
||||
#include <Common/Concepts.h>
|
||||
#include <Common/TargetSpecific.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <optional>
|
||||
#include <Common/findExtreme.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
template <typename T>
|
||||
concept is_any_native_number = (is_any_of<T, Int8, Int16, Int32, Int64, UInt8, UInt16, UInt32, UInt64, Float32, Float64>);
|
||||
|
||||
template <is_any_native_number T>
|
||||
struct MinComparator
|
||||
@ -28,8 +19,8 @@ struct MaxComparator
|
||||
|
||||
MULTITARGET_FUNCTION_AVX2_SSE42(
|
||||
MULTITARGET_FUNCTION_HEADER(template <is_any_native_number T, typename ComparatorClass, bool add_all_elements, bool add_if_cond_zero> static std::optional<T> NO_INLINE),
|
||||
findNumericExtremeImpl,
|
||||
MULTITARGET_FUNCTION_BODY((const T * __restrict ptr, const UInt8 * __restrict condition_map [[maybe_unused]], size_t row_begin, size_t row_end)
|
||||
findExtremeImpl,
|
||||
MULTITARGET_FUNCTION_BODY((const T * __restrict ptr, const UInt8 * __restrict condition_map [[maybe_unused]], size_t row_begin, size_t row_end) /// NOLINT
|
||||
{
|
||||
size_t count = row_end - row_begin;
|
||||
ptr += row_begin;
|
||||
@ -86,69 +77,67 @@ MULTITARGET_FUNCTION_AVX2_SSE42(
|
||||
}
|
||||
))
|
||||
|
||||
|
||||
/// Given a vector of T finds the extreme (MIN or MAX) value
|
||||
template <is_any_native_number T, class ComparatorClass, bool add_all_elements, bool add_if_cond_zero>
|
||||
static std::optional<T>
|
||||
findNumericExtreme(const T * __restrict ptr, const UInt8 * __restrict condition_map [[maybe_unused]], size_t start, size_t end)
|
||||
findExtreme(const T * __restrict ptr, const UInt8 * __restrict condition_map [[maybe_unused]], size_t start, size_t end)
|
||||
{
|
||||
#if USE_MULTITARGET_CODE
|
||||
/// We see no benefit from using AVX512BW or AVX512F (over AVX2), so we only declare SSE and AVX2
|
||||
if (isArchSupported(TargetArch::AVX2))
|
||||
return findNumericExtremeImplAVX2<T, ComparatorClass, add_all_elements, add_if_cond_zero>(ptr, condition_map, start, end);
|
||||
return findExtremeImplAVX2<T, ComparatorClass, add_all_elements, add_if_cond_zero>(ptr, condition_map, start, end);
|
||||
|
||||
if (isArchSupported(TargetArch::SSE42))
|
||||
return findNumericExtremeImplSSE42<T, ComparatorClass, add_all_elements, add_if_cond_zero>(ptr, condition_map, start, end);
|
||||
return findExtremeImplSSE42<T, ComparatorClass, add_all_elements, add_if_cond_zero>(ptr, condition_map, start, end);
|
||||
#endif
|
||||
return findNumericExtremeImpl<T, ComparatorClass, add_all_elements, add_if_cond_zero>(ptr, condition_map, start, end);
|
||||
return findExtremeImpl<T, ComparatorClass, add_all_elements, add_if_cond_zero>(ptr, condition_map, start, end);
|
||||
}
|
||||
|
||||
template <is_any_native_number T>
|
||||
std::optional<T> findNumericMin(const T * __restrict ptr, size_t start, size_t end)
|
||||
std::optional<T> findExtremeMin(const T * __restrict ptr, size_t start, size_t end)
|
||||
{
|
||||
return findNumericExtreme<T, MinComparator<T>, true, false>(ptr, nullptr, start, end);
|
||||
return findExtreme<T, MinComparator<T>, true, false>(ptr, nullptr, start, end);
|
||||
}
|
||||
|
||||
template <is_any_native_number T>
|
||||
std::optional<T> findNumericMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end)
|
||||
std::optional<T> findExtremeMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end)
|
||||
{
|
||||
return findNumericExtreme<T, MinComparator<T>, false, true>(ptr, condition_map, start, end);
|
||||
return findExtreme<T, MinComparator<T>, false, true>(ptr, condition_map, start, end);
|
||||
}
|
||||
|
||||
template <is_any_native_number T>
|
||||
std::optional<T> findNumericMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end)
|
||||
std::optional<T> findExtremeMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end)
|
||||
{
|
||||
return findNumericExtreme<T, MinComparator<T>, false, false>(ptr, condition_map, start, end);
|
||||
return findExtreme<T, MinComparator<T>, false, false>(ptr, condition_map, start, end);
|
||||
}
|
||||
|
||||
template <is_any_native_number T>
|
||||
std::optional<T> findNumericMax(const T * __restrict ptr, size_t start, size_t end)
|
||||
std::optional<T> findExtremeMax(const T * __restrict ptr, size_t start, size_t end)
|
||||
{
|
||||
return findNumericExtreme<T, MaxComparator<T>, true, false>(ptr, nullptr, start, end);
|
||||
return findExtreme<T, MaxComparator<T>, true, false>(ptr, nullptr, start, end);
|
||||
}
|
||||
|
||||
template <is_any_native_number T>
|
||||
std::optional<T> findNumericMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end)
|
||||
std::optional<T> findExtremeMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end)
|
||||
{
|
||||
return findNumericExtreme<T, MaxComparator<T>, false, true>(ptr, condition_map, start, end);
|
||||
return findExtreme<T, MaxComparator<T>, false, true>(ptr, condition_map, start, end);
|
||||
}
|
||||
|
||||
template <is_any_native_number T>
|
||||
std::optional<T> findNumericMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end)
|
||||
std::optional<T> findExtremeMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end)
|
||||
{
|
||||
return findNumericExtreme<T, MaxComparator<T>, false, false>(ptr, condition_map, start, end);
|
||||
return findExtreme<T, MaxComparator<T>, false, false>(ptr, condition_map, start, end);
|
||||
}
|
||||
|
||||
|
||||
#define EXTERN_INSTANTIATION(T) \
|
||||
extern template std::optional<T> findNumericMin(const T * __restrict ptr, size_t start, size_t end); \
|
||||
extern template std::optional<T> findNumericMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \
|
||||
extern template std::optional<T> findNumericMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \
|
||||
extern template std::optional<T> findNumericMax(const T * __restrict ptr, size_t start, size_t end); \
|
||||
extern template std::optional<T> findNumericMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \
|
||||
extern template std::optional<T> findNumericMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end);
|
||||
|
||||
FOR_BASIC_NUMERIC_TYPES(EXTERN_INSTANTIATION)
|
||||
#undef EXTERN_INSTANTIATION
|
||||
#define INSTANTIATION(T) \
|
||||
template std::optional<T> findExtremeMin(const T * __restrict ptr, size_t start, size_t end); \
|
||||
template std::optional<T> findExtremeMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \
|
||||
template std::optional<T> findExtremeMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \
|
||||
template std::optional<T> findExtremeMax(const T * __restrict ptr, size_t start, size_t end); \
|
||||
template std::optional<T> findExtremeMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \
|
||||
template std::optional<T> findExtremeMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end);
|
||||
|
||||
FOR_BASIC_NUMERIC_TYPES(INSTANTIATION)
|
||||
#undef INSTANTIATION
|
||||
}
|
45
src/Common/findExtreme.h
Normal file
45
src/Common/findExtreme.h
Normal file
@ -0,0 +1,45 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <base/defines.h>
|
||||
#include <base/types.h>
|
||||
#include <Common/Concepts.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <optional>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
template <typename T>
|
||||
concept is_any_native_number = (is_any_of<T, Int8, Int16, Int32, Int64, UInt8, UInt16, UInt32, UInt64, Float32, Float64>);
|
||||
|
||||
template <is_any_native_number T>
|
||||
std::optional<T> findExtremeMin(const T * __restrict ptr, size_t start, size_t end);
|
||||
|
||||
template <is_any_native_number T>
|
||||
std::optional<T> findExtremeMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end);
|
||||
|
||||
template <is_any_native_number T>
|
||||
std::optional<T> findExtremeMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end);
|
||||
|
||||
template <is_any_native_number T>
|
||||
std::optional<T> findExtremeMax(const T * __restrict ptr, size_t start, size_t end);
|
||||
|
||||
template <is_any_native_number T>
|
||||
std::optional<T> findExtremeMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end);
|
||||
|
||||
template <is_any_native_number T>
|
||||
std::optional<T> findExtremeMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end);
|
||||
|
||||
#define EXTERN_INSTANTIATION(T) \
|
||||
extern template std::optional<T> findExtremeMin(const T * __restrict ptr, size_t start, size_t end); \
|
||||
extern template std::optional<T> findExtremeMinNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \
|
||||
extern template std::optional<T> findExtremeMinIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \
|
||||
extern template std::optional<T> findExtremeMax(const T * __restrict ptr, size_t start, size_t end); \
|
||||
extern template std::optional<T> findExtremeMaxNotNull(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end); \
|
||||
extern template std::optional<T> findExtremeMaxIf(const T * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end);
|
||||
|
||||
FOR_BASIC_NUMERIC_TYPES(EXTERN_INSTANTIATION)
|
||||
#undef EXTERN_INSTANTIATION
|
||||
|
||||
}
|
36
src/Common/iota.cpp
Normal file
36
src/Common/iota.cpp
Normal file
@ -0,0 +1,36 @@
|
||||
#include <base/defines.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/TargetSpecific.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
MULTITARGET_FUNCTION_AVX2_SSE42(
|
||||
MULTITARGET_FUNCTION_HEADER(template <iota_supported_types T> void NO_INLINE),
|
||||
iotaImpl, MULTITARGET_FUNCTION_BODY((T * begin, size_t count, T first_value) /// NOLINT
|
||||
{
|
||||
for (size_t i = 0; i < count; i++)
|
||||
*(begin + i) = static_cast<T>(first_value + i);
|
||||
})
|
||||
)
|
||||
|
||||
template <iota_supported_types T>
|
||||
void iota(T * begin, size_t count, T first_value)
|
||||
{
|
||||
#if USE_MULTITARGET_CODE
|
||||
if (isArchSupported(TargetArch::AVX2))
|
||||
return iotaImplAVX2(begin, count, first_value);
|
||||
|
||||
if (isArchSupported(TargetArch::SSE42))
|
||||
return iotaImplSSE42(begin, count, first_value);
|
||||
#endif
|
||||
return iotaImpl(begin, count, first_value);
|
||||
}
|
||||
|
||||
template void iota(UInt8 * begin, size_t count, UInt8 first_value);
|
||||
template void iota(UInt32 * begin, size_t count, UInt32 first_value);
|
||||
template void iota(UInt64 * begin, size_t count, UInt64 first_value);
|
||||
#if defined(OS_DARWIN)
|
||||
template void iota(size_t * begin, size_t count, size_t first_value);
|
||||
#endif
|
||||
}
|
34
src/Common/iota.h
Normal file
34
src/Common/iota.h
Normal file
@ -0,0 +1,34 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
#include <Common/Concepts.h>
|
||||
|
||||
/// This is a replacement for std::iota to use dynamic dispatch
|
||||
/// Note that is only defined for containers with contiguous memory only
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Make sure to add any new type to the extern declaration at the end of the file and instantiate it in iota.cpp
|
||||
|
||||
template <typename T>
|
||||
concept iota_supported_types = (is_any_of<
|
||||
T,
|
||||
UInt8,
|
||||
UInt32,
|
||||
UInt64
|
||||
#if defined(OS_DARWIN)
|
||||
,
|
||||
size_t
|
||||
#endif
|
||||
>);
|
||||
|
||||
template <iota_supported_types T> void iota(T * begin, size_t count, T first_value);
|
||||
|
||||
extern template void iota(UInt8 * begin, size_t count, UInt8 first_value);
|
||||
extern template void iota(UInt32 * begin, size_t count, UInt32 first_value);
|
||||
extern template void iota(UInt64 * begin, size_t count, UInt64 first_value);
|
||||
#if defined(OS_DARWIN)
|
||||
extern template void iota(size_t * begin, size_t count, size_t first_value);
|
||||
#endif
|
||||
}
|
@ -1,5 +1,6 @@
|
||||
#include <Common/levenshteinDistance.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/levenshteinDistance.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -11,8 +12,7 @@ size_t levenshteinDistance(const String & lhs, const String & rhs)
|
||||
|
||||
PODArrayWithStackMemory<size_t, 64> row(n + 1);
|
||||
|
||||
for (size_t i = 1; i <= n; ++i)
|
||||
row[i] = i;
|
||||
iota(row.data() + 1, n, size_t(1));
|
||||
|
||||
for (size_t j = 1; j <= m; ++j)
|
||||
{
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
@ -20,7 +21,7 @@ namespace
|
||||
std::vector<UInt64> getVectorWithNumbersUpToN(size_t n)
|
||||
{
|
||||
std::vector<UInt64> res(n);
|
||||
std::iota(res.begin(), res.end(), 0);
|
||||
iota(res.data(), res.size(), UInt64(0));
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -185,6 +185,7 @@ class IColumn;
|
||||
M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \
|
||||
M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \
|
||||
M(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, "Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas'", 0) \
|
||||
M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \
|
||||
\
|
||||
M(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards. Shard is marked as unavailable when: 1) The shard cannot be reached due to a connection failure. 2) Shard is unresolvable through DNS. 3) Table does not exist on the shard.", 0) \
|
||||
\
|
||||
@ -584,6 +585,7 @@ class IColumn;
|
||||
M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \
|
||||
M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \
|
||||
M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \
|
||||
M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped taraget table during pushing to views", 0) \
|
||||
M(Bool, allow_experimental_refreshable_materialized_view, false, "Allow refreshable materialized views (CREATE MATERIALIZED VIEW <name> REFRESH ...).", 0) \
|
||||
M(Bool, stop_refreshable_materialized_views_on_startup, false, "On server startup, prevent scheduling of refreshable materialized views, as if with SYSTEM STOP VIEWS. You can manually start them with SYSTEM START VIEWS or SYSTEM START VIEW <name> afterwards. Also applies to newly created views. Has no effect on non-refreshable materialized views.", 0) \
|
||||
M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \
|
||||
|
@ -115,6 +115,8 @@ IMPLEMENT_SETTING_ENUM(DistributedDDLOutputMode, ErrorCodes::BAD_ARGUMENTS,
|
||||
{{"none", DistributedDDLOutputMode::NONE},
|
||||
{"throw", DistributedDDLOutputMode::THROW},
|
||||
{"null_status_on_timeout", DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT},
|
||||
{"throw_only_active", DistributedDDLOutputMode::THROW_ONLY_ACTIVE},
|
||||
{"null_status_on_timeout_only_active", DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE},
|
||||
{"never_throw", DistributedDDLOutputMode::NEVER_THROW}})
|
||||
|
||||
IMPLEMENT_SETTING_ENUM(StreamingHandleErrorMode, ErrorCodes::BAD_ARGUMENTS,
|
||||
|
@ -173,6 +173,8 @@ enum class DistributedDDLOutputMode
|
||||
THROW,
|
||||
NULL_STATUS_ON_TIMEOUT,
|
||||
NEVER_THROW,
|
||||
THROW_ONLY_ACTIVE,
|
||||
NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE,
|
||||
};
|
||||
|
||||
DECLARE_SETTING_ENUM(DistributedDDLOutputMode)
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Dictionaries/IDictionary.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
@ -53,7 +54,7 @@ public:
|
||||
LOG_TRACE(dictionary.log, "Will load the dictionary using {} threads (with {} backlog)", shards, backlog);
|
||||
|
||||
shards_slots.resize(shards);
|
||||
std::iota(shards_slots.begin(), shards_slots.end(), 0);
|
||||
iota(shards_slots.data(), shards_slots.size(), UInt64(0));
|
||||
|
||||
for (size_t shard = 0; shard < shards; ++shard)
|
||||
{
|
||||
|
@ -5,6 +5,7 @@
|
||||
|
||||
#include <base/sort.h>
|
||||
|
||||
#include <Common/iota.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
@ -507,7 +508,7 @@ const IColumn * unrollSimplePolygons(const ColumnPtr & column, Offset & offset)
|
||||
if (!ptr_polygons)
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a column containing arrays of points");
|
||||
offset.ring_offsets.assign(ptr_polygons->getOffsets());
|
||||
std::iota(offset.polygon_offsets.begin(), offset.polygon_offsets.end(), 1);
|
||||
iota<IColumn::Offsets::value_type>(offset.polygon_offsets.data(), offset.polygon_offsets.size(), IColumn::Offsets::value_type(1));
|
||||
offset.multi_polygon_offsets.assign(offset.polygon_offsets);
|
||||
|
||||
return ptr_polygons->getDataPtr().get();
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Poco/Logger.h>
|
||||
|
||||
@ -184,7 +185,7 @@ public:
|
||||
{
|
||||
setBoundingBox();
|
||||
std::vector<size_t> order(polygons.size());
|
||||
std::iota(order.begin(), order.end(), 0);
|
||||
iota(order.data(), order.size(), size_t(0));
|
||||
root = makeCell(min_x, min_y, max_x, max_y, order);
|
||||
}
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Functions/FunctionsStringSimilarity.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/UTF8Helpers.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
#ifdef __SSE4_2__
|
||||
# include <nmmintrin.h>
|
||||
@ -246,8 +247,7 @@ struct ByteEditDistanceImpl
|
||||
ResultType insertion = 0;
|
||||
ResultType deletion = 0;
|
||||
|
||||
for (size_t i = 0; i <= haystack_size; ++i)
|
||||
distances0[i] = i;
|
||||
iota(distances0.data(), haystack_size + 1, ResultType(0));
|
||||
|
||||
for (size_t pos_needle = 0; pos_needle < needle_size; ++pos_needle)
|
||||
{
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
@ -80,7 +81,7 @@ public:
|
||||
const size_t cur_samples = std::min(num_elements, samples);
|
||||
|
||||
indices.resize(num_elements);
|
||||
std::iota(indices.begin(), indices.end(), prev_array_offset);
|
||||
iota(indices.data(), indices.size(), prev_array_offset);
|
||||
std::shuffle(indices.begin(), indices.end(), rng);
|
||||
|
||||
for (UInt64 i = 0; i < cur_samples; i++)
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/shuffle.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -150,7 +151,7 @@ ColumnPtr FunctionArrayShuffleImpl<Traits>::executeGeneric(const ColumnArray & a
|
||||
size_t size = offsets.size();
|
||||
size_t nested_size = array.getData().size();
|
||||
IColumn::Permutation permutation(nested_size);
|
||||
std::iota(std::begin(permutation), std::end(permutation), 0);
|
||||
iota(permutation.data(), permutation.size(), IColumn::Permutation::value_type(0));
|
||||
|
||||
ColumnArray::Offset current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Functions/array/arraySort.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/array/arraySort.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -55,9 +56,7 @@ ColumnPtr ArraySortImpl<positive, is_partial>::execute(
|
||||
size_t size = offsets.size();
|
||||
size_t nested_size = array.getData().size();
|
||||
IColumn::Permutation permutation(nested_size);
|
||||
|
||||
for (size_t i = 0; i < nested_size; ++i)
|
||||
permutation[i] = i;
|
||||
iota(permutation.data(), nested_size, IColumn::Permutation::value_type(0));
|
||||
|
||||
ColumnArray::Offset current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
|
@ -56,8 +56,7 @@ public:
|
||||
auto column = ColumnUInt64::create();
|
||||
auto & data = column->getData();
|
||||
data.resize(input_rows_count);
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
data[i] = i;
|
||||
iota(data.data(), input_rows_count, UInt64(0));
|
||||
|
||||
return column;
|
||||
}
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include "config.h"
|
||||
|
||||
#ifdef ENABLE_SQIDS
|
||||
#if USE_SQIDS
|
||||
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
@ -57,9 +57,10 @@ public:
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
size_t num_args = arguments.size();
|
||||
auto col_res = ColumnString::create();
|
||||
col_res->reserve(input_rows_count);
|
||||
|
||||
const size_t num_args = arguments.size();
|
||||
std::vector<UInt64> numbers(num_args);
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
@ -83,7 +84,7 @@ REGISTER_FUNCTION(Sqid)
|
||||
{
|
||||
factory.registerFunction<FunctionSqid>(FunctionDocumentation{
|
||||
.description=R"(
|
||||
Transforms numbers into YouTube-like short URL hash called [Sqid](https://sqids.org/).)",
|
||||
Transforms numbers into a [Sqid](https://sqids.org/) which is a Youtube-like ID string.)",
|
||||
.syntax="sqid(number1, ...)",
|
||||
.arguments={{"number1, ...", "Arbitrarily many UInt8, UInt16, UInt32 or UInt64 arguments"}},
|
||||
.returned_value="A hash id [String](/docs/en/sql-reference/data-types/string.md).",
|
@ -3,6 +3,7 @@
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/UTF8Helpers.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
@ -31,7 +32,7 @@ struct TranslateImpl
|
||||
if (map_from.size() != map_to.size())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second and third arguments must be the same length");
|
||||
|
||||
std::iota(map.begin(), map.end(), 0);
|
||||
iota(map.data(), map.size(), UInt8(0));
|
||||
|
||||
for (size_t i = 0; i < map_from.size(); ++i)
|
||||
{
|
||||
@ -129,7 +130,7 @@ struct TranslateUTF8Impl
|
||||
if (map_from_size != map_to_size)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second and third arguments must be the same length");
|
||||
|
||||
std::iota(map_ascii.begin(), map_ascii.end(), 0);
|
||||
iota(map_ascii.data(), map_ascii.size(), UInt32(0));
|
||||
|
||||
const UInt8 * map_from_ptr = reinterpret_cast<const UInt8 *>(map_from.data());
|
||||
const UInt8 * map_from_end = map_from_ptr + map_from.size();
|
||||
|
@ -412,7 +412,8 @@ void executeQueryWithParallelReplicas(
|
||||
new_cluster = not_optimized_cluster->getClusterWithReplicasAsShards(settings, settings.max_parallel_replicas);
|
||||
}
|
||||
|
||||
auto coordinator = std::make_shared<ParallelReplicasReadingCoordinator>(new_cluster->getShardCount());
|
||||
auto coordinator
|
||||
= std::make_shared<ParallelReplicasReadingCoordinator>(new_cluster->getShardCount(), settings.parallel_replicas_mark_segment_size);
|
||||
auto external_tables = new_context->getExternalTables();
|
||||
auto read_from_remote = std::make_unique<ReadFromParallelRemoteReplicasStep>(
|
||||
query_ast,
|
||||
|
@ -215,20 +215,47 @@ ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const Z
|
||||
}
|
||||
|
||||
|
||||
bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, const ZooKeeperPtr & zookeeper)
|
||||
bool DDLTask::findCurrentHostID(ContextPtr global_context, Poco::Logger * log, const ZooKeeperPtr & zookeeper, const std::optional<std::string> & config_host_name)
|
||||
{
|
||||
bool host_in_hostlist = false;
|
||||
std::exception_ptr first_exception = nullptr;
|
||||
|
||||
const auto maybe_secure_port = global_context->getTCPPortSecure();
|
||||
const auto port = global_context->getTCPPort();
|
||||
|
||||
if (config_host_name)
|
||||
{
|
||||
bool is_local_port = (maybe_secure_port && HostID(*config_host_name, *maybe_secure_port).isLocalAddress(*maybe_secure_port)) ||
|
||||
HostID(*config_host_name, port).isLocalAddress(port);
|
||||
|
||||
if (!is_local_port)
|
||||
throw Exception(
|
||||
ErrorCodes::DNS_ERROR,
|
||||
"{} is not a local address. Check parameter 'host_name' in the configuration",
|
||||
*config_host_name);
|
||||
}
|
||||
|
||||
for (const HostID & host : entry.hosts)
|
||||
{
|
||||
auto maybe_secure_port = global_context->getTCPPortSecure();
|
||||
if (config_host_name)
|
||||
{
|
||||
if (config_host_name != host.host_name)
|
||||
continue;
|
||||
|
||||
if (maybe_secure_port != host.port && port != host.port)
|
||||
continue;
|
||||
|
||||
host_in_hostlist = true;
|
||||
host_id = host;
|
||||
host_id_str = host.toString();
|
||||
break;
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
/// The port is considered local if it matches TCP or TCP secure port that the server is listening.
|
||||
bool is_local_port
|
||||
= (maybe_secure_port && host.isLocalAddress(*maybe_secure_port)) || host.isLocalAddress(global_context->getTCPPort());
|
||||
= (maybe_secure_port && host.isLocalAddress(*maybe_secure_port)) || host.isLocalAddress(port);
|
||||
|
||||
if (!is_local_port)
|
||||
continue;
|
||||
|
@ -44,6 +44,9 @@ struct HostID
|
||||
explicit HostID(const Cluster::Address & address)
|
||||
: host_name(address.host_name), port(address.port) {}
|
||||
|
||||
HostID(const String & host_name_, UInt16 port_)
|
||||
: host_name(host_name_), port(port_) {}
|
||||
|
||||
static HostID fromString(const String & host_port_str);
|
||||
|
||||
String toString() const
|
||||
@ -143,7 +146,7 @@ struct DDLTask : public DDLTaskBase
|
||||
{
|
||||
DDLTask(const String & name, const String & path) : DDLTaskBase(name, path) {}
|
||||
|
||||
bool findCurrentHostID(ContextPtr global_context, Poco::Logger * log, const ZooKeeperPtr & zookeeper);
|
||||
bool findCurrentHostID(ContextPtr global_context, Poco::Logger * log, const ZooKeeperPtr & zookeeper, const std::optional<std::string> & config_host_name);
|
||||
|
||||
void setClusterInfo(ContextPtr context, Poco::Logger * log);
|
||||
|
||||
|
@ -107,6 +107,9 @@ DDLWorker::DDLWorker(
|
||||
cleanup_delay_period = config->getUInt64(prefix + ".cleanup_delay_period", static_cast<UInt64>(cleanup_delay_period));
|
||||
max_tasks_in_queue = std::max<UInt64>(1, config->getUInt64(prefix + ".max_tasks_in_queue", max_tasks_in_queue));
|
||||
|
||||
if (config->has(prefix + ".host_name"))
|
||||
config_host_name = config->getString(prefix + ".host_name");
|
||||
|
||||
if (config->has(prefix + ".profile"))
|
||||
context->setSetting("profile", config->getString(prefix + ".profile"));
|
||||
}
|
||||
@ -214,7 +217,7 @@ DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_r
|
||||
/// Stage 2: resolve host_id and check if we should execute query or not
|
||||
/// Multiple clusters can use single DDL queue path in ZooKeeper,
|
||||
/// So we should skip task if we cannot find current host in cluster hosts list.
|
||||
if (!task->findCurrentHostID(context, log, zookeeper))
|
||||
if (!task->findCurrentHostID(context, log, zookeeper, config_host_name))
|
||||
{
|
||||
out_reason = "There is no a local address in host list";
|
||||
return add_to_skip_set();
|
||||
|
@ -153,6 +153,8 @@ protected:
|
||||
ContextMutablePtr context;
|
||||
Poco::Logger * log;
|
||||
|
||||
std::optional<std::string> config_host_name; /// host_name from config
|
||||
|
||||
std::string host_fqdn; /// current host domain name
|
||||
std::string host_fqdn_id; /// host_name:port
|
||||
std::string queue_dir; /// dir with queue of queries
|
||||
|
@ -2501,7 +2501,12 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
|
||||
max_block_size = std::max<UInt64>(1, max_block_limited);
|
||||
max_threads_execute_query = max_streams = 1;
|
||||
}
|
||||
if (max_block_limited < local_limits.local_limits.size_limits.max_rows)
|
||||
if (local_limits.local_limits.size_limits.max_rows != 0)
|
||||
{
|
||||
if (max_block_limited < local_limits.local_limits.size_limits.max_rows)
|
||||
query_info.limit = max_block_limited;
|
||||
}
|
||||
else
|
||||
{
|
||||
query_info.limit = max_block_limited;
|
||||
}
|
||||
|
@ -200,8 +200,6 @@ public:
|
||||
Status prepare() override;
|
||||
|
||||
private:
|
||||
static Strings getChildrenAllowNoNode(const std::shared_ptr<zkutil::ZooKeeper> & zookeeper, const String & node_path);
|
||||
|
||||
static Block getSampleBlock(ContextPtr context_, bool hosts_to_wait);
|
||||
|
||||
Strings getNewAndUpdate(const Strings & current_list_of_finished_hosts);
|
||||
@ -228,7 +226,8 @@ private:
|
||||
NameSet waiting_hosts; /// hosts from task host list
|
||||
NameSet finished_hosts; /// finished hosts from host list
|
||||
NameSet ignoring_hosts; /// appeared hosts that are not in hosts list
|
||||
Strings current_active_hosts; /// Hosts that were in active state at the last check
|
||||
Strings current_active_hosts; /// Hosts that are currently executing the task
|
||||
NameSet offline_hosts; /// Hosts that are not currently running
|
||||
size_t num_hosts_finished = 0;
|
||||
|
||||
/// Save the first detected error and throw it at the end of execution
|
||||
@ -237,7 +236,10 @@ private:
|
||||
Int64 timeout_seconds = 120;
|
||||
bool is_replicated_database = false;
|
||||
bool throw_on_timeout = true;
|
||||
bool only_running_hosts = false;
|
||||
|
||||
bool timeout_exceeded = false;
|
||||
bool stop_waiting_offline_hosts = false;
|
||||
};
|
||||
|
||||
|
||||
@ -310,12 +312,15 @@ DDLQueryStatusSource::DDLQueryStatusSource(
|
||||
, log(&Poco::Logger::get("DDLQueryStatusSource"))
|
||||
{
|
||||
auto output_mode = context->getSettingsRef().distributed_ddl_output_mode;
|
||||
throw_on_timeout = output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::NONE;
|
||||
throw_on_timeout = output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE
|
||||
|| output_mode == DistributedDDLOutputMode::NONE;
|
||||
|
||||
if (hosts_to_wait)
|
||||
{
|
||||
waiting_hosts = NameSet(hosts_to_wait->begin(), hosts_to_wait->end());
|
||||
is_replicated_database = true;
|
||||
only_running_hosts = output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE ||
|
||||
output_mode == DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE;
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -377,6 +382,38 @@ Chunk DDLQueryStatusSource::generateChunkWithUnfinishedHosts() const
|
||||
return Chunk(std::move(columns), unfinished_hosts.size());
|
||||
}
|
||||
|
||||
static NameSet getOfflineHosts(const String & node_path, const NameSet & hosts_to_wait, const ZooKeeperPtr & zookeeper, Poco::Logger * log)
|
||||
{
|
||||
fs::path replicas_path;
|
||||
if (node_path.ends_with('/'))
|
||||
replicas_path = fs::path(node_path).parent_path().parent_path().parent_path() / "replicas";
|
||||
else
|
||||
replicas_path = fs::path(node_path).parent_path().parent_path() / "replicas";
|
||||
|
||||
Strings paths;
|
||||
Strings hosts_array;
|
||||
for (const auto & host : hosts_to_wait)
|
||||
{
|
||||
hosts_array.push_back(host);
|
||||
paths.push_back(replicas_path / host / "active");
|
||||
}
|
||||
|
||||
NameSet offline;
|
||||
auto res = zookeeper->tryGet(paths);
|
||||
for (size_t i = 0; i < res.size(); ++i)
|
||||
if (res[i].error == Coordination::Error::ZNONODE)
|
||||
offline.insert(hosts_array[i]);
|
||||
|
||||
if (offline.size() == hosts_to_wait.size())
|
||||
{
|
||||
/// Avoid reporting that all hosts are offline
|
||||
LOG_WARNING(log, "Did not find active hosts, will wait for all {} hosts. This should not happen often", offline.size());
|
||||
return {};
|
||||
}
|
||||
|
||||
return offline;
|
||||
}
|
||||
|
||||
Chunk DDLQueryStatusSource::generate()
|
||||
{
|
||||
bool all_hosts_finished = num_hosts_finished >= waiting_hosts.size();
|
||||
@ -398,7 +435,7 @@ Chunk DDLQueryStatusSource::generate()
|
||||
if (isCancelled())
|
||||
return {};
|
||||
|
||||
if (timeout_seconds >= 0 && watch.elapsedSeconds() > timeout_seconds)
|
||||
if (stop_waiting_offline_hosts || (timeout_seconds >= 0 && watch.elapsedSeconds() > timeout_seconds))
|
||||
{
|
||||
timeout_exceeded = true;
|
||||
|
||||
@ -406,7 +443,7 @@ Chunk DDLQueryStatusSource::generate()
|
||||
size_t num_active_hosts = current_active_hosts.size();
|
||||
|
||||
constexpr auto msg_format = "Watching task {} is executing longer than distributed_ddl_task_timeout (={}) seconds. "
|
||||
"There are {} unfinished hosts ({} of them are currently active), "
|
||||
"There are {} unfinished hosts ({} of them are currently executing the task), "
|
||||
"they are going to execute the query in background";
|
||||
if (throw_on_timeout)
|
||||
{
|
||||
@ -425,10 +462,7 @@ Chunk DDLQueryStatusSource::generate()
|
||||
return generateChunkWithUnfinishedHosts();
|
||||
}
|
||||
|
||||
if (num_hosts_finished != 0 || try_number != 0)
|
||||
{
|
||||
sleepForMilliseconds(std::min<size_t>(1000, 50 * (try_number + 1)));
|
||||
}
|
||||
sleepForMilliseconds(std::min<size_t>(1000, 50 * try_number));
|
||||
|
||||
bool node_exists = false;
|
||||
Strings tmp_hosts;
|
||||
@ -440,9 +474,21 @@ Chunk DDLQueryStatusSource::generate()
|
||||
retries_ctl.retryLoop([&]()
|
||||
{
|
||||
auto zookeeper = context->getZooKeeper();
|
||||
node_exists = zookeeper->exists(node_path);
|
||||
tmp_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / node_to_wait);
|
||||
tmp_active_hosts = getChildrenAllowNoNode(zookeeper, fs::path(node_path) / "active");
|
||||
Strings paths = {String(fs::path(node_path) / node_to_wait), String(fs::path(node_path) / "active")};
|
||||
auto res = zookeeper->tryGetChildren(paths);
|
||||
for (size_t i = 0; i < res.size(); ++i)
|
||||
if (res[i].error != Coordination::Error::ZOK && res[i].error != Coordination::Error::ZNONODE)
|
||||
throw Coordination::Exception::fromPath(res[i].error, paths[i]);
|
||||
|
||||
if (res[0].error == Coordination::Error::ZNONODE)
|
||||
node_exists = zookeeper->exists(node_path);
|
||||
else
|
||||
node_exists = true;
|
||||
tmp_hosts = res[0].names;
|
||||
tmp_active_hosts = res[1].names;
|
||||
|
||||
if (only_running_hosts)
|
||||
offline_hosts = getOfflineHosts(node_path, waiting_hosts, zookeeper, log);
|
||||
});
|
||||
}
|
||||
|
||||
@ -460,6 +506,17 @@ Chunk DDLQueryStatusSource::generate()
|
||||
|
||||
Strings new_hosts = getNewAndUpdate(tmp_hosts);
|
||||
++try_number;
|
||||
|
||||
if (only_running_hosts)
|
||||
{
|
||||
size_t num_finished_or_offline = 0;
|
||||
for (const auto & host : waiting_hosts)
|
||||
num_finished_or_offline += finished_hosts.contains(host) || offline_hosts.contains(host);
|
||||
|
||||
if (num_finished_or_offline == waiting_hosts.size())
|
||||
stop_waiting_offline_hosts = true;
|
||||
}
|
||||
|
||||
if (new_hosts.empty())
|
||||
continue;
|
||||
|
||||
@ -470,7 +527,13 @@ Chunk DDLQueryStatusSource::generate()
|
||||
{
|
||||
ExecutionStatus status(-1, "Cannot obtain error message");
|
||||
|
||||
if (node_to_wait == "finished")
|
||||
/// Replicated database retries in case of error, it should not write error status.
|
||||
#ifdef ABORT_ON_LOGICAL_ERROR
|
||||
bool need_check_status = true;
|
||||
#else
|
||||
bool need_check_status = !is_replicated_database;
|
||||
#endif
|
||||
if (need_check_status)
|
||||
{
|
||||
String status_data;
|
||||
bool finished_exists = false;
|
||||
@ -496,7 +559,6 @@ Chunk DDLQueryStatusSource::generate()
|
||||
if (status.code != 0 && !first_exception
|
||||
&& context->getSettingsRef().distributed_ddl_output_mode != DistributedDDLOutputMode::NEVER_THROW)
|
||||
{
|
||||
/// Replicated database retries in case of error, it should not write error status.
|
||||
if (is_replicated_database)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "There was an error on {}: {} (probably it's a bug)", host_id, status.message);
|
||||
|
||||
@ -555,15 +617,6 @@ IProcessor::Status DDLQueryStatusSource::prepare()
|
||||
return ISource::prepare();
|
||||
}
|
||||
|
||||
Strings DDLQueryStatusSource::getChildrenAllowNoNode(const std::shared_ptr<zkutil::ZooKeeper> & zookeeper, const String & node_path)
|
||||
{
|
||||
Strings res;
|
||||
Coordination::Error code = zookeeper->tryGetChildren(node_path, res);
|
||||
if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE)
|
||||
throw Coordination::Exception::fromPath(code, node_path);
|
||||
return res;
|
||||
}
|
||||
|
||||
Strings DDLQueryStatusSource::getNewAndUpdate(const Strings & current_list_of_finished_hosts)
|
||||
{
|
||||
Strings diff;
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
#ifdef __SSE2__
|
||||
#include <emmintrin.h>
|
||||
@ -155,8 +156,7 @@ void getBlockSortPermutationImpl(const Block & block, const SortDescription & de
|
||||
{
|
||||
size_t size = block.rows();
|
||||
permutation.resize(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
permutation[i] = i;
|
||||
iota(permutation.data(), size, IColumn::Permutation::value_type(0));
|
||||
|
||||
if (limit >= size)
|
||||
limit = 0;
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <memory>
|
||||
#include <thread>
|
||||
|
||||
#include <Common/iota.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
@ -788,7 +789,7 @@ TEST_F(FileCacheTest, writeBuffer)
|
||||
|
||||
/// get random permutation of indexes
|
||||
std::vector<size_t> indexes(data.size());
|
||||
std::iota(indexes.begin(), indexes.end(), 0);
|
||||
iota(indexes.data(), indexes.size(), size_t(0));
|
||||
std::shuffle(indexes.begin(), indexes.end(), rng);
|
||||
|
||||
for (auto i : indexes)
|
||||
|
@ -645,7 +645,12 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
max_threads_execute_query = 1;
|
||||
}
|
||||
|
||||
if (max_block_size_limited < select_query_info.local_storage_limits.local_limits.size_limits.max_rows)
|
||||
if (select_query_info.local_storage_limits.local_limits.size_limits.max_rows != 0)
|
||||
{
|
||||
if (max_block_size_limited < select_query_info.local_storage_limits.local_limits.size_limits.max_rows)
|
||||
table_expression_query_info.limit = max_block_size_limited;
|
||||
}
|
||||
else
|
||||
{
|
||||
table_expression_query_info.limit = max_block_size_limited;
|
||||
}
|
||||
|
@ -127,7 +127,12 @@ struct Pattern
|
||||
{
|
||||
hash.update(rule_type);
|
||||
hash.update(regexp_str);
|
||||
hash.update(function->getName());
|
||||
if (function)
|
||||
{
|
||||
hash.update(function->getName());
|
||||
for (const auto & p : function->getParameters())
|
||||
hash.update(toString(p));
|
||||
}
|
||||
for (const auto & r : retentions)
|
||||
{
|
||||
hash.update(r.age);
|
||||
|
@ -418,7 +418,13 @@ Pipe ReadFromMergeTree::readFromPool(
|
||||
&& settings.allow_prefetched_read_pool_for_local_filesystem
|
||||
&& MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.local_fs_method);
|
||||
|
||||
if (allow_prefetched_remote || allow_prefetched_local)
|
||||
/** Do not use prefetched read pool if query is trivial limit query.
|
||||
* Because time spend during filling per thread tasks can be greater than whole query
|
||||
* execution for big tables with small limit.
|
||||
*/
|
||||
bool use_prefetched_read_pool = query_info.limit == 0 && (allow_prefetched_remote || allow_prefetched_local);
|
||||
|
||||
if (use_prefetched_read_pool)
|
||||
{
|
||||
pool = std::make_shared<MergeTreePrefetchedReadPool>(
|
||||
std::move(parts_with_range),
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Storages/MergeTree/KeyCondition.h>
|
||||
#include <Storages/System/StorageSystemNumbers.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
namespace DB
|
||||
@ -40,11 +41,10 @@ protected:
|
||||
auto column = ColumnUInt64::create(block_size);
|
||||
ColumnUInt64::Container & vec = column->getData();
|
||||
|
||||
size_t curr = next; /// The local variable for some reason works faster (>20%) than member of class.
|
||||
UInt64 curr = next; /// The local variable for some reason works faster (>20%) than member of class.
|
||||
UInt64 * pos = vec.data(); /// This also accelerates the code.
|
||||
UInt64 * end = &vec[block_size];
|
||||
while (pos < end)
|
||||
*pos++ = curr++;
|
||||
iota(pos, static_cast<size_t>(end - pos), curr);
|
||||
|
||||
next += step;
|
||||
|
||||
@ -211,17 +211,18 @@ protected:
|
||||
{
|
||||
auto start_value_64 = static_cast<UInt64>(start_value);
|
||||
auto end_value_64 = static_cast<UInt64>(end_value);
|
||||
while (start_value_64 < end_value_64)
|
||||
*(pos++) = start_value_64++;
|
||||
auto size = end_value_64 - start_value_64;
|
||||
iota(pos, static_cast<size_t>(size), start_value_64);
|
||||
pos += size;
|
||||
}
|
||||
};
|
||||
|
||||
if (can_provide > need)
|
||||
{
|
||||
UInt64 start_value = first_value(range) + cursor.offset_in_range;
|
||||
UInt64 end_value = start_value + need; /// end_value will never overflow
|
||||
while (start_value < end_value)
|
||||
*(pos++) = start_value++;
|
||||
/// end_value will never overflow
|
||||
iota(pos, static_cast<size_t>(need), start_value);
|
||||
pos += need;
|
||||
|
||||
provided += need;
|
||||
cursor.offset_in_range += need;
|
||||
|
@ -18,31 +18,25 @@ protected:
|
||||
|
||||
void setKeyConditionImpl(const SelectQueryInfo & query_info, ContextPtr context, const Block & keys)
|
||||
{
|
||||
if (!context->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
key_condition = std::make_shared<const KeyCondition>(
|
||||
query_info,
|
||||
context,
|
||||
keys.getNames(),
|
||||
std::make_shared<ExpressionActions>(std::make_shared<ActionsDAG>(keys.getColumnsWithTypeAndName())));
|
||||
}
|
||||
key_condition = std::make_shared<const KeyCondition>(
|
||||
query_info,
|
||||
context,
|
||||
keys.getNames(),
|
||||
std::make_shared<ExpressionActions>(std::make_shared<ActionsDAG>(keys.getColumnsWithTypeAndName())));
|
||||
}
|
||||
|
||||
void setKeyConditionImpl(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context, const Block & keys)
|
||||
{
|
||||
if (context->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
std::unordered_map<std::string, DB::ColumnWithTypeAndName> node_name_to_input_column;
|
||||
for (const auto & column : keys.getColumnsWithTypeAndName())
|
||||
node_name_to_input_column.insert({column.name, column});
|
||||
std::unordered_map<std::string, DB::ColumnWithTypeAndName> node_name_to_input_column;
|
||||
for (const auto & column : keys.getColumnsWithTypeAndName())
|
||||
node_name_to_input_column.insert({column.name, column});
|
||||
|
||||
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_column, context);
|
||||
key_condition = std::make_shared<const KeyCondition>(
|
||||
filter_actions_dag,
|
||||
context,
|
||||
keys.getNames(),
|
||||
std::make_shared<ExpressionActions>(std::make_shared<ActionsDAG>(keys.getColumnsWithTypeAndName())));
|
||||
}
|
||||
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_column, context);
|
||||
key_condition = std::make_shared<const KeyCondition>(
|
||||
filter_actions_dag,
|
||||
context,
|
||||
keys.getNames(),
|
||||
std::make_shared<ExpressionActions>(std::make_shared<ActionsDAG>(keys.getColumnsWithTypeAndName())));
|
||||
}
|
||||
|
||||
public:
|
||||
|
@ -1,7 +1,8 @@
|
||||
#include <Processors/Transforms/PartialSortingTransform.h>
|
||||
#include <Interpreters/sortBlock.h>
|
||||
#include <Core/SortCursor.h>
|
||||
#include <Interpreters/sortBlock.h>
|
||||
#include <Processors/Transforms/PartialSortingTransform.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -36,9 +37,7 @@ size_t getFilterMask(const ColumnRawPtrs & raw_block_columns, const Columns & th
|
||||
else
|
||||
{
|
||||
rows_to_compare.resize(num_rows);
|
||||
|
||||
for (size_t i = 0; i < num_rows; ++i)
|
||||
rows_to_compare[i] = i;
|
||||
iota(rows_to_compare.data(), num_rows, UInt64(0));
|
||||
|
||||
size_t size = description.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
|
@ -39,6 +39,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
}
|
||||
|
||||
ThreadStatusesHolder::~ThreadStatusesHolder()
|
||||
@ -316,7 +317,21 @@ Chain buildPushingToViewsChain(
|
||||
type = QueryViewsLogElement::ViewType::MATERIALIZED;
|
||||
result_chain.addTableLock(lock);
|
||||
|
||||
StoragePtr inner_table = materialized_view->getTargetTable();
|
||||
StoragePtr inner_table = materialized_view->tryGetTargetTable();
|
||||
/// If target table was dropped, ignore this materialized view.
|
||||
if (!inner_table)
|
||||
{
|
||||
if (context->getSettingsRef().ignore_materialized_views_with_dropped_target_table)
|
||||
continue;
|
||||
|
||||
throw Exception(
|
||||
ErrorCodes::UNKNOWN_TABLE,
|
||||
"Target table '{}' of view '{}' doesn't exists. To ignore this view use setting "
|
||||
"ignore_materialized_views_with_dropped_target_table",
|
||||
materialized_view->getTargetTableId().getFullTableName(),
|
||||
view_id.getFullTableName());
|
||||
}
|
||||
|
||||
auto inner_table_id = inner_table->getStorageID();
|
||||
auto inner_metadata_snapshot = inner_table->getInMemoryMetadataPtr();
|
||||
|
||||
|
@ -1,14 +1,12 @@
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include "Core/UUID.h"
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/IJoin.h>
|
||||
#include <Interpreters/TableJoin.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Processors/ConcatProcessor.h>
|
||||
#include <Processors/DelayedPortsProcessor.h>
|
||||
#include <Processors/Executors/PipelineExecutor.h>
|
||||
@ -25,11 +23,14 @@
|
||||
#include <Processors/Transforms/ExtremesTransform.h>
|
||||
#include <Processors/Transforms/JoiningTransform.h>
|
||||
#include <Processors/Transforms/MergeJoinTransform.h>
|
||||
#include <Processors/Transforms/PasteJoinTransform.h>
|
||||
#include <Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h>
|
||||
#include <Processors/Transforms/PartialSortingTransform.h>
|
||||
#include <Processors/Transforms/PasteJoinTransform.h>
|
||||
#include <Processors/Transforms/TotalsHavingTransform.h>
|
||||
#include <QueryPipeline/narrowPipe.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -619,8 +620,7 @@ void QueryPipelineBuilder::addPipelineBefore(QueryPipelineBuilder pipeline)
|
||||
bool has_extremes = pipe.getExtremesPort();
|
||||
size_t num_extra_ports = (has_totals ? 1 : 0) + (has_extremes ? 1 : 0);
|
||||
IProcessor::PortNumbers delayed_streams(pipe.numOutputPorts() + num_extra_ports);
|
||||
for (size_t i = 0; i < delayed_streams.size(); ++i)
|
||||
delayed_streams[i] = i;
|
||||
iota(delayed_streams.data(), delayed_streams.size(), IProcessor::PortNumbers::value_type(0));
|
||||
|
||||
auto * collected_processors = pipe.collected_processors;
|
||||
|
||||
|
@ -15,6 +15,9 @@
|
||||
#include <Processors/Transforms/AddingDefaultsTransform.h>
|
||||
#include <Processors/Transforms/ExtractColumnsTransform.h>
|
||||
#include <Processors/Sources/ConstChunkGenerator.h>
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/SourceStepWithFilter.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/CompressionMethod.h>
|
||||
@ -408,22 +411,22 @@ ColumnsDescription StorageHDFS::getTableStructureFromData(
|
||||
class HDFSSource::DisclosedGlobIterator::Impl
|
||||
{
|
||||
public:
|
||||
Impl(const String & uri, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
Impl(const String & uri, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
{
|
||||
const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri);
|
||||
uris = getPathsList(path_from_uri, uri_without_path, context);
|
||||
ASTPtr filter_ast;
|
||||
ActionsDAGPtr filter_dag;
|
||||
if (!uris.empty())
|
||||
filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, uris[0].path, context);
|
||||
filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns);
|
||||
|
||||
if (filter_ast)
|
||||
if (filter_dag)
|
||||
{
|
||||
std::vector<String> paths;
|
||||
paths.reserve(uris.size());
|
||||
for (const auto & path_with_info : uris)
|
||||
paths.push_back(path_with_info.path);
|
||||
|
||||
VirtualColumnUtils::filterByPathOrFile(uris, paths, query, virtual_columns, context, filter_ast);
|
||||
VirtualColumnUtils::filterByPathOrFile(uris, paths, filter_dag, virtual_columns, context);
|
||||
}
|
||||
auto file_progress_callback = context->getFileProgressCallback();
|
||||
|
||||
@ -456,21 +459,21 @@ private:
|
||||
class HDFSSource::URISIterator::Impl : WithContext
|
||||
{
|
||||
public:
|
||||
explicit Impl(const std::vector<String> & uris_, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context_)
|
||||
explicit Impl(const std::vector<String> & uris_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context_)
|
||||
: WithContext(context_), uris(uris_), file_progress_callback(context_->getFileProgressCallback())
|
||||
{
|
||||
ASTPtr filter_ast;
|
||||
ActionsDAGPtr filter_dag;
|
||||
if (!uris.empty())
|
||||
filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, getPathFromUriAndUriWithoutPath(uris[0]).first, getContext());
|
||||
filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns);
|
||||
|
||||
if (filter_ast)
|
||||
if (filter_dag)
|
||||
{
|
||||
std::vector<String> paths;
|
||||
paths.reserve(uris.size());
|
||||
for (const auto & uri : uris)
|
||||
paths.push_back(getPathFromUriAndUriWithoutPath(uri).first);
|
||||
|
||||
VirtualColumnUtils::filterByPathOrFile(uris, paths, query, virtual_columns, getContext(), filter_ast);
|
||||
VirtualColumnUtils::filterByPathOrFile(uris, paths, filter_dag, virtual_columns, getContext());
|
||||
}
|
||||
|
||||
if (!uris.empty())
|
||||
@ -517,16 +520,16 @@ private:
|
||||
std::function<void(FileProgress)> file_progress_callback;
|
||||
};
|
||||
|
||||
HDFSSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
: pimpl(std::make_shared<HDFSSource::DisclosedGlobIterator::Impl>(uri, query, virtual_columns, context)) {}
|
||||
HDFSSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
: pimpl(std::make_shared<HDFSSource::DisclosedGlobIterator::Impl>(uri, predicate, virtual_columns, context)) {}
|
||||
|
||||
StorageHDFS::PathWithInfo HDFSSource::DisclosedGlobIterator::next()
|
||||
{
|
||||
return pimpl->next();
|
||||
}
|
||||
|
||||
HDFSSource::URISIterator::URISIterator(const std::vector<String> & uris_, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
: pimpl(std::make_shared<HDFSSource::URISIterator::Impl>(uris_, query, virtual_columns, context))
|
||||
HDFSSource::URISIterator::URISIterator(const std::vector<String> & uris_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
: pimpl(std::make_shared<HDFSSource::URISIterator::Impl>(uris_, predicate, virtual_columns, context))
|
||||
{
|
||||
}
|
||||
|
||||
@ -541,8 +544,7 @@ HDFSSource::HDFSSource(
|
||||
ContextPtr context_,
|
||||
UInt64 max_block_size_,
|
||||
std::shared_ptr<IteratorWrapper> file_iterator_,
|
||||
bool need_only_count_,
|
||||
const SelectQueryInfo & query_info_)
|
||||
bool need_only_count_)
|
||||
: ISource(info.source_header, false)
|
||||
, WithContext(context_)
|
||||
, storage(std::move(storage_))
|
||||
@ -553,7 +555,6 @@ HDFSSource::HDFSSource(
|
||||
, file_iterator(file_iterator_)
|
||||
, columns_description(info.columns_description)
|
||||
, need_only_count(need_only_count_)
|
||||
, query_info(query_info_)
|
||||
{
|
||||
initialize();
|
||||
}
|
||||
@ -843,7 +844,57 @@ bool StorageHDFS::supportsSubsetOfColumns(const ContextPtr & context_) const
|
||||
return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name, context_);
|
||||
}
|
||||
|
||||
Pipe StorageHDFS::read(
|
||||
class ReadFromHDFS : public SourceStepWithFilter
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "ReadFromHDFS"; }
|
||||
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
void applyFilters() override;
|
||||
|
||||
ReadFromHDFS(
|
||||
Block sample_block,
|
||||
ReadFromFormatInfo info_,
|
||||
bool need_only_count_,
|
||||
std::shared_ptr<StorageHDFS> storage_,
|
||||
ContextPtr context_,
|
||||
size_t max_block_size_,
|
||||
size_t num_streams_)
|
||||
: SourceStepWithFilter(DataStream{.header = std::move(sample_block)})
|
||||
, info(std::move(info_))
|
||||
, need_only_count(need_only_count_)
|
||||
, storage(std::move(storage_))
|
||||
, context(std::move(context_))
|
||||
, max_block_size(max_block_size_)
|
||||
, num_streams(num_streams_)
|
||||
{
|
||||
}
|
||||
|
||||
private:
|
||||
ReadFromFormatInfo info;
|
||||
const bool need_only_count;
|
||||
std::shared_ptr<StorageHDFS> storage;
|
||||
|
||||
ContextPtr context;
|
||||
size_t max_block_size;
|
||||
size_t num_streams;
|
||||
|
||||
std::shared_ptr<HDFSSource::IteratorWrapper> iterator_wrapper;
|
||||
|
||||
void createIterator(const ActionsDAG::Node * predicate);
|
||||
};
|
||||
|
||||
void ReadFromHDFS::applyFilters()
|
||||
{
|
||||
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context);
|
||||
const ActionsDAG::Node * predicate = nullptr;
|
||||
if (filter_actions_dag)
|
||||
predicate = filter_actions_dag->getOutputs().at(0);
|
||||
|
||||
createIterator(predicate);
|
||||
}
|
||||
|
||||
void StorageHDFS::read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
@ -852,18 +903,40 @@ Pipe StorageHDFS::read(
|
||||
size_t max_block_size,
|
||||
size_t num_streams)
|
||||
{
|
||||
std::shared_ptr<HDFSSource::IteratorWrapper> iterator_wrapper{nullptr};
|
||||
if (distributed_processing)
|
||||
auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context_), virtual_columns);
|
||||
bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty())
|
||||
&& context_->getSettingsRef().optimize_count_from_files;
|
||||
|
||||
auto this_ptr = std::static_pointer_cast<StorageHDFS>(shared_from_this());
|
||||
|
||||
auto reading = std::make_unique<ReadFromHDFS>(
|
||||
read_from_format_info.source_header,
|
||||
std::move(read_from_format_info),
|
||||
need_only_count,
|
||||
std::move(this_ptr),
|
||||
context_,
|
||||
max_block_size,
|
||||
num_streams);
|
||||
|
||||
query_plan.addStep(std::move(reading));
|
||||
}
|
||||
|
||||
void ReadFromHDFS::createIterator(const ActionsDAG::Node * predicate)
|
||||
{
|
||||
if (iterator_wrapper)
|
||||
return;
|
||||
|
||||
if (storage->distributed_processing)
|
||||
{
|
||||
iterator_wrapper = std::make_shared<HDFSSource::IteratorWrapper>(
|
||||
[callback = context_->getReadTaskCallback()]() -> StorageHDFS::PathWithInfo {
|
||||
[callback = context->getReadTaskCallback()]() -> StorageHDFS::PathWithInfo {
|
||||
return StorageHDFS::PathWithInfo{callback(), std::nullopt};
|
||||
});
|
||||
}
|
||||
else if (is_path_with_globs)
|
||||
else if (storage->is_path_with_globs)
|
||||
{
|
||||
/// Iterate through disclosed globs and make a source for each file
|
||||
auto glob_iterator = std::make_shared<HDFSSource::DisclosedGlobIterator>(uris[0], query_info.query, virtual_columns, context_);
|
||||
auto glob_iterator = std::make_shared<HDFSSource::DisclosedGlobIterator>(storage->uris[0], predicate, storage->virtual_columns, context);
|
||||
iterator_wrapper = std::make_shared<HDFSSource::IteratorWrapper>([glob_iterator]()
|
||||
{
|
||||
return glob_iterator->next();
|
||||
@ -871,31 +944,38 @@ Pipe StorageHDFS::read(
|
||||
}
|
||||
else
|
||||
{
|
||||
auto uris_iterator = std::make_shared<HDFSSource::URISIterator>(uris, query_info.query, virtual_columns, context_);
|
||||
auto uris_iterator = std::make_shared<HDFSSource::URISIterator>(storage->uris, predicate, storage->virtual_columns, context);
|
||||
iterator_wrapper = std::make_shared<HDFSSource::IteratorWrapper>([uris_iterator]()
|
||||
{
|
||||
return uris_iterator->next();
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context_), getVirtuals());
|
||||
bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty())
|
||||
&& context_->getSettingsRef().optimize_count_from_files;
|
||||
void ReadFromHDFS::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
createIterator(nullptr);
|
||||
|
||||
Pipes pipes;
|
||||
auto this_ptr = std::static_pointer_cast<StorageHDFS>(shared_from_this());
|
||||
for (size_t i = 0; i < num_streams; ++i)
|
||||
{
|
||||
pipes.emplace_back(std::make_shared<HDFSSource>(
|
||||
read_from_format_info,
|
||||
this_ptr,
|
||||
context_,
|
||||
info,
|
||||
storage,
|
||||
context,
|
||||
max_block_size,
|
||||
iterator_wrapper,
|
||||
need_only_count,
|
||||
query_info));
|
||||
need_only_count));
|
||||
}
|
||||
return Pipe::unitePipes(std::move(pipes));
|
||||
|
||||
auto pipe = Pipe::unitePipes(std::move(pipes));
|
||||
if (pipe.empty())
|
||||
pipe = Pipe(std::make_shared<NullSource>(info.source_header));
|
||||
|
||||
for (const auto & processor : pipe.getProcessors())
|
||||
processors.emplace_back(processor);
|
||||
|
||||
pipeline.init(std::move(pipe));
|
||||
}
|
||||
|
||||
SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context_, bool /*async_insert*/)
|
||||
|
@ -51,7 +51,8 @@ public:
|
||||
|
||||
String getName() const override { return "HDFS"; }
|
||||
|
||||
Pipe read(
|
||||
void read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
@ -93,6 +94,7 @@ public:
|
||||
|
||||
protected:
|
||||
friend class HDFSSource;
|
||||
friend class ReadFromHDFS;
|
||||
|
||||
private:
|
||||
std::vector<String> uris;
|
||||
@ -114,7 +116,7 @@ public:
|
||||
class DisclosedGlobIterator
|
||||
{
|
||||
public:
|
||||
DisclosedGlobIterator(const String & uri_, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context);
|
||||
DisclosedGlobIterator(const String & uri_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context);
|
||||
StorageHDFS::PathWithInfo next();
|
||||
private:
|
||||
class Impl;
|
||||
@ -125,7 +127,7 @@ public:
|
||||
class URISIterator
|
||||
{
|
||||
public:
|
||||
URISIterator(const std::vector<String> & uris_, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context);
|
||||
URISIterator(const std::vector<String> & uris_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context);
|
||||
StorageHDFS::PathWithInfo next();
|
||||
private:
|
||||
class Impl;
|
||||
@ -142,8 +144,7 @@ public:
|
||||
ContextPtr context_,
|
||||
UInt64 max_block_size_,
|
||||
std::shared_ptr<IteratorWrapper> file_iterator_,
|
||||
bool need_only_count_,
|
||||
const SelectQueryInfo & query_info_);
|
||||
bool need_only_count_);
|
||||
|
||||
String getName() const override;
|
||||
|
||||
@ -162,7 +163,6 @@ private:
|
||||
ColumnsDescription columns_description;
|
||||
bool need_only_count;
|
||||
size_t total_rows_in_file = 0;
|
||||
SelectQueryInfo query_info;
|
||||
|
||||
std::unique_ptr<ReadBuffer> read_buf;
|
||||
std::shared_ptr<IInputFormat> input_format;
|
||||
|
@ -79,9 +79,9 @@ void StorageHDFSCluster::addColumnsStructureToQuery(ASTPtr & query, const String
|
||||
}
|
||||
|
||||
|
||||
RemoteQueryExecutor::Extension StorageHDFSCluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const
|
||||
RemoteQueryExecutor::Extension StorageHDFSCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const
|
||||
{
|
||||
auto iterator = std::make_shared<HDFSSource::DisclosedGlobIterator>(uri, query, virtual_columns, context);
|
||||
auto iterator = std::make_shared<HDFSSource::DisclosedGlobIterator>(uri, predicate, virtual_columns, context);
|
||||
auto callback = std::make_shared<std::function<String()>>([iter = std::move(iterator)]() mutable -> String { return iter->next().path; });
|
||||
return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)};
|
||||
}
|
||||
|
@ -35,7 +35,7 @@ public:
|
||||
|
||||
NamesAndTypesList getVirtuals() const override;
|
||||
|
||||
RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override;
|
||||
RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override;
|
||||
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include "Storages/IStorageCluster.h"
|
||||
#include <Storages/IStorageCluster.h>
|
||||
|
||||
#include "Common/Exception.h"
|
||||
#include "Core/QueryProcessingStage.h"
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <IO/ConnectionTimeouts.h>
|
||||
#include <Interpreters/Context.h>
|
||||
@ -11,11 +11,14 @@
|
||||
#include <Interpreters/AddDefaultDatabaseVisitor.h>
|
||||
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Processors/Sources/RemoteSource.h>
|
||||
#include <Processors/QueryPlan/SourceStepWithFilter.h>
|
||||
#include <QueryPipeline/narrowPipe.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Processors/Sources/RemoteSource.h>
|
||||
#include <QueryPipeline/RemoteQueryExecutor.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/StorageDictionary.h>
|
||||
@ -38,9 +41,66 @@ IStorageCluster::IStorageCluster(
|
||||
{
|
||||
}
|
||||
|
||||
class ReadFromCluster : public SourceStepWithFilter
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "ReadFromCluster"; }
|
||||
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
void applyFilters() override;
|
||||
|
||||
ReadFromCluster(
|
||||
Block sample_block,
|
||||
std::shared_ptr<IStorageCluster> storage_,
|
||||
ASTPtr query_to_send_,
|
||||
QueryProcessingStage::Enum processed_stage_,
|
||||
ClusterPtr cluster_,
|
||||
Poco::Logger * log_,
|
||||
ContextPtr context_)
|
||||
: SourceStepWithFilter(DataStream{.header = std::move(sample_block)})
|
||||
, storage(std::move(storage_))
|
||||
, query_to_send(std::move(query_to_send_))
|
||||
, processed_stage(processed_stage_)
|
||||
, cluster(std::move(cluster_))
|
||||
, log(log_)
|
||||
, context(std::move(context_))
|
||||
{
|
||||
}
|
||||
|
||||
private:
|
||||
std::shared_ptr<IStorageCluster> storage;
|
||||
ASTPtr query_to_send;
|
||||
QueryProcessingStage::Enum processed_stage;
|
||||
ClusterPtr cluster;
|
||||
Poco::Logger * log;
|
||||
ContextPtr context;
|
||||
|
||||
std::optional<RemoteQueryExecutor::Extension> extension;
|
||||
|
||||
void createExtension(const ActionsDAG::Node * predicate);
|
||||
ContextPtr updateSettings(const Settings & settings);
|
||||
};
|
||||
|
||||
void ReadFromCluster::applyFilters()
|
||||
{
|
||||
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context);
|
||||
const ActionsDAG::Node * predicate = nullptr;
|
||||
if (filter_actions_dag)
|
||||
predicate = filter_actions_dag->getOutputs().at(0);
|
||||
|
||||
createExtension(predicate);
|
||||
}
|
||||
|
||||
void ReadFromCluster::createExtension(const ActionsDAG::Node * predicate)
|
||||
{
|
||||
if (extension)
|
||||
return;
|
||||
|
||||
extension = storage->getTaskIteratorExtension(predicate, context);
|
||||
}
|
||||
|
||||
/// The code executes on initiator
|
||||
Pipe IStorageCluster::read(
|
||||
void IStorageCluster::read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
@ -49,10 +109,10 @@ Pipe IStorageCluster::read(
|
||||
size_t /*max_block_size*/,
|
||||
size_t /*num_streams*/)
|
||||
{
|
||||
updateBeforeRead(context);
|
||||
storage_snapshot->check(column_names);
|
||||
|
||||
updateBeforeRead(context);
|
||||
auto cluster = getCluster(context);
|
||||
auto extension = getTaskIteratorExtension(query_info.query, context);
|
||||
|
||||
/// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*)
|
||||
|
||||
@ -70,12 +130,6 @@ Pipe IStorageCluster::read(
|
||||
query_to_send = interpreter.getQueryInfo().query->clone();
|
||||
}
|
||||
|
||||
const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{};
|
||||
|
||||
Pipes pipes;
|
||||
|
||||
const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState;
|
||||
|
||||
if (!structure_argument_was_provided)
|
||||
addColumnsStructureToQuery(query_to_send, storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(), context);
|
||||
|
||||
@ -89,7 +143,29 @@ Pipe IStorageCluster::read(
|
||||
/* only_replace_in_join_= */true);
|
||||
visitor.visit(query_to_send);
|
||||
|
||||
auto new_context = updateSettings(context, context->getSettingsRef());
|
||||
auto this_ptr = std::static_pointer_cast<IStorageCluster>(shared_from_this());
|
||||
|
||||
auto reading = std::make_unique<ReadFromCluster>(
|
||||
sample_block,
|
||||
std::move(this_ptr),
|
||||
std::move(query_to_send),
|
||||
processed_stage,
|
||||
cluster,
|
||||
log,
|
||||
context);
|
||||
|
||||
query_plan.addStep(std::move(reading));
|
||||
}
|
||||
|
||||
void ReadFromCluster::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
createExtension(nullptr);
|
||||
|
||||
const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{};
|
||||
const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState;
|
||||
|
||||
Pipes pipes;
|
||||
auto new_context = updateSettings(context->getSettingsRef());
|
||||
const auto & current_settings = new_context->getSettingsRef();
|
||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
|
||||
for (const auto & shard_info : cluster->getShardsInfo())
|
||||
@ -100,7 +176,7 @@ Pipe IStorageCluster::read(
|
||||
auto remote_query_executor = std::make_shared<RemoteQueryExecutor>(
|
||||
std::vector<IConnectionPool::Entry>{try_result},
|
||||
queryToString(query_to_send),
|
||||
sample_block,
|
||||
getOutputStream().header,
|
||||
new_context,
|
||||
/*throttler=*/nullptr,
|
||||
scalars,
|
||||
@ -113,8 +189,14 @@ Pipe IStorageCluster::read(
|
||||
}
|
||||
}
|
||||
|
||||
storage_snapshot->check(column_names);
|
||||
return Pipe::unitePipes(std::move(pipes));
|
||||
auto pipe = Pipe::unitePipes(std::move(pipes));
|
||||
if (pipe.empty())
|
||||
pipe = Pipe(std::make_shared<NullSource>(getOutputStream().header));
|
||||
|
||||
for (const auto & processor : pipe.getProcessors())
|
||||
processors.emplace_back(processor);
|
||||
|
||||
pipeline.init(std::move(pipe));
|
||||
}
|
||||
|
||||
QueryProcessingStage::Enum IStorageCluster::getQueryProcessingStage(
|
||||
@ -129,7 +211,7 @@ QueryProcessingStage::Enum IStorageCluster::getQueryProcessingStage(
|
||||
return QueryProcessingStage::Enum::FetchColumns;
|
||||
}
|
||||
|
||||
ContextPtr IStorageCluster::updateSettings(ContextPtr context, const Settings & settings)
|
||||
ContextPtr ReadFromCluster::updateSettings(const Settings & settings)
|
||||
{
|
||||
Settings new_settings = settings;
|
||||
|
||||
|
@ -22,7 +22,8 @@ public:
|
||||
Poco::Logger * log_,
|
||||
bool structure_argument_was_provided_);
|
||||
|
||||
Pipe read(
|
||||
void read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
@ -33,7 +34,7 @@ public:
|
||||
|
||||
ClusterPtr getCluster(ContextPtr context) const;
|
||||
/// Query is needed for pruning by virtual columns (_file, _path)
|
||||
virtual RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const = 0;
|
||||
virtual RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const = 0;
|
||||
|
||||
QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override;
|
||||
|
||||
@ -45,8 +46,6 @@ protected:
|
||||
virtual void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) = 0;
|
||||
|
||||
private:
|
||||
ContextPtr updateSettings(ContextPtr context, const Settings & settings);
|
||||
|
||||
Poco::Logger * log;
|
||||
String cluster_name;
|
||||
bool structure_argument_was_provided;
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -30,12 +31,10 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas(
|
||||
settings_,
|
||||
context_)
|
||||
, extension(std::move(extension_))
|
||||
, coordination_mode(CoordinationMode::Default)
|
||||
{
|
||||
extension.all_callback(InitialAllRangesAnnouncement(
|
||||
CoordinationMode::Default,
|
||||
parts_ranges.getDescriptions(),
|
||||
extension.number_of_current_replica
|
||||
));
|
||||
extension.all_callback(
|
||||
InitialAllRangesAnnouncement(coordination_mode, parts_ranges.getDescriptions(), extension.number_of_current_replica));
|
||||
}
|
||||
|
||||
MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t /*task_idx*/, MergeTreeReadTask * previous_task)
|
||||
@ -48,7 +47,7 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t /*task_id
|
||||
if (buffered_ranges.empty())
|
||||
{
|
||||
auto result = extension.callback(ParallelReadRequest(
|
||||
CoordinationMode::Default,
|
||||
coordination_mode,
|
||||
extension.number_of_current_replica,
|
||||
pool_settings.min_marks_for_concurrent_read * pool_settings.threads,
|
||||
/// For Default coordination mode we don't need to pass part names.
|
||||
|
@ -31,6 +31,7 @@ private:
|
||||
mutable std::mutex mutex;
|
||||
|
||||
const ParallelReadingExtension extension;
|
||||
const CoordinationMode coordination_mode;
|
||||
RangesInDataPartsDescription buffered_ranges;
|
||||
bool no_more_tasks_available{false};
|
||||
Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPoolParallelReplicas");
|
||||
|
@ -1,27 +1,77 @@
|
||||
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <cmath>
|
||||
#include <cstddef>
|
||||
#include <iterator>
|
||||
#include <map>
|
||||
#include <mutex>
|
||||
#include <numeric>
|
||||
#include <vector>
|
||||
#include <map>
|
||||
#include <set>
|
||||
|
||||
#include <string>
|
||||
#include <type_traits>
|
||||
#include <unordered_map>
|
||||
#include <vector>
|
||||
#include <consistent_hashing.h>
|
||||
|
||||
#include "Common/Exception.h"
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <base/types.h>
|
||||
#include "IO/WriteBufferFromString.h"
|
||||
#include <IO/Progress.h>
|
||||
#include "Storages/MergeTree/RangesInDataPart.h"
|
||||
#include "Storages/MergeTree/RequestResponse.h"
|
||||
#include <Storages/MergeTree/MarkRange.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Storages/MergeTree/IntersectionsIndexes.h>
|
||||
#include <Storages/MergeTree/MarkRange.h>
|
||||
#include <Storages/MergeTree/MergeTreePartInfo.h>
|
||||
#include <Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <Storages/MergeTree/RequestResponse.h>
|
||||
#include <base/defines.h>
|
||||
#include <base/types.h>
|
||||
#include <boost/algorithm/string/split.hpp>
|
||||
#include <fmt/core.h>
|
||||
#include <fmt/format.h>
|
||||
#include <Common/ElapsedTimeProfileEventIncrement.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
namespace
|
||||
{
|
||||
size_t roundDownToMultiple(size_t num, size_t multiple)
|
||||
{
|
||||
return (num / multiple) * multiple;
|
||||
}
|
||||
|
||||
size_t
|
||||
takeFromRange(const MarkRange & range, size_t min_number_of_marks, size_t & current_marks_amount, RangesInDataPartDescription & result)
|
||||
{
|
||||
const auto marks_needed = min_number_of_marks - current_marks_amount;
|
||||
chassert(marks_needed);
|
||||
auto range_we_take = MarkRange{range.begin, range.begin + std::min(marks_needed, range.getNumberOfMarks())};
|
||||
if (!result.ranges.empty() && result.ranges.back().end == range_we_take.begin)
|
||||
/// Can extend the previous range
|
||||
result.ranges.back().end = range_we_take.end;
|
||||
else
|
||||
result.ranges.emplace_back(range_we_take);
|
||||
current_marks_amount += range_we_take.getNumberOfMarks();
|
||||
return range_we_take.getNumberOfMarks();
|
||||
}
|
||||
}
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event ParallelReplicasHandleRequestMicroseconds;
|
||||
extern const Event ParallelReplicasHandleAnnouncementMicroseconds;
|
||||
|
||||
extern const Event ParallelReplicasStealingByHashMicroseconds;
|
||||
extern const Event ParallelReplicasProcessingPartsMicroseconds;
|
||||
extern const Event ParallelReplicasStealingLeftoversMicroseconds;
|
||||
extern const Event ParallelReplicasCollectingOwnedSegmentsMicroseconds;
|
||||
|
||||
extern const Event ParallelReplicasReadAssignedMarks;
|
||||
extern const Event ParallelReplicasReadUnassignedMarks;
|
||||
extern const Event ParallelReplicasReadAssignedForStealingMarks;
|
||||
}
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
@ -58,7 +108,8 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
class ParallelReplicasReadingCoordinator::ImplInterface
|
||||
@ -68,6 +119,15 @@ public:
|
||||
{
|
||||
size_t number_of_requests{0};
|
||||
size_t sum_marks{0};
|
||||
|
||||
/// Marks assigned to the given replica by consistent hash
|
||||
size_t assigned_to_me = 0;
|
||||
/// Marks stolen from other replicas
|
||||
size_t stolen_unassigned = 0;
|
||||
|
||||
/// Stolen marks that were assigned for stealing to the given replica by hash. Makes sense only for DefaultCoordinator
|
||||
size_t stolen_by_hash = 0;
|
||||
|
||||
bool is_unavailable{false};
|
||||
};
|
||||
using Stats = std::vector<Stat>;
|
||||
@ -76,7 +136,15 @@ public:
|
||||
String result = "Statistics: ";
|
||||
std::vector<String> stats_by_replica;
|
||||
for (size_t i = 0; i < stats.size(); ++i)
|
||||
stats_by_replica.push_back(fmt::format("replica {}{} - {{requests: {} marks: {}}}", i, stats[i].is_unavailable ? " is unavailable" : "", stats[i].number_of_requests, stats[i].sum_marks));
|
||||
stats_by_replica.push_back(fmt::format(
|
||||
"replica {}{} - {{requests: {} marks: {} assigned_to_me: {} stolen_by_hash: {} stolen_unassigned: {}}}",
|
||||
i,
|
||||
stats[i].is_unavailable ? " is unavailable" : "",
|
||||
stats[i].number_of_requests,
|
||||
stats[i].sum_marks,
|
||||
stats[i].assigned_to_me,
|
||||
stats[i].stolen_by_hash,
|
||||
stats[i].stolen_unassigned));
|
||||
result += fmt::format("{}", fmt::join(stats_by_replica, "; "));
|
||||
return result;
|
||||
}
|
||||
@ -92,6 +160,7 @@ public:
|
||||
{}
|
||||
|
||||
virtual ~ImplInterface() = default;
|
||||
|
||||
virtual ParallelReadResponse handleRequest(ParallelReadRequest request) = 0;
|
||||
virtual void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) = 0;
|
||||
virtual void markReplicaAsUnavailable(size_t replica_number) = 0;
|
||||
@ -103,165 +172,227 @@ using Parts = std::set<Part>;
|
||||
using PartRefs = std::deque<Parts::iterator>;
|
||||
|
||||
|
||||
/// This coordinator relies heavily on the fact that we work with a single shard,
|
||||
/// i.e. the difference in parts contained in each replica's snapshot is rather negligible (it is only recently inserted or merged parts).
|
||||
/// So the guarantees we provide here are basically the same as with single-node reading: we will read from parts as their were seen by some node at the moment when query started.
|
||||
///
|
||||
/// Knowing that almost each part could be read by each node, we suppose ranges of each part to be available to all the replicas and thus distribute them evenly between them
|
||||
/// (of course we still check if replica has access to the given part before scheduling a reading from it).
|
||||
///
|
||||
/// Of course we want to distribute marks evenly. Looks like it is better to split parts into reasonably small segments of equal size
|
||||
/// (something between 16 and 128 granules i.e. ~100K and ~1M rows should work).
|
||||
/// This approach seems to work ok for all three main cases: full scan, reading random sub-ranges and reading only {pre,suf}-fix of parts.
|
||||
/// Also we could expect that more granular division will make distribution more even up to a certain point.
|
||||
class DefaultCoordinator : public ParallelReplicasReadingCoordinator::ImplInterface
|
||||
{
|
||||
public:
|
||||
using ParallelReadRequestPtr = std::unique_ptr<ParallelReadRequest>;
|
||||
using PartToMarkRanges = std::map<PartToRead::PartAndProjectionNames, HalfIntervals>;
|
||||
|
||||
explicit DefaultCoordinator(size_t replicas_count_)
|
||||
explicit DefaultCoordinator(size_t replicas_count_, size_t mark_segment_size_)
|
||||
: ParallelReplicasReadingCoordinator::ImplInterface(replicas_count_)
|
||||
, reading_state(replicas_count_)
|
||||
, mark_segment_size(mark_segment_size_)
|
||||
, replica_status(replicas_count_)
|
||||
, distribution_by_hash_queue(replicas_count_)
|
||||
{
|
||||
if (mark_segment_size == 0)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Zero value provided for `mark_segment_size`");
|
||||
}
|
||||
|
||||
~DefaultCoordinator() override;
|
||||
|
||||
struct PartitionReading
|
||||
{
|
||||
PartSegments part_ranges;
|
||||
PartToMarkRanges mark_ranges_in_part;
|
||||
};
|
||||
ParallelReadResponse handleRequest(ParallelReadRequest request) override;
|
||||
|
||||
using PartitionToBlockRanges = std::map<String, PartitionReading>;
|
||||
PartitionToBlockRanges partitions;
|
||||
void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) override;
|
||||
|
||||
void markReplicaAsUnavailable(size_t replica_number) override;
|
||||
|
||||
private:
|
||||
/// This many granules will represent a single segment of marks that will be assigned to a replica
|
||||
const size_t mark_segment_size{0};
|
||||
|
||||
size_t sent_initial_requests{0};
|
||||
bool state_initialized{false};
|
||||
size_t finished_replicas{0};
|
||||
|
||||
Parts all_parts_to_read;
|
||||
/// Contains only parts which we haven't started to read from
|
||||
PartRefs delayed_parts;
|
||||
/// Per-replica preferred parts split by consistent hash
|
||||
/// Once all task will be done by some replica, it can steal tasks
|
||||
std::vector<PartRefs> reading_state;
|
||||
struct ReplicaStatus
|
||||
{
|
||||
bool is_finished{false};
|
||||
bool is_announcement_received{false};
|
||||
};
|
||||
std::vector<ReplicaStatus> replica_status;
|
||||
|
||||
Poco::Logger * log = &Poco::Logger::get("DefaultCoordinator");
|
||||
|
||||
std::atomic<bool> state_initialized{false};
|
||||
/// Workflow of a segment:
|
||||
/// 0. `all_parts_to_read` contains all the parts and thus all the segments initially present there (virtually)
|
||||
/// 1. when we traverse `all_parts_to_read` in selectPartsAndRanges() we either:
|
||||
/// * take this segment into output
|
||||
/// * put this segment into `distribution_by_hash_queue` for its owner if it's available and can read from it
|
||||
/// * otherwise put this segment into `distribution_by_hash_queue` for its stealer_by_hash if it's available and can read from it
|
||||
/// * otherwise put this segment into `ranges_for_stealing_queue`
|
||||
/// 2. when we traverse `distribution_by_hash_queue` in `selectPartsAndRanges` we either:
|
||||
/// * take this segment into output
|
||||
/// * otherwise put this segment into `distribution_by_hash_queue` for its stealer_by_hash if it's available and can read from it
|
||||
/// * otherwise put this segment into `ranges_for_stealing_queue`
|
||||
/// 3. when we figuring out that some replica is unavailable we move all segments from its `distribution_by_hash_queue` to their stealers by hash or to `ranges_for_stealing_queue`
|
||||
/// 4. when we get the announcement from a replica we move all segments it cannot read to their stealers by hash or to `ranges_for_stealing_queue`
|
||||
///
|
||||
/// So, segments always move in one direction down this path (possibly skipping some stops):
|
||||
/// `all_parts_to_read` -> `distribution_by_hash_queue[owner]` -> `distribution_by_hash_queue[stealer_by_hash]` -> `ranges_for_stealing_queue`
|
||||
|
||||
ParallelReadResponse handleRequest(ParallelReadRequest request) override;
|
||||
void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) override;
|
||||
void markReplicaAsUnavailable(size_t replica_number) override;
|
||||
/// We take the set of parts announced by this replica as the working set for the whole query.
|
||||
/// For this replica we know for sure that
|
||||
/// 1. it sees all the parts from this set
|
||||
/// 2. it was available in the beginning of execution (since we got announcement), so if it will become unavailable at some point - query will be failed with exception.
|
||||
/// this means that we can delegate reading of all leftover segments (i.e. segments that were not read by their owner or stealer by hash) to this node
|
||||
size_t source_replica_for_parts_snapshot{0};
|
||||
|
||||
void updateReadingState(InitialAllRangesAnnouncement announcement);
|
||||
void finalizeReadingState();
|
||||
/// Parts view from the first announcement we received
|
||||
std::vector<Part> all_parts_to_read;
|
||||
|
||||
size_t computeConsistentHash(const MergeTreePartInfo & info) const
|
||||
std::unordered_map<std::string, std::unordered_set<size_t>> part_visibility; /// part_name -> set of replicas announced that part
|
||||
|
||||
/// We order parts from biggest (= oldest) to newest and steal from newest. Because we assume
|
||||
/// that they're gonna be merged soon anyway and for them we should already expect worse cache hit.
|
||||
struct BiggerPartsFirst
|
||||
{
|
||||
auto hash = SipHash();
|
||||
hash.update(info.getPartNameV1());
|
||||
return ConsistentHashing(hash.get64(), replicas_count);
|
||||
}
|
||||
bool operator()(const auto & lhs, const auto & rhs) const { return lhs.info.getBlocksCount() > rhs.info.getBlocksCount(); }
|
||||
};
|
||||
|
||||
void selectPartsAndRanges(const PartRefs & container, size_t replica_num, size_t min_number_of_marks, size_t & current_mark_size, ParallelReadResponse & response) const;
|
||||
/// We don't precalculate the whole assignment for each node at the start.
|
||||
/// When replica asks coordinator for a new portion of data to read, it traverses `all_parts_to_read` to find ranges relevant to this replica (by consistent hash).
|
||||
/// Many hashes are being calculated during this process and just to not loose this time we save the information about all these ranges
|
||||
/// observed along the way to what node they belong to.
|
||||
/// Ranges in this queue might belong to a part that the given replica cannot read from - the corresponding check happens later.
|
||||
/// TODO: consider making it bounded in size
|
||||
std::vector<std::multiset<RangesInDataPartDescription, BiggerPartsFirst>> distribution_by_hash_queue;
|
||||
|
||||
/// For some ranges their owner and stealer (by consistent hash) cannot read from the given part at all. So this range have to be stolen anyway.
|
||||
/// TODO: consider making it bounded in size
|
||||
RangesInDataPartsDescription ranges_for_stealing_queue;
|
||||
|
||||
/// We take only first replica's set of parts as the whole working set for this query.
|
||||
/// For other replicas we'll just discard parts that they know, but that weren't present in the first request we received.
|
||||
/// The second and all subsequent announcements needed only to understand if we can schedule reading from the given part to the given replica.
|
||||
void initializeReadingState(InitialAllRangesAnnouncement announcement);
|
||||
|
||||
void setProgressCallback();
|
||||
|
||||
enum class ScanMode
|
||||
{
|
||||
/// Main working set for the replica
|
||||
TakeWhatsMineByHash,
|
||||
/// We need to steal to optimize tail latency, let's do it by hash nevertheless
|
||||
TakeWhatsMineForStealing,
|
||||
/// All bets are off, we need to steal "for correctness" - to not leave any segments unread
|
||||
TakeEverythingAvailable
|
||||
};
|
||||
|
||||
void selectPartsAndRanges(
|
||||
size_t replica_num,
|
||||
ScanMode scan_mode,
|
||||
size_t min_number_of_marks,
|
||||
size_t & current_marks_amount,
|
||||
RangesInDataPartsDescription & description);
|
||||
|
||||
size_t computeConsistentHash(const std::string & part_name, size_t segment_begin, ScanMode scan_mode) const;
|
||||
|
||||
void tryToTakeFromDistributionQueue(
|
||||
size_t replica_num, size_t min_number_of_marks, size_t & current_marks_amount, RangesInDataPartsDescription & description);
|
||||
|
||||
void tryToStealFromQueues(
|
||||
size_t replica_num,
|
||||
ScanMode scan_mode,
|
||||
size_t min_number_of_marks,
|
||||
size_t & current_marks_amount,
|
||||
RangesInDataPartsDescription & description);
|
||||
|
||||
void tryToStealFromQueue(
|
||||
auto & queue,
|
||||
ssize_t owner, /// In case `queue` is `distribution_by_hash_queue[replica]`
|
||||
size_t replica_num,
|
||||
ScanMode scan_mode,
|
||||
size_t min_number_of_marks,
|
||||
size_t & current_marks_amount,
|
||||
RangesInDataPartsDescription & description);
|
||||
|
||||
void processPartsFurther(
|
||||
size_t replica_num,
|
||||
ScanMode scan_mode,
|
||||
size_t min_number_of_marks,
|
||||
size_t & current_marks_amount,
|
||||
RangesInDataPartsDescription & description);
|
||||
|
||||
bool possiblyCanReadPart(size_t replica, const MergeTreePartInfo & info) const;
|
||||
void enqueueSegment(const MergeTreePartInfo & info, const MarkRange & segment, size_t owner);
|
||||
void enqueueToStealerOrStealingQueue(const MergeTreePartInfo & info, const MarkRange & segment);
|
||||
};
|
||||
|
||||
|
||||
DefaultCoordinator::~DefaultCoordinator()
|
||||
{
|
||||
LOG_DEBUG(log, "Coordination done: {}", toString(stats));
|
||||
try
|
||||
{
|
||||
LOG_DEBUG(log, "Coordination done: {}", toString(stats));
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log);
|
||||
}
|
||||
}
|
||||
|
||||
void DefaultCoordinator::updateReadingState(InitialAllRangesAnnouncement announcement)
|
||||
void DefaultCoordinator::initializeReadingState(InitialAllRangesAnnouncement announcement)
|
||||
{
|
||||
PartRefs parts_diff;
|
||||
|
||||
/// To get rid of duplicates
|
||||
for (auto && part_ranges: announcement.description)
|
||||
for (const auto & part : announcement.description)
|
||||
{
|
||||
Part part{.description = std::move(part_ranges), .replicas = {announcement.replica_num}};
|
||||
const MergeTreePartInfo & announced_part = part.description.info;
|
||||
|
||||
auto it = std::lower_bound(cbegin(all_parts_to_read), cend(all_parts_to_read), part);
|
||||
if (it != all_parts_to_read.cend())
|
||||
{
|
||||
const MergeTreePartInfo & found_part = it->description.info;
|
||||
if (found_part == announced_part)
|
||||
{
|
||||
/// We have the same part - add the info about presence on current replica
|
||||
it->replicas.insert(announcement.replica_num);
|
||||
continue;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// check if it is covering or covered part
|
||||
/// need to compare with 2 nearest parts in set, - lesser and greater than the part from the announcement
|
||||
bool is_disjoint = found_part.isDisjoint(announced_part);
|
||||
if (it != all_parts_to_read.cbegin() && is_disjoint)
|
||||
{
|
||||
const MergeTreePartInfo & lesser_part = (--it)->description.info;
|
||||
is_disjoint &= lesser_part.isDisjoint(announced_part);
|
||||
}
|
||||
if (!is_disjoint)
|
||||
continue;
|
||||
}
|
||||
}
|
||||
else if (!all_parts_to_read.empty())
|
||||
{
|
||||
/// the announced part is greatest - check if it's disjoint with lesser part
|
||||
const MergeTreePartInfo & lesser_part = all_parts_to_read.crbegin()->description.info;
|
||||
if (!lesser_part.isDisjoint(announced_part))
|
||||
continue;
|
||||
}
|
||||
|
||||
auto [insert_it, _] = all_parts_to_read.emplace(std::move(part));
|
||||
parts_diff.push_back(insert_it);
|
||||
/// We don't really care here if this part will be included into the working set or not
|
||||
part_visibility[part.info.getPartNameV1()].insert(announcement.replica_num);
|
||||
}
|
||||
|
||||
/// Split all parts by consistent hash
|
||||
while (!parts_diff.empty())
|
||||
/// If state is already initialized - just register availabitily info and leave
|
||||
if (state_initialized)
|
||||
return;
|
||||
|
||||
for (auto && part : announcement.description)
|
||||
{
|
||||
auto current_part_it = parts_diff.front();
|
||||
parts_diff.pop_front();
|
||||
auto consistent_hash = computeConsistentHash(current_part_it->description.info);
|
||||
auto intersecting_it = std::find_if(
|
||||
all_parts_to_read.begin(),
|
||||
all_parts_to_read.end(),
|
||||
[&part](const Part & other) { return !other.description.info.isDisjoint(part.info); });
|
||||
|
||||
/// Check whether the new part can easy go to replica queue
|
||||
if (current_part_it->replicas.contains(consistent_hash))
|
||||
{
|
||||
reading_state[consistent_hash].emplace_back(current_part_it);
|
||||
continue;
|
||||
}
|
||||
if (intersecting_it != all_parts_to_read.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Intersecting parts found in announcement");
|
||||
|
||||
/// Add to delayed parts
|
||||
delayed_parts.emplace_back(current_part_it);
|
||||
all_parts_to_read.push_back(Part{.description = std::move(part), .replicas = {announcement.replica_num}});
|
||||
}
|
||||
|
||||
std::ranges::sort(
|
||||
all_parts_to_read, [](const Part & lhs, const Part & rhs) { return BiggerPartsFirst()(lhs.description, rhs.description); });
|
||||
state_initialized = true;
|
||||
source_replica_for_parts_snapshot = announcement.replica_num;
|
||||
|
||||
LOG_DEBUG(log, "Reading state is fully initialized: {}", fmt::join(all_parts_to_read, "; "));
|
||||
}
|
||||
|
||||
void DefaultCoordinator::markReplicaAsUnavailable(size_t replica_number)
|
||||
{
|
||||
if (stats[replica_number].is_unavailable == false)
|
||||
LOG_DEBUG(log, "Replica number {} is unavailable", replica_number);
|
||||
|
||||
++unavailable_replicas_count;
|
||||
stats[replica_number].is_unavailable = true;
|
||||
|
||||
if (sent_initial_requests == replicas_count - unavailable_replicas_count)
|
||||
setProgressCallback();
|
||||
|
||||
for (const auto & segment : distribution_by_hash_queue[replica_number])
|
||||
{
|
||||
LOG_DEBUG(log, "Replica number {} is unavailable", replica_number);
|
||||
|
||||
stats[replica_number].is_unavailable = true;
|
||||
++unavailable_replicas_count;
|
||||
|
||||
if (sent_initial_requests == replicas_count - unavailable_replicas_count)
|
||||
finalizeReadingState();
|
||||
chassert(segment.ranges.size() == 1);
|
||||
enqueueToStealerOrStealingQueue(segment.info, segment.ranges.front());
|
||||
}
|
||||
distribution_by_hash_queue[replica_number].clear();
|
||||
}
|
||||
|
||||
void DefaultCoordinator::finalizeReadingState()
|
||||
void DefaultCoordinator::setProgressCallback()
|
||||
{
|
||||
/// Clear all the delayed queue
|
||||
while (!delayed_parts.empty())
|
||||
{
|
||||
auto current_part_it = delayed_parts.front();
|
||||
auto consistent_hash = computeConsistentHash(current_part_it->description.info);
|
||||
|
||||
if (current_part_it->replicas.contains(consistent_hash))
|
||||
{
|
||||
reading_state[consistent_hash].emplace_back(current_part_it);
|
||||
delayed_parts.pop_front();
|
||||
continue;
|
||||
}
|
||||
|
||||
/// In this situation just assign to a random replica which has this part
|
||||
auto replica = *(std::next(current_part_it->replicas.begin(), thread_local_rng() % current_part_it->replicas.size()));
|
||||
reading_state[replica].emplace_back(current_part_it);
|
||||
delayed_parts.pop_front();
|
||||
}
|
||||
|
||||
// update progress with total rows
|
||||
// Update progress with total rows
|
||||
if (progress_callback)
|
||||
{
|
||||
size_t total_rows_to_read = 0;
|
||||
@ -274,116 +405,378 @@ void DefaultCoordinator::finalizeReadingState()
|
||||
|
||||
LOG_DEBUG(log, "Total rows to read: {}", total_rows_to_read);
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Reading state is fully initialized: {}", fmt::join(all_parts_to_read, "; "));
|
||||
}
|
||||
|
||||
|
||||
void DefaultCoordinator::handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement)
|
||||
{
|
||||
const auto replica_num = announcement.replica_num;
|
||||
|
||||
updateReadingState(std::move(announcement));
|
||||
LOG_DEBUG(log, "Initial request from replica {}: {}", announcement.replica_num, announcement.describe());
|
||||
|
||||
initializeReadingState(std::move(announcement));
|
||||
|
||||
if (replica_num >= stats.size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Replica number ({}) is bigger than total replicas count ({})", replica_num, stats.size());
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR, "Replica number ({}) is bigger than total replicas count ({})", replica_num, stats.size());
|
||||
|
||||
++stats[replica_num].number_of_requests;
|
||||
replica_status[replica_num].is_announcement_received = true;
|
||||
|
||||
++sent_initial_requests;
|
||||
LOG_DEBUG(log, "Sent initial requests: {} Replicas count: {}", sent_initial_requests, replicas_count);
|
||||
|
||||
if (sent_initial_requests == replicas_count)
|
||||
finalizeReadingState();
|
||||
}
|
||||
setProgressCallback();
|
||||
|
||||
void DefaultCoordinator::selectPartsAndRanges(const PartRefs & container, size_t replica_num, size_t min_number_of_marks, size_t & current_mark_size, ParallelReadResponse & response) const
|
||||
{
|
||||
for (const auto & part : container)
|
||||
/// Sift the queue to move out all invisible segments
|
||||
for (const auto & segment : distribution_by_hash_queue[replica_num])
|
||||
{
|
||||
if (current_mark_size >= min_number_of_marks)
|
||||
if (!part_visibility[segment.info.getPartNameV1()].contains(replica_num))
|
||||
{
|
||||
LOG_TEST(log, "Current mark size {} is bigger than min_number_marks {}", current_mark_size, min_number_of_marks);
|
||||
break;
|
||||
}
|
||||
|
||||
if (part->description.ranges.empty())
|
||||
{
|
||||
LOG_TEST(log, "Part {} is already empty in reading state", part->description.info.getPartNameV1());
|
||||
continue;
|
||||
}
|
||||
|
||||
if (std::find(part->replicas.begin(), part->replicas.end(), replica_num) == part->replicas.end())
|
||||
{
|
||||
LOG_TEST(log, "Not found part {} on replica {}", part->description.info.getPartNameV1(), replica_num);
|
||||
continue;
|
||||
}
|
||||
|
||||
response.description.push_back({
|
||||
.info = part->description.info,
|
||||
.ranges = {},
|
||||
});
|
||||
|
||||
while (!part->description.ranges.empty() && current_mark_size < min_number_of_marks)
|
||||
{
|
||||
auto & range = part->description.ranges.front();
|
||||
const size_t needed = min_number_of_marks - current_mark_size;
|
||||
|
||||
if (range.getNumberOfMarks() > needed)
|
||||
{
|
||||
auto range_we_take = MarkRange{range.begin, range.begin + needed};
|
||||
response.description.back().ranges.emplace_back(range_we_take);
|
||||
current_mark_size += range_we_take.getNumberOfMarks();
|
||||
|
||||
range.begin += needed;
|
||||
break;
|
||||
}
|
||||
|
||||
response.description.back().ranges.emplace_back(range);
|
||||
current_mark_size += range.getNumberOfMarks();
|
||||
part->description.ranges.pop_front();
|
||||
chassert(segment.ranges.size() == 1);
|
||||
enqueueToStealerOrStealingQueue(segment.info, segment.ranges.front());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void DefaultCoordinator::tryToTakeFromDistributionQueue(
|
||||
size_t replica_num, size_t min_number_of_marks, size_t & current_marks_amount, RangesInDataPartsDescription & description)
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::ParallelReplicasCollectingOwnedSegmentsMicroseconds);
|
||||
|
||||
auto & distribution_queue = distribution_by_hash_queue[replica_num];
|
||||
auto replica_can_read_part = [&](auto replica, const auto & part) { return part_visibility[part.getPartNameV1()].contains(replica); };
|
||||
|
||||
RangesInDataPartDescription result;
|
||||
|
||||
while (!distribution_queue.empty() && current_marks_amount < min_number_of_marks)
|
||||
{
|
||||
if (result.ranges.empty() || distribution_queue.begin()->info != result.info)
|
||||
{
|
||||
if (!result.ranges.empty())
|
||||
/// We're switching to a different part, so have to save currently accumulated ranges
|
||||
description.push_back(result);
|
||||
result = {.info = distribution_queue.begin()->info};
|
||||
}
|
||||
|
||||
/// NOTE: this works because ranges are not considered by the comparator
|
||||
auto & part_ranges = const_cast<RangesInDataPartDescription &>(*distribution_queue.begin());
|
||||
chassert(part_ranges.ranges.size() == 1);
|
||||
auto & range = part_ranges.ranges.front();
|
||||
|
||||
if (replica_can_read_part(replica_num, part_ranges.info))
|
||||
{
|
||||
if (auto taken = takeFromRange(range, min_number_of_marks, current_marks_amount, result); taken == range.getNumberOfMarks())
|
||||
distribution_queue.erase(distribution_queue.begin());
|
||||
else
|
||||
{
|
||||
range.begin += taken;
|
||||
break;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// It might be that `replica_num` is the stealer by hash itself - no problem,
|
||||
/// we'll just have a redundant hash computation inside this function
|
||||
enqueueToStealerOrStealingQueue(part_ranges.info, range);
|
||||
distribution_queue.erase(distribution_queue.begin());
|
||||
}
|
||||
}
|
||||
|
||||
if (!result.ranges.empty())
|
||||
description.push_back(result);
|
||||
}
|
||||
|
||||
void DefaultCoordinator::tryToStealFromQueues(
|
||||
size_t replica_num,
|
||||
ScanMode scan_mode,
|
||||
size_t min_number_of_marks,
|
||||
size_t & current_marks_amount,
|
||||
RangesInDataPartsDescription & description)
|
||||
{
|
||||
auto steal_from_other_replicas = [&]()
|
||||
{
|
||||
/// Try to steal from other replicas starting from replicas with longest queues
|
||||
std::vector<size_t> order(replicas_count);
|
||||
std::iota(order.begin(), order.end(), 0);
|
||||
std::ranges::sort(
|
||||
order, [&](auto lhs, auto rhs) { return distribution_by_hash_queue[lhs].size() > distribution_by_hash_queue[rhs].size(); });
|
||||
|
||||
for (auto replica : order)
|
||||
tryToStealFromQueue(
|
||||
distribution_by_hash_queue[replica],
|
||||
replica,
|
||||
replica_num,
|
||||
scan_mode,
|
||||
min_number_of_marks,
|
||||
current_marks_amount,
|
||||
description);
|
||||
};
|
||||
|
||||
if (scan_mode == ScanMode::TakeWhatsMineForStealing)
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::ParallelReplicasStealingByHashMicroseconds);
|
||||
steal_from_other_replicas();
|
||||
}
|
||||
else
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::ParallelReplicasStealingLeftoversMicroseconds);
|
||||
/// Check orphaned ranges
|
||||
tryToStealFromQueue(
|
||||
ranges_for_stealing_queue, /*owner=*/-1, replica_num, scan_mode, min_number_of_marks, current_marks_amount, description);
|
||||
/// Last hope. In case we haven't yet figured out that some node is unavailable its segments are still in the distribution queue.
|
||||
steal_from_other_replicas();
|
||||
}
|
||||
}
|
||||
|
||||
void DefaultCoordinator::tryToStealFromQueue(
|
||||
auto & queue,
|
||||
ssize_t owner,
|
||||
size_t replica_num,
|
||||
ScanMode scan_mode,
|
||||
size_t min_number_of_marks,
|
||||
size_t & current_marks_amount,
|
||||
RangesInDataPartsDescription & description)
|
||||
{
|
||||
auto replica_can_read_part = [&](auto replica, const auto & part) { return part_visibility[part.getPartNameV1()].contains(replica); };
|
||||
|
||||
RangesInDataPartDescription result;
|
||||
|
||||
auto it = queue.rbegin();
|
||||
while (it != queue.rend() && current_marks_amount < min_number_of_marks)
|
||||
{
|
||||
auto & part_ranges = const_cast<RangesInDataPartDescription &>(*it);
|
||||
chassert(part_ranges.ranges.size() == 1);
|
||||
auto & range = part_ranges.ranges.front();
|
||||
|
||||
if (result.ranges.empty() || part_ranges.info != result.info)
|
||||
{
|
||||
if (!result.ranges.empty())
|
||||
/// We're switching to a different part, so have to save currently accumulated ranges
|
||||
description.push_back(result);
|
||||
result = {.info = part_ranges.info};
|
||||
}
|
||||
|
||||
if (replica_can_read_part(replica_num, part_ranges.info))
|
||||
{
|
||||
bool can_take = false;
|
||||
if (scan_mode == ScanMode::TakeWhatsMineForStealing)
|
||||
{
|
||||
chassert(owner >= 0);
|
||||
const size_t segment_begin = roundDownToMultiple(range.begin, mark_segment_size);
|
||||
can_take = computeConsistentHash(part_ranges.info.getPartNameV1(), segment_begin, scan_mode) == replica_num;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Don't steal segments with alive owner that sees them
|
||||
can_take = owner == -1 || stats[owner].is_unavailable || !replica_status[owner].is_announcement_received;
|
||||
}
|
||||
if (can_take)
|
||||
{
|
||||
if (auto taken = takeFromRange(range, min_number_of_marks, current_marks_amount, result); taken == range.getNumberOfMarks())
|
||||
{
|
||||
it = decltype(it)(queue.erase(std::next(it).base()));
|
||||
continue;
|
||||
}
|
||||
else
|
||||
range.begin += taken;
|
||||
}
|
||||
}
|
||||
|
||||
++it;
|
||||
}
|
||||
|
||||
if (!result.ranges.empty())
|
||||
description.push_back(result);
|
||||
}
|
||||
|
||||
void DefaultCoordinator::processPartsFurther(
|
||||
size_t replica_num,
|
||||
ScanMode scan_mode,
|
||||
size_t min_number_of_marks,
|
||||
size_t & current_marks_amount,
|
||||
RangesInDataPartsDescription & description)
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::ParallelReplicasProcessingPartsMicroseconds);
|
||||
|
||||
for (const auto & part : all_parts_to_read)
|
||||
{
|
||||
if (current_marks_amount >= min_number_of_marks)
|
||||
{
|
||||
LOG_TEST(log, "Current mark size {} is bigger than min_number_marks {}", current_marks_amount, min_number_of_marks);
|
||||
return;
|
||||
}
|
||||
|
||||
RangesInDataPartDescription result{.info = part.description.info};
|
||||
|
||||
while (!part.description.ranges.empty() && current_marks_amount < min_number_of_marks)
|
||||
{
|
||||
auto & range = part.description.ranges.front();
|
||||
|
||||
/// Parts are divided into segments of `mark_segment_size` granules staring from 0-th granule
|
||||
for (size_t segment_begin = roundDownToMultiple(range.begin, mark_segment_size);
|
||||
segment_begin < range.end && current_marks_amount < min_number_of_marks;
|
||||
segment_begin += mark_segment_size)
|
||||
{
|
||||
const auto cur_segment
|
||||
= MarkRange{std::max(range.begin, segment_begin), std::min(range.end, segment_begin + mark_segment_size)};
|
||||
|
||||
const auto owner = computeConsistentHash(part.description.info.getPartNameV1(), segment_begin, scan_mode);
|
||||
if (owner == replica_num)
|
||||
{
|
||||
const auto taken = takeFromRange(cur_segment, min_number_of_marks, current_marks_amount, result);
|
||||
if (taken == range.getNumberOfMarks())
|
||||
part.description.ranges.pop_front();
|
||||
else
|
||||
{
|
||||
range.begin += taken;
|
||||
break;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
chassert(scan_mode == ScanMode::TakeWhatsMineByHash);
|
||||
enqueueSegment(part.description.info, cur_segment, owner);
|
||||
range.begin += cur_segment.getNumberOfMarks();
|
||||
if (range.getNumberOfMarks() == 0)
|
||||
part.description.ranges.pop_front();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!result.ranges.empty())
|
||||
description.push_back(std::move(result));
|
||||
}
|
||||
}
|
||||
|
||||
void DefaultCoordinator::selectPartsAndRanges(
|
||||
size_t replica_num,
|
||||
ScanMode scan_mode,
|
||||
size_t min_number_of_marks,
|
||||
size_t & current_marks_amount,
|
||||
RangesInDataPartsDescription & description)
|
||||
{
|
||||
if (scan_mode == ScanMode::TakeWhatsMineByHash)
|
||||
{
|
||||
tryToTakeFromDistributionQueue(replica_num, min_number_of_marks, current_marks_amount, description);
|
||||
processPartsFurther(replica_num, scan_mode, min_number_of_marks, current_marks_amount, description);
|
||||
/// We might back-fill `distribution_by_hash_queue` for this replica in `enqueueToStealerOrStealingQueue`
|
||||
tryToTakeFromDistributionQueue(replica_num, min_number_of_marks, current_marks_amount, description);
|
||||
}
|
||||
else
|
||||
tryToStealFromQueues(replica_num, scan_mode, min_number_of_marks, current_marks_amount, description);
|
||||
}
|
||||
|
||||
bool DefaultCoordinator::possiblyCanReadPart(size_t replica, const MergeTreePartInfo & info) const
|
||||
{
|
||||
/// At this point we might not be sure if `owner` can read from the given part.
|
||||
/// Then we will check it while processing `owner`'s data requests - they are guaranteed to came after the announcement.
|
||||
return !stats[replica].is_unavailable && !replica_status[replica].is_finished
|
||||
&& (!replica_status[replica].is_announcement_received || part_visibility.at(info.getPartNameV1()).contains(replica));
|
||||
}
|
||||
|
||||
void DefaultCoordinator::enqueueSegment(const MergeTreePartInfo & info, const MarkRange & segment, size_t owner)
|
||||
{
|
||||
if (possiblyCanReadPart(owner, info))
|
||||
{
|
||||
/// TODO: optimize me (maybe we can store something lighter than RangesInDataPartDescription)
|
||||
distribution_by_hash_queue[owner].insert(RangesInDataPartDescription{.info = info, .ranges = {segment}});
|
||||
LOG_TEST(log, "Segment {} is added to its owner's ({}) queue", segment, owner);
|
||||
}
|
||||
else
|
||||
enqueueToStealerOrStealingQueue(info, segment);
|
||||
}
|
||||
|
||||
void DefaultCoordinator::enqueueToStealerOrStealingQueue(const MergeTreePartInfo & info, const MarkRange & segment)
|
||||
{
|
||||
auto && range = RangesInDataPartDescription{.info = info, .ranges = {segment}};
|
||||
const auto stealer_by_hash = computeConsistentHash(
|
||||
info.getPartNameV1(), roundDownToMultiple(segment.begin, mark_segment_size), ScanMode::TakeWhatsMineForStealing);
|
||||
if (possiblyCanReadPart(stealer_by_hash, info))
|
||||
{
|
||||
distribution_by_hash_queue[stealer_by_hash].insert(std::move(range));
|
||||
LOG_TEST(log, "Segment {} is added to its stealer's ({}) queue", segment, stealer_by_hash);
|
||||
}
|
||||
else
|
||||
{
|
||||
ranges_for_stealing_queue.push_back(std::move(range));
|
||||
LOG_TEST(log, "Segment {} is added to stealing queue", segment);
|
||||
}
|
||||
}
|
||||
|
||||
size_t DefaultCoordinator::computeConsistentHash(const std::string & part_name, size_t segment_begin, ScanMode scan_mode) const
|
||||
{
|
||||
chassert(segment_begin % mark_segment_size == 0);
|
||||
auto hash = SipHash();
|
||||
hash.update(part_name);
|
||||
hash.update(segment_begin);
|
||||
hash.update(scan_mode);
|
||||
return ConsistentHashing(hash.get64(), replicas_count);
|
||||
}
|
||||
|
||||
ParallelReadResponse DefaultCoordinator::handleRequest(ParallelReadRequest request)
|
||||
{
|
||||
LOG_TRACE(log, "Handling request from replica {}, minimal marks size is {}", request.replica_num, request.min_number_of_marks);
|
||||
|
||||
size_t current_mark_size = 0;
|
||||
ParallelReadResponse response;
|
||||
|
||||
/// 1. Try to select from preferred set of parts for current replica
|
||||
selectPartsAndRanges(reading_state[request.replica_num], request.replica_num, request.min_number_of_marks, current_mark_size, response);
|
||||
size_t current_mark_size = 0;
|
||||
|
||||
/// 2. Try to use parts from delayed queue
|
||||
while (!delayed_parts.empty() && current_mark_size < request.min_number_of_marks)
|
||||
{
|
||||
auto part = delayed_parts.front();
|
||||
delayed_parts.pop_front();
|
||||
reading_state[request.replica_num].emplace_back(part);
|
||||
selectPartsAndRanges(reading_state[request.replica_num], request.replica_num, request.min_number_of_marks, current_mark_size, response);
|
||||
}
|
||||
/// 1. Try to select ranges meant for this replica by consistent hash
|
||||
selectPartsAndRanges(
|
||||
request.replica_num, ScanMode::TakeWhatsMineByHash, request.min_number_of_marks, current_mark_size, response.description);
|
||||
const size_t assigned_to_me = current_mark_size;
|
||||
|
||||
/// 3. Try to steal tasks;
|
||||
if (current_mark_size < request.min_number_of_marks)
|
||||
{
|
||||
for (size_t i = 0; i < replicas_count; ++i)
|
||||
{
|
||||
if (i != request.replica_num)
|
||||
selectPartsAndRanges(reading_state[i], request.replica_num, request.min_number_of_marks, current_mark_size, response);
|
||||
/// 2. Try to steal but with caching again (with different key)
|
||||
selectPartsAndRanges(
|
||||
request.replica_num, ScanMode::TakeWhatsMineForStealing, request.min_number_of_marks, current_mark_size, response.description);
|
||||
const size_t stolen_by_hash = current_mark_size - assigned_to_me;
|
||||
|
||||
if (current_mark_size >= request.min_number_of_marks)
|
||||
break;
|
||||
}
|
||||
}
|
||||
/// 3. Try to steal with no preference. We're trying to postpone it as much as possible.
|
||||
if (current_mark_size == 0 && request.replica_num == source_replica_for_parts_snapshot)
|
||||
selectPartsAndRanges(
|
||||
request.replica_num, ScanMode::TakeEverythingAvailable, request.min_number_of_marks, current_mark_size, response.description);
|
||||
const size_t stolen_unassigned = current_mark_size - stolen_by_hash - assigned_to_me;
|
||||
|
||||
stats[request.replica_num].number_of_requests += 1;
|
||||
stats[request.replica_num].sum_marks += current_mark_size;
|
||||
|
||||
stats[request.replica_num].assigned_to_me += assigned_to_me;
|
||||
stats[request.replica_num].stolen_by_hash += stolen_by_hash;
|
||||
stats[request.replica_num].stolen_unassigned += stolen_unassigned;
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::ParallelReplicasReadAssignedMarks, assigned_to_me);
|
||||
ProfileEvents::increment(ProfileEvents::ParallelReplicasReadUnassignedMarks, stolen_unassigned);
|
||||
ProfileEvents::increment(ProfileEvents::ParallelReplicasReadAssignedForStealingMarks, stolen_by_hash);
|
||||
|
||||
if (response.description.empty())
|
||||
{
|
||||
response.finish = true;
|
||||
|
||||
LOG_TRACE(log, "Going to respond to replica {} with {}", request.replica_num, response.describe());
|
||||
replica_status[request.replica_num].is_finished = true;
|
||||
|
||||
if (++finished_replicas == replicas_count - unavailable_replicas_count)
|
||||
{
|
||||
/// Nobody will come to process any more data
|
||||
|
||||
if (!ranges_for_stealing_queue.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Some orphaned segments were left unread");
|
||||
|
||||
for (size_t replica = 0; replica < replicas_count; ++replica)
|
||||
if (!distribution_by_hash_queue[replica].empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Non-empty distribution_by_hash_queue for replica {}", replica);
|
||||
}
|
||||
}
|
||||
|
||||
LOG_DEBUG(
|
||||
log,
|
||||
"Going to respond to replica {} with {}; mine_marks={}, stolen_by_hash={}, stolen_rest={}",
|
||||
request.replica_num,
|
||||
response.describe(),
|
||||
assigned_to_me,
|
||||
stolen_by_hash,
|
||||
stolen_unassigned);
|
||||
|
||||
return response;
|
||||
}
|
||||
|
||||
@ -456,6 +849,8 @@ void InOrderCoordinator<mode>::handleInitialAllRangesAnnouncement(InitialAllRang
|
||||
std::sort(ranges.begin(), ranges.end());
|
||||
}
|
||||
|
||||
++stats[announcement.replica_num].number_of_requests;
|
||||
|
||||
if (new_rows_to_read > 0)
|
||||
{
|
||||
Progress progress;
|
||||
@ -557,6 +952,8 @@ ParallelReadResponse InOrderCoordinator<mode>::handleRequest(ParallelReadRequest
|
||||
|
||||
void ParallelReplicasReadingCoordinator::handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement)
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::ParallelReplicasHandleAnnouncementMicroseconds);
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
if (!pimpl)
|
||||
@ -570,6 +967,8 @@ void ParallelReplicasReadingCoordinator::handleInitialAllRangesAnnouncement(Init
|
||||
|
||||
ParallelReadResponse ParallelReplicasReadingCoordinator::handleRequest(ParallelReadRequest request)
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::ParallelReplicasHandleRequestMicroseconds);
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
if (!pimpl)
|
||||
@ -604,7 +1003,7 @@ void ParallelReplicasReadingCoordinator::initialize()
|
||||
switch (mode)
|
||||
{
|
||||
case CoordinationMode::Default:
|
||||
pimpl = std::make_unique<DefaultCoordinator>(replicas_count);
|
||||
pimpl = std::make_unique<DefaultCoordinator>(replicas_count, mark_segment_size);
|
||||
break;
|
||||
case CoordinationMode::WithOrder:
|
||||
pimpl = std::make_unique<InOrderCoordinator<CoordinationMode::WithOrder>>(replicas_count);
|
||||
@ -621,7 +1020,10 @@ void ParallelReplicasReadingCoordinator::initialize()
|
||||
pimpl->markReplicaAsUnavailable(replica);
|
||||
}
|
||||
|
||||
ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator(size_t replicas_count_) : replicas_count(replicas_count_) {}
|
||||
ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator(size_t replicas_count_, size_t mark_segment_size_)
|
||||
: replicas_count(replicas_count_), mark_segment_size(mark_segment_size_)
|
||||
{
|
||||
}
|
||||
|
||||
ParallelReplicasReadingCoordinator::~ParallelReplicasReadingCoordinator() = default;
|
||||
|
||||
|
@ -15,7 +15,7 @@ class ParallelReplicasReadingCoordinator
|
||||
public:
|
||||
class ImplInterface;
|
||||
|
||||
explicit ParallelReplicasReadingCoordinator(size_t replicas_count_);
|
||||
explicit ParallelReplicasReadingCoordinator(size_t replicas_count_, size_t mark_segment_size_ = 0);
|
||||
~ParallelReplicasReadingCoordinator();
|
||||
|
||||
void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement);
|
||||
@ -35,8 +35,8 @@ private:
|
||||
|
||||
std::mutex mutex;
|
||||
size_t replicas_count{0};
|
||||
size_t mark_segment_size{0};
|
||||
CoordinationMode mode{CoordinationMode::Default};
|
||||
std::atomic<bool> initialized{false};
|
||||
std::unique_ptr<ImplInterface> pimpl;
|
||||
ProgressCallback progress_callback; // store the callback only to bypass it to coordinator implementation
|
||||
std::set<size_t> replicas_used;
|
||||
|
@ -6,11 +6,14 @@
|
||||
#include <IO/CompressionMethod.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Processors/Executors/CompletedPipelineExecutor.h>
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
#include <Processors/ISource.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/SourceStepWithFilter.h>
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Storages/S3Queue/S3QueueTableMetadata.h>
|
||||
#include <Storages/S3Queue/StorageS3Queue.h>
|
||||
#include <Storages/S3Queue/S3QueueFilesMetadata.h>
|
||||
@ -20,6 +23,7 @@
|
||||
#include <Storages/StorageSnapshot.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Storages/prepareReadingFromFormat.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <filesystem>
|
||||
|
||||
|
||||
@ -204,10 +208,65 @@ bool StorageS3Queue::supportsSubsetOfColumns(const ContextPtr & context_) const
|
||||
return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context_, format_settings);
|
||||
}
|
||||
|
||||
Pipe StorageS3Queue::read(
|
||||
class ReadFromS3Queue : public SourceStepWithFilter
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "ReadFromS3Queue"; }
|
||||
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
void applyFilters() override;
|
||||
|
||||
ReadFromS3Queue(
|
||||
Block sample_block,
|
||||
ReadFromFormatInfo info_,
|
||||
std::shared_ptr<StorageS3Queue> storage_,
|
||||
ContextPtr context_,
|
||||
size_t max_block_size_,
|
||||
size_t num_streams_)
|
||||
: SourceStepWithFilter(DataStream{.header = std::move(sample_block)})
|
||||
, info(std::move(info_))
|
||||
, storage(std::move(storage_))
|
||||
, context(std::move(context_))
|
||||
, max_block_size(max_block_size_)
|
||||
, num_streams(num_streams_)
|
||||
{
|
||||
}
|
||||
|
||||
private:
|
||||
ReadFromFormatInfo info;
|
||||
std::shared_ptr<StorageS3Queue> storage;
|
||||
ContextPtr context;
|
||||
size_t max_block_size;
|
||||
size_t num_streams;
|
||||
|
||||
std::shared_ptr<StorageS3Queue::FileIterator> iterator;
|
||||
|
||||
void createIterator(const ActionsDAG::Node * predicate);
|
||||
};
|
||||
|
||||
void ReadFromS3Queue::createIterator(const ActionsDAG::Node * predicate)
|
||||
{
|
||||
if (iterator)
|
||||
return;
|
||||
|
||||
iterator = storage->createFileIterator(context, predicate);
|
||||
}
|
||||
|
||||
|
||||
void ReadFromS3Queue::applyFilters()
|
||||
{
|
||||
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context);
|
||||
const ActionsDAG::Node * predicate = nullptr;
|
||||
if (filter_actions_dag)
|
||||
predicate = filter_actions_dag->getOutputs().at(0);
|
||||
|
||||
createIterator(predicate);
|
||||
}
|
||||
|
||||
void StorageS3Queue::read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
SelectQueryInfo & /*query_info*/,
|
||||
ContextPtr local_context,
|
||||
QueryProcessingStage::Enum /*processed_stage*/,
|
||||
size_t max_block_size,
|
||||
@ -225,27 +284,49 @@ Pipe StorageS3Queue::read(
|
||||
"Cannot read from {} with attached materialized views", getName());
|
||||
}
|
||||
|
||||
Pipes pipes;
|
||||
const size_t adjusted_num_streams = std::min<size_t>(num_streams, s3queue_settings->s3queue_processing_threads_num);
|
||||
auto this_ptr = std::static_pointer_cast<StorageS3Queue>(shared_from_this());
|
||||
auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals());
|
||||
|
||||
auto file_iterator = createFileIterator(local_context, query_info.query);
|
||||
auto reading = std::make_unique<ReadFromS3Queue>(
|
||||
read_from_format_info.source_header,
|
||||
read_from_format_info,
|
||||
std::move(this_ptr),
|
||||
local_context,
|
||||
max_block_size,
|
||||
num_streams);
|
||||
|
||||
query_plan.addStep(std::move(reading));
|
||||
}
|
||||
|
||||
void ReadFromS3Queue::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
Pipes pipes;
|
||||
const size_t adjusted_num_streams = std::min<size_t>(num_streams, storage->s3queue_settings->s3queue_processing_threads_num);
|
||||
|
||||
createIterator(nullptr);
|
||||
for (size_t i = 0; i < adjusted_num_streams; ++i)
|
||||
pipes.emplace_back(createSource(file_iterator, column_names, storage_snapshot, max_block_size, local_context));
|
||||
return Pipe::unitePipes(std::move(pipes));
|
||||
pipes.emplace_back(storage->createSource(info, iterator, max_block_size, context));
|
||||
|
||||
auto pipe = Pipe::unitePipes(std::move(pipes));
|
||||
if (pipe.empty())
|
||||
pipe = Pipe(std::make_shared<NullSource>(info.source_header));
|
||||
|
||||
for (const auto & processor : pipe.getProcessors())
|
||||
processors.emplace_back(processor);
|
||||
|
||||
pipeline.init(std::move(pipe));
|
||||
}
|
||||
|
||||
std::shared_ptr<StorageS3QueueSource> StorageS3Queue::createSource(
|
||||
const ReadFromFormatInfo & info,
|
||||
std::shared_ptr<StorageS3Queue::FileIterator> file_iterator,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
size_t max_block_size,
|
||||
ContextPtr local_context)
|
||||
{
|
||||
auto configuration_snapshot = updateConfigurationAndGetCopy(local_context);
|
||||
auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals());
|
||||
|
||||
auto internal_source = std::make_unique<StorageS3Source>(
|
||||
read_from_format_info, configuration.format, getName(), local_context, format_settings,
|
||||
info, configuration.format, getName(), local_context, format_settings,
|
||||
max_block_size,
|
||||
configuration_snapshot.request_settings,
|
||||
configuration_snapshot.compression_method,
|
||||
@ -253,7 +334,7 @@ std::shared_ptr<StorageS3QueueSource> StorageS3Queue::createSource(
|
||||
configuration_snapshot.url.bucket,
|
||||
configuration_snapshot.url.version_id,
|
||||
configuration_snapshot.url.uri.getHost() + std::to_string(configuration_snapshot.url.uri.getPort()),
|
||||
file_iterator, local_context->getSettingsRef().max_download_threads, false, /* query_info */ std::nullopt);
|
||||
file_iterator, local_context->getSettingsRef().max_download_threads, false);
|
||||
|
||||
auto file_deleter = [this, bucket = configuration_snapshot.url.bucket, client = configuration_snapshot.client, blob_storage_log = BlobStorageLogWriter::create()](const std::string & path) mutable
|
||||
{
|
||||
@ -277,8 +358,8 @@ std::shared_ptr<StorageS3QueueSource> StorageS3Queue::createSource(
|
||||
};
|
||||
auto s3_queue_log = s3queue_settings->s3queue_enable_logging_to_s3queue_log ? local_context->getS3QueueLog() : nullptr;
|
||||
return std::make_shared<StorageS3QueueSource>(
|
||||
getName(), read_from_format_info.source_header, std::move(internal_source),
|
||||
files_metadata, after_processing, file_deleter, read_from_format_info.requested_virtual_columns,
|
||||
getName(), info.source_header, std::move(internal_source),
|
||||
files_metadata, after_processing, file_deleter, info.requested_virtual_columns,
|
||||
local_context, shutdown_called, table_is_being_dropped, s3_queue_log, getStorageID(), log);
|
||||
}
|
||||
|
||||
@ -375,13 +456,14 @@ bool StorageS3Queue::streamToViews()
|
||||
auto block_io = interpreter.execute();
|
||||
auto file_iterator = createFileIterator(s3queue_context, nullptr);
|
||||
|
||||
auto read_from_format_info = prepareReadingFromFormat(block_io.pipeline.getHeader().getNames(), storage_snapshot, supportsSubsetOfColumns(s3queue_context), getVirtuals());
|
||||
|
||||
Pipes pipes;
|
||||
pipes.reserve(s3queue_settings->s3queue_processing_threads_num);
|
||||
for (size_t i = 0; i < s3queue_settings->s3queue_processing_threads_num; ++i)
|
||||
{
|
||||
auto source = createSource(
|
||||
file_iterator, block_io.pipeline.getHeader().getNames(),
|
||||
storage_snapshot, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context);
|
||||
read_from_format_info, file_iterator, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context);
|
||||
|
||||
pipes.emplace_back(std::move(source));
|
||||
}
|
||||
@ -479,10 +561,10 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const
|
||||
}
|
||||
}
|
||||
|
||||
std::shared_ptr<StorageS3Queue::FileIterator> StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query)
|
||||
std::shared_ptr<StorageS3Queue::FileIterator> StorageS3Queue::createFileIterator(ContextPtr local_context, const ActionsDAG::Node * predicate)
|
||||
{
|
||||
auto glob_iterator = std::make_unique<StorageS3QueueSource::GlobIterator>(
|
||||
*configuration.client, configuration.url, query, virtual_columns, local_context,
|
||||
*configuration.client, configuration.url, predicate, virtual_columns, local_context,
|
||||
/* read_keys */nullptr, configuration.request_settings);
|
||||
return std::make_shared<FileIterator>(files_metadata, std::move(glob_iterator), shutdown_called);
|
||||
}
|
||||
|
@ -39,10 +39,11 @@ public:
|
||||
|
||||
String getName() const override { return "S3Queue"; }
|
||||
|
||||
Pipe read(
|
||||
void read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
SelectQueryInfo & /*query_info*/,
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t max_block_size,
|
||||
@ -57,6 +58,7 @@ public:
|
||||
zkutil::ZooKeeperPtr getZooKeeper() const;
|
||||
|
||||
private:
|
||||
friend class ReadFromS3Queue;
|
||||
using FileIterator = StorageS3QueueSource::FileIterator;
|
||||
|
||||
const std::unique_ptr<S3QueueSettings> s3queue_settings;
|
||||
@ -85,11 +87,10 @@ private:
|
||||
bool supportsSubsetOfColumns(const ContextPtr & context_) const;
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
|
||||
std::shared_ptr<FileIterator> createFileIterator(ContextPtr local_context, ASTPtr query);
|
||||
std::shared_ptr<FileIterator> createFileIterator(ContextPtr local_context, const ActionsDAG::Node * predicate);
|
||||
std::shared_ptr<StorageS3QueueSource> createSource(
|
||||
const ReadFromFormatInfo & info,
|
||||
std::shared_ptr<StorageS3Queue::FileIterator> file_iterator,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
size_t max_block_size,
|
||||
ContextPtr local_context);
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <Storages/StorageAzureBlob.h>
|
||||
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
@ -21,6 +20,9 @@
|
||||
#include <Processors/Transforms/ExtractColumnsTransform.h>
|
||||
#include <Processors/Formats/IOutputFormat.h>
|
||||
#include <Processors/Sources/ConstChunkGenerator.h>
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/SourceStepWithFilter.h>
|
||||
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/StorageSnapshot.h>
|
||||
@ -666,7 +668,58 @@ private:
|
||||
|
||||
}
|
||||
|
||||
Pipe StorageAzureBlob::read(
|
||||
class ReadFromAzureBlob : public SourceStepWithFilter
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "ReadFromAzureBlob"; }
|
||||
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
void applyFilters() override;
|
||||
|
||||
ReadFromAzureBlob(
|
||||
Block sample_block,
|
||||
std::shared_ptr<StorageAzureBlob> storage_,
|
||||
ReadFromFormatInfo info_,
|
||||
const bool need_only_count_,
|
||||
ContextPtr context_,
|
||||
size_t max_block_size_,
|
||||
size_t num_streams_)
|
||||
: SourceStepWithFilter(DataStream{.header = std::move(sample_block)})
|
||||
, storage(std::move(storage_))
|
||||
, info(std::move(info_))
|
||||
, need_only_count(need_only_count_)
|
||||
, context(std::move(context_))
|
||||
, max_block_size(max_block_size_)
|
||||
, num_streams(num_streams_)
|
||||
{
|
||||
}
|
||||
|
||||
private:
|
||||
std::shared_ptr<StorageAzureBlob> storage;
|
||||
ReadFromFormatInfo info;
|
||||
const bool need_only_count;
|
||||
|
||||
ContextPtr context;
|
||||
|
||||
size_t max_block_size;
|
||||
const size_t num_streams;
|
||||
|
||||
std::shared_ptr<StorageAzureBlobSource::IIterator> iterator_wrapper;
|
||||
|
||||
void createIterator(const ActionsDAG::Node * predicate);
|
||||
};
|
||||
|
||||
void ReadFromAzureBlob::applyFilters()
|
||||
{
|
||||
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context);
|
||||
const ActionsDAG::Node * predicate = nullptr;
|
||||
if (filter_actions_dag)
|
||||
predicate = filter_actions_dag->getOutputs().at(0);
|
||||
|
||||
createIterator(predicate);
|
||||
}
|
||||
|
||||
void StorageAzureBlob::read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
@ -678,51 +731,83 @@ Pipe StorageAzureBlob::read(
|
||||
if (partition_by && configuration.withWildcard())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned Azure storage is not implemented yet");
|
||||
|
||||
Pipes pipes;
|
||||
|
||||
std::shared_ptr<StorageAzureBlobSource::IIterator> iterator_wrapper;
|
||||
if (distributed_processing)
|
||||
{
|
||||
iterator_wrapper = std::make_shared<StorageAzureBlobSource::ReadIterator>(local_context,
|
||||
local_context->getReadTaskCallback());
|
||||
}
|
||||
else if (configuration.withGlobs())
|
||||
{
|
||||
/// Iterate through disclosed globs and make a source for each file
|
||||
iterator_wrapper = std::make_shared<StorageAzureBlobSource::GlobIterator>(
|
||||
object_storage.get(), configuration.container, configuration.blob_path,
|
||||
query_info.query, virtual_columns, local_context, nullptr, local_context->getFileProgressCallback());
|
||||
}
|
||||
else
|
||||
{
|
||||
iterator_wrapper = std::make_shared<StorageAzureBlobSource::KeysIterator>(
|
||||
object_storage.get(), configuration.container, configuration.blobs_paths,
|
||||
query_info.query, virtual_columns, local_context, nullptr, local_context->getFileProgressCallback());
|
||||
}
|
||||
auto this_ptr = std::static_pointer_cast<StorageAzureBlob>(shared_from_this());
|
||||
|
||||
auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals());
|
||||
bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty())
|
||||
&& local_context->getSettingsRef().optimize_count_from_files;
|
||||
|
||||
auto reading = std::make_unique<ReadFromAzureBlob>(
|
||||
read_from_format_info.source_header,
|
||||
std::move(this_ptr),
|
||||
std::move(read_from_format_info),
|
||||
need_only_count,
|
||||
local_context,
|
||||
max_block_size,
|
||||
num_streams);
|
||||
|
||||
query_plan.addStep(std::move(reading));
|
||||
}
|
||||
|
||||
void ReadFromAzureBlob::createIterator(const ActionsDAG::Node * predicate)
|
||||
{
|
||||
if (iterator_wrapper)
|
||||
return;
|
||||
|
||||
const auto & configuration = storage->configuration;
|
||||
|
||||
if (storage->distributed_processing)
|
||||
{
|
||||
iterator_wrapper = std::make_shared<StorageAzureBlobSource::ReadIterator>(context,
|
||||
context->getReadTaskCallback());
|
||||
}
|
||||
else if (configuration.withGlobs())
|
||||
{
|
||||
/// Iterate through disclosed globs and make a source for each file
|
||||
iterator_wrapper = std::make_shared<StorageAzureBlobSource::GlobIterator>(
|
||||
storage->object_storage.get(), configuration.container, configuration.blob_path,
|
||||
predicate, storage->virtual_columns, context, nullptr, context->getFileProgressCallback());
|
||||
}
|
||||
else
|
||||
{
|
||||
iterator_wrapper = std::make_shared<StorageAzureBlobSource::KeysIterator>(
|
||||
storage->object_storage.get(), configuration.container, configuration.blobs_paths,
|
||||
predicate, storage->virtual_columns, context, nullptr, context->getFileProgressCallback());
|
||||
}
|
||||
}
|
||||
|
||||
void ReadFromAzureBlob::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
createIterator(nullptr);
|
||||
|
||||
const auto & configuration = storage->configuration;
|
||||
Pipes pipes;
|
||||
|
||||
for (size_t i = 0; i < num_streams; ++i)
|
||||
{
|
||||
pipes.emplace_back(std::make_shared<StorageAzureBlobSource>(
|
||||
read_from_format_info,
|
||||
info,
|
||||
configuration.format,
|
||||
getName(),
|
||||
local_context,
|
||||
format_settings,
|
||||
context,
|
||||
storage->format_settings,
|
||||
max_block_size,
|
||||
configuration.compression_method,
|
||||
object_storage.get(),
|
||||
storage->object_storage.get(),
|
||||
configuration.container,
|
||||
configuration.connection_url,
|
||||
iterator_wrapper,
|
||||
need_only_count,
|
||||
query_info));
|
||||
need_only_count));
|
||||
}
|
||||
|
||||
return Pipe::unitePipes(std::move(pipes));
|
||||
auto pipe = Pipe::unitePipes(std::move(pipes));
|
||||
if (pipe.empty())
|
||||
pipe = Pipe(std::make_shared<NullSource>(info.source_header));
|
||||
|
||||
for (const auto & processor : pipe.getProcessors())
|
||||
processors.emplace_back(processor);
|
||||
|
||||
pipeline.init(std::move(pipe));
|
||||
}
|
||||
|
||||
SinkToStoragePtr StorageAzureBlob::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/)
|
||||
@ -829,7 +914,7 @@ StorageAzureBlobSource::GlobIterator::GlobIterator(
|
||||
AzureObjectStorage * object_storage_,
|
||||
const std::string & container_,
|
||||
String blob_path_with_globs_,
|
||||
ASTPtr query_,
|
||||
const ActionsDAG::Node * predicate,
|
||||
const NamesAndTypesList & virtual_columns_,
|
||||
ContextPtr context_,
|
||||
RelativePathsWithMetadata * outer_blobs_,
|
||||
@ -838,7 +923,6 @@ StorageAzureBlobSource::GlobIterator::GlobIterator(
|
||||
, object_storage(object_storage_)
|
||||
, container(container_)
|
||||
, blob_path_with_globs(blob_path_with_globs_)
|
||||
, query(query_)
|
||||
, virtual_columns(virtual_columns_)
|
||||
, outer_blobs(outer_blobs_)
|
||||
, file_progress_callback(file_progress_callback_)
|
||||
@ -870,6 +954,8 @@ StorageAzureBlobSource::GlobIterator::GlobIterator(
|
||||
ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", blob_path_with_globs, matcher->error());
|
||||
|
||||
recursive = blob_path_with_globs == "/**" ? true : false;
|
||||
|
||||
filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns);
|
||||
}
|
||||
|
||||
RelativePathWithMetadata StorageAzureBlobSource::GlobIterator::next()
|
||||
@ -909,20 +995,15 @@ RelativePathWithMetadata StorageAzureBlobSource::GlobIterator::next()
|
||||
}
|
||||
|
||||
index = 0;
|
||||
if (!is_initialized)
|
||||
{
|
||||
filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, fs::path(container) / new_batch.front().relative_path, getContext());
|
||||
is_initialized = true;
|
||||
}
|
||||
|
||||
if (filter_ast)
|
||||
if (filter_dag)
|
||||
{
|
||||
std::vector<String> paths;
|
||||
paths.reserve(new_batch.size());
|
||||
for (auto & path_with_metadata : new_batch)
|
||||
paths.push_back(fs::path(container) / path_with_metadata.relative_path);
|
||||
|
||||
VirtualColumnUtils::filterByPathOrFile(new_batch, paths, query, virtual_columns, getContext(), filter_ast);
|
||||
VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_dag, virtual_columns, getContext());
|
||||
}
|
||||
|
||||
if (outer_blobs)
|
||||
@ -948,7 +1029,7 @@ StorageAzureBlobSource::KeysIterator::KeysIterator(
|
||||
AzureObjectStorage * object_storage_,
|
||||
const std::string & container_,
|
||||
const Strings & keys_,
|
||||
ASTPtr query_,
|
||||
const ActionsDAG::Node * predicate,
|
||||
const NamesAndTypesList & virtual_columns_,
|
||||
ContextPtr context_,
|
||||
RelativePathsWithMetadata * outer_blobs,
|
||||
@ -956,23 +1037,22 @@ StorageAzureBlobSource::KeysIterator::KeysIterator(
|
||||
: IIterator(context_)
|
||||
, object_storage(object_storage_)
|
||||
, container(container_)
|
||||
, query(query_)
|
||||
, virtual_columns(virtual_columns_)
|
||||
{
|
||||
Strings all_keys = keys_;
|
||||
|
||||
ASTPtr filter_ast;
|
||||
if (!all_keys.empty())
|
||||
filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, fs::path(container) / all_keys[0], getContext());
|
||||
filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns);
|
||||
|
||||
if (filter_ast)
|
||||
if (filter_dag)
|
||||
{
|
||||
Strings paths;
|
||||
paths.reserve(all_keys.size());
|
||||
for (const auto & key : all_keys)
|
||||
paths.push_back(fs::path(container) / key);
|
||||
|
||||
VirtualColumnUtils::filterByPathOrFile(all_keys, paths, query, virtual_columns, getContext(), filter_ast);
|
||||
VirtualColumnUtils::filterByPathOrFile(all_keys, paths, filter_dag, virtual_columns, getContext());
|
||||
}
|
||||
|
||||
for (auto && key : all_keys)
|
||||
@ -1078,8 +1158,7 @@ StorageAzureBlobSource::StorageAzureBlobSource(
|
||||
const String & container_,
|
||||
const String & connection_url_,
|
||||
std::shared_ptr<IIterator> file_iterator_,
|
||||
bool need_only_count_,
|
||||
const SelectQueryInfo & query_info_)
|
||||
bool need_only_count_)
|
||||
:ISource(info.source_header, false)
|
||||
, WithContext(context_)
|
||||
, requested_columns(info.requested_columns)
|
||||
@ -1096,7 +1175,6 @@ StorageAzureBlobSource::StorageAzureBlobSource(
|
||||
, connection_url(connection_url_)
|
||||
, file_iterator(file_iterator_)
|
||||
, need_only_count(need_only_count_)
|
||||
, query_info(query_info_)
|
||||
, create_reader_pool(CurrentMetrics::ObjectStorageAzureThreads, CurrentMetrics::ObjectStorageAzureThreadsActive, CurrentMetrics::ObjectStorageAzureThreadsScheduled, 1)
|
||||
, create_reader_scheduler(threadPoolCallbackRunner<ReaderHolder>(create_reader_pool, "AzureReader"))
|
||||
{
|
||||
|
@ -88,7 +88,8 @@ public:
|
||||
return name;
|
||||
}
|
||||
|
||||
Pipe read(
|
||||
void read(
|
||||
QueryPlan & query_plan,
|
||||
const Names &,
|
||||
const StorageSnapshotPtr &,
|
||||
SelectQueryInfo &,
|
||||
@ -126,6 +127,8 @@ public:
|
||||
bool distributed_processing = false);
|
||||
|
||||
private:
|
||||
friend class ReadFromAzureBlob;
|
||||
|
||||
std::string name;
|
||||
Configuration configuration;
|
||||
std::unique_ptr<AzureObjectStorage> object_storage;
|
||||
@ -156,7 +159,7 @@ public:
|
||||
AzureObjectStorage * object_storage_,
|
||||
const std::string & container_,
|
||||
String blob_path_with_globs_,
|
||||
ASTPtr query_,
|
||||
const ActionsDAG::Node * predicate,
|
||||
const NamesAndTypesList & virtual_columns_,
|
||||
ContextPtr context_,
|
||||
RelativePathsWithMetadata * outer_blobs_,
|
||||
@ -169,8 +172,7 @@ public:
|
||||
AzureObjectStorage * object_storage;
|
||||
std::string container;
|
||||
String blob_path_with_globs;
|
||||
ASTPtr query;
|
||||
ASTPtr filter_ast;
|
||||
ActionsDAGPtr filter_dag;
|
||||
NamesAndTypesList virtual_columns;
|
||||
|
||||
size_t index = 0;
|
||||
@ -184,7 +186,6 @@ public:
|
||||
|
||||
void createFilterAST(const String & any_key);
|
||||
bool is_finished = false;
|
||||
bool is_initialized = false;
|
||||
std::mutex next_mutex;
|
||||
|
||||
std::function<void(FileProgress)> file_progress_callback;
|
||||
@ -212,7 +213,7 @@ public:
|
||||
AzureObjectStorage * object_storage_,
|
||||
const std::string & container_,
|
||||
const Strings & keys_,
|
||||
ASTPtr query_,
|
||||
const ActionsDAG::Node * predicate,
|
||||
const NamesAndTypesList & virtual_columns_,
|
||||
ContextPtr context_,
|
||||
RelativePathsWithMetadata * outer_blobs,
|
||||
@ -226,7 +227,7 @@ public:
|
||||
std::string container;
|
||||
RelativePathsWithMetadata keys;
|
||||
|
||||
ASTPtr query;
|
||||
ActionsDAGPtr filter_dag;
|
||||
NamesAndTypesList virtual_columns;
|
||||
|
||||
std::atomic<size_t> index = 0;
|
||||
@ -244,8 +245,7 @@ public:
|
||||
const String & container_,
|
||||
const String & connection_url_,
|
||||
std::shared_ptr<IIterator> file_iterator_,
|
||||
bool need_only_count_,
|
||||
const SelectQueryInfo & query_info_);
|
||||
bool need_only_count_);
|
||||
~StorageAzureBlobSource() override;
|
||||
|
||||
Chunk generate() override;
|
||||
@ -271,7 +271,6 @@ private:
|
||||
std::shared_ptr<IIterator> file_iterator;
|
||||
bool need_only_count;
|
||||
size_t total_rows_in_file = 0;
|
||||
SelectQueryInfo query_info;
|
||||
|
||||
struct ReaderHolder
|
||||
{
|
||||
|
@ -69,11 +69,11 @@ void StorageAzureBlobCluster::addColumnsStructureToQuery(ASTPtr & query, const S
|
||||
TableFunctionAzureBlobStorageCluster::addColumnsStructureToArguments(expression_list->children, structure, context);
|
||||
}
|
||||
|
||||
RemoteQueryExecutor::Extension StorageAzureBlobCluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const
|
||||
RemoteQueryExecutor::Extension StorageAzureBlobCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const
|
||||
{
|
||||
auto iterator = std::make_shared<StorageAzureBlobSource::GlobIterator>(
|
||||
object_storage.get(), configuration.container, configuration.blob_path,
|
||||
query, virtual_columns, context, nullptr);
|
||||
predicate, virtual_columns, context, nullptr);
|
||||
auto callback = std::make_shared<std::function<String()>>([iterator]() mutable -> String{ return iterator->next().relative_path; });
|
||||
return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) };
|
||||
}
|
||||
|
@ -34,7 +34,7 @@ public:
|
||||
|
||||
NamesAndTypesList getVirtuals() const override;
|
||||
|
||||
RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override;
|
||||
RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override;
|
||||
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
|
||||
|
@ -91,6 +91,7 @@
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
|
||||
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Processors/Sources/RemoteSource.h>
|
||||
#include <Processors/Sinks/EmptySink.h>
|
||||
@ -1068,15 +1069,67 @@ std::optional<QueryPipeline> StorageDistributed::distributedWriteBetweenDistribu
|
||||
return pipeline;
|
||||
}
|
||||
|
||||
static ActionsDAGPtr getFilterFromQuery(const ASTPtr & ast, ContextPtr context)
|
||||
{
|
||||
QueryPlan plan;
|
||||
SelectQueryOptions options;
|
||||
options.only_analyze = true;
|
||||
if (context->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
InterpreterSelectQueryAnalyzer interpreter(ast, context, options);
|
||||
plan = std::move(interpreter).extractQueryPlan();
|
||||
}
|
||||
else
|
||||
{
|
||||
InterpreterSelectWithUnionQuery interpreter(ast, context, options);
|
||||
interpreter.buildQueryPlan(plan);
|
||||
}
|
||||
|
||||
plan.optimize(QueryPlanOptimizationSettings::fromContext(context));
|
||||
|
||||
std::stack<QueryPlan::Node *> nodes;
|
||||
nodes.push(plan.getRootNode());
|
||||
|
||||
SourceStepWithFilter * source = nullptr;
|
||||
|
||||
while (!nodes.empty())
|
||||
{
|
||||
const auto * node = nodes.top();
|
||||
nodes.pop();
|
||||
|
||||
if (auto * with_filter = dynamic_cast<SourceStepWithFilter *>(node->step.get()))
|
||||
{
|
||||
if (source)
|
||||
{
|
||||
WriteBufferFromOwnString buf;
|
||||
plan.explainPlan(buf, {});
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Found multiple source steps for query\n{}\nPlan\n{}",
|
||||
queryToString(ast), buf.str());
|
||||
}
|
||||
|
||||
source = with_filter;
|
||||
}
|
||||
}
|
||||
|
||||
if (!source)
|
||||
return nullptr;
|
||||
|
||||
return ActionsDAG::buildFilterActionsDAG(source->getFilterNodes().nodes, {}, context);
|
||||
}
|
||||
|
||||
|
||||
std::optional<QueryPipeline> StorageDistributed::distributedWriteFromClusterStorage(const IStorageCluster & src_storage_cluster, const ASTInsertQuery & query, ContextPtr local_context) const
|
||||
{
|
||||
const auto & settings = local_context->getSettingsRef();
|
||||
auto & select = query.select->as<ASTSelectWithUnionQuery &>();
|
||||
|
||||
auto filter = getFilterFromQuery(query.select, local_context);
|
||||
const ActionsDAG::Node * predicate = nullptr;
|
||||
if (filter)
|
||||
predicate = filter->getOutputs().at(0);
|
||||
|
||||
/// Select query is needed for pruining on virtual columns
|
||||
auto extension = src_storage_cluster.getTaskIteratorExtension(
|
||||
select.list_of_selects->children.at(0)->as<ASTSelectQuery>()->clone(),
|
||||
local_context);
|
||||
auto extension = src_storage_cluster.getTaskIteratorExtension(predicate, local_context);
|
||||
|
||||
auto dst_cluster = getCluster();
|
||||
|
||||
|
@ -9,6 +9,7 @@
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier_fwd.h>
|
||||
@ -37,6 +38,8 @@
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Processors/Sources/ConstChunkGenerator.h>
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/SourceStepWithFilter.h>
|
||||
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -929,22 +932,21 @@ static std::chrono::seconds getLockTimeout(ContextPtr context)
|
||||
|
||||
using StorageFilePtr = std::shared_ptr<StorageFile>;
|
||||
|
||||
|
||||
StorageFileSource::FilesIterator::FilesIterator(
|
||||
const Strings & files_,
|
||||
std::optional<StorageFile::ArchiveInfo> archive_info_,
|
||||
ASTPtr query,
|
||||
const ActionsDAG::Node * predicate,
|
||||
const NamesAndTypesList & virtual_columns,
|
||||
ContextPtr context_,
|
||||
bool distributed_processing_)
|
||||
: files(files_), archive_info(std::move(archive_info_)), distributed_processing(distributed_processing_), context(context_)
|
||||
{
|
||||
ASTPtr filter_ast;
|
||||
if (!distributed_processing && !archive_info && !files.empty() && !files[0].empty())
|
||||
filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, files[0], context_);
|
||||
ActionsDAGPtr filter_dag;
|
||||
if (!distributed_processing && !archive_info && !files.empty())
|
||||
filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns);
|
||||
|
||||
if (filter_ast)
|
||||
VirtualColumnUtils::filterByPathOrFile(files, files, query, virtual_columns, context_, filter_ast);
|
||||
if (filter_dag)
|
||||
VirtualColumnUtils::filterByPathOrFile(files, files, filter_dag, virtual_columns, context_);
|
||||
}
|
||||
|
||||
String StorageFileSource::FilesIterator::next()
|
||||
@ -974,16 +976,13 @@ const String & StorageFileSource::FilesIterator::getFileNameInArchive()
|
||||
StorageFileSource::StorageFileSource(
|
||||
const ReadFromFormatInfo & info,
|
||||
std::shared_ptr<StorageFile> storage_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
ContextPtr context_,
|
||||
const SelectQueryInfo & query_info_,
|
||||
UInt64 max_block_size_,
|
||||
FilesIteratorPtr files_iterator_,
|
||||
std::unique_ptr<ReadBuffer> read_buf_,
|
||||
bool need_only_count_)
|
||||
: SourceWithKeyCondition(info.source_header, false)
|
||||
, storage(std::move(storage_))
|
||||
, storage_snapshot(storage_snapshot_)
|
||||
, files_iterator(std::move(files_iterator_))
|
||||
, read_buf(std::move(read_buf_))
|
||||
, columns_description(info.columns_description)
|
||||
@ -991,7 +990,6 @@ StorageFileSource::StorageFileSource(
|
||||
, requested_virtual_columns(info.requested_virtual_columns)
|
||||
, block_for_format(info.format_header)
|
||||
, context(context_)
|
||||
, query_info(query_info_)
|
||||
, max_block_size(max_block_size_)
|
||||
, need_only_count(need_only_count_)
|
||||
{
|
||||
@ -1322,14 +1320,64 @@ std::optional<size_t> StorageFileSource::tryGetNumRowsFromCache(const String & p
|
||||
return schema_cache.tryGetNumRows(key, get_last_mod_time);
|
||||
}
|
||||
|
||||
Pipe StorageFile::read(
|
||||
class ReadFromFile : public SourceStepWithFilter
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "ReadFromFile"; }
|
||||
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
void applyFilters() override;
|
||||
|
||||
ReadFromFile(
|
||||
Block sample_block,
|
||||
std::shared_ptr<StorageFile> storage_,
|
||||
ReadFromFormatInfo info_,
|
||||
const bool need_only_count_,
|
||||
ContextPtr context_,
|
||||
size_t max_block_size_,
|
||||
size_t num_streams_)
|
||||
: SourceStepWithFilter(DataStream{.header = std::move(sample_block)})
|
||||
, storage(std::move(storage_))
|
||||
, info(std::move(info_))
|
||||
, need_only_count(need_only_count_)
|
||||
, context(std::move(context_))
|
||||
, max_block_size(max_block_size_)
|
||||
, max_num_streams(num_streams_)
|
||||
{
|
||||
}
|
||||
|
||||
private:
|
||||
std::shared_ptr<StorageFile> storage;
|
||||
ReadFromFormatInfo info;
|
||||
const bool need_only_count;
|
||||
|
||||
ContextPtr context;
|
||||
size_t max_block_size;
|
||||
const size_t max_num_streams;
|
||||
|
||||
std::shared_ptr<StorageFileSource::FilesIterator> files_iterator;
|
||||
|
||||
void createIterator(const ActionsDAG::Node * predicate);
|
||||
};
|
||||
|
||||
void ReadFromFile::applyFilters()
|
||||
{
|
||||
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context);
|
||||
const ActionsDAG::Node * predicate = nullptr;
|
||||
if (filter_actions_dag)
|
||||
predicate = filter_actions_dag->getOutputs().at(0);
|
||||
|
||||
createIterator(predicate);
|
||||
}
|
||||
|
||||
void StorageFile::read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum /*processed_stage*/,
|
||||
size_t max_block_size,
|
||||
const size_t max_num_streams)
|
||||
size_t num_streams)
|
||||
{
|
||||
if (use_table_fd)
|
||||
{
|
||||
@ -1346,24 +1394,58 @@ Pipe StorageFile::read(
|
||||
|
||||
if (p->size() == 1 && !fs::exists(p->at(0)))
|
||||
{
|
||||
if (context->getSettingsRef().engine_file_empty_if_not_exists)
|
||||
return Pipe(std::make_shared<NullSource>(storage_snapshot->getSampleBlockForColumns(column_names)));
|
||||
else
|
||||
if (!context->getSettingsRef().engine_file_empty_if_not_exists)
|
||||
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File {} doesn't exist", p->at(0));
|
||||
|
||||
auto header = storage_snapshot->getSampleBlockForColumns(column_names);
|
||||
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, context);
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
auto files_iterator = std::make_shared<StorageFileSource::FilesIterator>(paths, archive_info, query_info.query, virtual_columns, context, distributed_processing);
|
||||
|
||||
auto this_ptr = std::static_pointer_cast<StorageFile>(shared_from_this());
|
||||
|
||||
auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context), getVirtuals());
|
||||
bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty())
|
||||
&& context->getSettingsRef().optimize_count_from_files;
|
||||
|
||||
auto reading = std::make_unique<ReadFromFile>(
|
||||
read_from_format_info.source_header,
|
||||
std::move(this_ptr),
|
||||
std::move(read_from_format_info),
|
||||
need_only_count,
|
||||
context,
|
||||
max_block_size,
|
||||
num_streams);
|
||||
|
||||
query_plan.addStep(std::move(reading));
|
||||
}
|
||||
|
||||
void ReadFromFile::createIterator(const ActionsDAG::Node * predicate)
|
||||
{
|
||||
if (files_iterator)
|
||||
return;
|
||||
|
||||
files_iterator = std::make_shared<StorageFileSource::FilesIterator>(
|
||||
storage->paths,
|
||||
storage->archive_info,
|
||||
predicate,
|
||||
storage->virtual_columns,
|
||||
context,
|
||||
storage->distributed_processing);
|
||||
}
|
||||
|
||||
void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
createIterator(nullptr);
|
||||
|
||||
size_t num_streams = max_num_streams;
|
||||
|
||||
size_t files_to_read = 0;
|
||||
if (archive_info)
|
||||
files_to_read = archive_info->paths_to_archives.size();
|
||||
if (storage->archive_info)
|
||||
files_to_read = storage->archive_info->paths_to_archives.size();
|
||||
else
|
||||
files_to_read = paths.size();
|
||||
files_to_read = storage->paths.size();
|
||||
|
||||
if (max_num_streams > files_to_read)
|
||||
num_streams = files_to_read;
|
||||
@ -1374,12 +1456,8 @@ Pipe StorageFile::read(
|
||||
/// Set total number of bytes to process. For progress bar.
|
||||
auto progress_callback = context->getFileProgressCallback();
|
||||
|
||||
if (progress_callback && !archive_info)
|
||||
progress_callback(FileProgress(0, total_bytes_to_read));
|
||||
|
||||
auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context), getVirtuals());
|
||||
bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty())
|
||||
&& context->getSettingsRef().optimize_count_from_files;
|
||||
if (progress_callback && !storage->archive_info)
|
||||
progress_callback(FileProgress(0, storage->total_bytes_to_read));
|
||||
|
||||
for (size_t i = 0; i < num_streams; ++i)
|
||||
{
|
||||
@ -1388,22 +1466,35 @@ Pipe StorageFile::read(
|
||||
/// If yes, then we should use it in StorageFileSource. Atomic bool flag is needed
|
||||
/// to prevent data race in case of parallel reads.
|
||||
std::unique_ptr<ReadBuffer> read_buffer;
|
||||
if (has_peekable_read_buffer_from_fd.exchange(false))
|
||||
read_buffer = std::move(peekable_read_buffer_from_fd);
|
||||
if (storage->has_peekable_read_buffer_from_fd.exchange(false))
|
||||
read_buffer = std::move(storage->peekable_read_buffer_from_fd);
|
||||
|
||||
pipes.emplace_back(std::make_shared<StorageFileSource>(
|
||||
read_from_format_info,
|
||||
this_ptr,
|
||||
storage_snapshot,
|
||||
auto source = std::make_shared<StorageFileSource>(
|
||||
info,
|
||||
storage,
|
||||
context,
|
||||
query_info,
|
||||
max_block_size,
|
||||
files_iterator,
|
||||
std::move(read_buffer),
|
||||
need_only_count));
|
||||
need_only_count);
|
||||
|
||||
source->setKeyCondition(filter_nodes.nodes, context);
|
||||
pipes.emplace_back(std::move(source));
|
||||
}
|
||||
|
||||
return Pipe::unitePipes(std::move(pipes));
|
||||
auto pipe = Pipe::unitePipes(std::move(pipes));
|
||||
size_t output_ports = pipe.numOutputPorts();
|
||||
const bool parallelize_output = context->getSettingsRef().parallelize_output_from_storages;
|
||||
if (parallelize_output && storage->parallelizeOutputAfterReading(context) && output_ports > 0 && output_ports < max_num_streams)
|
||||
pipe.resize(max_num_streams);
|
||||
|
||||
if (pipe.empty())
|
||||
pipe = Pipe(std::make_shared<NullSource>(info.source_header));
|
||||
|
||||
for (const auto & processor : pipe.getProcessors())
|
||||
processors.emplace_back(processor);
|
||||
|
||||
pipeline.init(std::move(pipe));
|
||||
}
|
||||
|
||||
|
||||
|
@ -53,7 +53,8 @@ public:
|
||||
|
||||
std::string getName() const override { return "File"; }
|
||||
|
||||
Pipe read(
|
||||
void read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
@ -137,6 +138,7 @@ public:
|
||||
protected:
|
||||
friend class StorageFileSource;
|
||||
friend class StorageFileSink;
|
||||
friend class ReadFromFile;
|
||||
|
||||
private:
|
||||
void setStorageMetadata(CommonArguments args);
|
||||
@ -194,7 +196,7 @@ public:
|
||||
explicit FilesIterator(
|
||||
const Strings & files_,
|
||||
std::optional<StorageFile::ArchiveInfo> archive_info_,
|
||||
ASTPtr query,
|
||||
const ActionsDAG::Node * predicate,
|
||||
const NamesAndTypesList & virtual_columns,
|
||||
ContextPtr context_,
|
||||
bool distributed_processing_ = false);
|
||||
@ -234,9 +236,7 @@ private:
|
||||
StorageFileSource(
|
||||
const ReadFromFormatInfo & info,
|
||||
std::shared_ptr<StorageFile> storage_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
ContextPtr context_,
|
||||
const SelectQueryInfo & query_info_,
|
||||
UInt64 max_block_size_,
|
||||
FilesIteratorPtr files_iterator_,
|
||||
std::unique_ptr<ReadBuffer> read_buf_,
|
||||
@ -269,7 +269,6 @@ private:
|
||||
std::optional<size_t> tryGetNumRowsFromCache(const String & path, time_t last_mod_time) const;
|
||||
|
||||
std::shared_ptr<StorageFile> storage;
|
||||
StorageSnapshotPtr storage_snapshot;
|
||||
FilesIteratorPtr files_iterator;
|
||||
String current_path;
|
||||
std::optional<size_t> current_file_size;
|
||||
@ -290,7 +289,6 @@ private:
|
||||
Block block_for_format;
|
||||
|
||||
ContextPtr context; /// TODO Untangle potential issues with context lifetime.
|
||||
SelectQueryInfo query_info;
|
||||
UInt64 max_block_size;
|
||||
|
||||
bool finished_generate = false;
|
||||
|
@ -71,9 +71,9 @@ void StorageFileCluster::addColumnsStructureToQuery(ASTPtr & query, const String
|
||||
TableFunctionFileCluster::addColumnsStructureToArguments(expression_list->children, structure, context);
|
||||
}
|
||||
|
||||
RemoteQueryExecutor::Extension StorageFileCluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const
|
||||
RemoteQueryExecutor::Extension StorageFileCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const
|
||||
{
|
||||
auto iterator = std::make_shared<StorageFileSource::FilesIterator>(paths, std::nullopt, query, virtual_columns, context);
|
||||
auto iterator = std::make_shared<StorageFileSource::FilesIterator>(paths, std::nullopt, predicate, virtual_columns, context);
|
||||
auto callback = std::make_shared<TaskIterator>([iter = std::move(iterator)]() mutable -> String { return iter->next(); });
|
||||
return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)};
|
||||
}
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
|
||||
NamesAndTypesList getVirtuals() const override { return virtual_columns; }
|
||||
|
||||
RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override;
|
||||
RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override;
|
||||
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
|
||||
|
@ -73,6 +73,7 @@ public:
|
||||
|
||||
StoragePtr getTargetTable() const;
|
||||
StoragePtr tryGetTargetTable() const;
|
||||
StorageID getTargetTableId() const;
|
||||
|
||||
/// Get the virtual column of the target table;
|
||||
NamesAndTypesList getVirtuals() const override;
|
||||
@ -119,7 +120,6 @@ private:
|
||||
std::tuple<ContextMutablePtr, std::shared_ptr<ASTInsertQuery>> prepareRefresh() const;
|
||||
StorageID exchangeTargetTable(StorageID fresh_table, ContextPtr refresh_context);
|
||||
|
||||
StorageID getTargetTableId() const;
|
||||
void setTargetTableId(StorageID id);
|
||||
void updateTargetTableId(std::optional<String> database_name, std::optional<String> table_name);
|
||||
};
|
||||
|
@ -1,6 +1,4 @@
|
||||
#include "config.h"
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include "Parsers/ASTCreateQuery.h"
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
@ -16,6 +14,7 @@
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/StorageS3.h>
|
||||
@ -42,6 +41,7 @@
|
||||
#include <Processors/Formats/IOutputFormat.h>
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Processors/Sources/ConstChunkGenerator.h>
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Processors/QueryPlan/SourceStepWithFilter.h>
|
||||
|
||||
|
||||
@ -57,6 +57,7 @@
|
||||
#include <Common/parseGlobs.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
|
||||
#include <Processors/ISource.h>
|
||||
#include <Processors/Sinks/SinkToStorage.h>
|
||||
@ -146,7 +147,8 @@ public:
|
||||
const Names & column_names_,
|
||||
StorageSnapshotPtr storage_snapshot_,
|
||||
StorageS3 & storage_,
|
||||
SelectQueryInfo query_info_,
|
||||
ReadFromFormatInfo read_from_format_info_,
|
||||
bool need_only_count_,
|
||||
ContextPtr context_,
|
||||
size_t max_block_size_,
|
||||
size_t num_streams_)
|
||||
@ -154,106 +156,36 @@ public:
|
||||
, column_names(column_names_)
|
||||
, storage_snapshot(std::move(storage_snapshot_))
|
||||
, storage(storage_)
|
||||
, query_info(std::move(query_info_))
|
||||
, read_from_format_info(std::move(read_from_format_info_))
|
||||
, need_only_count(need_only_count_)
|
||||
, local_context(std::move(context_))
|
||||
, max_block_size(max_block_size_)
|
||||
, num_streams(num_streams_)
|
||||
{
|
||||
query_configuration = storage.updateConfigurationAndGetCopy(local_context);
|
||||
virtual_columns = storage.getVirtuals();
|
||||
}
|
||||
|
||||
private:
|
||||
Names column_names;
|
||||
StorageSnapshotPtr storage_snapshot;
|
||||
StorageS3 & storage;
|
||||
SelectQueryInfo query_info;
|
||||
ReadFromFormatInfo read_from_format_info;
|
||||
bool need_only_count;
|
||||
StorageS3::Configuration query_configuration;
|
||||
NamesAndTypesList virtual_columns;
|
||||
|
||||
ContextPtr local_context;
|
||||
|
||||
size_t max_block_size;
|
||||
size_t num_streams;
|
||||
|
||||
std::shared_ptr<StorageS3Source::IIterator> iterator_wrapper;
|
||||
|
||||
void createIterator(const ActionsDAG::Node * predicate);
|
||||
};
|
||||
|
||||
|
||||
static Block getBlockWithVirtuals(const NamesAndTypesList & virtual_columns, const String & bucket, const std::unordered_set<String> & keys)
|
||||
{
|
||||
Block virtual_columns_block;
|
||||
fs::path bucket_path(bucket);
|
||||
|
||||
for (const auto & [column_name, column_type] : virtual_columns)
|
||||
{
|
||||
if (column_name == "_path")
|
||||
{
|
||||
auto column = column_type->createColumn();
|
||||
for (const auto & key : keys)
|
||||
column->insert((bucket_path / key).string());
|
||||
virtual_columns_block.insert({std::move(column), column_type, column_name});
|
||||
}
|
||||
else if (column_name == "_file")
|
||||
{
|
||||
auto column = column_type->createColumn();
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
auto pos = key.find_last_of('/');
|
||||
if (pos != std::string::npos)
|
||||
column->insert(key.substr(pos + 1));
|
||||
else
|
||||
column->insert(key);
|
||||
}
|
||||
virtual_columns_block.insert({std::move(column), column_type, column_name});
|
||||
}
|
||||
else if (column_name == "_key")
|
||||
{
|
||||
auto column = column_type->createColumn();
|
||||
for (const auto & key : keys)
|
||||
column->insert(key);
|
||||
virtual_columns_block.insert({std::move(column), column_type, column_name});
|
||||
}
|
||||
else
|
||||
{
|
||||
auto column = column_type->createColumn();
|
||||
column->insertManyDefaults(keys.size());
|
||||
virtual_columns_block.insert({std::move(column), column_type, column_name});
|
||||
}
|
||||
}
|
||||
|
||||
/// Column _key is mandatory and may not be in virtual_columns list
|
||||
if (!virtual_columns_block.has("_key"))
|
||||
{
|
||||
auto column_type = std::make_shared<DataTypeString>();
|
||||
auto column = column_type->createColumn(); for (const auto & key : keys)
|
||||
column->insert(key);
|
||||
virtual_columns_block.insert({std::move(column), column_type, "_key"});
|
||||
}
|
||||
|
||||
return virtual_columns_block;
|
||||
}
|
||||
|
||||
static std::vector<String> filterKeysForPartitionPruning(
|
||||
const std::vector<String> & keys,
|
||||
const String & bucket,
|
||||
const NamesAndTypesList & virtual_columns,
|
||||
const std::vector<ActionsDAGPtr> & filter_dags,
|
||||
ContextPtr context)
|
||||
{
|
||||
std::unordered_set<String> result_keys(keys.begin(), keys.end());
|
||||
for (const auto & filter_dag : filter_dags)
|
||||
{
|
||||
if (result_keys.empty())
|
||||
break;
|
||||
|
||||
auto block = getBlockWithVirtuals(virtual_columns, bucket, result_keys);
|
||||
|
||||
auto filter_actions = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), block);
|
||||
if (!filter_actions)
|
||||
continue;
|
||||
VirtualColumnUtils::filterBlockWithDAG(filter_actions, block, context);
|
||||
|
||||
result_keys = VirtualColumnUtils::extractSingleValueFromBlock<String>(block, "_key");
|
||||
}
|
||||
|
||||
LOG_DEBUG(&Poco::Logger::get("StorageS3"), "Applied partition pruning {} from {} keys left", result_keys.size(), keys.size());
|
||||
return std::vector<String>(result_keys.begin(), result_keys.end());
|
||||
}
|
||||
|
||||
class IOutputFormat;
|
||||
using OutputFormatPtr = std::shared_ptr<IOutputFormat>;
|
||||
|
||||
@ -263,7 +195,7 @@ public:
|
||||
Impl(
|
||||
const S3::Client & client_,
|
||||
const S3::URI & globbed_uri_,
|
||||
ASTPtr & query_,
|
||||
const ActionsDAG::Node * predicate,
|
||||
const NamesAndTypesList & virtual_columns_,
|
||||
ContextPtr context_,
|
||||
KeysWithInfo * read_keys_,
|
||||
@ -272,7 +204,6 @@ public:
|
||||
: WithContext(context_)
|
||||
, client(client_.clone())
|
||||
, globbed_uri(globbed_uri_)
|
||||
, query(query_)
|
||||
, virtual_columns(virtual_columns_)
|
||||
, read_keys(read_keys_)
|
||||
, request_settings(request_settings_)
|
||||
@ -306,6 +237,8 @@ public:
|
||||
"Cannot compile regex from glob ({}): {}", globbed_uri.key, matcher->error());
|
||||
|
||||
recursive = globbed_uri.key == "/**" ? true : false;
|
||||
|
||||
filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns);
|
||||
fillInternalBufferAssumeLocked();
|
||||
}
|
||||
|
||||
@ -424,20 +357,14 @@ private:
|
||||
return;
|
||||
}
|
||||
|
||||
if (!is_initialized)
|
||||
{
|
||||
filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, fs::path(globbed_uri.bucket) / temp_buffer.front()->key, getContext());
|
||||
is_initialized = true;
|
||||
}
|
||||
|
||||
if (filter_ast)
|
||||
if (filter_dag)
|
||||
{
|
||||
std::vector<String> paths;
|
||||
paths.reserve(temp_buffer.size());
|
||||
for (const auto & key_with_info : temp_buffer)
|
||||
paths.push_back(fs::path(globbed_uri.bucket) / key_with_info->key);
|
||||
|
||||
VirtualColumnUtils::filterByPathOrFile(temp_buffer, paths, query, virtual_columns, getContext(), filter_ast);
|
||||
VirtualColumnUtils::filterByPathOrFile(temp_buffer, paths, filter_dag, virtual_columns, getContext());
|
||||
}
|
||||
|
||||
buffer = std::move(temp_buffer);
|
||||
@ -479,8 +406,7 @@ private:
|
||||
S3::URI globbed_uri;
|
||||
ASTPtr query;
|
||||
NamesAndTypesList virtual_columns;
|
||||
bool is_initialized{false};
|
||||
ASTPtr filter_ast;
|
||||
ActionsDAGPtr filter_dag;
|
||||
std::unique_ptr<re2::RE2> matcher;
|
||||
bool recursive{false};
|
||||
bool is_finished{false};
|
||||
@ -498,13 +424,13 @@ private:
|
||||
StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator(
|
||||
const S3::Client & client_,
|
||||
const S3::URI & globbed_uri_,
|
||||
ASTPtr query,
|
||||
const ActionsDAG::Node * predicate,
|
||||
const NamesAndTypesList & virtual_columns_,
|
||||
ContextPtr context,
|
||||
KeysWithInfo * read_keys_,
|
||||
const S3Settings::RequestSettings & request_settings_,
|
||||
std::function<void(FileProgress)> file_progress_callback_)
|
||||
: pimpl(std::make_shared<StorageS3Source::DisclosedGlobIterator::Impl>(client_, globbed_uri_, query, virtual_columns_, context, read_keys_, request_settings_, file_progress_callback_))
|
||||
: pimpl(std::make_shared<StorageS3Source::DisclosedGlobIterator::Impl>(client_, globbed_uri_, predicate, virtual_columns_, context, read_keys_, request_settings_, file_progress_callback_))
|
||||
{
|
||||
}
|
||||
|
||||
@ -646,8 +572,7 @@ StorageS3Source::StorageS3Source(
|
||||
const String & url_host_and_port_,
|
||||
std::shared_ptr<IIterator> file_iterator_,
|
||||
const size_t max_parsing_threads_,
|
||||
bool need_only_count_,
|
||||
std::optional<SelectQueryInfo> query_info_)
|
||||
bool need_only_count_)
|
||||
: SourceWithKeyCondition(info.source_header, false)
|
||||
, WithContext(context_)
|
||||
, name(std::move(name_))
|
||||
@ -663,7 +588,6 @@ StorageS3Source::StorageS3Source(
|
||||
, client(client_)
|
||||
, sample_block(info.format_header)
|
||||
, format_settings(format_settings_)
|
||||
, query_info(std::move(query_info_))
|
||||
, requested_virtual_columns(info.requested_virtual_columns)
|
||||
, file_iterator(file_iterator_)
|
||||
, max_parsing_threads(max_parsing_threads_)
|
||||
@ -1151,8 +1075,7 @@ static std::shared_ptr<StorageS3Source::IIterator> createFileIterator(
|
||||
const StorageS3::Configuration & configuration,
|
||||
bool distributed_processing,
|
||||
ContextPtr local_context,
|
||||
ASTPtr query,
|
||||
const std::vector<ActionsDAGPtr> & filter_dags,
|
||||
const ActionsDAG::Node * predicate,
|
||||
const NamesAndTypesList & virtual_columns,
|
||||
StorageS3::KeysWithInfo * read_keys = nullptr,
|
||||
std::function<void(FileProgress)> file_progress_callback = {})
|
||||
@ -1165,12 +1088,22 @@ static std::shared_ptr<StorageS3Source::IIterator> createFileIterator(
|
||||
{
|
||||
/// Iterate through disclosed globs and make a source for each file
|
||||
return std::make_shared<StorageS3Source::DisclosedGlobIterator>(
|
||||
*configuration.client, configuration.url, query, virtual_columns,
|
||||
*configuration.client, configuration.url, predicate, virtual_columns,
|
||||
local_context, read_keys, configuration.request_settings, file_progress_callback);
|
||||
}
|
||||
else
|
||||
{
|
||||
Strings keys = filterKeysForPartitionPruning(configuration.keys, configuration.url.bucket, virtual_columns, filter_dags, local_context);
|
||||
Strings keys = configuration.keys;
|
||||
auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns);
|
||||
if (filter_dag)
|
||||
{
|
||||
std::vector<String> paths;
|
||||
paths.reserve(keys.size());
|
||||
for (const auto & key : keys)
|
||||
paths.push_back(fs::path(configuration.url.bucket) / key);
|
||||
VirtualColumnUtils::filterByPathOrFile(keys, paths, filter_dag, virtual_columns, local_context);
|
||||
}
|
||||
|
||||
return std::make_shared<StorageS3Source::KeysIterator>(
|
||||
*configuration.client, configuration.url.version_id, keys,
|
||||
configuration.url.bucket, configuration.request_settings, read_keys, file_progress_callback);
|
||||
@ -1204,12 +1137,16 @@ void StorageS3::read(
|
||||
{
|
||||
auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), virtual_columns);
|
||||
|
||||
bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty())
|
||||
&& local_context->getSettingsRef().optimize_count_from_files;
|
||||
|
||||
auto reading = std::make_unique<ReadFromStorageS3Step>(
|
||||
read_from_format_info.source_header,
|
||||
column_names,
|
||||
storage_snapshot,
|
||||
*this,
|
||||
query_info,
|
||||
std::move(read_from_format_info),
|
||||
need_only_count,
|
||||
local_context,
|
||||
max_block_size,
|
||||
num_streams);
|
||||
@ -1217,19 +1154,32 @@ void StorageS3::read(
|
||||
query_plan.addStep(std::move(reading));
|
||||
}
|
||||
|
||||
void ReadFromStorageS3Step::applyFilters()
|
||||
{
|
||||
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, local_context);
|
||||
const ActionsDAG::Node * predicate = nullptr;
|
||||
if (filter_actions_dag)
|
||||
predicate = filter_actions_dag->getOutputs().at(0);
|
||||
|
||||
createIterator(predicate);
|
||||
}
|
||||
|
||||
void ReadFromStorageS3Step::createIterator(const ActionsDAG::Node * predicate)
|
||||
{
|
||||
if (iterator_wrapper)
|
||||
return;
|
||||
|
||||
iterator_wrapper = createFileIterator(
|
||||
query_configuration, storage.distributed_processing, local_context, predicate,
|
||||
virtual_columns, nullptr, local_context->getFileProgressCallback());
|
||||
}
|
||||
|
||||
void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
auto query_configuration = storage.updateConfigurationAndGetCopy(local_context);
|
||||
|
||||
if (storage.partition_by && query_configuration.withWildcard())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned S3 storage is not implemented yet");
|
||||
|
||||
auto virtual_columns = storage.getVirtuals();
|
||||
auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, storage.supportsSubsetOfColumns(local_context), virtual_columns);
|
||||
|
||||
std::shared_ptr<StorageS3Source::IIterator> iterator_wrapper = createFileIterator(
|
||||
query_configuration, storage.distributed_processing, local_context, query_info.query, filter_dags,
|
||||
virtual_columns, nullptr, local_context->getFileProgressCallback());
|
||||
createIterator(nullptr);
|
||||
|
||||
size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount();
|
||||
if (estimated_keys_count > 1)
|
||||
@ -1238,9 +1188,6 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline,
|
||||
/// Disclosed glob iterator can underestimate the amount of keys in some cases. We will keep one stream for this particular case.
|
||||
num_streams = 1;
|
||||
|
||||
bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty())
|
||||
&& local_context->getSettingsRef().optimize_count_from_files;
|
||||
|
||||
const size_t max_threads = local_context->getSettingsRef().max_threads;
|
||||
const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / std::max(num_streams, 1ul));
|
||||
LOG_DEBUG(&Poco::Logger::get("StorageS3"), "Reading in {} streams, {} threads per stream", num_streams, max_parsing_threads);
|
||||
@ -1249,7 +1196,7 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline,
|
||||
pipes.reserve(num_streams);
|
||||
for (size_t i = 0; i < num_streams; ++i)
|
||||
{
|
||||
pipes.emplace_back(std::make_shared<StorageS3Source>(
|
||||
auto source = std::make_shared<StorageS3Source>(
|
||||
read_from_format_info,
|
||||
query_configuration.format,
|
||||
storage.getName(),
|
||||
@ -1264,17 +1211,20 @@ void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline,
|
||||
query_configuration.url.uri.getHost() + std::to_string(query_configuration.url.uri.getPort()),
|
||||
iterator_wrapper,
|
||||
max_parsing_threads,
|
||||
need_only_count,
|
||||
query_info));
|
||||
need_only_count);
|
||||
|
||||
source->setKeyCondition(filter_nodes.nodes, local_context);
|
||||
pipes.emplace_back(std::move(source));
|
||||
}
|
||||
|
||||
pipeline.init(Pipe::unitePipes(std::move(pipes)));
|
||||
}
|
||||
auto pipe = Pipe::unitePipes(std::move(pipes));
|
||||
if (pipe.empty())
|
||||
pipe = Pipe(std::make_shared<NullSource>(read_from_format_info.source_header));
|
||||
|
||||
for (const auto & processor : pipe.getProcessors())
|
||||
processors.emplace_back(processor);
|
||||
|
||||
void ReadFromStorageS3Step::applyFilters()
|
||||
{
|
||||
/// We will use filter_dags in filterKeysForPartitionPruning called from initializePipeline, nothing to do here
|
||||
pipeline.init(std::move(pipe));
|
||||
}
|
||||
|
||||
SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/)
|
||||
@ -1858,7 +1808,7 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl(
|
||||
{
|
||||
KeysWithInfo read_keys;
|
||||
|
||||
auto file_iterator = createFileIterator(configuration, false, ctx, {}, {}, {}, &read_keys);
|
||||
auto file_iterator = createFileIterator(configuration, false, ctx, {}, {}, &read_keys);
|
||||
|
||||
ReadBufferIterator read_buffer_iterator(file_iterator, read_keys, configuration, format_settings, ctx);
|
||||
return readSchemaFromFormat(configuration.format, format_settings, read_buffer_iterator, configuration.withGlobs(), ctx);
|
||||
|
@ -78,7 +78,7 @@ public:
|
||||
DisclosedGlobIterator(
|
||||
const S3::Client & client_,
|
||||
const S3::URI & globbed_uri_,
|
||||
ASTPtr query,
|
||||
const ActionsDAG::Node * predicate,
|
||||
const NamesAndTypesList & virtual_columns,
|
||||
ContextPtr context,
|
||||
KeysWithInfo * read_keys_ = nullptr,
|
||||
@ -145,8 +145,7 @@ public:
|
||||
const String & url_host_and_port,
|
||||
std::shared_ptr<IIterator> file_iterator_,
|
||||
size_t max_parsing_threads,
|
||||
bool need_only_count_,
|
||||
std::optional<SelectQueryInfo> query_info);
|
||||
bool need_only_count_);
|
||||
|
||||
~StorageS3Source() override;
|
||||
|
||||
@ -180,7 +179,6 @@ private:
|
||||
std::shared_ptr<const S3::Client> client;
|
||||
Block sample_block;
|
||||
std::optional<FormatSettings> format_settings;
|
||||
std::optional<SelectQueryInfo> query_info;
|
||||
|
||||
struct ReaderHolder
|
||||
{
|
||||
|
@ -78,10 +78,10 @@ void StorageS3Cluster::updateConfigurationIfChanged(ContextPtr local_context)
|
||||
s3_configuration.update(local_context);
|
||||
}
|
||||
|
||||
RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const
|
||||
RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const
|
||||
{
|
||||
auto iterator = std::make_shared<StorageS3Source::DisclosedGlobIterator>(
|
||||
*s3_configuration.client, s3_configuration.url, query, virtual_columns, context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback());
|
||||
*s3_configuration.client, s3_configuration.url, predicate, virtual_columns, context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback());
|
||||
|
||||
auto callback = std::make_shared<std::function<String()>>([iterator]() mutable -> String
|
||||
{
|
||||
|
@ -34,7 +34,7 @@ public:
|
||||
|
||||
NamesAndTypesList getVirtuals() const override;
|
||||
|
||||
RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override;
|
||||
RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override;
|
||||
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
|
||||
|
@ -26,6 +26,8 @@
|
||||
#include <Processors/Transforms/AddingDefaultsTransform.h>
|
||||
#include <Processors/Transforms/ExtractColumnsTransform.h>
|
||||
#include <Processors/Sources/ConstChunkGenerator.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/SourceStepWithFilter.h>
|
||||
|
||||
#include <Common/ThreadStatus.h>
|
||||
#include <Common/parseRemoteDescription.h>
|
||||
@ -182,22 +184,22 @@ namespace
|
||||
class StorageURLSource::DisclosedGlobIterator::Impl
|
||||
{
|
||||
public:
|
||||
Impl(const String & uri_, size_t max_addresses, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
Impl(const String & uri_, size_t max_addresses, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
{
|
||||
uris = parseRemoteDescription(uri_, 0, uri_.size(), ',', max_addresses);
|
||||
|
||||
ASTPtr filter_ast;
|
||||
ActionsDAGPtr filter_dag;
|
||||
if (!uris.empty())
|
||||
filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, Poco::URI(uris[0]).getPath(), context);
|
||||
filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns);
|
||||
|
||||
if (filter_ast)
|
||||
if (filter_dag)
|
||||
{
|
||||
std::vector<String> paths;
|
||||
paths.reserve(uris.size());
|
||||
for (const auto & uri : uris)
|
||||
paths.push_back(Poco::URI(uri).getPath());
|
||||
|
||||
VirtualColumnUtils::filterByPathOrFile(uris, paths, query, virtual_columns, context, filter_ast);
|
||||
VirtualColumnUtils::filterByPathOrFile(uris, paths, filter_dag, virtual_columns, context);
|
||||
}
|
||||
}
|
||||
|
||||
@ -220,8 +222,8 @@ private:
|
||||
std::atomic_size_t index = 0;
|
||||
};
|
||||
|
||||
StorageURLSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, size_t max_addresses, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
: pimpl(std::make_shared<StorageURLSource::DisclosedGlobIterator::Impl>(uri, max_addresses, query, virtual_columns, context)) {}
|
||||
StorageURLSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, size_t max_addresses, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
: pimpl(std::make_shared<StorageURLSource::DisclosedGlobIterator::Impl>(uri, max_addresses, predicate, virtual_columns, context)) {}
|
||||
|
||||
String StorageURLSource::DisclosedGlobIterator::next()
|
||||
{
|
||||
@ -260,7 +262,6 @@ StorageURLSource::StorageURLSource(
|
||||
const ConnectionTimeouts & timeouts,
|
||||
CompressionMethod compression_method,
|
||||
size_t max_parsing_threads,
|
||||
const SelectQueryInfo &,
|
||||
const HTTPHeaderEntries & headers_,
|
||||
const URIParams & params,
|
||||
bool glob_url,
|
||||
@ -874,7 +875,70 @@ bool IStorageURLBase::parallelizeOutputAfterReading(ContextPtr context) const
|
||||
return FormatFactory::instance().checkParallelizeOutputAfterReading(format_name, context);
|
||||
}
|
||||
|
||||
Pipe IStorageURLBase::read(
|
||||
class ReadFromURL : public SourceStepWithFilter
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "ReadFromURL"; }
|
||||
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
void applyFilters() override;
|
||||
|
||||
ReadFromURL(
|
||||
Block sample_block,
|
||||
std::shared_ptr<IStorageURLBase> storage_,
|
||||
std::vector<String> * uri_options_,
|
||||
ReadFromFormatInfo info_,
|
||||
const bool need_only_count_,
|
||||
std::vector<std::pair<std::string, std::string>> read_uri_params_,
|
||||
std::function<void(std::ostream &)> read_post_data_callback_,
|
||||
ContextPtr context_,
|
||||
size_t max_block_size_,
|
||||
size_t num_streams_)
|
||||
: SourceStepWithFilter(DataStream{.header = std::move(sample_block)})
|
||||
, storage(std::move(storage_))
|
||||
, uri_options(uri_options_)
|
||||
, info(std::move(info_))
|
||||
, need_only_count(need_only_count_)
|
||||
, read_uri_params(std::move(read_uri_params_))
|
||||
, read_post_data_callback(std::move(read_post_data_callback_))
|
||||
, context(std::move(context_))
|
||||
, max_block_size(max_block_size_)
|
||||
, num_streams(num_streams_)
|
||||
{
|
||||
}
|
||||
|
||||
private:
|
||||
std::shared_ptr<IStorageURLBase> storage;
|
||||
std::vector<String> * uri_options;
|
||||
|
||||
ReadFromFormatInfo info;
|
||||
const bool need_only_count;
|
||||
std::vector<std::pair<std::string, std::string>> read_uri_params;
|
||||
std::function<void(std::ostream &)> read_post_data_callback;
|
||||
|
||||
ContextPtr context;
|
||||
|
||||
size_t max_block_size;
|
||||
size_t num_streams;
|
||||
|
||||
std::shared_ptr<StorageURLSource::IteratorWrapper> iterator_wrapper;
|
||||
bool is_url_with_globs = false;
|
||||
bool is_empty_glob = false;
|
||||
|
||||
void createIterator(const ActionsDAG::Node * predicate);
|
||||
};
|
||||
|
||||
void ReadFromURL::applyFilters()
|
||||
{
|
||||
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context);
|
||||
const ActionsDAG::Node * predicate = nullptr;
|
||||
if (filter_actions_dag)
|
||||
predicate = filter_actions_dag->getOutputs().at(0);
|
||||
|
||||
createIterator(predicate);
|
||||
}
|
||||
|
||||
void IStorageURLBase::read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
@ -884,16 +948,61 @@ Pipe IStorageURLBase::read(
|
||||
size_t num_streams)
|
||||
{
|
||||
auto params = getReadURIParams(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size);
|
||||
|
||||
std::shared_ptr<StorageURLSource::IteratorWrapper> iterator_wrapper{nullptr};
|
||||
bool is_url_with_globs = urlWithGlobs(uri);
|
||||
size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements;
|
||||
auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals());
|
||||
|
||||
if (distributed_processing)
|
||||
bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty())
|
||||
&& local_context->getSettingsRef().optimize_count_from_files;
|
||||
|
||||
auto read_post_data_callback = getReadPOSTDataCallback(
|
||||
read_from_format_info.columns_description.getNamesOfPhysical(),
|
||||
read_from_format_info.columns_description,
|
||||
query_info,
|
||||
local_context,
|
||||
processed_stage,
|
||||
max_block_size);
|
||||
|
||||
auto this_ptr = std::static_pointer_cast<IStorageURLBase>(shared_from_this());
|
||||
|
||||
auto reading = std::make_unique<ReadFromURL>(
|
||||
read_from_format_info.source_header,
|
||||
std::move(this_ptr),
|
||||
nullptr,
|
||||
std::move(read_from_format_info),
|
||||
need_only_count,
|
||||
std::move(params),
|
||||
std::move(read_post_data_callback),
|
||||
local_context,
|
||||
max_block_size,
|
||||
num_streams);
|
||||
|
||||
query_plan.addStep(std::move(reading));
|
||||
}
|
||||
|
||||
void ReadFromURL::createIterator(const ActionsDAG::Node * predicate)
|
||||
{
|
||||
if (iterator_wrapper || is_empty_glob)
|
||||
return;
|
||||
|
||||
if (uri_options)
|
||||
{
|
||||
iterator_wrapper = std::make_shared<StorageURLSource::IteratorWrapper>([&, done = false]() mutable
|
||||
{
|
||||
if (done)
|
||||
return StorageURLSource::FailoverOptions{};
|
||||
done = true;
|
||||
return *uri_options;
|
||||
});
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
|
||||
is_url_with_globs = urlWithGlobs(storage->uri);
|
||||
|
||||
if (storage->distributed_processing)
|
||||
{
|
||||
iterator_wrapper = std::make_shared<StorageURLSource::IteratorWrapper>(
|
||||
[callback = local_context->getReadTaskCallback(), max_addresses]()
|
||||
[callback = context->getReadTaskCallback(), max_addresses]()
|
||||
{
|
||||
String next_uri = callback();
|
||||
if (next_uri.empty())
|
||||
@ -904,11 +1013,14 @@ Pipe IStorageURLBase::read(
|
||||
else if (is_url_with_globs)
|
||||
{
|
||||
/// Iterate through disclosed globs and make a source for each file
|
||||
auto glob_iterator = std::make_shared<StorageURLSource::DisclosedGlobIterator>(uri, max_addresses, query_info.query, virtual_columns, local_context);
|
||||
auto glob_iterator = std::make_shared<StorageURLSource::DisclosedGlobIterator>(storage->uri, max_addresses, predicate, storage->virtual_columns, context);
|
||||
|
||||
/// check if we filtered out all the paths
|
||||
if (glob_iterator->size() == 0)
|
||||
return Pipe(std::make_shared<NullSource>(read_from_format_info.source_header));
|
||||
{
|
||||
is_empty_glob = true;
|
||||
return;
|
||||
}
|
||||
|
||||
iterator_wrapper = std::make_shared<StorageURLSource::IteratorWrapper>([glob_iterator, max_addresses]()
|
||||
{
|
||||
@ -923,7 +1035,7 @@ Pipe IStorageURLBase::read(
|
||||
}
|
||||
else
|
||||
{
|
||||
iterator_wrapper = std::make_shared<StorageURLSource::IteratorWrapper>([&, max_addresses, done = false]() mutable
|
||||
iterator_wrapper = std::make_shared<StorageURLSource::IteratorWrapper>([max_addresses, done = false, &uri = storage->uri]() mutable
|
||||
{
|
||||
if (done)
|
||||
return StorageURLSource::FailoverOptions{};
|
||||
@ -932,49 +1044,69 @@ Pipe IStorageURLBase::read(
|
||||
});
|
||||
num_streams = 1;
|
||||
}
|
||||
}
|
||||
|
||||
bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty())
|
||||
&& local_context->getSettingsRef().optimize_count_from_files;
|
||||
void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
createIterator(nullptr);
|
||||
|
||||
if (is_empty_glob)
|
||||
{
|
||||
pipeline.init(Pipe(std::make_shared<NullSource>(info.source_header)));
|
||||
return;
|
||||
}
|
||||
|
||||
Pipes pipes;
|
||||
pipes.reserve(num_streams);
|
||||
|
||||
const size_t max_threads = local_context->getSettingsRef().max_threads;
|
||||
const size_t max_threads = context->getSettingsRef().max_threads;
|
||||
const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / num_streams);
|
||||
|
||||
for (size_t i = 0; i < num_streams; ++i)
|
||||
{
|
||||
pipes.emplace_back(std::make_shared<StorageURLSource>(
|
||||
read_from_format_info,
|
||||
auto source = std::make_shared<StorageURLSource>(
|
||||
info,
|
||||
iterator_wrapper,
|
||||
getReadMethod(),
|
||||
getReadPOSTDataCallback(
|
||||
read_from_format_info.columns_description.getNamesOfPhysical(),
|
||||
read_from_format_info.columns_description,
|
||||
query_info,
|
||||
local_context,
|
||||
processed_stage,
|
||||
max_block_size),
|
||||
format_name,
|
||||
format_settings,
|
||||
getName(),
|
||||
local_context,
|
||||
storage->getReadMethod(),
|
||||
read_post_data_callback,
|
||||
storage->format_name,
|
||||
storage->format_settings,
|
||||
storage->getName(),
|
||||
context,
|
||||
max_block_size,
|
||||
getHTTPTimeouts(local_context),
|
||||
compression_method,
|
||||
getHTTPTimeouts(context),
|
||||
storage->compression_method,
|
||||
max_parsing_threads,
|
||||
query_info,
|
||||
headers,
|
||||
params,
|
||||
storage->headers,
|
||||
read_uri_params,
|
||||
is_url_with_globs,
|
||||
need_only_count));
|
||||
need_only_count);
|
||||
|
||||
source->setKeyCondition(filter_nodes.nodes, context);
|
||||
pipes.emplace_back(std::move(source));
|
||||
}
|
||||
|
||||
return Pipe::unitePipes(std::move(pipes));
|
||||
if (uri_options)
|
||||
std::shuffle(uri_options->begin(), uri_options->end(), thread_local_rng);
|
||||
|
||||
auto pipe = Pipe::unitePipes(std::move(pipes));
|
||||
size_t output_ports = pipe.numOutputPorts();
|
||||
const bool parallelize_output = context->getSettingsRef().parallelize_output_from_storages;
|
||||
if (parallelize_output && storage->parallelizeOutputAfterReading(context) && output_ports > 0 && output_ports < num_streams)
|
||||
pipe.resize(num_streams);
|
||||
|
||||
if (pipe.empty())
|
||||
pipe = Pipe(std::make_shared<NullSource>(info.source_header));
|
||||
|
||||
for (const auto & processor : pipe.getProcessors())
|
||||
processors.emplace_back(processor);
|
||||
|
||||
pipeline.init(std::move(pipe));
|
||||
}
|
||||
|
||||
|
||||
Pipe StorageURLWithFailover::read(
|
||||
void StorageURLWithFailover::read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
@ -984,38 +1116,34 @@ Pipe StorageURLWithFailover::read(
|
||||
size_t num_streams)
|
||||
{
|
||||
auto params = getReadURIParams(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size);
|
||||
|
||||
auto iterator_wrapper = std::make_shared<StorageURLSource::IteratorWrapper>([&, done = false]() mutable
|
||||
{
|
||||
if (done)
|
||||
return StorageURLSource::FailoverOptions{};
|
||||
done = true;
|
||||
return uri_options;
|
||||
});
|
||||
|
||||
auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals());
|
||||
|
||||
const size_t max_threads = local_context->getSettingsRef().max_threads;
|
||||
const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / num_streams);
|
||||
bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty())
|
||||
&& local_context->getSettingsRef().optimize_count_from_files;
|
||||
|
||||
auto pipe = Pipe(std::make_shared<StorageURLSource>(
|
||||
read_from_format_info,
|
||||
iterator_wrapper,
|
||||
getReadMethod(),
|
||||
getReadPOSTDataCallback(read_from_format_info.columns_description.getNamesOfPhysical(), read_from_format_info.columns_description, query_info, local_context, processed_stage, max_block_size),
|
||||
format_name,
|
||||
format_settings,
|
||||
getName(),
|
||||
auto read_post_data_callback = getReadPOSTDataCallback(
|
||||
read_from_format_info.columns_description.getNamesOfPhysical(),
|
||||
read_from_format_info.columns_description,
|
||||
query_info,
|
||||
local_context,
|
||||
processed_stage,
|
||||
max_block_size);
|
||||
|
||||
auto this_ptr = std::static_pointer_cast<StorageURL>(shared_from_this());
|
||||
|
||||
auto reading = std::make_unique<ReadFromURL>(
|
||||
read_from_format_info.source_header,
|
||||
std::move(this_ptr),
|
||||
&uri_options,
|
||||
std::move(read_from_format_info),
|
||||
need_only_count,
|
||||
std::move(params),
|
||||
std::move(read_post_data_callback),
|
||||
local_context,
|
||||
max_block_size,
|
||||
getHTTPTimeouts(local_context),
|
||||
compression_method,
|
||||
max_parsing_threads,
|
||||
query_info,
|
||||
headers,
|
||||
params));
|
||||
std::shuffle(uri_options.begin(), uri_options.end(), thread_local_rng);
|
||||
return pipe;
|
||||
num_streams);
|
||||
|
||||
query_plan.addStep(std::move(reading));
|
||||
}
|
||||
|
||||
|
||||
|
@ -34,7 +34,8 @@ class PullingPipelineExecutor;
|
||||
class IStorageURLBase : public IStorage
|
||||
{
|
||||
public:
|
||||
Pipe read(
|
||||
void read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
@ -67,6 +68,8 @@ public:
|
||||
const ContextPtr & context);
|
||||
|
||||
protected:
|
||||
friend class ReadFromURL;
|
||||
|
||||
IStorageURLBase(
|
||||
const String & uri_,
|
||||
ContextPtr context_,
|
||||
@ -136,7 +139,7 @@ public:
|
||||
class DisclosedGlobIterator
|
||||
{
|
||||
public:
|
||||
DisclosedGlobIterator(const String & uri_, size_t max_addresses, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context);
|
||||
DisclosedGlobIterator(const String & uri_, size_t max_addresses, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context);
|
||||
|
||||
String next();
|
||||
size_t size();
|
||||
@ -162,7 +165,6 @@ public:
|
||||
const ConnectionTimeouts & timeouts,
|
||||
CompressionMethod compression_method,
|
||||
size_t max_parsing_threads,
|
||||
const SelectQueryInfo & query_info,
|
||||
const HTTPHeaderEntries & headers_ = {},
|
||||
const URIParams & params = {},
|
||||
bool glob_url = false,
|
||||
@ -317,7 +319,8 @@ public:
|
||||
ContextPtr context_,
|
||||
const String & compression_method_);
|
||||
|
||||
Pipe read(
|
||||
void read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
|
@ -81,9 +81,9 @@ void StorageURLCluster::addColumnsStructureToQuery(ASTPtr & query, const String
|
||||
TableFunctionURLCluster::addColumnsStructureToArguments(expression_list->children, structure, context);
|
||||
}
|
||||
|
||||
RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const
|
||||
RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const
|
||||
{
|
||||
auto iterator = std::make_shared<StorageURLSource::DisclosedGlobIterator>(uri, context->getSettingsRef().glob_expansion_max_elements, query, virtual_columns, context);
|
||||
auto iterator = std::make_shared<StorageURLSource::DisclosedGlobIterator>(uri, context->getSettingsRef().glob_expansion_max_elements, predicate, virtual_columns, context);
|
||||
auto callback = std::make_shared<TaskIterator>([iter = std::move(iterator)]() mutable -> String { return iter->next(); });
|
||||
return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)};
|
||||
}
|
||||
|
@ -34,7 +34,7 @@ public:
|
||||
|
||||
NamesAndTypesList getVirtuals() const override { return virtual_columns; }
|
||||
|
||||
RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override;
|
||||
RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override;
|
||||
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
|
||||
|
@ -102,7 +102,8 @@ std::function<void(std::ostream &)> StorageXDBC::getReadPOSTDataCallback(
|
||||
return write_body_callback;
|
||||
}
|
||||
|
||||
Pipe StorageXDBC::read(
|
||||
void StorageXDBC::read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
@ -114,7 +115,7 @@ Pipe StorageXDBC::read(
|
||||
storage_snapshot->check(column_names);
|
||||
|
||||
bridge_helper->startBridgeSync();
|
||||
return IStorageURLBase::read(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams);
|
||||
IStorageURLBase::read(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams);
|
||||
}
|
||||
|
||||
SinkToStoragePtr StorageXDBC::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/)
|
||||
|
@ -19,7 +19,8 @@ namespace DB
|
||||
class StorageXDBC : public IStorageURLBase
|
||||
{
|
||||
public:
|
||||
Pipe read(
|
||||
void read(
|
||||
QueryPlan & query_plan,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
|
@ -36,7 +36,10 @@
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include "Functions/FunctionsLogical.h"
|
||||
#include "Functions/IFunction.h"
|
||||
#include "Functions/IFunctionAdaptors.h"
|
||||
#include "Functions/indexHint.h"
|
||||
#include <Parsers/makeASTForLogicalFunction.h>
|
||||
#include <Columns/ColumnSet.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
@ -390,9 +393,9 @@ static void addPathAndFileToVirtualColumns(Block & block, const String & path, s
|
||||
block.getByName("_idx").column->assumeMutableRef().insert(idx);
|
||||
}
|
||||
|
||||
ASTPtr createPathAndFileFilterAst(const ASTPtr & query, const NamesAndTypesList & virtual_columns, const String & path_example, const ContextPtr & context)
|
||||
ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns)
|
||||
{
|
||||
if (!query || virtual_columns.empty())
|
||||
if (!predicate || virtual_columns.empty())
|
||||
return {};
|
||||
|
||||
Block block;
|
||||
@ -401,16 +404,12 @@ ASTPtr createPathAndFileFilterAst(const ASTPtr & query, const NamesAndTypesList
|
||||
if (column.name == "_file" || column.name == "_path")
|
||||
block.insert({column.type->createColumn(), column.type, column.name});
|
||||
}
|
||||
/// Create a block with one row to construct filter
|
||||
/// Append "idx" column as the filter result
|
||||
|
||||
block.insert({ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "_idx"});
|
||||
addPathAndFileToVirtualColumns(block, path_example, 0);
|
||||
ASTPtr filter_ast;
|
||||
prepareFilterBlockWithQuery(query, context, block, filter_ast);
|
||||
return filter_ast;
|
||||
return splitFilterDagForAllowedInputs(predicate, block);
|
||||
}
|
||||
|
||||
ColumnPtr getFilterByPathAndFileIndexes(const std::vector<String> & paths, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context, ASTPtr filter_ast)
|
||||
ColumnPtr getFilterByPathAndFileIndexes(const std::vector<String> & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
{
|
||||
Block block;
|
||||
for (const auto & column : virtual_columns)
|
||||
@ -423,7 +422,7 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector<String> & paths, const
|
||||
for (size_t i = 0; i != paths.size(); ++i)
|
||||
addPathAndFileToVirtualColumns(block, paths[i], i);
|
||||
|
||||
filterBlockWithQuery(query, block, context, filter_ast);
|
||||
filterBlockWithDAG(dag, block, context);
|
||||
|
||||
return block.getByName("_idx").column;
|
||||
}
|
||||
@ -523,6 +522,37 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs(
|
||||
|
||||
return &node_copy;
|
||||
}
|
||||
else if (node->function_base->getName() == "indexHint")
|
||||
{
|
||||
if (const auto * adaptor = typeid_cast<const FunctionToFunctionBaseAdaptor *>(node->function_base.get()))
|
||||
{
|
||||
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()))
|
||||
{
|
||||
auto index_hint_dag = index_hint->getActions()->clone();
|
||||
ActionsDAG::NodeRawConstPtrs atoms;
|
||||
for (const auto & output : index_hint_dag->getOutputs())
|
||||
if (const auto * child_copy = splitFilterNodeForAllowedInputs(output, allowed_inputs, additional_nodes))
|
||||
atoms.push_back(child_copy);
|
||||
|
||||
if (!atoms.empty())
|
||||
{
|
||||
const auto * res = atoms.at(0);
|
||||
|
||||
if (atoms.size() > 1)
|
||||
{
|
||||
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
|
||||
res = &index_hint_dag->addFunction(func_builder_and, atoms, {});
|
||||
}
|
||||
|
||||
if (!res->result_type->equals(*node->result_type))
|
||||
res = &index_hint_dag->addCast(*res, node->result_type, {});
|
||||
|
||||
additional_nodes.splice(additional_nodes.end(), ActionsDAG::detachNodes(std::move(*index_hint_dag)));
|
||||
return res;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!canEvaluateSubtree(node, allowed_inputs))
|
||||
|
@ -58,14 +58,14 @@ auto extractSingleValueFromBlock(const Block & block, const String & name)
|
||||
|
||||
NamesAndTypesList getPathFileAndSizeVirtualsForStorage(NamesAndTypesList storage_columns);
|
||||
|
||||
ASTPtr createPathAndFileFilterAst(const ASTPtr & query, const NamesAndTypesList & virtual_columns, const String & path_example, const ContextPtr & context);
|
||||
ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns);
|
||||
|
||||
ColumnPtr getFilterByPathAndFileIndexes(const std::vector<String> & paths, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context, ASTPtr filter_ast);
|
||||
ColumnPtr getFilterByPathAndFileIndexes(const std::vector<String> & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context);
|
||||
|
||||
template <typename T>
|
||||
void filterByPathOrFile(std::vector<T> & sources, const std::vector<String> & paths, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context, ASTPtr filter_ast)
|
||||
void filterByPathOrFile(std::vector<T> & sources, const std::vector<String> & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
{
|
||||
auto indexes_column = getFilterByPathAndFileIndexes(paths, query, virtual_columns, context, filter_ast);
|
||||
auto indexes_column = getFilterByPathAndFileIndexes(paths, dag, virtual_columns, context);
|
||||
const auto & indexes = typeid_cast<const ColumnUInt64 &>(*indexes_column).getData();
|
||||
if (indexes.size() == sources.size())
|
||||
return;
|
||||
|
@ -26,4 +26,28 @@
|
||||
</retention>
|
||||
</default>
|
||||
</graphite_rollup_alternative>
|
||||
<graphite_rollup_alternative_no_function>
|
||||
<version_column_name>Version</version_column_name>
|
||||
<pattern>
|
||||
<regexp>sum</regexp>
|
||||
<retention>
|
||||
<age>0</age>
|
||||
<precision>600</precision>
|
||||
</retention>
|
||||
<retention>
|
||||
<age>17280</age>
|
||||
<precision>6000</precision>
|
||||
</retention>
|
||||
</pattern>
|
||||
<default>
|
||||
<retention>
|
||||
<age>0</age>
|
||||
<precision>600</precision>
|
||||
</retention>
|
||||
<retention>
|
||||
<age>17280</age>
|
||||
<precision>6000</precision>
|
||||
</retention>
|
||||
</default>
|
||||
</graphite_rollup_alternative_no_function>
|
||||
</clickhouse>
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user