Merge branch 'master' into shared_ptr_helper3

This commit is contained in:
mergify[bot] 2022-05-03 20:46:16 +00:00 committed by GitHub
commit 64084b5e32
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
192 changed files with 3539 additions and 1522 deletions

View File

@ -20,7 +20,9 @@ if (NOT DEFINED ENV{CLION_IDE} AND NOT DEFINED ENV{XCODE_IDE})
endif()
# Check if environment is polluted.
if (DEFINED ENV{CFLAGS} OR DEFINED ENV{CXXFLAGS} OR DEFINED ENV{LDFLAGS}
if (NOT $ENV{CFLAGS} STREQUAL ""
OR NOT $ENV{CXXFLAGS} STREQUAL ""
OR NOT $ENV{LDFLAGS} STREQUAL ""
OR CMAKE_C_FLAGS OR CMAKE_CXX_FLAGS OR CMAKE_EXE_LINKER_FLAGS OR CMAKE_SHARED_LINKER_FLAGS OR CMAKE_MODULE_LINKER_FLAGS
OR CMAKE_C_FLAGS_INIT OR CMAKE_CXX_FLAGS_INIT OR CMAKE_EXE_LINKER_FLAGS_INIT OR CMAKE_SHARED_LINKER_FLAGS_INIT OR CMAKE_MODULE_LINKER_FLAGS_INIT)

View File

@ -37,7 +37,8 @@ struct StringRef
size_t size = 0;
/// Non-constexpr due to reinterpret_cast.
template <typename CharT, typename = std::enable_if_t<sizeof(CharT) == 1>>
template <typename CharT>
requires (sizeof(CharT) == 1)
StringRef(const CharT * data_, size_t size_) : data(reinterpret_cast<const char *>(data_)), size(size_)
{
/// Sanity check for overflowed values.

View File

@ -21,10 +21,12 @@ public:
return *this;
}
template <typename G, typename = std::enable_if_t<std::is_convertible_v<G, F>, void>>
template <typename G>
requires std::is_convertible_v<G, F>
constexpr basic_scope_guard(basic_scope_guard<G> && src) : function{src.release()} {}
template <typename G, typename = std::enable_if_t<std::is_convertible_v<G, F>, void>>
template <typename G>
requires std::is_convertible_v<G, F>
constexpr basic_scope_guard & operator=(basic_scope_guard<G> && src)
{
if (this != &src)
@ -35,10 +37,12 @@ public:
return *this;
}
template <typename G, typename = std::enable_if_t<std::is_convertible_v<G, F>, void>>
template <typename G>
requires std::is_convertible_v<G, F>
constexpr basic_scope_guard(const G & function_) : function{function_} {}
template <typename G, typename = std::enable_if_t<std::is_convertible_v<G, F>, void>>
template <typename G>
requires std::is_convertible_v<G, F>
constexpr basic_scope_guard(G && function_) : function{std::move(function_)} {}
~basic_scope_guard() { invoke(); }
@ -64,7 +68,8 @@ public:
return std::exchange(function, {});
}
template <typename G, typename = std::enable_if_t<std::is_convertible_v<G, F>, void>>
template <typename G>
requires std::is_convertible_v<G, F>
basic_scope_guard<F> & join(basic_scope_guard<G> && other)
{
if (other.function)

2
contrib/minizip-ng vendored

@ -1 +1 @@
Subproject commit 6cffc951851620e0fac1993be75e4713c334de03
Subproject commit f3d400e999056ca290998b3fd89cc5a74e4b8b58

View File

@ -256,20 +256,6 @@ for conn_index, c in enumerate(all_connections):
reportStageEnd("settings")
# Check tables that should exist. If they don't exist, just skip this test.
tables = [e.text for e in root.findall("preconditions/table_exists")]
for t in tables:
for c in all_connections:
try:
res = c.execute("select 1 from {} limit 1".format(t))
except:
exception_message = traceback.format_exception_only(*sys.exc_info()[:2])[-1]
skipped_message = " ".join(exception_message.split("\n")[:2])
print(f"skipped\t{tsv_escape(skipped_message)}")
sys.exit(0)
reportStageEnd("preconditions")
if not args.use_existing_tables:
# Run create and fill queries. We will run them simultaneously for both
# servers, to save time. The weird XML search + filter is because we want to

View File

@ -167,6 +167,34 @@ Config is read from multiple files (in XML or YAML format) and merged into singl
For queries and subsystems other than `Server` config is accessible using `Context::getConfigRef()` method. Every subsystem that is capable of reloading it's config without server restart should register itself in reload callback in `Server::main()` method. Note that if newer config has an error, most subsystems will ignore new config, log warning messages and keep working with previously loaded config. Due to the nature of `AbstractConfiguration` it is not possible to pass reference to specific section, so `String config_prefix` is usually used instead.
## Threads and jobs {#threads-and-jobs}
To execute queries and do side activities ClickHouse allocates threads from one of thread pools to avoid frequent thread creation and destruction. There are a few thread pools, which are selected depending on a purpose and structure of a job:
* Server pool for incoming client sessions.
* Global thread pool for general purpose jobs, background activities and standalone threads.
* IO thread pool for jobs that are mostly blocked on some IO and are not CPU-intensive.
* Background pools for periodic tasks.
* Pools for preemptable tasks that can be split into steps.
Server pool is a `Poco::ThreadPool` class instance defined in `Server::main()` method. It can have at most `max_connection` threads. Every thread is dedicated to a single active connection.
Global thread pool is `GlobalThreadPool` singleton class. To allocate thread from it `ThreadFromGlobalPool` is used. It has an interface similar to `std::thread`, but pulls thread from the global pool and does all necessary initializations. It is configured with the following settings:
* `max_thread_pool_size` - limit on thread count in pool.
* `max_thread_pool_free_size` - limit on idle thread count waiting for new jobs.
* `thread_pool_queue_size` - limit on scheduled job count.
Global pool is universal and all pools described below are implemented on top of it. This can be thought of as a hierarchy of pools. Any specialized pool takes its threads from the global pool using `ThreadPool` class. So the main purpose of any specialized pool is to apply limit on the number of simultaneous jobs and do job scheduling. If there are more jobs scheduled than threads in a pool, `ThreadPool` accumulates jobs in a queue with priorities. Each job has an integer priority. Default priority is zero. All jobs with higher priority values are started before any job with lower priority value. But there is no difference between already executing jobs, thus priority matters only when the pool in overloaded.
IO thread pool is implemented as a plain `ThreadPool` accessible via `IOThreadPool::get()` method. It is configured in the same way as global pool with `max_io_thread_pool_size`, `max_io_thread_pool_free_size` and `io_thread_pool_queue_size` settings. The main purpose of IO thread pool is to avoid exhaustion of the global pool with IO jobs, which could prevent queries from fully utilizing CPU.
For periodic task execution there is `BackgroundSchedulePool` class. You can register tasks using `BackgroundSchedulePool::TaskHolder` objects and the pool ensures that no task runs two jobs at the same time. It also allows you to postpone task execution to a specific instant in the future or temporarily deactivate task. Global `Context` provides a few instances of this class for different purposes. For general purpose tasks `Context::getSchedulePool()` is used.
There are also specialized thread pools for preemptable tasks. Such `IExecutableTask` task can be split into ordered sequence of jobs, called steps. To schedule these tasks in a manner allowing short tasks to be prioritied over long ones `MergeTreeBackgroundExecutor` is used. As name suggests it is used for background MergeTree related operations such as merges, mutations, fetches and moves. Pool instances are available using `Context::getCommonExecutor()` and other similar methods.
No matter what pool is used for a job, at start `ThreadStatus` instance is created for this job. It encapsulates all per-thread information: thread id, query id, performance counters, resource consumption and many other useful data. Job can access it via thread local pointer by `CurrentThread::get()` call, so we do not need to pass it to every function.
If thread is related to query execution, then the most important thing attached to `ThreadStatus` is query context `ContextPtr`. Every query has its master thread in the server pool. Master thread does the attachment by holding an `ThreadStatus::QueryScope query_scope(query_context)` object. Master thread also creates a thread group represented with `ThreadGroupStatus` object. Every additional thread that is allocated during this query execution is attached to its thread group by `CurrentThread::attachTo(thread_group)` call. Thread groups are used to aggregate profile event counters and track memory consumption by all threads dedicated to a single task (see `MemoryTracker` and `ProfileEvents::Counters` classes for more information).
## Distributed Query Execution {#distributed-query-execution}
Servers in a cluster setup are mostly independent. You can create a `Distributed` table on one or all servers in a cluster. The `Distributed` table does not store data itself it only provides a “view” to all local tables on multiple nodes of a cluster. When you SELECT from a `Distributed` table, it rewrites that query, chooses remote nodes according to load balancing settings, and sends the query to them. The `Distributed` table requests remote servers to process a query just up to a stage where intermediate results from different servers can be merged. Then it receives the intermediate results and merges them. The distributed table tries to distribute as much work as possible to remote servers and does not send much intermediate data over the network.

View File

@ -83,7 +83,7 @@ $ ./src/unit_tests_dbms --gtest_filter=LocalAddress*
Performance tests allow to measure and compare performance of some isolated part of ClickHouse on synthetic queries. Tests are located at `tests/performance`. Each test is represented by `.xml` file with description of test case. Tests are run with `docker/tests/performance-comparison` tool . See the readme file for invocation.
Each test run one or multiple queries (possibly with combinations of parameters) in a loop. Some tests can contain preconditions on preloaded test dataset.
Each test run one or multiple queries (possibly with combinations of parameters) in a loop.
If you want to improve performance of ClickHouse in some scenario, and if improvements can be observed on simple queries, it is highly recommended to write a performance test. It always makes sense to use `perf top` or other perf tools during your tests.

View File

@ -6,6 +6,29 @@ slug: /en/operations/settings/settings
# Settings {#settings}
## allow_nondeterministic_mutations {#allow_nondeterministic_mutations}
User-level setting that allows mutations on replicated tables to make use of non-deterministic functions such as `dictGet`.
Given that, for example, dictionaries, can be out of sync across nodes, mutations that pull values from them are disallowed on replicated tables by default. Enabling this setting allows this behavior, making it the user's responsibility to ensure that the data used is in sync across all nodes.
Default value: 0.
**Example**
``` xml
<profiles>
<default>
<allow_nondeterministic_mutations>1</allow_nondeterministic_mutations>
<!-- ... -->
</default>
<!-- ... -->
</profiles>
```
## distributed_product_mode {#distributed-product-mode}
Changes the behaviour of [distributed subqueries](../../sql-reference/operators/in.md).

View File

@ -84,6 +84,7 @@ Result:
Returns the inclusive lower bound of the corresponding tumbling window.
``` sql
tumbleStart(bounds_tuple);
tumbleStart(time_attr, interval [, timezone]);
```
@ -92,6 +93,7 @@ tumbleStart(time_attr, interval [, timezone]);
Returns the exclusive upper bound of the corresponding tumbling window.
``` sql
tumbleEnd(bounds_tuple);
tumbleEnd(time_attr, interval [, timezone]);
```
@ -100,6 +102,7 @@ tumbleEnd(time_attr, interval [, timezone]);
Returns the inclusive lower bound of the corresponding hopping window.
``` sql
hopStart(bounds_tuple);
hopStart(time_attr, hop_interval, window_interval [, timezone]);
```
@ -108,5 +111,6 @@ hopStart(time_attr, hop_interval, window_interval [, timezone]);
Returns the exclusive upper bound of the corresponding hopping window.
``` sql
hopEnd(bounds_tuple);
hopEnd(time_attr, hop_interval, window_interval [, timezone]);
```

View File

@ -233,7 +233,7 @@
vertical-align: top;
}
td.right
.right
{
text-align: right;
}
@ -272,6 +272,26 @@
max-width: none;
}
td.transposed
{
max-width: none;
overflow: auto;
white-space: pre-wrap;
}
td.empty-result
{
text-align: center;
vertical-align: middle;
}
div.empty-result
{
opacity: 10%;
font-size: 7vw;
font-family: Liberation Sans, DejaVu Sans, sans-serif;
}
/* The style for SQL NULL */
.null
{
@ -613,32 +633,9 @@
}
}
function renderTable(response)
function renderCell(cell, col_idx, settings)
{
let thead = document.createElement('thead');
for (let idx in response.meta) {
let th = document.createElement('th');
const name = document.createTextNode(response.meta[idx].name);
th.appendChild(name);
thead.appendChild(th);
}
/// To prevent hanging the browser, limit the number of cells in a table.
/// It's important to have the limit on number of cells, not just rows, because tables may be wide or narrow.
const max_rows = 10000 / response.meta.length;
let row_num = 0;
const column_is_number = response.meta.map(elem => !!elem.type.match(/^(U?Int|Decimal|Float)/));
const column_maximums = column_is_number.map((elem, idx) => elem ? Math.max(...response.data.map(row => row[idx])) : 0);
const column_minimums = column_is_number.map((elem, idx) => elem ? Math.min(...response.data.map(row => Math.max(0, row[idx]))) : 0);
const column_need_render_bars = column_is_number.map((elem, idx) => column_maximums[idx] > 0 && column_maximums[idx] > column_minimums[idx]);
let tbody = document.createElement('tbody');
for (let row_idx in response.data) {
let tr = document.createElement('tr');
for (let col_idx in response.data[row_idx]) {
let td = document.createElement('td');
let cell = response.data[row_idx][col_idx];
let is_null = (cell === null);
let is_link = false;
@ -667,14 +664,18 @@
node = link;
}
td.className = column_is_number[col_idx] ? 'right' : 'left';
if (settings.is_transposed) {
td.className = 'left transposed';
} else {
td.className = settings.column_is_number[col_idx] ? 'right' : 'left';
}
if (is_null) {
td.className += ' null';
}
/// If it's a number, render bar in background.
if (column_need_render_bars[col_idx] && text > 0) {
const ratio = 100 * text / column_maximums[col_idx];
if (!settings.is_transposed && settings.column_need_render_bars[col_idx] && text > 0) {
const ratio = 100 * text / settings.column_maximums[col_idx];
let div = document.createElement('div');
@ -688,6 +689,84 @@
}
td.appendChild(node);
return td;
}
function renderTableTransposed(response)
{
let tbody = document.createElement('tbody');
for (let col_idx in response.meta) {
let tr = document.createElement('tr');
{
let th = document.createElement('th');
th.className = 'right';
th.style.width = '0';
th.appendChild(document.createTextNode(response.meta[col_idx].name));
tr.appendChild(th);
}
for (let row_idx in response.data)
{
let cell = response.data[row_idx][col_idx];
const td = renderCell(cell, col_idx, {is_transposed: true});
tr.appendChild(td);
}
if (response.data.length == 0 && col_idx == 0)
{
/// If result is empty, show this fact with a style.
let td = document.createElement('td');
td.rowSpan = response.meta.length;
td.className = 'empty-result';
let div = document.createElement('div');
div.appendChild(document.createTextNode("empty result"));
div.className = 'empty-result';
td.appendChild(div);
tr.appendChild(td);
}
tbody.appendChild(tr);
}
let table = document.getElementById('data-table');
table.appendChild(tbody);
}
function renderTable(response)
{
if (response.data.length <= 1 && response.meta.length >= 5) {
renderTableTransposed(response)
return;
}
let thead = document.createElement('thead');
for (let idx in response.meta) {
let th = document.createElement('th');
const name = document.createTextNode(response.meta[idx].name);
th.appendChild(name);
thead.appendChild(th);
}
/// To prevent hanging the browser, limit the number of cells in a table.
/// It's important to have the limit on number of cells, not just rows, because tables may be wide or narrow.
const max_rows = 10000 / response.meta.length;
let row_num = 0;
const column_is_number = response.meta.map(elem => !!elem.type.match(/^(U?Int|Decimal|Float)/));
const column_maximums = column_is_number.map((elem, idx) => elem ? Math.max(...response.data.map(row => row[idx])) : 0);
const column_minimums = column_is_number.map((elem, idx) => elem ? Math.min(...response.data.map(row => Math.max(0, row[idx]))) : 0);
const column_need_render_bars = column_is_number.map((elem, idx) => column_maximums[idx] > 0 && column_maximums[idx] > column_minimums[idx]);
const settings = {
is_transposed: false,
column_is_number: column_is_number,
column_maximums: column_maximums,
column_minimums: column_minimums,
column_need_render_bars: column_need_render_bars,
};
let tbody = document.createElement('tbody');
for (let row_idx in response.data) {
let tr = document.createElement('tr');
for (let col_idx in response.data[row_idx]) {
let cell = response.data[row_idx][col_idx];
const td = renderCell(cell, col_idx, settings);
tr.appendChild(td);
}
tbody.appendChild(tr);
@ -787,10 +866,7 @@
document.documentElement.setAttribute('data-theme', theme);
}
/**
* First we check if theme is set via the 'theme' GET parameter, if not, we check localStorage,
* otherwise we check OS preference
*/
/// First we check if theme is set via the 'theme' GET parameter, if not, we check localStorage, otherwise we check OS preference.
let theme = current_url.searchParams.get('theme');
if (['dark', 'light'].indexOf(theme) === -1) {
theme = window.localStorage.getItem('theme');

View File

@ -225,26 +225,38 @@ public:
}
void
addBatchSinglePlace(size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena *, ssize_t if_argument_pos) const final
addBatchSinglePlace(
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena *,
ssize_t if_argument_pos) const final
{
AggregateFunctionSumData<Numerator> sum_data;
const auto & column = assert_cast<const ColVecType &>(*columns[0]);
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
sum_data.addManyConditional(column.getData().data(), flags.data(), batch_size);
this->data(place).denominator += countBytesInFilter(flags.data(), batch_size);
sum_data.addManyConditional(column.getData().data(), flags.data(), row_begin, row_end);
this->data(place).denominator += countBytesInFilter(flags.data(), row_begin, row_end);
}
else
{
sum_data.addMany(column.getData().data(), batch_size);
this->data(place).denominator += batch_size;
sum_data.addMany(column.getData().data(), row_begin, row_end);
this->data(place).denominator += (row_end - row_begin);
}
increment(place, sum_data.sum);
}
void addBatchSinglePlaceNotNull(
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, const UInt8 * null_map, Arena *, ssize_t if_argument_pos)
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
const UInt8 * null_map,
Arena *,
ssize_t if_argument_pos)
const final
{
AggregateFunctionSumData<Numerator> sum_data;
@ -253,22 +265,22 @@ public:
{
/// Merge the 2 sets of flags (null and if) into a single one. This allows us to use parallelizable sums when available
const auto * if_flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData().data();
auto final_flags = std::make_unique<UInt8[]>(batch_size);
auto final_flags = std::make_unique<UInt8[]>(row_end);
size_t used_value = 0;
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
{
UInt8 kept = (!null_map[i]) & !!if_flags[i];
final_flags[i] = kept;
used_value += kept;
}
sum_data.addManyConditional(column.getData().data(), final_flags.get(), batch_size);
sum_data.addManyConditional(column.getData().data(), final_flags.get(), row_begin, row_end);
this->data(place).denominator += used_value;
}
else
{
sum_data.addManyNotNull(column.getData().data(), null_map, batch_size);
this->data(place).denominator += batch_size - countBytesInFilter(null_map, batch_size);
sum_data.addManyNotNull(column.getData().data(), null_map, row_begin, row_end);
this->data(place).denominator += (row_end - row_begin) - countBytesInFilter(null_map, row_begin, row_end);
}
increment(place, sum_data.sum);
}

View File

@ -54,7 +54,12 @@ public:
}
void addBatchSinglePlace(
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena *, ssize_t if_argument_pos) const override
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena *,
ssize_t if_argument_pos) const override
{
if (if_argument_pos >= 0)
{
@ -63,12 +68,13 @@ public:
}
else
{
data(place).count += batch_size;
data(place).count += row_end - row_begin;
}
}
void addBatchSinglePlaceNotNull(
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
const UInt8 * null_map,
@ -78,11 +84,12 @@ public:
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
data(place).count += countBytesInFilterWithNull(flags, null_map);
data(place).count += countBytesInFilterWithNull(flags, null_map, row_begin, row_end);
}
else
{
data(place).count += batch_size - countBytesInFilter(null_map, batch_size);
size_t rows = row_end - row_begin;
data(place).count += rows - countBytesInFilter(null_map, row_begin, row_end);
}
}
@ -204,17 +211,23 @@ public:
}
void addBatchSinglePlace(
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena *, ssize_t if_argument_pos) const override
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena *,
ssize_t if_argument_pos) const override
{
const auto & nc = assert_cast<const ColumnNullable &>(*columns[0]);
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
data(place).count += countBytesInFilterWithNull(flags, nc.getNullMapData().data());
data(place).count += countBytesInFilterWithNull(flags, nc.getNullMapData().data(), row_begin, row_end);
}
else
{
data(place).count += batch_size - countBytesInFilter(nc.getNullMapData().data(), batch_size);
size_t rows = row_end - row_begin;
data(place).count += rows - countBytesInFilter(nc.getNullMapData().data(), row_begin, row_end);
}
}

View File

@ -200,7 +200,7 @@ public:
arguments_raw[i] = arguments[i].get();
assert(!arguments.empty());
nested_func->addBatchSinglePlace(arguments[0]->size(), getNestedPlace(place), arguments_raw.data(), arena);
nested_func->addBatchSinglePlace(0, arguments[0]->size(), getNestedPlace(place), arguments_raw.data(), arena);
nested_func->insertResultInto(getNestedPlace(place), to, arena);
}

View File

@ -37,18 +37,18 @@ inline TColumn readItem(const IColumn * column, Arena * arena, size_t row)
template <typename TColumn, typename TFilter = void>
size_t
getFirstNElements_low_threshold(const TColumn * data, int num_elements, int threshold, size_t * results, const TFilter * filter = nullptr)
getFirstNElements_low_threshold(const TColumn * data, size_t row_begin, size_t row_end, size_t threshold, size_t * results, const TFilter * filter = nullptr)
{
for (int i = 0; i < threshold; i++)
for (size_t i = 0; i < threshold; i++)
{
results[i] = 0;
}
threshold = std::min(num_elements, threshold);
int current_max = 0;
int cur;
int z;
for (int i = 0; i < num_elements; i++)
threshold = std::min(row_end - row_begin, threshold);
size_t current_max = 0;
size_t cur;
size_t z;
for (size_t i = row_begin; i < row_end; i++)
{
if constexpr (!std::is_same_v<TFilter, void>)
{
@ -90,12 +90,12 @@ struct SortableItem
template <typename TColumn, typename TFilter = void>
size_t getFirstNElements_high_threshold(
const TColumn * data, size_t num_elements, size_t threshold, size_t * results, const TFilter * filter = nullptr)
const TColumn * data, size_t row_begin, size_t row_end, size_t threshold, size_t * results, const TFilter * filter = nullptr)
{
std::vector<SortableItem<TColumn>> dataIndexed(num_elements);
std::vector<SortableItem<TColumn>> dataIndexed(row_end);
size_t num_elements_filtered = 0;
for (size_t i = 0; i < num_elements; i++)
for (size_t i = row_begin; i < row_end; i++)
{
if constexpr (!std::is_same_v<TFilter, void>)
{
@ -124,21 +124,21 @@ size_t getFirstNElements_high_threshold(
static const size_t THRESHOLD_MAX_CUSTOM_FUNCTION = 1000;
template <typename TColumn>
size_t getFirstNElements(const TColumn * data, size_t num_elements, size_t threshold, size_t * results, const UInt8 * filter = nullptr)
size_t getFirstNElements(const TColumn * data, size_t row_begin, size_t row_end, size_t threshold, size_t * results, const UInt8 * filter = nullptr)
{
if (threshold < THRESHOLD_MAX_CUSTOM_FUNCTION)
{
if (filter != nullptr)
return getFirstNElements_low_threshold(data, num_elements, threshold, results, filter);
return getFirstNElements_low_threshold(data, row_begin, row_end, threshold, results, filter);
else
return getFirstNElements_low_threshold(data, num_elements, threshold, results);
return getFirstNElements_low_threshold(data, row_begin, row_end, threshold, results);
}
else
{
if (filter != nullptr)
return getFirstNElements_high_threshold(data, num_elements, threshold, results, filter);
return getFirstNElements_high_threshold(data, row_begin, row_end, threshold, results, filter);
else
return getFirstNElements_high_threshold(data, num_elements, threshold, results);
return getFirstNElements_high_threshold(data, row_begin, row_end, threshold, results);
}
}
@ -203,7 +203,7 @@ public:
template <typename TColumn, bool is_plain, typename TFunc>
void
forFirstRows(size_t batch_size, const IColumn ** columns, size_t data_column, Arena * arena, ssize_t if_argument_pos, TFunc func) const
forFirstRows(size_t row_begin, size_t row_end, const IColumn ** columns, size_t data_column, Arena * arena, ssize_t if_argument_pos, TFunc func) const
{
const TColumn * values = nullptr;
std::unique_ptr<std::vector<TColumn>> values_vector;
@ -211,8 +211,8 @@ public:
if constexpr (std::is_same_v<TColumn, StringRef>)
{
values_vector.reset(new std::vector<TColumn>(batch_size));
for (size_t i = 0; i < batch_size; i++)
values_vector.reset(new std::vector<TColumn>(row_end));
for (size_t i = row_begin; i < row_end; i++)
(*values_vector)[i] = readItem<TColumn, is_plain>(columns[data_column], arena, i);
values = (*values_vector).data();
}
@ -231,7 +231,7 @@ public:
filter = reinterpret_cast<const UInt8 *>(refFilter.data);
}
size_t num_elements = getFirstNElements(values, batch_size, threshold, best_rows.data(), filter);
size_t num_elements = getFirstNElements(values, row_begin, row_end, threshold, best_rows.data(), filter);
for (size_t i = 0; i < num_elements; i++)
{
func(best_rows[i], values);
@ -239,14 +239,19 @@ public:
}
void addBatchSinglePlace(
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos) const override
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena * arena,
ssize_t if_argument_pos) const override
{
State & data = this->data(place);
if constexpr (use_column_b)
{
forFirstRows<TColumnB, is_plain_b>(
batch_size, columns, 1, arena, if_argument_pos, [columns, &arena, &data](size_t row, const TColumnB * values)
row_begin, row_end, columns, 1, arena, if_argument_pos, [columns, &arena, &data](size_t row, const TColumnB * values)
{
data.add(readItem<TColumnA, is_plain_a>(columns[0], arena, row), values[row]);
});
@ -254,7 +259,7 @@ public:
else
{
forFirstRows<TColumnA, is_plain_a>(
batch_size, columns, 0, arena, if_argument_pos, [&data](size_t row, const TColumnA * values)
row_begin, row_end, columns, 0, arena, if_argument_pos, [&data](size_t row, const TColumnA * values)
{
data.add(values[row]);
});

View File

@ -119,7 +119,13 @@ public:
}
}
void addBatchSinglePlace(size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t) const override
void addBatchSinglePlace(
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena * arena,
ssize_t) const override
{
const ColumnNullable * column = assert_cast<const ColumnNullable *>(columns[0]);
const UInt8 * null_map = column->getNullMapData().data();
@ -142,25 +148,31 @@ public:
/// Combine the 2 flag arrays so we can call a simplified version (one check vs 2)
/// Note that now the null map will contain 0 if not null and not filtered, or 1 for null or filtered (or both)
auto final_nulls = std::make_unique<UInt8[]>(batch_size);
auto final_nulls = std::make_unique<UInt8[]>(row_end);
if (filter_null_map)
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
final_nulls[i] = (!!null_map[i]) | (!filter_values[i]) | (!!filter_null_map[i]);
else
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
final_nulls[i] = (!!null_map[i]) | (!filter_values[i]);
if constexpr (result_is_nullable)
{
if (!memoryIsByte(final_nulls.get(), batch_size, 1))
if (!memoryIsByte(final_nulls.get(), row_begin, row_end, 1))
this->setFlag(place);
else
return; /// No work to do.
}
this->nested_function->addBatchSinglePlaceNotNull(
batch_size, this->nestedPlace(place), columns_param, final_nulls.get(), arena, -1);
row_begin,
row_end,
this->nestedPlace(place),
columns_param,
final_nulls.get(),
arena,
-1);
}
#if USE_EMBEDDED_COMPILER

View File

@ -98,31 +98,38 @@ public:
}
void addBatch(
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
const IColumn ** columns,
Arena * arena,
ssize_t) const override
{
nested_func->addBatch(batch_size, places, place_offset, columns, arena, num_arguments - 1);
nested_func->addBatch(row_begin, row_end, places, place_offset, columns, arena, num_arguments - 1);
}
void addBatchSinglePlace(
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t) const override
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena * arena,
ssize_t) const override
{
nested_func->addBatchSinglePlace(batch_size, place, columns, arena, num_arguments - 1);
nested_func->addBatchSinglePlace(row_begin, row_end, place, columns, arena, num_arguments - 1);
}
void addBatchSinglePlaceNotNull(
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
const UInt8 * null_map,
Arena * arena,
ssize_t) const override
{
nested_func->addBatchSinglePlaceNotNull(batch_size, place, columns, null_map, arena, num_arguments - 1);
nested_func->addBatchSinglePlaceNotNull(row_begin, row_end, place, columns, null_map, arena, num_arguments - 1);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
@ -131,13 +138,14 @@ public:
}
void mergeBatch(
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
const AggregateDataPtr * rhs,
Arena * arena) const override
{
nested_func->mergeBatch(batch_size, places, place_offset, rhs, arena);
nested_func->mergeBatch(row_begin, row_end, places, place_offset, rhs, arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> version) const override

View File

@ -1159,7 +1159,12 @@ public:
}
void addBatchSinglePlace(
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos) const override
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena * arena,
ssize_t if_argument_pos) const override
{
if constexpr (is_any)
if (this->data(place).has())
@ -1167,7 +1172,7 @@ public:
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
{
if (flags[i])
{
@ -1179,7 +1184,7 @@ public:
}
else
{
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
{
this->data(place).changeIfBetter(*columns[0], i, arena);
if constexpr (is_any)
@ -1189,7 +1194,8 @@ public:
}
void addBatchSinglePlaceNotNull( /// NOLINT
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
const UInt8 * null_map,
@ -1203,7 +1209,7 @@ public:
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
{
if (!null_map[i] && flags[i])
{
@ -1215,7 +1221,7 @@ public:
}
else
{
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
{
if (!null_map[i])
{

View File

@ -307,17 +307,22 @@ public:
}
void addBatchSinglePlace( /// NOLINT
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1) const override
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena * arena,
ssize_t if_argument_pos = -1) const override
{
const ColumnNullable * column = assert_cast<const ColumnNullable *>(columns[0]);
const IColumn * nested_column = &column->getNestedColumn();
const UInt8 * null_map = column->getNullMapData().data();
this->nested_function->addBatchSinglePlaceNotNull(
batch_size, this->nestedPlace(place), &nested_column, null_map, arena, if_argument_pos);
row_begin, row_end, this->nestedPlace(place), &nested_column, null_map, arena, if_argument_pos);
if constexpr (result_is_nullable)
if (!memoryIsByte(null_map, batch_size, 1))
if (!memoryIsByte(null_map, row_begin, row_end, 1))
this->setFlag(place);
}

View File

@ -109,7 +109,8 @@ public:
}
void addBatch( /// NOLINT
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
const IColumn ** columns,
@ -119,7 +120,7 @@ public:
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
{
if (flags[i] && places[i])
add(places[i] + place_offset, columns, i, arena);
@ -127,21 +128,26 @@ public:
}
else
{
nested_function->addBatch(batch_size, places, place_offset, columns, arena, if_argument_pos);
for (size_t i = 0; i < batch_size; ++i)
nested_function->addBatch(row_begin, row_end, places, place_offset, columns, arena, if_argument_pos);
for (size_t i = row_begin; i < row_end; ++i)
if (places[i])
(places[i] + place_offset)[size_of_data] = 1;
}
}
void addBatchSinglePlace( /// NOLINT
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1) const override
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena * arena,
ssize_t if_argument_pos = -1) const override
{
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
nested_function->addBatchSinglePlace(batch_size, place, columns, arena, if_argument_pos);
for (size_t i = 0; i < batch_size; ++i)
nested_function->addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos);
for (size_t i = row_begin; i < row_end; ++i)
{
if (flags[i])
{
@ -152,16 +158,17 @@ public:
}
else
{
if (batch_size)
if (row_end != row_begin)
{
nested_function->addBatchSinglePlace(batch_size, place, columns, arena, if_argument_pos);
nested_function->addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos);
place[size_of_data] = 1;
}
}
}
void addBatchSinglePlaceNotNull( /// NOLINT
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
const UInt8 * null_map,
@ -171,8 +178,8 @@ public:
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
nested_function->addBatchSinglePlaceNotNull(batch_size, place, columns, null_map, arena, if_argument_pos);
for (size_t i = 0; i < batch_size; ++i)
nested_function->addBatchSinglePlaceNotNull(row_begin, row_end, place, columns, null_map, arena, if_argument_pos);
for (size_t i = row_begin; i < row_end; ++i)
{
if (flags[i] && !null_map[i])
{
@ -183,10 +190,10 @@ public:
}
else
{
if (batch_size)
if (row_end != row_begin)
{
nested_function->addBatchSinglePlaceNotNull(batch_size, place, columns, null_map, arena, if_argument_pos);
for (size_t i = 0; i < batch_size; ++i)
nested_function->addBatchSinglePlaceNotNull(row_begin, row_end, place, columns, null_map, arena, if_argument_pos);
for (size_t i = row_begin; i < row_end; ++i)
{
if (!null_map[i])
{
@ -208,14 +215,15 @@ public:
}
void mergeBatch(
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
const AggregateDataPtr * rhs,
Arena * arena) const override
{
nested_function->mergeBatch(batch_size, places, place_offset, rhs, arena);
for (size_t i = 0; i < batch_size; ++i)
nested_function->mergeBatch(row_begin, row_end, places, place_offset, rhs, arena);
for (size_t i = row_begin; i < row_end; ++i)
(places[i] + place_offset)[size_of_data] |= rhs[i][size_of_data];
}

View File

@ -52,6 +52,8 @@ public:
return nested_func->getDefaultVersion();
}
size_t getVersionFromRevision(size_t revision) const override { return nested_func->getVersionFromRevision(revision); }
void create(AggregateDataPtr __restrict place) const override
{
nested_func->create(place);

View File

@ -59,9 +59,11 @@ struct AggregateFunctionSumData
/// Vectorized version
template <typename Value>
void NO_SANITIZE_UNDEFINED NO_INLINE addMany(const Value * __restrict ptr, size_t count)
void NO_SANITIZE_UNDEFINED NO_INLINE addMany(const Value * __restrict ptr, size_t start, size_t end)
{
const auto * end = ptr + count;
ptr += start;
size_t count = end - start;
const auto * end_ptr = ptr + count;
if constexpr (std::is_floating_point_v<T>)
{
@ -87,7 +89,7 @@ struct AggregateFunctionSumData
/// clang cannot vectorize the loop if accumulator is class member instead of local variable.
T local_sum{};
while (ptr < end)
while (ptr < end_ptr)
{
Impl::add(local_sum, *ptr);
++ptr;
@ -97,9 +99,11 @@ struct AggregateFunctionSumData
template <typename Value, bool add_if_zero>
void NO_SANITIZE_UNDEFINED NO_INLINE
addManyConditionalInternal(const Value * __restrict ptr, const UInt8 * __restrict condition_map, size_t count)
addManyConditionalInternal(const Value * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end)
{
const auto * end = ptr + count;
ptr += start;
size_t count = end - start;
const auto * end_ptr = ptr + count;
if constexpr (
(is_integer<T> && !is_big_int_v<T>)
@ -108,7 +112,7 @@ struct AggregateFunctionSumData
/// For integers we can vectorize the operation if we replace the null check using a multiplication (by 0 for null, 1 for not null)
/// https://quick-bench.com/q/MLTnfTvwC2qZFVeWHfOBR3U7a8I
T local_sum{};
while (ptr < end)
while (ptr < end_ptr)
{
T multiplier = !*condition_map == add_if_zero;
Impl::add(local_sum, *ptr * multiplier);
@ -151,7 +155,7 @@ struct AggregateFunctionSumData
}
T local_sum{};
while (ptr < end)
while (ptr < end_ptr)
{
if (!*condition_map == add_if_zero)
Impl::add(local_sum, *ptr);
@ -162,15 +166,15 @@ struct AggregateFunctionSumData
}
template <typename Value>
void ALWAYS_INLINE addManyNotNull(const Value * __restrict ptr, const UInt8 * __restrict null_map, size_t count)
void ALWAYS_INLINE addManyNotNull(const Value * __restrict ptr, const UInt8 * __restrict null_map, size_t start, size_t end)
{
return addManyConditionalInternal<Value, true>(ptr, null_map, count);
return addManyConditionalInternal<Value, true>(ptr, null_map, start, end);
}
template <typename Value>
void ALWAYS_INLINE addManyConditional(const Value * __restrict ptr, const UInt8 * __restrict cond_map, size_t count)
void ALWAYS_INLINE addManyConditional(const Value * __restrict ptr, const UInt8 * __restrict cond_map, size_t start, size_t end)
{
return addManyConditionalInternal<Value, false>(ptr, cond_map, count);
return addManyConditionalInternal<Value, false>(ptr, cond_map, start, end);
}
void NO_SANITIZE_UNDEFINED merge(const AggregateFunctionSumData & rhs)
@ -220,7 +224,7 @@ struct AggregateFunctionSumKahanData
/// Vectorized version
template <typename Value>
void NO_INLINE addMany(const Value * __restrict ptr, size_t count)
void NO_INLINE addMany(const Value * __restrict ptr, size_t start, size_t end)
{
/// Less than in ordinary sum, because the algorithm is more complicated and too large loop unrolling is questionable.
/// But this is just a guess.
@ -228,7 +232,10 @@ struct AggregateFunctionSumKahanData
T partial_sums[unroll_count]{};
T partial_compensations[unroll_count]{};
const auto * end = ptr + count;
ptr += start;
size_t count = end - start;
const auto * end_ptr = ptr + count;
const auto * unrolled_end = ptr + (count / unroll_count * unroll_count);
while (ptr < unrolled_end)
@ -241,7 +248,7 @@ struct AggregateFunctionSumKahanData
for (size_t i = 0; i < unroll_count; ++i)
mergeImpl(sum, compensation, partial_sums[i], partial_compensations[i]);
while (ptr < end)
while (ptr < end_ptr)
{
addImpl(*ptr, sum, compensation);
++ptr;
@ -249,13 +256,16 @@ struct AggregateFunctionSumKahanData
}
template <typename Value, bool add_if_zero>
void NO_INLINE addManyConditionalInternal(const Value * __restrict ptr, const UInt8 * __restrict condition_map, size_t count)
void NO_INLINE addManyConditionalInternal(const Value * __restrict ptr, const UInt8 * __restrict condition_map, size_t start, size_t end)
{
constexpr size_t unroll_count = 4;
T partial_sums[unroll_count]{};
T partial_compensations[unroll_count]{};
const auto * end = ptr + count;
ptr += start;
size_t count = end - start;
const auto * end_ptr = ptr + count;
const auto * unrolled_end = ptr + (count / unroll_count * unroll_count);
while (ptr < unrolled_end)
@ -270,7 +280,7 @@ struct AggregateFunctionSumKahanData
for (size_t i = 0; i < unroll_count; ++i)
mergeImpl(sum, compensation, partial_sums[i], partial_compensations[i]);
while (ptr < end)
while (ptr < end_ptr)
{
if ((!*condition_map) == add_if_zero)
addImpl(*ptr, sum, compensation);
@ -280,15 +290,15 @@ struct AggregateFunctionSumKahanData
}
template <typename Value>
void ALWAYS_INLINE addManyNotNull(const Value * __restrict ptr, const UInt8 * __restrict null_map, size_t count)
void ALWAYS_INLINE addManyNotNull(const Value * __restrict ptr, const UInt8 * __restrict null_map, size_t start, size_t end)
{
return addManyConditionalInternal<Value, true>(ptr, null_map, count);
return addManyConditionalInternal<Value, true>(ptr, null_map, start, end);
}
template <typename Value>
void ALWAYS_INLINE addManyConditional(const Value * __restrict ptr, const UInt8 * __restrict cond_map, size_t count)
void ALWAYS_INLINE addManyConditional(const Value * __restrict ptr, const UInt8 * __restrict cond_map, size_t start, size_t end)
{
return addManyConditionalInternal<Value, false>(ptr, cond_map, count);
return addManyConditionalInternal<Value, false>(ptr, cond_map, start, end);
}
void ALWAYS_INLINE mergeImpl(T & to_sum, T & to_compensation, T from_sum, T from_compensation)
@ -385,22 +395,33 @@ public:
}
void addBatchSinglePlace(
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena *, ssize_t if_argument_pos) const override
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena *,
ssize_t if_argument_pos) const override
{
const auto & column = assert_cast<const ColVecType &>(*columns[0]);
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
this->data(place).addManyConditional(column.getData().data(), flags.data(), batch_size);
this->data(place).addManyConditional(column.getData().data(), flags.data(), row_begin, row_end);
}
else
{
this->data(place).addMany(column.getData().data(), batch_size);
this->data(place).addMany(column.getData().data(), row_begin, row_end);
}
}
void addBatchSinglePlaceNotNull(
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, const UInt8 * null_map, Arena *, ssize_t if_argument_pos)
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
const UInt8 * null_map,
Arena *,
ssize_t if_argument_pos)
const override
{
const auto & column = assert_cast<const ColVecType &>(*columns[0]);
@ -408,15 +429,15 @@ public:
{
/// Merge the 2 sets of flags (null and if) into a single one. This allows us to use parallelizable sums when available
const auto * if_flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData().data();
auto final_flags = std::make_unique<UInt8[]>(batch_size);
for (size_t i = 0; i < batch_size; ++i)
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];
this->data(place).addManyConditional(column.getData().data(), final_flags.get(), batch_size);
this->data(place).addManyConditional(column.getData().data(), final_flags.get(), row_begin, row_end);
}
else
{
this->data(place).addManyNotNull(column.getData().data(), null_map, batch_size);
this->data(place).addManyNotNull(column.getData().data(), null_map, row_begin, row_end);
}
}

View File

@ -175,7 +175,8 @@ public:
* and do a single call to "addBatch" for devirtualization and inlining.
*/
virtual void addBatch( /// NOLINT
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
const IColumn ** columns,
@ -184,13 +185,16 @@ public:
/// The version of "addBatch", that handle sparse columns as arguments.
virtual void addBatchSparse(
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
const IColumn ** columns,
Arena * arena) const = 0;
virtual void mergeBatch(
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
const AggregateDataPtr * rhs,
@ -199,17 +203,27 @@ public:
/** The same for single place.
*/
virtual void addBatchSinglePlace( /// NOLINT
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1) const = 0;
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena * arena,
ssize_t if_argument_pos = -1) const = 0;
/// The version of "addBatchSinglePlace", that handle sparse columns as arguments.
virtual void addBatchSparseSinglePlace(
AggregateDataPtr place, const IColumn ** columns, Arena * arena) const = 0;
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena * arena) const = 0;
/** The same for single place when need to aggregate only filtered data.
* Instead of using an if-column, the condition is combined inside the null_map
*/
virtual void addBatchSinglePlaceNotNull( /// NOLINT
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
const UInt8 * null_map,
@ -217,7 +231,12 @@ public:
ssize_t if_argument_pos = -1) const = 0;
virtual void addBatchSinglePlaceFromInterval( /// NOLINT
size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1)
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena * arena,
ssize_t if_argument_pos = -1)
const = 0;
/** In addition to addBatch, this method collects multiple rows of arguments into array "places"
@ -226,7 +245,8 @@ public:
* "places" contains a large number of same values consecutively.
*/
virtual void addBatchArray(
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
const IColumn ** columns,
@ -237,7 +257,8 @@ public:
* and pointers to aggregation states are stored in AggregateDataPtr[256] lookup table.
*/
virtual void addBatchLookupTable8(
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
std::function<void(AggregateDataPtr &)> init,
@ -251,7 +272,8 @@ public:
* All places that were not inserted must be destroyed if there was exception during insert into result column.
*/
virtual void insertResultIntoBatch(
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
IColumn & to,
@ -261,7 +283,8 @@ public:
/** Destroy batch of aggregate places.
*/
virtual void destroyBatch(
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset) const noexcept = 0;
@ -355,7 +378,8 @@ public:
AddFunc getAddressOfAddFunction() const override { return &addFree; }
void addBatch( /// NOLINT
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
const IColumn ** columns,
@ -365,7 +389,7 @@ public:
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
{
if (flags[i] && places[i])
static_cast<const Derived *>(this)->add(places[i] + place_offset, columns, i, arena);
@ -373,13 +397,15 @@ public:
}
else
{
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
if (places[i])
static_cast<const Derived *>(this)->add(places[i] + place_offset, columns, i, arena);
}
}
void addBatchSparse(
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
const IColumn ** columns,
@ -387,33 +413,42 @@ public:
{
const auto & column_sparse = assert_cast<const ColumnSparse &>(*columns[0]);
const auto * values = &column_sparse.getValuesColumn();
size_t batch_size = column_sparse.size();
auto offset_it = column_sparse.begin();
for (size_t i = 0; i < batch_size; ++i, ++offset_it)
/// FIXME: make it more optimal
for (size_t i = 0; i < row_begin; ++i, ++offset_it)
;
for (size_t i = 0; i < row_end; ++i, ++offset_it)
static_cast<const Derived *>(this)->add(places[offset_it.getCurrentRow()] + place_offset,
&values, offset_it.getValueIndex(), arena);
}
void mergeBatch(
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
const AggregateDataPtr * rhs,
Arena * arena) const override
{
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
if (places[i])
static_cast<const Derived *>(this)->merge(places[i] + place_offset, rhs[i], arena);
}
void addBatchSinglePlace( /// NOLINT
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1) const override
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena * arena,
ssize_t if_argument_pos = -1) const override
{
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
{
if (flags[i])
static_cast<const Derived *>(this)->add(place, columns, i, arena);
@ -421,26 +456,34 @@ public:
}
else
{
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
static_cast<const Derived *>(this)->add(place, columns, i, arena);
}
}
void addBatchSparseSinglePlace(
AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena * arena) const override
{
/// TODO: add values and defaults separately if order of adding isn't important.
const auto & column_sparse = assert_cast<const ColumnSparse &>(*columns[0]);
const auto * values = &column_sparse.getValuesColumn();
size_t batch_size = column_sparse.size();
auto offset_it = column_sparse.begin();
for (size_t i = 0; i < batch_size; ++i, ++offset_it)
/// FIXME: make it more optimal
for (size_t i = 0; i < row_begin; ++i, ++offset_it)
;
for (size_t i = 0; i < row_end; ++i, ++offset_it)
static_cast<const Derived *>(this)->add(place, &values, offset_it.getValueIndex(), arena);
}
void addBatchSinglePlaceNotNull( /// NOLINT
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
const UInt8 * null_map,
@ -450,26 +493,31 @@ public:
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
if (!null_map[i] && flags[i])
static_cast<const Derived *>(this)->add(place, columns, i, arena);
}
else
{
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
if (!null_map[i])
static_cast<const Derived *>(this)->add(place, columns, i, arena);
}
}
void addBatchSinglePlaceFromInterval( /// NOLINT
size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1)
size_t row_begin,
size_t row_end,
AggregateDataPtr place,
const IColumn ** columns,
Arena * arena,
ssize_t if_argument_pos = -1)
const override
{
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
for (size_t i = batch_begin; i < batch_end; ++i)
for (size_t i = row_begin; i < row_end; ++i)
{
if (flags[i])
static_cast<const Derived *>(this)->add(place, columns, i, arena);
@ -477,17 +525,23 @@ public:
}
else
{
for (size_t i = batch_begin; i < batch_end; ++i)
for (size_t i = row_begin; i < row_end; ++i)
static_cast<const Derived *>(this)->add(place, columns, i, arena);
}
}
void addBatchArray(
size_t batch_size, AggregateDataPtr * places, size_t place_offset, const IColumn ** columns, const UInt64 * offsets, Arena * arena)
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
const IColumn ** columns,
const UInt64 * offsets,
Arena * arena)
const override
{
size_t current_offset = 0;
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
{
size_t next_offset = offsets[i];
for (size_t j = current_offset; j < next_offset; ++j)
@ -498,7 +552,8 @@ public:
}
void addBatchLookupTable8(
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr * map,
size_t place_offset,
std::function<void(AggregateDataPtr &)> init,
@ -508,10 +563,10 @@ public:
{
static constexpr size_t UNROLL_COUNT = 8;
size_t i = 0;
size_t i = row_begin;
size_t batch_size_unrolled = batch_size / UNROLL_COUNT * UNROLL_COUNT;
for (; i < batch_size_unrolled; i += UNROLL_COUNT)
size_t size_unrolled = (row_end - row_begin) / UNROLL_COUNT * UNROLL_COUNT;
for (; i < size_unrolled; i += UNROLL_COUNT)
{
AggregateDataPtr places[UNROLL_COUNT];
for (size_t j = 0; j < UNROLL_COUNT; ++j)
@ -527,7 +582,7 @@ public:
static_cast<const Derived *>(this)->add(places[j] + place_offset, columns, i + j, arena);
}
for (; i < batch_size; ++i)
for (; i < row_end; ++i)
{
AggregateDataPtr & place = map[key[i]];
if (unlikely(!place))
@ -536,13 +591,20 @@ public:
}
}
void insertResultIntoBatch(size_t batch_size, AggregateDataPtr * places, size_t place_offset, IColumn & to, Arena * arena, bool destroy_place_after_insert) const override
void insertResultIntoBatch(
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset,
IColumn & to,
Arena * arena,
bool destroy_place_after_insert) const override
{
size_t batch_index = 0;
size_t batch_index = row_begin;
try
{
for (; batch_index < batch_size; ++batch_index)
for (; batch_index < row_end; ++batch_index)
{
static_cast<const Derived *>(this)->insertResultInto(places[batch_index] + place_offset, to, arena);
@ -552,16 +614,20 @@ public:
}
catch (...)
{
for (size_t destroy_index = batch_index; destroy_index < batch_size; ++destroy_index)
for (size_t destroy_index = batch_index; destroy_index < row_end; ++destroy_index)
static_cast<const Derived *>(this)->destroy(places[destroy_index] + place_offset);
throw;
}
}
void destroyBatch(size_t batch_size, AggregateDataPtr * places, size_t place_offset) const noexcept override
void destroyBatch(
size_t row_begin,
size_t row_end,
AggregateDataPtr * places,
size_t place_offset) const noexcept override
{
for (size_t i = 0; i < batch_size; ++i)
for (size_t i = row_begin; i < row_end; ++i)
{
static_cast<const Derived *>(this)->destroy(places[i] + place_offset);
}
@ -612,7 +678,8 @@ public:
}
void addBatchLookupTable8(
size_t batch_size,
size_t row_begin,
size_t row_end,
AggregateDataPtr * map,
size_t place_offset,
std::function<void(AggregateDataPtr &)> init,
@ -626,7 +693,7 @@ public:
if (func.allocatesMemoryInArena() || sizeof(Data) > 16 || func.sizeOfData() != sizeof(Data))
{
IAggregateFunctionHelper<Derived>::addBatchLookupTable8(batch_size, map, place_offset, init, key, columns, arena);
IAggregateFunctionHelper<Derived>::addBatchLookupTable8(row_begin, row_end, map, place_offset, init, key, columns, arena);
return;
}
@ -637,12 +704,12 @@ public:
std::unique_ptr<Data[]> places{new Data[256 * UNROLL_COUNT]};
bool has_data[256 * UNROLL_COUNT]{}; /// Separate flags array to avoid heavy initialization.
size_t i = 0;
size_t i = row_begin;
/// Aggregate data into different lookup tables.
size_t batch_size_unrolled = batch_size / UNROLL_COUNT * UNROLL_COUNT;
for (; i < batch_size_unrolled; i += UNROLL_COUNT)
size_t size_unrolled = (row_end - row_begin) / UNROLL_COUNT * UNROLL_COUNT;
for (; i < size_unrolled; i += UNROLL_COUNT)
{
for (size_t j = 0; j < UNROLL_COUNT; ++j)
{
@ -676,7 +743,7 @@ public:
/// Process tails and add directly to the final destination.
for (; i < batch_size; ++i)
for (; i < row_end; ++i)
{
size_t k = key[i];
AggregateDataPtr & place = map[k];

View File

@ -161,16 +161,17 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
}
catch (Poco::TimeoutException & e)
{
/// disconnect() will reset the socket, get timeouts before.
const std::string & message = fmt::format("{} ({}, receive timeout {} ms, send timeout {} ms)",
e.displayText(), getDescription(),
socket->getReceiveTimeout().totalMilliseconds(),
socket->getSendTimeout().totalMilliseconds());
disconnect();
/// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost.
throw NetException(message, ErrorCodes::SOCKET_TIMEOUT);
/// This exception can only be thrown from socket->connect(), so add information about connection timeout.
const auto & connection_timeout = static_cast<bool>(secure) ? timeouts.secure_connection_timeout : timeouts.connection_timeout;
throw NetException(
ErrorCodes::SOCKET_TIMEOUT,
"{} ({}, connection timeout {} ms)",
e.displayText(),
getDescription(),
connection_timeout.totalMilliseconds());
}
}

View File

@ -197,7 +197,10 @@ bool ConnectionEstablisherAsync::checkReceiveTimeout()
destroyFiber();
/// In not async case this exception would be thrown and caught in ConnectionEstablisher::run,
/// but in async case we process timeout outside and cannot throw exception. So, we just save fail message.
fail_message = "Timeout exceeded while reading from socket (" + result.entry->getDescription() + ")";
fail_message = fmt::format(
"Timeout exceeded while reading from socket ({}, receive timeout {} ms)",
result.entry->getDescription(),
result.entry->getSocket()->getReceiveTimeout().totalMilliseconds());
epoll.remove(socket_fd);
resetResult();
return false;

View File

@ -181,6 +181,7 @@ void HedgedConnections::sendQuery(
replica.connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data);
replica.change_replica_timeout.setRelative(timeouts.receive_data_timeout);
replica.packet_receiver->setReceiveTimeout(hedged_connections_factory.getConnectionTimeouts().receive_timeout);
};
for (auto & offset_status : offset_states)
@ -354,11 +355,16 @@ bool HedgedConnections::resumePacketReceiver(const HedgedConnections::ReplicaLoc
}
else if (std::holds_alternative<Poco::Timespan>(res))
{
const String & description = replica_state.connection->getDescription();
finishProcessReplica(replica_state, true);
/// Check if there is no more active connections with the same offset and there is no new replica in process.
if (offset_states[location.offset].active_connection_count == 0 && !offset_states[location.offset].next_replica_in_process)
throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT);
throw NetException(
ErrorCodes::SOCKET_TIMEOUT,
"Timeout exceeded while reading from socket ({}, receive timeout {} ms)",
description,
std::get<Poco::Timespan>(res).totalMilliseconds());
}
else if (std::holds_alternative<std::exception_ptr>(res))
{

View File

@ -37,7 +37,7 @@ public:
if (!connection->hasReadPendingData() && !checkReceiveTimeout())
{
/// Receive timeout expired.
return Poco::Timespan();
return connection->getSocket()->getReceiveTimeout();
}
/// Resume fiber.
@ -60,6 +60,11 @@ public:
int getFileDescriptor() const { return epoll.getFileDescriptor(); }
void setReceiveTimeout(const Poco::Timespan & timeout)
{
receive_timeout.setRelative(timeout);
}
private:
/// When epoll file descriptor is ready, check if it's an expired timeout.
/// Return false if receive timeout expired and socket is not ready, return true otherwise.

View File

@ -54,7 +54,7 @@ MutableColumnPtr ColumnFixedString::cloneResized(size_t size) const
bool ColumnFixedString::isDefaultAt(size_t index) const
{
assert(index < size());
return memoryIsZero(chars.data() + index * n, n);
return memoryIsZero(chars.data() + index * n, 0, n);
}
void ColumnFixedString::insert(const Field & x)

View File

@ -27,7 +27,7 @@ static UInt64 toBits64(const Int8 * bytes64)
}
#endif
size_t countBytesInFilter(const UInt8 * filt, size_t sz)
size_t countBytesInFilter(const UInt8 * filt, size_t start, size_t end)
{
size_t count = 0;
@ -37,18 +37,20 @@ size_t countBytesInFilter(const UInt8 * filt, size_t sz)
*/
const Int8 * pos = reinterpret_cast<const Int8 *>(filt);
const Int8 * end = pos + sz;
pos += start;
const Int8 * end_pos = pos + (end - start);
#if defined(__SSE2__) && defined(__POPCNT__)
const Int8 * end64 = pos + sz / 64 * 64;
const Int8 * end_pos64 = pos + (end - start) / 64 * 64;
for (; pos < end64; pos += 64)
for (; pos < end_pos64; pos += 64)
count += __builtin_popcountll(toBits64(pos));
/// TODO Add duff device for tail?
#endif
for (; pos < end; ++pos)
for (; pos < end_pos; ++pos)
count += *pos != 0;
return count;
@ -56,10 +58,10 @@ size_t countBytesInFilter(const UInt8 * filt, size_t sz)
size_t countBytesInFilter(const IColumn::Filter & filt)
{
return countBytesInFilter(filt.data(), filt.size());
return countBytesInFilter(filt.data(), 0, filt.size());
}
size_t countBytesInFilterWithNull(const IColumn::Filter & filt, const UInt8 * null_map)
size_t countBytesInFilterWithNull(const IColumn::Filter & filt, const UInt8 * null_map, size_t start, size_t end)
{
size_t count = 0;
@ -68,20 +70,20 @@ size_t countBytesInFilterWithNull(const IColumn::Filter & filt, const UInt8 * nu
* It would be better to use != 0, then this does not allow SSE2.
*/
const Int8 * pos = reinterpret_cast<const Int8 *>(filt.data());
const Int8 * pos2 = reinterpret_cast<const Int8 *>(null_map);
const Int8 * end = pos + filt.size();
const Int8 * pos = reinterpret_cast<const Int8 *>(filt.data()) + start;
const Int8 * pos2 = reinterpret_cast<const Int8 *>(null_map) + start;
const Int8 * end_pos = pos + (end - start);
#if defined(__SSE2__) && defined(__POPCNT__)
const Int8 * end64 = pos + filt.size() / 64 * 64;
const Int8 * end_pos64 = pos + (end - start) / 64 * 64;
for (; pos < end64; pos += 64, pos2 += 64)
for (; pos < end_pos64; pos += 64, pos2 += 64)
count += __builtin_popcountll(toBits64(pos) & ~toBits64(pos2));
/// TODO Add duff device for tail?
#endif
for (; pos < end; ++pos, ++pos2)
for (; pos < end_pos; ++pos, ++pos2)
count += (*pos & ~*pos2) != 0;
return count;
@ -96,17 +98,18 @@ std::vector<size_t> countColumnsSizeInSelector(IColumn::ColumnIndex num_columns,
return counts;
}
bool memoryIsByte(const void * data, size_t size, uint8_t byte)
bool memoryIsByte(const void * data, size_t start, size_t end, uint8_t byte)
{
size_t size = end - start;
if (size == 0)
return true;
const auto * ptr = reinterpret_cast<const uint8_t *>(data);
const auto * ptr = reinterpret_cast<const uint8_t *>(data) + start;
return *ptr == byte && memcmp(ptr, ptr + 1, size - 1) == 0;
}
bool memoryIsZero(const void * data, size_t size)
bool memoryIsZero(const void * data, size_t start, size_t end)
{
return memoryIsByte(data, size, 0x0);
return memoryIsByte(data, start, end, 0x0);
}
namespace ErrorCodes

View File

@ -53,17 +53,17 @@ inline UInt64 bytes64MaskToBits64Mask(const UInt8 * bytes64)
}
/// Counts how many bytes of `filt` are greater than zero.
size_t countBytesInFilter(const UInt8 * filt, size_t sz);
size_t countBytesInFilter(const UInt8 * filt, size_t start, size_t end);
size_t countBytesInFilter(const IColumn::Filter & filt);
size_t countBytesInFilterWithNull(const IColumn::Filter & filt, const UInt8 * null_map);
size_t countBytesInFilterWithNull(const IColumn::Filter & filt, const UInt8 * null_map, size_t start, size_t end);
/// Returns vector with num_columns elements. vector[i] is the count of i values in selector.
/// Selector must contain values from 0 to num_columns - 1. NOTE: this is not checked.
std::vector<size_t> countColumnsSizeInSelector(IColumn::ColumnIndex num_columns, const IColumn::Selector & selector);
/// Returns true, if the memory contains only zeros.
bool memoryIsZero(const void * data, size_t size);
bool memoryIsByte(const void * data, size_t size, uint8_t byte);
bool memoryIsZero(const void * data, size_t start, size_t end);
bool memoryIsByte(const void * data, size_t start, size_t end, uint8_t byte);
/// The general implementation of `filter` function for ColumnArray and ColumnString.
template <typename T>

View File

@ -256,10 +256,10 @@ private:
return lut[toLUTIndex(v)];
}
template <typename T, typename Divisor>
inline T roundDown(T x, Divisor divisor) const
template <typename DateOrTime, typename Divisor>
inline DateOrTime roundDown(DateOrTime x, Divisor divisor) const
{
static_assert(std::is_integral_v<T> && std::is_integral_v<Divisor>);
static_assert(std::is_integral_v<DateOrTime> && std::is_integral_v<Divisor>);
assert(divisor > 0);
if (likely(offset_is_whole_number_of_hours_during_epoch))
@ -273,7 +273,15 @@ private:
}
Time date = find(x).date;
return date + (x - date) / divisor * divisor;
Time res = date + (x - date) / divisor * divisor;
if constexpr (std::is_unsigned_v<DateOrTime> || std::is_same_v<DateOrTime, DayNum>)
{
if (unlikely(res < 0))
return 0;
return res;
}
else
return res;
}
public:
@ -564,11 +572,16 @@ public:
}
/// NOTE: Assuming timezone offset is a multiple of 15 minutes.
inline Time toStartOfMinute(Time t) const { return toStartOfMinuteInterval(t, 1); }
inline Time toStartOfFiveMinutes(Time t) const { return toStartOfMinuteInterval(t, 5); }
inline Time toStartOfFifteenMinutes(Time t) const { return toStartOfMinuteInterval(t, 15); }
inline Time toStartOfTenMinutes(Time t) const { return toStartOfMinuteInterval(t, 10); }
inline Time toStartOfHour(Time t) const { return roundDown(t, 3600); }
template <typename DateOrTime>
DateOrTime toStartOfMinute(DateOrTime t) const { return toStartOfMinuteInterval(t, 1); }
template <typename DateOrTime>
DateOrTime toStartOfFiveMinutes(DateOrTime t) const { return toStartOfMinuteInterval(t, 5); }
template <typename DateOrTime>
DateOrTime toStartOfFifteenMinutes(DateOrTime t) const { return toStartOfMinuteInterval(t, 15); }
template <typename DateOrTime>
DateOrTime toStartOfTenMinutes(DateOrTime t) const { return toStartOfMinuteInterval(t, 10); }
template <typename DateOrTime>
DateOrTime toStartOfHour(DateOrTime t) const { return roundDown(t, 3600); }
/** Number of calendar day since the beginning of UNIX epoch (1970-01-01 is zero)
* We use just two bytes for it. It covers the range up to 2105 and slightly more.
@ -953,7 +966,8 @@ public:
return lut[toLUTIndex(ExtendedDayNum(d / days * days))].date;
}
inline Time toStartOfHourInterval(Time t, UInt64 hours) const
template <typename DateOrTime>
DateOrTime toStartOfHourInterval(DateOrTime t, UInt64 hours) const
{
if (hours == 1)
return toStartOfHour(t);
@ -993,10 +1007,19 @@ public:
time = time / seconds * seconds;
}
return values.date + time;
Time res = values.date + time;
if constexpr (std::is_unsigned_v<DateOrTime> || std::is_same_v<DateOrTime, DayNum>)
{
if (unlikely(res < 0))
return 0;
return res;
}
else
return res;
}
inline Time toStartOfMinuteInterval(Time t, UInt64 minutes) const
template <typename DateOrTime>
DateOrTime toStartOfMinuteInterval(DateOrTime t, UInt64 minutes) const
{
UInt64 divisor = 60 * minutes;
if (likely(offset_is_whole_number_of_minutes_during_epoch))
@ -1007,10 +1030,19 @@ public:
}
Time date = find(t).date;
return date + (t - date) / divisor * divisor;
Time res = date + (t - date) / divisor * divisor;
if constexpr (std::is_unsigned_v<DateOrTime> || std::is_same_v<DateOrTime, DayNum>)
{
if (unlikely(res < 0))
return 0;
return res;
}
else
return res;
}
inline Time toStartOfSecondInterval(Time t, UInt64 seconds) const
template <typename DateOrTime>
DateOrTime toStartOfSecondInterval(DateOrTime t, UInt64 seconds) const
{
if (seconds == 1)
return t;

View File

@ -1036,14 +1036,14 @@ public:
return const_cast<std::decay_t<decltype(*this)> *>(this)->find(x, hash_value);
}
std::enable_if_t<Grower::performs_linear_probing_with_single_step, bool>
ALWAYS_INLINE erase(const Key & x)
ALWAYS_INLINE bool erase(const Key & x)
requires Grower::performs_linear_probing_with_single_step
{
return erase(x, hash(x));
}
std::enable_if_t<Grower::performs_linear_probing_with_single_step, bool>
ALWAYS_INLINE erase(const Key & x, size_t hash_value)
ALWAYS_INLINE bool erase(const Key & x, size_t hash_value)
requires Grower::performs_linear_probing_with_single_step
{
/** Deletion from open addressing hash table without tombstones
*

View File

@ -117,6 +117,25 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n)
bytes_read += size_decompressed;
bytes += size_decompressed;
}
else if (nextimpl_working_buffer_offset > 0)
{
/// Need to skip some bytes in decompressed data (seek happened before readBig call).
size_compressed = new_size_compressed;
bytes += offset();
/// This is for clang static analyzer.
assert(size_decompressed + additional_size_at_the_end_of_buffer > 0);
memory.resize(size_decompressed + additional_size_at_the_end_of_buffer);
working_buffer = Buffer(memory.data(), &memory[size_decompressed]);
decompress(working_buffer, size_decompressed, size_compressed_without_checksum);
/// Read partial data from first block. Won't run here at second block.
/// Avoid to call nextImpl and unnecessary memcpy in read when the second block fits entirely to output buffer.
size_t size_partial = std::min((size_decompressed - nextimpl_working_buffer_offset),(n - bytes_read));
pos = working_buffer.begin() + nextimpl_working_buffer_offset;
nextimpl_working_buffer_offset = 0;
bytes_read += read(to + bytes_read, size_partial);
}
else
{
size_compressed = new_size_compressed;
@ -124,17 +143,12 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n)
/// This is for clang static analyzer.
assert(size_decompressed + additional_size_at_the_end_of_buffer > 0);
memory.resize(size_decompressed + additional_size_at_the_end_of_buffer);
working_buffer = Buffer(memory.data(), &memory[size_decompressed]);
decompress(working_buffer, size_decompressed, size_compressed_without_checksum);
/// Manually take nextimpl_working_buffer_offset into account, because we don't use
/// nextImpl in this method.
pos = working_buffer.begin() + nextimpl_working_buffer_offset;
nextimpl_working_buffer_offset = 0;
///Read partial data from last block.
pos = working_buffer.begin();
bytes_read += read(to + bytes_read, n - bytes_read);
break;
}

View File

@ -249,6 +249,7 @@ class IColumn;
/** Settings for testing hedged requests */ \
M(Milliseconds, sleep_in_send_tables_status_ms, 0, "Time to sleep in sending tables status response in TCPHandler", 0) \
M(Milliseconds, sleep_in_send_data_ms, 0, "Time to sleep in sending data in TCPHandler", 0) \
M(Milliseconds, sleep_after_receiving_query_ms, 0, "Time to sleep after receiving query in TCPHandler", 0) \
M(UInt64, unknown_packet_in_send_data, 0, "Send unknown packet instead of data Nth data packet", 0) \
/** Settings for testing connection collector */ \
M(Milliseconds, sleep_in_receive_cancel_ms, 0, "Time to sleep in receiving cancel in TCPHandler", 0) \
@ -656,6 +657,8 @@ class IColumn;
M(Bool, input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference, false, "Allow to skip columns with unsupported types while schema inference for format Arrow", 0) \
M(String, column_names_for_schema_inference, "", "The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'", 0) \
M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \
M(Bool, input_format_protobuf_flatten_google_wrappers, false, "Enable Google wrappers for regular non-nested columns, e.g. google.protobuf.StringValue 'str' for String column 'str'. For Nullable columns empty wrappers are recognized as defaults, and missing as nulls", 0) \
M(Bool, output_format_protobuf_nullables_with_google_wrappers, false, "When serializing Nullable columns with Google wrappers, serialize default values as empty wrappers. If turned off, default and null values are not serialized", 0) \
\
M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \
M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \

View File

@ -91,8 +91,8 @@ inline UInt32 getDecimalScale(const DataTypeDecimal<T> & data_type)
}
template <typename FromDataType, typename ToDataType, typename ReturnType = void>
inline std::enable_if_t<IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>, ReturnType>
convertDecimalsImpl(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType& result)
requires (IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>)
inline ReturnType convertDecimalsImpl(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType& result)
{
using FromFieldType = typename FromDataType::FieldType;
using ToFieldType = typename ToDataType::FieldType;
@ -136,8 +136,8 @@ convertDecimalsImpl(const typename FromDataType::FieldType & value, UInt32 scale
}
template <typename FromDataType, typename ToDataType>
inline std::enable_if_t<IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>, typename ToDataType::FieldType>
convertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to)
requires (IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>)
inline typename ToDataType::FieldType convertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to)
{
using ToFieldType = typename ToDataType::FieldType;
ToFieldType result;
@ -148,15 +148,15 @@ convertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_fro
}
template <typename FromDataType, typename ToDataType>
inline std::enable_if_t<IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>, bool>
tryConvertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType& result)
requires (IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>)
inline bool tryConvertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType& result)
{
return convertDecimalsImpl<FromDataType, ToDataType, bool>(value, scale_from, scale_to, result);
}
template <typename FromDataType, typename ToDataType, typename ReturnType>
inline std::enable_if_t<IsDataTypeDecimal<FromDataType> && is_arithmetic_v<typename ToDataType::FieldType>, ReturnType>
convertFromDecimalImpl(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result)
requires (IsDataTypeDecimal<FromDataType> && is_arithmetic_v<typename ToDataType::FieldType>)
inline ReturnType convertFromDecimalImpl(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result)
{
using FromFieldType = typename FromDataType::FieldType;
using ToFieldType = typename ToDataType::FieldType;
@ -165,8 +165,8 @@ convertFromDecimalImpl(const typename FromDataType::FieldType & value, UInt32 sc
}
template <typename FromDataType, typename ToDataType>
inline std::enable_if_t<IsDataTypeDecimal<FromDataType> && is_arithmetic_v<typename ToDataType::FieldType>, typename ToDataType::FieldType>
convertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale)
requires (IsDataTypeDecimal<FromDataType> && is_arithmetic_v<typename ToDataType::FieldType>)
inline typename ToDataType::FieldType convertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale)
{
typename ToDataType::FieldType result;
@ -176,15 +176,15 @@ convertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale)
}
template <typename FromDataType, typename ToDataType>
inline std::enable_if_t<IsDataTypeDecimal<FromDataType> && is_arithmetic_v<typename ToDataType::FieldType>, bool>
tryConvertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result)
requires (IsDataTypeDecimal<FromDataType> && is_arithmetic_v<typename ToDataType::FieldType>)
inline bool tryConvertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result)
{
return convertFromDecimalImpl<FromDataType, ToDataType, bool>(value, scale, result);
}
template <typename FromDataType, typename ToDataType, typename ReturnType>
inline std::enable_if_t<is_arithmetic_v<typename FromDataType::FieldType> && IsDataTypeDecimal<ToDataType>, ReturnType>
convertToDecimalImpl(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result)
requires (is_arithmetic_v<typename FromDataType::FieldType> && IsDataTypeDecimal<ToDataType>)
inline ReturnType convertToDecimalImpl(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result)
{
using FromFieldType = typename FromDataType::FieldType;
using ToFieldType = typename ToDataType::FieldType;
@ -230,8 +230,8 @@ convertToDecimalImpl(const typename FromDataType::FieldType & value, UInt32 scal
}
template <typename FromDataType, typename ToDataType>
inline std::enable_if_t<is_arithmetic_v<typename FromDataType::FieldType> && IsDataTypeDecimal<ToDataType>, typename ToDataType::FieldType>
convertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale)
requires (is_arithmetic_v<typename FromDataType::FieldType> && IsDataTypeDecimal<ToDataType>)
inline typename ToDataType::FieldType convertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale)
{
typename ToDataType::FieldType result;
convertToDecimalImpl<FromDataType, ToDataType, void>(value, scale, result);
@ -239,8 +239,8 @@ convertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale)
}
template <typename FromDataType, typename ToDataType>
inline std::enable_if_t<is_arithmetic_v<typename FromDataType::FieldType> && IsDataTypeDecimal<ToDataType>, bool>
tryConvertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result)
requires (is_arithmetic_v<typename FromDataType::FieldType> && IsDataTypeDecimal<ToDataType>)
inline bool tryConvertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale, typename ToDataType::FieldType& result)
{
return convertToDecimalImpl<FromDataType, ToDataType, bool>(value, scale, result);
}

View File

@ -10,8 +10,6 @@ namespace DB
namespace ErrorCodes
{
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int INCONSISTENT_RESERVATIONS;
extern const int NO_RESERVATIONS_PROVIDED;
}
IVolume::IVolume(
@ -45,43 +43,4 @@ UInt64 IVolume::getMaxUnreservedFreeSpace() const
return res;
}
MultiDiskReservation::MultiDiskReservation(Reservations & reservations_, UInt64 size_)
: reservations(std::move(reservations_))
, size(size_)
{
if (reservations.empty())
{
throw Exception("At least one reservation must be provided to MultiDiskReservation", ErrorCodes::NO_RESERVATIONS_PROVIDED);
}
for (auto & reservation : reservations)
{
if (reservation->getSize() != size_)
{
throw Exception("Reservations must have same size", ErrorCodes::INCONSISTENT_RESERVATIONS);
}
}
}
Disks MultiDiskReservation::getDisks() const
{
Disks res;
res.reserve(reservations.size());
for (const auto & reservation : reservations)
{
res.push_back(reservation->getDisk());
}
return res;
}
void MultiDiskReservation::update(UInt64 new_size)
{
for (auto & reservation : reservations)
{
reservation->update(new_size);
}
size = new_size;
}
}

View File

@ -81,22 +81,4 @@ public:
bool perform_ttl_move_on_insert = true;
};
/// Reservation for multiple disks at once. Can be used in RAID1 implementation.
class MultiDiskReservation : public IReservation
{
public:
MultiDiskReservation(Reservations & reservations, UInt64 size);
UInt64 getSize() const override { return size; }
DiskPtr getDisk(size_t i) const override { return reservations[i]->getDisk(); }
Disks getDisks() const override;
void update(UInt64 new_size) override;
private:
Reservations reservations;
UInt64 size;
};
}

View File

@ -1,6 +1,7 @@
#include "StoragePolicy.h"
#include "DiskFactory.h"
#include "DiskLocal.h"
#include "createVolume.h"
#include <Interpreters/Context.h>
#include <Common/escapeForFileName.h>

View File

@ -5,7 +5,6 @@
#include <Disks/IDisk.h>
#include <Disks/IVolume.h>
#include <Disks/VolumeJBOD.h>
#include <Disks/VolumeRAID1.h>
#include <Disks/SingleDiskVolume.h>
#include <IO/WriteHelpers.h>
#include <Common/CurrentMetrics.h>

View File

@ -1,29 +0,0 @@
#include "VolumeRAID1.h"
#include <Common/StringUtils/StringUtils.h>
#include <Common/quoteString.h>
namespace DB
{
ReservationPtr VolumeRAID1::reserve(UInt64 bytes)
{
/// This volume can not store data which size is greater than `max_data_part_size`
/// to ensure that parts of size greater than that go to another volume(s).
if (max_data_part_size != 0 && bytes > max_data_part_size)
return {};
Reservations res(disks.size());
for (size_t i = 0; i < disks.size(); ++i)
{
res[i] = disks[i]->reserve(bytes);
if (!res[i])
return {};
}
return std::make_unique<MultiDiskReservation>(res, bytes);
}
}

View File

@ -1,49 +0,0 @@
#pragma once
#include <Disks/createVolume.h>
#include <Disks/VolumeJBOD.h>
namespace DB
{
class VolumeRAID1;
using VolumeRAID1Ptr = std::shared_ptr<VolumeRAID1>;
/// Volume which reserves space on each underlying disk.
///
/// NOTE: Just interface implementation, doesn't used in codebase,
/// also not available for user.
class VolumeRAID1 : public VolumeJBOD
{
public:
VolumeRAID1(String name_, Disks disks_, UInt64 max_data_part_size_, bool are_merges_avoided_in_config_)
: VolumeJBOD(name_, disks_, max_data_part_size_, are_merges_avoided_in_config_)
{
}
VolumeRAID1(
String name_,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
DiskSelectorPtr disk_selector)
: VolumeJBOD(name_, config, config_prefix, disk_selector)
{
}
VolumeRAID1(
VolumeRAID1 & volume_raid1,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
DiskSelectorPtr disk_selector)
: VolumeJBOD(volume_raid1, config, config_prefix, disk_selector)
{
}
VolumeType getType() const override { return VolumeType::RAID1; }
ReservationPtr reserve(UInt64 bytes) override;
};
}

View File

@ -2,7 +2,6 @@
#include <Disks/SingleDiskVolume.h>
#include <Disks/VolumeJBOD.h>
#include <Disks/VolumeRAID1.h>
#include <boost/algorithm/string.hpp>
@ -23,11 +22,6 @@ VolumePtr createVolumeFromReservation(const ReservationPtr & reservation, Volume
/// for such type of reservation will be with one disk.
return std::make_shared<SingleDiskVolume>(other_volume->getName(), reservation->getDisk(), other_volume->max_data_part_size);
}
if (other_volume->getType() == VolumeType::RAID1)
{
auto volume = std::dynamic_pointer_cast<VolumeRAID1>(other_volume);
return std::make_shared<VolumeRAID1>(volume->getName(), reservation->getDisks(), volume->max_data_part_size, volume->are_merges_avoided);
}
return nullptr;
}

View File

@ -105,6 +105,8 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.pretty.max_rows = settings.output_format_pretty_max_rows;
format_settings.pretty.max_value_width = settings.output_format_pretty_max_value_width;
format_settings.pretty.output_format_pretty_row_numbers = settings.output_format_pretty_row_numbers;
format_settings.protobuf.input_flatten_google_wrappers = settings.input_format_protobuf_flatten_google_wrappers;
format_settings.protobuf.output_nullables_with_google_wrappers = settings.output_format_protobuf_nullables_with_google_wrappers;
format_settings.regexp.escaping_rule = settings.format_regexp_escaping_rule;
format_settings.regexp.regexp = settings.format_regexp;
format_settings.regexp.skip_unmatched = settings.format_regexp_skip_unmatched;
@ -218,7 +220,6 @@ InputFormatPtr FormatFactory::getInput(
(ReadBuffer & input) -> InputFormatPtr
{ return input_getter(input, sample, row_input_format_params, format_settings); };
ParallelParsingInputFormat::Params params{
buf, sample, parser_creator, file_segmentation_engine, name, settings.max_threads, settings.min_chunk_bytes_for_parallel_parsing,
context->getApplicationType() == Context::ApplicationType::SERVER};

View File

@ -170,6 +170,8 @@ struct FormatSettings
struct
{
bool input_flatten_google_wrappers = false;
bool output_nullables_with_google_wrappers = false;
/**
* Some buffers (kafka / rabbit) split the rows internally using callback,
* and always send one row per message, so we can push there formats

View File

@ -102,14 +102,43 @@ namespace
}
};
bool isGoogleWrapperMessage(const MessageDescriptor & message_descriptor)
{
auto message_type = message_descriptor.well_known_type();
return (message_type >= google::protobuf::Descriptor::WELLKNOWNTYPE_DOUBLEVALUE)
&& (message_type <= google::protobuf::Descriptor::WELLKNOWNTYPE_BOOLVALUE);
}
bool isGoogleWrapperField(const FieldDescriptor & field_descriptor)
{
const auto * message_descriptor = field_descriptor.message_type();
if (message_descriptor == nullptr)
return false;
return isGoogleWrapperMessage(*message_descriptor);
}
bool isGoogleWrapperField(const FieldDescriptor * field_descriptor)
{
if (field_descriptor == nullptr)
return false;
return isGoogleWrapperField(*field_descriptor);
}
std::string_view googleWrapperColumnName(const FieldDescriptor & field_descriptor)
{
assert(isGoogleWrapperField(field_descriptor));
return field_descriptor.message_type()->field(0)->name();
}
// Should we omit null values (zero for numbers / empty string for strings) while storing them.
bool shouldSkipZeroOrEmpty(const FieldDescriptor & field_descriptor)
bool shouldSkipZeroOrEmpty(const FieldDescriptor & field_descriptor, bool google_wrappers_special_treatment = false)
{
if (!field_descriptor.is_optional())
return false;
if (field_descriptor.containing_type()->options().map_entry())
return false;
if (google_wrappers_special_treatment && isGoogleWrapperField(field_descriptor))
return false;
return field_descriptor.message_type() || (field_descriptor.file()->syntax() == google::protobuf::FileDescriptor::SYNTAX_PROTO3);
}
@ -143,7 +172,6 @@ namespace
return field_descriptor.file()->syntax() == google::protobuf::FileDescriptor::SYNTAX_PROTO3;
}
WriteBuffer & writeIndent(WriteBuffer & out, size_t size) { return out << String(size * 4, ' '); }
@ -1783,6 +1811,15 @@ namespace
column_nullable.insertDefault();
}
void insertNestedDefaults(size_t row_num)
{
auto & column_nullable = assert_cast<ColumnNullable &>(column->assumeMutableRef());
if (row_num < column_nullable.size())
return;
column_nullable.getNestedColumn().insertDefault();
column_nullable.getNullMapData().push_back(0);
}
void describeTree(WriteBuffer & out, size_t indent) const override
{
writeIndent(out, indent) << "ProtobufSerializerNullable ->\n";
@ -2151,12 +2188,15 @@ namespace
std::vector<FieldDesc> && field_descs_,
const FieldDescriptor * parent_field_descriptor_,
bool with_length_delimiter_,
bool google_wrappers_special_treatment_,
std::unique_ptr<RowInputMissingColumnsFiller> missing_columns_filler_,
const ProtobufReaderOrWriter & reader_or_writer_)
: parent_field_descriptor(parent_field_descriptor_)
, with_length_delimiter(with_length_delimiter_)
, google_wrappers_special_treatment(google_wrappers_special_treatment_)
, missing_columns_filler(std::move(missing_columns_filler_))
, should_skip_if_empty(parent_field_descriptor ? shouldSkipZeroOrEmpty(*parent_field_descriptor) : false)
, should_skip_if_empty(parent_field_descriptor
? shouldSkipZeroOrEmpty(*parent_field_descriptor, google_wrappers_special_treatment_) : false)
, reader(reader_or_writer_.reader)
, writer(reader_or_writer_.writer)
{
@ -2195,7 +2235,7 @@ namespace
info.field_serializer->setColumns(field_columns.data(), field_columns.size());
}
if (reader)
if (reader || (google_wrappers_special_treatment && isGoogleWrapperField(parent_field_descriptor)))
{
mutable_columns.resize(num_columns_);
for (size_t i : collections::range(num_columns_))
@ -2239,7 +2279,8 @@ namespace
if (parent_field_descriptor)
{
bool is_group = (parent_field_descriptor->type() == FieldTypeId::TYPE_GROUP);
writer->endNestedMessage(parent_field_descriptor->number(), is_group, should_skip_if_empty);
writer->endNestedMessage(parent_field_descriptor->number(), is_group,
should_skip_if_empty || (google_wrappers_special_treatment && isNullGoogleWrapper(row_num)));
}
else if (has_envelope_as_parent)
{
@ -2280,9 +2321,19 @@ namespace
if (info.field_read)
info.field_read = false;
else
{
if (google_wrappers_special_treatment && isNullableGoogleWrapper())
{
auto * nullable_ser = dynamic_cast<ProtobufSerializerNullable*>(info.field_serializer.get());
nullable_ser->insertNestedDefaults(row_num);
}
else
{
info.field_serializer->insertDefaults(row_num);
}
}
}
}
catch (...)
{
for (auto & column : mutable_columns)
@ -2362,6 +2413,16 @@ namespace
missing_columns_filler->addDefaults(mutable_columns, row_num);
}
bool isNullGoogleWrapper(size_t row_num)
{
return isGoogleWrapperField(parent_field_descriptor) && mutable_columns[0].get()->isNullAt(row_num);
}
bool isNullableGoogleWrapper()
{
return isGoogleWrapperField(parent_field_descriptor) && mutable_columns[0].get()->isNullable();
}
struct FieldInfo
{
FieldInfo(
@ -2386,6 +2447,7 @@ namespace
const FieldDescriptor * const parent_field_descriptor;
bool has_envelope_as_parent = false;
const bool with_length_delimiter;
const bool google_wrappers_special_treatment;
const std::unique_ptr<RowInputMissingColumnsFiller> missing_columns_filler;
const bool should_skip_if_empty;
ProtobufReader * const reader;
@ -2701,7 +2763,8 @@ namespace
std::vector<size_t> & missing_column_indices,
const MessageDescriptor & message_descriptor,
bool with_length_delimiter,
bool with_envelope)
bool with_envelope,
bool google_wrappers_special_treatment)
{
root_serializer_ptr = std::make_shared<ProtobufSerializer *>();
get_root_desc_function = [root_serializer_ptr = root_serializer_ptr](size_t indent) -> String
@ -2718,6 +2781,7 @@ namespace
data_types.data(),
message_descriptor,
with_length_delimiter,
google_wrappers_special_treatment,
/* parent_field_descriptor = */ nullptr,
used_column_indices,
/* columns_are_reordered_outside = */ false,
@ -2825,7 +2889,8 @@ namespace
static bool findFieldsByColumnName(
const std::string_view & column_name,
const MessageDescriptor & message_descriptor,
std::vector<std::pair<const FieldDescriptor *, std::string_view /* suffix */>> & out_field_descriptors_with_suffixes)
std::vector<std::pair<const FieldDescriptor *, std::string_view /* suffix */>> & out_field_descriptors_with_suffixes,
bool google_wrappers_special_treatment)
{
out_field_descriptors_with_suffixes.clear();
@ -2836,7 +2901,11 @@ namespace
const auto & field_descriptor = *message_descriptor.field(i);
if (columnNameEqualsToFieldName(column_name, field_descriptor))
{
out_field_descriptors_with_suffixes.emplace_back(&field_descriptor, std::string_view{});
std::string_view suffix =
google_wrappers_special_treatment && isGoogleWrapperField(field_descriptor)
? googleWrapperColumnName(field_descriptor)
: "";
out_field_descriptors_with_suffixes.emplace_back(&field_descriptor, suffix);
break;
}
}
@ -2916,6 +2985,7 @@ namespace
const DataTypePtr * data_types,
const MessageDescriptor & message_descriptor,
bool with_length_delimiter,
bool google_wrappers_special_treatment,
const FieldDescriptor * parent_field_descriptor,
std::vector<size_t> & used_column_indices,
bool columns_are_reordered_outside,
@ -2932,6 +3002,7 @@ namespace
data_types,
message_descriptor,
with_length_delimiter,
google_wrappers_special_treatment,
parent_field_descriptor,
used_column_indices,
columns_are_reordered_outside,
@ -2944,6 +3015,7 @@ namespace
const DataTypePtr * data_types,
const MessageDescriptor & message_descriptor,
bool with_length_delimiter,
bool google_wrappers_special_treatment,
const FieldDescriptor * parent_field_descriptor,
std::vector<size_t> & used_column_indices,
bool columns_are_reordered_outside,
@ -2998,14 +3070,15 @@ namespace
const auto & column_name = column_names[column_idx];
const auto & data_type = data_types[column_idx];
if (!findFieldsByColumnName(column_name, message_descriptor, field_descriptors_with_suffixes))
if (!findFieldsByColumnName(column_name, message_descriptor, field_descriptors_with_suffixes, google_wrappers_special_treatment))
continue;
if ((field_descriptors_with_suffixes.size() == 1) && field_descriptors_with_suffixes[0].second.empty())
{
/// Simple case: one column is serialized as one field.
const auto & field_descriptor = *field_descriptors_with_suffixes[0].first;
auto field_serializer = buildFieldSerializer(column_name, data_type, field_descriptor, field_descriptor.is_repeated());
auto field_serializer = buildFieldSerializer(column_name, data_type,
field_descriptor, field_descriptor.is_repeated(), google_wrappers_special_treatment);
if (field_serializer)
{
@ -3068,6 +3141,7 @@ namespace
nested_data_types.data(),
*field_descriptor->message_type(),
/* with_length_delimiter = */ false,
google_wrappers_special_treatment,
field_descriptor,
used_column_indices_in_nested,
/* columns_are_reordered_outside = */ true,
@ -3108,6 +3182,7 @@ namespace
nested_data_types.data(),
*field_descriptor->message_type(),
/* with_length_delimiter = */ false,
google_wrappers_special_treatment,
field_descriptor,
used_column_indices_in_nested,
/* columns_are_reordered_outside = */ true,
@ -3161,7 +3236,7 @@ namespace
}
return std::make_unique<ProtobufSerializerMessage>(
std::move(field_descs), parent_field_descriptor, with_length_delimiter,
std::move(field_descs), parent_field_descriptor, with_length_delimiter, google_wrappers_special_treatment,
std::move(missing_columns_filler), reader_or_writer);
}
@ -3171,7 +3246,8 @@ namespace
const std::string_view & column_name,
const DataTypePtr & data_type,
const FieldDescriptor & field_descriptor,
bool allow_repeat)
bool allow_repeat,
bool google_wrappers_special_treatment)
{
auto data_type_id = data_type->getTypeId();
switch (data_type_id)
@ -3208,7 +3284,8 @@ namespace
case TypeIndex::Nullable:
{
const auto & nullable_data_type = assert_cast<const DataTypeNullable &>(*data_type);
auto nested_serializer = buildFieldSerializer(column_name, nullable_data_type.getNestedType(), field_descriptor, allow_repeat);
auto nested_serializer = buildFieldSerializer(column_name, nullable_data_type.getNestedType(),
field_descriptor, allow_repeat, google_wrappers_special_treatment);
if (!nested_serializer)
return nullptr;
return std::make_unique<ProtobufSerializerNullable>(std::move(nested_serializer));
@ -3218,7 +3295,8 @@ namespace
{
const auto & low_cardinality_data_type = assert_cast<const DataTypeLowCardinality &>(*data_type);
auto nested_serializer
= buildFieldSerializer(column_name, low_cardinality_data_type.getDictionaryType(), field_descriptor, allow_repeat);
= buildFieldSerializer(column_name, low_cardinality_data_type.getDictionaryType(),
field_descriptor, allow_repeat, google_wrappers_special_treatment);
if (!nested_serializer)
return nullptr;
return std::make_unique<ProtobufSerializerLowCardinality>(std::move(nested_serializer));
@ -3227,7 +3305,8 @@ namespace
case TypeIndex::Map:
{
const auto & map_data_type = assert_cast<const DataTypeMap &>(*data_type);
auto nested_serializer = buildFieldSerializer(column_name, map_data_type.getNestedType(), field_descriptor, allow_repeat);
auto nested_serializer = buildFieldSerializer(column_name, map_data_type.getNestedType(),
field_descriptor, allow_repeat, google_wrappers_special_treatment);
if (!nested_serializer)
return nullptr;
return std::make_unique<ProtobufSerializerMap>(std::move(nested_serializer));
@ -3242,7 +3321,8 @@ namespace
throwFieldNotRepeated(field_descriptor, column_name);
auto nested_serializer = buildFieldSerializer(column_name, array_data_type.getNestedType(), field_descriptor,
/* allow_repeat = */ false); // We do our repeating now, so for nested type we forget about the repeating.
/* allow_repeat = */ false, // We do our repeating now, so for nested type we forget about the repeating.
google_wrappers_special_treatment);
if (!nested_serializer)
return nullptr;
return std::make_unique<ProtobufSerializerArray>(std::move(nested_serializer));
@ -3266,6 +3346,7 @@ namespace
tuple_data_type.getElements().data(),
*field_descriptor.message_type(),
/* with_length_delimiter = */ false,
google_wrappers_special_treatment,
&field_descriptor,
used_column_indices,
/* columns_are_reordered_outside = */ false,
@ -3292,7 +3373,8 @@ namespace
for (const auto & nested_data_type : tuple_data_type.getElements())
{
auto nested_serializer = buildFieldSerializer(column_name, nested_data_type, field_descriptor,
/* allow_repeat = */ false); // We do our repeating now, so for nested type we forget about the repeating.
/* allow_repeat = */ false, // We do our repeating now, so for nested type we forget about the repeating.
google_wrappers_special_treatment);
if (!nested_serializer)
break;
nested_serializers.push_back(std::move(nested_serializer));
@ -3439,9 +3521,10 @@ std::unique_ptr<ProtobufSerializer> ProtobufSerializer::create(
const google::protobuf::Descriptor & message_descriptor,
bool with_length_delimiter,
bool with_envelope,
bool flatten_google_wrappers,
ProtobufReader & reader)
{
return ProtobufSerializerBuilder(reader).buildMessageSerializer(column_names, data_types, missing_column_indices, message_descriptor, with_length_delimiter, with_envelope);
return ProtobufSerializerBuilder(reader).buildMessageSerializer(column_names, data_types, missing_column_indices, message_descriptor, with_length_delimiter, with_envelope, flatten_google_wrappers);
}
std::unique_ptr<ProtobufSerializer> ProtobufSerializer::create(
@ -3450,10 +3533,11 @@ std::unique_ptr<ProtobufSerializer> ProtobufSerializer::create(
const google::protobuf::Descriptor & message_descriptor,
bool with_length_delimiter,
bool with_envelope,
bool defaults_for_nullable_google_wrappers,
ProtobufWriter & writer)
{
std::vector<size_t> missing_column_indices;
return ProtobufSerializerBuilder(writer).buildMessageSerializer(column_names, data_types, missing_column_indices, message_descriptor, with_length_delimiter, with_envelope);
return ProtobufSerializerBuilder(writer).buildMessageSerializer(column_names, data_types, missing_column_indices, message_descriptor, with_length_delimiter, with_envelope, defaults_for_nullable_google_wrappers);
}
NamesAndTypesList protobufSchemaToCHSchema(const google::protobuf::Descriptor * message_descriptor)

View File

@ -41,6 +41,7 @@ public:
const google::protobuf::Descriptor & message_descriptor,
bool with_length_delimiter,
bool with_envelope,
bool flatten_google_wrappers,
ProtobufReader & reader);
static std::unique_ptr<ProtobufSerializer> create(
@ -49,6 +50,7 @@ public:
const google::protobuf::Descriptor & message_descriptor,
bool with_length_delimiter,
bool with_envelope,
bool defaults_for_nullable_google_wrappers,
ProtobufWriter & writer);
};

View File

@ -15,6 +15,9 @@ void registerFileSegmentationEngineRegexp(FormatFactory & factory);
void registerFileSegmentationEngineJSONAsString(FormatFactory & factory);
void registerFileSegmentationEngineJSONAsObject(FormatFactory & factory);
void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory);
#if USE_HIVE
void registerFileSegmentationEngineHiveText(FormatFactory & factory);
#endif
/// Formats for both input/output.
@ -128,6 +131,10 @@ void registerFormats()
registerFileSegmentationEngineJSONAsString(factory);
registerFileSegmentationEngineJSONAsObject(factory);
registerFileSegmentationEngineJSONCompactEachRow(factory);
#if USE_HIVE
registerFileSegmentationEngineHiveText(factory);
#endif
registerInputFormatNative(factory);
registerOutputFormatNative(factory);

View File

@ -3576,7 +3576,7 @@ private:
const auto & nullable_col = assert_cast<const ColumnNullable &>(*col);
const auto & null_map = nullable_col.getNullMapData();
if (!memoryIsZero(null_map.data(), null_map.size()))
if (!memoryIsZero(null_map.data(), 0, null_map.size()))
throw Exception{"Cannot convert NULL value to non-Nullable type",
ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN};
}

View File

@ -186,7 +186,7 @@ ColumnPtr FunctionArrayReduce::executeImpl(const ColumnsWithTypeAndName & argume
while (const auto * func = typeid_cast<const AggregateFunctionState *>(that))
that = func->getNestedFunction().get();
that->addBatchArray(input_rows_count, places.data(), 0, aggregate_arguments, offsets->data(), arena.get());
that->addBatchArray(0, input_rows_count, places.data(), 0, aggregate_arguments, offsets->data(), arena.get());
}
for (size_t i = 0; i < input_rows_count; ++i)

View File

@ -147,7 +147,7 @@ ColumnPtr FunctionInitializeAggregation::executeImpl(const ColumnsWithTypeAndNam
/// Unnest consecutive trailing -State combinators
while (const auto * func = typeid_cast<const AggregateFunctionState *>(that))
that = func->getNestedFunction().get();
that->addBatch(input_rows_count, places.data(), 0, aggregate_arguments, arena.get());
that->addBatch(0, input_rows_count, places.data(), 0, aggregate_arguments, arena.get());
}
for (size_t i = 0; i < input_rows_count; ++i)

View File

@ -125,7 +125,7 @@ public:
if (in)
{
const auto & in_data = in->getData();
if (!memoryIsZero(in_data.data(), in_data.size() * sizeof(in_data[0])))
if (!memoryIsZero(in_data.data(), 0, in_data.size() * sizeof(in_data[0])))
{
throw Exception(ErrorCodes::FUNCTION_THROW_IF_VALUE_IS_NON_ZERO,
message.value_or("Value passed to '" + getName() + "' function is non zero"));

View File

@ -164,7 +164,7 @@ namespace
return time_zone.toStartOfHourInterval(t, hours);
}
static UInt32 execute(Int64 t, Int64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 hours, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfHourInterval(t / scale_multiplier, hours);
}
@ -182,7 +182,7 @@ namespace
return time_zone.toStartOfMinuteInterval(t, minutes);
}
static UInt32 execute(Int64 t, Int64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 minutes, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfMinuteInterval(t / scale_multiplier, minutes);
}
@ -200,7 +200,7 @@ namespace
return time_zone.toStartOfSecondInterval(t, seconds);
}
static UInt32 execute(Int64 t, Int64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier)
static Int64 execute(Int64 t, Int64 seconds, const DateLUTImpl & time_zone, Int64 scale_multiplier)
{
return time_zone.toStartOfSecondInterval(t / scale_multiplier, seconds);
}

View File

@ -19,6 +19,8 @@ public:
const ReadBuffer & getWrappedReadBuffer() const { return *in; }
ReadBuffer & getWrappedReadBuffer() { return *in; }
void prefetch() override { in->prefetch(); }
protected:
std::unique_ptr<ReadBuffer> in;
};

View File

@ -24,6 +24,8 @@ public:
~PeekableReadBuffer() override;
void prefetch() override { sub_buf.prefetch(); }
/// Sets checkpoint at current position
ALWAYS_INLINE inline void setCheckpoint()
{

View File

@ -354,11 +354,6 @@ Aggregator::Params::StatsCollectingParams::StatsCollectingParams(
{
}
bool Aggregator::Params::StatsCollectingParams::isCollectionAndUseEnabled() const
{
return key != 0;
}
Block Aggregator::getHeader(bool final) const
{
return params.getHeader(final);
@ -569,13 +564,12 @@ Aggregator::Aggregator(const Params & params_) : params(params_)
void Aggregator::compileAggregateFunctionsIfNeeded()
{
static std::unordered_map<UInt128, UInt64, UInt128Hash> aggregate_functions_description_to_count;
static std::mutex mtx;
static std::mutex mutex;
if (!params.compile_aggregate_expressions)
return;
std::vector<AggregateFunctionWithOffset> functions_to_compile;
size_t aggregate_instructions_size = 0;
String functions_description;
is_aggregate_function_compiled.resize(aggregate_functions.size());
@ -603,7 +597,6 @@ void Aggregator::compileAggregateFunctionsIfNeeded()
functions_description += ' ';
}
++aggregate_instructions_size;
is_aggregate_function_compiled[i] = function->isCompilable();
}
@ -617,10 +610,11 @@ void Aggregator::compileAggregateFunctionsIfNeeded()
aggregate_functions_description_hash.get128(aggregate_functions_description_hash_key);
{
std::lock_guard<std::mutex> lock(mtx);
std::lock_guard<std::mutex> lock(mutex);
if (aggregate_functions_description_to_count[aggregate_functions_description_hash_key]++ < params.min_count_to_compile_aggregate_expression)
return;
}
if (auto * compilation_cache = CompiledExpressionCacheFactory::instance().tryGetCache())
{
@ -639,7 +633,6 @@ void Aggregator::compileAggregateFunctionsIfNeeded()
auto compiled_aggregate_functions = compileAggregateFunctions(getJITInstance(), functions_to_compile, functions_description);
compiled_aggregate_functions_holder = std::make_shared<CompiledAggregateFunctionsHolder>(std::move(compiled_aggregate_functions));
}
}
}
#endif
@ -851,6 +844,45 @@ bool Aggregator::hasSparseArguments(AggregateFunctionInstruction * aggregate_ins
return false;
}
void Aggregator::executeOnBlockSmall(
AggregatedDataVariants & result,
size_t row_begin,
size_t row_end,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions) const
{
/// `result` will destroy the states of aggregate functions in the destructor
result.aggregator = this;
/// How to perform the aggregation?
if (result.empty())
{
initDataVariantsWithSizeHint(result, method_chosen, params);
result.keys_size = params.keys_size;
result.key_sizes = key_sizes;
}
executeImpl(result, row_begin, row_end, key_columns, aggregate_instructions);
}
void Aggregator::executeImpl(
AggregatedDataVariants & result,
size_t row_begin,
size_t row_end,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions,
bool no_more_keys,
AggregateDataPtr overflow_row) const
{
#define M(NAME, IS_TWO_LEVEL) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
executeImpl(*result.NAME, result.aggregates_pool, row_begin, row_end, key_columns, aggregate_instructions, no_more_keys, overflow_row);
if (false) {} // NOLINT
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
}
/** It's interesting - if you remove `noinline`, then gcc for some reason will inline this function, and the performance decreases (~ 10%).
* (Probably because after the inline of this function, more internal functions no longer be inlined.)
* Inline does not make sense, since the inner loop is entirely inside this function.
@ -859,7 +891,8 @@ template <typename Method>
void NO_INLINE Aggregator::executeImpl(
Method & method,
Arena * aggregates_pool,
size_t rows,
size_t row_begin,
size_t row_end,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions,
bool no_more_keys,
@ -872,17 +905,17 @@ void NO_INLINE Aggregator::executeImpl(
#if USE_EMBEDDED_COMPILER
if (compiled_aggregate_functions_holder && !hasSparseArguments(aggregate_instructions))
{
executeImplBatch<false, true>(method, state, aggregates_pool, rows, aggregate_instructions, overflow_row);
executeImplBatch<false, true>(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row);
}
else
#endif
{
executeImplBatch<false, false>(method, state, aggregates_pool, rows, aggregate_instructions, overflow_row);
executeImplBatch<false, false>(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row);
}
}
else
{
executeImplBatch<true, false>(method, state, aggregates_pool, rows, aggregate_instructions, overflow_row);
executeImplBatch<true, false>(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row);
}
}
@ -891,7 +924,8 @@ void NO_INLINE Aggregator::executeImplBatch(
Method & method,
typename Method::State & state,
Arena * aggregates_pool,
size_t rows,
size_t row_begin,
size_t row_end,
AggregateFunctionInstruction * aggregate_instructions,
AggregateDataPtr overflow_row) const
{
@ -903,7 +937,7 @@ void NO_INLINE Aggregator::executeImplBatch(
/// For all rows.
AggregateDataPtr place = aggregates_pool->alloc(0);
for (size_t i = 0; i < rows; ++i)
for (size_t i = row_begin; i < row_end; ++i)
state.emplaceKey(method.data, i, *aggregates_pool).setMapped(place);
return;
}
@ -927,7 +961,8 @@ void NO_INLINE Aggregator::executeImplBatch(
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
{
inst->batch_that->addBatchLookupTable8(
rows,
row_begin,
row_end,
reinterpret_cast<AggregateDataPtr *>(method.data.data()),
inst->state_offset,
[&](AggregateDataPtr & aggregate_data)
@ -943,10 +978,14 @@ void NO_INLINE Aggregator::executeImplBatch(
}
}
std::unique_ptr<AggregateDataPtr[]> places(new AggregateDataPtr[rows]);
/// NOTE: only row_end-row_start is required, but:
/// - this affects only optimize_aggregation_in_order,
/// - this is just a pointer, so it should not be significant,
/// - and plus this will require other changes in the interface.
std::unique_ptr<AggregateDataPtr[]> places(new AggregateDataPtr[row_end]);
/// For all rows.
for (size_t i = 0; i < rows; ++i)
for (size_t i = row_begin; i < row_end; ++i)
{
AggregateDataPtr aggregate_data = nullptr;
@ -1031,7 +1070,7 @@ void NO_INLINE Aggregator::executeImplBatch(
}
auto add_into_aggregate_states_function = compiled_aggregate_functions_holder->compiled_aggregate_functions.add_into_aggregate_states_function;
add_into_aggregate_states_function(rows, columns_data.data(), places.get());
add_into_aggregate_states_function(row_begin, row_end, columns_data.data(), places.get());
}
#endif
@ -1047,11 +1086,11 @@ void NO_INLINE Aggregator::executeImplBatch(
AggregateFunctionInstruction * inst = aggregate_instructions + i;
if (inst->offsets)
inst->batch_that->addBatchArray(rows, places.get(), inst->state_offset, inst->batch_arguments, inst->offsets, aggregates_pool);
inst->batch_that->addBatchArray(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, inst->offsets, aggregates_pool);
else if (inst->has_sparse_arguments)
inst->batch_that->addBatchSparse(places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool);
inst->batch_that->addBatchSparse(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool);
else
inst->batch_that->addBatch(rows, places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool);
inst->batch_that->addBatch(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool);
}
}
@ -1059,10 +1098,13 @@ void NO_INLINE Aggregator::executeImplBatch(
template <bool use_compiled_functions>
void NO_INLINE Aggregator::executeWithoutKeyImpl(
AggregatedDataWithoutKey & res,
size_t rows,
size_t row_begin, size_t row_end,
AggregateFunctionInstruction * aggregate_instructions,
Arena * arena) const
{
if (row_begin == row_end)
return;
#if USE_EMBEDDED_COMPILER
if constexpr (use_compiled_functions)
{
@ -1083,7 +1125,7 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl(
}
auto add_into_aggregate_states_function_single_place = compiled_aggregate_functions_holder->compiled_aggregate_functions.add_into_aggregate_states_function_single_place;
add_into_aggregate_states_function_single_place(rows, columns_data.data(), res);
add_into_aggregate_states_function_single_place(row_begin, row_end, columns_data.data(), res);
#if defined(MEMORY_SANITIZER)
@ -1114,11 +1156,23 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl(
if (inst->offsets)
inst->batch_that->addBatchSinglePlace(
inst->offsets[static_cast<ssize_t>(rows - 1)], res + inst->state_offset, inst->batch_arguments, arena);
0,
inst->offsets[static_cast<ssize_t>(row_end - 1)],
res + inst->state_offset,
inst->batch_arguments,
arena);
else if (inst->has_sparse_arguments)
inst->batch_that->addBatchSparseSinglePlace(res + inst->state_offset, inst->batch_arguments, arena);
inst->batch_that->addBatchSparseSinglePlace(
row_begin, row_end,
res + inst->state_offset,
inst->batch_arguments,
arena);
else
inst->batch_that->addBatchSinglePlace(rows, res + inst->state_offset, inst->batch_arguments, arena);
inst->batch_that->addBatchSinglePlace(
row_begin, row_end,
res + inst->state_offset,
inst->batch_arguments,
arena);
}
}
@ -1210,16 +1264,27 @@ void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns
}
bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & result,
ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, bool & no_more_keys) const
bool Aggregator::executeOnBlock(const Block & block,
AggregatedDataVariants & result,
ColumnRawPtrs & key_columns,
AggregateColumns & aggregate_columns,
bool & no_more_keys) const
{
UInt64 num_rows = block.rows();
return executeOnBlock(block.getColumns(), num_rows, result, key_columns, aggregate_columns, no_more_keys);
return executeOnBlock(block.getColumns(),
/* row_begin= */ 0, block.rows(),
result,
key_columns,
aggregate_columns,
no_more_keys);
}
bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedDataVariants & result,
ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, bool & no_more_keys) const
bool Aggregator::executeOnBlock(Columns columns,
size_t row_begin, size_t row_end,
AggregatedDataVariants & result,
ColumnRawPtrs & key_columns,
AggregateColumns & aggregate_columns,
bool & no_more_keys) const
{
/// `result` will destroy the states of aggregate functions in the destructor
result.aggregator = this;
@ -1275,27 +1340,19 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData
// #if USE_EMBEDDED_COMPILER
// if (compiled_aggregate_functions_holder)
// {
// executeWithoutKeyImpl<true>(result.without_key, num_rows, aggregate_functions_instructions.data(), result.aggregates_pool);
// executeWithoutKeyImpl<true>(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
// }
// else
// #endif
{
executeWithoutKeyImpl<false>(result.without_key, num_rows, aggregate_functions_instructions.data(), result.aggregates_pool);
executeWithoutKeyImpl<false>(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
}
}
else
{
/// This is where data is written that does not fit in `max_rows_to_group_by` with `group_by_overflow_mode = any`.
AggregateDataPtr overflow_row_ptr = params.overflow_row ? result.without_key : nullptr;
#define M(NAME, IS_TWO_LEVEL) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
executeImpl(*result.NAME, result.aggregates_pool, num_rows, key_columns, aggregate_functions_instructions.data(), \
no_more_keys, overflow_row_ptr);
if (false) {} // NOLINT
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
executeImpl(result, row_begin, row_end, key_columns, aggregate_functions_instructions.data(), no_more_keys, overflow_row_ptr);
}
size_t result_size = result.sizeWithoutOverflowRow();
@ -1717,7 +1774,7 @@ void NO_INLINE Aggregator::convertToBlockImplFinal(
}
auto insert_aggregates_into_columns_function = compiled_functions.insert_aggregates_into_columns_function;
insert_aggregates_into_columns_function(places.size(), columns_data.data(), places.data());
insert_aggregates_into_columns_function(0, places.size(), columns_data.data(), places.data());
}
#endif
@ -1746,7 +1803,7 @@ void NO_INLINE Aggregator::convertToBlockImplFinal(
bool is_state = aggregate_functions[destroy_index]->isState();
bool destroy_place_after_insert = !is_state;
aggregate_functions[destroy_index]->insertResultIntoBatch(places.size(), places.data(), offset, *final_aggregate_column, arena, destroy_place_after_insert);
aggregate_functions[destroy_index]->insertResultIntoBatch(0, places.size(), places.data(), offset, *final_aggregate_column, arena, destroy_place_after_insert);
}
}
catch (...)
@ -1766,7 +1823,7 @@ void NO_INLINE Aggregator::convertToBlockImplFinal(
}
size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index];
aggregate_functions[aggregate_functions_destroy_index]->destroyBatch(places.size(), places.data(), offset);
aggregate_functions[aggregate_functions_destroy_index]->destroyBatch(0, places.size(), places.data(), offset);
}
if (exception)
@ -2526,6 +2583,7 @@ void NO_INLINE Aggregator::mergeStreamsImplCase(
/// For all rows.
size_t rows = block.rows();
std::unique_ptr<AggregateDataPtr[]> places(new AggregateDataPtr[rows]);
for (size_t i = 0; i < rows; ++i)
@ -2564,7 +2622,8 @@ void NO_INLINE Aggregator::mergeStreamsImplCase(
{
/// Merge state of aggregate functions.
aggregate_functions[j]->mergeBatch(
rows, places.get(), offsets_of_aggregate_states[j],
0, rows,
places.get(), offsets_of_aggregate_states[j],
aggregate_columns[j]->data(),
aggregates_pool);
}

View File

@ -941,9 +941,10 @@ public:
size_t max_entries_for_hash_table_stats_,
size_t max_size_to_preallocate_for_aggregation_);
bool isCollectionAndUseEnabled() const;
bool isCollectionAndUseEnabled() const { return key != 0; }
void disable() { key = 0; }
const UInt64 key = 0;
UInt64 key = 0;
const size_t max_entries_for_hash_table_stats = 0;
const size_t max_size_to_preallocate_for_aggregation = 0;
};
@ -1022,12 +1023,17 @@ public:
using AggregateFunctionsPlainPtrs = std::vector<const IAggregateFunction *>;
/// Process one block. Return false if the processing should be aborted (with group_by_overflow_mode = 'break').
bool executeOnBlock(const Block & block, AggregatedDataVariants & result,
ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block
bool executeOnBlock(const Block & block,
AggregatedDataVariants & result,
ColumnRawPtrs & key_columns,
AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block
bool & no_more_keys) const;
bool executeOnBlock(Columns columns, UInt64 num_rows, AggregatedDataVariants & result,
ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block
bool executeOnBlock(Columns columns,
size_t row_begin, size_t row_end,
AggregatedDataVariants & result,
ColumnRawPtrs & key_columns,
AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block
bool & no_more_keys) const;
/// Used for aggregate projection.
@ -1160,12 +1166,33 @@ private:
void destroyAllAggregateStates(AggregatedDataVariants & result) const;
/// Used for optimize_aggregation_in_order:
/// - No two-level aggregation
/// - No external aggregation
/// - No without_key support (it is implemented using executeOnIntervalWithoutKeyImpl())
void executeOnBlockSmall(
AggregatedDataVariants & result,
size_t row_begin,
size_t row_end,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions) const;
void executeImpl(
AggregatedDataVariants & result,
size_t row_begin,
size_t row_end,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions,
bool no_more_keys = false,
AggregateDataPtr overflow_row = nullptr) const;
/// Process one data block, aggregate the data into a hash table.
template <typename Method>
void executeImpl(
Method & method,
Arena * aggregates_pool,
size_t rows,
size_t row_begin,
size_t row_end,
ColumnRawPtrs & key_columns,
AggregateFunctionInstruction * aggregate_instructions,
bool no_more_keys,
@ -1177,7 +1204,8 @@ private:
Method & method,
typename Method::State & state,
Arena * aggregates_pool,
size_t rows,
size_t row_begin,
size_t row_end,
AggregateFunctionInstruction * aggregate_instructions,
AggregateDataPtr overflow_row) const;
@ -1185,7 +1213,8 @@ private:
template <bool use_compiled_functions>
void executeWithoutKeyImpl(
AggregatedDataWithoutKey & res,
size_t rows,
size_t row_begin,
size_t row_end,
AggregateFunctionInstruction * aggregate_instructions,
Arena * arena) const;

View File

@ -3411,6 +3411,7 @@ ReadSettings Context::getReadSettings() const
res.remote_read_min_bytes_for_seek = settings.remote_read_min_bytes_for_seek;
res.local_fs_buffer_size = settings.max_read_buffer_size;
res.remote_fs_buffer_size = settings.max_read_buffer_size;
res.direct_io_threshold = settings.min_bytes_to_use_direct_io;
res.mmap_threshold = settings.min_bytes_to_use_mmap_io;
res.priority = settings.read_priority;

View File

@ -589,7 +589,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
res.add(std::move(column));
}
if (context_->getSettingsRef().flatten_nested)
if (!attach && context_->getSettingsRef().flatten_nested)
res.flattenNested();
if (res.getAllPhysical().empty())

View File

@ -309,11 +309,12 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module, const
auto * places_type = b.getInt8Ty()->getPointerTo()->getPointerTo();
auto * column_data_type = llvm::StructType::get(b.getInt8PtrTy(), b.getInt8PtrTy());
auto * aggregate_loop_func_declaration = llvm::FunctionType::get(b.getVoidTy(), { size_type, column_data_type->getPointerTo(), places_type }, false);
auto * aggregate_loop_func_declaration = llvm::FunctionType::get(b.getVoidTy(), { size_type, size_type, column_data_type->getPointerTo(), places_type }, false);
auto * aggregate_loop_func_definition = llvm::Function::Create(aggregate_loop_func_declaration, llvm::Function::ExternalLinkage, name, module);
auto * arguments = aggregate_loop_func_definition->args().begin();
llvm::Value * rows_count_arg = arguments++;
llvm::Value * row_start_arg = arguments++;
llvm::Value * row_end_arg = arguments++;
llvm::Value * columns_arg = arguments++;
llvm::Value * places_arg = arguments++;
@ -322,6 +323,9 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module, const
auto * entry = llvm::BasicBlock::Create(b.getContext(), "entry", aggregate_loop_func_definition);
b.SetInsertPoint(entry);
llvm::IRBuilder<> entry_builder(entry);
auto * places_start_arg = entry_builder.CreateInBoundsGEP(nullptr, places_arg, row_start_arg);
std::vector<ColumnDataPlaceholder> columns;
size_t previous_columns_size = 0;
@ -338,7 +342,16 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module, const
const auto & argument_type = argument_types[column_argument_index];
auto * data = b.CreateLoad(column_data_type, b.CreateConstInBoundsGEP1_64(column_data_type, columns_arg, previous_columns_size + column_argument_index));
data_placeholder.data_init = b.CreatePointerCast(b.CreateExtractValue(data, {0}), toNativeType(b, removeNullable(argument_type))->getPointerTo());
data_placeholder.null_init = argument_type->isNullable() ? b.CreateExtractValue(data, {1}) : nullptr;
data_placeholder.data_init = entry_builder.CreateInBoundsGEP(nullptr, data_placeholder.data_init, row_start_arg);
if (argument_type->isNullable())
{
data_placeholder.null_init = b.CreateExtractValue(data, {1});
data_placeholder.null_init = entry_builder.CreateInBoundsGEP(nullptr, data_placeholder.null_init, row_start_arg);
}
else
{
data_placeholder.null_init = nullptr;
}
columns.emplace_back(data_placeholder);
}
@ -350,15 +363,15 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module, const
auto * end = llvm::BasicBlock::Create(b.getContext(), "end", aggregate_loop_func_definition);
auto * loop = llvm::BasicBlock::Create(b.getContext(), "loop", aggregate_loop_func_definition);
b.CreateCondBr(b.CreateICmpEQ(rows_count_arg, llvm::ConstantInt::get(size_type, 0)), end, loop);
b.CreateCondBr(b.CreateICmpEQ(row_start_arg, row_end_arg), end, loop);
b.SetInsertPoint(loop);
auto * counter_phi = b.CreatePHI(rows_count_arg->getType(), 2);
counter_phi->addIncoming(llvm::ConstantInt::get(size_type, 0), entry);
auto * counter_phi = b.CreatePHI(row_start_arg->getType(), 2);
counter_phi->addIncoming(row_start_arg, entry);
auto * places_phi = b.CreatePHI(places_arg->getType(), 2);
places_phi->addIncoming(places_arg, entry);
auto * places_phi = b.CreatePHI(places_start_arg->getType(), 2);
places_phi->addIncoming(places_start_arg, entry);
for (auto & col : columns)
{
@ -428,7 +441,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module, const
auto * value = b.CreateAdd(counter_phi, llvm::ConstantInt::get(size_type, 1));
counter_phi->addIncoming(value, cur_block);
b.CreateCondBr(b.CreateICmpEQ(value, rows_count_arg), end, loop);
b.CreateCondBr(b.CreateICmpEQ(value, row_end_arg), end, loop);
b.SetInsertPoint(end);
b.CreateRetVoid();
@ -443,11 +456,12 @@ static void compileAddIntoAggregateStatesFunctionsSinglePlace(llvm::Module & mod
auto * places_type = b.getInt8Ty()->getPointerTo();
auto * column_data_type = llvm::StructType::get(b.getInt8PtrTy(), b.getInt8PtrTy());
auto * aggregate_loop_func_declaration = llvm::FunctionType::get(b.getVoidTy(), { size_type, column_data_type->getPointerTo(), places_type }, false);
auto * aggregate_loop_func_declaration = llvm::FunctionType::get(b.getVoidTy(), { size_type, size_type, column_data_type->getPointerTo(), places_type }, false);
auto * aggregate_loop_func_definition = llvm::Function::Create(aggregate_loop_func_declaration, llvm::Function::ExternalLinkage, name, module);
auto * arguments = aggregate_loop_func_definition->args().begin();
llvm::Value * rows_count_arg = arguments++;
llvm::Value * row_start_arg = arguments++;
llvm::Value * row_end_arg = arguments++;
llvm::Value * columns_arg = arguments++;
llvm::Value * place_arg = arguments++;
@ -456,6 +470,8 @@ static void compileAddIntoAggregateStatesFunctionsSinglePlace(llvm::Module & mod
auto * entry = llvm::BasicBlock::Create(b.getContext(), "entry", aggregate_loop_func_definition);
b.SetInsertPoint(entry);
llvm::IRBuilder<> entry_builder(entry);
std::vector<ColumnDataPlaceholder> columns;
size_t previous_columns_size = 0;
@ -472,7 +488,16 @@ static void compileAddIntoAggregateStatesFunctionsSinglePlace(llvm::Module & mod
const auto & argument_type = argument_types[column_argument_index];
auto * data = b.CreateLoad(column_data_type, b.CreateConstInBoundsGEP1_64(column_data_type, columns_arg, previous_columns_size + column_argument_index));
data_placeholder.data_init = b.CreatePointerCast(b.CreateExtractValue(data, {0}), toNativeType(b, removeNullable(argument_type))->getPointerTo());
data_placeholder.null_init = argument_type->isNullable() ? b.CreateExtractValue(data, {1}) : nullptr;
data_placeholder.data_init = entry_builder.CreateInBoundsGEP(nullptr, data_placeholder.data_init, row_start_arg);
if (argument_type->isNullable())
{
data_placeholder.null_init = b.CreateExtractValue(data, {1});
data_placeholder.null_init = entry_builder.CreateInBoundsGEP(nullptr, data_placeholder.null_init, row_start_arg);
}
else
{
data_placeholder.null_init = nullptr;
}
columns.emplace_back(data_placeholder);
}
@ -484,12 +509,12 @@ static void compileAddIntoAggregateStatesFunctionsSinglePlace(llvm::Module & mod
auto * end = llvm::BasicBlock::Create(b.getContext(), "end", aggregate_loop_func_definition);
auto * loop = llvm::BasicBlock::Create(b.getContext(), "loop", aggregate_loop_func_definition);
b.CreateCondBr(b.CreateICmpEQ(rows_count_arg, llvm::ConstantInt::get(size_type, 0)), end, loop);
b.CreateCondBr(b.CreateICmpEQ(row_start_arg, row_end_arg), end, loop);
b.SetInsertPoint(loop);
auto * counter_phi = b.CreatePHI(rows_count_arg->getType(), 2);
counter_phi->addIncoming(llvm::ConstantInt::get(size_type, 0), entry);
auto * counter_phi = b.CreatePHI(row_start_arg->getType(), 2);
counter_phi->addIncoming(row_start_arg, entry);
for (auto & col : columns)
{
@ -555,7 +580,7 @@ static void compileAddIntoAggregateStatesFunctionsSinglePlace(llvm::Module & mod
auto * value = b.CreateAdd(counter_phi, llvm::ConstantInt::get(size_type, 1));
counter_phi->addIncoming(value, cur_block);
b.CreateCondBr(b.CreateICmpEQ(value, rows_count_arg), end, loop);
b.CreateCondBr(b.CreateICmpEQ(value, row_end_arg), end, loop);
b.SetInsertPoint(end);
b.CreateRetVoid();
@ -600,35 +625,47 @@ static void compileInsertAggregatesIntoResultColumns(llvm::Module & module, cons
auto * column_data_type = llvm::StructType::get(b.getInt8PtrTy(), b.getInt8PtrTy());
auto * aggregate_data_places_type = b.getInt8Ty()->getPointerTo()->getPointerTo();
auto * aggregate_loop_func_declaration = llvm::FunctionType::get(b.getVoidTy(), { size_type, column_data_type->getPointerTo(), aggregate_data_places_type }, false);
auto * aggregate_loop_func_declaration = llvm::FunctionType::get(b.getVoidTy(), { size_type, size_type, column_data_type->getPointerTo(), aggregate_data_places_type }, false);
auto * aggregate_loop_func = llvm::Function::Create(aggregate_loop_func_declaration, llvm::Function::ExternalLinkage, name, module);
auto * arguments = aggregate_loop_func->args().begin();
llvm::Value * rows_count_arg = &*arguments++;
llvm::Value * row_start_arg = &*arguments++;
llvm::Value * row_end_arg = &*arguments++;
llvm::Value * columns_arg = &*arguments++;
llvm::Value * aggregate_data_places_arg = &*arguments++;
auto * entry = llvm::BasicBlock::Create(b.getContext(), "entry", aggregate_loop_func);
b.SetInsertPoint(entry);
llvm::IRBuilder<> entry_builder(entry);
std::vector<ColumnDataPlaceholder> columns(functions.size());
for (size_t i = 0; i < functions.size(); ++i)
{
auto return_type = functions[i].function->getReturnType();
auto * data = b.CreateLoad(column_data_type, b.CreateConstInBoundsGEP1_64(column_data_type, columns_arg, i));
columns[i].data_init = b.CreatePointerCast(b.CreateExtractValue(data, {0}), toNativeType(b, removeNullable(return_type))->getPointerTo());
columns[i].null_init = return_type->isNullable() ? b.CreateExtractValue(data, {1}) : nullptr;
columns[i].data_init = entry_builder.CreateInBoundsGEP(nullptr, columns[i].data_init, row_start_arg);
if (return_type->isNullable())
{
columns[i].null_init = b.CreateExtractValue(data, {1});
columns[i].null_init = entry_builder.CreateInBoundsGEP(nullptr, columns[i].null_init, row_start_arg);
}
else
{
columns[i].null_init = nullptr;
}
}
auto * end = llvm::BasicBlock::Create(b.getContext(), "end", aggregate_loop_func);
auto * loop = llvm::BasicBlock::Create(b.getContext(), "loop", aggregate_loop_func);
b.CreateCondBr(b.CreateICmpEQ(rows_count_arg, llvm::ConstantInt::get(size_type, 0)), end, loop);
b.CreateCondBr(b.CreateICmpEQ(row_start_arg, row_end_arg), end, loop);
b.SetInsertPoint(loop);
auto * counter_phi = b.CreatePHI(rows_count_arg->getType(), 2);
counter_phi->addIncoming(llvm::ConstantInt::get(size_type, 0), entry);
auto * counter_phi = b.CreatePHI(row_start_arg->getType(), 2);
counter_phi->addIncoming(row_start_arg, entry);
auto * aggregate_data_place_phi = b.CreatePHI(aggregate_data_places_type, 2);
aggregate_data_place_phi->addIncoming(aggregate_data_places_arg, entry);
@ -682,7 +719,7 @@ static void compileInsertAggregatesIntoResultColumns(llvm::Module & module, cons
aggregate_data_place_phi->addIncoming(b.CreateConstInBoundsGEP1_64(nullptr, aggregate_data_place_phi, 1), cur_block);
b.CreateCondBr(b.CreateICmpEQ(value, rows_count_arg), end, loop);
b.CreateCondBr(b.CreateICmpEQ(value, row_end_arg), end, loop);
b.SetInsertPoint(end);
b.CreateRetVoid();

View File

@ -26,6 +26,7 @@ struct ColumnData
*/
ColumnData getColumnData(const IColumn * column);
using ColumnDataRowsOffset = size_t;
using ColumnDataRowsSize = size_t;
using JITCompiledFunction = void (*)(ColumnDataRowsSize, ColumnData *);
@ -51,10 +52,10 @@ struct AggregateFunctionWithOffset
};
using JITCreateAggregateStatesFunction = void (*)(AggregateDataPtr);
using JITAddIntoAggregateStatesFunction = void (*)(ColumnDataRowsSize, ColumnData *, AggregateDataPtr *);
using JITAddIntoAggregateStatesFunctionSinglePlace = void (*)(ColumnDataRowsSize, ColumnData *, AggregateDataPtr);
using JITAddIntoAggregateStatesFunction = void (*)(ColumnDataRowsOffset, ColumnDataRowsOffset, ColumnData *, AggregateDataPtr *);
using JITAddIntoAggregateStatesFunctionSinglePlace = void (*)(ColumnDataRowsOffset, ColumnDataRowsOffset, ColumnData *, AggregateDataPtr);
using JITMergeAggregateStatesFunction = void (*)(AggregateDataPtr, AggregateDataPtr);
using JITInsertAggregateStatesIntoColumnsFunction = void (*)(ColumnDataRowsSize, ColumnData *, AggregateDataPtr *);
using JITInsertAggregateStatesIntoColumnsFunction = void (*)(ColumnDataRowsOffset, ColumnDataRowsOffset, ColumnData *, AggregateDataPtr *);
struct CompiledAggregateFunctions
{

View File

@ -48,7 +48,7 @@ void UserDefinedSQLFunctionFactory::registerFunction(ContextPtr context, const S
if (if_not_exists)
return;
throw Exception(ErrorCodes::CANNOT_DROP_FUNCTION, "User defined executable function '{}'", function_name);
throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "User defined executable function '{}' already exists", function_name);
}
std::lock_guard lock(mutex);

View File

@ -13,6 +13,7 @@
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ExpressionElementParsers.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/typeid_cast.h>
@ -37,7 +38,19 @@ std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(co
return std::make_pair(literal->value, applyVisitor(FieldToDataType(), literal->value));
NamesAndTypesList source_columns = {{ "_dummy", std::make_shared<DataTypeUInt8>() }};
auto ast = node->clone();
if (ast->as<ASTSubquery>() != nullptr)
{
/** For subqueries getColumnName if there are no alias will return __subquery_ + 'hash'.
* If there is alias getColumnName for subquery will return alias.
* In result block name of subquery after QueryAliasesVisitor pass will be _subquery1.
* We specify alias for subquery, because we need to get column from result block.
*/
ast->setAlias("constant_expression");
}
ReplaceQueryParameterVisitor param_visitor(context->getQueryParameters());
param_visitor.visit(ast);
@ -46,6 +59,12 @@ std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(co
String name = ast->getColumnName();
auto syntax_result = TreeRewriter(context).analyze(ast, source_columns);
/// AST potentially could be transformed to literal during TreeRewriter analyze.
/// For example if we have SQL user defined function that return literal AS subquery.
if (ASTLiteral * literal = ast->as<ASTLiteral>())
return std::make_pair(literal->value, applyVisitor(FieldToDataType(), literal->value));
ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(ast, syntax_result, context).getConstActions();
/// There must be at least one column in the block so that it knows the number of rows.

View File

@ -298,7 +298,7 @@ void registerInputFormatCSV(FormatFactory & factory)
registerWithNamesAndTypes("CSV", register_func);
}
static std::pair<bool, size_t> fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, size_t min_rows)
std::pair<bool, size_t> fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, size_t min_rows)
{
char * pos = in.position();
bool quotes = false;

View File

@ -82,4 +82,6 @@ private:
CSVFormatReader reader;
};
std::pair<bool, size_t> fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, size_t min_rows);
}

View File

@ -10,16 +10,20 @@ namespace ErrorCodes
}
static FormatSettings updateFormatSettings(const FormatSettings & settings)
static FormatSettings updateFormatSettings(const FormatSettings & settings, const Block & header)
{
FormatSettings updated = settings;
updated.skip_unknown_fields = true;
updated.date_time_input_format = FormatSettings::DateTimeInputFormat::BestEffort;
updated.csv.delimiter = updated.hive_text.fields_delimiter;
if (settings.hive_text.input_field_names.empty())
updated.hive_text.input_field_names = header.getNames();
return updated;
}
HiveTextRowInputFormat::HiveTextRowInputFormat(
const Block & header_, ReadBuffer & in_, const Params & params_, const FormatSettings & format_settings_)
: HiveTextRowInputFormat(header_, std::make_unique<PeekableReadBuffer>(in_), params_, updateFormatSettings(format_settings_))
: HiveTextRowInputFormat(header_, std::make_unique<PeekableReadBuffer>(in_), params_, updateFormatSettings(format_settings_, header_))
{
}
@ -56,5 +60,15 @@ void registerInputFormatHiveText(FormatFactory & factory)
return std::make_shared<HiveTextRowInputFormat>(sample, buf, params, settings);
});
}
void registerFileSegmentationEngineHiveText(FormatFactory & factory)
{
factory.registerFileSegmentationEngine(
"HiveText",
[](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) -> std::pair<bool, size_t> {
return fileSegmentationEngineCSVImpl(in, memory, min_chunk_size, 0);
});
}
}
#endif

View File

@ -11,7 +11,6 @@ namespace DB
{
/// A stream for input data in Hive Text format.
/// Parallel parsing is disabled currently.
class HiveTextRowInputFormat final : public CSVRowInputFormat
{
public:

View File

@ -10,7 +10,12 @@
namespace DB
{
ProtobufListInputFormat::ProtobufListInputFormat(ReadBuffer & in_, const Block & header_, const Params & params_, const FormatSchemaInfo & schema_info_)
ProtobufListInputFormat::ProtobufListInputFormat(
ReadBuffer & in_,
const Block & header_,
const Params & params_,
const FormatSchemaInfo & schema_info_,
bool flatten_google_wrappers_)
: IRowInputFormat(header_, in_, params_)
, reader(std::make_unique<ProtobufReader>(in_))
, serializer(ProtobufSerializer::create(
@ -20,6 +25,7 @@ ProtobufListInputFormat::ProtobufListInputFormat(ReadBuffer & in_, const Block &
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::Yes),
/* with_length_delimiter = */ true,
/* with_envelope = */ true,
flatten_google_wrappers_,
*reader))
{
}
@ -70,7 +76,8 @@ void registerInputFormatProtobufList(FormatFactory & factory)
RowInputFormatParams params,
const FormatSettings & settings)
{
return std::make_shared<ProtobufListInputFormat>(buf, sample, std::move(params), FormatSchemaInfo(settings, "Protobuf", true));
return std::make_shared<ProtobufListInputFormat>(buf, sample, std::move(params),
FormatSchemaInfo(settings, "Protobuf", true), settings.protobuf.input_flatten_google_wrappers);
});
factory.markFormatAsColumnOriented("ProtobufList");
}

View File

@ -24,7 +24,12 @@ class ReadBuffer;
class ProtobufListInputFormat final : public IRowInputFormat
{
public:
ProtobufListInputFormat(ReadBuffer & in_, const Block & header_, const Params & params_, const FormatSchemaInfo & schema_info_);
ProtobufListInputFormat(
ReadBuffer & in_,
const Block & header_,
const Params & params_,
const FormatSchemaInfo & schema_info_,
bool flatten_google_wrappers_);
String getName() const override { return "ProtobufListInputFormat"; }

View File

@ -14,7 +14,8 @@ ProtobufListOutputFormat::ProtobufListOutputFormat(
WriteBuffer & out_,
const Block & header_,
const RowOutputFormatParams & params_,
const FormatSchemaInfo & schema_info_)
const FormatSchemaInfo & schema_info_,
bool defaults_for_nullable_google_wrappers_)
: IRowOutputFormat(header_, out_, params_)
, writer(std::make_unique<ProtobufWriter>(out))
, serializer(ProtobufSerializer::create(
@ -23,6 +24,7 @@ ProtobufListOutputFormat::ProtobufListOutputFormat(
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::Yes),
/* with_length_delimiter = */ true,
/* with_envelope = */ true,
defaults_for_nullable_google_wrappers_,
*writer))
{
}
@ -51,7 +53,8 @@ void registerOutputFormatProtobufList(FormatFactory & factory)
{
return std::make_shared<ProtobufListOutputFormat>(
buf, header, params,
FormatSchemaInfo(settings, "Protobuf", true));
FormatSchemaInfo(settings, "Protobuf", true),
settings.protobuf.output_nullables_with_google_wrappers);
});
}

View File

@ -27,7 +27,8 @@ public:
WriteBuffer & out_,
const Block & header_,
const RowOutputFormatParams & params_,
const FormatSchemaInfo & schema_info_);
const FormatSchemaInfo & schema_info_,
bool defaults_for_nullable_google_wrappers_);
String getName() const override { return "ProtobufListOutputFormat"; }

View File

@ -10,7 +10,8 @@
namespace DB
{
ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, const Params & params_, const FormatSchemaInfo & schema_info_, bool with_length_delimiter_)
ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, const Params & params_,
const FormatSchemaInfo & schema_info_, bool with_length_delimiter_, bool flatten_google_wrappers_)
: IRowInputFormat(header_, in_, params_)
, reader(std::make_unique<ProtobufReader>(in_))
, serializer(ProtobufSerializer::create(
@ -20,6 +21,7 @@ ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & h
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::No),
with_length_delimiter_,
/* with_envelope = */ false,
flatten_google_wrappers_,
*reader))
{
}
@ -64,7 +66,8 @@ void registerInputFormatProtobuf(FormatFactory & factory)
{
return std::make_shared<ProtobufRowInputFormat>(buf, sample, std::move(params),
FormatSchemaInfo(settings, "Protobuf", true),
with_length_delimiter);
with_length_delimiter,
settings.protobuf.input_flatten_google_wrappers);
});
}
}

View File

@ -28,7 +28,13 @@ class ReadBuffer;
class ProtobufRowInputFormat final : public IRowInputFormat
{
public:
ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, const Params & params_, const FormatSchemaInfo & schema_info_, bool with_length_delimiter_);
ProtobufRowInputFormat(
ReadBuffer & in_,
const Block & header_,
const Params & params_,
const FormatSchemaInfo & schema_info_,
bool with_length_delimiter_,
bool flatten_google_wrappers_);
String getName() const override { return "ProtobufRowInputFormat"; }

View File

@ -32,6 +32,7 @@ ProtobufRowOutputFormat::ProtobufRowOutputFormat(
*ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::No),
with_length_delimiter_,
/* with_envelope = */ false,
settings_.protobuf.output_nullables_with_google_wrappers,
*writer))
, allow_multiple_rows(with_length_delimiter_ || settings_.protobuf.allow_multiple_rows_without_delimiter)
{

View File

@ -57,6 +57,17 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
pipeline.dropTotalsAndExtremes();
bool allow_to_use_two_level_group_by = pipeline.getNumStreams() > 1 || params.max_bytes_before_external_group_by != 0;
/// optimize_aggregation_in_order
if (group_by_info)
{
/// two-level aggregation is not supported anyway for in order aggregation.
allow_to_use_two_level_group_by = false;
/// It is incorrect for in order aggregation.
params.stats_collecting_params.disable();
}
if (!allow_to_use_two_level_group_by)
{
params.group_by_two_level_threshold = 0;
@ -70,14 +81,6 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
auto transform_params = std::make_shared<AggregatingTransformParams>(std::move(params), final);
if (group_by_info)
{
bool need_finish_sorting = (group_by_info->order_key_prefix_descr.size() < group_by_sort_description.size());
if (need_finish_sorting)
{
/// TOO SLOW
}
else
{
if (pipeline.getNumStreams() > 1)
{
@ -96,7 +99,8 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
/// not greater than 'aggregation_in_order_max_block_bytes'.
/// So, we reduce 'max_bytes' value for aggregation in 'merge_threads' times.
return std::make_shared<AggregatingInOrderTransform>(
header, transform_params, group_by_sort_description,
header, transform_params,
group_by_info, group_by_sort_description,
max_block_size, aggregation_in_order_max_block_bytes / merge_threads,
many_data, counter++);
});
@ -128,7 +132,8 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
pipeline.addSimpleTransform([&](const Block & header)
{
return std::make_shared<AggregatingInOrderTransform>(
header, transform_params, group_by_sort_description,
header, transform_params,
group_by_info, group_by_sort_description,
max_block_size, aggregation_in_order_max_block_bytes);
});
@ -143,7 +148,6 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
finalizing = collector.detachProcessors(2);
return;
}
}
/// If there are several sources, then we perform parallel aggregation
if (pipeline.getNumStreams() > 1)

View File

@ -159,8 +159,9 @@ void TTLAggregationAlgorithm::calculateAggregates(const MutableColumns & aggrega
aggregate_chunk.emplace_back(std::move(chunk_column));
}
aggregator->executeOnBlock(aggregate_chunk, length, aggregation_result, key_columns,
columns_for_aggregator, no_more_keys);
aggregator->executeOnBlock(
aggregate_chunk, /* row_begin= */ 0, length,
aggregation_result, key_columns, columns_for_aggregator, no_more_keys);
}

View File

@ -1,6 +1,8 @@
#include <Processors/Transforms/AggregatingInOrderTransform.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Storages/SelectQueryInfo.h>
#include <Core/SortCursor.h>
#include <Interpreters/sortBlock.h>
#include <base/range.h>
namespace DB
@ -9,16 +11,19 @@ namespace DB
AggregatingInOrderTransform::AggregatingInOrderTransform(
Block header,
AggregatingTransformParamsPtr params_,
InputOrderInfoPtr group_by_info_,
const SortDescription & group_by_description_,
size_t max_block_size_, size_t max_block_bytes_)
: AggregatingInOrderTransform(std::move(header), std::move(params_),
group_by_description_, max_block_size_, max_block_bytes_,
group_by_info_, group_by_description_,
max_block_size_, max_block_bytes_,
std::make_unique<ManyAggregatedData>(1), 0)
{
}
AggregatingInOrderTransform::AggregatingInOrderTransform(
Block header, AggregatingTransformParamsPtr params_,
InputOrderInfoPtr group_by_info_,
const SortDescription & group_by_description_,
size_t max_block_size_, size_t max_block_bytes_,
ManyAggregatedDataPtr many_data_, size_t current_variant)
@ -26,6 +31,8 @@ AggregatingInOrderTransform::AggregatingInOrderTransform(
, max_block_size(max_block_size_)
, max_block_bytes(max_block_bytes_)
, params(std::move(params_))
, group_by_info(group_by_info_)
, sort_description(group_by_description_)
, aggregate_columns(params->params.aggregates_size)
, many_data(std::move(many_data_))
, variants(*many_data->variants[current_variant])
@ -33,8 +40,18 @@ AggregatingInOrderTransform::AggregatingInOrderTransform(
/// We won't finalize states in order to merge same states (generated due to multi-thread execution) in AggregatingSortedTransform
res_header = params->getCustomHeader(false);
for (const auto & column_description : group_by_description_)
for (size_t i = 0; i < group_by_info->order_key_prefix_descr.size(); ++i)
{
const auto & column_description = group_by_description_[i];
group_by_description.emplace_back(column_description, res_header.getPositionByName(column_description.column_name));
}
if (group_by_info->order_key_prefix_descr.size() < group_by_description_.size())
{
group_by_key = true;
/// group_by_description may contains duplicates, so we use keys_size from Aggregator::params
key_columns_raw.resize(params->params.keys_size);
}
}
AggregatingInOrderTransform::~AggregatingInOrderTransform() = default;
@ -70,6 +87,8 @@ void AggregatingInOrderTransform::consume(Chunk chunk)
{
materialized_columns.push_back(chunk.getColumns().at(params->params.keys[i])->convertToFullColumnIfConst());
key_columns[i] = materialized_columns.back();
if (group_by_key)
key_columns_raw[i] = materialized_columns.back().get();
}
Aggregator::NestedColumnsHolder nested_columns_holder;
@ -83,16 +102,19 @@ void AggregatingInOrderTransform::consume(Chunk chunk)
if (!cur_block_size)
{
res_key_columns.resize(params->params.keys_size);
res_aggregate_columns.resize(params->params.aggregates_size);
for (size_t i = 0; i < params->params.keys_size; ++i)
res_key_columns[i] = res_header.safeGetByPosition(i).type->createColumn();
params->aggregator.createStatesAndFillKeyColumnsWithSingleKey(variants, key_columns, key_begin, res_key_columns);
if (!group_by_key)
{
res_aggregate_columns.resize(params->params.aggregates_size);
for (size_t i = 0; i < params->params.aggregates_size; ++i)
res_aggregate_columns[i] = res_header.safeGetByPosition(i + params->params.keys_size).type->createColumn();
params->aggregator.createStatesAndFillKeyColumnsWithSingleKey(variants, key_columns, key_begin, res_key_columns);
params->aggregator.addArenasToAggregateColumns(variants, res_aggregate_columns);
}
++cur_block_size;
}
@ -113,18 +135,26 @@ void AggregatingInOrderTransform::consume(Chunk chunk)
/// Add data to aggr. state if interval is not empty. Empty when haven't found current key in new block.
if (key_begin != key_end)
{
if (group_by_key)
params->aggregator.executeOnBlockSmall(variants, key_begin, key_end, key_columns_raw, aggregate_function_instructions.data());
else
params->aggregator.executeOnIntervalWithoutKeyImpl(variants, key_begin, key_end, aggregate_function_instructions.data(), variants.aggregates_pool);
}
current_memory_usage = getCurrentMemoryUsage() - initial_memory_usage;
/// We finalize last key aggregation state if a new key found.
if (key_end != rows)
{
if (!group_by_key)
params->aggregator.addSingleKeyToAggregateColumns(variants, res_aggregate_columns);
/// If max_block_size is reached we have to stop consuming and generate the block. Save the extra rows into new chunk.
if (cur_block_size >= max_block_size || cur_block_bytes + current_memory_usage >= max_block_bytes)
{
if (group_by_key)
group_by_block = params->aggregator.prepareBlockAndFillSingleLevel(variants, /* final= */ false);
cur_block_bytes += current_memory_usage;
finalizeCurrentChunk(std::move(chunk), key_end);
return;
@ -155,7 +185,7 @@ void AggregatingInOrderTransform::finalizeCurrentChunk(Chunk chunk, size_t key_e
block_end_reached = true;
need_generate = true;
variants.without_key = nullptr;
variants.invalidate();
}
void AggregatingInOrderTransform::work()
@ -234,10 +264,16 @@ void AggregatingInOrderTransform::generate()
{
if (cur_block_size && is_consume_finished)
{
if (group_by_key)
group_by_block = params->aggregator.prepareBlockAndFillSingleLevel(variants, /* final= */ false);
else
params->aggregator.addSingleKeyToAggregateColumns(variants, res_aggregate_columns);
variants.without_key = nullptr;
variants.invalidate();
}
bool group_by_key_needs_empty_block = is_consume_finished && !cur_block_size;
if (!group_by_key || group_by_key_needs_empty_block)
{
Block res = res_header.cloneEmpty();
for (size_t i = 0; i < res_key_columns.size(); ++i)
@ -247,6 +283,15 @@ void AggregatingInOrderTransform::generate()
res.getByPosition(i + res_key_columns.size()).column = std::move(res_aggregate_columns[i]);
to_push_chunk = convertToChunk(res);
}
else
{
/// Sorting is required after aggregation, for proper merging, via
/// FinishAggregatingInOrderTransform/MergingAggregatedBucketTransform
sortBlock(group_by_block, sort_description);
to_push_chunk = convertToChunk(group_by_block);
}
if (!to_push_chunk.getNumRows())
return;

View File

@ -9,6 +9,9 @@
namespace DB
{
struct InputOrderInfo;
using InputOrderInfoPtr = std::shared_ptr<const InputOrderInfo>;
struct ChunkInfoWithAllocatedBytes : public ChunkInfo
{
explicit ChunkInfoWithAllocatedBytes(Int64 allocated_bytes_)
@ -20,12 +23,14 @@ class AggregatingInOrderTransform : public IProcessor
{
public:
AggregatingInOrderTransform(Block header, AggregatingTransformParamsPtr params,
const SortDescription & group_by_description,
InputOrderInfoPtr group_by_info_,
const SortDescription & group_by_description_,
size_t max_block_size_, size_t max_block_bytes_,
ManyAggregatedDataPtr many_data, size_t current_variant);
AggregatingInOrderTransform(Block header, AggregatingTransformParamsPtr params,
const SortDescription & group_by_description,
InputOrderInfoPtr group_by_info_,
const SortDescription & group_by_description_,
size_t max_block_size_, size_t max_block_bytes_);
~AggregatingInOrderTransform() override;
@ -51,7 +56,14 @@ private:
MutableColumns res_aggregate_columns;
AggregatingTransformParamsPtr params;
InputOrderInfoPtr group_by_info;
/// For sortBlock()
SortDescription sort_description;
SortDescriptionWithPositions group_by_description;
bool group_by_key = false;
Block group_by_block;
ColumnRawPtrs key_columns_raw;
Aggregator::AggregateColumns aggregate_columns;

View File

@ -533,7 +533,7 @@ void AggregatingTransform::consume(Chunk chunk)
}
else
{
if (!params->aggregator.executeOnBlock(chunk.detachColumns(), num_rows, variants, key_columns, aggregate_columns, no_more_keys))
if (!params->aggregator.executeOnBlock(chunk.detachColumns(), 0, num_rows, variants, key_columns, aggregate_columns, no_more_keys))
is_consume_finished = true;
}
}

View File

@ -64,7 +64,7 @@ void CheckConstraintsTransform::onConsume(Chunk chunk)
/// Check if constraint value is nullable
const auto & null_map = column_nullable->getNullMapColumn();
const PaddedPODArray<UInt8> & null_map_data = null_map.getData();
bool null_map_contains_null = !memoryIsZero(null_map_data.raw_data(), null_map_data.size() * sizeof(UInt8));
bool null_map_contains_null = !memoryIsZero(null_map_data.raw_data(), 0, null_map_data.size() * sizeof(UInt8));
if (null_map_contains_null)
throw Exception(
@ -84,7 +84,7 @@ void CheckConstraintsTransform::onConsume(Chunk chunk)
size_t size = res_column_uint8.size();
/// Is violated.
if (!memoryIsByte(res_data, size, 1))
if (!memoryIsByte(res_data, 0, size, 1))
{
size_t row_idx = 0;
for (; row_idx < size; ++row_idx)

View File

@ -114,6 +114,8 @@ bool RemoteQueryExecutorReadContext::checkTimeout(bool blocking)
{
if (last_used_socket)
e.addMessage(" while reading from socket ({})", last_used_socket->peerAddress().toString());
if (e.code() == ErrorCodes::SOCKET_TIMEOUT)
e.addMessage(" (receive timeout {} ms)", receive_timeout_usec / 1000);
throw;
}
}

View File

@ -532,6 +532,7 @@ void TCPHandler::extractConnectionSettingsFromContext(const ContextPtr & context
sleep_in_send_tables_status = settings.sleep_in_send_tables_status_ms;
unknown_packet_in_send_data = settings.unknown_packet_in_send_data;
sleep_in_receive_cancel = settings.sleep_in_receive_cancel_ms;
sleep_after_receiving_query = settings.sleep_after_receiving_query_ms;
}
@ -766,7 +767,7 @@ void TCPHandler::processTablesStatusRequest()
writeVarUInt(Protocol::Server::TablesStatusResponse, *out);
/// For testing hedged requests
if (sleep_in_send_tables_status.totalMilliseconds())
if (unlikely(sleep_in_send_tables_status.totalMilliseconds()))
{
out->next();
std::chrono::milliseconds ms(sleep_in_send_tables_status.totalMilliseconds());
@ -1104,7 +1105,7 @@ bool TCPHandler::receivePacket()
case Protocol::Client::Cancel:
{
/// For testing connection collector.
if (sleep_in_receive_cancel.totalMilliseconds())
if (unlikely(sleep_in_receive_cancel.totalMilliseconds()))
{
std::chrono::milliseconds ms(sleep_in_receive_cancel.totalMilliseconds());
std::this_thread::sleep_for(ms);
@ -1153,7 +1154,7 @@ String TCPHandler::receiveReadTaskResponseAssumeLocked()
{
state.is_cancelled = true;
/// For testing connection collector.
if (sleep_in_receive_cancel.totalMilliseconds())
if (unlikely(sleep_in_receive_cancel.totalMilliseconds()))
{
std::chrono::milliseconds ms(sleep_in_receive_cancel.totalMilliseconds());
std::this_thread::sleep_for(ms);
@ -1186,7 +1187,7 @@ std::optional<PartitionReadResponse> TCPHandler::receivePartitionMergeTreeReadTa
{
state.is_cancelled = true;
/// For testing connection collector.
if (sleep_in_receive_cancel.totalMilliseconds())
if (unlikely(sleep_in_receive_cancel.totalMilliseconds()))
{
std::chrono::milliseconds ms(sleep_in_receive_cancel.totalMilliseconds());
std::this_thread::sleep_for(ms);
@ -1364,6 +1365,13 @@ void TCPHandler::receiveQuery()
{
query_context->setSetting("normalize_function_names", false);
}
/// For testing hedged requests
if (unlikely(sleep_after_receiving_query.totalMilliseconds()))
{
std::chrono::milliseconds ms(sleep_after_receiving_query.totalMilliseconds());
std::this_thread::sleep_for(ms);
}
}
void TCPHandler::receiveUnexpectedQuery()
@ -1601,7 +1609,7 @@ bool TCPHandler::isQueryCancelled()
state.is_cancelled = true;
/// For testing connection collector.
{
if (sleep_in_receive_cancel.totalMilliseconds())
if (unlikely(sleep_in_receive_cancel.totalMilliseconds()))
{
std::chrono::milliseconds ms(sleep_in_receive_cancel.totalMilliseconds());
std::this_thread::sleep_for(ms);

View File

@ -163,6 +163,7 @@ private:
Poco::Timespan sleep_in_send_tables_status;
UInt64 unknown_packet_in_send_data = 0;
Poco::Timespan sleep_in_receive_cancel;
Poco::Timespan sleep_after_receiving_query;
std::unique_ptr<Session> session;
ContextMutablePtr query_context;

View File

@ -94,7 +94,7 @@ bool RemoteReadBuffer::nextImpl()
return status;
}
//file_buffer::pos should increase correspondingly when RemoteReadBuffer is consumed, otherwise start_offset will be incorrect.
/// file_buffer::pos should increase correspondingly when RemoteReadBuffer is consumed, otherwise start_offset will be incorrect.
local_file_holder->file_buffer->position() = local_file_holder->file_buffer->buffer().begin() + BufferBase::offset();
auto start_offset = local_file_holder->file_buffer->getPosition();
auto end_offset = start_offset + local_file_holder->file_buffer->internalBuffer().size();

View File

@ -16,7 +16,6 @@
namespace DB
{
/** Accepts HDFS path to file and opens it.
* Closes file by himself (thus "owns" a file descriptor).
*/

View File

@ -155,8 +155,8 @@ public:
if (current_idx >= source_info->hive_files.size())
return {};
const auto & curr_file = source_info->hive_files[current_idx];
current_path = curr_file->getPath();
const auto & current_file = source_info->hive_files[current_idx];
current_path = current_file->getPath();
String uri_with_path = hdfs_namenode_url + current_path;
auto compression = chooseCompressionMethod(current_path, compression_method);
@ -186,7 +186,7 @@ public:
remote_read_buf = RemoteReadBuffer::create(
getContext(),
std::make_shared<StorageHiveMetadata>(
"Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getSize(), curr_file->getLastModTs()),
"Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, current_file->getSize(), current_file->getLastModTs()),
std::move(raw_read_buf),
buff_size,
format == "Parquet" || format == "ORC");
@ -194,13 +194,13 @@ public:
else
remote_read_buf = std::move(raw_read_buf);
if (curr_file->getFormat() == FileFormat::TEXT)
if (current_file->getFormat() == FileFormat::TEXT)
read_buf = wrapReadBufferWithCompressionMethod(std::move(remote_read_buf), compression);
else
read_buf = std::move(remote_read_buf);
auto input_format = FormatFactory::instance().getInputFormat(
format, *read_buf, to_read_block, getContext(), max_block_size, updateFormatSettings(curr_file));
format, *read_buf, to_read_block, getContext(), max_block_size, updateFormatSettings(current_file));
QueryPipelineBuilder builder;
builder.init(Pipe(input_format));

View File

@ -128,7 +128,7 @@ private:
};
/**
* Executor for a background MergeTree related operations such as merges, mutations, fetches an so on.
* Executor for a background MergeTree related operations such as merges, mutations, fetches and so on.
* It can execute only successors of ExecutableTask interface.
* Which is a self-written coroutine. It suspends, when returns true from executeStep() method.
*

View File

@ -54,9 +54,20 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor(
{
header_without_virtual_columns = getPort().getHeader();
/// Reverse order is to minimize reallocations when removing columns from the block
for (auto it = virt_column_names.rbegin(); it != virt_column_names.rend(); ++it)
{
if (*it == "_part_offset")
{
non_const_virtual_column_names.emplace_back(*it);
}
else
{
/// Remove virtual columns that are going to be filled with const values
if (header_without_virtual_columns.has(*it))
header_without_virtual_columns.erase(*it);
}
}
if (prewhere_info)
{
@ -199,23 +210,23 @@ void MergeTreeBaseSelectProcessor::initializeRangeReaders(MergeTreeReadTask & cu
{
if (reader->getColumns().empty())
{
current_task.range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_actions.get(), true);
current_task.range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_actions.get(), true, non_const_virtual_column_names);
}
else
{
MergeTreeRangeReader * pre_reader_ptr = nullptr;
if (pre_reader != nullptr)
{
current_task.pre_range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_actions.get(), false);
current_task.pre_range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_actions.get(), false, non_const_virtual_column_names);
pre_reader_ptr = &current_task.pre_range_reader;
}
current_task.range_reader = MergeTreeRangeReader(reader.get(), pre_reader_ptr, nullptr, true);
current_task.range_reader = MergeTreeRangeReader(reader.get(), pre_reader_ptr, nullptr, true, non_const_virtual_column_names);
}
}
else
{
current_task.range_reader = MergeTreeRangeReader(reader.get(), nullptr, nullptr, true);
current_task.range_reader = MergeTreeRangeReader(reader.get(), nullptr, nullptr, true, non_const_virtual_column_names);
}
}
@ -341,7 +352,25 @@ namespace
};
}
static void injectVirtualColumnsImpl(
/// Adds virtual columns that are not const for all rows
static void injectNonConstVirtualColumns(
size_t rows,
VirtualColumnsInserter & inserter,
const Names & virtual_columns)
{
if (unlikely(rows))
throw Exception("Cannot insert non-constant virtual column to non-empty chunk.",
ErrorCodes::LOGICAL_ERROR);
for (const auto & virtual_column_name : virtual_columns)
{
if (virtual_column_name == "_part_offset")
inserter.insertUInt64Column(DataTypeUInt64().createColumn(), virtual_column_name);
}
}
/// Adds virtual columns that are const for the whole part
static void injectPartConstVirtualColumns(
size_t rows,
VirtualColumnsInserter & inserter,
MergeTreeReadTask * task,
@ -502,7 +531,11 @@ void MergeTreeBaseSelectProcessor::injectVirtualColumns(
Block & block, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns)
{
VirtualColumnsInserterIntoBlock inserter{block};
injectVirtualColumnsImpl(block.rows(), inserter, task, partition_value_type, virtual_columns);
/// First add non-const columns that are filled by the range reader and then const columns that we will fill ourselves.
/// Note that the order is important: virtual columns filled by the range reader must go first
injectNonConstVirtualColumns(block.rows(), inserter, virtual_columns);
injectPartConstVirtualColumns(block.rows(), inserter, task, partition_value_type, virtual_columns);
}
void MergeTreeBaseSelectProcessor::injectVirtualColumns(
@ -512,7 +545,8 @@ void MergeTreeBaseSelectProcessor::injectVirtualColumns(
auto columns = chunk.detachColumns();
VirtualColumnsInserterIntoColumns inserter{columns};
injectVirtualColumnsImpl(num_rows, inserter, task, partition_value_type, virtual_columns);
/// Only add const virtual columns because non-const ones have already been added
injectPartConstVirtualColumns(num_rows, inserter, task, partition_value_type, virtual_columns);
chunk.setColumns(columns, num_rows);
}

View File

@ -102,6 +102,9 @@ protected:
Names virt_column_names;
/// These columns will be filled by the merge tree range reader
Names non_const_virtual_column_names;
DataTypePtr partition_value_type;
/// This header is used for chunks from readFromPart().

View File

@ -6416,6 +6416,7 @@ NamesAndTypesList MergeTreeData::getVirtuals() const
NameAndTypePair("_partition_id", std::make_shared<DataTypeString>()),
NameAndTypePair("_partition_value", getPartitionValueType()),
NameAndTypePair("_sample_factor", std::make_shared<DataTypeFloat64>()),
NameAndTypePair("_part_offset", std::make_shared<DataTypeUInt64>()),
};
}

View File

@ -1145,6 +1145,10 @@ static void selectColumnNames(
{
virt_column_names.push_back(name);
}
else if (name == "_part_offset")
{
virt_column_names.push_back(name);
}
else if (name == "_part_uuid")
{
virt_column_names.push_back(name);

View File

@ -22,7 +22,7 @@ std::unique_ptr<MergeTreeReaderStream> makeIndexReader(
std::move(settings), mark_cache, uncompressed_cache,
part->getFileSizeOrZero(index->getFileName() + extension),
&part->index_granularity_info,
ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE);
ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE, false);
}
}

View File

@ -537,13 +537,15 @@ MergeTreeRangeReader::MergeTreeRangeReader(
IMergeTreeReader * merge_tree_reader_,
MergeTreeRangeReader * prev_reader_,
const PrewhereExprInfo * prewhere_info_,
bool last_reader_in_chain_)
bool last_reader_in_chain_,
const Names & non_const_virtual_column_names_)
: merge_tree_reader(merge_tree_reader_)
, index_granularity(&(merge_tree_reader->data_part->index_granularity))
, prev_reader(prev_reader_)
, prewhere_info(prewhere_info_)
, last_reader_in_chain(last_reader_in_chain_)
, is_initialized(true)
, non_const_virtual_column_names(non_const_virtual_column_names_)
{
if (prev_reader)
sample_block = prev_reader->getSampleBlock();
@ -551,6 +553,15 @@ MergeTreeRangeReader::MergeTreeRangeReader(
for (const auto & name_and_type : merge_tree_reader->getColumns())
sample_block.insert({name_and_type.type->createColumn(), name_and_type.type, name_and_type.name});
for (const auto & column_name : non_const_virtual_column_names)
{
if (sample_block.has(column_name))
continue;
if (column_name == "_part_offset")
sample_block.insert(ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), column_name));
}
if (prewhere_info)
{
if (prewhere_info->alias_actions)
@ -616,6 +627,16 @@ size_t MergeTreeRangeReader::Stream::numPendingRows() const
return rows_between_marks - offset_after_current_mark;
}
UInt64 MergeTreeRangeReader::Stream::currentPartOffset() const
{
return index_granularity->getMarkStartingRow(current_mark) + offset_after_current_mark;
}
UInt64 MergeTreeRangeReader::Stream::lastPartOffset() const
{
return index_granularity->getMarkStartingRow(last_mark);
}
size_t MergeTreeRangeReader::Stream::ceilRowsToCompleteGranules(size_t rows_num) const
{
@ -730,16 +751,23 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
if (read_result.num_rows)
{
/// Physical columns go first and then some virtual columns follow
const size_t physical_columns_count = read_result.columns.size() - non_const_virtual_column_names.size();
Columns physical_columns(read_result.columns.begin(), read_result.columns.begin() + physical_columns_count);
bool should_evaluate_missing_defaults;
merge_tree_reader->fillMissingColumns(read_result.columns, should_evaluate_missing_defaults,
merge_tree_reader->fillMissingColumns(physical_columns, should_evaluate_missing_defaults,
read_result.num_rows);
/// If some columns absent in part, then evaluate default values
if (should_evaluate_missing_defaults)
merge_tree_reader->evaluateMissingDefaults({}, read_result.columns);
merge_tree_reader->evaluateMissingDefaults({}, physical_columns);
/// If result not empty, then apply on-fly alter conversions if any required
merge_tree_reader->performRequiredConversions(read_result.columns);
merge_tree_reader->performRequiredConversions(physical_columns);
for (size_t i = 0; i < physical_columns.size(); ++i)
read_result.columns[i] = std::move(physical_columns[i]);
}
else
read_result.columns.clear();
@ -767,6 +795,17 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t
size_t current_task_last_mark = getLastMark(ranges);
/// The stream could be unfinished by the previous read request because of max_rows limit.
/// In this case it will have some rows from the previously started range. We need to save their begin and
/// end offsets to properly fill _part_offset column.
UInt64 leading_begin_part_offset = 0;
UInt64 leading_end_part_offset = 0;
if (!stream.isFinished())
{
leading_begin_part_offset = stream.currentPartOffset();
leading_end_part_offset = stream.lastPartOffset();
}
/// Stream is lazy. result.num_added_rows is the number of rows added to block which is not equal to
/// result.num_rows_read until call to stream.finalize(). Also result.num_added_rows may be less than
/// result.num_rows_read if the last granule in range also the last in part (so we have to adjust last granule).
@ -803,9 +842,42 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t
/// Last granule may be incomplete.
result.adjustLastGranule();
for (const auto & column_name : non_const_virtual_column_names)
{
if (column_name == "_part_offset")
fillPartOffsetColumn(result, leading_begin_part_offset, leading_end_part_offset);
}
return result;
}
void MergeTreeRangeReader::fillPartOffsetColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset)
{
size_t num_rows = result.numReadRows();
auto column = ColumnUInt64::create(num_rows);
ColumnUInt64::Container & vec = column->getData();
UInt64 * pos = vec.data();
UInt64 * end = &vec[num_rows];
while (pos < end && leading_begin_part_offset < leading_end_part_offset)
*pos++ = leading_begin_part_offset++;
const auto start_ranges = result.startedRanges();
for (const auto & start_range : start_ranges)
{
UInt64 start_part_offset = index_granularity->getMarkStartingRow(start_range.range.begin);
UInt64 end_part_offset = index_granularity->getMarkStartingRow(start_range.range.end);
while (pos < end && start_part_offset < end_part_offset)
*pos++ = start_part_offset++;
}
result.columns.emplace_back(std::move(column));
}
Columns MergeTreeRangeReader::continueReadingChain(ReadResult & result, size_t & num_rows)
{
Columns columns;
@ -922,7 +994,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r
const auto & header = merge_tree_reader->getColumns();
size_t num_columns = header.size();
if (result.columns.size() != num_columns)
if (result.columns.size() != (num_columns + non_const_virtual_column_names.size()))
throw Exception("Invalid number of columns passed to MergeTreeRangeReader. "
"Expected " + toString(num_columns) + ", "
"got " + toString(result.columns.size()), ErrorCodes::LOGICAL_ERROR);
@ -948,6 +1020,15 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r
for (auto name_and_type = header.begin(); pos < num_columns; ++pos, ++name_and_type)
block.insert({result.columns[pos], name_and_type->type, name_and_type->name});
for (const auto & column_name : non_const_virtual_column_names)
{
if (column_name == "_part_offset")
block.insert({result.columns[pos], std::make_shared<DataTypeUInt64>(), column_name});
else
throw Exception("Unexpected non-const virtual column: " + column_name, ErrorCodes::LOGICAL_ERROR);
++pos;
}
if (prewhere_info->alias_actions)
prewhere_info->alias_actions->execute(block);

View File

@ -44,7 +44,8 @@ public:
IMergeTreeReader * merge_tree_reader_,
MergeTreeRangeReader * prev_reader_,
const PrewhereExprInfo * prewhere_info_,
bool last_reader_in_chain_);
bool last_reader_in_chain_,
const Names & non_const_virtual_column_names);
MergeTreeRangeReader() = default;
@ -58,6 +59,7 @@ public:
bool isCurrentRangeFinished() const;
bool isInitialized() const { return is_initialized; }
/// Accumulates sequential read() requests to perform a large read instead of multiple small reads
class DelayedStream
{
public:
@ -120,6 +122,8 @@ public:
size_t numPendingGranules() const { return last_mark - current_mark; }
size_t numPendingRows() const;
size_t currentMark() const { return current_mark; }
UInt64 currentPartOffset() const;
UInt64 lastPartOffset() const;
size_t current_mark = 0;
/// Invariant: offset_after_current_mark + skipped_rows_after_offset < index_granularity
@ -236,6 +240,7 @@ private:
ReadResult startReadingChain(size_t max_rows, MarkRanges & ranges);
Columns continueReadingChain(ReadResult & result, size_t & num_rows);
void executePrewhereActionsAndFilterColumns(ReadResult & result);
void fillPartOffsetColumn(ReadResult & result, UInt64 leading_begin_part_offset, UInt64 leading_end_part_offset);
IMergeTreeReader * merge_tree_reader = nullptr;
const MergeTreeIndexGranularity * index_granularity = nullptr;
@ -248,6 +253,7 @@ private:
bool last_reader_in_chain = false;
bool is_initialized = false;
Names non_const_virtual_column_names;
};
}

View File

@ -22,10 +22,12 @@ MergeTreeReaderStream::MergeTreeReaderStream(
MarkCache * mark_cache_,
UncompressedCache * uncompressed_cache, size_t file_size_,
const MergeTreeIndexGranularityInfo * index_granularity_info_,
const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type)
const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type,
bool is_low_cardinality_dictionary_)
: disk(std::move(disk_))
, path_prefix(path_prefix_)
, data_file_extension(data_file_extension_)
, is_low_cardinality_dictionary(is_low_cardinality_dictionary_)
, marks_count(marks_count_)
, file_size(file_size_)
, mark_cache(mark_cache_)
@ -126,7 +128,42 @@ size_t MergeTreeReaderStream::getRightOffset(size_t right_mark_non_included)
auto right_mark = marks_loader.getMark(right_mark_non_included);
result_right_offset = right_mark.offset_in_compressed_file;
bool need_to_check_marks_from_the_right = false;
/// If the end of range is inside the block, we will need to read it too.
if (right_mark.offset_in_decompressed_block > 0)
{
need_to_check_marks_from_the_right = true;
}
else if (is_low_cardinality_dictionary)
{
/// Also, in LowCardinality dictionary several consecutive marks can point to
/// the same offset. So to get true bytes offset we have to get first
/// non-equal mark.
/// Example:
/// Mark 186, points to [2003111, 0]
/// Mark 187, points to [2003111, 0]
/// Mark 188, points to [2003111, 0] <--- for example need to read until 188
/// Mark 189, points to [2003111, 0] <--- not suitable, because have same offset
/// Mark 190, points to [2003111, 0]
/// Mark 191, points to [2003111, 0]
/// Mark 192, points to [2081424, 0] <--- what we are looking for
/// Mark 193, points to [2081424, 0]
/// Mark 194, points to [2081424, 0]
/// Also, in some cases, when one granule is not-atomically written (which is possible at merges)
/// one granule may require reading of two dictionaries which starts from different marks.
/// The only correct way is to take offset from at least next different granule from the right one.
/// Check test_s3_low_cardinality_right_border.
need_to_check_marks_from_the_right = true;
}
/// Let's go to the right and find mark with bigger offset in compressed file
if (need_to_check_marks_from_the_right)
{
bool found_bigger_mark = false;
for (size_t i = right_mark_non_included + 1; i < marks_count; ++i)
{
@ -145,6 +182,7 @@ size_t MergeTreeReaderStream::getRightOffset(size_t right_mark_non_included)
result_right_offset = file_size;
}
}
}
else if (right_mark_non_included == 0)
result_right_offset = marks_loader.getMark(right_mark_non_included).offset_in_compressed_file;
else

View File

@ -24,7 +24,8 @@ public:
const MergeTreeReaderSettings & settings_,
MarkCache * mark_cache, UncompressedCache * uncompressed_cache,
size_t file_size_, const MergeTreeIndexGranularityInfo * index_granularity_info_,
const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type);
const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type,
bool is_low_cardinality_dictionary_);
void seekToMark(size_t index);
@ -46,6 +47,8 @@ private:
std::string path_prefix;
std::string data_file_extension;
bool is_low_cardinality_dictionary = false;
size_t marks_count;
size_t file_size;

View File

@ -180,12 +180,14 @@ void MergeTreeReaderWide::addStreams(const NameAndTypePair & name_and_type,
if (!data_file_exists)
return;
bool is_lc_dict = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys;
streams.emplace(stream_name, std::make_unique<MergeTreeReaderStream>(
disk, data_part->getFullRelativePath() + stream_name, DATA_FILE_EXTENSION,
data_part->getMarksCount(), all_mark_ranges, settings, mark_cache,
uncompressed_cache, data_part->getFileSizeOrZero(stream_name + DATA_FILE_EXTENSION),
&data_part->index_granularity_info,
profile_callback, clock_type));
profile_callback, clock_type, is_lc_dict));
};
data_part->getSerialization(name_and_type)->enumerateStreams(callback);

View File

@ -314,6 +314,7 @@ NamesAndTypesList StorageDistributed::getVirtuals() const
NameAndTypePair("_part_uuid", std::make_shared<DataTypeUUID>()),
NameAndTypePair("_partition_id", std::make_shared<DataTypeString>()),
NameAndTypePair("_sample_factor", std::make_shared<DataTypeFloat64>()),
NameAndTypePair("_part_offset", std::make_shared<DataTypeUInt64>()),
NameAndTypePair("_shard_num", std::make_shared<DataTypeUInt32>()), /// deprecated
};
}

View File

@ -59,6 +59,7 @@
#include <IO/Operators.h>
#include <IO/ConnectionTimeouts.h>
#include <IO/ConnectionTimeoutsContext.h>
#include <Disks/createVolume.h>
#include <Interpreters/InterpreterAlterQuery.h>
#include <Interpreters/PartLog.h>

View File

@ -25,6 +25,17 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
void ITableFunctionFileLike::parseFirstArguments(const ASTPtr & arg, ContextPtr context)
{
auto ast = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
filename = ast->as<ASTLiteral &>().value.safeGet<String>();
}
String ITableFunctionFileLike::getFormatFromFirstArgument()
{
return FormatFactory::instance().getFormatFromFileName(filename, true);
}
void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, ContextPtr context)
{
/// Parse args
@ -38,16 +49,16 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context
if (args.empty())
throw Exception("Table function '" + getName() + "' requires at least 1 argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
parseFirstArguments(args[0], context);
filename = args[0]->as<ASTLiteral &>().value.safeGet<String>();
for (size_t i = 1; i < args.size(); ++i)
args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(args[i], context);
if (args.size() > 1)
format = args[1]->as<ASTLiteral &>().value.safeGet<String>();
if (format == "auto")
format = FormatFactory::instance().getFormatFromFileName(filename, true);
format = getFormatFromFirstArgument();
if (args.size() <= 2)
return;

View File

@ -1,6 +1,7 @@
#pragma once
#include <TableFunctions/ITableFunction.h>
#include "Parsers/IAST_fwd.h"
namespace DB
{
@ -19,6 +20,8 @@ public:
protected:
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
virtual void parseFirstArguments(const ASTPtr & arg, ContextPtr context);
virtual String getFormatFromFirstArgument();
String filename;
String format = "auto";

View File

@ -1,16 +1,74 @@
#include <TableFunctions/TableFunctionFile.h>
#include <TableFunctions/parseColumnsListForTableFunction.h>
#include "Parsers/IAST_fwd.h"
#include "registerTableFunctions.h"
#include <Access/Common/AccessFlags.h>
#include <Interpreters/Context.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/StorageFile.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Formats/FormatFactory.h>
#include <Parsers/ASTIdentifier_fwd.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, ContextPtr context)
{
if (context->getApplicationType() != Context::ApplicationType::LOCAL)
{
ITableFunctionFileLike::parseFirstArguments(arg, context);
return;
}
if (auto opt_name = tryGetIdentifierName(arg))
{
if (*opt_name == "stdin")
fd = STDIN_FILENO;
else if (*opt_name == "stdout")
fd = STDOUT_FILENO;
else if (*opt_name == "stderr")
fd = STDERR_FILENO;
else
filename = *opt_name;
}
else if (const auto * literal = arg->as<ASTLiteral>())
{
auto type = literal->value.getType();
if (type == Field::Types::Int64 || type == Field::Types::UInt64)
{
fd = (type == Field::Types::Int64) ? static_cast<int>(literal->value.get<Int64>()) : static_cast<int>(literal->value.get<UInt64>());
if (fd < 0)
throw Exception("File descriptor must be non-negative", ErrorCodes::BAD_ARGUMENTS);
}
else if (type == Field::Types::String)
{
filename = literal->value.get<String>();
if (filename == "-")
fd = STDIN_FILENO;
}
else
throw Exception(
"The first argument of table function '" + getName() + "' mush be path or file descriptor", ErrorCodes::BAD_ARGUMENTS);
}
}
String TableFunctionFile::getFormatFromFirstArgument()
{
if (fd >= 0)
return FormatFactory::instance().getFormatFromFileDescriptor(fd);
else
return FormatFactory::instance().getFormatFromFileName(filename, true);
}
StoragePtr TableFunctionFile::getStorage(const String & source,
const String & format_, const ColumnsDescription & columns,
ContextPtr global_context, const std::string & table_name,
@ -28,6 +86,8 @@ StoragePtr TableFunctionFile::getStorage(const String & source,
ConstraintsDescription{},
String{},
};
if (fd >= 0)
return StorageFile::create(fd, args);
return std::make_shared<StorageFile>(source, global_context->getUserFilesPath(), args);
}
@ -36,6 +96,9 @@ ColumnsDescription TableFunctionFile::getActualTableStructure(ContextPtr context
{
if (structure == "auto")
{
if (fd >= 0)
throw Exception(
"Schema inference is not supported for table function '" + getName() + "' with file descriptor", ErrorCodes::LOGICAL_ERROR);
size_t total_bytes_to_read = 0;
Strings paths = StorageFile::getPathsList(filename, context->getUserFilesPath(), context, total_bytes_to_read);
return StorageFile::getTableStructureFromFile(format, paths, compression_method, std::nullopt, context);

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