mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge branch 'master' into uniqExactIf
This commit is contained in:
commit
758beda39a
@ -2234,7 +2234,7 @@ Result:
|
||||
|
||||
## Regular Expression Tree Dictionary {#regexp-tree-dictionary}
|
||||
|
||||
Regular expression tree dictionaries are a special type of dictionary which represent the mapping from key to attributes using a tree of regular expressions. There are some use cases, e.g. parsing of (user agent)[https://en.wikipedia.org/wiki/User_agent] strings, which can be expressed elegantly with regexp tree dictionaries.
|
||||
Regular expression tree dictionaries are a special type of dictionary which represent the mapping from key to attributes using a tree of regular expressions. There are some use cases, e.g. parsing of [user agent](https://en.wikipedia.org/wiki/User_agent) strings, which can be expressed elegantly with regexp tree dictionaries.
|
||||
|
||||
### Use Regular Expression Tree Dictionary in ClickHouse Open-Source
|
||||
|
||||
@ -2280,7 +2280,7 @@ This config consists of a list of regular expression tree nodes. Each node has t
|
||||
- The value of an attribute may contain **back references**, referring to capture groups of the matched regular expression. In the example, the value of attribute `version` in the first node consists of a back-reference `\1` to capture group `(\d+[\.\d]*)` in the regular expression. Back-reference numbers range from 1 to 9 and are written as `$1` or `\1` (for number 1). The back reference is replaced by the matched capture group during query execution.
|
||||
- **child nodes**: a list of children of a regexp tree node, each of which has its own attributes and (potentially) children nodes. String matching proceeds in a depth-first fashion. If a string matches a regexp node, the dictionary checks if it also matches the nodes' child nodes. If that is the case, the attributes of the deepest matching node are assigned. Attributes of a child node overwrite equally named attributes of parent nodes. The name of child nodes in YAML files can be arbitrary, e.g. `versions` in above example.
|
||||
|
||||
Regexp tree dictionaries only allow access using functions `dictGet`, `dictGetOrDefault` and `dictGetOrNull`.
|
||||
Regexp tree dictionaries only allow access using the functions `dictGet` and `dictGetOrDefault`.
|
||||
|
||||
Example:
|
||||
|
||||
|
@ -34,7 +34,7 @@ For the `SAMPLE` clause the following syntax is supported:
|
||||
| `SAMPLE k OFFSET m` | Here `k` and `m` are the numbers from 0 to 1. The query is executed on a sample of `k` fraction of the data. The data used for the sample is offset by `m` fraction. [Read more](#select-sample-offset) |
|
||||
|
||||
|
||||
## SAMPLE K
|
||||
## SAMPLE K {#select-sample-k}
|
||||
|
||||
Here `k` is the number from 0 to 1 (both fractional and decimal notations are supported). For example, `SAMPLE 1/2` or `SAMPLE 0.5`.
|
||||
|
||||
@ -54,7 +54,7 @@ ORDER BY PageViews DESC LIMIT 1000
|
||||
|
||||
In this example, the query is executed on a sample from 0.1 (10%) of data. Values of aggregate functions are not corrected automatically, so to get an approximate result, the value `count()` is manually multiplied by 10.
|
||||
|
||||
## SAMPLE N
|
||||
## SAMPLE N {#select-sample-n}
|
||||
|
||||
Here `n` is a sufficiently large integer. For example, `SAMPLE 10000000`.
|
||||
|
||||
@ -90,7 +90,7 @@ FROM visits
|
||||
SAMPLE 10000000
|
||||
```
|
||||
|
||||
## SAMPLE K OFFSET M
|
||||
## SAMPLE K OFFSET M {#select-sample-offset}
|
||||
|
||||
Here `k` and `m` are numbers from 0 to 1. Examples are shown below.
|
||||
|
||||
|
@ -433,7 +433,7 @@ const String & AsyncLoader::getPoolName(size_t pool) const
|
||||
return pools[pool].name; // NOTE: lock is not needed because `name` is const and `pools` are immutable
|
||||
}
|
||||
|
||||
ssize_t AsyncLoader::getPoolPriority(size_t pool) const
|
||||
Priority AsyncLoader::getPoolPriority(size_t pool) const
|
||||
{
|
||||
return pools[pool].priority; // NOTE: lock is not needed because `priority` is const and `pools` are immutable
|
||||
}
|
||||
@ -576,7 +576,7 @@ void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool_id, std::un
|
||||
{
|
||||
Pool & old_pool = pools[job->pool_id];
|
||||
Pool & new_pool = pools[new_pool_id];
|
||||
if (old_pool.priority >= new_pool.priority)
|
||||
if (old_pool.priority <= new_pool.priority)
|
||||
return; // Never lower priority or change pool leaving the same priority
|
||||
|
||||
// Update priority and push job forward through ready queue if needed
|
||||
@ -590,7 +590,7 @@ void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool_id, std::un
|
||||
spawn(new_pool, lock);
|
||||
}
|
||||
|
||||
// Set user-facing pool and priority (may affect executing jobs)
|
||||
// Set user-facing pool (may affect executing jobs)
|
||||
job->pool_id.store(new_pool_id);
|
||||
|
||||
// Recurse into dependencies
|
||||
@ -621,7 +621,7 @@ bool AsyncLoader::canSpawnWorker(Pool & pool, std::unique_lock<std::mutex> &)
|
||||
return is_running
|
||||
&& !pool.ready_queue.empty()
|
||||
&& pool.workers < pool.max_threads
|
||||
&& (!current_priority || *current_priority <= pool.priority);
|
||||
&& (!current_priority || *current_priority >= pool.priority);
|
||||
}
|
||||
|
||||
bool AsyncLoader::canWorkerLive(Pool & pool, std::unique_lock<std::mutex> &)
|
||||
@ -629,17 +629,17 @@ bool AsyncLoader::canWorkerLive(Pool & pool, std::unique_lock<std::mutex> &)
|
||||
return is_running
|
||||
&& !pool.ready_queue.empty()
|
||||
&& pool.workers <= pool.max_threads
|
||||
&& (!current_priority || *current_priority <= pool.priority);
|
||||
&& (!current_priority || *current_priority >= pool.priority);
|
||||
}
|
||||
|
||||
void AsyncLoader::updateCurrentPriorityAndSpawn(std::unique_lock<std::mutex> & lock)
|
||||
{
|
||||
// Find current priority.
|
||||
// NOTE: We assume low number of pools, so O(N) scans are fine.
|
||||
std::optional<ssize_t> priority;
|
||||
std::optional<Priority> priority;
|
||||
for (Pool & pool : pools)
|
||||
{
|
||||
if (pool.isActive() && (!priority || *priority < pool.priority))
|
||||
if (pool.isActive() && (!priority || *priority > pool.priority))
|
||||
priority = pool.priority;
|
||||
}
|
||||
current_priority = priority;
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <base/types.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/Priority.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/ThreadPool_fwd.h>
|
||||
|
||||
@ -268,10 +269,10 @@ inline LoadTaskPtrs joinTasks(const LoadTaskPtrs & tasks1, const LoadTaskPtrs &
|
||||
|
||||
// `AsyncLoader` is a scheduler for DAG of `LoadJob`s. It tracks job dependencies and priorities.
|
||||
// Basic usage example:
|
||||
// // Start async_loader with two thread pools (0=bg, 1=fg):
|
||||
// // Start async_loader with two thread pools (0=fg, 1=bg):
|
||||
// AsyncLoader async_loader({
|
||||
// {"BgPool", CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, .max_threads = 1, .priority = 0}
|
||||
// {"FgPool", CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, .max_threads = 2, .priority = 1}
|
||||
// {"FgPool", CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, .max_threads = 2, .priority{0}}
|
||||
// {"BgPool", CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, .max_threads = 1, .priority{1}}
|
||||
// });
|
||||
//
|
||||
// // Create and schedule a task consisting of three jobs. Job1 has no dependencies and is run first.
|
||||
@ -279,19 +280,19 @@ inline LoadTaskPtrs joinTasks(const LoadTaskPtrs & tasks1, const LoadTaskPtrs &
|
||||
// auto job_func = [&] (const LoadJobPtr & self) {
|
||||
// LOG_TRACE(log, "Executing load job '{}' in pool '{}'", self->name, async_loader->getPoolName(self->pool()));
|
||||
// };
|
||||
// auto job1 = makeLoadJob({}, "job1", /* pool_id = */ 0, job_func);
|
||||
// auto job2 = makeLoadJob({ job1 }, "job2", /* pool_id = */ 0, job_func);
|
||||
// auto job3 = makeLoadJob({ job1 }, "job3", /* pool_id = */ 0, job_func);
|
||||
// auto job1 = makeLoadJob({}, "job1", /* pool_id = */ 1, job_func);
|
||||
// auto job2 = makeLoadJob({ job1 }, "job2", /* pool_id = */ 1, job_func);
|
||||
// auto job3 = makeLoadJob({ job1 }, "job3", /* pool_id = */ 1, job_func);
|
||||
// auto task = makeLoadTask(async_loader, { job1, job2, job3 });
|
||||
// task.schedule();
|
||||
//
|
||||
// // Another thread may prioritize a job by changing its pool and wait for it:
|
||||
// async_loader->prioritize(job3, /* pool_id = */ 1); // higher priority jobs are run first, default priority is zero.
|
||||
// job3->wait(); // blocks until job completion or cancellation and rethrow an exception (if any)
|
||||
// async_loader->prioritize(job3, /* pool_id = */ 0); // Increase priority: 1 -> 0 (lower is better)
|
||||
// job3->wait(); // Blocks until job completion or cancellation and rethrow an exception (if any)
|
||||
//
|
||||
// Every job has a pool associated with it. AsyncLoader starts every job in its thread pool.
|
||||
// Each pool has a constant priority and a mutable maximum number of threads.
|
||||
// Higher priority (greater `pool.priority` value) jobs are run first.
|
||||
// Higher priority (lower `pool.priority` value) jobs are run first.
|
||||
// No job with lower priority is started while there is at least one higher priority job ready or running.
|
||||
//
|
||||
// Job priority can be elevated (but cannot be lowered)
|
||||
@ -301,7 +302,8 @@ inline LoadTaskPtrs joinTasks(const LoadTaskPtrs & tasks1, const LoadTaskPtrs &
|
||||
// this also leads to a priority inheritance for all the dependencies.
|
||||
// Value stored in load job `pool_id` field is atomic and can be changed even during job execution.
|
||||
// Job is, of course, not moved from its initial thread pool, but it should use `self->pool()` for
|
||||
// all new jobs it create to avoid priority inversion.
|
||||
// all new jobs it create to avoid priority inversion. To obtain pool in which job is being executed
|
||||
// call `self->execution_pool()` instead.
|
||||
//
|
||||
// === IMPLEMENTATION DETAILS ===
|
||||
// All possible states and statuses of a job:
|
||||
@ -335,7 +337,7 @@ private:
|
||||
struct Pool
|
||||
{
|
||||
const String name;
|
||||
const ssize_t priority;
|
||||
const Priority priority;
|
||||
std::unique_ptr<ThreadPool> thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools.
|
||||
std::map<UInt64, LoadJobPtr> ready_queue; // FIFO queue of jobs to be executed in this pool. Map is used for faster erasing. Key is `ready_seqno`
|
||||
size_t max_threads; // Max number of workers to be spawn
|
||||
@ -367,7 +369,7 @@ public:
|
||||
Metric metric_threads;
|
||||
Metric metric_active_threads;
|
||||
size_t max_threads;
|
||||
ssize_t priority;
|
||||
Priority priority;
|
||||
};
|
||||
|
||||
AsyncLoader(std::vector<PoolInitializer> pool_initializers, bool log_failures_, bool log_progress_);
|
||||
@ -412,7 +414,7 @@ public:
|
||||
|
||||
size_t getMaxThreads(size_t pool) const;
|
||||
const String & getPoolName(size_t pool) const;
|
||||
ssize_t getPoolPriority(size_t pool) const;
|
||||
Priority getPoolPriority(size_t pool) const;
|
||||
|
||||
size_t getScheduledJobCount() const;
|
||||
|
||||
@ -451,7 +453,7 @@ private:
|
||||
|
||||
mutable std::mutex mutex; // Guards all the fields below.
|
||||
bool is_running = true;
|
||||
std::optional<ssize_t> current_priority; // highest priority among active pools
|
||||
std::optional<Priority> current_priority; // highest priority among active pools
|
||||
UInt64 last_ready_seqno = 0; // Increasing counter for ready queue keys.
|
||||
std::unordered_map<LoadJobPtr, Info> scheduled_jobs; // Full set of scheduled pending jobs along with scheduling info.
|
||||
std::vector<Pool> pools; // Thread pools for job execution and ready queues
|
||||
|
@ -32,7 +32,7 @@ namespace DB::ErrorCodes
|
||||
|
||||
struct Initializer {
|
||||
size_t max_threads = 1;
|
||||
ssize_t priority = 0;
|
||||
Priority priority;
|
||||
};
|
||||
|
||||
struct AsyncLoaderTest
|
||||
@ -144,11 +144,11 @@ struct AsyncLoaderTest
|
||||
TEST(AsyncLoader, Smoke)
|
||||
{
|
||||
AsyncLoaderTest t({
|
||||
{.max_threads = 2, .priority = 0},
|
||||
{.max_threads = 2, .priority = -1},
|
||||
{.max_threads = 2, .priority = Priority{0}},
|
||||
{.max_threads = 2, .priority = Priority{1}},
|
||||
});
|
||||
|
||||
static constexpr ssize_t low_priority_pool = 1;
|
||||
static constexpr size_t low_priority_pool = 1;
|
||||
|
||||
std::atomic<size_t> jobs_done{0};
|
||||
std::atomic<size_t> low_priority_jobs_done{0};
|
||||
@ -419,6 +419,8 @@ TEST(AsyncLoader, CancelExecutingTask)
|
||||
}
|
||||
}
|
||||
|
||||
// This test is disabled due to `MemorySanitizer: use-of-uninitialized-value` issue in `collectSymbolsFromProgramHeaders` function
|
||||
// More details: https://github.com/ClickHouse/ClickHouse/pull/48923#issuecomment-1545415482
|
||||
TEST(AsyncLoader, DISABLED_JobFailure)
|
||||
{
|
||||
AsyncLoaderTest t;
|
||||
@ -595,16 +597,16 @@ TEST(AsyncLoader, TestOverload)
|
||||
TEST(AsyncLoader, StaticPriorities)
|
||||
{
|
||||
AsyncLoaderTest t({
|
||||
{.max_threads = 1, .priority = 0},
|
||||
{.max_threads = 1, .priority = 1},
|
||||
{.max_threads = 1, .priority = 2},
|
||||
{.max_threads = 1, .priority = 3},
|
||||
{.max_threads = 1, .priority = 4},
|
||||
{.max_threads = 1, .priority = 5},
|
||||
{.max_threads = 1, .priority = 6},
|
||||
{.max_threads = 1, .priority = 7},
|
||||
{.max_threads = 1, .priority = 8},
|
||||
{.max_threads = 1, .priority = 9},
|
||||
{.max_threads = 1, .priority{0}},
|
||||
{.max_threads = 1, .priority{-1}},
|
||||
{.max_threads = 1, .priority{-2}},
|
||||
{.max_threads = 1, .priority{-3}},
|
||||
{.max_threads = 1, .priority{-4}},
|
||||
{.max_threads = 1, .priority{-5}},
|
||||
{.max_threads = 1, .priority{-6}},
|
||||
{.max_threads = 1, .priority{-7}},
|
||||
{.max_threads = 1, .priority{-8}},
|
||||
{.max_threads = 1, .priority{-9}},
|
||||
});
|
||||
|
||||
std::string schedule;
|
||||
@ -634,9 +636,9 @@ TEST(AsyncLoader, StaticPriorities)
|
||||
TEST(AsyncLoader, SimplePrioritization)
|
||||
{
|
||||
AsyncLoaderTest t({
|
||||
{.max_threads = 1, .priority = 0},
|
||||
{.max_threads = 1, .priority = 1},
|
||||
{.max_threads = 1, .priority = 2},
|
||||
{.max_threads = 1, .priority{0}},
|
||||
{.max_threads = 1, .priority{-1}},
|
||||
{.max_threads = 1, .priority{-2}},
|
||||
});
|
||||
|
||||
t.loader.start();
|
||||
@ -674,16 +676,16 @@ TEST(AsyncLoader, SimplePrioritization)
|
||||
TEST(AsyncLoader, DynamicPriorities)
|
||||
{
|
||||
AsyncLoaderTest t({
|
||||
{.max_threads = 1, .priority = 0},
|
||||
{.max_threads = 1, .priority = 1},
|
||||
{.max_threads = 1, .priority = 2},
|
||||
{.max_threads = 1, .priority = 3},
|
||||
{.max_threads = 1, .priority = 4},
|
||||
{.max_threads = 1, .priority = 5},
|
||||
{.max_threads = 1, .priority = 6},
|
||||
{.max_threads = 1, .priority = 7},
|
||||
{.max_threads = 1, .priority = 8},
|
||||
{.max_threads = 1, .priority = 9},
|
||||
{.max_threads = 1, .priority{0}},
|
||||
{.max_threads = 1, .priority{-1}},
|
||||
{.max_threads = 1, .priority{-2}},
|
||||
{.max_threads = 1, .priority{-3}},
|
||||
{.max_threads = 1, .priority{-4}},
|
||||
{.max_threads = 1, .priority{-5}},
|
||||
{.max_threads = 1, .priority{-6}},
|
||||
{.max_threads = 1, .priority{-7}},
|
||||
{.max_threads = 1, .priority{-8}},
|
||||
{.max_threads = 1, .priority{-9}},
|
||||
});
|
||||
|
||||
for (bool prioritize : {false, true})
|
||||
@ -890,8 +892,8 @@ TEST(AsyncLoader, DynamicPools)
|
||||
const size_t max_threads[] { 2, 10 };
|
||||
const int jobs_in_chain = 16;
|
||||
AsyncLoaderTest t({
|
||||
{.max_threads = max_threads[0], .priority = 0},
|
||||
{.max_threads = max_threads[1], .priority = 1},
|
||||
{.max_threads = max_threads[0], .priority{0}},
|
||||
{.max_threads = max_threads[1], .priority{-1}},
|
||||
});
|
||||
|
||||
t.loader.start();
|
||||
|
@ -47,7 +47,8 @@ struct Settings;
|
||||
M(Bool, compress_snapshots_with_zstd_format, true, "Write compressed snapshots in ZSTD format (instead of custom LZ4)", 0) \
|
||||
M(UInt64, configuration_change_tries_count, 20, "How many times we will try to apply configuration change (add/remove server) to the cluster", 0) \
|
||||
M(UInt64, max_log_file_size, 50 * 1024 * 1024, "Max size of the Raft log file. If possible, each created log file will preallocate this amount of bytes on disk. Set to 0 to disable the limit", 0) \
|
||||
M(UInt64, log_file_overallocate_size, 50 * 1024 * 1024, "If max_log_file_size is not set to 0, this value will be added to it for preallocating bytes on disk. If a log record is larger than this value, it could lead to uncaught out-of-space issues so a larger value is preferred", 0)
|
||||
M(UInt64, log_file_overallocate_size, 50 * 1024 * 1024, "If max_log_file_size is not set to 0, this value will be added to it for preallocating bytes on disk. If a log record is larger than this value, it could lead to uncaught out-of-space issues so a larger value is preferred", 0) \
|
||||
M(UInt64, min_request_size_for_cache, 50 * 1024, "Minimal size of the request to cache the deserialization result. Caching can have negative effect on latency for smaller requests, set to 0 to disable", 0)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS)
|
||||
|
||||
|
@ -471,17 +471,6 @@ void KeeperServer::shutdown()
|
||||
namespace
|
||||
{
|
||||
|
||||
// Serialize the request with all the necessary information for the leader
|
||||
// we don't know ZXID and digest yet so we don't serialize it
|
||||
nuraft::ptr<nuraft::buffer> getZooKeeperRequestMessage(const KeeperStorage::RequestForSession & request_for_session)
|
||||
{
|
||||
DB::WriteBufferFromNuraftBuffer write_buf;
|
||||
DB::writeIntBinary(request_for_session.session_id, write_buf);
|
||||
request_for_session.request->write(write_buf);
|
||||
DB::writeIntBinary(request_for_session.time, write_buf);
|
||||
return write_buf.getBuffer();
|
||||
}
|
||||
|
||||
// Serialize the request for the log entry
|
||||
nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(const KeeperStorage::RequestForSession & request_for_session)
|
||||
{
|
||||
@ -489,12 +478,11 @@ nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(const KeeperStorage::RequestFor
|
||||
DB::writeIntBinary(request_for_session.session_id, write_buf);
|
||||
request_for_session.request->write(write_buf);
|
||||
DB::writeIntBinary(request_for_session.time, write_buf);
|
||||
DB::writeIntBinary(request_for_session.zxid, write_buf);
|
||||
assert(request_for_session.digest);
|
||||
DB::writeIntBinary(request_for_session.digest->version, write_buf);
|
||||
if (request_for_session.digest->version != KeeperStorage::DigestVersion::NO_DIGEST)
|
||||
DB::writeIntBinary(request_for_session.digest->value, write_buf);
|
||||
|
||||
/// we fill with dummy values to eliminate unnecessary copy later on when we will write correct values
|
||||
DB::writeIntBinary(static_cast<int64_t>(0), write_buf); /// zxid
|
||||
DB::writeIntBinary(KeeperStorage::DigestVersion::NO_DIGEST, write_buf); /// digest version or NO_DIGEST flag
|
||||
DB::writeIntBinary(static_cast<uint64_t>(0), write_buf); /// digest value
|
||||
/// if new fields are added, update KeeperStateMachine::ZooKeeperLogSerializationVersion along with parseRequest function and PreAppendLog callback handler
|
||||
return write_buf.getBuffer();
|
||||
}
|
||||
|
||||
@ -512,9 +500,7 @@ RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorage::RequestsForS
|
||||
{
|
||||
std::vector<nuraft::ptr<nuraft::buffer>> entries;
|
||||
for (const auto & request_for_session : requests_for_sessions)
|
||||
{
|
||||
entries.push_back(getZooKeeperRequestMessage(request_for_session));
|
||||
}
|
||||
entries.push_back(getZooKeeperLogEntry(request_for_session));
|
||||
|
||||
std::lock_guard lock{server_write_mutex};
|
||||
if (is_recovering)
|
||||
@ -635,14 +621,52 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
|
||||
assert(entry->get_val_type() == nuraft::app_log);
|
||||
auto next_zxid = state_machine->getNextZxid();
|
||||
|
||||
auto & entry_buf = entry->get_buf();
|
||||
auto request_for_session = state_machine->parseRequest(entry_buf);
|
||||
request_for_session.zxid = next_zxid;
|
||||
if (!state_machine->preprocess(request_for_session))
|
||||
auto entry_buf = entry->get_buf_ptr();
|
||||
|
||||
KeeperStateMachine::ZooKeeperLogSerializationVersion serialization_version;
|
||||
auto request_for_session = state_machine->parseRequest(*entry_buf, /*final=*/false, &serialization_version);
|
||||
request_for_session->zxid = next_zxid;
|
||||
if (!state_machine->preprocess(*request_for_session))
|
||||
return nuraft::cb_func::ReturnCode::ReturnNull;
|
||||
|
||||
request_for_session.digest = state_machine->getNodesDigest();
|
||||
entry = nuraft::cs_new<nuraft::log_entry>(entry->get_term(), getZooKeeperLogEntry(request_for_session), entry->get_val_type());
|
||||
request_for_session->digest = state_machine->getNodesDigest();
|
||||
|
||||
using enum KeeperStateMachine::ZooKeeperLogSerializationVersion;
|
||||
|
||||
/// older versions of Keeper can send logs that are missing some fields
|
||||
size_t bytes_missing = 0;
|
||||
if (serialization_version < WITH_TIME)
|
||||
bytes_missing += sizeof(request_for_session->time);
|
||||
|
||||
if (serialization_version < WITH_ZXID_DIGEST)
|
||||
bytes_missing += sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value);
|
||||
|
||||
if (bytes_missing != 0)
|
||||
{
|
||||
auto new_buffer = nuraft::buffer::alloc(entry_buf->size() + bytes_missing);
|
||||
memcpy(new_buffer->data_begin(), entry_buf->data_begin(), entry_buf->size());
|
||||
entry_buf = std::move(new_buffer);
|
||||
entry = nuraft::cs_new<nuraft::log_entry>(entry->get_term(), entry_buf, entry->get_val_type());
|
||||
}
|
||||
|
||||
size_t write_buffer_header_size
|
||||
= sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value);
|
||||
|
||||
if (serialization_version < WITH_TIME)
|
||||
write_buffer_header_size += sizeof(request_for_session->time);
|
||||
|
||||
auto * buffer_start = reinterpret_cast<BufferBase::Position>(entry_buf->data_begin() + entry_buf->size() - write_buffer_header_size);
|
||||
|
||||
WriteBuffer write_buf(buffer_start, write_buffer_header_size);
|
||||
|
||||
if (serialization_version < WITH_TIME)
|
||||
writeIntBinary(request_for_session->time, write_buf);
|
||||
|
||||
writeIntBinary(request_for_session->zxid, write_buf);
|
||||
writeIntBinary(request_for_session->digest->version, write_buf);
|
||||
if (request_for_session->digest->version != KeeperStorage::NO_DIGEST)
|
||||
writeIntBinary(request_for_session->digest->value, write_buf);
|
||||
|
||||
break;
|
||||
}
|
||||
case nuraft::cb_func::AppendLogFailed:
|
||||
@ -654,8 +678,8 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
|
||||
assert(entry->get_val_type() == nuraft::app_log);
|
||||
|
||||
auto & entry_buf = entry->get_buf();
|
||||
auto request_for_session = state_machine->parseRequest(entry_buf);
|
||||
state_machine->rollbackRequest(request_for_session, true);
|
||||
auto request_for_session = state_machine->parseRequest(entry_buf, true);
|
||||
state_machine->rollbackRequest(*request_for_session, true);
|
||||
break;
|
||||
}
|
||||
default:
|
||||
|
@ -1,16 +1,16 @@
|
||||
#include <cerrno>
|
||||
#include <base/errnoToString.h>
|
||||
#include <base/defines.h>
|
||||
#include <future>
|
||||
#include <Coordination/KeeperSnapshotManager.h>
|
||||
#include <Coordination/KeeperStateMachine.h>
|
||||
#include <Coordination/ReadBufferFromNuraftBuffer.h>
|
||||
#include <Coordination/WriteBufferFromNuraftBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <base/defines.h>
|
||||
#include <base/errnoToString.h>
|
||||
#include <sys/mman.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperCommon.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperIO.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include "Coordination/KeeperStorage.h"
|
||||
|
||||
@ -60,6 +60,7 @@ KeeperStateMachine::KeeperStateMachine(
|
||||
coordination_settings->dead_session_check_period_ms.totalMilliseconds())
|
||||
, responses_queue(responses_queue_)
|
||||
, snapshots_queue(snapshots_queue_)
|
||||
, min_request_size_to_cache(coordination_settings_->min_request_size_for_cache)
|
||||
, last_committed_idx(0)
|
||||
, log(&Poco::Logger::get("KeeperStateMachine"))
|
||||
, superdigest(superdigest_)
|
||||
@ -149,19 +150,19 @@ void assertDigest(
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperStateMachine::pre_commit(uint64_t log_idx, nuraft::buffer & data)
|
||||
{
|
||||
auto request_for_session = parseRequest(data);
|
||||
if (!request_for_session.zxid)
|
||||
request_for_session.zxid = log_idx;
|
||||
auto request_for_session = parseRequest(data, /*final=*/false);
|
||||
if (!request_for_session->zxid)
|
||||
request_for_session->zxid = log_idx;
|
||||
|
||||
preprocess(request_for_session);
|
||||
preprocess(*request_for_session);
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
KeeperStorage::RequestForSession KeeperStateMachine::parseRequest(nuraft::buffer & data)
|
||||
std::shared_ptr<KeeperStorage::RequestForSession> KeeperStateMachine::parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version)
|
||||
{
|
||||
ReadBufferFromNuraftBuffer buffer(data);
|
||||
KeeperStorage::RequestForSession request_for_session;
|
||||
readIntBinary(request_for_session.session_id, buffer);
|
||||
auto request_for_session = std::make_shared<KeeperStorage::RequestForSession>();
|
||||
readIntBinary(request_for_session->session_id, buffer);
|
||||
|
||||
int32_t length;
|
||||
Coordination::read(length, buffer);
|
||||
@ -169,29 +170,81 @@ KeeperStorage::RequestForSession KeeperStateMachine::parseRequest(nuraft::buffer
|
||||
int32_t xid;
|
||||
Coordination::read(xid, buffer);
|
||||
|
||||
static constexpr std::array non_cacheable_xids{
|
||||
Coordination::WATCH_XID,
|
||||
Coordination::PING_XID,
|
||||
Coordination::AUTH_XID,
|
||||
Coordination::CLOSE_XID,
|
||||
};
|
||||
|
||||
const bool should_cache
|
||||
= min_request_size_to_cache != 0 && request_for_session->session_id != -1 && data.size() >= min_request_size_to_cache
|
||||
&& std::all_of(
|
||||
non_cacheable_xids.begin(), non_cacheable_xids.end(), [&](const auto non_cacheable_xid) { return xid != non_cacheable_xid; });
|
||||
|
||||
if (should_cache)
|
||||
{
|
||||
std::lock_guard lock(request_cache_mutex);
|
||||
if (auto xid_to_request_it = parsed_request_cache.find(request_for_session->session_id);
|
||||
xid_to_request_it != parsed_request_cache.end())
|
||||
{
|
||||
auto & xid_to_request = xid_to_request_it->second;
|
||||
if (auto request_it = xid_to_request.find(xid); request_it != xid_to_request.end())
|
||||
{
|
||||
if (final)
|
||||
{
|
||||
auto request = std::move(request_it->second);
|
||||
xid_to_request.erase(request_it);
|
||||
return request;
|
||||
}
|
||||
else
|
||||
return request_it->second;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Coordination::OpNum opnum;
|
||||
|
||||
Coordination::read(opnum, buffer);
|
||||
|
||||
request_for_session.request = Coordination::ZooKeeperRequestFactory::instance().get(opnum);
|
||||
request_for_session.request->xid = xid;
|
||||
request_for_session.request->readImpl(buffer);
|
||||
request_for_session->request = Coordination::ZooKeeperRequestFactory::instance().get(opnum);
|
||||
request_for_session->request->xid = xid;
|
||||
request_for_session->request->readImpl(buffer);
|
||||
|
||||
if (!buffer.eof())
|
||||
readIntBinary(request_for_session.time, buffer);
|
||||
else /// backward compatibility
|
||||
request_for_session.time
|
||||
= std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::system_clock::now().time_since_epoch()).count();
|
||||
|
||||
if (!buffer.eof())
|
||||
readIntBinary(request_for_session.zxid, buffer);
|
||||
using enum ZooKeeperLogSerializationVersion;
|
||||
ZooKeeperLogSerializationVersion version = INITIAL;
|
||||
|
||||
if (!buffer.eof())
|
||||
{
|
||||
request_for_session.digest.emplace();
|
||||
readIntBinary(request_for_session.digest->version, buffer);
|
||||
if (request_for_session.digest->version != KeeperStorage::DigestVersion::NO_DIGEST)
|
||||
readIntBinary(request_for_session.digest->value, buffer);
|
||||
version = WITH_TIME;
|
||||
readIntBinary(request_for_session->time, buffer);
|
||||
}
|
||||
else
|
||||
request_for_session->time
|
||||
= std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::system_clock::now().time_since_epoch()).count();
|
||||
|
||||
if (!buffer.eof())
|
||||
{
|
||||
version = WITH_ZXID_DIGEST;
|
||||
|
||||
readIntBinary(request_for_session->zxid, buffer);
|
||||
|
||||
chassert(!buffer.eof());
|
||||
|
||||
request_for_session->digest.emplace();
|
||||
readIntBinary(request_for_session->digest->version, buffer);
|
||||
if (request_for_session->digest->version != KeeperStorage::DigestVersion::NO_DIGEST || !buffer.eof())
|
||||
readIntBinary(request_for_session->digest->value, buffer);
|
||||
}
|
||||
|
||||
if (serialization_version)
|
||||
*serialization_version = version;
|
||||
|
||||
if (should_cache && !final)
|
||||
{
|
||||
std::lock_guard lock(request_cache_mutex);
|
||||
parsed_request_cache[request_for_session->session_id].emplace(xid, request_for_session);
|
||||
}
|
||||
|
||||
return request_for_session;
|
||||
@ -231,15 +284,15 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data)
|
||||
{
|
||||
auto request_for_session = parseRequest(data);
|
||||
if (!request_for_session.zxid)
|
||||
request_for_session.zxid = log_idx;
|
||||
auto request_for_session = parseRequest(data, true);
|
||||
if (!request_for_session->zxid)
|
||||
request_for_session->zxid = log_idx;
|
||||
|
||||
/// Special processing of session_id request
|
||||
if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID)
|
||||
if (request_for_session->request->getOpNum() == Coordination::OpNum::SessionID)
|
||||
{
|
||||
const Coordination::ZooKeeperSessionIDRequest & session_id_request
|
||||
= dynamic_cast<const Coordination::ZooKeeperSessionIDRequest &>(*request_for_session.request);
|
||||
= dynamic_cast<const Coordination::ZooKeeperSessionIDRequest &>(*request_for_session->request);
|
||||
int64_t session_id;
|
||||
std::shared_ptr<Coordination::ZooKeeperSessionIDResponse> response = std::make_shared<Coordination::ZooKeeperSessionIDResponse>();
|
||||
response->internal_id = session_id_request.internal_id;
|
||||
@ -261,25 +314,34 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
|
||||
}
|
||||
else
|
||||
{
|
||||
if (request_for_session->request->getOpNum() == Coordination::OpNum::Close)
|
||||
{
|
||||
std::lock_guard lock(request_cache_mutex);
|
||||
parsed_request_cache.erase(request_for_session->session_id);
|
||||
}
|
||||
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
KeeperStorage::ResponsesForSessions responses_for_sessions = storage->processRequest(
|
||||
request_for_session.request, request_for_session.session_id, request_for_session.zxid);
|
||||
KeeperStorage::ResponsesForSessions responses_for_sessions
|
||||
= storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid);
|
||||
for (auto & response_for_session : responses_for_sessions)
|
||||
if (!responses_queue.push(response_for_session))
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed);
|
||||
LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response_for_session.session_id);
|
||||
LOG_WARNING(
|
||||
log,
|
||||
"Failed to push response with session id {} to the queue, probably because of shutdown",
|
||||
response_for_session.session_id);
|
||||
}
|
||||
|
||||
if (keeper_context->digest_enabled && request_for_session.digest)
|
||||
assertDigest(*request_for_session.digest, storage->getNodesDigest(true), *request_for_session.request, true);
|
||||
if (keeper_context->digest_enabled && request_for_session->digest)
|
||||
assertDigest(*request_for_session->digest, storage->getNodesDigest(true), *request_for_session->request, true);
|
||||
}
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::KeeperCommits);
|
||||
last_committed_idx = log_idx;
|
||||
|
||||
if (commit_callback)
|
||||
commit_callback(request_for_session);
|
||||
commit_callback(*request_for_session);
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
@ -330,14 +392,14 @@ void KeeperStateMachine::commit_config(const uint64_t /* log_idx */, nuraft::ptr
|
||||
|
||||
void KeeperStateMachine::rollback(uint64_t log_idx, nuraft::buffer & data)
|
||||
{
|
||||
auto request_for_session = parseRequest(data);
|
||||
auto request_for_session = parseRequest(data, true);
|
||||
// If we received a log from an older node, use the log_idx as the zxid
|
||||
// log_idx will always be larger or equal to the zxid so we can safely do this
|
||||
// (log_idx is increased for all logs, while zxid is only increased for requests)
|
||||
if (!request_for_session.zxid)
|
||||
request_for_session.zxid = log_idx;
|
||||
if (!request_for_session->zxid)
|
||||
request_for_session->zxid = log_idx;
|
||||
|
||||
rollbackRequest(request_for_session, false);
|
||||
rollbackRequest(*request_for_session, false);
|
||||
}
|
||||
|
||||
void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing)
|
||||
@ -541,11 +603,7 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSessi
|
||||
/// Pure local request, just process it with storage
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
auto responses = storage->processRequest(
|
||||
request_for_session.request,
|
||||
request_for_session.session_id,
|
||||
std::nullopt,
|
||||
true /*check_acl*/,
|
||||
true /*is_local*/);
|
||||
request_for_session.request, request_for_session.session_id, std::nullopt, true /*check_acl*/, true /*is_local*/);
|
||||
for (const auto & response : responses)
|
||||
if (!responses_queue.push(response))
|
||||
LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response.session_id);
|
||||
|
@ -36,7 +36,22 @@ public:
|
||||
/// Read state from the latest snapshot
|
||||
void init();
|
||||
|
||||
static KeeperStorage::RequestForSession parseRequest(nuraft::buffer & data);
|
||||
enum ZooKeeperLogSerializationVersion
|
||||
{
|
||||
INITIAL = 0,
|
||||
WITH_TIME = 1,
|
||||
WITH_ZXID_DIGEST = 2,
|
||||
};
|
||||
|
||||
/// lifetime of a parsed request is:
|
||||
/// [preprocess/PreAppendLog -> commit]
|
||||
/// [preprocess/PreAppendLog -> rollback]
|
||||
/// on events like commit and rollback we can remove the parsed request to keep the memory usage at minimum
|
||||
/// request cache is also cleaned on session close in case something strange happened
|
||||
///
|
||||
/// final - whether it's the final time we will fetch the request so we can safely remove it from cache
|
||||
/// serialization_version - information about which fields were parsed from the buffer so we can modify the buffer accordingly
|
||||
std::shared_ptr<KeeperStorage::RequestForSession> parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version = nullptr);
|
||||
|
||||
bool preprocess(const KeeperStorage::RequestForSession & request_for_session);
|
||||
|
||||
@ -138,6 +153,13 @@ private:
|
||||
/// for request.
|
||||
mutable std::mutex storage_and_responses_lock;
|
||||
|
||||
std::unordered_map<int64_t, std::unordered_map<Coordination::XID, std::shared_ptr<KeeperStorage::RequestForSession>>> parsed_request_cache;
|
||||
uint64_t min_request_size_to_cache{0};
|
||||
/// we only need to protect the access to the map itself
|
||||
/// requests can be modified from anywhere without lock because a single request
|
||||
/// can be processed only in 1 thread at any point
|
||||
std::mutex request_cache_mutex;
|
||||
|
||||
/// Last committed Raft log number.
|
||||
std::atomic<uint64_t> last_committed_idx;
|
||||
|
||||
|
@ -110,7 +110,7 @@ public:
|
||||
struct RequestForSession
|
||||
{
|
||||
int64_t session_id;
|
||||
int64_t time;
|
||||
int64_t time{0};
|
||||
Coordination::ZooKeeperRequestPtr request;
|
||||
int64_t zxid{0};
|
||||
std::optional<Digest> digest;
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <Coordination/WriteBufferFromNuraftBuffer.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1230,8 +1230,11 @@ public:
|
||||
/// The case when arguments are the same (tautological comparison). Return constant.
|
||||
/// NOTE: Nullable types are special case.
|
||||
/// (BTW, this function use default implementation for Nullable, so Nullable types cannot be here. Check just in case.)
|
||||
/// NOTE: We consider NaN comparison to be implementation specific (and in our implementation NaNs are sometimes equal sometimes not).
|
||||
if (left_type->equals(*right_type) && !left_type->isNullable() && !isTuple(left_type) && col_left_untyped == col_right_untyped)
|
||||
if (left_type->equals(*right_type) &&
|
||||
!left_type->isNullable() &&
|
||||
!isTuple(left_type) &&
|
||||
!WhichDataType(left_type).isFloat() &&
|
||||
col_left_untyped == col_right_untyped)
|
||||
{
|
||||
ColumnPtr result_column;
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Common/ErrorCodes.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/Priority.h>
|
||||
|
||||
#include <IO/ResourceRequest.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
@ -37,7 +38,7 @@ inline const Poco::Util::AbstractConfiguration & emptyConfig()
|
||||
struct SchedulerNodeInfo
|
||||
{
|
||||
double weight = 1.0; /// Weight of this node among it's siblings
|
||||
Int64 priority = 0; /// Priority of this node among it's siblings (higher value means higher priority)
|
||||
Priority priority; /// Priority of this node among it's siblings (lower value means higher priority)
|
||||
|
||||
/// Arbitrary data accessed/stored by parent
|
||||
union {
|
||||
@ -65,7 +66,7 @@ struct SchedulerNodeInfo
|
||||
|
||||
void setPriority(Int64 value)
|
||||
{
|
||||
priority = value;
|
||||
priority.value = value;
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -26,12 +26,12 @@ class PriorityPolicy : public ISchedulerNode
|
||||
struct Item
|
||||
{
|
||||
ISchedulerNode * child = nullptr;
|
||||
Int64 priority = 0; // higher value means higher priority
|
||||
Priority priority; // lower value means higher priority
|
||||
|
||||
/// For max-heap by priority
|
||||
bool operator<(const Item& rhs) const noexcept
|
||||
{
|
||||
return priority < rhs.priority;
|
||||
return priority > rhs.priority; // Reversed for heap top to yield highest priority (lowest value) child first
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -22,9 +22,9 @@ TEST(IOResourcePriorityPolicy, Priorities)
|
||||
ResourceTest t;
|
||||
|
||||
t.add<PriorityPolicy>("/");
|
||||
t.add<FifoQueue>("/A", "<priority>1</priority>");
|
||||
t.add<FifoQueue>("/A", "<priority>3</priority>");
|
||||
t.add<FifoQueue>("/B", "<priority>2</priority>");
|
||||
t.add<FifoQueue>("/C", "<priority>3</priority>");
|
||||
t.add<FifoQueue>("/C", "<priority>1</priority>");
|
||||
|
||||
t.enqueue("/A", {10, 10, 10});
|
||||
t.enqueue("/B", {10, 10, 10});
|
||||
@ -56,9 +56,9 @@ TEST(IOResourcePriorityPolicy, Activation)
|
||||
ResourceTest t;
|
||||
|
||||
t.add<PriorityPolicy>("/");
|
||||
t.add<FifoQueue>("/A", "<priority>1</priority>");
|
||||
t.add<FifoQueue>("/A", "<priority>3</priority>");
|
||||
t.add<FifoQueue>("/B", "<priority>2</priority>");
|
||||
t.add<FifoQueue>("/C", "<priority>3</priority>");
|
||||
t.add<FifoQueue>("/C", "<priority>1</priority>");
|
||||
|
||||
t.enqueue("/A", {10, 10, 10, 10, 10, 10});
|
||||
t.enqueue("/B", {10});
|
||||
|
@ -49,7 +49,7 @@ TEST(IOResourceStaticResourceManager, Prioritization)
|
||||
{
|
||||
// Lock is not required here because this is called during request execution and we have max_requests = 1
|
||||
if (last_priority)
|
||||
EXPECT_TRUE(priority <= *last_priority); // Should be true if every queue arrived at the same time at busy period start
|
||||
EXPECT_TRUE(priority >= *last_priority); // Should be true if every queue arrived at the same time at busy period start
|
||||
last_priority = priority;
|
||||
};
|
||||
|
||||
@ -63,8 +63,8 @@ TEST(IOResourceStaticResourceManager, Prioritization)
|
||||
<res1>
|
||||
<node path="/"> <type>inflight_limit</type><max_requests>1</max_requests></node>
|
||||
<node path="/prio"> <type>priority</type></node>
|
||||
<node path="/prio/A"> <priority>-1</priority></node>
|
||||
<node path="/prio/B"> <priority>1</priority></node>
|
||||
<node path="/prio/A"> <priority>1</priority></node>
|
||||
<node path="/prio/B"> <priority>-1</priority></node>
|
||||
<node path="/prio/C"> </node>
|
||||
<node path="/prio/D"> </node>
|
||||
<node path="/prio/leader"></node>
|
||||
|
@ -2041,7 +2041,7 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena
|
||||
*/
|
||||
if (data.hasNullKeyData())
|
||||
{
|
||||
has_null_key_data = Method::one_key_nullable_optimization;
|
||||
has_null_key_data = true;
|
||||
out_cols->key_columns[0]->insertDefault();
|
||||
insertAggregatesIntoColumns(data.getNullKeyData(), out_cols->final_aggregate_columns, arena);
|
||||
data.hasNullKeyData() = false;
|
||||
@ -2076,6 +2076,7 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena
|
||||
res.emplace_back(insertResultsIntoColumns<use_compiled_functions>(places, std::move(out_cols.value()), arena, has_null_key_data));
|
||||
places.clear();
|
||||
out_cols.reset();
|
||||
has_null_key_data = false;
|
||||
}
|
||||
}
|
||||
});
|
||||
|
@ -19,6 +19,8 @@
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
@ -68,17 +70,6 @@ void ExecuteScalarSubqueriesMatcher::visit(ASTPtr & ast, Data & data)
|
||||
visit(*t, ast, data);
|
||||
}
|
||||
|
||||
/// Converting to literal values might take a fair amount of overhead when the value is large, (e.g.
|
||||
/// Array, BitMap, etc.), This conversion is required for constant folding, index lookup, branch
|
||||
/// elimination. However, these optimizations should never be related to large values, thus we
|
||||
/// blacklist them here.
|
||||
static bool worthConvertingToLiteral(const Block & scalar)
|
||||
{
|
||||
const auto * scalar_type_name = scalar.safeGetByPosition(0).type->getFamilyName();
|
||||
static const std::set<std::string_view> useless_literal_types = {"Array", "Tuple", "AggregateFunction", "Function", "Set", "LowCardinality"};
|
||||
return !useless_literal_types.contains(scalar_type_name);
|
||||
}
|
||||
|
||||
static auto getQueryInterpreter(const ASTSubquery & subquery, ExecuteScalarSubqueriesMatcher::Data & data)
|
||||
{
|
||||
auto subquery_context = Context::createCopy(data.getContext());
|
||||
@ -255,7 +246,9 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr
|
||||
const Settings & settings = data.getContext()->getSettingsRef();
|
||||
|
||||
// Always convert to literals when there is no query context.
|
||||
if (data.only_analyze || !settings.enable_scalar_subquery_optimization || worthConvertingToLiteral(scalar)
|
||||
if (data.only_analyze
|
||||
|| !settings.enable_scalar_subquery_optimization
|
||||
|| worthConvertingScalarToLiteral(scalar, data.max_literal_size)
|
||||
|| !data.getContext()->hasQueryContext())
|
||||
{
|
||||
/// subquery and ast can be the same object and ast will be moved.
|
||||
@ -278,7 +271,7 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr
|
||||
ast = std::move(func);
|
||||
}
|
||||
}
|
||||
else
|
||||
else if (!data.replace_only_to_literals)
|
||||
{
|
||||
auto func = makeASTFunction("__getScalar", std::make_shared<ASTLiteral>(scalar_query_hash_str));
|
||||
func->alias = subquery.alias;
|
||||
@ -318,4 +311,31 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTFunction & func, ASTPtr & as
|
||||
Visitor(data).visit(*add_node);
|
||||
}
|
||||
|
||||
static size_t getSizeOfSerializedLiteral(const Field & field)
|
||||
{
|
||||
auto field_str = applyVisitor(FieldVisitorToString(), field);
|
||||
return field_str.size();
|
||||
}
|
||||
|
||||
bool worthConvertingScalarToLiteral(const Block & scalar, std::optional<size_t> max_literal_size)
|
||||
{
|
||||
/// Converting to literal values might take a fair amount of overhead when the value is large, (e.g.
|
||||
/// Array, BitMap, etc.), This conversion is required for constant folding, index lookup, branch
|
||||
/// elimination. However, these optimizations should never be related to large values, thus we blacklist them here.
|
||||
const auto * scalar_type_name = scalar.safeGetByPosition(0).type->getFamilyName();
|
||||
static const std::set<std::string_view> maybe_large_literal_types = {"Array", "Tuple", "AggregateFunction", "Function", "Set", "LowCardinality"};
|
||||
|
||||
if (!maybe_large_literal_types.contains(scalar_type_name))
|
||||
return true;
|
||||
|
||||
if (!max_literal_size)
|
||||
return false;
|
||||
|
||||
/// Size of serialized literal cannot be less than size in bytes.
|
||||
if (scalar.bytes() > *max_literal_size)
|
||||
return false;
|
||||
|
||||
return getSizeOfSerializedLiteral((*scalar.safeGetByPosition(0).column)[0]) <= *max_literal_size;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -37,6 +37,8 @@ public:
|
||||
Scalars & local_scalars;
|
||||
bool only_analyze;
|
||||
bool is_create_parameterized_view;
|
||||
bool replace_only_to_literals;
|
||||
std::optional<size_t> max_literal_size;
|
||||
};
|
||||
|
||||
static bool needChildVisit(ASTPtr & node, const ASTPtr &);
|
||||
@ -49,4 +51,6 @@ private:
|
||||
|
||||
using ExecuteScalarSubqueriesVisitor = ExecuteScalarSubqueriesMatcher::Visitor;
|
||||
|
||||
bool worthConvertingScalarToLiteral(const Block & scalar, std::optional<size_t> max_literal_size);
|
||||
|
||||
}
|
||||
|
@ -8,12 +8,14 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/FunctionNameNormalizer.h>
|
||||
#include <Interpreters/MutationsInterpreter.h>
|
||||
#include <Interpreters/MutationsNonDeterministicHelpers.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Parsers/ASTAssignment.h>
|
||||
#include <Parsers/ASTIdentifier_fwd.h>
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/LiveView/LiveViewCommands.h>
|
||||
@ -67,7 +69,6 @@ BlockIO InterpreterAlterQuery::execute()
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown alter object type");
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
|
||||
{
|
||||
BlockIO res;
|
||||
@ -156,7 +157,8 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
|
||||
if (mutation_commands.hasNonEmptyMutationCommands())
|
||||
{
|
||||
table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef());
|
||||
MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate();
|
||||
MutationsInterpreter::Settings settings(false);
|
||||
MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), settings).validate();
|
||||
table->mutate(mutation_commands, getContext());
|
||||
}
|
||||
|
||||
@ -236,6 +238,7 @@ BlockIO InterpreterAlterQuery::executeToDatabase(const ASTAlterQuery & alter)
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
AccessRightsElements InterpreterAlterQuery::getRequiredAccess() const
|
||||
{
|
||||
AccessRightsElements required_access;
|
||||
|
@ -72,7 +72,8 @@ BlockIO InterpreterDeleteQuery::execute()
|
||||
mutation_commands.emplace_back(mut_command);
|
||||
|
||||
table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef());
|
||||
MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate();
|
||||
MutationsInterpreter::Settings settings(false);
|
||||
MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), settings).validate();
|
||||
table->mutate(mutation_commands, getContext());
|
||||
return {};
|
||||
}
|
||||
|
@ -437,7 +437,10 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
if (!metadata_snapshot)
|
||||
metadata_snapshot = storage->getInMemoryMetadataPtr();
|
||||
|
||||
storage_snapshot = storage->getStorageSnapshotForQuery(metadata_snapshot, query_ptr, context);
|
||||
if (options.only_analyze)
|
||||
storage_snapshot = storage->getStorageSnapshotWithoutData(metadata_snapshot, context);
|
||||
else
|
||||
storage_snapshot = storage->getStorageSnapshotForQuery(metadata_snapshot, query_ptr, context);
|
||||
}
|
||||
|
||||
if (has_input || !joined_tables.resolveTables())
|
||||
|
@ -1,9 +1,9 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/MutationsInterpreter.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/MutationsNonDeterministicHelpers.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/StorageFromMergeTreeDataPart.h>
|
||||
#include <Storages/StorageMergeTree.h>
|
||||
@ -31,7 +31,6 @@
|
||||
#include <Interpreters/PreparedSets.h>
|
||||
#include <Storages/LightweightDeleteDescription.h>
|
||||
#include <Storages/MergeTree/MergeTreeSequentialSource.h>
|
||||
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
|
||||
#include <Processors/Sources/ThrowingExceptionSource.h>
|
||||
#include <Analyzer/QueryTreeBuilder.h>
|
||||
#include <Analyzer/QueryTreePassManager.h>
|
||||
@ -53,90 +52,12 @@ namespace ErrorCodes
|
||||
extern const int NO_SUCH_COLUMN_IN_TABLE;
|
||||
extern const int CANNOT_UPDATE_COLUMN;
|
||||
extern const int UNEXPECTED_EXPRESSION;
|
||||
extern const int THERE_IS_NO_COLUMN;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Helps to detect situations, where non-deterministic functions may be used in mutations of Replicated*MergeTree.
|
||||
class FirstNonDeterministicFunctionMatcher
|
||||
{
|
||||
public:
|
||||
struct Data
|
||||
{
|
||||
ContextPtr context;
|
||||
std::optional<String> nondeterministic_function_name;
|
||||
bool subquery = false;
|
||||
};
|
||||
|
||||
static bool needChildVisit(const ASTPtr & /*node*/, const ASTPtr & /*child*/)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
static void visit(const ASTPtr & node, Data & data)
|
||||
{
|
||||
if (data.nondeterministic_function_name || data.subquery)
|
||||
return;
|
||||
|
||||
if (node->as<ASTSelectQuery>())
|
||||
{
|
||||
/// We cannot determine if subquery is deterministic or not,
|
||||
/// so we do not allow to use subqueries in mutation without allow_nondeterministic_mutations=1
|
||||
data.subquery = true;
|
||||
}
|
||||
else if (const auto * function = typeid_cast<const ASTFunction *>(node.get()))
|
||||
{
|
||||
/// Property of being deterministic for lambda expression is completely determined
|
||||
/// by the contents of its definition, so we just proceed to it.
|
||||
if (function->name != "lambda")
|
||||
{
|
||||
/// NOTE It may be an aggregate function, so get(...) may throw.
|
||||
/// However, an aggregate function can be used only in subquery and we do not go into subquery.
|
||||
const auto func = FunctionFactory::instance().get(function->name, data.context);
|
||||
if (!func->isDeterministic())
|
||||
data.nondeterministic_function_name = func->getName();
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
using FirstNonDeterministicFunctionFinder = InDepthNodeVisitor<FirstNonDeterministicFunctionMatcher, true>;
|
||||
using FirstNonDeterministicFunctionData = FirstNonDeterministicFunctionMatcher::Data;
|
||||
|
||||
FirstNonDeterministicFunctionData findFirstNonDeterministicFunctionName(const MutationCommand & command, ContextPtr context)
|
||||
{
|
||||
FirstNonDeterministicFunctionMatcher::Data finder_data{context, std::nullopt, false};
|
||||
|
||||
switch (command.type)
|
||||
{
|
||||
case MutationCommand::UPDATE:
|
||||
{
|
||||
auto update_assignments_ast = command.ast->as<const ASTAlterCommand &>().update_assignments->clone();
|
||||
FirstNonDeterministicFunctionFinder(finder_data).visit(update_assignments_ast);
|
||||
|
||||
if (finder_data.nondeterministic_function_name)
|
||||
return finder_data;
|
||||
|
||||
/// Currently UPDATE and DELETE both always have predicates so we can use fallthrough
|
||||
[[fallthrough]];
|
||||
}
|
||||
|
||||
case MutationCommand::DELETE:
|
||||
{
|
||||
auto predicate_ast = command.predicate->clone();
|
||||
FirstNonDeterministicFunctionFinder(finder_data).visit(predicate_ast);
|
||||
|
||||
return finder_data;
|
||||
}
|
||||
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
ASTPtr prepareQueryAffectedAST(const std::vector<MutationCommand> & commands, const StoragePtr & storage, ContextPtr context)
|
||||
{
|
||||
/// Execute `SELECT count() FROM storage WHERE predicate1 OR predicate2 OR ...` query.
|
||||
@ -326,10 +247,10 @@ MutationsInterpreter::Source::Source(MergeTreeData & storage_, MergeTreeData::Da
|
||||
|
||||
StorageSnapshotPtr MutationsInterpreter::Source::getStorageSnapshot(const StorageMetadataPtr & snapshot_, const ContextPtr & context_) const
|
||||
{
|
||||
if (data)
|
||||
return data->getStorageSnapshot(snapshot_, context_);
|
||||
if (const auto * merge_tree = getMergeTreeData())
|
||||
return merge_tree->getStorageSnapshotWithoutData(snapshot_, context_);
|
||||
|
||||
return storage->getStorageSnapshot(snapshot_, context_);
|
||||
return storage->getStorageSnapshotWithoutData(snapshot_, context_);
|
||||
}
|
||||
|
||||
StoragePtr MutationsInterpreter::Source::getStorage() const
|
||||
@ -367,20 +288,27 @@ bool MutationsInterpreter::Source::materializeTTLRecalculateOnly() const
|
||||
return data && data->getSettings()->materialize_ttl_recalculate_only;
|
||||
}
|
||||
|
||||
static Names getAvailableColumnsWithVirtuals(StorageMetadataPtr metadata_snapshot, const IStorage & storage)
|
||||
{
|
||||
auto all_columns = metadata_snapshot->getColumns().getNamesOfPhysical();
|
||||
for (const auto & column : storage.getVirtuals())
|
||||
all_columns.push_back(column.name);
|
||||
return all_columns;
|
||||
}
|
||||
|
||||
MutationsInterpreter::MutationsInterpreter(
|
||||
StoragePtr storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
StorageMetadataPtr metadata_snapshot_,
|
||||
MutationCommands commands_,
|
||||
ContextPtr context_,
|
||||
bool can_execute_,
|
||||
bool return_all_columns_,
|
||||
bool return_mutated_rows_)
|
||||
Settings settings_)
|
||||
: MutationsInterpreter(
|
||||
Source(std::move(storage_)),
|
||||
metadata_snapshot_, std::move(commands_), std::move(context_),
|
||||
can_execute_, return_all_columns_, return_mutated_rows_)
|
||||
Source(storage_),
|
||||
metadata_snapshot_, std::move(commands_),
|
||||
getAvailableColumnsWithVirtuals(metadata_snapshot_, *storage_),
|
||||
std::move(context_), std::move(settings_))
|
||||
{
|
||||
if (can_execute_ && dynamic_cast<const MergeTreeData *>(source.getStorage().get()))
|
||||
if (settings.can_execute && dynamic_cast<const MergeTreeData *>(source.getStorage().get()))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
@ -392,37 +320,34 @@ MutationsInterpreter::MutationsInterpreter(
|
||||
MutationsInterpreter::MutationsInterpreter(
|
||||
MergeTreeData & storage_,
|
||||
MergeTreeData::DataPartPtr source_part_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
StorageMetadataPtr metadata_snapshot_,
|
||||
MutationCommands commands_,
|
||||
Names available_columns_,
|
||||
ContextPtr context_,
|
||||
bool can_execute_,
|
||||
bool return_all_columns_,
|
||||
bool return_mutated_rows_)
|
||||
Settings settings_)
|
||||
: MutationsInterpreter(
|
||||
Source(storage_, std::move(source_part_)),
|
||||
metadata_snapshot_, std::move(commands_), std::move(context_),
|
||||
can_execute_, return_all_columns_, return_mutated_rows_)
|
||||
std::move(metadata_snapshot_), std::move(commands_),
|
||||
std::move(available_columns_), std::move(context_), std::move(settings_))
|
||||
{
|
||||
}
|
||||
|
||||
MutationsInterpreter::MutationsInterpreter(
|
||||
Source source_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
StorageMetadataPtr metadata_snapshot_,
|
||||
MutationCommands commands_,
|
||||
Names available_columns_,
|
||||
ContextPtr context_,
|
||||
bool can_execute_,
|
||||
bool return_all_columns_,
|
||||
bool return_mutated_rows_)
|
||||
Settings settings_)
|
||||
: source(std::move(source_))
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, commands(std::move(commands_))
|
||||
, available_columns(std::move(available_columns_))
|
||||
, context(Context::createCopy(context_))
|
||||
, can_execute(can_execute_)
|
||||
, select_limits(SelectQueryOptions().analyze(!can_execute).ignoreLimits().ignoreProjections())
|
||||
, return_all_columns(return_all_columns_)
|
||||
, return_mutated_rows(return_mutated_rows_)
|
||||
, settings(std::move(settings_))
|
||||
, select_limits(SelectQueryOptions().analyze(!settings.can_execute).ignoreLimits().ignoreProjections())
|
||||
{
|
||||
prepare(!can_execute);
|
||||
prepare(!settings.can_execute);
|
||||
}
|
||||
|
||||
static NameSet getKeyColumns(const MutationsInterpreter::Source & source, const StorageMetadataPtr & metadata_snapshot)
|
||||
@ -546,16 +471,18 @@ void MutationsInterpreter::prepare(bool dry_run)
|
||||
const ColumnsDescription & columns_desc = metadata_snapshot->getColumns();
|
||||
const IndicesDescription & indices_desc = metadata_snapshot->getSecondaryIndices();
|
||||
const ProjectionsDescription & projections_desc = metadata_snapshot->getProjections();
|
||||
NamesAndTypesList all_columns = columns_desc.getAllPhysical();
|
||||
|
||||
auto storage_snapshot = std::make_shared<StorageSnapshot>(*source.getStorage(), metadata_snapshot);
|
||||
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withVirtuals();
|
||||
|
||||
auto all_columns = storage_snapshot->getColumnsByNames(options, available_columns);
|
||||
NameSet available_columns_set(available_columns.begin(), available_columns.end());
|
||||
|
||||
/// Add _row_exists column if it is physically present in the part
|
||||
if (source.hasLightweightDeleteMask())
|
||||
all_columns.push_back({LightweightDeleteDescription::FILTER_COLUMN});
|
||||
|
||||
if (return_all_columns)
|
||||
{
|
||||
for (const auto & column : source.getStorage()->getVirtuals())
|
||||
all_columns.push_back(column);
|
||||
all_columns.push_back({LightweightDeleteDescription::FILTER_COLUMN});
|
||||
available_columns_set.insert(LightweightDeleteDescription::FILTER_COLUMN.name);
|
||||
}
|
||||
|
||||
NameSet updated_columns;
|
||||
@ -567,9 +494,13 @@ void MutationsInterpreter::prepare(bool dry_run)
|
||||
|| command.type == MutationCommand::Type::DELETE)
|
||||
materialize_ttl_recalculate_only = false;
|
||||
|
||||
for (const auto & kv : command.column_to_update_expression)
|
||||
for (const auto & [name, _] : command.column_to_update_expression)
|
||||
{
|
||||
updated_columns.insert(kv.first);
|
||||
if (!available_columns_set.contains(name) && name != LightweightDeleteDescription::FILTER_COLUMN.name)
|
||||
throw Exception(ErrorCodes::THERE_IS_NO_COLUMN,
|
||||
"Column {} is updated but not requested to read", name);
|
||||
|
||||
updated_columns.insert(name);
|
||||
}
|
||||
}
|
||||
|
||||
@ -580,29 +511,28 @@ void MutationsInterpreter::prepare(bool dry_run)
|
||||
{
|
||||
for (const auto & column : columns_desc)
|
||||
{
|
||||
if (column.default_desc.kind == ColumnDefaultKind::Materialized)
|
||||
if (column.default_desc.kind == ColumnDefaultKind::Materialized && available_columns_set.contains(column.name))
|
||||
{
|
||||
auto query = column.default_desc.expression->clone();
|
||||
auto syntax_result = TreeRewriter(context).analyze(query, all_columns);
|
||||
for (const String & dependency : syntax_result->requiredSourceColumns())
|
||||
{
|
||||
for (const auto & dependency : syntax_result->requiredSourceColumns())
|
||||
if (updated_columns.contains(dependency))
|
||||
column_to_affected_materialized[dependency].push_back(column.name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
validateUpdateColumns(source, metadata_snapshot, updated_columns, column_to_affected_materialized);
|
||||
}
|
||||
|
||||
dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns);
|
||||
if (settings.recalculate_dependencies_of_updated_columns)
|
||||
dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns);
|
||||
|
||||
std::vector<String> read_columns;
|
||||
/// First, break a sequence of commands into stages.
|
||||
for (auto & command : commands)
|
||||
{
|
||||
// we can return deleted rows only if it's the only present command
|
||||
assert(command.type == MutationCommand::DELETE || command.type == MutationCommand::UPDATE || !return_mutated_rows);
|
||||
assert(command.type == MutationCommand::DELETE || command.type == MutationCommand::UPDATE || !settings.return_mutated_rows);
|
||||
|
||||
if (command.type == MutationCommand::DELETE)
|
||||
{
|
||||
@ -612,7 +542,7 @@ void MutationsInterpreter::prepare(bool dry_run)
|
||||
|
||||
auto predicate = getPartitionAndPredicateExpressionForMutationCommand(command);
|
||||
|
||||
if (!return_mutated_rows)
|
||||
if (!settings.return_mutated_rows)
|
||||
predicate = makeASTFunction("isZeroOrNull", predicate);
|
||||
|
||||
stages.back().filters.push_back(predicate);
|
||||
@ -700,7 +630,7 @@ void MutationsInterpreter::prepare(bool dry_run)
|
||||
|
||||
stages.back().column_to_updated.emplace(column, updated_column);
|
||||
|
||||
if (condition && return_mutated_rows)
|
||||
if (condition && settings.return_mutated_rows)
|
||||
stages.back().filters.push_back(condition);
|
||||
}
|
||||
|
||||
@ -909,17 +839,15 @@ void MutationsInterpreter::prepare(bool dry_run)
|
||||
}
|
||||
|
||||
is_prepared = true;
|
||||
|
||||
prepareMutationStages(stages, dry_run);
|
||||
}
|
||||
|
||||
void MutationsInterpreter::prepareMutationStages(std::vector<Stage> & prepared_stages, bool dry_run)
|
||||
{
|
||||
auto storage_snapshot = source.getStorageSnapshot(metadata_snapshot, context);
|
||||
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects();
|
||||
if (return_all_columns)
|
||||
options.withVirtuals();
|
||||
auto all_columns = storage_snapshot->getColumns(options);
|
||||
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects().withVirtuals();
|
||||
|
||||
auto all_columns = storage_snapshot->getColumnsByNames(options, available_columns);
|
||||
|
||||
/// Add _row_exists column if it is present in the part
|
||||
if (source.hasLightweightDeleteMask())
|
||||
@ -928,7 +856,7 @@ void MutationsInterpreter::prepareMutationStages(std::vector<Stage> & prepared_s
|
||||
/// Next, for each stage calculate columns changed by this and previous stages.
|
||||
for (size_t i = 0; i < prepared_stages.size(); ++i)
|
||||
{
|
||||
if (return_all_columns || !prepared_stages[i].filters.empty())
|
||||
if (settings.return_all_columns || !prepared_stages[i].filters.empty())
|
||||
{
|
||||
for (const auto & column : all_columns)
|
||||
prepared_stages[i].output_columns.insert(column.name);
|
||||
@ -1054,8 +982,7 @@ struct VirtualColumns
|
||||
{
|
||||
if (columns_to_read[i] == LightweightDeleteDescription::FILTER_COLUMN.name)
|
||||
{
|
||||
LoadedMergeTreeDataPartInfoForReader part_info_reader(part);
|
||||
if (!part_info_reader.getColumns().contains(LightweightDeleteDescription::FILTER_COLUMN.name))
|
||||
if (!part->getColumns().contains(LightweightDeleteDescription::FILTER_COLUMN.name))
|
||||
{
|
||||
ColumnWithTypeAndName mask_column;
|
||||
mask_column.type = LightweightDeleteDescription::FILTER_COLUMN.type;
|
||||
@ -1144,7 +1071,6 @@ void MutationsInterpreter::Source::read(
|
||||
ActionsDAGPtr filter;
|
||||
if (!first_stage.filter_column_names.empty())
|
||||
{
|
||||
|
||||
ActionsDAG::NodeRawConstPtrs nodes(num_filters);
|
||||
for (size_t i = 0; i < num_filters; ++i)
|
||||
nodes[i] = &steps[i]->actions()->findInOutputs(names[i]);
|
||||
@ -1155,7 +1081,9 @@ void MutationsInterpreter::Source::read(
|
||||
VirtualColumns virtual_columns(std::move(required_columns), part);
|
||||
|
||||
createMergeTreeSequentialSource(
|
||||
plan, *data, storage_snapshot, part, std::move(virtual_columns.columns_to_read), apply_deleted_mask_, filter, context_,
|
||||
plan, *data, storage_snapshot, part,
|
||||
std::move(virtual_columns.columns_to_read),
|
||||
apply_deleted_mask_, filter, context_,
|
||||
&Poco::Logger::get("MutationsInterpreter"));
|
||||
|
||||
virtual_columns.addVirtuals(plan);
|
||||
@ -1208,7 +1136,7 @@ void MutationsInterpreter::Source::read(
|
||||
|
||||
void MutationsInterpreter::initQueryPlan(Stage & first_stage, QueryPlan & plan)
|
||||
{
|
||||
source.read(first_stage, plan, metadata_snapshot, context, apply_deleted_mask, can_execute);
|
||||
source.read(first_stage, plan, metadata_snapshot, context, settings.apply_deleted_mask, settings.can_execute);
|
||||
addCreatingSetsStep(plan, first_stage.analyzer->getPreparedSets(), context);
|
||||
}
|
||||
|
||||
@ -1221,6 +1149,7 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v
|
||||
const auto & step = stage.expressions_chain.steps[i];
|
||||
if (step->actions()->hasArrayJoin())
|
||||
throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "arrayJoin is not allowed in mutations");
|
||||
|
||||
if (i < stage.filter_column_names.size())
|
||||
{
|
||||
/// Execute DELETEs.
|
||||
@ -1253,15 +1182,13 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v
|
||||
|
||||
void MutationsInterpreter::validate()
|
||||
{
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
/// For Replicated* storages mutations cannot employ non-deterministic functions
|
||||
/// because that produces inconsistencies between replicas
|
||||
if (startsWith(source.getStorage()->getName(), "Replicated") && !settings.allow_nondeterministic_mutations)
|
||||
if (startsWith(source.getStorage()->getName(), "Replicated") && !context->getSettingsRef().allow_nondeterministic_mutations)
|
||||
{
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
const auto nondeterministic_func_data = findFirstNonDeterministicFunctionName(command, context);
|
||||
const auto nondeterministic_func_data = findFirstNonDeterministicFunction(command, context);
|
||||
if (nondeterministic_func_data.subquery)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ALTER UPDATE/ALTER DELETE statement with subquery may be nondeterministic, "
|
||||
"see allow_nondeterministic_mutations setting");
|
||||
@ -1281,7 +1208,7 @@ void MutationsInterpreter::validate()
|
||||
|
||||
QueryPipelineBuilder MutationsInterpreter::execute()
|
||||
{
|
||||
if (!can_execute)
|
||||
if (!settings.can_execute)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot execute mutations interpreter because can_execute flag set to false");
|
||||
|
||||
QueryPlan plan;
|
||||
|
@ -36,30 +36,44 @@ ASTPtr getPartitionAndPredicateExpressionForMutationCommand(
|
||||
/// to this data.
|
||||
class MutationsInterpreter
|
||||
{
|
||||
private:
|
||||
struct Stage;
|
||||
|
||||
public:
|
||||
struct Settings
|
||||
{
|
||||
explicit Settings(bool can_execute_) : can_execute(can_execute_) {}
|
||||
|
||||
/// If false only analyze mutation expressions.
|
||||
bool can_execute = false;
|
||||
/// Whether all columns should be returned, not just updated
|
||||
bool return_all_columns = false;
|
||||
/// Whether we should return mutated or all existing rows
|
||||
bool return_mutated_rows = false;
|
||||
/// Where we should filter deleted rows by lightweight DELETE.
|
||||
bool apply_deleted_mask = true;
|
||||
/// Where we should recalculate skip indexes, TTL expressions, etc. that depend on updated columns.
|
||||
bool recalculate_dependencies_of_updated_columns = true;
|
||||
};
|
||||
|
||||
/// Storage to mutate, array of mutations commands and context. If you really want to execute mutation
|
||||
/// use can_execute = true, in other cases (validation, amount of commands) it can be false
|
||||
MutationsInterpreter(
|
||||
StoragePtr storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
StorageMetadataPtr metadata_snapshot_,
|
||||
MutationCommands commands_,
|
||||
ContextPtr context_,
|
||||
bool can_execute_,
|
||||
bool return_all_columns_ = false,
|
||||
bool return_mutated_rows_ = false);
|
||||
Settings settings_);
|
||||
|
||||
/// Special case for *MergeTree
|
||||
MutationsInterpreter(
|
||||
MergeTreeData & storage_,
|
||||
MergeTreeData::DataPartPtr source_part_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
StorageMetadataPtr metadata_snapshot_,
|
||||
MutationCommands commands_,
|
||||
Names available_columns_,
|
||||
ContextPtr context_,
|
||||
bool can_execute_,
|
||||
bool return_all_columns_ = false,
|
||||
bool return_mutated_rows_ = false);
|
||||
Settings settings_);
|
||||
|
||||
void validate();
|
||||
size_t evaluateCommandsSize();
|
||||
@ -93,8 +107,6 @@ public:
|
||||
|
||||
MutationKind::MutationKindEnum getMutationKind() const { return mutation_kind.mutation_kind; }
|
||||
|
||||
void setApplyDeletedMask(bool apply) { apply_deleted_mask = apply; }
|
||||
|
||||
/// Internal class which represents a data part for MergeTree
|
||||
/// or just storage for other storages.
|
||||
/// The main idea is to create a dedicated reading from MergeTree part.
|
||||
@ -131,12 +143,11 @@ public:
|
||||
private:
|
||||
MutationsInterpreter(
|
||||
Source source_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
StorageMetadataPtr metadata_snapshot_,
|
||||
MutationCommands commands_,
|
||||
Names available_columns_,
|
||||
ContextPtr context_,
|
||||
bool can_execute_,
|
||||
bool return_all_columns_,
|
||||
bool return_mutated_rows_);
|
||||
Settings settings_);
|
||||
|
||||
void prepare(bool dry_run);
|
||||
|
||||
@ -151,12 +162,11 @@ private:
|
||||
Source source;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
MutationCommands commands;
|
||||
Names available_columns;
|
||||
ContextPtr context;
|
||||
bool can_execute;
|
||||
Settings settings;
|
||||
SelectQueryOptions select_limits;
|
||||
|
||||
bool apply_deleted_mask = true;
|
||||
|
||||
/// A sequence of mutation commands is executed as a sequence of stages. Each stage consists of several
|
||||
/// filters, followed by updating values of some columns. Commands can reuse expressions calculated by the
|
||||
/// previous commands in the same stage, but at the end of each stage intermediate columns are thrown away
|
||||
@ -206,12 +216,6 @@ private:
|
||||
|
||||
/// Columns, that we need to read for calculation of skip indices, projections or TTL expressions.
|
||||
ColumnDependencies dependencies;
|
||||
|
||||
// whether all columns should be returned, not just updated
|
||||
bool return_all_columns;
|
||||
|
||||
// whether we should return mutated or all existing rows
|
||||
bool return_mutated_rows;
|
||||
};
|
||||
|
||||
}
|
||||
|
100
src/Interpreters/MutationsNonDeterministicHelpers.cpp
Normal file
100
src/Interpreters/MutationsNonDeterministicHelpers.cpp
Normal file
@ -0,0 +1,100 @@
|
||||
#include "Parsers/IAST_fwd.h"
|
||||
#include <Interpreters/MutationsNonDeterministicHelpers.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Storages/MutationCommands.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/ExecuteScalarSubqueriesVisitor.h>
|
||||
#include <Interpreters/addTypeConversionToAST.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Helps to detect situations, where non-deterministic functions may be used in mutations.
|
||||
class FirstNonDeterministicFunctionMatcher
|
||||
{
|
||||
public:
|
||||
struct Data
|
||||
{
|
||||
ContextPtr context;
|
||||
FirstNonDeterministicFunctionResult result;
|
||||
};
|
||||
|
||||
static bool needChildVisit(const ASTPtr & /*node*/, const ASTPtr & /*child*/)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
static void visit(const ASTPtr & node, Data & data)
|
||||
{
|
||||
if (data.result.nondeterministic_function_name || data.result.subquery)
|
||||
return;
|
||||
|
||||
if (node->as<ASTSelectQuery>())
|
||||
{
|
||||
/// We cannot determine if subquery is deterministic or not,
|
||||
/// so we do not allow to use subqueries in mutation without allow_nondeterministic_mutations=1
|
||||
data.result.subquery = true;
|
||||
}
|
||||
else if (const auto * function = typeid_cast<const ASTFunction *>(node.get()))
|
||||
{
|
||||
/// Property of being deterministic for lambda expression is completely determined
|
||||
/// by the contents of its definition, so we just proceed to it.
|
||||
if (function->name != "lambda")
|
||||
{
|
||||
/// NOTE It may be an aggregate function, so get(...) may throw.
|
||||
/// However, an aggregate function can be used only in subquery and we do not go into subquery.
|
||||
const auto func = FunctionFactory::instance().get(function->name, data.context);
|
||||
if (!func->isDeterministic())
|
||||
data.result.nondeterministic_function_name = func->getName();
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
using FirstNonDeterministicFunctionFinder = InDepthNodeVisitor<FirstNonDeterministicFunctionMatcher, true>;
|
||||
using FirstNonDeterministicFunctionData = FirstNonDeterministicFunctionMatcher::Data;
|
||||
|
||||
}
|
||||
|
||||
FirstNonDeterministicFunctionResult findFirstNonDeterministicFunction(const MutationCommand & command, ContextPtr context)
|
||||
{
|
||||
FirstNonDeterministicFunctionMatcher::Data finder_data{context, {}};
|
||||
|
||||
switch (command.type)
|
||||
{
|
||||
case MutationCommand::UPDATE:
|
||||
{
|
||||
auto update_assignments_ast = command.ast->as<const ASTAlterCommand &>().update_assignments->clone();
|
||||
FirstNonDeterministicFunctionFinder(finder_data).visit(update_assignments_ast);
|
||||
|
||||
if (finder_data.result.nondeterministic_function_name)
|
||||
return finder_data.result;
|
||||
|
||||
/// Currently UPDATE and DELETE both always have predicates so we can use fallthrough
|
||||
[[fallthrough]];
|
||||
}
|
||||
|
||||
case MutationCommand::DELETE:
|
||||
{
|
||||
auto predicate_ast = command.predicate->clone();
|
||||
FirstNonDeterministicFunctionFinder(finder_data).visit(predicate_ast);
|
||||
return finder_data.result;
|
||||
}
|
||||
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
}
|
21
src/Interpreters/MutationsNonDeterministicHelpers.h
Normal file
21
src/Interpreters/MutationsNonDeterministicHelpers.h
Normal file
@ -0,0 +1,21 @@
|
||||
#pragma once
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Core/Types.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct MutationCommand;
|
||||
|
||||
struct FirstNonDeterministicFunctionResult
|
||||
{
|
||||
std::optional<String> nondeterministic_function_name;
|
||||
bool subquery = false;
|
||||
};
|
||||
|
||||
/// Searches for non-deterministic functions and subqueries which
|
||||
/// may also be non-deterministic in expressions of mutation command.
|
||||
FirstNonDeterministicFunctionResult findFirstNonDeterministicFunction(const MutationCommand & command, ContextPtr context);
|
||||
|
||||
}
|
@ -455,7 +455,11 @@ void executeScalarSubqueries(
|
||||
ASTPtr & query, ContextPtr context, size_t subquery_depth, Scalars & scalars, Scalars & local_scalars, bool only_analyze, bool is_create_parameterized_view)
|
||||
{
|
||||
LogAST log;
|
||||
ExecuteScalarSubqueriesVisitor::Data visitor_data{WithContext{context}, subquery_depth, scalars, local_scalars, only_analyze, is_create_parameterized_view};
|
||||
ExecuteScalarSubqueriesVisitor::Data visitor_data{
|
||||
WithContext{context}, subquery_depth, scalars,
|
||||
local_scalars, only_analyze, is_create_parameterized_view,
|
||||
/*replace_only_to_literals=*/ false, /*max_literal_size=*/ std::nullopt};
|
||||
|
||||
ExecuteScalarSubqueriesVisitor(visitor_data, log.stream()).visit(query);
|
||||
}
|
||||
|
||||
|
@ -666,9 +666,13 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
|
||||
interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal));
|
||||
|
||||
if (context->getCurrentTransaction() && !interpreter->supportsTransactions() &&
|
||||
context->getSettingsRef().throw_on_unsupported_query_inside_transaction)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported for this type of query ({})", ast->getID());
|
||||
const auto & query_settings = context->getSettingsRef();
|
||||
if (context->getCurrentTransaction() && query_settings.throw_on_unsupported_query_inside_transaction)
|
||||
{
|
||||
if (!interpreter->supportsTransactions())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported for this type of query ({})", ast->getID());
|
||||
|
||||
}
|
||||
|
||||
if (!interpreter->ignoreQuota() && !quota_checked)
|
||||
{
|
||||
|
@ -76,7 +76,7 @@ public:
|
||||
|
||||
bool checkEndOfRow();
|
||||
bool checkForSuffixImpl(bool check_eof);
|
||||
inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(*buf); }
|
||||
inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(*buf, true); }
|
||||
|
||||
EscapingRule getEscapingRule() const override { return format_settings.custom.escaping_rule; }
|
||||
|
||||
|
@ -582,7 +582,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
|
||||
ContextPtr context = reading->getContext();
|
||||
MergeTreeDataSelectExecutor reader(reading->getMergeTreeData());
|
||||
|
||||
auto ordinary_reading_select_result = reading->selectRangesToRead(parts);
|
||||
auto ordinary_reading_select_result = reading->selectRangesToRead(parts, /* alter_conversions = */ {});
|
||||
size_t ordinary_reading_marks = ordinary_reading_select_result->marks();
|
||||
|
||||
/// Selecting best candidate.
|
||||
@ -640,7 +640,8 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
|
||||
query_info_copy.prewhere_info = nullptr;
|
||||
|
||||
projection_reading = reader.readFromParts(
|
||||
{},
|
||||
/* parts = */ {},
|
||||
/* alter_conversions = */ {},
|
||||
best_candidate->dag->getRequiredColumnsNames(),
|
||||
proj_snapshot,
|
||||
query_info_copy,
|
||||
|
@ -110,10 +110,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
|
||||
return false;
|
||||
|
||||
if (query.dag)
|
||||
{
|
||||
query.dag->removeUnusedActions();
|
||||
// LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Query DAG: {}", query.dag->dumpDAG());
|
||||
}
|
||||
}
|
||||
|
||||
std::list<NormalProjectionCandidate> candidates;
|
||||
@ -125,12 +122,9 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
|
||||
ContextPtr context = reading->getContext();
|
||||
MergeTreeDataSelectExecutor reader(reading->getMergeTreeData());
|
||||
|
||||
auto ordinary_reading_select_result = reading->selectRangesToRead(parts);
|
||||
auto ordinary_reading_select_result = reading->selectRangesToRead(parts, /* alter_conversions = */ {});
|
||||
size_t ordinary_reading_marks = ordinary_reading_select_result->marks();
|
||||
|
||||
// LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"),
|
||||
// "Marks for ordinary reading {}", ordinary_reading_marks);
|
||||
|
||||
std::shared_ptr<PartitionIdToMaxBlock> max_added_blocks = getMaxAddedBlocks(reading);
|
||||
|
||||
for (const auto * projection : normal_projections)
|
||||
@ -152,9 +146,6 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
|
||||
if (!analyzed)
|
||||
continue;
|
||||
|
||||
// LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"),
|
||||
// "Marks for projection {} {}", projection->name ,candidate.sum_marks);
|
||||
|
||||
if (candidate.sum_marks >= ordinary_reading_marks)
|
||||
continue;
|
||||
|
||||
@ -173,14 +164,12 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
|
||||
storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data);
|
||||
proj_snapshot->addProjection(best_candidate->projection);
|
||||
|
||||
// LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Proj snapshot {}",
|
||||
// proj_snapshot->getColumns(GetColumnsOptions::Kind::All).toString());
|
||||
|
||||
auto query_info_copy = query_info;
|
||||
query_info_copy.prewhere_info = nullptr;
|
||||
|
||||
auto projection_reading = reader.readFromParts(
|
||||
{},
|
||||
/*parts=*/ {},
|
||||
/*alter_conversions=*/ {},
|
||||
required_columns,
|
||||
proj_snapshot,
|
||||
query_info_copy,
|
||||
@ -201,9 +190,6 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
|
||||
if (has_ordinary_parts)
|
||||
reading->setAnalyzedResult(std::move(best_candidate->merge_tree_ordinary_select_result_ptr));
|
||||
|
||||
// LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}",
|
||||
// projection_reading->getOutputStream().header.dumpStructure());
|
||||
|
||||
projection_reading->setStepDescription(best_candidate->projection->name);
|
||||
|
||||
auto & projection_reading_node = nodes.emplace_back(QueryPlan::Node{.step = std::move(projection_reading)});
|
||||
|
@ -248,7 +248,7 @@ bool analyzeProjectionCandidate(
|
||||
|
||||
if (!normal_parts.empty())
|
||||
{
|
||||
auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts));
|
||||
auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts), /* alter_conversions = */ {});
|
||||
|
||||
if (normal_result_ptr->error())
|
||||
return false;
|
||||
|
@ -142,8 +142,10 @@ std::pair<std::vector<Values>, std::vector<RangesInDataParts>> split(RangesInDat
|
||||
{
|
||||
result_layers.back().emplace_back(
|
||||
parts[part_idx].data_part,
|
||||
parts[part_idx].alter_conversions,
|
||||
parts[part_idx].part_index_in_query,
|
||||
MarkRanges{{current_part_range_begin[part_idx], current.range.end}});
|
||||
|
||||
current_part_range_begin.erase(part_idx);
|
||||
current_part_range_end.erase(part_idx);
|
||||
continue;
|
||||
@ -170,8 +172,10 @@ std::pair<std::vector<Values>, std::vector<RangesInDataParts>> split(RangesInDat
|
||||
{
|
||||
result_layers.back().emplace_back(
|
||||
parts[part_idx].data_part,
|
||||
parts[part_idx].alter_conversions,
|
||||
parts[part_idx].part_index_in_query,
|
||||
MarkRanges{{current_part_range_begin[part_idx], last_mark + 1}});
|
||||
|
||||
current_part_range_begin[part_idx] = current_part_range_end[part_idx];
|
||||
}
|
||||
}
|
||||
|
@ -171,6 +171,7 @@ void ReadFromMergeTree::AnalysisResult::checkLimits(const Settings & settings, c
|
||||
|
||||
ReadFromMergeTree::ReadFromMergeTree(
|
||||
MergeTreeData::DataPartsVector parts_,
|
||||
std::vector<AlterConversionsPtr> alter_conversions_,
|
||||
Names real_column_names_,
|
||||
Names virt_column_names_,
|
||||
const MergeTreeData & data_,
|
||||
@ -191,6 +192,7 @@ ReadFromMergeTree::ReadFromMergeTree(
|
||||
virt_column_names_)})
|
||||
, reader_settings(getMergeTreeReaderSettings(context_, query_info_))
|
||||
, prepared_parts(std::move(parts_))
|
||||
, alter_conversions_for_parts(std::move(alter_conversions_))
|
||||
, real_column_names(std::move(real_column_names_))
|
||||
, virt_column_names(std::move(virt_column_names_))
|
||||
, data(data_)
|
||||
@ -307,8 +309,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas(
|
||||
reader_settings,
|
||||
required_columns,
|
||||
virt_column_names,
|
||||
min_marks_for_concurrent_read
|
||||
);
|
||||
min_marks_for_concurrent_read);
|
||||
|
||||
Pipes pipes;
|
||||
const auto & settings = context->getSettingsRef();
|
||||
@ -459,7 +460,7 @@ ProcessorPtr ReadFromMergeTree::createSource(
|
||||
bool set_rows_approx = !is_parallel_reading_from_replicas && !reader_settings.read_in_order;
|
||||
|
||||
auto algorithm = std::make_unique<Algorithm>(
|
||||
data, storage_snapshot, part.data_part, max_block_size, preferred_block_size_bytes,
|
||||
data, storage_snapshot, part.data_part, part.alter_conversions, max_block_size, preferred_block_size_bytes,
|
||||
preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, prewhere_info,
|
||||
actions_settings, reader_settings, pool, virt_column_names, part.part_index_in_query, has_limit_below_one_block);
|
||||
|
||||
@ -808,7 +809,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
|
||||
}
|
||||
|
||||
ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction);
|
||||
new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part));
|
||||
new_parts.emplace_back(part.data_part, part.alter_conversions, part.part_index_in_query, std::move(ranges_to_get_from_part));
|
||||
}
|
||||
|
||||
splitted_parts_and_ranges.emplace_back(std::move(new_parts));
|
||||
@ -1000,7 +1001,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
||||
{
|
||||
for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it)
|
||||
{
|
||||
new_parts.emplace_back(part_it->data_part, part_it->part_index_in_query, part_it->ranges);
|
||||
new_parts.emplace_back(part_it->data_part, part_it->alter_conversions, part_it->part_index_in_query, part_it->ranges);
|
||||
}
|
||||
}
|
||||
|
||||
@ -1111,10 +1112,13 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
||||
return Pipe::unitePipes(std::move(partition_pipes));
|
||||
}
|
||||
|
||||
MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(MergeTreeData::DataPartsVector parts) const
|
||||
MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
|
||||
MergeTreeData::DataPartsVector parts,
|
||||
std::vector<AlterConversionsPtr> alter_conversions) const
|
||||
{
|
||||
return selectRangesToRead(
|
||||
std::move(parts),
|
||||
std::move(alter_conversions),
|
||||
prewhere_info,
|
||||
filter_nodes,
|
||||
storage_snapshot->metadata,
|
||||
@ -1131,6 +1135,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge
|
||||
|
||||
MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
|
||||
MergeTreeData::DataPartsVector parts,
|
||||
std::vector<AlterConversionsPtr> alter_conversions,
|
||||
const PrewhereInfoPtr & prewhere_info,
|
||||
const ActionDAGNodes & added_filter_nodes,
|
||||
const StorageMetadataPtr & metadata_snapshot_base,
|
||||
@ -1182,7 +1187,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
|
||||
updated_query_info_with_filter_dag.filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context);
|
||||
|
||||
return selectRangesToReadImpl(
|
||||
parts,
|
||||
std::move(parts),
|
||||
std::move(alter_conversions),
|
||||
metadata_snapshot_base,
|
||||
metadata_snapshot,
|
||||
updated_query_info_with_filter_dag,
|
||||
@ -1196,7 +1202,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
|
||||
}
|
||||
|
||||
return selectRangesToReadImpl(
|
||||
parts,
|
||||
std::move(parts),
|
||||
std::move(alter_conversions),
|
||||
metadata_snapshot_base,
|
||||
metadata_snapshot,
|
||||
query_info,
|
||||
@ -1211,6 +1218,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
|
||||
|
||||
MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
|
||||
MergeTreeData::DataPartsVector parts,
|
||||
std::vector<AlterConversionsPtr> alter_conversions,
|
||||
const StorageMetadataPtr & metadata_snapshot_base,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
@ -1284,6 +1292,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
|
||||
{
|
||||
MergeTreeDataSelectExecutor::filterPartsByPartition(
|
||||
parts,
|
||||
alter_conversions,
|
||||
part_values,
|
||||
metadata_snapshot_base,
|
||||
data,
|
||||
@ -1321,6 +1330,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
|
||||
|
||||
result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes(
|
||||
std::move(parts),
|
||||
std::move(alter_conversions),
|
||||
metadata_snapshot,
|
||||
query_info,
|
||||
context,
|
||||
@ -1491,7 +1501,7 @@ bool ReadFromMergeTree::requestOutputEachPartitionThroughSeparatePort()
|
||||
|
||||
ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const
|
||||
{
|
||||
auto result_ptr = analyzed_result_ptr ? analyzed_result_ptr : selectRangesToRead(prepared_parts);
|
||||
auto result_ptr = analyzed_result_ptr ? analyzed_result_ptr : selectRangesToRead(prepared_parts, alter_conversions_for_parts);
|
||||
if (std::holds_alternative<std::exception_ptr>(result_ptr->result))
|
||||
std::rethrow_exception(std::get<std::exception_ptr>(result_ptr->result));
|
||||
|
||||
@ -1720,7 +1730,6 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
|
||||
for (const auto & processor : pipe.getProcessors())
|
||||
processors.emplace_back(processor);
|
||||
|
||||
|
||||
pipeline.init(std::move(pipe));
|
||||
// Attach QueryIdHolder if needed
|
||||
if (query_id_holder)
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPool.h>
|
||||
#include <Storages/MergeTree/AlterConversions.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -97,6 +98,7 @@ public:
|
||||
|
||||
ReadFromMergeTree(
|
||||
MergeTreeData::DataPartsVector parts_,
|
||||
std::vector<AlterConversionsPtr> alter_conversions_,
|
||||
Names real_column_names_,
|
||||
Names virt_column_names_,
|
||||
const MergeTreeData & data_,
|
||||
@ -134,6 +136,7 @@ public:
|
||||
|
||||
static MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(
|
||||
MergeTreeData::DataPartsVector parts,
|
||||
std::vector<AlterConversionsPtr> alter_conversions,
|
||||
const PrewhereInfoPtr & prewhere_info,
|
||||
const ActionDAGNodes & added_filter_nodes,
|
||||
const StorageMetadataPtr & metadata_snapshot_base,
|
||||
@ -147,7 +150,9 @@ public:
|
||||
bool sample_factor_column_queried,
|
||||
Poco::Logger * log);
|
||||
|
||||
MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts) const;
|
||||
MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(
|
||||
MergeTreeData::DataPartsVector parts,
|
||||
std::vector<AlterConversionsPtr> alter_conversions) const;
|
||||
|
||||
ContextPtr getContext() const { return context; }
|
||||
const SelectQueryInfo & getQueryInfo() const { return query_info; }
|
||||
@ -168,7 +173,12 @@ public:
|
||||
|
||||
bool hasAnalyzedResult() const { return analyzed_result_ptr != nullptr; }
|
||||
void setAnalyzedResult(MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr_) { analyzed_result_ptr = std::move(analyzed_result_ptr_); }
|
||||
void resetParts(MergeTreeData::DataPartsVector parts) { prepared_parts = std::move(parts); }
|
||||
|
||||
void resetParts(MergeTreeData::DataPartsVector parts)
|
||||
{
|
||||
prepared_parts = std::move(parts);
|
||||
alter_conversions_for_parts = {};
|
||||
}
|
||||
|
||||
const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; }
|
||||
const MergeTreeData & getMergeTreeData() const { return data; }
|
||||
@ -179,6 +189,7 @@ public:
|
||||
private:
|
||||
static MergeTreeDataSelectAnalysisResultPtr selectRangesToReadImpl(
|
||||
MergeTreeData::DataPartsVector parts,
|
||||
std::vector<AlterConversionsPtr> alter_conversions,
|
||||
const StorageMetadataPtr & metadata_snapshot_base,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
@ -202,6 +213,8 @@ private:
|
||||
MergeTreeReaderSettings reader_settings;
|
||||
|
||||
MergeTreeData::DataPartsVector prepared_parts;
|
||||
std::vector<AlterConversionsPtr> alter_conversions_for_parts;
|
||||
|
||||
Names real_column_names;
|
||||
Names virt_column_names;
|
||||
|
||||
|
@ -661,6 +661,12 @@ public:
|
||||
return getStorageSnapshot(metadata_snapshot, query_context);
|
||||
}
|
||||
|
||||
/// Creates a storage snapshot but without holding a data specific to storage.
|
||||
virtual StorageSnapshotPtr getStorageSnapshotWithoutData(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const
|
||||
{
|
||||
return getStorageSnapshot(metadata_snapshot, query_context);
|
||||
}
|
||||
|
||||
/// A helper to implement read()
|
||||
static void readFromPipe(
|
||||
QueryPlan & query_plan,
|
||||
|
@ -9,6 +9,13 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
void AlterConversions::addMutationCommand(const MutationCommand & command)
|
||||
{
|
||||
/// Currently only RENAME_COLUMN is applied on-fly.
|
||||
if (command.type == MutationCommand::Type::RENAME_COLUMN)
|
||||
rename_map.emplace_back(RenamePair{command.rename_to, command.column_name});
|
||||
}
|
||||
|
||||
bool AlterConversions::columnHasNewName(const std::string & old_name) const
|
||||
{
|
||||
for (const auto & [new_name, prev_name] : rename_map)
|
||||
@ -31,7 +38,6 @@ std::string AlterConversions::getColumnNewName(const std::string & old_name) con
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Column {} was not renamed", old_name);
|
||||
}
|
||||
|
||||
|
||||
bool AlterConversions::isColumnRenamed(const std::string & new_name) const
|
||||
{
|
||||
for (const auto & [name_to, name_from] : rename_map)
|
||||
@ -41,6 +47,7 @@ bool AlterConversions::isColumnRenamed(const std::string & new_name) const
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Get column old name before rename (lookup by key in rename_map)
|
||||
std::string AlterConversions::getColumnOldName(const std::string & new_name) const
|
||||
{
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/MutationCommands.h>
|
||||
#include <string>
|
||||
#include <unordered_map>
|
||||
|
||||
@ -7,20 +8,23 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Alter conversions which should be applied on-fly for part. Build from of
|
||||
/// the most recent mutation commands for part. Now we have only rename_map
|
||||
/// here (from ALTER_RENAME) command, because for all other type of alters
|
||||
/// we can deduce conversions for part from difference between
|
||||
/// part->getColumns() and storage->getColumns().
|
||||
struct AlterConversions
|
||||
|
||||
/// Alter conversions which should be applied on-fly for part.
|
||||
/// Built from of the most recent mutation commands for part.
|
||||
/// Now only ALTER RENAME COLUMN is applied.
|
||||
class AlterConversions : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
AlterConversions() = default;
|
||||
|
||||
struct RenamePair
|
||||
{
|
||||
std::string rename_to;
|
||||
std::string rename_from;
|
||||
};
|
||||
/// Rename map new_name -> old_name
|
||||
std::vector<RenamePair> rename_map;
|
||||
|
||||
void addMutationCommand(const MutationCommand & command);
|
||||
const std::vector<RenamePair> & getRenameMap() const { return rename_map; }
|
||||
|
||||
/// Column was renamed (lookup by value in rename_map)
|
||||
bool columnHasNewName(const std::string & old_name) const;
|
||||
@ -30,6 +34,12 @@ struct AlterConversions
|
||||
bool isColumnRenamed(const std::string & new_name) const;
|
||||
/// Get column old name before rename (lookup by key in rename_map)
|
||||
std::string getColumnOldName(const std::string & new_name) const;
|
||||
|
||||
private:
|
||||
/// Rename map new_name -> old_name.
|
||||
std::vector<RenamePair> rename_map;
|
||||
};
|
||||
|
||||
using AlterConversionsPtr = std::shared_ptr<const AlterConversions>;
|
||||
|
||||
}
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/LightweightDeleteDescription.h>
|
||||
#include <Storages/MergeTree/AlterConversions.h>
|
||||
#include <Storages/MergeTree/IDataPartStorage.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartState.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexGranularity.h>
|
||||
@ -92,6 +93,7 @@ public:
|
||||
const MarkRanges & mark_ranges,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
MarkCache * mark_cache,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const ValueSizeMap & avg_value_size_hints_,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback_) const = 0;
|
||||
|
@ -52,7 +52,7 @@ public:
|
||||
|
||||
virtual const MergeTreeDataPartChecksums & getChecksums() const = 0;
|
||||
|
||||
virtual AlterConversions getAlterConversions() const = 0;
|
||||
virtual AlterConversionsPtr getAlterConversions() const = 0;
|
||||
|
||||
virtual size_t getMarksCount() const = 0;
|
||||
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
@ -5,8 +6,6 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Interpreters/inplaceBlockConversions.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -133,9 +132,9 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns
|
||||
String IMergeTreeReader::getColumnNameInPart(const NameAndTypePair & required_column) const
|
||||
{
|
||||
auto name_in_storage = required_column.getNameInStorage();
|
||||
if (alter_conversions.isColumnRenamed(name_in_storage))
|
||||
if (alter_conversions->isColumnRenamed(name_in_storage))
|
||||
{
|
||||
name_in_storage = alter_conversions.getColumnOldName(name_in_storage);
|
||||
name_in_storage = alter_conversions->getColumnOldName(name_in_storage);
|
||||
return Nested::concatenateName(name_in_storage, required_column.getSubcolumnName());
|
||||
}
|
||||
|
||||
|
@ -50,8 +50,8 @@ public:
|
||||
/// Evaluate defaulted columns if necessary.
|
||||
void evaluateMissingDefaults(Block additional_columns, Columns & res_columns) const;
|
||||
|
||||
/// If part metadata is not equal to storage metadata, than
|
||||
/// try to perform conversions of columns.
|
||||
/// If part metadata is not equal to storage metadata,
|
||||
/// then try to perform conversions of columns.
|
||||
void performRequiredConversions(Columns & res_columns) const;
|
||||
|
||||
const NamesAndTypesList & getColumns() const { return requested_columns; }
|
||||
@ -104,7 +104,7 @@ protected:
|
||||
|
||||
private:
|
||||
/// Alter conversions, which must be applied on fly if required
|
||||
AlterConversions alter_conversions;
|
||||
AlterConversionsPtr alter_conversions;
|
||||
|
||||
/// Columns that are requested to read.
|
||||
NamesAndTypesList requested_columns;
|
||||
|
@ -1,4 +1,5 @@
|
||||
#pragma once
|
||||
#include <Storages/MergeTree/AlterConversions.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPartInfoForReader.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
|
||||
@ -9,9 +10,11 @@ namespace DB
|
||||
class LoadedMergeTreeDataPartInfoForReader final : public IMergeTreeDataPartInfoForReader
|
||||
{
|
||||
public:
|
||||
explicit LoadedMergeTreeDataPartInfoForReader(MergeTreeData::DataPartPtr data_part_)
|
||||
LoadedMergeTreeDataPartInfoForReader(
|
||||
MergeTreeData::DataPartPtr data_part_, AlterConversionsPtr alter_conversions_)
|
||||
: IMergeTreeDataPartInfoForReader(data_part_->storage.getContext())
|
||||
, data_part(data_part_)
|
||||
, data_part(std::move(data_part_))
|
||||
, alter_conversions(std::move(alter_conversions_))
|
||||
{
|
||||
}
|
||||
|
||||
@ -33,7 +36,7 @@ public:
|
||||
|
||||
std::optional<size_t> getColumnPosition(const String & column_name) const override { return data_part->getColumnPosition(column_name); }
|
||||
|
||||
AlterConversions getAlterConversions() const override { return data_part->storage.getAlterConversionsForPart(data_part); }
|
||||
AlterConversionsPtr getAlterConversions() const override { return alter_conversions; }
|
||||
|
||||
String getColumnNameWithMinimumCompressedSize(bool with_subcolumns) const override { return data_part->getColumnNameWithMinimumCompressedSize(with_subcolumns); }
|
||||
|
||||
@ -53,8 +56,11 @@ public:
|
||||
|
||||
SerializationPtr getSerialization(const NameAndTypePair & column) const override { return data_part->getSerialization(column.name); }
|
||||
|
||||
MergeTreeData::DataPartPtr getDataPart() const { return data_part; }
|
||||
|
||||
private:
|
||||
MergeTreeData::DataPartPtr data_part;
|
||||
AlterConversionsPtr alter_conversions;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -193,8 +193,9 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
|
||||
global_ctx->storage_columns = global_ctx->metadata_snapshot->getColumns().getAllPhysical();
|
||||
|
||||
auto object_columns = MergeTreeData::getConcreteObjectColumns(global_ctx->future_part->parts, global_ctx->metadata_snapshot->getColumns());
|
||||
global_ctx->storage_snapshot = std::make_shared<StorageSnapshot>(*global_ctx->data, global_ctx->metadata_snapshot, object_columns);
|
||||
|
||||
extendObjectColumns(global_ctx->storage_columns, object_columns, false);
|
||||
global_ctx->storage_snapshot = std::make_shared<StorageSnapshot>(*global_ctx->data, global_ctx->metadata_snapshot, std::move(object_columns));
|
||||
|
||||
extractMergingAndGatheringColumns(
|
||||
global_ctx->storage_columns,
|
||||
@ -544,8 +545,8 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const
|
||||
global_ctx->future_part->parts[part_num],
|
||||
column_names,
|
||||
ctx->read_with_direct_io,
|
||||
true,
|
||||
false,
|
||||
/*take_column_types_from_storage=*/ true,
|
||||
/*quiet=*/ false,
|
||||
global_ctx->input_rows_filtered);
|
||||
|
||||
pipes.emplace_back(std::move(pipe));
|
||||
@ -896,8 +897,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
|
||||
part,
|
||||
global_ctx->merging_column_names,
|
||||
ctx->read_with_direct_io,
|
||||
true,
|
||||
false,
|
||||
/*take_column_types_from_storage=*/ true,
|
||||
/*quiet=*/ false,
|
||||
global_ctx->input_rows_filtered);
|
||||
|
||||
if (global_ctx->metadata_snapshot->hasSortingKey())
|
||||
|
@ -80,49 +80,51 @@ IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm(
|
||||
result_header = header_without_const_virtual_columns;
|
||||
injectPartConstVirtualColumns(0, result_header, nullptr, partition_value_type, virt_column_names);
|
||||
|
||||
if (prewhere_actions)
|
||||
LOG_TRACE(log, "PREWHERE condition was split into {} steps: {}", prewhere_actions->steps.size(), prewhere_actions->dumpConditions());
|
||||
if (!prewhere_actions.steps.empty())
|
||||
LOG_TRACE(log, "PREWHERE condition was split into {} steps: {}", prewhere_actions.steps.size(), prewhere_actions.dumpConditions());
|
||||
|
||||
if (prewhere_info)
|
||||
LOG_TEST(log, "Original PREWHERE DAG:\n{}\nPREWHERE actions:\n{}",
|
||||
(prewhere_info->prewhere_actions ? prewhere_info->prewhere_actions->dumpDAG(): std::string("<nullptr>")),
|
||||
(prewhere_actions ? prewhere_actions->dump() : std::string("<nullptr>")));
|
||||
(!prewhere_actions.steps.empty() ? prewhere_actions.dump() : std::string("<nullptr>")));
|
||||
}
|
||||
|
||||
bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere);
|
||||
|
||||
std::unique_ptr<PrewhereExprInfo> IMergeTreeSelectAlgorithm::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps)
|
||||
PrewhereExprInfo IMergeTreeSelectAlgorithm::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps)
|
||||
{
|
||||
std::unique_ptr<PrewhereExprInfo> prewhere_actions;
|
||||
PrewhereExprInfo prewhere_actions;
|
||||
if (prewhere_info)
|
||||
{
|
||||
prewhere_actions = std::make_unique<PrewhereExprInfo>();
|
||||
|
||||
if (prewhere_info->row_level_filter)
|
||||
{
|
||||
PrewhereExprStep row_level_filter_step
|
||||
{
|
||||
.type = PrewhereExprStep::Filter,
|
||||
.actions = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter, actions_settings),
|
||||
.column_name = prewhere_info->row_level_column_name,
|
||||
.remove_column = true,
|
||||
.need_filter = true
|
||||
.filter_column_name = prewhere_info->row_level_column_name,
|
||||
.remove_filter_column = true,
|
||||
.need_filter = true,
|
||||
.perform_alter_conversions = true,
|
||||
};
|
||||
|
||||
prewhere_actions->steps.emplace_back(std::move(row_level_filter_step));
|
||||
prewhere_actions.steps.emplace_back(std::make_shared<PrewhereExprStep>(std::move(row_level_filter_step)));
|
||||
}
|
||||
|
||||
if (!enable_multiple_prewhere_read_steps ||
|
||||
!tryBuildPrewhereSteps(prewhere_info, actions_settings, *prewhere_actions))
|
||||
!tryBuildPrewhereSteps(prewhere_info, actions_settings, prewhere_actions))
|
||||
{
|
||||
PrewhereExprStep prewhere_step
|
||||
{
|
||||
.type = PrewhereExprStep::Filter,
|
||||
.actions = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions, actions_settings),
|
||||
.column_name = prewhere_info->prewhere_column_name,
|
||||
.remove_column = prewhere_info->remove_prewhere_column,
|
||||
.need_filter = prewhere_info->need_filter
|
||||
.filter_column_name = prewhere_info->prewhere_column_name,
|
||||
.remove_filter_column = prewhere_info->remove_prewhere_column,
|
||||
.need_filter = prewhere_info->need_filter,
|
||||
.perform_alter_conversions = true,
|
||||
};
|
||||
|
||||
prewhere_actions->steps.emplace_back(std::move(prewhere_step));
|
||||
prewhere_actions.steps.emplace_back(std::make_shared<PrewhereExprStep>(std::move(prewhere_step)));
|
||||
}
|
||||
}
|
||||
|
||||
@ -213,7 +215,7 @@ void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForCurrentTask(
|
||||
reader = task->data_part->getReader(
|
||||
task->task_columns.columns, metadata_snapshot, task->mark_ranges,
|
||||
owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
reader_settings, value_size_map, profile_callback);
|
||||
task->alter_conversions, reader_settings, value_size_map, profile_callback);
|
||||
}
|
||||
|
||||
if (!task->pre_reader_for_step.empty())
|
||||
@ -226,13 +228,15 @@ void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForCurrentTask(
|
||||
else
|
||||
{
|
||||
initializeMergeTreePreReadersForPart(
|
||||
task->data_part, task->task_columns, metadata_snapshot,
|
||||
task->data_part, task->alter_conversions,
|
||||
task->task_columns, metadata_snapshot,
|
||||
task->mark_ranges, value_size_map, profile_callback);
|
||||
}
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForPart(
|
||||
MergeTreeData::DataPartPtr & data_part,
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MarkRanges & mark_ranges,
|
||||
@ -242,15 +246,16 @@ void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForPart(
|
||||
reader = data_part->getReader(
|
||||
task_columns.columns, metadata_snapshot, mark_ranges,
|
||||
owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
reader_settings, value_size_map, profile_callback);
|
||||
alter_conversions, reader_settings, value_size_map, profile_callback);
|
||||
|
||||
initializeMergeTreePreReadersForPart(
|
||||
data_part, task_columns, metadata_snapshot,
|
||||
data_part, alter_conversions, task_columns, metadata_snapshot,
|
||||
mark_ranges, value_size_map, profile_callback);
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::initializeMergeTreePreReadersForPart(
|
||||
MergeTreeData::DataPartPtr & data_part,
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MarkRanges & mark_ranges,
|
||||
@ -266,36 +271,37 @@ void IMergeTreeSelectAlgorithm::initializeMergeTreePreReadersForPart(
|
||||
data_part->getReader(
|
||||
{LightweightDeleteDescription::FILTER_COLUMN}, metadata_snapshot,
|
||||
mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
reader_settings, value_size_map, profile_callback));
|
||||
alter_conversions, reader_settings, value_size_map, profile_callback));
|
||||
}
|
||||
|
||||
if (prewhere_info)
|
||||
for (const auto & pre_columns_per_step : task_columns.pre_columns)
|
||||
{
|
||||
for (const auto & pre_columns_per_step : task_columns.pre_columns)
|
||||
{
|
||||
pre_reader_for_step.push_back(
|
||||
data_part->getReader(
|
||||
pre_columns_per_step, metadata_snapshot, mark_ranges,
|
||||
owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
reader_settings, value_size_map, profile_callback));
|
||||
}
|
||||
pre_reader_for_step.push_back(
|
||||
data_part->getReader(
|
||||
pre_columns_per_step, metadata_snapshot, mark_ranges,
|
||||
owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
alter_conversions, reader_settings, value_size_map, profile_callback));
|
||||
}
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::initializeRangeReaders(MergeTreeReadTask & current_task)
|
||||
{
|
||||
return initializeRangeReadersImpl(
|
||||
current_task.range_reader, current_task.pre_range_readers, prewhere_info, prewhere_actions.get(),
|
||||
current_task.range_reader, current_task.pre_range_readers, prewhere_actions,
|
||||
reader.get(), current_task.data_part->hasLightweightDelete(), reader_settings,
|
||||
pre_reader_for_step, lightweight_delete_filter_step, non_const_virtual_column_names);
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::initializeRangeReadersImpl(
|
||||
MergeTreeRangeReader & range_reader, std::deque<MergeTreeRangeReader> & pre_range_readers,
|
||||
PrewhereInfoPtr prewhere_info, const PrewhereExprInfo * prewhere_actions,
|
||||
IMergeTreeReader * reader, bool has_lightweight_delete, const MergeTreeReaderSettings & reader_settings,
|
||||
MergeTreeRangeReader & range_reader,
|
||||
std::deque<MergeTreeRangeReader> & pre_range_readers,
|
||||
const PrewhereExprInfo & prewhere_actions,
|
||||
IMergeTreeReader * reader,
|
||||
bool has_lightweight_delete,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
const std::vector<std::unique_ptr<IMergeTreeReader>> & pre_reader_for_step,
|
||||
const PrewhereExprStep & lightweight_delete_filter_step, const Names & non_const_virtual_column_names)
|
||||
const PrewhereExprStep & lightweight_delete_filter_step,
|
||||
const Names & non_const_virtual_column_names)
|
||||
{
|
||||
MergeTreeRangeReader * prev_reader = nullptr;
|
||||
bool last_reader = false;
|
||||
@ -310,25 +316,25 @@ void IMergeTreeSelectAlgorithm::initializeRangeReadersImpl(
|
||||
pre_readers_shift++;
|
||||
}
|
||||
|
||||
if (prewhere_info)
|
||||
if (prewhere_actions.steps.size() + pre_readers_shift != pre_reader_for_step.size())
|
||||
{
|
||||
if (prewhere_actions->steps.size() + pre_readers_shift != pre_reader_for_step.size())
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"PREWHERE steps count mismatch, actions: {}, readers: {}",
|
||||
prewhere_actions->steps.size(), pre_reader_for_step.size());
|
||||
}
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"PREWHERE steps count mismatch, actions: {}, readers: {}",
|
||||
prewhere_actions.steps.size(), pre_reader_for_step.size());
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < prewhere_actions->steps.size(); ++i)
|
||||
{
|
||||
last_reader = reader->getColumns().empty() && (i + 1 == prewhere_actions->steps.size());
|
||||
for (size_t i = 0; i < prewhere_actions.steps.size(); ++i)
|
||||
{
|
||||
last_reader = reader->getColumns().empty() && (i + 1 == prewhere_actions.steps.size());
|
||||
|
||||
MergeTreeRangeReader current_reader(pre_reader_for_step[i + pre_readers_shift].get(), prev_reader, &prewhere_actions->steps[i], last_reader, non_const_virtual_column_names);
|
||||
MergeTreeRangeReader current_reader(
|
||||
pre_reader_for_step[i + pre_readers_shift].get(),
|
||||
prev_reader, prewhere_actions.steps[i].get(),
|
||||
last_reader, non_const_virtual_column_names);
|
||||
|
||||
pre_range_readers.push_back(std::move(current_reader));
|
||||
prev_reader = &pre_range_readers.back();
|
||||
}
|
||||
pre_range_readers.push_back(std::move(current_reader));
|
||||
prev_reader = &pre_range_readers.back();
|
||||
}
|
||||
|
||||
if (!last_reader)
|
||||
|
@ -74,7 +74,7 @@ public:
|
||||
|
||||
virtual std::string getName() const = 0;
|
||||
|
||||
static std::unique_ptr<PrewhereExprInfo> getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps);
|
||||
static PrewhereExprInfo getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps);
|
||||
|
||||
protected:
|
||||
/// This struct allow to return block with no columns but with non-zero number of rows similar to Chunk
|
||||
@ -110,8 +110,7 @@ protected:
|
||||
static void initializeRangeReadersImpl(
|
||||
MergeTreeRangeReader & range_reader,
|
||||
std::deque<MergeTreeRangeReader> & pre_range_readers,
|
||||
PrewhereInfoPtr prewhere_info,
|
||||
const PrewhereExprInfo * prewhere_actions,
|
||||
const PrewhereExprInfo & prewhere_actions,
|
||||
IMergeTreeReader * reader,
|
||||
bool has_lightweight_delete,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
@ -126,7 +125,8 @@ protected:
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback);
|
||||
|
||||
void initializeMergeTreeReadersForPart(
|
||||
MergeTreeData::DataPartPtr & data_part,
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MarkRanges & mark_ranges,
|
||||
@ -140,10 +140,19 @@ protected:
|
||||
StorageSnapshotPtr storage_snapshot;
|
||||
|
||||
/// This step is added when the part has lightweight delete mask
|
||||
const PrewhereExprStep lightweight_delete_filter_step { nullptr, LightweightDeleteDescription::FILTER_COLUMN.name, true, true };
|
||||
const PrewhereExprStep lightweight_delete_filter_step
|
||||
{
|
||||
.type = PrewhereExprStep::Filter,
|
||||
.actions = nullptr,
|
||||
.filter_column_name = LightweightDeleteDescription::FILTER_COLUMN.name,
|
||||
.remove_filter_column = true,
|
||||
.need_filter = true,
|
||||
.perform_alter_conversions = true,
|
||||
};
|
||||
|
||||
PrewhereInfoPtr prewhere_info;
|
||||
ExpressionActionsSettings actions_settings;
|
||||
std::unique_ptr<PrewhereExprInfo> prewhere_actions;
|
||||
PrewhereExprInfo prewhere_actions;
|
||||
|
||||
UInt64 max_block_size_rows;
|
||||
UInt64 preferred_block_size_bytes;
|
||||
@ -195,7 +204,8 @@ private:
|
||||
|
||||
/// Initialize pre readers.
|
||||
void initializeMergeTreePreReadersForPart(
|
||||
MergeTreeData::DataPartPtr & data_part,
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MarkRanges & mark_ranges,
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPartInfoForReader.h>
|
||||
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
@ -30,7 +31,7 @@ namespace
|
||||
bool injectRequiredColumnsRecursively(
|
||||
const String & column_name,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
const AlterConversions & alter_conversions,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const IMergeTreeDataPartInfoForReader & data_part_info_for_reader,
|
||||
const GetColumnsOptions & options,
|
||||
Names & columns,
|
||||
@ -46,8 +47,8 @@ bool injectRequiredColumnsRecursively(
|
||||
if (column_in_storage)
|
||||
{
|
||||
auto column_name_in_part = column_in_storage->getNameInStorage();
|
||||
if (alter_conversions.isColumnRenamed(column_name_in_part))
|
||||
column_name_in_part = alter_conversions.getColumnOldName(column_name_in_part);
|
||||
if (alter_conversions && alter_conversions->isColumnRenamed(column_name_in_part))
|
||||
column_name_in_part = alter_conversions->getColumnOldName(column_name_in_part);
|
||||
|
||||
auto column_in_part = data_part_info_for_reader.getColumns().tryGetByName(column_name_in_part);
|
||||
|
||||
@ -98,13 +99,14 @@ NameSet injectRequiredColumns(
|
||||
NameSet injected_columns;
|
||||
|
||||
bool have_at_least_one_physical_column = false;
|
||||
AlterConversions alter_conversions;
|
||||
AlterConversionsPtr alter_conversions;
|
||||
if (!data_part_info_for_reader.isProjectionPart())
|
||||
alter_conversions = data_part_info_for_reader.getAlterConversions();
|
||||
|
||||
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical)
|
||||
.withExtendedObjects()
|
||||
.withSystemColumns();
|
||||
|
||||
if (with_subcolumns)
|
||||
options.withSubcolumns();
|
||||
|
||||
@ -137,6 +139,7 @@ NameSet injectRequiredColumns(
|
||||
|
||||
MergeTreeReadTask::MergeTreeReadTask(
|
||||
const DataPartPtr & data_part_,
|
||||
const AlterConversionsPtr & alter_conversions_,
|
||||
const MarkRanges & mark_ranges_,
|
||||
size_t part_index_in_query_,
|
||||
const NameSet & column_name_set_,
|
||||
@ -146,6 +149,7 @@ MergeTreeReadTask::MergeTreeReadTask(
|
||||
std::future<MergeTreeReaderPtr> reader_,
|
||||
std::vector<std::future<MergeTreeReaderPtr>> && pre_reader_for_step_)
|
||||
: data_part{data_part_}
|
||||
, alter_conversions{alter_conversions_}
|
||||
, mark_ranges{mark_ranges_}
|
||||
, part_index_in_query{part_index_in_query_}
|
||||
, column_name_set{column_name_set_}
|
||||
@ -306,10 +310,8 @@ MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
|
||||
/// Read system columns such as lightweight delete mask "_row_exists" if it is persisted in the part
|
||||
for (const auto & name : system_columns)
|
||||
{
|
||||
if (data_part_info_for_reader.getColumns().contains(name))
|
||||
column_to_read_after_prewhere.push_back(name);
|
||||
}
|
||||
|
||||
/// Inject columns required for defaults evaluation
|
||||
injectRequiredColumns(
|
||||
@ -319,44 +321,50 @@ MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
auto options = GetColumnsOptions(GetColumnsOptions::All)
|
||||
.withExtendedObjects()
|
||||
.withSystemColumns();
|
||||
|
||||
if (with_subcolumns)
|
||||
options.withSubcolumns();
|
||||
|
||||
NameSet columns_from_previous_steps;
|
||||
auto add_step = [&](const PrewhereExprStep & step)
|
||||
{
|
||||
Names step_column_names = step.actions->getActionsDAG().getRequiredColumnsNames();
|
||||
|
||||
injectRequiredColumns(
|
||||
data_part_info_for_reader, storage_snapshot, with_subcolumns, step_column_names);
|
||||
|
||||
Names columns_to_read_in_step;
|
||||
for (const auto & name : step_column_names)
|
||||
{
|
||||
if (columns_from_previous_steps.contains(name))
|
||||
continue;
|
||||
|
||||
columns_to_read_in_step.push_back(name);
|
||||
columns_from_previous_steps.insert(name);
|
||||
}
|
||||
|
||||
result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, columns_to_read_in_step));
|
||||
};
|
||||
|
||||
if (prewhere_info)
|
||||
{
|
||||
auto prewhere_actions = IMergeTreeSelectAlgorithm::getPrewhereActions(
|
||||
prewhere_info, actions_settings, reader_settings.enable_multiple_prewhere_read_steps);
|
||||
prewhere_info,
|
||||
actions_settings,
|
||||
reader_settings.enable_multiple_prewhere_read_steps);
|
||||
|
||||
NameSet columns_from_previous_steps;
|
||||
|
||||
for (const auto & step : prewhere_actions->steps)
|
||||
{
|
||||
Names step_column_names = step.actions->getActionsDAG().getRequiredColumnsNames();
|
||||
|
||||
injectRequiredColumns(
|
||||
data_part_info_for_reader, storage_snapshot, with_subcolumns, step_column_names);
|
||||
|
||||
Names columns_to_read_in_step;
|
||||
for (const auto & name : step_column_names)
|
||||
{
|
||||
if (columns_from_previous_steps.contains(name))
|
||||
continue;
|
||||
columns_to_read_in_step.push_back(name);
|
||||
columns_from_previous_steps.insert(name);
|
||||
}
|
||||
|
||||
result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, columns_to_read_in_step));
|
||||
}
|
||||
|
||||
/// Remove columns read in prewehere from the list of columns to read
|
||||
Names post_column_names;
|
||||
for (const auto & name : column_to_read_after_prewhere)
|
||||
if (!columns_from_previous_steps.contains(name))
|
||||
post_column_names.push_back(name);
|
||||
|
||||
column_to_read_after_prewhere = std::move(post_column_names);
|
||||
for (const auto & step : prewhere_actions.steps)
|
||||
add_step(*step);
|
||||
}
|
||||
|
||||
/// Remove columns read in prewehere from the list of columns to read
|
||||
Names post_column_names;
|
||||
for (const auto & name : column_to_read_after_prewhere)
|
||||
if (!columns_from_previous_steps.contains(name))
|
||||
post_column_names.push_back(name);
|
||||
|
||||
column_to_read_after_prewhere = std::move(post_column_names);
|
||||
|
||||
/// Rest of the requested columns
|
||||
result.columns = storage_snapshot->getColumnsByNames(options, column_to_read_after_prewhere);
|
||||
return result;
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/AlterConversions.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -35,7 +36,6 @@ NameSet injectRequiredColumns(
|
||||
bool with_subcolumns,
|
||||
Names & columns);
|
||||
|
||||
|
||||
struct MergeTreeReadTaskColumns
|
||||
{
|
||||
/// column names to read during WHERE
|
||||
@ -49,8 +49,10 @@ struct MergeTreeReadTaskColumns
|
||||
/// A batch of work for MergeTreeThreadSelectProcessor
|
||||
struct MergeTreeReadTask
|
||||
{
|
||||
/// data part which should be read while performing this task
|
||||
/// Data part which should be read while performing this task
|
||||
DataPartPtr data_part;
|
||||
/// Alter converversionss that should be applied on-fly for part.
|
||||
AlterConversionsPtr alter_conversions;
|
||||
/// Ranges to read from `data_part`.
|
||||
MarkRanges mark_ranges;
|
||||
/// for virtual `part_index` virtual column
|
||||
@ -77,6 +79,7 @@ struct MergeTreeReadTask
|
||||
|
||||
MergeTreeReadTask(
|
||||
const DataPartPtr & data_part_,
|
||||
const AlterConversionsPtr & alter_conversions_,
|
||||
const MarkRanges & mark_ranges_,
|
||||
size_t part_index_in_query_,
|
||||
const NameSet & column_name_set_,
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include "Storages/MergeTree/MergeTreeDataPartBuilder.h"
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionCount.h>
|
||||
@ -76,6 +75,8 @@
|
||||
#include <Storages/StorageMergeTree.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartBuilder.h>
|
||||
#include <Storages/MutationCommands.h>
|
||||
|
||||
#include <boost/range/algorithm_ext/erase.hpp>
|
||||
#include <boost/algorithm/string/join.hpp>
|
||||
@ -7982,25 +7983,14 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S
|
||||
return true;
|
||||
}
|
||||
|
||||
AlterConversions MergeTreeData::getAlterConversionsForPart(const MergeTreeDataPartPtr part) const
|
||||
AlterConversionsPtr MergeTreeData::getAlterConversionsForPart(MergeTreeDataPartPtr part) const
|
||||
{
|
||||
std::map<int64_t, MutationCommands> commands_map = getAlterMutationCommandsForPart(part);
|
||||
auto commands_map = getAlterMutationCommandsForPart(part);
|
||||
|
||||
AlterConversions result{};
|
||||
auto & rename_map = result.rename_map;
|
||||
for (const auto & [version, commands] : commands_map)
|
||||
{
|
||||
auto result = std::make_shared<AlterConversions>();
|
||||
for (const auto & [_, commands] : commands_map)
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
/// Currently we need explicit conversions only for RENAME alter
|
||||
/// all other conversions can be deduced from diff between part columns
|
||||
/// and columns in storage.
|
||||
if (command.type == MutationCommand::Type::RENAME_COLUMN)
|
||||
{
|
||||
rename_map.emplace_back(AlterConversions::RenamePair{command.rename_to, command.column_name});
|
||||
}
|
||||
}
|
||||
}
|
||||
result->addMutationCommand(command);
|
||||
|
||||
return result;
|
||||
}
|
||||
@ -8331,13 +8321,22 @@ void MergeTreeData::updateObjectColumns(const DataPartPtr & part, const DataPart
|
||||
StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const
|
||||
{
|
||||
auto snapshot_data = std::make_unique<SnapshotData>();
|
||||
ColumnsDescription object_columns_copy;
|
||||
|
||||
auto lock = lockParts();
|
||||
snapshot_data->parts = getVisibleDataPartsVectorUnlocked(query_context, lock);
|
||||
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, object_columns, std::move(snapshot_data));
|
||||
{
|
||||
auto lock = lockParts();
|
||||
snapshot_data->parts = getVisibleDataPartsVectorUnlocked(query_context, lock);
|
||||
object_columns_copy = object_columns;
|
||||
}
|
||||
|
||||
snapshot_data->alter_conversions.reserve(snapshot_data->parts.size());
|
||||
for (const auto & part : snapshot_data->parts)
|
||||
snapshot_data->alter_conversions.push_back(getAlterConversionsForPart(part));
|
||||
|
||||
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, std::move(object_columns_copy), std::move(snapshot_data));
|
||||
}
|
||||
|
||||
StorageSnapshotPtr MergeTreeData::getStorageSnapshotWithoutParts(const StorageMetadataPtr & metadata_snapshot) const
|
||||
StorageSnapshotPtr MergeTreeData::getStorageSnapshotWithoutData(const StorageMetadataPtr & metadata_snapshot, ContextPtr) const
|
||||
{
|
||||
auto lock = lockParts();
|
||||
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, object_columns, std::make_unique<SnapshotData>());
|
||||
|
@ -443,12 +443,13 @@ public:
|
||||
struct SnapshotData : public StorageSnapshot::Data
|
||||
{
|
||||
DataPartsVector parts;
|
||||
std::vector<AlterConversionsPtr> alter_conversions;
|
||||
};
|
||||
|
||||
StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const override;
|
||||
|
||||
/// The same as above but does not hold vector of data parts.
|
||||
StorageSnapshotPtr getStorageSnapshotWithoutParts(const StorageMetadataPtr & metadata_snapshot) const;
|
||||
StorageSnapshotPtr getStorageSnapshotWithoutData(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const override;
|
||||
|
||||
/// Load the set of data parts from disk. Call once - immediately after the object is created.
|
||||
void loadDataParts(bool skip_sanity_checks);
|
||||
@ -914,7 +915,7 @@ public:
|
||||
Disks getDisks() const { return getStoragePolicy()->getDisks(); }
|
||||
|
||||
/// Return alter conversions for part which must be applied on fly.
|
||||
AlterConversions getAlterConversionsForPart(MergeTreeDataPartPtr part) const;
|
||||
AlterConversionsPtr getAlterConversionsForPart(MergeTreeDataPartPtr part) const;
|
||||
|
||||
/// Returns destination disk or volume for the TTL rule according to current storage policy.
|
||||
SpacePtr getDestinationForMoveTTL(const TTLDescription & move_ttl) const;
|
||||
|
@ -34,11 +34,12 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader(
|
||||
const MarkRanges & mark_ranges,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
MarkCache * mark_cache,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
const ValueSizeMap & avg_value_size_hints,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback) const
|
||||
{
|
||||
auto read_info = std::make_shared<LoadedMergeTreeDataPartInfoForReader>(shared_from_this());
|
||||
auto read_info = std::make_shared<LoadedMergeTreeDataPartInfoForReader>(shared_from_this(), alter_conversions);
|
||||
auto * load_marks_threadpool = reader_settings.read_settings.load_marks_asynchronously ? &read_info->getContext()->getLoadMarksThreadpool() : nullptr;
|
||||
|
||||
return std::make_unique<MergeTreeReaderCompact>(
|
||||
|
@ -34,6 +34,7 @@ public:
|
||||
const MarkRanges & mark_ranges,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
MarkCache * mark_cache,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const ValueSizeMap & avg_value_size_hints,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override;
|
||||
|
@ -36,12 +36,14 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader(
|
||||
const MarkRanges & mark_ranges,
|
||||
UncompressedCache * /* uncompressed_cache */,
|
||||
MarkCache * /* mark_cache */,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
const ValueSizeMap & /* avg_value_size_hints */,
|
||||
const ReadBufferFromFileBase::ProfileCallback & /* profile_callback */) const
|
||||
{
|
||||
auto read_info = std::make_shared<LoadedMergeTreeDataPartInfoForReader>(shared_from_this());
|
||||
auto read_info = std::make_shared<LoadedMergeTreeDataPartInfoForReader>(shared_from_this(), alter_conversions);
|
||||
auto ptr = std::static_pointer_cast<const MergeTreeDataPartInMemory>(shared_from_this());
|
||||
|
||||
return std::make_unique<MergeTreeReaderInMemory>(
|
||||
read_info, ptr, columns_to_read, metadata_snapshot, mark_ranges, reader_settings);
|
||||
}
|
||||
|
@ -23,6 +23,7 @@ public:
|
||||
const MarkRanges & mark_ranges,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
MarkCache * mark_cache,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const ValueSizeMap & avg_value_size_hints,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override;
|
||||
|
@ -33,11 +33,12 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader(
|
||||
const MarkRanges & mark_ranges,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
MarkCache * mark_cache,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
const ValueSizeMap & avg_value_size_hints,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback) const
|
||||
{
|
||||
auto read_info = std::make_shared<LoadedMergeTreeDataPartInfoForReader>(shared_from_this());
|
||||
auto read_info = std::make_shared<LoadedMergeTreeDataPartInfoForReader>(shared_from_this(), alter_conversions);
|
||||
return std::make_unique<MergeTreeReaderWide>(
|
||||
read_info, columns_to_read,
|
||||
metadata_snapshot, uncompressed_cache,
|
||||
|
@ -29,6 +29,7 @@ public:
|
||||
const MarkRanges & mark_ranges,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
MarkCache * mark_cache,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const ValueSizeMap & avg_value_size_hints,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override;
|
||||
|
@ -160,11 +160,13 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read(
|
||||
const auto & snapshot_data = assert_cast<const MergeTreeData::SnapshotData &>(*storage_snapshot->data);
|
||||
|
||||
const auto & parts = snapshot_data.parts;
|
||||
const auto & alter_conversions = snapshot_data.alter_conversions;
|
||||
|
||||
if (!query_info.projection)
|
||||
{
|
||||
auto step = readFromParts(
|
||||
query_info.merge_tree_select_result_ptr ? MergeTreeData::DataPartsVector{} : parts,
|
||||
query_info.merge_tree_select_result_ptr ? std::vector<AlterConversionsPtr>{} : alter_conversions,
|
||||
column_names_to_return,
|
||||
storage_snapshot,
|
||||
query_info,
|
||||
@ -207,7 +209,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read(
|
||||
{
|
||||
LOG_DEBUG(log, "projection required columns: {}", fmt::join(query_info.projection->required_columns, ", "));
|
||||
projection_plan->addStep(readFromParts(
|
||||
{},
|
||||
/*parts=*/ {},
|
||||
/*alter_conversions=*/ {},
|
||||
query_info.projection->required_columns,
|
||||
storage_snapshot,
|
||||
query_info,
|
||||
@ -794,6 +797,7 @@ std::optional<std::unordered_set<String>> MergeTreeDataSelectExecutor::filterPar
|
||||
|
||||
void MergeTreeDataSelectExecutor::filterPartsByPartition(
|
||||
MergeTreeData::DataPartsVector & parts,
|
||||
std::vector<AlterConversionsPtr> & alter_conversions,
|
||||
const std::optional<std::unordered_set<String>> & part_values,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeData & data,
|
||||
@ -803,6 +807,8 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition(
|
||||
Poco::Logger * log,
|
||||
ReadFromMergeTree::IndexStats & index_stats)
|
||||
{
|
||||
chassert(alter_conversions.empty() || parts.size() == alter_conversions.size());
|
||||
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
std::optional<PartitionPruner> partition_pruner;
|
||||
@ -836,6 +842,7 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition(
|
||||
if (query_context->getSettingsRef().allow_experimental_query_deduplication)
|
||||
selectPartsToReadWithUUIDFilter(
|
||||
parts,
|
||||
alter_conversions,
|
||||
part_values,
|
||||
data.getPinnedPartUUIDs(),
|
||||
minmax_idx_condition,
|
||||
@ -848,6 +855,7 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition(
|
||||
else
|
||||
selectPartsToRead(
|
||||
parts,
|
||||
alter_conversions,
|
||||
part_values,
|
||||
minmax_idx_condition,
|
||||
minmax_columns_types,
|
||||
@ -886,6 +894,7 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition(
|
||||
|
||||
RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes(
|
||||
MergeTreeData::DataPartsVector && parts,
|
||||
std::vector<AlterConversionsPtr> && alter_conversions,
|
||||
StorageMetadataPtr metadata_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
const ContextPtr & context,
|
||||
@ -896,6 +905,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
||||
ReadFromMergeTree::IndexStats & index_stats,
|
||||
bool use_skip_indexes)
|
||||
{
|
||||
chassert(alter_conversions.empty() || parts.size() == alter_conversions.size());
|
||||
|
||||
RangesInDataParts parts_with_ranges;
|
||||
parts_with_ranges.resize(parts.size());
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
@ -1003,9 +1014,11 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
||||
auto process_part = [&](size_t part_index)
|
||||
{
|
||||
auto & part = parts[part_index];
|
||||
auto alter_conversions_for_part = !alter_conversions.empty()
|
||||
? alter_conversions[part_index]
|
||||
: std::make_shared<AlterConversions>();
|
||||
|
||||
RangesInDataPart ranges(part, part_index);
|
||||
|
||||
RangesInDataPart ranges(part, alter_conversions_for_part, part_index);
|
||||
size_t total_marks_count = part->index_granularity.getMarksCountWithoutFinal();
|
||||
|
||||
if (metadata_snapshot->hasPrimaryKey())
|
||||
@ -1024,8 +1037,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
||||
break;
|
||||
|
||||
index_and_condition.stat.total_parts.fetch_add(1, std::memory_order_relaxed);
|
||||
index_and_condition.stat.total_granules.fetch_add(ranges.ranges.getNumberOfMarks(), std::memory_order_relaxed);
|
||||
|
||||
size_t total_granules = 0;
|
||||
size_t granules_dropped = 0;
|
||||
ranges.ranges = filterMarksUsingIndex(
|
||||
index_and_condition.index,
|
||||
@ -1034,15 +1047,12 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
||||
ranges.ranges,
|
||||
settings,
|
||||
reader_settings,
|
||||
total_granules,
|
||||
granules_dropped,
|
||||
mark_cache.get(),
|
||||
uncompressed_cache.get(),
|
||||
log);
|
||||
|
||||
index_and_condition.stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed);
|
||||
index_and_condition.stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed);
|
||||
|
||||
if (ranges.ranges.empty())
|
||||
index_and_condition.stat.parts_dropped.fetch_add(1, std::memory_order_relaxed);
|
||||
}
|
||||
@ -1293,6 +1303,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar
|
||||
|
||||
return ReadFromMergeTree::selectRangesToRead(
|
||||
std::move(parts),
|
||||
/*alter_conversions=*/ {},
|
||||
prewhere_info,
|
||||
added_filter_nodes,
|
||||
metadata_snapshot_base,
|
||||
@ -1309,6 +1320,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar
|
||||
|
||||
QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts(
|
||||
MergeTreeData::DataPartsVector parts,
|
||||
std::vector<AlterConversionsPtr> alter_conversions,
|
||||
const Names & column_names_to_return,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
@ -1338,6 +1350,7 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
return std::make_unique<ReadFromMergeTree>(
|
||||
std::move(parts),
|
||||
std::move(alter_conversions),
|
||||
real_column_names,
|
||||
virt_column_names,
|
||||
data,
|
||||
@ -1609,7 +1622,6 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
|
||||
const MarkRanges & ranges,
|
||||
const Settings & settings,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
size_t & total_granules,
|
||||
size_t & granules_dropped,
|
||||
MarkCache * mark_cache,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
@ -1670,8 +1682,6 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
|
||||
if (last_index_mark != index_range.begin || !granule)
|
||||
reader.seek(index_range.begin);
|
||||
|
||||
total_granules += index_range.end - index_range.begin;
|
||||
|
||||
for (size_t index_mark = index_range.begin; index_mark < index_range.end; ++index_mark)
|
||||
{
|
||||
if (index_mark != index_range.begin || !granule || last_index_mark != index_range.begin)
|
||||
@ -1833,6 +1843,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingMergedIndex(
|
||||
|
||||
void MergeTreeDataSelectExecutor::selectPartsToRead(
|
||||
MergeTreeData::DataPartsVector & parts,
|
||||
std::vector<AlterConversionsPtr> & alter_conversions,
|
||||
const std::optional<std::unordered_set<String>> & part_values,
|
||||
const std::optional<KeyCondition> & minmax_idx_condition,
|
||||
const DataTypes & minmax_columns_types,
|
||||
@ -1841,10 +1852,14 @@ void MergeTreeDataSelectExecutor::selectPartsToRead(
|
||||
PartFilterCounters & counters)
|
||||
{
|
||||
MergeTreeData::DataPartsVector prev_parts;
|
||||
std::vector<AlterConversionsPtr> prev_conversions;
|
||||
|
||||
std::swap(prev_parts, parts);
|
||||
for (const auto & part_or_projection : prev_parts)
|
||||
std::swap(prev_conversions, alter_conversions);
|
||||
|
||||
for (size_t i = 0; i < prev_parts.size(); ++i)
|
||||
{
|
||||
const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get();
|
||||
const auto * part = prev_parts[i]->isProjectionPart() ? prev_parts[i]->getParentPart() : prev_parts[i].get();
|
||||
if (part_values && part_values->find(part->name) == part_values->end())
|
||||
continue;
|
||||
|
||||
@ -1881,12 +1896,15 @@ void MergeTreeDataSelectExecutor::selectPartsToRead(
|
||||
counters.num_parts_after_partition_pruner += 1;
|
||||
counters.num_granules_after_partition_pruner += num_granules;
|
||||
|
||||
parts.push_back(part_or_projection);
|
||||
parts.push_back(prev_parts[i]);
|
||||
if (!prev_conversions.empty())
|
||||
alter_conversions.push_back(prev_conversions[i]);
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter(
|
||||
MergeTreeData::DataPartsVector & parts,
|
||||
std::vector<AlterConversionsPtr> & alter_conversions,
|
||||
const std::optional<std::unordered_set<String>> & part_values,
|
||||
MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids,
|
||||
const std::optional<KeyCondition> & minmax_idx_condition,
|
||||
@ -1899,16 +1917,22 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter(
|
||||
{
|
||||
/// process_parts prepare parts that have to be read for the query,
|
||||
/// returns false if duplicated parts' UUID have been met
|
||||
auto select_parts = [&] (MergeTreeData::DataPartsVector & selected_parts) -> bool
|
||||
auto select_parts = [&] (
|
||||
MergeTreeData::DataPartsVector & selected_parts,
|
||||
std::vector<AlterConversionsPtr> & selected_conversions) -> bool
|
||||
{
|
||||
auto ignored_part_uuids = query_context->getIgnoredPartUUIDs();
|
||||
std::unordered_set<UUID> temp_part_uuids;
|
||||
|
||||
MergeTreeData::DataPartsVector prev_parts;
|
||||
std::vector<AlterConversionsPtr> prev_conversions;
|
||||
|
||||
std::swap(prev_parts, selected_parts);
|
||||
for (const auto & part_or_projection : prev_parts)
|
||||
std::swap(prev_conversions, selected_conversions);
|
||||
|
||||
for (size_t i = 0; i < prev_parts.size(); ++i)
|
||||
{
|
||||
const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get();
|
||||
const auto * part = prev_parts[i]->isProjectionPart() ? prev_parts[i]->getParentPart() : prev_parts[i].get();
|
||||
if (part_values && part_values->find(part->name) == part_values->end())
|
||||
continue;
|
||||
|
||||
@ -1958,7 +1982,9 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter(
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Found a part with the same UUID on the same replica.");
|
||||
}
|
||||
|
||||
selected_parts.push_back(part_or_projection);
|
||||
selected_parts.push_back(prev_parts[i]);
|
||||
if (!prev_conversions.empty())
|
||||
selected_conversions.push_back(prev_conversions[i]);
|
||||
}
|
||||
|
||||
if (!temp_part_uuids.empty())
|
||||
@ -1977,7 +2003,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter(
|
||||
};
|
||||
|
||||
/// Process parts that have to be read for a query.
|
||||
auto needs_retry = !select_parts(parts);
|
||||
auto needs_retry = !select_parts(parts, alter_conversions);
|
||||
|
||||
/// If any duplicated part UUIDs met during the first step, try to ignore them in second pass.
|
||||
/// This may happen when `prefer_localhost_replica` is set and "distributed" stage runs in the same process with "remote" stage.
|
||||
@ -1988,7 +2014,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter(
|
||||
counters = PartFilterCounters();
|
||||
|
||||
/// Second attempt didn't help, throw an exception
|
||||
if (!select_parts(parts))
|
||||
if (!select_parts(parts, alter_conversions))
|
||||
throw Exception(ErrorCodes::DUPLICATED_PART_UUIDS, "Found duplicate UUIDs while processing query.");
|
||||
}
|
||||
}
|
||||
|
@ -41,6 +41,7 @@ public:
|
||||
/// The same as read, but with specified set of parts.
|
||||
QueryPlanStepPtr readFromParts(
|
||||
MergeTreeData::DataPartsVector parts,
|
||||
std::vector<AlterConversionsPtr> alter_conversions,
|
||||
const Names & column_names,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
@ -92,7 +93,6 @@ private:
|
||||
const MarkRanges & ranges,
|
||||
const Settings & settings,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
size_t & total_granules,
|
||||
size_t & granules_dropped,
|
||||
MarkCache * mark_cache,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
@ -125,6 +125,7 @@ private:
|
||||
/// as well as `max_block_number_to_read`.
|
||||
static void selectPartsToRead(
|
||||
MergeTreeData::DataPartsVector & parts,
|
||||
std::vector<AlterConversionsPtr> & alter_conversions,
|
||||
const std::optional<std::unordered_set<String>> & part_values,
|
||||
const std::optional<KeyCondition> & minmax_idx_condition,
|
||||
const DataTypes & minmax_columns_types,
|
||||
@ -135,6 +136,7 @@ private:
|
||||
/// Same as previous but also skip parts uuids if any to the query context, or skip parts which uuids marked as excluded.
|
||||
static void selectPartsToReadWithUUIDFilter(
|
||||
MergeTreeData::DataPartsVector & parts,
|
||||
std::vector<AlterConversionsPtr> & alter_conversions,
|
||||
const std::optional<std::unordered_set<String>> & part_values,
|
||||
MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids,
|
||||
const std::optional<KeyCondition> & minmax_idx_condition,
|
||||
@ -174,6 +176,7 @@ public:
|
||||
/// Filter parts using minmax index and partition key.
|
||||
static void filterPartsByPartition(
|
||||
MergeTreeData::DataPartsVector & parts,
|
||||
std::vector<AlterConversionsPtr> & alter_conversions,
|
||||
const std::optional<std::unordered_set<String>> & part_values,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeData & data,
|
||||
@ -188,6 +191,7 @@ public:
|
||||
/// If 'check_limits = true' it will throw exception if the amount of data exceed the limits from settings.
|
||||
static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes(
|
||||
MergeTreeData::DataPartsVector && parts,
|
||||
std::vector<AlterConversionsPtr> && alter_conversions,
|
||||
StorageMetadataPtr metadata_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
const ContextPtr & context,
|
||||
|
@ -55,7 +55,12 @@ try
|
||||
: getSizePredictor(data_part, task_columns, sample_block);
|
||||
|
||||
task = std::make_unique<MergeTreeReadTask>(
|
||||
data_part, mark_ranges_for_task, part_index_in_query, column_name_set, task_columns,
|
||||
data_part,
|
||||
alter_conversions,
|
||||
mark_ranges_for_task,
|
||||
part_index_in_query,
|
||||
column_name_set,
|
||||
task_columns,
|
||||
std::move(size_predictor));
|
||||
|
||||
return true;
|
||||
|
@ -21,7 +21,7 @@ std::unique_ptr<MergeTreeReaderStream> makeIndexReader(
|
||||
auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr;
|
||||
|
||||
return std::make_unique<MergeTreeReaderStream>(
|
||||
std::make_shared<LoadedMergeTreeDataPartInfoForReader>(part),
|
||||
std::make_shared<LoadedMergeTreeDataPartInfoForReader>(part, std::make_shared<AlterConversions>()),
|
||||
index->getFileName(), extension, marks_count,
|
||||
all_mark_ranges,
|
||||
std::move(settings), mark_cache, uncompressed_cache,
|
||||
|
@ -5,6 +5,8 @@
|
||||
#include <Storages/MergeTree/MarkRange.h>
|
||||
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/AlterConversions.h>
|
||||
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/ElapsedTimeProfileEventIncrement.h>
|
||||
@ -71,6 +73,7 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool(
|
||||
struct MergeTreePrefetchedReadPool::PartInfo
|
||||
{
|
||||
MergeTreeData::DataPartPtr data_part;
|
||||
AlterConversionsPtr alter_conversions;
|
||||
size_t part_index_in_query;
|
||||
size_t sum_marks = 0;
|
||||
MarkRanges ranges;
|
||||
@ -89,12 +92,13 @@ struct MergeTreePrefetchedReadPool::PartInfo
|
||||
std::future<MergeTreeReaderPtr> MergeTreePrefetchedReadPool::createPrefetchedReader(
|
||||
const IMergeTreeDataPart & data_part,
|
||||
const NamesAndTypesList & columns,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MarkRanges & required_ranges,
|
||||
Priority priority) const
|
||||
{
|
||||
auto reader = data_part.getReader(
|
||||
columns, storage_snapshot->metadata, required_ranges,
|
||||
uncompressed_cache, mark_cache, reader_settings,
|
||||
uncompressed_cache, mark_cache, alter_conversions, reader_settings,
|
||||
IMergeTreeReader::ValueSizeMap{}, profile_callback);
|
||||
|
||||
/// In order to make a prefetch we need to wait for marks to be loaded. But we just created
|
||||
@ -120,21 +124,18 @@ void MergeTreePrefetchedReadPool::createPrefetchedReaderForTask(MergeTreeReadTas
|
||||
if (task.reader.valid())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Task already has a reader");
|
||||
|
||||
task.reader = createPrefetchedReader(*task.data_part, task.task_columns.columns, task.mark_ranges, task.priority);
|
||||
task.reader = createPrefetchedReader(*task.data_part, task.task_columns.columns, task.alter_conversions, task.mark_ranges, task.priority);
|
||||
|
||||
if (reader_settings.apply_deleted_mask && task.data_part->hasLightweightDelete())
|
||||
{
|
||||
auto pre_reader = createPrefetchedReader(*task.data_part, {LightweightDeleteDescription::FILTER_COLUMN}, task.mark_ranges, task.priority);
|
||||
auto pre_reader = createPrefetchedReader(*task.data_part, {LightweightDeleteDescription::FILTER_COLUMN}, task.alter_conversions, task.mark_ranges, task.priority);
|
||||
task.pre_reader_for_step.push_back(std::move(pre_reader));
|
||||
}
|
||||
|
||||
if (prewhere_info)
|
||||
for (const auto & pre_columns_per_step : task.task_columns.pre_columns)
|
||||
{
|
||||
for (const auto & pre_columns_per_step : task.task_columns.pre_columns)
|
||||
{
|
||||
auto pre_reader = createPrefetchedReader(*task.data_part, pre_columns_per_step, task.mark_ranges, task.priority);
|
||||
task.pre_reader_for_step.push_back(std::move(pre_reader));
|
||||
}
|
||||
auto pre_reader = createPrefetchedReader(*task.data_part, pre_columns_per_step, task.alter_conversions, task.mark_ranges, task.priority);
|
||||
task.pre_reader_for_step.push_back(std::move(pre_reader));
|
||||
}
|
||||
}
|
||||
|
||||
@ -335,27 +336,28 @@ MergeTreePrefetchedReadPool::PartsInfos MergeTreePrefetchedReadPool::getPartsInf
|
||||
auto part_info = std::make_unique<PartInfo>();
|
||||
|
||||
part_info->data_part = part.data_part;
|
||||
part_info->alter_conversions = part.alter_conversions;
|
||||
part_info->part_index_in_query = part.part_index_in_query;
|
||||
part_info->ranges = part.ranges;
|
||||
std::sort(part_info->ranges.begin(), part_info->ranges.end());
|
||||
|
||||
LoadedMergeTreeDataPartInfoForReader part_reader_info(part.data_part, part_info->alter_conversions);
|
||||
|
||||
/// Sum up total size of all mark ranges in a data part.
|
||||
for (const auto & range : part.ranges)
|
||||
{
|
||||
part_info->sum_marks += range.end - range.begin;
|
||||
}
|
||||
|
||||
part_info->approx_size_of_mark = getApproxSizeOfGranule(*part_info->data_part);
|
||||
|
||||
const auto task_columns = getReadTaskColumns(
|
||||
LoadedMergeTreeDataPartInfoForReader(part.data_part),
|
||||
part_reader_info,
|
||||
storage_snapshot,
|
||||
column_names,
|
||||
virtual_column_names,
|
||||
prewhere_info,
|
||||
actions_settings,
|
||||
reader_settings,
|
||||
/* with_subcolumns */true);
|
||||
/*with_subcolumns=*/ true);
|
||||
|
||||
part_info->size_predictor = !predict_block_size_bytes
|
||||
? nullptr
|
||||
@ -571,8 +573,12 @@ MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThr
|
||||
: std::make_unique<MergeTreeBlockSizePredictor>(*part.size_predictor); /// make a copy
|
||||
|
||||
auto read_task = std::make_unique<MergeTreeReadTask>(
|
||||
part.data_part, ranges_to_get_from_part, part.part_index_in_query,
|
||||
part.column_name_set, part.task_columns,
|
||||
part.data_part,
|
||||
part.alter_conversions,
|
||||
ranges_to_get_from_part,
|
||||
part.part_index_in_query,
|
||||
part.column_name_set,
|
||||
part.task_columns,
|
||||
std::move(curr_task_size_predictor));
|
||||
|
||||
read_task->priority = priority;
|
||||
|
@ -56,6 +56,7 @@ private:
|
||||
std::future<MergeTreeReaderPtr> createPrefetchedReader(
|
||||
const IMergeTreeDataPart & data_part,
|
||||
const NamesAndTypesList & columns,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MarkRanges & required_ranges,
|
||||
Priority priority) const;
|
||||
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Columns/FilterDescription.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
@ -842,8 +843,8 @@ MergeTreeRangeReader::MergeTreeRangeReader(
|
||||
if (step.actions)
|
||||
step.actions->execute(result_sample_block, true);
|
||||
|
||||
if (step.remove_column)
|
||||
result_sample_block.erase(step.column_name);
|
||||
if (step.remove_filter_column)
|
||||
result_sample_block.erase(step.filter_column_name);
|
||||
}
|
||||
}
|
||||
|
||||
@ -981,12 +982,9 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
|
||||
/// Calculate and update read bytes
|
||||
size_t total_bytes = 0;
|
||||
for (auto & column : columns)
|
||||
{
|
||||
if (column)
|
||||
{
|
||||
total_bytes += column->byteSize();
|
||||
}
|
||||
}
|
||||
|
||||
read_result.addNumBytesRead(total_bytes);
|
||||
|
||||
if (!columns.empty())
|
||||
@ -1000,8 +998,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
|
||||
/// some columns (e.g. arrays) might be only partially filled and thus not be valid and
|
||||
/// fillMissingColumns() fixes this.
|
||||
bool should_evaluate_missing_defaults;
|
||||
merge_tree_reader->fillMissingColumns(columns, should_evaluate_missing_defaults,
|
||||
num_read_rows);
|
||||
merge_tree_reader->fillMissingColumns(columns, should_evaluate_missing_defaults, num_read_rows);
|
||||
|
||||
if (read_result.total_rows_per_granule == num_read_rows && read_result.num_rows != num_read_rows)
|
||||
{
|
||||
@ -1026,7 +1023,8 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
|
||||
}
|
||||
|
||||
/// If columns not empty, then apply on-fly alter conversions if any required
|
||||
merge_tree_reader->performRequiredConversions(columns);
|
||||
if (!prewhere_info || prewhere_info->perform_alter_conversions)
|
||||
merge_tree_reader->performRequiredConversions(columns);
|
||||
}
|
||||
|
||||
read_result.columns.reserve(read_result.columns.size() + columns.size());
|
||||
@ -1050,15 +1048,15 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
|
||||
Columns physical_columns(read_result.columns.begin(), read_result.columns.begin() + physical_columns_count);
|
||||
|
||||
bool should_evaluate_missing_defaults;
|
||||
merge_tree_reader->fillMissingColumns(physical_columns, should_evaluate_missing_defaults,
|
||||
read_result.num_rows);
|
||||
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({}, physical_columns);
|
||||
|
||||
/// If result not empty, then apply on-fly alter conversions if any required
|
||||
merge_tree_reader->performRequiredConversions(physical_columns);
|
||||
if (!prewhere_info || prewhere_info->perform_alter_conversions)
|
||||
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]);
|
||||
@ -1317,91 +1315,87 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r
|
||||
"Invalid number of columns passed to MergeTreeRangeReader. Expected {}, got {}",
|
||||
num_columns, result.columns.size());
|
||||
|
||||
/// Filter computed at the current step. Its size is equal to num_rows which is <= total_rows_per_granule
|
||||
ColumnPtr current_step_filter;
|
||||
size_t prewhere_column_pos;
|
||||
/// Restore block from columns list.
|
||||
Block block;
|
||||
size_t pos = 0;
|
||||
|
||||
if (prev_reader)
|
||||
{
|
||||
/// Restore block from columns list.
|
||||
Block block;
|
||||
size_t pos = 0;
|
||||
|
||||
if (prev_reader)
|
||||
for (const auto & col : prev_reader->getSampleBlock())
|
||||
{
|
||||
for (const auto & col : prev_reader->getSampleBlock())
|
||||
{
|
||||
block.insert({result.columns[pos], col.type, col.name});
|
||||
++pos;
|
||||
}
|
||||
block.insert({result.columns[pos], col.type, col.name});
|
||||
++pos;
|
||||
}
|
||||
|
||||
for (auto name_and_type = header.begin(); name_and_type != header.end() && pos < result.columns.size(); ++pos, ++name_and_type)
|
||||
block.insert({result.columns[pos], name_and_type->type, name_and_type->name});
|
||||
|
||||
{
|
||||
/// Columns might be projected out. We need to store them here so that default columns can be evaluated later.
|
||||
Block additional_columns = block;
|
||||
|
||||
if (prewhere_info->actions)
|
||||
{
|
||||
const String dummy_column = addDummyColumnWithRowCount(block, result.num_rows);
|
||||
|
||||
LOG_TEST(log, "Executing prewhere actions on block: {}", block.dumpStructure());
|
||||
|
||||
prewhere_info->actions->execute(block);
|
||||
|
||||
if (!dummy_column.empty())
|
||||
block.erase(dummy_column);
|
||||
}
|
||||
|
||||
result.additional_columns.clear();
|
||||
/// Additional columns might only be needed if there are more steps in the chain.
|
||||
if (!last_reader_in_chain)
|
||||
{
|
||||
for (auto & col : additional_columns)
|
||||
{
|
||||
/// Exclude columns that are present in the result block to avoid storing them and filtering twice.
|
||||
/// TODO: also need to exclude the columns that are not needed for the next steps.
|
||||
if (block.has(col.name))
|
||||
continue;
|
||||
result.additional_columns.insert(col);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
prewhere_column_pos = block.getPositionByName(prewhere_info->column_name);
|
||||
|
||||
result.columns.clear();
|
||||
result.columns.reserve(block.columns());
|
||||
for (auto & col : block)
|
||||
result.columns.emplace_back(std::move(col.column));
|
||||
|
||||
current_step_filter = result.columns[prewhere_column_pos];
|
||||
}
|
||||
|
||||
/// In case when we are returning prewhere column the caller expects it to serve as a final filter:
|
||||
/// it must contain 0s not only from the current step but also from all the previous steps.
|
||||
/// One way to achieve this is to apply the final_filter if we know that the final_filter was not applied at
|
||||
/// several previous steps but was accumulated instead.
|
||||
result.can_return_prewhere_column_without_filtering = result.filterWasApplied();
|
||||
for (auto name_and_type = header.begin(); name_and_type != header.end() && pos < result.columns.size(); ++pos, ++name_and_type)
|
||||
block.insert({result.columns[pos], name_and_type->type, name_and_type->name});
|
||||
|
||||
if (prewhere_info->remove_column)
|
||||
result.columns.erase(result.columns.begin() + prewhere_column_pos);
|
||||
|
||||
FilterWithCachedCount current_filter(current_step_filter);
|
||||
|
||||
result.optimize(current_filter, merge_tree_reader->canReadIncompleteGranules());
|
||||
|
||||
if (prewhere_info->need_filter && !result.filterWasApplied())
|
||||
{
|
||||
/// Depending on whether the final filter was applied at the previous step or not we need to apply either
|
||||
/// just the current step filter or the accumulated filter.
|
||||
FilterWithCachedCount filter_to_apply =
|
||||
current_filter.size() == result.total_rows_per_granule ?
|
||||
result.final_filter :
|
||||
current_filter;
|
||||
/// Columns might be projected out. We need to store them here so that default columns can be evaluated later.
|
||||
Block additional_columns = block;
|
||||
|
||||
if (prewhere_info->actions)
|
||||
{
|
||||
const String dummy_column = addDummyColumnWithRowCount(block, result.num_rows);
|
||||
|
||||
LOG_TEST(log, "Executing prewhere actions on block: {}", block.dumpStructure());
|
||||
|
||||
prewhere_info->actions->execute(block);
|
||||
|
||||
if (!dummy_column.empty())
|
||||
block.erase(dummy_column);
|
||||
}
|
||||
|
||||
result.additional_columns.clear();
|
||||
/// Additional columns might only be needed if there are more steps in the chain.
|
||||
if (!last_reader_in_chain)
|
||||
{
|
||||
for (auto & col : additional_columns)
|
||||
{
|
||||
/// Exclude columns that are present in the result block to avoid storing them and filtering twice.
|
||||
/// TODO: also need to exclude the columns that are not needed for the next steps.
|
||||
if (block.has(col.name))
|
||||
continue;
|
||||
result.additional_columns.insert(col);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
result.columns.clear();
|
||||
result.columns.reserve(block.columns());
|
||||
for (auto & col : block)
|
||||
result.columns.emplace_back(std::move(col.column));
|
||||
|
||||
if (prewhere_info->type == PrewhereExprStep::Filter)
|
||||
{
|
||||
/// Filter computed at the current step. Its size is equal to num_rows which is <= total_rows_per_granule
|
||||
size_t filter_column_pos = block.getPositionByName(prewhere_info->filter_column_name);
|
||||
auto current_step_filter = result.columns[filter_column_pos];
|
||||
|
||||
/// In case when we are returning prewhere column the caller expects it to serve as a final filter:
|
||||
/// it must contain 0s not only from the current step but also from all the previous steps.
|
||||
/// One way to achieve this is to apply the final_filter if we know that the final_filter was not applied at
|
||||
/// several previous steps but was accumulated instead.
|
||||
result.can_return_prewhere_column_without_filtering = result.filterWasApplied();
|
||||
|
||||
if (prewhere_info->remove_filter_column)
|
||||
result.columns.erase(result.columns.begin() + filter_column_pos);
|
||||
|
||||
FilterWithCachedCount current_filter(current_step_filter);
|
||||
result.optimize(current_filter, merge_tree_reader->canReadIncompleteGranules());
|
||||
|
||||
if (prewhere_info->need_filter && !result.filterWasApplied())
|
||||
{
|
||||
/// Depending on whether the final filter was applied at the previous step or not we need to apply either
|
||||
/// just the current step filter or the accumulated filter.
|
||||
FilterWithCachedCount filter_to_apply =
|
||||
current_filter.size() == result.total_rows_per_granule
|
||||
? result.final_filter
|
||||
: current_filter;
|
||||
|
||||
result.applyFilter(filter_to_apply);
|
||||
}
|
||||
}
|
||||
|
||||
LOG_TEST(log, "After execute prewhere {}", result.dumpInfo());
|
||||
@ -1415,12 +1409,12 @@ std::string PrewhereExprInfo::dump() const
|
||||
for (size_t i = 0; i < steps.size(); ++i)
|
||||
{
|
||||
s << "STEP " << i << ":\n"
|
||||
<< " ACTIONS: " << (steps[i].actions ?
|
||||
(indent + boost::replace_all_copy(steps[i].actions->dumpActions(), "\n", indent)) :
|
||||
<< " ACTIONS: " << (steps[i]->actions ?
|
||||
(indent + boost::replace_all_copy(steps[i]->actions->dumpActions(), "\n", indent)) :
|
||||
"nullptr") << "\n"
|
||||
<< " COLUMN: " << steps[i].column_name << "\n"
|
||||
<< " REMOVE_COLUMN: " << steps[i].remove_column << "\n"
|
||||
<< " NEED_FILTER: " << steps[i].need_filter << "\n\n";
|
||||
<< " COLUMN: " << steps[i]->filter_column_name << "\n"
|
||||
<< " REMOVE_COLUMN: " << steps[i]->remove_filter_column << "\n"
|
||||
<< " NEED_FILTER: " << steps[i]->need_filter << "\n\n";
|
||||
}
|
||||
|
||||
return s.str();
|
||||
@ -1431,7 +1425,7 @@ std::string PrewhereExprInfo::dumpConditions() const
|
||||
WriteBufferFromOwnString s;
|
||||
|
||||
for (size_t i = 0; i < steps.size(); ++i)
|
||||
s << (i == 0 ? "\"" : ", \"") << steps[i].column_name << "\"";
|
||||
s << (i == 0 ? "\"" : ", \"") << steps[i]->filter_column_name << "\"";
|
||||
|
||||
return s.str();
|
||||
}
|
||||
|
@ -22,19 +22,33 @@ using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
|
||||
struct PrewhereExprStep
|
||||
{
|
||||
enum Type
|
||||
{
|
||||
Filter,
|
||||
Expression,
|
||||
};
|
||||
|
||||
Type type = Type::Filter;
|
||||
ExpressionActionsPtr actions;
|
||||
String column_name;
|
||||
bool remove_column = false;
|
||||
String filter_column_name;
|
||||
|
||||
bool remove_filter_column = false;
|
||||
bool need_filter = false;
|
||||
|
||||
/// Some PREWHERE steps should be executed without conversions.
|
||||
/// A step without alter conversion cannot be executed after step with alter conversions.
|
||||
bool perform_alter_conversions = false;
|
||||
};
|
||||
|
||||
using PrewhereExprStepPtr = std::shared_ptr<PrewhereExprStep>;
|
||||
using PrewhereExprSteps = std::vector<PrewhereExprStepPtr>;
|
||||
|
||||
/// The same as PrewhereInfo, but with ExpressionActions instead of ActionsDAG
|
||||
struct PrewhereExprInfo
|
||||
{
|
||||
std::vector<PrewhereExprStep> steps;
|
||||
PrewhereExprSteps steps;
|
||||
|
||||
std::string dump() const;
|
||||
|
||||
std::string dumpConditions() const;
|
||||
};
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeReadPool.h>
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/logger_useful.h>
|
||||
@ -92,16 +93,18 @@ std::vector<size_t> MergeTreeReadPool::fillPerPartInfo(
|
||||
|
||||
per_part_sum_marks.push_back(sum_marks);
|
||||
|
||||
auto & per_part = per_part_params.emplace_back();
|
||||
per_part.data_part = part;
|
||||
|
||||
LoadedMergeTreeDataPartInfoForReader part_info(part.data_part, part.alter_conversions);
|
||||
auto task_columns = getReadTaskColumns(
|
||||
LoadedMergeTreeDataPartInfoForReader(part.data_part), storage_snapshot,
|
||||
column_names, virtual_column_names, prewhere_info, actions_settings, reader_settings, /*with_subcolumns=*/ true);
|
||||
part_info, storage_snapshot, column_names, virtual_column_names,
|
||||
prewhere_info, actions_settings,
|
||||
reader_settings, /*with_subcolumns=*/ true);
|
||||
|
||||
auto size_predictor = !predict_block_size_bytes ? nullptr
|
||||
: IMergeTreeSelectAlgorithm::getSizePredictor(part.data_part, task_columns, sample_block);
|
||||
|
||||
auto & per_part = per_part_params.emplace_back();
|
||||
|
||||
per_part.data_part = part;
|
||||
per_part.size_predictor = std::move(size_predictor);
|
||||
|
||||
/// will be used to distinguish between PREWHERE and WHERE columns when applying filter
|
||||
@ -202,13 +205,16 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t thread)
|
||||
}
|
||||
|
||||
const auto & per_part = per_part_params[part_idx];
|
||||
|
||||
auto curr_task_size_predictor = !per_part.size_predictor ? nullptr
|
||||
: std::make_unique<MergeTreeBlockSizePredictor>(*per_part.size_predictor); /// make a copy
|
||||
|
||||
return std::make_unique<MergeTreeReadTask>(
|
||||
part.data_part, ranges_to_get_from_part, part.part_index_in_query,
|
||||
per_part.column_name_set, per_part.task_columns,
|
||||
part.data_part,
|
||||
part.alter_conversions,
|
||||
ranges_to_get_from_part,
|
||||
part.part_index_in_query,
|
||||
per_part.column_name_set,
|
||||
per_part.task_columns,
|
||||
std::move(curr_task_size_predictor));
|
||||
}
|
||||
|
||||
@ -456,6 +462,7 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t thread)
|
||||
|
||||
return std::make_unique<MergeTreeReadTask>(
|
||||
part.data_part,
|
||||
part.alter_conversions,
|
||||
ranges_to_read,
|
||||
part.part_index_in_query,
|
||||
per_part.column_name_set,
|
||||
@ -515,5 +522,4 @@ MarkRanges MergeTreeInOrderReadPoolParallelReplicas::getNewTask(RangesInDataPart
|
||||
return {};
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -8,6 +8,8 @@
|
||||
#include <Storages/MergeTree/RequestResponse.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReadPool.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/MergeTree/AlterConversions.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
#include <mutex>
|
||||
|
||||
@ -164,7 +166,6 @@ private:
|
||||
class MergeTreeReadPoolParallelReplicas : public IMergeTreeReadPool
|
||||
{
|
||||
public:
|
||||
|
||||
MergeTreeReadPoolParallelReplicas(
|
||||
StorageSnapshotPtr storage_snapshot_,
|
||||
size_t threads_,
|
||||
|
@ -48,8 +48,13 @@ bool MergeTreeReverseSelectAlgorithm::getNewTaskOrdinaryReading()
|
||||
: getSizePredictor(data_part, task_columns, sample_block);
|
||||
|
||||
task = std::make_unique<MergeTreeReadTask>(
|
||||
data_part, mark_ranges_for_task, part_index_in_query, column_name_set,
|
||||
task_columns, std::move(size_predictor));
|
||||
data_part,
|
||||
alter_conversions,
|
||||
mark_ranges_for_task,
|
||||
part_index_in_query,
|
||||
column_name_set,
|
||||
task_columns,
|
||||
std::move(size_predictor));
|
||||
|
||||
return true;
|
||||
|
||||
@ -86,8 +91,13 @@ bool MergeTreeReverseSelectAlgorithm::getNewTaskParallelReplicas()
|
||||
: getSizePredictor(data_part, task_columns, sample_block);
|
||||
|
||||
task = std::make_unique<MergeTreeReadTask>(
|
||||
data_part, mark_ranges_for_task, part_index_in_query, column_name_set,
|
||||
task_columns, std::move(size_predictor));
|
||||
data_part,
|
||||
alter_conversions,
|
||||
mark_ranges_for_task,
|
||||
part_index_in_query,
|
||||
column_name_set,
|
||||
task_columns,
|
||||
std::move(size_predictor));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
@ -12,6 +12,7 @@ MergeTreeSelectAlgorithm::MergeTreeSelectAlgorithm(
|
||||
const MergeTreeData & storage_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const MergeTreeData::DataPartPtr & owned_data_part_,
|
||||
const AlterConversionsPtr & alter_conversions_,
|
||||
UInt64 max_block_size_rows_,
|
||||
size_t preferred_block_size_bytes_,
|
||||
size_t preferred_max_column_in_block_size_bytes_,
|
||||
@ -32,6 +33,7 @@ MergeTreeSelectAlgorithm::MergeTreeSelectAlgorithm(
|
||||
reader_settings_, use_uncompressed_cache_, virt_column_names_},
|
||||
required_columns{std::move(required_columns_)},
|
||||
data_part{owned_data_part_},
|
||||
alter_conversions(alter_conversions_),
|
||||
sample_block(storage_snapshot_->metadata->getSampleBlock()),
|
||||
all_mark_ranges(std::move(mark_ranges_)),
|
||||
part_index_in_query(part_index_in_query_),
|
||||
@ -44,9 +46,13 @@ MergeTreeSelectAlgorithm::MergeTreeSelectAlgorithm(
|
||||
|
||||
void MergeTreeSelectAlgorithm::initializeReaders()
|
||||
{
|
||||
LoadedMergeTreeDataPartInfoForReader part_info(data_part, alter_conversions);
|
||||
|
||||
task_columns = getReadTaskColumns(
|
||||
LoadedMergeTreeDataPartInfoForReader(data_part), storage_snapshot,
|
||||
required_columns, virt_column_names, prewhere_info, actions_settings, reader_settings, /*with_subcolumns=*/ true);
|
||||
part_info, storage_snapshot,
|
||||
required_columns, virt_column_names,
|
||||
prewhere_info,
|
||||
actions_settings, reader_settings, /*with_subcolumns=*/ true);
|
||||
|
||||
/// Will be used to distinguish between PREWHERE and WHERE columns when applying filter
|
||||
const auto & column_names = task_columns.columns.getNames();
|
||||
@ -58,7 +64,8 @@ void MergeTreeSelectAlgorithm::initializeReaders()
|
||||
owned_mark_cache = storage.getContext()->getMarkCache();
|
||||
|
||||
initializeMergeTreeReadersForPart(
|
||||
data_part, task_columns, storage_snapshot->getMetadataForQuery(), all_mark_ranges, {}, {});
|
||||
data_part, alter_conversions, task_columns,
|
||||
storage_snapshot->getMetadataForQuery(), all_mark_ranges, {}, {});
|
||||
}
|
||||
|
||||
|
||||
|
@ -20,7 +20,8 @@ public:
|
||||
MergeTreeSelectAlgorithm(
|
||||
const MergeTreeData & storage,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const MergeTreeData::DataPartPtr & owned_data_part,
|
||||
const MergeTreeData::DataPartPtr & owned_data_part_,
|
||||
const AlterConversionsPtr & alter_conversions_,
|
||||
UInt64 max_block_size_rows,
|
||||
size_t preferred_block_size_bytes,
|
||||
size_t preferred_max_column_in_block_size_bytes,
|
||||
@ -54,6 +55,9 @@ protected:
|
||||
/// Data part will not be removed if the pointer owns it
|
||||
MergeTreeData::DataPartPtr data_part;
|
||||
|
||||
/// Alter converversionss that should be applied on-fly for part.
|
||||
AlterConversionsPtr alter_conversions;
|
||||
|
||||
/// Cache getSampleBlock call, which might be heavy.
|
||||
Block sample_block;
|
||||
|
||||
|
@ -111,12 +111,14 @@ MergeTreeSequentialSource::MergeTreeSequentialSource(
|
||||
data_part->getMarksCount(), data_part->name, data_part->rows_count);
|
||||
}
|
||||
|
||||
auto alter_conversions = storage.getAlterConversionsForPart(data_part);
|
||||
|
||||
/// Note, that we don't check setting collaborate_with_coordinator presence, because this source
|
||||
/// is only used in background merges.
|
||||
addTotalRowsApprox(data_part->rows_count);
|
||||
|
||||
/// Add columns because we don't want to read empty blocks
|
||||
injectRequiredColumns(LoadedMergeTreeDataPartInfoForReader(data_part), storage_snapshot, /*with_subcolumns=*/ false, columns_to_read);
|
||||
injectRequiredColumns(LoadedMergeTreeDataPartInfoForReader(data_part, alter_conversions), storage_snapshot, /*with_subcolumns=*/ false, columns_to_read);
|
||||
|
||||
NamesAndTypesList columns_for_reader;
|
||||
if (take_column_types_from_storage)
|
||||
@ -146,9 +148,10 @@ MergeTreeSequentialSource::MergeTreeSequentialSource(
|
||||
if (!mark_ranges)
|
||||
mark_ranges.emplace(MarkRanges{MarkRange(0, data_part->getMarksCount())});
|
||||
|
||||
reader = data_part->getReader(columns_for_reader, storage_snapshot->metadata,
|
||||
*mark_ranges,
|
||||
/* uncompressed_cache = */ nullptr, mark_cache.get(), reader_settings, {}, {});
|
||||
reader = data_part->getReader(
|
||||
columns_for_reader, storage_snapshot->metadata,
|
||||
*mark_ranges, /* uncompressed_cache = */ nullptr,
|
||||
mark_cache.get(), alter_conversions, reader_settings, {}, {});
|
||||
}
|
||||
|
||||
Chunk MergeTreeSequentialSource::generate()
|
||||
@ -241,10 +244,9 @@ Pipe createMergeTreeSequentialSource(
|
||||
if (need_to_filter_deleted_rows)
|
||||
columns.emplace_back(LightweightDeleteDescription::FILTER_COLUMN.name);
|
||||
|
||||
bool apply_deleted_mask = false;
|
||||
|
||||
auto column_part_source = std::make_shared<MergeTreeSequentialSource>(
|
||||
storage, storage_snapshot, data_part, columns, std::optional<MarkRanges>{}, apply_deleted_mask, read_with_direct_io, take_column_types_from_storage, quiet);
|
||||
storage, storage_snapshot, data_part, columns, std::optional<MarkRanges>{},
|
||||
/*apply_deleted_mask=*/ false, read_with_direct_io, take_column_types_from_storage, quiet);
|
||||
|
||||
Pipe pipe(std::move(column_part_source));
|
||||
|
||||
@ -316,7 +318,8 @@ public:
|
||||
}
|
||||
|
||||
auto source = std::make_unique<MergeTreeSequentialSource>(
|
||||
storage, storage_snapshot, data_part, columns_to_read, std::move(mark_ranges), apply_deleted_mask, false, true);
|
||||
storage, storage_snapshot, data_part, columns_to_read,
|
||||
std::move(mark_ranges), apply_deleted_mask, false, true);
|
||||
|
||||
pipeline.init(Pipe(std::move(source)));
|
||||
}
|
||||
@ -344,7 +347,9 @@ void createMergeTreeSequentialSource(
|
||||
Poco::Logger * log)
|
||||
{
|
||||
auto reading = std::make_unique<ReadFromPart>(
|
||||
storage, storage_snapshot, std::move(data_part), std::move(columns_to_read), apply_deleted_mask, filter, std::move(context), log);
|
||||
storage, storage_snapshot, std::move(data_part),
|
||||
std::move(columns_to_read), apply_deleted_mask,
|
||||
filter, std::move(context), log);
|
||||
|
||||
plan.addStep(std::move(reading));
|
||||
}
|
||||
|
@ -39,7 +39,7 @@ MergeTreeSink::MergeTreeSink(
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, max_parts_per_block(max_parts_per_block_)
|
||||
, context(context_)
|
||||
, storage_snapshot(storage.getStorageSnapshotWithoutParts(metadata_snapshot))
|
||||
, storage_snapshot(storage.getStorageSnapshotWithoutData(metadata_snapshot, context_))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -330,15 +330,20 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction
|
||||
{
|
||||
for (const auto & step : steps)
|
||||
{
|
||||
prewhere.steps.push_back(
|
||||
PrewhereExprStep new_step
|
||||
{
|
||||
.type = PrewhereExprStep::Filter,
|
||||
.actions = std::make_shared<ExpressionActions>(step.actions, actions_settings),
|
||||
.column_name = step.column_name,
|
||||
.remove_column = !all_output_names.contains(step.column_name), /// Don't remove if it's in the list of original outputs
|
||||
.filter_column_name = step.column_name,
|
||||
.remove_filter_column = !all_output_names.contains(step.column_name), /// Don't remove if it's in the list of original outputs
|
||||
.need_filter = false,
|
||||
});
|
||||
.perform_alter_conversions = true,
|
||||
};
|
||||
|
||||
prewhere.steps.push_back(std::make_shared<PrewhereExprStep>(std::move(new_step)));
|
||||
}
|
||||
prewhere.steps.back().need_filter = prewhere_info->need_filter;
|
||||
|
||||
prewhere.steps.back()->need_filter = prewhere_info->need_filter;
|
||||
}
|
||||
|
||||
return true;
|
||||
|
@ -57,9 +57,7 @@ void MergeTreeThreadSelectAlgorithm::finalizeNewTask()
|
||||
/// task->reader.valid() means there is a prefetched reader in this test, use it.
|
||||
const bool init_new_readers = !reader || task->reader.valid() || part_name != last_read_part_name;
|
||||
if (init_new_readers)
|
||||
{
|
||||
initializeMergeTreeReadersForCurrentTask(metadata_snapshot, value_size_map, profile_callback);
|
||||
}
|
||||
|
||||
last_read_part_name = part_name;
|
||||
}
|
||||
|
@ -56,10 +56,10 @@ static bool checkOperationIsNotCanceled(ActionBlocker & merges_blocker, MergeLis
|
||||
*/
|
||||
static void splitAndModifyMutationCommands(
|
||||
MergeTreeData::DataPartPtr part,
|
||||
StorageMetadataPtr metadata_snapshot,
|
||||
const MutationCommands & commands,
|
||||
MutationCommands & for_interpreter,
|
||||
MutationCommands & for_file_renames,
|
||||
const StorageMetadataPtr & table_metadata_snapshot,
|
||||
Poco::Logger * log)
|
||||
{
|
||||
auto part_columns = part->getColumnsDescription();
|
||||
@ -114,7 +114,7 @@ static void splitAndModifyMutationCommands(
|
||||
/// It's important because required renames depend not only on part's data version (i.e. mutation version)
|
||||
/// but also on part's metadata version. Why we have such logic only for renames? Because all other types of alter
|
||||
/// can be deduced based on difference between part's schema and table schema.
|
||||
for (const auto & [rename_to, rename_from] : alter_conversions.rename_map)
|
||||
for (const auto & [rename_to, rename_from] : alter_conversions->getRenameMap())
|
||||
{
|
||||
if (part_columns.has(rename_from))
|
||||
{
|
||||
@ -144,14 +144,14 @@ static void splitAndModifyMutationCommands(
|
||||
{
|
||||
if (!mutated_columns.contains(column.name))
|
||||
{
|
||||
if (!table_metadata_snapshot->getColumns().has(column.name) && !part->storage.getVirtuals().contains(column.name))
|
||||
if (!metadata_snapshot->getColumns().has(column.name) && !part->storage.getVirtuals().contains(column.name))
|
||||
{
|
||||
/// We cannot add the column because there's no such column in table.
|
||||
/// It's okay if the column was dropped. It may also absent in dropped_columns
|
||||
/// if the corresponding MUTATE_PART entry was not created yet or was created separately from current MUTATE_PART.
|
||||
/// But we don't know for sure what happened.
|
||||
auto part_metadata_version = part->getMetadataVersion();
|
||||
auto table_metadata_version = table_metadata_snapshot->getMetadataVersion();
|
||||
auto table_metadata_version = metadata_snapshot->getMetadataVersion();
|
||||
/// StorageMergeTree does not have metadata version
|
||||
if (table_metadata_version <= part_metadata_version && part->storage.supportsReplication())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} with metadata version {} contains column {} that is absent "
|
||||
@ -219,7 +219,7 @@ static void splitAndModifyMutationCommands(
|
||||
/// but also on part's metadata version. Why we have such logic only for renames? Because all other types of alter
|
||||
/// can be deduced based on difference between part's schema and table schema.
|
||||
|
||||
for (const auto & [rename_to, rename_from] : alter_conversions.rename_map)
|
||||
for (const auto & [rename_to, rename_from] : alter_conversions->getRenameMap())
|
||||
{
|
||||
for_file_renames.push_back({.type = MutationCommand::Type::RENAME_COLUMN, .column_name = rename_from, .rename_to = rename_to});
|
||||
}
|
||||
@ -1653,7 +1653,7 @@ MutateTask::MutateTask(
|
||||
ctx->source_part = ctx->future_part->parts[0];
|
||||
ctx->need_prefix = need_prefix_;
|
||||
|
||||
auto storage_snapshot = ctx->data->getStorageSnapshot(ctx->metadata_snapshot, context_);
|
||||
auto storage_snapshot = ctx->data->getStorageSnapshotWithoutData(ctx->metadata_snapshot, context_);
|
||||
extendObjectColumns(ctx->storage_columns, storage_snapshot->object_columns, /*with_subcolumns=*/ false);
|
||||
}
|
||||
|
||||
@ -1748,6 +1748,7 @@ bool MutateTask::prepare()
|
||||
/// Allow mutations to work when force_index_by_date or force_primary_key is on.
|
||||
context_for_reading->setSetting("force_index_by_date", false);
|
||||
context_for_reading->setSetting("force_primary_key", false);
|
||||
context_for_reading->setSetting("apply_mutations_on_fly", false);
|
||||
/// Skip using large sets in KeyCondition
|
||||
context_for_reading->setSetting("use_index_for_in_with_subqueries_max_values", 100000);
|
||||
|
||||
@ -1800,20 +1801,25 @@ bool MutateTask::prepare()
|
||||
context_for_reading->setSetting("allow_asynchronous_read_from_io_pool_for_merge_tree", false);
|
||||
context_for_reading->setSetting("max_streams_for_merge_tree_reading", Field(0));
|
||||
|
||||
MutationHelpers::splitAndModifyMutationCommands(ctx->source_part, ctx->commands_for_part, ctx->for_interpreter,
|
||||
ctx->for_file_renames, ctx->metadata_snapshot, ctx->log);
|
||||
MutationHelpers::splitAndModifyMutationCommands(
|
||||
ctx->source_part, ctx->metadata_snapshot,
|
||||
ctx->commands_for_part, ctx->for_interpreter, ctx->for_file_renames, ctx->log);
|
||||
|
||||
ctx->stage_progress = std::make_unique<MergeStageProgress>(1.0);
|
||||
|
||||
if (!ctx->for_interpreter.empty())
|
||||
{
|
||||
/// Always disable filtering in mutations: we want to read and write all rows because for updates we rewrite only some of the
|
||||
/// columns and preserve the columns that are not affected, but after the update all columns must have the same number of row
|
||||
MutationsInterpreter::Settings settings(true);
|
||||
settings.apply_deleted_mask = false;
|
||||
|
||||
ctx->interpreter = std::make_unique<MutationsInterpreter>(
|
||||
*ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->for_interpreter, context_for_reading, true);
|
||||
*ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->for_interpreter,
|
||||
ctx->metadata_snapshot->getColumns().getNamesOfPhysical(), context_for_reading, settings);
|
||||
|
||||
ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices();
|
||||
ctx->materialized_projections = ctx->interpreter->grabMaterializedProjections();
|
||||
/// Always disable filtering in mutations: we want to read and write all rows because for updates we rewrite only some of the
|
||||
/// columns and preserve the columns that are not affected, but after the update all columns must have the same number of rows.
|
||||
ctx->interpreter->setApplyDeletedMask(false);
|
||||
ctx->mutating_pipeline_builder = ctx->interpreter->execute();
|
||||
ctx->updated_header = ctx->interpreter->getUpdatedHeader();
|
||||
ctx->progress_callback = MergeProgressCallback((*ctx->mutate_entry)->ptr(), ctx->watch_prev_elapsed, *ctx->stage_progress);
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <Storages/MergeTree/MarkRange.h>
|
||||
#include "Storages/MergeTree/AlterConversions.h"
|
||||
#include "Storages/MergeTree/MergeTreePartInfo.h"
|
||||
|
||||
|
||||
@ -40,6 +41,7 @@ struct RangesInDataPartsDescription: public std::deque<RangesInDataPartDescripti
|
||||
struct RangesInDataPart
|
||||
{
|
||||
DataPartPtr data_part;
|
||||
AlterConversionsPtr alter_conversions;
|
||||
size_t part_index_in_query;
|
||||
MarkRanges ranges;
|
||||
|
||||
@ -47,9 +49,11 @@ struct RangesInDataPart
|
||||
|
||||
RangesInDataPart(
|
||||
const DataPartPtr & data_part_,
|
||||
const AlterConversionsPtr & alter_conversions_,
|
||||
const size_t part_index_in_query_,
|
||||
const MarkRanges & ranges_ = MarkRanges{})
|
||||
: data_part{data_part_}
|
||||
, alter_conversions{alter_conversions_}
|
||||
, part_index_in_query{part_index_in_query_}
|
||||
, ranges{ranges_}
|
||||
{}
|
||||
|
@ -1770,36 +1770,40 @@ ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue::getMergePredicate(zk
|
||||
std::map<int64_t, MutationCommands> ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const
|
||||
{
|
||||
std::unique_lock lock(state_mutex);
|
||||
|
||||
auto in_partition = mutations_by_partition.find(part->info.partition_id);
|
||||
if (in_partition == mutations_by_partition.end())
|
||||
return {};
|
||||
|
||||
Int64 part_data_version = part->info.getDataVersion();
|
||||
Int64 part_metadata_version = part->getMetadataVersion();
|
||||
LOG_DEBUG(log, "Looking for mutations for part {} (part data version {}, part metadata version {})", part->name, part_data_version, part_metadata_version);
|
||||
|
||||
std::map<int64_t, MutationCommands> result;
|
||||
/// Here we return mutation commands for part which has bigger alter version than part metadata version.
|
||||
/// Please note, we don't use getDataVersion(). It's because these alter commands are used for in-fly conversions
|
||||
/// of part's metadata.
|
||||
for (const auto & [mutation_version, mutation_status] : in_partition->second | std::views::reverse)
|
||||
{
|
||||
int32_t alter_version = mutation_status->entry->alter_version;
|
||||
auto alter_version = mutation_status->entry->alter_version;
|
||||
if (alter_version != -1)
|
||||
{
|
||||
if (alter_version > storage.getInMemoryMetadataPtr()->getMetadataVersion())
|
||||
continue;
|
||||
|
||||
/// we take commands with bigger metadata version
|
||||
/// We take commands with bigger metadata version
|
||||
if (alter_version > part_metadata_version)
|
||||
{
|
||||
result[mutation_version] = mutation_status->entry->commands;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// entries are ordered, we processing them in reverse order so we can break
|
||||
break;
|
||||
}
|
||||
}
|
||||
else if (mutation_version > part_data_version)
|
||||
{
|
||||
result[mutation_version] = mutation_status->entry->commands;
|
||||
}
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "Got {} commands for part {} (part data version {}, part metadata version {})",
|
||||
result.size(), part->name, part_data_version, part_metadata_version);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
|
@ -275,7 +275,7 @@ ReplicatedMergeTreeSinkImpl<async_insert>::ReplicatedMergeTreeSinkImpl(
|
||||
, deduplicate(deduplicate_)
|
||||
, log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)"))
|
||||
, context(context_)
|
||||
, storage_snapshot(storage.getStorageSnapshotWithoutParts(metadata_snapshot))
|
||||
, storage_snapshot(storage.getStorageSnapshotWithoutData(metadata_snapshot, context_))
|
||||
{
|
||||
/// The quorum value `1` has the same meaning as if it is disabled.
|
||||
if (required_quorum_size == 1)
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
||||
#include <Storages/MergeTree/AlterConversions.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
@ -28,6 +29,7 @@ public:
|
||||
explicit StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_)
|
||||
: IStorage(getIDFromPart(part_))
|
||||
, parts({part_})
|
||||
, alter_conversions({part_->storage.getAlterConversionsForPart(part_)})
|
||||
, storage(part_->storage)
|
||||
, partition_id(part_->info.partition_id)
|
||||
{
|
||||
@ -54,7 +56,7 @@ public:
|
||||
parts.begin(), parts.end(),
|
||||
storage_columns, [](const auto & part) -> const auto & { return part->getColumns(); });
|
||||
|
||||
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, object_columns);
|
||||
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, std::move(object_columns));
|
||||
}
|
||||
|
||||
void read(
|
||||
@ -70,6 +72,7 @@ public:
|
||||
query_plan.addStep(MergeTreeDataSelectExecutor(storage)
|
||||
.readFromParts(
|
||||
parts,
|
||||
alter_conversions,
|
||||
column_names,
|
||||
storage_snapshot,
|
||||
query_info,
|
||||
@ -126,6 +129,7 @@ public:
|
||||
|
||||
private:
|
||||
const MergeTreeData::DataPartsVector parts;
|
||||
const std::vector<AlterConversionsPtr> alter_conversions;
|
||||
const MergeTreeData & storage;
|
||||
const String partition_id;
|
||||
const MergeTreeDataSelectAnalysisResultPtr analysis_result_ptr;
|
||||
|
@ -185,7 +185,6 @@ void MutationCommands::readText(ReadBuffer & in)
|
||||
String commands_str;
|
||||
readEscapedString(commands_str, in);
|
||||
|
||||
|
||||
ParserAlterCommandList p_alter_commands;
|
||||
auto commands_ast = parseQuery(
|
||||
p_alter_commands, commands_str.data(), commands_str.data() + commands_str.length(), "mutation commands list", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||
|
@ -230,14 +230,17 @@ void StorageEmbeddedRocksDB::mutate(const MutationCommands & commands, ContextPt
|
||||
|
||||
if (commands.front().type == MutationCommand::Type::DELETE)
|
||||
{
|
||||
MutationsInterpreter::Settings settings(true);
|
||||
settings.return_all_columns = true;
|
||||
settings.return_mutated_rows = true;
|
||||
|
||||
auto interpreter = std::make_unique<MutationsInterpreter>(
|
||||
storage_ptr,
|
||||
metadata_snapshot,
|
||||
commands,
|
||||
context_,
|
||||
/*can_execute_*/ true,
|
||||
/*return_all_columns_*/ true,
|
||||
/*return_mutated_rows*/ true);
|
||||
settings);
|
||||
|
||||
auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute());
|
||||
PullingPipelineExecutor executor(pipeline);
|
||||
|
||||
@ -278,14 +281,17 @@ void StorageEmbeddedRocksDB::mutate(const MutationCommands & commands, ContextPt
|
||||
if (commands.front().column_to_update_expression.contains(primary_key))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key cannot be updated (cannot update column {})", primary_key);
|
||||
|
||||
MutationsInterpreter::Settings settings(true);
|
||||
settings.return_all_columns = true;
|
||||
settings.return_mutated_rows = true;
|
||||
|
||||
auto interpreter = std::make_unique<MutationsInterpreter>(
|
||||
storage_ptr,
|
||||
metadata_snapshot,
|
||||
commands,
|
||||
context_,
|
||||
/*can_execute_*/ true,
|
||||
/*return_all_columns*/ true,
|
||||
/*return_mutated_rows*/ true);
|
||||
settings);
|
||||
|
||||
auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute());
|
||||
PullingPipelineExecutor executor(pipeline);
|
||||
|
||||
|
@ -644,7 +644,7 @@ StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery(
|
||||
metadata_snapshot->getColumns(),
|
||||
[](const auto & shard_num_and_columns) -> const auto & { return shard_num_and_columns.second; });
|
||||
|
||||
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, object_columns, std::move(snapshot_data));
|
||||
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, std::move(object_columns), std::move(snapshot_data));
|
||||
}
|
||||
|
||||
namespace
|
||||
|
@ -138,7 +138,8 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context)
|
||||
// New scope controls lifetime of pipeline.
|
||||
{
|
||||
auto storage_ptr = DatabaseCatalog::instance().getTable(getStorageID(), context);
|
||||
auto interpreter = std::make_unique<MutationsInterpreter>(storage_ptr, metadata_snapshot, commands, context, true);
|
||||
MutationsInterpreter::Settings settings(true);
|
||||
auto interpreter = std::make_unique<MutationsInterpreter>(storage_ptr, metadata_snapshot, commands, context, settings);
|
||||
auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute());
|
||||
PullingPipelineExecutor executor(pipeline);
|
||||
|
||||
|
@ -857,14 +857,17 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca
|
||||
|
||||
if (commands.front().type == MutationCommand::Type::DELETE)
|
||||
{
|
||||
MutationsInterpreter::Settings settings(true);
|
||||
settings.return_all_columns = true;
|
||||
settings.return_mutated_rows = true;
|
||||
|
||||
auto interpreter = std::make_unique<MutationsInterpreter>(
|
||||
storage_ptr,
|
||||
metadata_snapshot,
|
||||
commands,
|
||||
local_context,
|
||||
/*can_execute_*/ true,
|
||||
/*return_all_columns_*/ true,
|
||||
/*return_mutated_rows*/ true);
|
||||
settings);
|
||||
|
||||
auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute());
|
||||
PullingPipelineExecutor executor(pipeline);
|
||||
|
||||
@ -926,14 +929,17 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca
|
||||
if (commands.front().column_to_update_expression.contains(primary_key))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key cannot be updated (cannot update column {})", primary_key);
|
||||
|
||||
MutationsInterpreter::Settings settings(true);
|
||||
settings.return_all_columns = true;
|
||||
settings.return_mutated_rows = true;
|
||||
|
||||
auto interpreter = std::make_unique<MutationsInterpreter>(
|
||||
storage_ptr,
|
||||
metadata_snapshot,
|
||||
commands,
|
||||
local_context,
|
||||
/*can_execute_*/ true,
|
||||
/*return_all_columns*/ true,
|
||||
/*return_mutated_rows*/ true);
|
||||
settings);
|
||||
|
||||
auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute());
|
||||
PullingPipelineExecutor executor(pipeline);
|
||||
|
||||
|
@ -142,7 +142,7 @@ StorageSnapshotPtr StorageMemory::getStorageSnapshot(const StorageMetadataPtr &
|
||||
metadata_snapshot->getColumns(),
|
||||
[](const auto & block) -> const auto & { return block.getColumnsWithTypeAndName(); });
|
||||
|
||||
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, object_columns, std::move(snapshot_data));
|
||||
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, std::move(object_columns), std::move(snapshot_data));
|
||||
}
|
||||
|
||||
void StorageMemory::read(
|
||||
@ -200,7 +200,8 @@ void StorageMemory::mutate(const MutationCommands & commands, ContextPtr context
|
||||
new_context->setSetting("max_streams_to_max_threads_ratio", 1);
|
||||
new_context->setSetting("max_threads", 1);
|
||||
|
||||
auto interpreter = std::make_unique<MutationsInterpreter>(storage_ptr, metadata_snapshot, commands, new_context, true);
|
||||
MutationsInterpreter::Settings settings(true);
|
||||
auto interpreter = std::make_unique<MutationsInterpreter>(storage_ptr, metadata_snapshot, commands, new_context, settings);
|
||||
auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute());
|
||||
PullingPipelineExecutor executor(pipeline);
|
||||
|
||||
|
@ -254,6 +254,7 @@ void StorageMergeTree::read(
|
||||
/// reset them to avoid holding them.
|
||||
auto & snapshot_data = assert_cast<MergeTreeData::SnapshotData &>(*storage_snapshot->data);
|
||||
snapshot_data.parts = {};
|
||||
snapshot_data.alter_conversions = {};
|
||||
}
|
||||
|
||||
std::optional<UInt64> StorageMergeTree::totalRows(const Settings &) const
|
||||
@ -1164,8 +1165,9 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate(
|
||||
auto fake_query_context = Context::createCopy(getContext());
|
||||
fake_query_context->makeQueryContext();
|
||||
fake_query_context->setCurrentQueryId("");
|
||||
MutationsInterpreter::Settings settings(false);
|
||||
MutationsInterpreter interpreter(
|
||||
shared_from_this(), metadata_snapshot, commands_for_size_validation, fake_query_context, false);
|
||||
shared_from_this(), metadata_snapshot, commands_for_size_validation, fake_query_context, settings);
|
||||
commands_size += interpreter.evaluateCommandsSize();
|
||||
}
|
||||
catch (...)
|
||||
@ -2193,17 +2195,17 @@ std::map<int64_t, MutationCommands> StorageMergeTree::getAlterMutationCommandsFo
|
||||
{
|
||||
std::lock_guard lock(currently_processing_in_background_mutex);
|
||||
|
||||
Int64 part_data_version = part->info.getDataVersion();
|
||||
|
||||
UInt64 part_data_version = part->info.getDataVersion();
|
||||
std::map<int64_t, MutationCommands> result;
|
||||
if (!current_mutations_by_version.empty())
|
||||
|
||||
for (const auto & [mutation_version, entry] : current_mutations_by_version | std::views::reverse)
|
||||
{
|
||||
const auto & [latest_mutation_id, latest_commands] = *current_mutations_by_version.rbegin();
|
||||
if (part_data_version < static_cast<int64_t>(latest_mutation_id))
|
||||
{
|
||||
result[latest_mutation_id] = latest_commands.commands;
|
||||
}
|
||||
if (mutation_version > part_data_version)
|
||||
result[mutation_version] = entry.commands;
|
||||
else
|
||||
break;
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
|
@ -196,7 +196,6 @@ private:
|
||||
void waitForMutation(Int64 version, const String & mutation_id, bool wait_for_another_mutation = false);
|
||||
void setMutationCSN(const String & mutation_id, CSN csn) override;
|
||||
|
||||
|
||||
friend struct CurrentlyMergingPartsTagger;
|
||||
|
||||
MergeMutateSelectedEntryPtr selectPartsToMerge(
|
||||
@ -280,7 +279,6 @@ private:
|
||||
|
||||
|
||||
protected:
|
||||
|
||||
std::map<int64_t, MutationCommands> getAlterMutationCommandsForPart(const DataPartPtr & part) const override;
|
||||
};
|
||||
|
||||
|
@ -4668,6 +4668,7 @@ void StorageReplicatedMergeTree::read(
|
||||
/// reset them to avoid holding them.
|
||||
auto & snapshot_data = assert_cast<MergeTreeData::SnapshotData &>(*storage_snapshot->data);
|
||||
snapshot_data.parts = {};
|
||||
snapshot_data.alter_conversions = {};
|
||||
});
|
||||
|
||||
/** The `select_sequential_consistency` setting has two meanings:
|
||||
@ -8116,7 +8117,6 @@ std::unique_ptr<MergeTreeSettings> StorageReplicatedMergeTree::getDefaultSetting
|
||||
return std::make_unique<MergeTreeSettings>(getContext()->getReplicatedMergeTreeSettings());
|
||||
}
|
||||
|
||||
|
||||
String StorageReplicatedMergeTree::getTableSharedID() const
|
||||
{
|
||||
std::lock_guard lock(table_shared_id_mutex);
|
||||
|
@ -30,27 +30,32 @@ struct StorageSnapshot
|
||||
|
||||
StorageSnapshot(
|
||||
const IStorage & storage_,
|
||||
const StorageMetadataPtr & metadata_)
|
||||
: storage(storage_), metadata(metadata_)
|
||||
StorageMetadataPtr metadata_)
|
||||
: storage(storage_), metadata(std::move(metadata_))
|
||||
{
|
||||
init();
|
||||
}
|
||||
|
||||
StorageSnapshot(
|
||||
const IStorage & storage_,
|
||||
const StorageMetadataPtr & metadata_,
|
||||
const ColumnsDescription & object_columns_)
|
||||
: storage(storage_), metadata(metadata_), object_columns(object_columns_)
|
||||
StorageMetadataPtr metadata_,
|
||||
ColumnsDescription object_columns_)
|
||||
: storage(storage_)
|
||||
, metadata(std::move(metadata_))
|
||||
, object_columns(std::move(object_columns_))
|
||||
{
|
||||
init();
|
||||
}
|
||||
|
||||
StorageSnapshot(
|
||||
const IStorage & storage_,
|
||||
const StorageMetadataPtr & metadata_,
|
||||
const ColumnsDescription & object_columns_,
|
||||
StorageMetadataPtr metadata_,
|
||||
ColumnsDescription object_columns_,
|
||||
DataPtr data_)
|
||||
: storage(storage_), metadata(metadata_), object_columns(object_columns_), data(std::move(data_))
|
||||
: storage(storage_)
|
||||
, metadata(std::move(metadata_))
|
||||
, object_columns(std::move(object_columns_))
|
||||
, data(std::move(data_))
|
||||
{
|
||||
init();
|
||||
}
|
||||
|
@ -24,12 +24,12 @@
|
||||
Name: t_minmax
|
||||
Description: minmax GRANULARITY 2
|
||||
Parts: 1/2
|
||||
Granules: 2/6
|
||||
Granules: 4/6
|
||||
Skip
|
||||
Name: t_set
|
||||
Description: set GRANULARITY 2
|
||||
Parts: 1/1
|
||||
Granules: 1/2
|
||||
Granules: 2/4
|
||||
-----------------
|
||||
"Node Type": "ReadFromMergeTree",
|
||||
"Description": "default.test_index",
|
||||
@ -68,7 +68,7 @@
|
||||
"Initial Parts": 2,
|
||||
"Selected Parts": 1,
|
||||
"Initial Granules": 6,
|
||||
"Selected Granules": 2
|
||||
"Selected Granules": 4
|
||||
},
|
||||
{
|
||||
"Type": "Skip",
|
||||
@ -76,8 +76,8 @@
|
||||
"Description": "set GRANULARITY 2",
|
||||
"Initial Parts": 1,
|
||||
"Selected Parts": 1,
|
||||
"Initial Granules": 2,
|
||||
"Selected Granules": 1
|
||||
"Initial Granules": 4,
|
||||
"Selected Granules": 2
|
||||
}
|
||||
]
|
||||
}
|
||||
|
@ -0,0 +1 @@
|
||||
unquoted_string
|
@ -0,0 +1 @@
|
||||
select * from format(CustomSeparatedIgnoreSpaces, 'x String', ' unquoted_string\n') settings format_custom_escaping_rule='CSV';
|
@ -6,6 +6,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
out="explain1.$CLICKHOUSE_TEST_UNIQUE_NAME.out"
|
||||
# only EXPLAIN triggers the problem under MSan
|
||||
$CLICKHOUSE_CLIENT -q "explain select * from numbers(1) into outfile '$out'"
|
||||
$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q "explain select * from numbers(1) into outfile '$out'"
|
||||
cat "$out"
|
||||
rm -f "$out"
|
||||
|
@ -0,0 +1,10 @@
|
||||
nan 0 0 0 0
|
||||
nan 0 0 0 0
|
||||
nan nan 0 0 0 0
|
||||
--
|
||||
--
|
||||
nan 1 1 1 1
|
||||
nan 1 1 1 1
|
||||
nan nan 1 1 1 1
|
||||
--
|
||||
nan
|
42
tests/queries/0_stateless/02769_nan_equality_comparison.sql
Normal file
42
tests/queries/0_stateless/02769_nan_equality_comparison.sql
Normal file
@ -0,0 +1,42 @@
|
||||
SET compile_expressions = 1;
|
||||
SET min_count_to_compile_expression = 0;
|
||||
|
||||
SELECT nan AS value, value = value, value = materialize(value), materialize(value) = value, materialize(value) = materialize(value);
|
||||
SELECT cast(nan, 'Float32') AS value, value = value, value = materialize(value), materialize(value) = value, materialize(value) = materialize(value);
|
||||
SELECT nan AS lhs, cast(nan, 'Float32') AS rhs, lhs = rhs, lhs = materialize(rhs), materialize(lhs) = rhs, materialize(lhs) = materialize(rhs);
|
||||
|
||||
SELECT '--';
|
||||
|
||||
CREATE TABLE test_table
|
||||
(
|
||||
id UInt32,
|
||||
value UInt32
|
||||
) ENGINE = MergeTree ORDER BY id;
|
||||
INSERT INTO test_table VALUES (76, 57);
|
||||
|
||||
SELECT value FROM (SELECT stddevSamp(id) AS value FROM test_table) as subquery
|
||||
WHERE ((value = value) AND (NOT (value = value)));
|
||||
|
||||
DROP TABLE test_table;
|
||||
|
||||
SELECT '--';
|
||||
|
||||
SELECT nan AS value, value != value, value != materialize(value), materialize(value) != value, materialize(value) != materialize(value);
|
||||
SELECT cast(nan, 'Float32') AS value, value != value, value != materialize(value), materialize(value) != value, materialize(value) != materialize(value);
|
||||
SELECT nan AS lhs, cast(nan, 'Float32') AS rhs, lhs != rhs, lhs != materialize(rhs), materialize(lhs) != rhs, materialize(lhs) != materialize(rhs);
|
||||
|
||||
SELECT '--';
|
||||
|
||||
CREATE TABLE test_table
|
||||
(
|
||||
id UInt32,
|
||||
value_1 UInt32,
|
||||
value_2 Float32
|
||||
) ENGINE = MergeTree ORDER BY id;
|
||||
INSERT INTO test_table VALUES (12000, 36, 77.94);
|
||||
|
||||
SELECT value
|
||||
FROM (SELECT (corr(value_1, value_1) OVER test_window) AS value FROM test_table WINDOW test_window AS (PARTITION BY value_2 ORDER BY id ASC)) as subquery
|
||||
WHERE not (not (value <> value));
|
||||
|
||||
DROP TABLE test_table;
|
@ -0,0 +1,4 @@
|
||||
1048576
|
||||
65411
|
||||
1048576
|
||||
65411
|
@ -0,0 +1,39 @@
|
||||
SET compile_aggregate_expressions = 1;
|
||||
SET min_count_to_compile_aggregate_expression = 0;
|
||||
SET group_by_use_nulls = 0;
|
||||
|
||||
SELECT count() FROM
|
||||
(
|
||||
SELECT
|
||||
count([NULL, NULL]),
|
||||
count([2147483646, -2147483647, 3, 3]),
|
||||
uniqExact(if(number >= 1048577, number, NULL), NULL)
|
||||
FROM numbers(1048577)
|
||||
GROUP BY if(number >= 2., number, NULL)
|
||||
);
|
||||
|
||||
SELECT count() FROM
|
||||
(
|
||||
SELECT count()
|
||||
FROM numbers(65411)
|
||||
GROUP BY if(number < 1, NULL, number)
|
||||
);
|
||||
|
||||
SET group_by_use_nulls = 1;
|
||||
|
||||
SELECT count() FROM
|
||||
(
|
||||
SELECT
|
||||
count([NULL, NULL]),
|
||||
count([2147483646, -2147483647, 3, 3]),
|
||||
uniqExact(if(number >= 1048577, number, NULL), NULL)
|
||||
FROM numbers(1048577)
|
||||
GROUP BY if(number >= 2., number, NULL)
|
||||
);
|
||||
|
||||
SELECT count() FROM
|
||||
(
|
||||
SELECT count()
|
||||
FROM numbers(65411)
|
||||
GROUP BY if(number < 1, NULL, number)
|
||||
);
|
Loading…
Reference in New Issue
Block a user