mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-09-20 00:30:49 +00:00
Merge branch 'ClickHouse:master' into master
This commit is contained in:
commit
c673bac0fe
2
contrib/arrow
vendored
2
contrib/arrow
vendored
@ -1 +1 @@
|
||||
Subproject commit 1f1b3d35fb6eb73e6492d3afd8a85cde848d174f
|
||||
Subproject commit 1d93838f69a802639ca144ea5704a98e2481810d
|
@ -90,34 +90,117 @@ Process 1 stopped
|
||||
|
||||
## Visual Studio Code integration
|
||||
|
||||
- [CodeLLDB extension](https://github.com/vadimcn/vscode-lldb) is required for visual debugging, the [Command Variable](https://github.com/rioj7/command-variable) extension can help dynamic launches if using [cmake variants](https://github.com/microsoft/vscode-cmake-tools/blob/main/docs/variants.md).
|
||||
- Make sure to set the backend to your llvm installation eg. `"lldb.library": "/usr/lib/x86_64-linux-gnu/liblldb-15.so"`
|
||||
- Launcher:
|
||||
- [CodeLLDB](https://github.com/vadimcn/vscode-lldb) extension is required for visual debugging.
|
||||
- [Command Variable](https://github.com/rioj7/command-variable) extension can help dynamic launches if using [CMake Variants](https://github.com/microsoft/vscode-cmake-tools/blob/main/docs/variants.md).
|
||||
- Make sure to set the backend to your LLVM installation eg. `"lldb.library": "/usr/lib/x86_64-linux-gnu/liblldb-15.so"`
|
||||
- Make sure to run the clickhouse executable in debug mode prior to launch. (It is also possible to create a `preLaunchTask` that automates this)
|
||||
|
||||
### Example configurations
|
||||
#### cmake-variants.yaml
|
||||
```yaml
|
||||
buildType:
|
||||
default: relwithdebinfo
|
||||
choices:
|
||||
debug:
|
||||
short: Debug
|
||||
long: Emit debug information
|
||||
buildType: Debug
|
||||
release:
|
||||
short: Release
|
||||
long: Optimize generated code
|
||||
buildType: Release
|
||||
relwithdebinfo:
|
||||
short: RelWithDebInfo
|
||||
long: Release with Debug Info
|
||||
buildType: RelWithDebInfo
|
||||
tsan:
|
||||
short: MinSizeRel
|
||||
long: Minimum Size Release
|
||||
buildType: MinSizeRel
|
||||
|
||||
toolchain:
|
||||
default: default
|
||||
description: Select toolchain
|
||||
choices:
|
||||
default:
|
||||
short: x86_64
|
||||
long: x86_64
|
||||
s390x:
|
||||
short: s390x
|
||||
long: s390x
|
||||
settings:
|
||||
CMAKE_TOOLCHAIN_FILE: cmake/linux/toolchain-s390x.cmake
|
||||
```
|
||||
|
||||
#### launch.json
|
||||
```json
|
||||
{
|
||||
"version": "0.2.0",
|
||||
"configurations": [
|
||||
{
|
||||
"name": "Debug",
|
||||
"type": "lldb",
|
||||
"request": "custom",
|
||||
"targetCreateCommands": ["target create ${command:cmake.launchTargetDirectory}/clickhouse"],
|
||||
"processCreateCommands": ["settings set target.source-map ${input:targetdir} ${workspaceFolder}", "gdb-remote 31338"],
|
||||
"sourceMap": { "${input:targetdir}": "${workspaceFolder}" },
|
||||
}
|
||||
],
|
||||
"inputs": [
|
||||
{
|
||||
"id": "targetdir",
|
||||
"type": "command",
|
||||
"command": "extension.commandvariable.transform",
|
||||
"args": {
|
||||
"text": "${command:cmake.launchTargetDirectory}",
|
||||
"find": ".*/([^/]+)/[^/]+$",
|
||||
"replace": "$1"
|
||||
}
|
||||
"name": "(lldb) Launch s390x with qemu",
|
||||
"targetCreateCommands": ["target create ${command:cmake.launchTargetPath}"],
|
||||
"processCreateCommands": ["gdb-remote 2159"],
|
||||
"preLaunchTask": "Run ClickHouse"
|
||||
}
|
||||
]
|
||||
}
|
||||
```
|
||||
|
||||
#### settings.json
|
||||
This would also put different builds under different subfolders of the `build` folder.
|
||||
```json
|
||||
{
|
||||
"cmake.buildDirectory": "${workspaceFolder}/build/${buildKitVendor}-${buildKitVersion}-${variant:toolchain}-${variant:buildType}",
|
||||
"lldb.library": "/usr/lib/x86_64-linux-gnu/liblldb-15.so"
|
||||
}
|
||||
```
|
||||
|
||||
#### run-debug.sh
|
||||
```sh
|
||||
#! /bin/sh
|
||||
echo 'Starting debugger session'
|
||||
cd $1
|
||||
qemu-s390x-static -g 2159 -L /usr/s390x-linux-gnu $2 $3 $4
|
||||
```
|
||||
|
||||
#### tasks.json
|
||||
Defines a task to run the compiled executable in `server` mode under a `tmp` folder next to the binaries, with configuration from under `programs/server/config.xml`.
|
||||
```json
|
||||
{
|
||||
"version": "2.0.0",
|
||||
"tasks": [
|
||||
{
|
||||
"label": "Run ClickHouse",
|
||||
"type": "shell",
|
||||
"isBackground": true,
|
||||
"command": "${workspaceFolder}/.vscode/run-debug.sh",
|
||||
"args": [
|
||||
"${command:cmake.launchTargetDirectory}/tmp",
|
||||
"${command:cmake.launchTargetPath}",
|
||||
"server",
|
||||
"--config-file=${workspaceFolder}/programs/server/config.xml"
|
||||
],
|
||||
"problemMatcher": [
|
||||
{
|
||||
"pattern": [
|
||||
{
|
||||
"regexp": ".",
|
||||
"file": 1,
|
||||
"location": 2,
|
||||
"message": 3
|
||||
}
|
||||
],
|
||||
"background": {
|
||||
"activeOnStart": true,
|
||||
"beginsPattern": "^Starting debugger session",
|
||||
"endsPattern": ".*"
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
```
|
||||
- Make sure to run the clickhouse executable in debug mode prior to launch. (It is also possible to create a `preLaunchTask` that automates this)
|
@ -219,6 +219,10 @@ LIMIT N
|
||||
SETTINGS annoy_index_search_k_nodes=100;
|
||||
```
|
||||
|
||||
:::note
|
||||
The Annoy index currently does not work with per-table, non-default `index_granularity` settings (see
|
||||
[here](https://github.com/ClickHouse/ClickHouse/pull/51325#issuecomment-1605920475)). If necessary, the value must be changed in config.xml.
|
||||
:::
|
||||
## USearch {#usearch}
|
||||
|
||||
This type of ANN index is based on the [the USearch library](https://github.com/unum-cloud/usearch), which implements the [HNSW
|
||||
@ -274,4 +278,4 @@ USearch currently supports two distance functions:
|
||||
([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)).
|
||||
|
||||
For normalized data, `L2Distance` is usually a better choice, otherwise `cosineDistance` is recommended to compensate for scale. If no
|
||||
distance function was specified during index creation, `L2Distance` is used as default.
|
||||
distance function was specified during index creation, `L2Distance` is used as default.
|
@ -217,6 +217,14 @@ Type: UInt32
|
||||
Default: 1024
|
||||
|
||||
|
||||
## index_mark_cache_policy
|
||||
|
||||
Index mark cache policy name.
|
||||
|
||||
Type: String
|
||||
|
||||
Default: SLRU
|
||||
|
||||
## index_mark_cache_size
|
||||
|
||||
Size of cache for index marks. Zero means disabled.
|
||||
@ -229,6 +237,21 @@ Type: UInt64
|
||||
|
||||
Default: 0
|
||||
|
||||
## index_mark_cache_size_ratio
|
||||
|
||||
The size of the protected queue in the index mark cache relative to the cache's total size.
|
||||
|
||||
Type: Double
|
||||
|
||||
Default: 0.5
|
||||
|
||||
## index_uncompressed_cache_policy
|
||||
|
||||
Index uncompressed cache policy name.
|
||||
|
||||
Type: String
|
||||
|
||||
Default: SLRU
|
||||
|
||||
## index_uncompressed_cache_size
|
||||
|
||||
@ -242,6 +265,13 @@ Type: UInt64
|
||||
|
||||
Default: 0
|
||||
|
||||
## index_uncompressed_cache_size_ratio
|
||||
|
||||
The size of the protected queue in the index uncompressed cache relative to the cache's total size.
|
||||
|
||||
Type: Double
|
||||
|
||||
Default: 0.5
|
||||
|
||||
## io_thread_pool_queue_size
|
||||
|
||||
@ -271,6 +301,14 @@ Type: UInt64
|
||||
|
||||
Default: 5368709120
|
||||
|
||||
## mark_cache_size_ratio
|
||||
|
||||
The size of the protected queue in the mark cache relative to the cache's total size.
|
||||
|
||||
Type: Double
|
||||
|
||||
Default: 0.5
|
||||
|
||||
## max_backup_bandwidth_for_server
|
||||
|
||||
The maximum read speed in bytes per second for all backups on server. Zero means unlimited.
|
||||
@ -629,6 +667,14 @@ Type: UInt64
|
||||
|
||||
Default: 0
|
||||
|
||||
## uncompressed_cache_size_ratio
|
||||
|
||||
The size of the protected queue in the uncompressed cache relative to the cache's total size.
|
||||
|
||||
Type: Double
|
||||
|
||||
Default: 0.5
|
||||
|
||||
## builtin_dictionaries_reload_interval {#builtin-dictionaries-reload-interval}
|
||||
|
||||
The interval in seconds before reloading built-in dictionaries.
|
||||
|
@ -2383,6 +2383,17 @@ See also:
|
||||
|
||||
- [optimize_functions_to_subcolumns](#optimize-functions-to-subcolumns)
|
||||
|
||||
## optimize_count_from_files {#optimize_count_from_files}
|
||||
|
||||
Enables or disables the optimization of counting number of rows from files in different input formats. It applies to table functions/engines `file`/`s3`/`url`/`hdfs`/`azureBlobStorage`.
|
||||
|
||||
Possible values:
|
||||
|
||||
- 0 — Optimization disabled.
|
||||
- 1 — Optimization enabled.
|
||||
|
||||
Default value: `1`.
|
||||
|
||||
## distributed_replica_error_half_life {#settings-distributed_replica_error_half_life}
|
||||
|
||||
- Type: seconds
|
||||
|
@ -657,21 +657,23 @@ void LocalServer::processConfig()
|
||||
/// There is no need for concurrent queries, override max_concurrent_queries.
|
||||
global_context->getProcessList().setMaxSize(0);
|
||||
|
||||
const size_t memory_amount = getMemoryAmount();
|
||||
const size_t physical_server_memory = getMemoryAmount();
|
||||
const double cache_size_to_ram_max_ratio = config().getDouble("cache_size_to_ram_max_ratio", 0.5);
|
||||
const size_t max_cache_size = static_cast<size_t>(memory_amount * cache_size_to_ram_max_ratio);
|
||||
const size_t max_cache_size = static_cast<size_t>(physical_server_memory * cache_size_to_ram_max_ratio);
|
||||
|
||||
String uncompressed_cache_policy = config().getString("uncompressed_cache_policy", DEFAULT_UNCOMPRESSED_CACHE_POLICY);
|
||||
size_t uncompressed_cache_size = config().getUInt64("uncompressed_cache_size", DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE);
|
||||
double uncompressed_cache_size_ratio = config().getDouble("uncompressed_cache_size_ratio", DEFAULT_UNCOMPRESSED_CACHE_SIZE_RATIO);
|
||||
if (uncompressed_cache_size > max_cache_size)
|
||||
{
|
||||
uncompressed_cache_size = max_cache_size;
|
||||
LOG_INFO(log, "Lowered uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size));
|
||||
}
|
||||
global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size);
|
||||
global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size, uncompressed_cache_size_ratio);
|
||||
|
||||
String mark_cache_policy = config().getString("mark_cache_policy", DEFAULT_MARK_CACHE_POLICY);
|
||||
size_t mark_cache_size = config().getUInt64("mark_cache_size", DEFAULT_MARK_CACHE_MAX_SIZE);
|
||||
double mark_cache_size_ratio = config().getDouble("mark_cache_size_ratio", DEFAULT_MARK_CACHE_SIZE_RATIO);
|
||||
if (!mark_cache_size)
|
||||
LOG_ERROR(log, "Too low mark cache size will lead to severe performance degradation.");
|
||||
if (mark_cache_size > max_cache_size)
|
||||
@ -679,23 +681,27 @@ void LocalServer::processConfig()
|
||||
mark_cache_size = max_cache_size;
|
||||
LOG_INFO(log, "Lowered mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(mark_cache_size));
|
||||
}
|
||||
global_context->setMarkCache(mark_cache_policy, mark_cache_size);
|
||||
global_context->setMarkCache(mark_cache_policy, mark_cache_size, mark_cache_size_ratio);
|
||||
|
||||
String index_uncompressed_cache_policy = config().getString("index_uncompressed_cache_policy", DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY);
|
||||
size_t index_uncompressed_cache_size = config().getUInt64("index_uncompressed_cache_size", DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE);
|
||||
double index_uncompressed_cache_size_ratio = config().getDouble("index_uncompressed_cache_size_ratio", DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO);
|
||||
if (index_uncompressed_cache_size > max_cache_size)
|
||||
{
|
||||
index_uncompressed_cache_size = max_cache_size;
|
||||
LOG_INFO(log, "Lowered index uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size));
|
||||
}
|
||||
global_context->setIndexUncompressedCache(index_uncompressed_cache_size);
|
||||
global_context->setIndexUncompressedCache(index_uncompressed_cache_policy, index_uncompressed_cache_size, index_uncompressed_cache_size_ratio);
|
||||
|
||||
String index_mark_cache_policy = config().getString("index_mark_cache_policy", DEFAULT_INDEX_MARK_CACHE_POLICY);
|
||||
size_t index_mark_cache_size = config().getUInt64("index_mark_cache_size", DEFAULT_INDEX_MARK_CACHE_MAX_SIZE);
|
||||
double index_mark_cache_size_ratio = config().getDouble("index_mark_cache_size_ratio", DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO);
|
||||
if (index_mark_cache_size > max_cache_size)
|
||||
{
|
||||
index_mark_cache_size = max_cache_size;
|
||||
LOG_INFO(log, "Lowered index mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size));
|
||||
}
|
||||
global_context->setIndexMarkCache(index_mark_cache_size);
|
||||
global_context->setIndexMarkCache(index_mark_cache_policy, index_mark_cache_size, index_mark_cache_size_ratio);
|
||||
|
||||
size_t mmap_cache_size = config().getUInt64("mmap_cache_size", DEFAULT_MMAP_CACHE_MAX_SIZE);
|
||||
if (mmap_cache_size > max_cache_size)
|
||||
|
@ -1111,37 +1111,43 @@ try
|
||||
|
||||
String uncompressed_cache_policy = server_settings.uncompressed_cache_policy;
|
||||
size_t uncompressed_cache_size = server_settings.uncompressed_cache_size;
|
||||
double uncompressed_cache_size_ratio = server_settings.uncompressed_cache_size_ratio;
|
||||
if (uncompressed_cache_size > max_cache_size)
|
||||
{
|
||||
uncompressed_cache_size = max_cache_size;
|
||||
LOG_INFO(log, "Lowered uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size));
|
||||
}
|
||||
global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size);
|
||||
global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size, uncompressed_cache_size_ratio);
|
||||
|
||||
String mark_cache_policy = server_settings.mark_cache_policy;
|
||||
size_t mark_cache_size = server_settings.mark_cache_size;
|
||||
double mark_cache_size_ratio = server_settings.mark_cache_size_ratio;
|
||||
if (mark_cache_size > max_cache_size)
|
||||
{
|
||||
mark_cache_size = max_cache_size;
|
||||
LOG_INFO(log, "Lowered mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(mark_cache_size));
|
||||
}
|
||||
global_context->setMarkCache(mark_cache_policy, mark_cache_size);
|
||||
global_context->setMarkCache(mark_cache_policy, mark_cache_size, mark_cache_size_ratio);
|
||||
|
||||
String index_uncompressed_cache_policy = server_settings.index_uncompressed_cache_policy;
|
||||
size_t index_uncompressed_cache_size = server_settings.index_uncompressed_cache_size;
|
||||
double index_uncompressed_cache_size_ratio = server_settings.index_uncompressed_cache_size_ratio;
|
||||
if (index_uncompressed_cache_size > max_cache_size)
|
||||
{
|
||||
index_uncompressed_cache_size = max_cache_size;
|
||||
LOG_INFO(log, "Lowered index uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size));
|
||||
}
|
||||
global_context->setIndexUncompressedCache(index_uncompressed_cache_size);
|
||||
global_context->setIndexUncompressedCache(index_uncompressed_cache_policy, index_uncompressed_cache_size, index_uncompressed_cache_size_ratio);
|
||||
|
||||
String index_mark_cache_policy = server_settings.index_mark_cache_policy;
|
||||
size_t index_mark_cache_size = server_settings.index_mark_cache_size;
|
||||
double index_mark_cache_size_ratio = server_settings.index_mark_cache_size_ratio;
|
||||
if (index_mark_cache_size > max_cache_size)
|
||||
{
|
||||
index_mark_cache_size = max_cache_size;
|
||||
LOG_INFO(log, "Lowered index mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size));
|
||||
}
|
||||
global_context->setIndexMarkCache(index_mark_cache_size);
|
||||
global_context->setIndexMarkCache(index_mark_cache_policy, index_mark_cache_size, index_mark_cache_size_ratio);
|
||||
|
||||
size_t mmap_cache_size = server_settings.mmap_cache_size;
|
||||
if (mmap_cache_size > max_cache_size)
|
||||
|
@ -40,14 +40,17 @@ public:
|
||||
using MappedPtr = typename CachePolicy::MappedPtr;
|
||||
using KeyMapped = typename CachePolicy::KeyMapped;
|
||||
|
||||
/// Use this ctor if you don't care about the internal cache policy.
|
||||
explicit CacheBase(size_t max_size_in_bytes, size_t max_count = 0, double size_ratio = 0.5)
|
||||
static constexpr auto NO_MAX_COUNT = 0uz;
|
||||
static constexpr auto DEFAULT_SIZE_RATIO = 0.5l;
|
||||
|
||||
/// Use this ctor if you only care about the cache size but not internals like the cache policy.
|
||||
explicit CacheBase(size_t max_size_in_bytes, size_t max_count = NO_MAX_COUNT, double size_ratio = DEFAULT_SIZE_RATIO)
|
||||
: CacheBase("SLRU", max_size_in_bytes, max_count, size_ratio)
|
||||
{
|
||||
}
|
||||
|
||||
/// Use this ctor if you want the user to configure the cache policy via some setting. Supports only general-purpose policies LRU and SLRU.
|
||||
explicit CacheBase(std::string_view cache_policy_name, size_t max_size_in_bytes, size_t max_count = 0, double size_ratio = 0.5)
|
||||
/// Use this ctor if the user should be able to configure the cache policy and cache sizes via settings. Supports only general-purpose policies LRU and SLRU.
|
||||
explicit CacheBase(std::string_view cache_policy_name, size_t max_size_in_bytes, size_t max_count, double size_ratio)
|
||||
{
|
||||
auto on_weight_loss_function = [&](size_t weight_loss) { onRemoveOverflowWeightLoss(weight_loss); };
|
||||
|
||||
@ -79,7 +82,7 @@ public:
|
||||
MappedPtr get(const Key & key)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto res = cache_policy->get(key, lock);
|
||||
auto res = cache_policy->get(key);
|
||||
if (res)
|
||||
++hits;
|
||||
else
|
||||
@ -90,7 +93,7 @@ public:
|
||||
std::optional<KeyMapped> getWithKey(const Key & key)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto res = cache_policy->getWithKey(key, lock);
|
||||
auto res = cache_policy->getWithKey(key);
|
||||
if (res.has_value())
|
||||
++hits;
|
||||
else
|
||||
@ -101,7 +104,7 @@ public:
|
||||
void set(const Key & key, const MappedPtr & mapped)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
cache_policy->set(key, mapped, lock);
|
||||
cache_policy->set(key, mapped);
|
||||
}
|
||||
|
||||
/// If the value for the key is in the cache, returns it. If it is not, calls load_func() to
|
||||
@ -118,7 +121,7 @@ public:
|
||||
InsertTokenHolder token_holder;
|
||||
{
|
||||
std::lock_guard cache_lock(mutex);
|
||||
auto val = cache_policy->get(key, cache_lock);
|
||||
auto val = cache_policy->get(key);
|
||||
if (val)
|
||||
{
|
||||
++hits;
|
||||
@ -156,7 +159,7 @@ public:
|
||||
auto token_it = insert_tokens.find(key);
|
||||
if (token_it != insert_tokens.end() && token_it->second.get() == token)
|
||||
{
|
||||
cache_policy->set(key, token->value, cache_lock);
|
||||
cache_policy->set(key, token->value);
|
||||
result = true;
|
||||
}
|
||||
|
||||
@ -185,49 +188,49 @@ public:
|
||||
insert_tokens.clear();
|
||||
hits = 0;
|
||||
misses = 0;
|
||||
cache_policy->clear(lock);
|
||||
cache_policy->clear();
|
||||
}
|
||||
|
||||
void remove(const Key & key)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
cache_policy->remove(key, lock);
|
||||
cache_policy->remove(key);
|
||||
}
|
||||
|
||||
size_t weight() const
|
||||
size_t sizeInBytes() const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return cache_policy->weight(lock);
|
||||
return cache_policy->sizeInBytes();
|
||||
}
|
||||
|
||||
size_t count() const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return cache_policy->count(lock);
|
||||
return cache_policy->count();
|
||||
}
|
||||
|
||||
size_t maxSize() const
|
||||
size_t maxSizeInBytes() const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return cache_policy->maxSize(lock);
|
||||
return cache_policy->maxSizeInBytes();
|
||||
}
|
||||
|
||||
void setMaxCount(size_t max_count)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
cache_policy->setMaxCount(max_count, lock);
|
||||
cache_policy->setMaxCount(max_count);
|
||||
}
|
||||
|
||||
void setMaxSize(size_t max_size_in_bytes)
|
||||
void setMaxSizeInBytes(size_t max_size_in_bytes)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
cache_policy->setMaxSize(max_size_in_bytes, lock);
|
||||
cache_policy->setMaxSizeInBytes(max_size_in_bytes);
|
||||
}
|
||||
|
||||
void setQuotaForUser(const String & user_name, size_t max_size_in_bytes, size_t max_entries)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
cache_policy->setQuotaForUser(user_name, max_size_in_bytes, max_entries, lock);
|
||||
cache_policy->setQuotaForUser(user_name, max_size_in_bytes, max_entries);
|
||||
}
|
||||
|
||||
virtual ~CacheBase() = default;
|
||||
|
@ -37,25 +37,25 @@ public:
|
||||
explicit ICachePolicy(CachePolicyUserQuotaPtr user_quotas_) : user_quotas(std::move(user_quotas_)) {}
|
||||
virtual ~ICachePolicy() = default;
|
||||
|
||||
virtual size_t weight(std::lock_guard<std::mutex> & /*cache_lock*/) const = 0;
|
||||
virtual size_t count(std::lock_guard<std::mutex> & /*cache_lock*/) const = 0;
|
||||
virtual size_t maxSize(std::lock_guard<std::mutex>& /*cache_lock*/) const = 0;
|
||||
virtual size_t sizeInBytes() const = 0;
|
||||
virtual size_t count() const = 0;
|
||||
virtual size_t maxSizeInBytes() const = 0;
|
||||
|
||||
virtual void setMaxCount(size_t /*max_count*/, std::lock_guard<std::mutex> & /* cache_lock */) = 0;
|
||||
virtual void setMaxSize(size_t /*max_size_in_bytes*/, std::lock_guard<std::mutex> & /* cache_lock */) = 0;
|
||||
virtual void setQuotaForUser(const String & user_name, size_t max_size_in_bytes, size_t max_entries, std::lock_guard<std::mutex> & /*cache_lock*/) { user_quotas->setQuotaForUser(user_name, max_size_in_bytes, max_entries); }
|
||||
virtual void setMaxCount(size_t /*max_count*/) = 0;
|
||||
virtual void setMaxSizeInBytes(size_t /*max_size_in_bytes*/) = 0;
|
||||
virtual void setQuotaForUser(const String & user_name, size_t max_size_in_bytes, size_t max_entries) { user_quotas->setQuotaForUser(user_name, max_size_in_bytes, max_entries); }
|
||||
|
||||
/// HashFunction usually hashes the entire key and the found key will be equal the provided key. In such cases, use get(). It is also
|
||||
/// possible to store other, non-hashed data in the key. In that case, the found key is potentially different from the provided key.
|
||||
/// Then use getWithKey() to also return the found key including it's non-hashed data.
|
||||
virtual MappedPtr get(const Key & key, std::lock_guard<std::mutex> & /* cache_lock */) = 0;
|
||||
virtual std::optional<KeyMapped> getWithKey(const Key &, std::lock_guard<std::mutex> & /*cache_lock*/) = 0;
|
||||
virtual MappedPtr get(const Key & key) = 0;
|
||||
virtual std::optional<KeyMapped> getWithKey(const Key &) = 0;
|
||||
|
||||
virtual void set(const Key & key, const MappedPtr & mapped, std::lock_guard<std::mutex> & /*cache_lock*/) = 0;
|
||||
virtual void set(const Key & key, const MappedPtr & mapped) = 0;
|
||||
|
||||
virtual void remove(const Key & key, std::lock_guard<std::mutex> & /*cache_lock*/) = 0;
|
||||
virtual void remove(const Key & key) = 0;
|
||||
|
||||
virtual void clear(std::lock_guard<std::mutex> & /*cache_lock*/) = 0;
|
||||
virtual void clear() = 0;
|
||||
virtual std::vector<KeyMapped> dump() const = 0;
|
||||
|
||||
protected:
|
||||
|
@ -34,41 +34,41 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
size_t weight(std::lock_guard<std::mutex> & /* cache_lock */) const override
|
||||
size_t sizeInBytes() const override
|
||||
{
|
||||
return current_size_in_bytes;
|
||||
}
|
||||
|
||||
size_t count(std::lock_guard<std::mutex> & /* cache_lock */) const override
|
||||
size_t count() const override
|
||||
{
|
||||
return cells.size();
|
||||
}
|
||||
|
||||
size_t maxSize(std::lock_guard<std::mutex> & /* cache_lock */) const override
|
||||
size_t maxSizeInBytes() const override
|
||||
{
|
||||
return max_size_in_bytes;
|
||||
}
|
||||
|
||||
void setMaxCount(size_t max_count_, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void setMaxCount(size_t max_count_) override
|
||||
{
|
||||
max_count = max_count_;
|
||||
removeOverflow();
|
||||
}
|
||||
|
||||
void setMaxSize(size_t max_size_in_bytes_, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void setMaxSizeInBytes(size_t max_size_in_bytes_) override
|
||||
{
|
||||
max_size_in_bytes = max_size_in_bytes_;
|
||||
removeOverflow();
|
||||
}
|
||||
|
||||
void clear(std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void clear() override
|
||||
{
|
||||
queue.clear();
|
||||
cells.clear();
|
||||
current_size_in_bytes = 0;
|
||||
}
|
||||
|
||||
void remove(const Key & key, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void remove(const Key & key) override
|
||||
{
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
@ -79,7 +79,7 @@ public:
|
||||
cells.erase(it);
|
||||
}
|
||||
|
||||
MappedPtr get(const Key & key, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
MappedPtr get(const Key & key) override
|
||||
{
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
@ -93,7 +93,7 @@ public:
|
||||
return cell.value;
|
||||
}
|
||||
|
||||
std::optional<KeyMapped> getWithKey(const Key & key, std::lock_guard<std::mutex> & /*cache_lock*/) override
|
||||
std::optional<KeyMapped> getWithKey(const Key & key) override
|
||||
{
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
@ -107,7 +107,7 @@ public:
|
||||
return std::make_optional<KeyMapped>({it->first, cell.value});
|
||||
}
|
||||
|
||||
void set(const Key & key, const MappedPtr & mapped, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void set(const Key & key, const MappedPtr & mapped) override
|
||||
{
|
||||
auto [it, inserted] = cells.emplace(std::piecewise_construct,
|
||||
std::forward_as_tuple(key),
|
||||
|
@ -366,6 +366,8 @@ The server successfully detected this situation and will download merged part fr
|
||||
M(DiskS3PutObject, "Number of DiskS3 API PutObject calls.") \
|
||||
M(DiskS3GetObject, "Number of DiskS3 API GetObject calls.") \
|
||||
\
|
||||
M(EngineFileLikeReadFiles, "Number of files read in table engines working with files (like File/S3/URL/HDFS).") \
|
||||
\
|
||||
M(ReadBufferFromS3Microseconds, "Time spent on reading from S3.") \
|
||||
M(ReadBufferFromS3InitMicroseconds, "Time spent initializing connection to S3.") \
|
||||
M(ReadBufferFromS3Bytes, "Bytes read from S3.") \
|
||||
|
@ -31,45 +31,45 @@ public:
|
||||
/// TODO: construct from special struct with cache policy parameters (also with max_protected_size).
|
||||
SLRUCachePolicy(size_t max_size_in_bytes_, size_t max_count_, double size_ratio_, OnWeightLossFunction on_weight_loss_function_)
|
||||
: Base(std::make_unique<NoCachePolicyUserQuota>())
|
||||
, size_ratio(size_ratio_)
|
||||
, max_protected_size(static_cast<size_t>(max_size_in_bytes_ * std::min(1.0, size_ratio)))
|
||||
, max_size_in_bytes(max_size_in_bytes_)
|
||||
, max_protected_size(calculateMaxProtectedSize(max_size_in_bytes_, size_ratio_))
|
||||
, max_count(max_count_)
|
||||
, size_ratio(size_ratio_)
|
||||
, on_weight_loss_function(on_weight_loss_function_)
|
||||
{
|
||||
}
|
||||
|
||||
size_t weight(std::lock_guard<std::mutex> & /* cache_lock */) const override
|
||||
size_t sizeInBytes() const override
|
||||
{
|
||||
return current_size_in_bytes;
|
||||
}
|
||||
|
||||
size_t count(std::lock_guard<std::mutex> & /* cache_lock */) const override
|
||||
size_t count() const override
|
||||
{
|
||||
return cells.size();
|
||||
}
|
||||
|
||||
size_t maxSize(std::lock_guard<std::mutex> & /* cache_lock */) const override
|
||||
size_t maxSizeInBytes() const override
|
||||
{
|
||||
return max_size_in_bytes;
|
||||
}
|
||||
|
||||
void setMaxCount(size_t max_count_, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void setMaxCount(size_t max_count_) override
|
||||
{
|
||||
max_count = max_count_;
|
||||
removeOverflow(protected_queue, max_protected_size, current_protected_size, /*is_protected=*/true);
|
||||
removeOverflow(probationary_queue, max_size_in_bytes, current_size_in_bytes, /*is_protected=*/false);
|
||||
}
|
||||
|
||||
void setMaxSize(size_t max_size_in_bytes_, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void setMaxSizeInBytes(size_t max_size_in_bytes_) override
|
||||
{
|
||||
max_protected_size = static_cast<size_t>(max_size_in_bytes_ * std::min(1.0, size_ratio));
|
||||
max_protected_size = calculateMaxProtectedSize(max_size_in_bytes_, size_ratio);
|
||||
max_size_in_bytes = max_size_in_bytes_;
|
||||
removeOverflow(protected_queue, max_protected_size, current_protected_size, /*is_protected=*/true);
|
||||
removeOverflow(probationary_queue, max_size_in_bytes, current_size_in_bytes, /*is_protected=*/false);
|
||||
}
|
||||
|
||||
void clear(std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void clear() override
|
||||
{
|
||||
cells.clear();
|
||||
probationary_queue.clear();
|
||||
@ -78,7 +78,7 @@ public:
|
||||
current_protected_size = 0;
|
||||
}
|
||||
|
||||
void remove(const Key & key, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void remove(const Key & key) override
|
||||
{
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
@ -95,7 +95,7 @@ public:
|
||||
cells.erase(it);
|
||||
}
|
||||
|
||||
MappedPtr get(const Key & key, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
MappedPtr get(const Key & key) override
|
||||
{
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
@ -116,7 +116,7 @@ public:
|
||||
return cell.value;
|
||||
}
|
||||
|
||||
std::optional<KeyMapped> getWithKey(const Key & key, std::lock_guard<std::mutex> & /*cache_lock*/) override
|
||||
std::optional<KeyMapped> getWithKey(const Key & key) override
|
||||
{
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
@ -137,7 +137,7 @@ public:
|
||||
return std::make_optional<KeyMapped>({it->first, cell.value});
|
||||
}
|
||||
|
||||
void set(const Key & key, const MappedPtr & mapped, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void set(const Key & key, const MappedPtr & mapped) override
|
||||
{
|
||||
auto [it, inserted] = cells.emplace(std::piecewise_construct,
|
||||
std::forward_as_tuple(key),
|
||||
@ -208,16 +208,21 @@ private:
|
||||
|
||||
Cells cells;
|
||||
|
||||
size_t max_size_in_bytes;
|
||||
size_t max_protected_size;
|
||||
size_t max_count;
|
||||
const double size_ratio;
|
||||
size_t current_protected_size = 0;
|
||||
size_t current_size_in_bytes = 0;
|
||||
size_t max_protected_size;
|
||||
size_t max_size_in_bytes;
|
||||
size_t max_count;
|
||||
|
||||
WeightFunction weight_function;
|
||||
OnWeightLossFunction on_weight_loss_function;
|
||||
|
||||
static size_t calculateMaxProtectedSize(size_t max_size_in_bytes, double size_ratio)
|
||||
{
|
||||
return static_cast<size_t>(max_size_in_bytes * std::max(0.0, std::min(1.0, size_ratio)));
|
||||
}
|
||||
|
||||
void removeOverflow(SLRUQueue & queue, size_t max_weight_size, size_t & current_weight_size, bool is_protected)
|
||||
{
|
||||
size_t current_weight_lost = 0;
|
||||
|
@ -94,39 +94,39 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
size_t weight(std::lock_guard<std::mutex> & /* cache_lock */) const override
|
||||
size_t sizeInBytes() const override
|
||||
{
|
||||
return size_in_bytes;
|
||||
}
|
||||
|
||||
size_t count(std::lock_guard<std::mutex> & /* cache_lock */) const override
|
||||
size_t count() const override
|
||||
{
|
||||
return cache.size();
|
||||
}
|
||||
|
||||
size_t maxSize(std::lock_guard<std::mutex> & /* cache_lock */) const override
|
||||
size_t maxSizeInBytes() const override
|
||||
{
|
||||
return max_size_in_bytes;
|
||||
}
|
||||
|
||||
void setMaxCount(size_t max_count_, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void setMaxCount(size_t max_count_) override
|
||||
{
|
||||
/// lazy behavior: the cache only shrinks upon the next insert
|
||||
max_count = max_count_;
|
||||
}
|
||||
|
||||
void setMaxSize(size_t max_size_in_bytes_, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void setMaxSizeInBytes(size_t max_size_in_bytes_) override
|
||||
{
|
||||
/// lazy behavior: the cache only shrinks upon the next insert
|
||||
max_size_in_bytes = max_size_in_bytes_;
|
||||
}
|
||||
|
||||
void clear(std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void clear() override
|
||||
{
|
||||
cache.clear();
|
||||
}
|
||||
|
||||
void remove(const Key & key, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void remove(const Key & key) override
|
||||
{
|
||||
auto it = cache.find(key);
|
||||
if (it == cache.end())
|
||||
@ -137,7 +137,7 @@ public:
|
||||
size_in_bytes -= sz;
|
||||
}
|
||||
|
||||
MappedPtr get(const Key & key, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
MappedPtr get(const Key & key) override
|
||||
{
|
||||
auto it = cache.find(key);
|
||||
if (it == cache.end())
|
||||
@ -145,7 +145,7 @@ public:
|
||||
return it->second;
|
||||
}
|
||||
|
||||
std::optional<KeyMapped> getWithKey(const Key & key, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
std::optional<KeyMapped> getWithKey(const Key & key) override
|
||||
{
|
||||
auto it = cache.find(key);
|
||||
if (it == cache.end())
|
||||
@ -154,7 +154,7 @@ public:
|
||||
}
|
||||
|
||||
/// Evicts on a best-effort basis. If there are too many non-stale entries, the new entry may not be cached at all!
|
||||
void set(const Key & key, const MappedPtr & mapped, std::lock_guard<std::mutex> & /* cache_lock */) override
|
||||
void set(const Key & key, const MappedPtr & mapped) override
|
||||
{
|
||||
chassert(mapped.get());
|
||||
|
||||
|
@ -5,11 +5,11 @@
|
||||
TEST(LRUCache, set)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, int>;
|
||||
auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10);
|
||||
auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10, /*size_ratio*/ 0.5);
|
||||
lru_cache.set(1, std::make_shared<int>(2));
|
||||
lru_cache.set(2, std::make_shared<int>(3));
|
||||
|
||||
auto w = lru_cache.weight();
|
||||
auto w = lru_cache.sizeInBytes();
|
||||
auto n = lru_cache.count();
|
||||
ASSERT_EQ(w, 2);
|
||||
ASSERT_EQ(n, 2);
|
||||
@ -18,7 +18,7 @@ TEST(LRUCache, set)
|
||||
TEST(LRUCache, update)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, int>;
|
||||
auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10);
|
||||
auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10, /*size_ratio*/ 0.5);
|
||||
lru_cache.set(1, std::make_shared<int>(2));
|
||||
lru_cache.set(1, std::make_shared<int>(3));
|
||||
auto val = lru_cache.get(1);
|
||||
@ -29,7 +29,7 @@ TEST(LRUCache, update)
|
||||
TEST(LRUCache, get)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, int>;
|
||||
auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10);
|
||||
auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10, /*size_ratio*/ 0.5);
|
||||
lru_cache.set(1, std::make_shared<int>(2));
|
||||
lru_cache.set(2, std::make_shared<int>(3));
|
||||
SimpleCacheBase::MappedPtr value = lru_cache.get(1);
|
||||
@ -49,7 +49,7 @@ struct ValueWeight
|
||||
TEST(LRUCache, evictOnSize)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, size_t>;
|
||||
auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 20, /*max_count*/ 3);
|
||||
auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 20, /*max_count*/ 3, /*size_ratio*/ 0.5);
|
||||
lru_cache.set(1, std::make_shared<size_t>(2));
|
||||
lru_cache.set(2, std::make_shared<size_t>(3));
|
||||
lru_cache.set(3, std::make_shared<size_t>(4));
|
||||
@ -65,7 +65,7 @@ TEST(LRUCache, evictOnSize)
|
||||
TEST(LRUCache, evictOnWeight)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, size_t, std::hash<int>, ValueWeight>;
|
||||
auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10);
|
||||
auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10, /*size_ratio*/ 0.5);
|
||||
lru_cache.set(1, std::make_shared<size_t>(2));
|
||||
lru_cache.set(2, std::make_shared<size_t>(3));
|
||||
lru_cache.set(3, std::make_shared<size_t>(4));
|
||||
@ -74,7 +74,7 @@ TEST(LRUCache, evictOnWeight)
|
||||
auto n = lru_cache.count();
|
||||
ASSERT_EQ(n, 2);
|
||||
|
||||
auto w = lru_cache.weight();
|
||||
auto w = lru_cache.sizeInBytes();
|
||||
ASSERT_EQ(w, 9);
|
||||
|
||||
auto value = lru_cache.get(1);
|
||||
@ -86,7 +86,7 @@ TEST(LRUCache, evictOnWeight)
|
||||
TEST(LRUCache, getOrSet)
|
||||
{
|
||||
using SimpleCacheBase = DB::CacheBase<int, size_t, std::hash<int>, ValueWeight>;
|
||||
auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10);
|
||||
auto lru_cache = SimpleCacheBase("LRU", /*max_size_in_bytes*/ 10, /*max_count*/ 10, /*size_ratio*/ 0.5);
|
||||
size_t x = 10;
|
||||
auto load_func = [&] { return std::make_shared<size_t>(x); };
|
||||
auto [value, loaded] = lru_cache.getOrSet(1, load_func);
|
||||
|
@ -9,7 +9,7 @@ TEST(SLRUCache, set)
|
||||
slru_cache.set(1, std::make_shared<int>(2));
|
||||
slru_cache.set(2, std::make_shared<int>(3));
|
||||
|
||||
auto w = slru_cache.weight();
|
||||
auto w = slru_cache.sizeInBytes();
|
||||
auto n = slru_cache.count();
|
||||
ASSERT_EQ(w, 2);
|
||||
ASSERT_EQ(n, 2);
|
||||
@ -125,7 +125,7 @@ TEST(SLRUCache, evictOnElements)
|
||||
auto n = slru_cache.count();
|
||||
ASSERT_EQ(n, 1);
|
||||
|
||||
auto w = slru_cache.weight();
|
||||
auto w = slru_cache.sizeInBytes();
|
||||
ASSERT_EQ(w, 3);
|
||||
|
||||
auto value = slru_cache.get(1);
|
||||
@ -148,7 +148,7 @@ TEST(SLRUCache, evictOnWeight)
|
||||
auto n = slru_cache.count();
|
||||
ASSERT_EQ(n, 2);
|
||||
|
||||
auto w = slru_cache.weight();
|
||||
auto w = slru_cache.sizeInBytes();
|
||||
ASSERT_EQ(w, 9);
|
||||
|
||||
auto value = slru_cache.get(1);
|
||||
|
@ -23,7 +23,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
try
|
||||
{
|
||||
UncompressedCache cache(1024);
|
||||
UncompressedCache cache("SLRU", 1024, 0.5);
|
||||
std::string path = argv[1];
|
||||
|
||||
std::cerr << std::fixed << std::setprecision(3);
|
||||
|
@ -66,12 +66,18 @@
|
||||
#define DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH 1000
|
||||
|
||||
/// Default maximum (total and entry) sizes and policies of various caches
|
||||
static constexpr auto DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE = 0_MiB;
|
||||
static constexpr auto DEFAULT_UNCOMPRESSED_CACHE_POLICY = "SLRU";
|
||||
static constexpr auto DEFAULT_MARK_CACHE_MAX_SIZE = 5368_MiB;
|
||||
static constexpr auto DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE = 0_MiB;
|
||||
static constexpr auto DEFAULT_UNCOMPRESSED_CACHE_SIZE_RATIO = 0.5l;
|
||||
static constexpr auto DEFAULT_MARK_CACHE_POLICY = "SLRU";
|
||||
static constexpr auto DEFAULT_MARK_CACHE_MAX_SIZE = 5368_MiB;
|
||||
static constexpr auto DEFAULT_MARK_CACHE_SIZE_RATIO = 0.5l;
|
||||
static constexpr auto DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY = "SLRU";
|
||||
static constexpr auto DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE = 0_MiB;
|
||||
static constexpr auto DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO = 0.5l;
|
||||
static constexpr auto DEFAULT_INDEX_MARK_CACHE_POLICY = "SLRU";
|
||||
static constexpr auto DEFAULT_INDEX_MARK_CACHE_MAX_SIZE = 0_MiB;
|
||||
static constexpr auto DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO = 0.5l;
|
||||
static constexpr auto DEFAULT_MMAP_CACHE_MAX_SIZE = 1_KiB; /// chosen by rolling dice
|
||||
static constexpr auto DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE = 128_MiB;
|
||||
static constexpr auto DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES = 10'000;
|
||||
|
@ -60,10 +60,16 @@ namespace DB
|
||||
M(Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro RAM max ratio. Allows to lower cache size on low-memory systems.", 0) \
|
||||
M(String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0) \
|
||||
M(UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0) \
|
||||
M(UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0) \
|
||||
M(Double, uncompressed_cache_size_ratio, DEFAULT_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the uncompressed cache relative to the cache's total size.", 0) \
|
||||
M(String, mark_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Mark cache policy name.", 0) \
|
||||
M(UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0) \
|
||||
M(Double, mark_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the mark cache relative to the cache's total size.", 0) \
|
||||
M(String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, "Index uncompressed cache policy name.", 0) \
|
||||
M(UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled.", 0) \
|
||||
M(Double, index_uncompressed_cache_size_ratio, DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the index uncompressed cache relative to the cache's total size.", 0) \
|
||||
M(String, index_mark_cache_policy, DEFAULT_INDEX_MARK_CACHE_POLICY, "Index mark cache policy name.", 0) \
|
||||
M(UInt64, index_mark_cache_size, DEFAULT_INDEX_MARK_CACHE_MAX_SIZE, "Size of cache for index marks. Zero means disabled.", 0) \
|
||||
M(Double, index_mark_cache_size_ratio, DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the index mark cache relative to the cache's total size.", 0) \
|
||||
M(UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0) \
|
||||
\
|
||||
M(Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0) \
|
||||
|
@ -535,6 +535,7 @@ class IColumn;
|
||||
M(Bool, database_atomic_wait_for_drop_and_detach_synchronously, false, "When executing DROP or DETACH TABLE in Atomic database, wait for table data to be finally dropped or detached.", 0) \
|
||||
M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \
|
||||
M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \
|
||||
M(Bool, optimize_count_from_files, true, "Optimize counting rows from files in supported input formats", 0) \
|
||||
M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \
|
||||
M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \
|
||||
M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \
|
||||
|
@ -126,6 +126,86 @@ namespace JSONUtils
|
||||
return fileSegmentationEngineJSONEachRowImpl<'[', ']'>(in, memory, min_bytes, min_rows, max_rows);
|
||||
}
|
||||
|
||||
template <const char opening_bracket, const char closing_bracket>
|
||||
void skipRowForJSONEachRowImpl(ReadBuffer & in)
|
||||
{
|
||||
size_t balance = 0;
|
||||
bool quotes = false;
|
||||
while (!in.eof())
|
||||
{
|
||||
if (quotes)
|
||||
{
|
||||
auto * pos = find_first_symbols<'\\', '"'>(in.position(), in.buffer().end());
|
||||
in.position() = pos;
|
||||
|
||||
if (in.position() > in.buffer().end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Position in buffer is out of bounds. There must be a bug.");
|
||||
else if (in.position() == in.buffer().end())
|
||||
continue;
|
||||
|
||||
if (*in.position() == '\\')
|
||||
{
|
||||
++in.position();
|
||||
if (!in.eof())
|
||||
++in.position();
|
||||
}
|
||||
else if (*in.position() == '"')
|
||||
{
|
||||
++in.position();
|
||||
quotes = false;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
auto * pos = find_first_symbols<opening_bracket, closing_bracket, '\\', '"'>(in.position(), in.buffer().end());
|
||||
in.position() = pos;
|
||||
|
||||
if (in.position() > in.buffer().end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Position in buffer is out of bounds. There must be a bug.");
|
||||
else if (in.position() == in.buffer().end())
|
||||
continue;
|
||||
|
||||
else if (*in.position() == opening_bracket)
|
||||
{
|
||||
++balance;
|
||||
++in.position();
|
||||
}
|
||||
else if (*in.position() == closing_bracket)
|
||||
{
|
||||
--balance;
|
||||
++in.position();
|
||||
}
|
||||
else if (*in.position() == '\\')
|
||||
{
|
||||
++in.position();
|
||||
if (!in.eof())
|
||||
++in.position();
|
||||
}
|
||||
else if (*in.position() == '"')
|
||||
{
|
||||
quotes = true;
|
||||
++in.position();
|
||||
}
|
||||
|
||||
if (balance == 0)
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected eof");
|
||||
|
||||
}
|
||||
|
||||
void skipRowForJSONEachRow(ReadBuffer & in)
|
||||
{
|
||||
return skipRowForJSONEachRowImpl<'{', '}'>(in);
|
||||
}
|
||||
|
||||
void skipRowForJSONCompactEachRow(ReadBuffer & in)
|
||||
{
|
||||
return skipRowForJSONEachRowImpl<'[', ']'>(in);
|
||||
}
|
||||
|
||||
NamesAndTypesList readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, const FormatSettings & settings, JSONInferenceInfo * inference_info)
|
||||
{
|
||||
skipWhitespaceIfAny(in);
|
||||
@ -612,8 +692,11 @@ namespace JSONUtils
|
||||
auto names_and_types = JSONUtils::readMetadata(in);
|
||||
for (const auto & [name, type] : names_and_types)
|
||||
{
|
||||
if (!header.has(name))
|
||||
continue;
|
||||
|
||||
auto header_type = header.getByName(name).type;
|
||||
if (header.has(name) && !type->equals(*header_type))
|
||||
if (!type->equals(*header_type))
|
||||
throw Exception(
|
||||
ErrorCodes::INCORRECT_DATA,
|
||||
"Type {} of column '{}' from metadata is not the same as type in header {}",
|
||||
|
@ -20,6 +20,9 @@ namespace JSONUtils
|
||||
std::pair<bool, size_t> fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_bytes, size_t max_rows);
|
||||
std::pair<bool, size_t> fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_bytes, size_t min_rows, size_t max_rows);
|
||||
|
||||
void skipRowForJSONEachRow(ReadBuffer & in);
|
||||
void skipRowForJSONCompactEachRow(ReadBuffer & in);
|
||||
|
||||
/// Read row in JSONEachRow format and try to determine type for each field.
|
||||
/// Return list of names and types.
|
||||
/// If cannot determine the type of some field, return nullptr for it.
|
||||
|
@ -42,11 +42,8 @@ private:
|
||||
using Base = CacheBase<UInt128, UncompressedCacheCell, UInt128TrivialHash, UncompressedSizeWeightFunction>;
|
||||
|
||||
public:
|
||||
explicit UncompressedCache(size_t max_size_in_bytes)
|
||||
: Base(max_size_in_bytes) {}
|
||||
|
||||
UncompressedCache(const String & uncompressed_cache_policy, size_t max_size_in_bytes)
|
||||
: Base(uncompressed_cache_policy, max_size_in_bytes) {}
|
||||
UncompressedCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio)
|
||||
: Base(cache_policy, max_size_in_bytes, 0, size_ratio) {}
|
||||
|
||||
/// Calculate key from path to file and offset.
|
||||
static UInt128 hash(const String & path_to_file, size_t offset)
|
||||
|
@ -175,7 +175,7 @@ public:
|
||||
private:
|
||||
CachePtr getHashTableStatsCache(const Params & params, const std::lock_guard<std::mutex> &)
|
||||
{
|
||||
if (!hash_table_stats || hash_table_stats->maxSize() != params.max_entries_for_hash_table_stats)
|
||||
if (!hash_table_stats || hash_table_stats->maxSizeInBytes() != params.max_entries_for_hash_table_stats)
|
||||
hash_table_stats = std::make_shared<Cache>(params.max_entries_for_hash_table_stats);
|
||||
return hash_table_stats;
|
||||
}
|
||||
|
@ -480,7 +480,7 @@ QueryCache::QueryCache(size_t max_size_in_bytes, size_t max_entries, size_t max_
|
||||
void QueryCache::updateConfiguration(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
cache.setMaxSize(max_size_in_bytes);
|
||||
cache.setMaxSizeInBytes(max_size_in_bytes);
|
||||
cache.setMaxCount(max_entries);
|
||||
max_entry_size_in_bytes = max_entry_size_in_bytes_;
|
||||
max_entry_size_in_rows = max_entry_size_in_rows_;
|
||||
@ -510,9 +510,9 @@ void QueryCache::clear()
|
||||
times_executed.clear();
|
||||
}
|
||||
|
||||
size_t QueryCache::weight() const
|
||||
size_t QueryCache::sizeInBytes() const
|
||||
{
|
||||
return cache.weight();
|
||||
return cache.sizeInBytes();
|
||||
}
|
||||
|
||||
size_t QueryCache::count() const
|
||||
|
@ -182,7 +182,7 @@ public:
|
||||
|
||||
void clear();
|
||||
|
||||
size_t weight() const;
|
||||
size_t sizeInBytes() const;
|
||||
size_t count() const;
|
||||
|
||||
/// Record new execution of query represented by key. Returns number of executions so far.
|
||||
|
@ -2268,14 +2268,14 @@ QueryStatusPtr Context::getProcessListElement() const
|
||||
}
|
||||
|
||||
|
||||
void Context::setUncompressedCache(const String & uncompressed_cache_policy, size_t max_size_in_bytes)
|
||||
void Context::setUncompressedCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio)
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (shared->uncompressed_cache)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Uncompressed cache has been already created.");
|
||||
|
||||
shared->uncompressed_cache = std::make_shared<UncompressedCache>(uncompressed_cache_policy, max_size_in_bytes);
|
||||
shared->uncompressed_cache = std::make_shared<UncompressedCache>(cache_policy, max_size_in_bytes, size_ratio);
|
||||
}
|
||||
|
||||
void Context::updateUncompressedCacheConfiguration(const Poco::Util::AbstractConfiguration & config)
|
||||
@ -2286,7 +2286,7 @@ void Context::updateUncompressedCacheConfiguration(const Poco::Util::AbstractCon
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Uncompressed cache was not created yet.");
|
||||
|
||||
size_t max_size_in_bytes = config.getUInt64("uncompressed_cache_size", DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE);
|
||||
shared->uncompressed_cache->setMaxSize(max_size_in_bytes);
|
||||
shared->uncompressed_cache->setMaxSizeInBytes(max_size_in_bytes);
|
||||
}
|
||||
|
||||
UncompressedCachePtr Context::getUncompressedCache() const
|
||||
@ -2303,14 +2303,14 @@ void Context::clearUncompressedCache() const
|
||||
shared->uncompressed_cache->clear();
|
||||
}
|
||||
|
||||
void Context::setMarkCache(const String & mark_cache_policy, size_t cache_size_in_bytes)
|
||||
void Context::setMarkCache(const String & cache_policy, size_t max_cache_size_in_bytes, double size_ratio)
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (shared->mark_cache)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Mark cache has been already created.");
|
||||
|
||||
shared->mark_cache = std::make_shared<MarkCache>(mark_cache_policy, cache_size_in_bytes);
|
||||
shared->mark_cache = std::make_shared<MarkCache>(cache_policy, max_cache_size_in_bytes, size_ratio);
|
||||
}
|
||||
|
||||
void Context::updateMarkCacheConfiguration(const Poco::Util::AbstractConfiguration & config)
|
||||
@ -2321,7 +2321,7 @@ void Context::updateMarkCacheConfiguration(const Poco::Util::AbstractConfigurati
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Mark cache was not created yet.");
|
||||
|
||||
size_t max_size_in_bytes = config.getUInt64("mark_cache_size", DEFAULT_MARK_CACHE_MAX_SIZE);
|
||||
shared->mark_cache->setMaxSize(max_size_in_bytes);
|
||||
shared->mark_cache->setMaxSizeInBytes(max_size_in_bytes);
|
||||
}
|
||||
|
||||
MarkCachePtr Context::getMarkCache() const
|
||||
@ -2353,14 +2353,14 @@ ThreadPool & Context::getLoadMarksThreadpool() const
|
||||
return *shared->load_marks_threadpool;
|
||||
}
|
||||
|
||||
void Context::setIndexUncompressedCache(size_t max_size_in_bytes)
|
||||
void Context::setIndexUncompressedCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio)
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (shared->index_uncompressed_cache)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Index uncompressed cache has been already created.");
|
||||
|
||||
shared->index_uncompressed_cache = std::make_shared<UncompressedCache>(max_size_in_bytes);
|
||||
shared->index_uncompressed_cache = std::make_shared<UncompressedCache>(cache_policy, max_size_in_bytes, size_ratio);
|
||||
}
|
||||
|
||||
void Context::updateIndexUncompressedCacheConfiguration(const Poco::Util::AbstractConfiguration & config)
|
||||
@ -2371,7 +2371,7 @@ void Context::updateIndexUncompressedCacheConfiguration(const Poco::Util::Abstra
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Index uncompressed cache was not created yet.");
|
||||
|
||||
size_t max_size_in_bytes = config.getUInt64("index_uncompressed_cache_size", DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE);
|
||||
shared->index_uncompressed_cache->setMaxSize(max_size_in_bytes);
|
||||
shared->index_uncompressed_cache->setMaxSizeInBytes(max_size_in_bytes);
|
||||
}
|
||||
|
||||
UncompressedCachePtr Context::getIndexUncompressedCache() const
|
||||
@ -2388,14 +2388,14 @@ void Context::clearIndexUncompressedCache() const
|
||||
shared->index_uncompressed_cache->clear();
|
||||
}
|
||||
|
||||
void Context::setIndexMarkCache(size_t cache_size_in_bytes)
|
||||
void Context::setIndexMarkCache(const String & cache_policy, size_t max_cache_size_in_bytes, double size_ratio)
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (shared->index_mark_cache)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Index mark cache has been already created.");
|
||||
|
||||
shared->index_mark_cache = std::make_shared<MarkCache>(cache_size_in_bytes);
|
||||
shared->index_mark_cache = std::make_shared<MarkCache>(cache_policy, max_cache_size_in_bytes, size_ratio);
|
||||
}
|
||||
|
||||
void Context::updateIndexMarkCacheConfiguration(const Poco::Util::AbstractConfiguration & config)
|
||||
@ -2406,7 +2406,7 @@ void Context::updateIndexMarkCacheConfiguration(const Poco::Util::AbstractConfig
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Index mark cache was not created yet.");
|
||||
|
||||
size_t max_size_in_bytes = config.getUInt64("index_mark_cache_size", DEFAULT_INDEX_MARK_CACHE_MAX_SIZE);
|
||||
shared->index_mark_cache->setMaxSize(max_size_in_bytes);
|
||||
shared->index_mark_cache->setMaxSizeInBytes(max_size_in_bytes);
|
||||
}
|
||||
|
||||
MarkCachePtr Context::getIndexMarkCache() const
|
||||
@ -2423,14 +2423,14 @@ void Context::clearIndexMarkCache() const
|
||||
shared->index_mark_cache->clear();
|
||||
}
|
||||
|
||||
void Context::setMMappedFileCache(size_t cache_size_in_num_entries)
|
||||
void Context::setMMappedFileCache(size_t max_cache_size_in_num_entries)
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (shared->mmap_cache)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Mapped file cache has been already created.");
|
||||
|
||||
shared->mmap_cache = std::make_shared<MMappedFileCache>(cache_size_in_num_entries);
|
||||
shared->mmap_cache = std::make_shared<MMappedFileCache>(max_cache_size_in_num_entries);
|
||||
}
|
||||
|
||||
void Context::updateMMappedFileCacheConfiguration(const Poco::Util::AbstractConfiguration & config)
|
||||
@ -2441,7 +2441,7 @@ void Context::updateMMappedFileCacheConfiguration(const Poco::Util::AbstractConf
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Mapped file cache was not created yet.");
|
||||
|
||||
size_t max_size_in_bytes = config.getUInt64("mmap_cache_size", DEFAULT_MMAP_CACHE_MAX_SIZE);
|
||||
shared->mmap_cache->setMaxSize(max_size_in_bytes);
|
||||
shared->mmap_cache->setMaxSizeInBytes(max_size_in_bytes);
|
||||
}
|
||||
|
||||
MMappedFileCachePtr Context::getMMappedFileCache() const
|
||||
|
@ -922,28 +922,28 @@ public:
|
||||
|
||||
/// --- Caches ------------------------------------------------------------------------------------------
|
||||
|
||||
void setUncompressedCache(const String & uncompressed_cache_policy, size_t max_size_in_bytes);
|
||||
void setUncompressedCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio);
|
||||
void updateUncompressedCacheConfiguration(const Poco::Util::AbstractConfiguration & config);
|
||||
std::shared_ptr<UncompressedCache> getUncompressedCache() const;
|
||||
void clearUncompressedCache() const;
|
||||
|
||||
void setMarkCache(const String & mark_cache_policy, size_t cache_size_in_bytes);
|
||||
void setMarkCache(const String & cache_policy, size_t max_cache_size_in_bytes, double size_ratio);
|
||||
void updateMarkCacheConfiguration(const Poco::Util::AbstractConfiguration & config);
|
||||
std::shared_ptr<MarkCache> getMarkCache() const;
|
||||
void clearMarkCache() const;
|
||||
ThreadPool & getLoadMarksThreadpool() const;
|
||||
|
||||
void setIndexUncompressedCache(size_t max_size_in_bytes);
|
||||
void setIndexUncompressedCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio);
|
||||
void updateIndexUncompressedCacheConfiguration(const Poco::Util::AbstractConfiguration & config);
|
||||
std::shared_ptr<UncompressedCache> getIndexUncompressedCache() const;
|
||||
void clearIndexUncompressedCache() const;
|
||||
|
||||
void setIndexMarkCache(size_t cache_size_in_bytes);
|
||||
void setIndexMarkCache(const String & cache_policy, size_t max_cache_size_in_bytes, double size_ratio);
|
||||
void updateIndexMarkCacheConfiguration(const Poco::Util::AbstractConfiguration & config);
|
||||
std::shared_ptr<MarkCache> getIndexMarkCache() const;
|
||||
void clearIndexMarkCache() const;
|
||||
|
||||
void setMMappedFileCache(size_t cache_size_in_num_entries);
|
||||
void setMMappedFileCache(size_t max_cache_size_in_num_entries);
|
||||
void updateMMappedFileCacheConfiguration(const Poco::Util::AbstractConfiguration & config);
|
||||
std::shared_ptr<MMappedFileCache> getMMappedFileCache() const;
|
||||
void clearMMappedFileCache() const;
|
||||
|
@ -2266,6 +2266,10 @@ std::optional<UInt64> InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle
|
||||
auto & query = getSelectQuery();
|
||||
if (!query.prewhere() && !query.where() && !context->getCurrentTransaction())
|
||||
{
|
||||
/// Some storages can optimize trivial count in read() method instead of totalRows() because it still can
|
||||
/// require reading some data (but much faster than reading columns).
|
||||
/// Set a special flag in query info so the storage will see it and optimize count in read() method.
|
||||
query_info.optimize_trivial_count = optimize_trivial_count;
|
||||
return storage->totalRows(settings);
|
||||
}
|
||||
else
|
||||
|
@ -68,13 +68,13 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values
|
||||
{
|
||||
if (auto mark_cache = getContext()->getMarkCache())
|
||||
{
|
||||
new_values["MarkCacheBytes"] = { mark_cache->weight(), "Total size of mark cache in bytes" };
|
||||
new_values["MarkCacheBytes"] = { mark_cache->sizeInBytes(), "Total size of mark cache in bytes" };
|
||||
new_values["MarkCacheFiles"] = { mark_cache->count(), "Total number of mark files cached in the mark cache" };
|
||||
}
|
||||
|
||||
if (auto uncompressed_cache = getContext()->getUncompressedCache())
|
||||
{
|
||||
new_values["UncompressedCacheBytes"] = { uncompressed_cache->weight(),
|
||||
new_values["UncompressedCacheBytes"] = { uncompressed_cache->sizeInBytes(),
|
||||
"Total size of uncompressed cache in bytes. Uncompressed cache does not usually improve the performance and should be mostly avoided." };
|
||||
new_values["UncompressedCacheCells"] = { uncompressed_cache->count(),
|
||||
"Total number of entries in the uncompressed cache. Each entry represents a decompressed block of data. Uncompressed cache does not usually improve performance and should be mostly avoided." };
|
||||
@ -82,13 +82,13 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values
|
||||
|
||||
if (auto index_mark_cache = getContext()->getIndexMarkCache())
|
||||
{
|
||||
new_values["IndexMarkCacheBytes"] = { index_mark_cache->weight(), "Total size of mark cache for secondary indices in bytes." };
|
||||
new_values["IndexMarkCacheBytes"] = { index_mark_cache->sizeInBytes(), "Total size of mark cache for secondary indices in bytes." };
|
||||
new_values["IndexMarkCacheFiles"] = { index_mark_cache->count(), "Total number of mark files cached in the mark cache for secondary indices." };
|
||||
}
|
||||
|
||||
if (auto index_uncompressed_cache = getContext()->getIndexUncompressedCache())
|
||||
{
|
||||
new_values["IndexUncompressedCacheBytes"] = { index_uncompressed_cache->weight(),
|
||||
new_values["IndexUncompressedCacheBytes"] = { index_uncompressed_cache->sizeInBytes(),
|
||||
"Total size of uncompressed cache in bytes for secondary indices. Uncompressed cache does not usually improve the performance and should be mostly avoided." };
|
||||
new_values["IndexUncompressedCacheCells"] = { index_uncompressed_cache->count(),
|
||||
"Total number of entries in the uncompressed cache for secondary indices. Each entry represents a decompressed block of data. Uncompressed cache does not usually improve performance and should be mostly avoided." };
|
||||
@ -104,7 +104,7 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values
|
||||
|
||||
if (auto query_cache = getContext()->getQueryCache())
|
||||
{
|
||||
new_values["QueryCacheBytes"] = { query_cache->weight(), "Total size of the query cache in bytes." };
|
||||
new_values["QueryCacheBytes"] = { query_cache->sizeInBytes(), "Total size of the query cache in bytes." };
|
||||
new_values["QueryCacheEntries"] = { query_cache->count(), "Total number of entries in the query cache." };
|
||||
}
|
||||
|
||||
@ -136,7 +136,7 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
if (auto * compiled_expression_cache = CompiledExpressionCacheFactory::instance().tryGetCache())
|
||||
{
|
||||
new_values["CompiledExpressionCacheBytes"] = { compiled_expression_cache->weight(),
|
||||
new_values["CompiledExpressionCacheBytes"] = { compiled_expression_cache->sizeInBytes(),
|
||||
"Total bytes used for the cache of JIT-compiled code." };
|
||||
new_values["CompiledExpressionCacheCount"] = { compiled_expression_cache->count(),
|
||||
"Total entries in the cache of JIT-compiled code." };
|
||||
|
@ -42,6 +42,7 @@
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/StorageDummy.h>
|
||||
#include <Storages/StorageDistributed.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
#include <Analyzer/Utils.h>
|
||||
@ -138,6 +139,84 @@ void checkStoragesSupportTransactions(const PlannerContextPtr & planner_context)
|
||||
}
|
||||
}
|
||||
|
||||
/** Storages can rely that filters that for storage will be available for analysis before
|
||||
* getQueryProcessingStage method will be called.
|
||||
*
|
||||
* StorageDistributed skip unused shards optimization relies on this.
|
||||
*
|
||||
* To collect filters that will be applied to specific table in case we have JOINs requires
|
||||
* to run query plan optimization pipeline.
|
||||
*
|
||||
* Algorithm:
|
||||
* 1. Replace all table expressions in query tree with dummy tables.
|
||||
* 2. Build query plan.
|
||||
* 3. Optimize query plan.
|
||||
* 4. Extract filters from ReadFromDummy query plan steps from query plan leaf nodes.
|
||||
*/
|
||||
void collectFiltersForAnalysis(const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context)
|
||||
{
|
||||
bool collect_filters = false;
|
||||
|
||||
for (auto & [table_expression, table_expression_data] : planner_context->getTableExpressionNodeToData())
|
||||
{
|
||||
auto * table_node = table_expression->as<TableNode>();
|
||||
auto * table_function_node = table_expression->as<TableFunctionNode>();
|
||||
if (!table_node && !table_function_node)
|
||||
continue;
|
||||
|
||||
const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage();
|
||||
if (typeid_cast<const StorageDistributed *>(storage.get()))
|
||||
{
|
||||
collect_filters = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!collect_filters)
|
||||
return;
|
||||
|
||||
ResultReplacementMap replacement_map;
|
||||
auto updated_query_tree = replaceTableExpressionsWithDummyTables(query_tree, planner_context->getQueryContext(), &replacement_map);
|
||||
|
||||
std::unordered_map<const IStorage *, TableExpressionData *> dummy_storage_to_table_expression_data;
|
||||
|
||||
for (auto & [from_table_expression, dummy_table_expression] : replacement_map)
|
||||
{
|
||||
auto * dummy_storage = dummy_table_expression->as<TableNode &>().getStorage().get();
|
||||
auto * table_expression_data = &planner_context->getTableExpressionDataOrThrow(from_table_expression);
|
||||
dummy_storage_to_table_expression_data.emplace(dummy_storage, table_expression_data);
|
||||
}
|
||||
|
||||
const auto & query_context = planner_context->getQueryContext();
|
||||
|
||||
SelectQueryOptions select_query_options;
|
||||
Planner planner(updated_query_tree, select_query_options);
|
||||
planner.buildQueryPlanIfNeeded();
|
||||
|
||||
auto & result_query_plan = planner.getQueryPlan();
|
||||
|
||||
auto optimization_settings = QueryPlanOptimizationSettings::fromContext(query_context);
|
||||
result_query_plan.optimize(optimization_settings);
|
||||
|
||||
std::vector<QueryPlan::Node *> nodes_to_process;
|
||||
nodes_to_process.push_back(result_query_plan.getRootNode());
|
||||
|
||||
while (!nodes_to_process.empty())
|
||||
{
|
||||
const auto * node_to_process = nodes_to_process.back();
|
||||
nodes_to_process.pop_back();
|
||||
nodes_to_process.insert(nodes_to_process.end(), node_to_process->children.begin(), node_to_process->children.end());
|
||||
|
||||
auto * read_from_dummy = typeid_cast<ReadFromDummy *>(node_to_process->step.get());
|
||||
if (!read_from_dummy)
|
||||
continue;
|
||||
|
||||
auto filter_actions = ActionsDAG::buildFilterActionsDAG(read_from_dummy->getFilterNodes().nodes, {}, query_context);
|
||||
auto & table_expression_data = dummy_storage_to_table_expression_data.at(&read_from_dummy->getStorage());
|
||||
table_expression_data->setFilterActions(std::move(filter_actions));
|
||||
}
|
||||
}
|
||||
|
||||
/// Extend lifetime of query context, storages, and table locks
|
||||
void extendQueryContextAndStoragesLifetime(QueryPlan & query_plan, const PlannerContextPtr & planner_context)
|
||||
{
|
||||
@ -1226,6 +1305,9 @@ void Planner::buildPlanForQueryNode()
|
||||
collectSets(query_tree, *planner_context);
|
||||
collectTableExpressionData(query_tree, planner_context);
|
||||
|
||||
if (!select_query_options.only_analyze)
|
||||
collectFiltersForAnalysis(query_tree, planner_context);
|
||||
|
||||
const auto & settings = query_context->getSettingsRef();
|
||||
|
||||
/// Check support for JOIN for parallel replicas with custom key
|
||||
|
@ -186,6 +186,7 @@ NameAndTypePair chooseSmallestColumnToReadFromStorage(const StoragePtr & storage
|
||||
|
||||
bool applyTrivialCountIfPossible(
|
||||
QueryPlan & query_plan,
|
||||
SelectQueryInfo & select_query_info,
|
||||
const TableNode & table_node,
|
||||
const QueryTreeNodePtr & query_tree,
|
||||
ContextMutablePtr & query_context,
|
||||
@ -243,6 +244,11 @@ bool applyTrivialCountIfPossible(
|
||||
if (!count_func)
|
||||
return false;
|
||||
|
||||
/// Some storages can optimize trivial count in read() method instead of totalRows() because it still can
|
||||
/// require reading some data (but much faster than reading columns).
|
||||
/// Set a special flag in query info so the storage will see it and optimize count in read() method.
|
||||
select_query_info.optimize_trivial_count = true;
|
||||
|
||||
/// Get number of rows
|
||||
std::optional<UInt64> num_rows = storage->totalRows(settings);
|
||||
if (!num_rows)
|
||||
@ -505,7 +511,7 @@ FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage,
|
||||
}
|
||||
|
||||
JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression,
|
||||
const SelectQueryInfo & select_query_info,
|
||||
SelectQueryInfo & select_query_info,
|
||||
const SelectQueryOptions & select_query_options,
|
||||
PlannerContextPtr & planner_context,
|
||||
bool is_single_table_expression,
|
||||
@ -538,6 +544,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
|
||||
auto table_expression_query_info = select_query_info;
|
||||
table_expression_query_info.table_expression = table_expression;
|
||||
table_expression_query_info.filter_actions_dag = table_expression_data.getFilterActions();
|
||||
|
||||
size_t max_streams = settings.max_threads;
|
||||
size_t max_threads_execute_query = settings.max_threads;
|
||||
@ -651,7 +658,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
is_single_table_expression &&
|
||||
table_node &&
|
||||
select_query_info.has_aggregates &&
|
||||
applyTrivialCountIfPossible(query_plan, *table_node, select_query_info.query_tree, planner_context->getMutableQueryContext(), table_expression_data.getColumnNames());
|
||||
applyTrivialCountIfPossible(query_plan, select_query_info, *table_node, select_query_info.query_tree, planner_context->getMutableQueryContext(), table_expression_data.getColumnNames());
|
||||
|
||||
if (is_trivial_count_applied)
|
||||
{
|
||||
@ -1389,7 +1396,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
|
||||
}
|
||||
|
||||
JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
|
||||
const SelectQueryInfo & select_query_info,
|
||||
SelectQueryInfo & select_query_info,
|
||||
SelectQueryOptions & select_query_options,
|
||||
const ColumnIdentifierSet & outer_scope_columns,
|
||||
PlannerContextPtr & planner_context)
|
||||
|
@ -19,7 +19,7 @@ struct JoinTreeQueryPlan
|
||||
|
||||
/// Build JOIN TREE query plan for query node
|
||||
JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
|
||||
const SelectQueryInfo & select_query_info,
|
||||
SelectQueryInfo & select_query_info,
|
||||
SelectQueryOptions & select_query_options,
|
||||
const ColumnIdentifierSet & outer_scope_columns,
|
||||
PlannerContextPtr & planner_context);
|
||||
|
@ -355,24 +355,52 @@ QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, con
|
||||
return function_node;
|
||||
}
|
||||
|
||||
QueryTreeNodePtr replaceTablesAndTableFunctionsWithDummyTables(const QueryTreeNodePtr & query_node,
|
||||
QueryTreeNodePtr replaceTableExpressionsWithDummyTables(const QueryTreeNodePtr & query_node,
|
||||
const ContextPtr & context,
|
||||
ResultReplacementMap * result_replacement_map)
|
||||
{
|
||||
auto & query_node_typed = query_node->as<QueryNode &>();
|
||||
auto table_expressions = extractTableExpressions(query_node_typed.getJoinTree());
|
||||
std::unordered_map<const IQueryTreeNode *, QueryTreeNodePtr> replacement_map;
|
||||
size_t subquery_index = 0;
|
||||
|
||||
for (auto & table_expression : table_expressions)
|
||||
{
|
||||
auto * table_node = table_expression->as<TableNode>();
|
||||
auto * table_function_node = table_expression->as<TableFunctionNode>();
|
||||
if (!table_node && !table_function_node)
|
||||
continue;
|
||||
auto * subquery_node = table_expression->as<QueryNode>();
|
||||
auto * union_node = table_expression->as<UnionNode>();
|
||||
|
||||
StoragePtr storage_dummy;
|
||||
|
||||
if (table_node || table_function_node)
|
||||
{
|
||||
const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot();
|
||||
auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals();
|
||||
|
||||
storage_dummy
|
||||
= std::make_shared<StorageDummy>(storage_snapshot->storage.getStorageID(), ColumnsDescription(storage_snapshot->getColumns(get_column_options)));
|
||||
}
|
||||
else if (subquery_node || union_node)
|
||||
{
|
||||
const auto & subquery_projection_columns
|
||||
= subquery_node ? subquery_node->getProjectionColumns() : union_node->computeProjectionColumns();
|
||||
|
||||
NameSet unique_column_names;
|
||||
NamesAndTypes storage_dummy_columns;
|
||||
storage_dummy_columns.reserve(subquery_projection_columns.size());
|
||||
|
||||
for (const auto & projection_column : subquery_projection_columns)
|
||||
{
|
||||
auto [_, inserted] = unique_column_names.insert(projection_column.name);
|
||||
if (inserted)
|
||||
storage_dummy_columns.emplace_back(projection_column);
|
||||
}
|
||||
|
||||
storage_dummy = std::make_shared<StorageDummy>(StorageID{"dummy", "subquery_" + std::to_string(subquery_index)}, ColumnsDescription(storage_dummy_columns));
|
||||
++subquery_index;
|
||||
}
|
||||
|
||||
const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot();
|
||||
auto storage_dummy = std::make_shared<StorageDummy>(storage_snapshot->storage.getStorageID(),
|
||||
storage_snapshot->metadata->getColumns());
|
||||
auto dummy_table_node = std::make_shared<TableNode>(std::move(storage_dummy), context);
|
||||
|
||||
if (result_replacement_map)
|
||||
|
@ -65,9 +65,9 @@ bool queryHasWithTotalsInAnySubqueryInJoinTree(const QueryTreeNodePtr & query_no
|
||||
/// Returns `and` function node that has condition nodes as its arguments
|
||||
QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, const ContextPtr & context);
|
||||
|
||||
/// Replace tables nodes and table function nodes with dummy table nodes
|
||||
/// Replace table expressions from query JOIN TREE with dummy tables
|
||||
using ResultReplacementMap = std::unordered_map<QueryTreeNodePtr, QueryTreeNodePtr>;
|
||||
QueryTreeNodePtr replaceTablesAndTableFunctionsWithDummyTables(const QueryTreeNodePtr & query_node,
|
||||
QueryTreeNodePtr replaceTableExpressionsWithDummyTables(const QueryTreeNodePtr & query_node,
|
||||
const ContextPtr & context,
|
||||
ResultReplacementMap * result_replacement_map = nullptr);
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Columns/ColumnSparse.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
|
||||
namespace DB
|
||||
@ -221,4 +222,16 @@ void convertToFullIfSparse(Chunk & chunk)
|
||||
chunk.setColumns(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
Chunk cloneConstWithDefault(const Chunk & chunk, size_t num_rows)
|
||||
{
|
||||
auto columns = chunk.cloneEmptyColumns();
|
||||
for (auto & column : columns)
|
||||
{
|
||||
column->insertDefault();
|
||||
column = ColumnConst::create(std::move(column), num_rows);
|
||||
}
|
||||
|
||||
return Chunk(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -156,4 +156,7 @@ private:
|
||||
void convertToFullIfConst(Chunk & chunk);
|
||||
void convertToFullIfSparse(Chunk & chunk);
|
||||
|
||||
/// Creates chunks with same columns but makes them const with default value and specified number of rows.
|
||||
Chunk cloneConstWithDefault(const Chunk & chunk, size_t num_rows);
|
||||
|
||||
}
|
||||
|
@ -28,4 +28,10 @@ void IInputFormat::setReadBuffer(ReadBuffer & in_)
|
||||
in = &in_;
|
||||
}
|
||||
|
||||
Chunk IInputFormat::getChunkForCount(size_t rows)
|
||||
{
|
||||
const auto & header = getPort().getHeader();
|
||||
return cloneConstWithDefault(Chunk{header.getColumns(), 0}, rows);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -61,11 +61,17 @@ public:
|
||||
|
||||
virtual size_t getApproxBytesReadForChunk() const { return 0; }
|
||||
|
||||
void needOnlyCount() { need_only_count = true; }
|
||||
|
||||
protected:
|
||||
virtual Chunk getChunkForCount(size_t rows);
|
||||
|
||||
ColumnMappingPtr column_mapping{};
|
||||
|
||||
InputFormatErrorsLoggerPtr errors_logger;
|
||||
|
||||
bool need_only_count = false;
|
||||
|
||||
private:
|
||||
/// Number of currently parsed chunk (if parallel parsing is enabled)
|
||||
size_t current_unit_number = 0;
|
||||
|
@ -113,6 +113,19 @@ Chunk IRowInputFormat::generate()
|
||||
size_t chunk_start_offset = getDataOffsetMaybeCompressed(getReadBuffer());
|
||||
try
|
||||
{
|
||||
if (need_only_count && supportsCountRows())
|
||||
{
|
||||
num_rows = countRows(params.max_block_size);
|
||||
if (num_rows == 0)
|
||||
{
|
||||
readSuffix();
|
||||
return {};
|
||||
}
|
||||
total_rows += num_rows;
|
||||
approx_bytes_read_for_chunk = getDataOffsetMaybeCompressed(getReadBuffer()) - chunk_start_offset;
|
||||
return getChunkForCount(num_rows);
|
||||
}
|
||||
|
||||
RowReadExtension info;
|
||||
bool continue_reading = true;
|
||||
for (size_t rows = 0; rows < params.max_block_size && continue_reading; ++rows)
|
||||
@ -262,7 +275,7 @@ Chunk IRowInputFormat::generate()
|
||||
|
||||
void IRowInputFormat::syncAfterError()
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method syncAfterError is not implemented for input format");
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method syncAfterError is not implemented for input format {}", getName());
|
||||
}
|
||||
|
||||
void IRowInputFormat::resetParser()
|
||||
@ -273,5 +286,10 @@ void IRowInputFormat::resetParser()
|
||||
block_missing_values.clear();
|
||||
}
|
||||
|
||||
size_t IRowInputFormat::countRows(size_t)
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method countRows is not implemented for input format {}", getName());
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -52,6 +52,13 @@ protected:
|
||||
*/
|
||||
virtual bool readRow(MutableColumns & columns, RowReadExtension & extra) = 0;
|
||||
|
||||
/// Count some rows. Called in a loop until it returns 0, and the return values are added up.
|
||||
/// `max_block_size` is the recommended number of rows after which to stop, if the implementation
|
||||
/// involves scanning the data. If the implementation just takes the count from metadata,
|
||||
/// `max_block_size` can be ignored.
|
||||
virtual size_t countRows(size_t max_block_size);
|
||||
virtual bool supportsCountRows() const { return false; }
|
||||
|
||||
virtual void readPrefix() {} /// delimiter before begin of result
|
||||
virtual void readSuffix() {} /// delimiter after end of result
|
||||
|
||||
|
@ -45,6 +45,9 @@ Chunk ArrowBlockInputFormat::generate()
|
||||
batch_result = stream_reader->Next();
|
||||
if (batch_result.ok() && !(*batch_result))
|
||||
return res;
|
||||
|
||||
if (need_only_count && batch_result.ok())
|
||||
return getChunkForCount((*batch_result)->num_rows());
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -57,6 +60,15 @@ Chunk ArrowBlockInputFormat::generate()
|
||||
if (record_batch_current >= record_batch_total)
|
||||
return res;
|
||||
|
||||
if (need_only_count)
|
||||
{
|
||||
auto rows = file_reader->RecordBatchCountRows(record_batch_current++);
|
||||
if (!rows.ok())
|
||||
throw ParsingException(
|
||||
ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading batch of Arrow data: {}", rows.status().ToString());
|
||||
return getChunkForCount(*rows);
|
||||
}
|
||||
|
||||
batch_result = file_reader->ReadRecordBatch(record_batch_current);
|
||||
}
|
||||
|
||||
|
@ -912,6 +912,19 @@ bool AvroRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &ext
|
||||
return false;
|
||||
}
|
||||
|
||||
size_t AvroRowInputFormat::countRows(size_t max_block_size)
|
||||
{
|
||||
size_t num_rows = 0;
|
||||
while (file_reader_ptr->hasMore() && num_rows < max_block_size)
|
||||
{
|
||||
file_reader_ptr->decr();
|
||||
file_reader_ptr->decoder().drain();
|
||||
++num_rows;
|
||||
}
|
||||
|
||||
return num_rows;
|
||||
}
|
||||
|
||||
class AvroConfluentRowInputFormat::SchemaRegistry
|
||||
{
|
||||
public:
|
||||
|
@ -160,6 +160,9 @@ private:
|
||||
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
|
||||
void readPrefix() override;
|
||||
|
||||
bool supportsCountRows() const override { return true; }
|
||||
size_t countRows(size_t max_block_size) override;
|
||||
|
||||
std::unique_ptr<avro::DataFileReaderBase> file_reader_ptr;
|
||||
std::unique_ptr<AvroDeserializer> deserializer_ptr;
|
||||
FormatSettings format_settings;
|
||||
|
@ -372,6 +372,9 @@ void BSONEachRowRowInputFormat::readArray(IColumn & column, const DataTypePtr &
|
||||
size_t document_start = in->count();
|
||||
BSONSizeT document_size;
|
||||
readBinary(document_size, *in);
|
||||
if (document_size < sizeof(BSONSizeT) + sizeof(BSON_DOCUMENT_END))
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid document size: {}", document_size);
|
||||
|
||||
while (in->count() - document_start + sizeof(BSON_DOCUMENT_END) != document_size)
|
||||
{
|
||||
auto nested_bson_type = getBSONType(readBSONType(*in));
|
||||
@ -399,6 +402,9 @@ void BSONEachRowRowInputFormat::readTuple(IColumn & column, const DataTypePtr &
|
||||
size_t document_start = in->count();
|
||||
BSONSizeT document_size;
|
||||
readBinary(document_size, *in);
|
||||
if (document_size < sizeof(BSONSizeT) + sizeof(BSON_DOCUMENT_END))
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid document size: {}", document_size);
|
||||
|
||||
while (in->count() - document_start + sizeof(BSON_DOCUMENT_END) != document_size)
|
||||
{
|
||||
auto nested_bson_type = getBSONType(readBSONType(*in));
|
||||
@ -457,6 +463,9 @@ void BSONEachRowRowInputFormat::readMap(IColumn & column, const DataTypePtr & da
|
||||
size_t document_start = in->count();
|
||||
BSONSizeT document_size;
|
||||
readBinary(document_size, *in);
|
||||
if (document_size < sizeof(BSONSizeT) + sizeof(BSON_DOCUMENT_END))
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid document size: {}", document_size);
|
||||
|
||||
while (in->count() - document_start + sizeof(BSON_DOCUMENT_END) != document_size)
|
||||
{
|
||||
auto nested_bson_type = getBSONType(readBSONType(*in));
|
||||
@ -696,6 +705,8 @@ static void skipBSONField(ReadBuffer & in, BSONType type)
|
||||
{
|
||||
BSONSizeT size;
|
||||
readBinary(size, in);
|
||||
if (size < sizeof(BSONSizeT) + sizeof(BSON_DOCUMENT_END))
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid document size: {}", size);
|
||||
in.ignore(size - sizeof(size));
|
||||
break;
|
||||
}
|
||||
@ -735,6 +746,8 @@ static void skipBSONField(ReadBuffer & in, BSONType type)
|
||||
{
|
||||
BSONSizeT size;
|
||||
readBinary(size, in);
|
||||
if (size < sizeof(BSONSizeT))
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid java code_w_scope size: {}", size);
|
||||
in.ignore(size - sizeof(size));
|
||||
break;
|
||||
}
|
||||
@ -775,6 +788,9 @@ bool BSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi
|
||||
|
||||
current_document_start = in->count();
|
||||
readBinary(current_document_size, *in);
|
||||
if (current_document_size < sizeof(BSONSizeT) + sizeof(BSON_DOCUMENT_END))
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid document size: {}", current_document_size);
|
||||
|
||||
while (in->count() - current_document_start + sizeof(BSON_DOCUMENT_END) != current_document_size)
|
||||
{
|
||||
auto type = getBSONType(readBSONType(*in));
|
||||
@ -822,6 +838,22 @@ void BSONEachRowRowInputFormat::resetParser()
|
||||
prev_positions.clear();
|
||||
}
|
||||
|
||||
size_t BSONEachRowRowInputFormat::countRows(size_t max_block_size)
|
||||
{
|
||||
size_t num_rows = 0;
|
||||
BSONSizeT document_size;
|
||||
while (!in->eof() && num_rows < max_block_size)
|
||||
{
|
||||
readBinary(document_size, *in);
|
||||
if (document_size < sizeof(BSONSizeT) + sizeof(BSON_DOCUMENT_END))
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid document size: {}", document_size);
|
||||
in->ignore(document_size - sizeof(BSONSizeT));
|
||||
++num_rows;
|
||||
}
|
||||
|
||||
return num_rows;
|
||||
}
|
||||
|
||||
BSONEachRowSchemaReader::BSONEachRowSchemaReader(ReadBuffer & in_, const FormatSettings & settings_)
|
||||
: IRowWithNamesSchemaReader(in_, settings_)
|
||||
{
|
||||
|
@ -64,6 +64,9 @@ private:
|
||||
bool allowSyncAfterError() const override { return true; }
|
||||
void syncAfterError() override;
|
||||
|
||||
bool supportsCountRows() const override { return true; }
|
||||
size_t countRows(size_t max_block_size) override;
|
||||
|
||||
size_t columnIndex(const StringRef & name, size_t key_index);
|
||||
|
||||
using ColumnReader = std::function<void(StringRef name, BSONType type)>;
|
||||
|
@ -114,6 +114,66 @@ void CSVRowInputFormat::resetParser()
|
||||
buf->reset();
|
||||
}
|
||||
|
||||
void CSVFormatReader::skipRow()
|
||||
{
|
||||
bool quotes = false;
|
||||
ReadBuffer & istr = *buf;
|
||||
|
||||
while (!istr.eof())
|
||||
{
|
||||
if (quotes)
|
||||
{
|
||||
auto * pos = find_first_symbols<'"'>(istr.position(), istr.buffer().end());
|
||||
istr.position() = pos;
|
||||
|
||||
if (pos > istr.buffer().end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Position in buffer is out of bounds. There must be a bug.");
|
||||
else if (pos == istr.buffer().end())
|
||||
continue;
|
||||
else if (*pos == '"')
|
||||
{
|
||||
++istr.position();
|
||||
if (!istr.eof() && *istr.position() == '"')
|
||||
++istr.position();
|
||||
else
|
||||
quotes = false;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
auto * pos = find_first_symbols<'"', '\r', '\n'>(istr.position(), istr.buffer().end());
|
||||
istr.position() = pos;
|
||||
|
||||
if (pos > istr.buffer().end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Position in buffer is out of bounds. There must be a bug.");
|
||||
else if (pos == istr.buffer().end())
|
||||
continue;
|
||||
|
||||
if (*pos == '"')
|
||||
{
|
||||
quotes = true;
|
||||
++istr.position();
|
||||
continue;
|
||||
}
|
||||
|
||||
if (*pos == '\n')
|
||||
{
|
||||
++istr.position();
|
||||
if (!istr.eof() && *istr.position() == '\r')
|
||||
++istr.position();
|
||||
return;
|
||||
}
|
||||
else if (*pos == '\r')
|
||||
{
|
||||
++istr.position();
|
||||
if (!istr.eof() && *pos == '\n')
|
||||
++pos;
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static void skipEndOfLine(ReadBuffer & in)
|
||||
{
|
||||
/// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic)
|
||||
|
@ -40,6 +40,8 @@ private:
|
||||
bool allowSyncAfterError() const override { return true; }
|
||||
void syncAfterError() override;
|
||||
|
||||
bool supportsCountRows() const override { return true; }
|
||||
|
||||
protected:
|
||||
std::shared_ptr<PeekableReadBuffer> buf;
|
||||
};
|
||||
@ -59,6 +61,8 @@ public:
|
||||
|
||||
bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override;
|
||||
|
||||
void skipRow() override;
|
||||
|
||||
void skipField(size_t /*file_column*/) override { skipField(); }
|
||||
void skipField();
|
||||
|
||||
|
@ -27,7 +27,7 @@ CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in_, Block header_
|
||||
serializer = std::make_unique<CapnProtoSerializer>(header.getDataTypes(), header.getNames(), schema, format_settings.capn_proto);
|
||||
}
|
||||
|
||||
kj::Array<capnp::word> CapnProtoRowInputFormat::readMessage()
|
||||
std::pair<kj::Array<capnp::word>, size_t> CapnProtoRowInputFormat::readMessagePrefix()
|
||||
{
|
||||
uint32_t segment_count;
|
||||
in->readStrict(reinterpret_cast<char*>(&segment_count), sizeof(uint32_t));
|
||||
@ -48,6 +48,14 @@ kj::Array<capnp::word> CapnProtoRowInputFormat::readMessage()
|
||||
for (size_t i = 0; i <= segment_count; ++i)
|
||||
in->readStrict(prefix_chars.begin() + ((i + 1) * sizeof(uint32_t)), sizeof(uint32_t));
|
||||
|
||||
return {std::move(prefix), prefix_size};
|
||||
}
|
||||
|
||||
kj::Array<capnp::word> CapnProtoRowInputFormat::readMessage()
|
||||
{
|
||||
auto [prefix, prefix_size] = readMessagePrefix();
|
||||
auto prefix_chars = prefix.asChars();
|
||||
|
||||
// calculate size of message
|
||||
const auto expected_words = capnp::expectedSizeInWordsFromPrefix(prefix);
|
||||
const auto expected_bytes = expected_words * sizeof(capnp::word);
|
||||
@ -62,6 +70,18 @@ kj::Array<capnp::word> CapnProtoRowInputFormat::readMessage()
|
||||
return msg;
|
||||
}
|
||||
|
||||
void CapnProtoRowInputFormat::skipMessage()
|
||||
{
|
||||
auto [prefix, prefix_size] = readMessagePrefix();
|
||||
|
||||
// calculate size of message
|
||||
const auto expected_bytes = capnp::expectedSizeInWordsFromPrefix(prefix) * sizeof(capnp::word);
|
||||
const auto data_size = expected_bytes - prefix_size;
|
||||
|
||||
// skip full message
|
||||
in->ignore(data_size);
|
||||
}
|
||||
|
||||
bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &)
|
||||
{
|
||||
if (in->eof())
|
||||
@ -82,6 +102,18 @@ bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t CapnProtoRowInputFormat::countRows(size_t max_block_size)
|
||||
{
|
||||
size_t num_rows = 0;
|
||||
while (!in->eof() && num_rows < max_block_size)
|
||||
{
|
||||
skipMessage();
|
||||
++num_rows;
|
||||
}
|
||||
|
||||
return num_rows;
|
||||
}
|
||||
|
||||
CapnProtoSchemaReader::CapnProtoSchemaReader(const FormatSettings & format_settings_) : format_settings(format_settings_)
|
||||
{
|
||||
}
|
||||
|
@ -31,7 +31,12 @@ public:
|
||||
private:
|
||||
bool readRow(MutableColumns & columns, RowReadExtension &) override;
|
||||
|
||||
bool supportsCountRows() const override { return true; }
|
||||
size_t countRows(size_t max_block_size) override;
|
||||
|
||||
std::pair<kj::Array<capnp::word>, size_t> readMessagePrefix();
|
||||
kj::Array<capnp::word> readMessage();
|
||||
void skipMessage();
|
||||
|
||||
std::shared_ptr<CapnProtoSchemaParser> parser;
|
||||
capnp::StructSchema schema;
|
||||
|
@ -221,19 +221,34 @@ std::vector<String> CustomSeparatedFormatReader::readRowImpl()
|
||||
return values;
|
||||
}
|
||||
|
||||
void CustomSeparatedFormatReader::skipHeaderRow()
|
||||
void CustomSeparatedFormatReader::skipRow()
|
||||
{
|
||||
skipRowStartDelimiter();
|
||||
bool first = true;
|
||||
do
|
||||
{
|
||||
if (!first)
|
||||
skipFieldDelimiter();
|
||||
first = false;
|
||||
|
||||
skipField();
|
||||
/// If the number of columns in row is unknown,
|
||||
/// we should check for end of row after each field.
|
||||
if (columns == 0 || allowVariableNumberOfColumns())
|
||||
{
|
||||
bool first = true;
|
||||
do
|
||||
{
|
||||
if (!first)
|
||||
skipFieldDelimiter();
|
||||
first = false;
|
||||
|
||||
skipField();
|
||||
}
|
||||
while (!checkForEndOfRow());
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i != columns; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
skipFieldDelimiter();
|
||||
skipField();
|
||||
}
|
||||
}
|
||||
while (!checkForEndOfRow());
|
||||
|
||||
skipRowEndDelimiter();
|
||||
}
|
||||
|
@ -33,6 +33,8 @@ private:
|
||||
void syncAfterError() override;
|
||||
void readPrefix() override;
|
||||
|
||||
bool supportsCountRows() const override { return true; }
|
||||
|
||||
std::unique_ptr<PeekableReadBuffer> buf;
|
||||
bool ignore_spaces;
|
||||
};
|
||||
@ -48,9 +50,9 @@ public:
|
||||
|
||||
void skipField(size_t /*file_column*/) override { skipField(); }
|
||||
void skipField();
|
||||
void skipNames() override { skipHeaderRow(); }
|
||||
void skipTypes() override { skipHeaderRow(); }
|
||||
void skipHeaderRow();
|
||||
void skipNames() override { skipRow(); }
|
||||
void skipTypes() override { skipRow(); }
|
||||
void skipRow() override;
|
||||
|
||||
void skipPrefixBeforeHeader() override;
|
||||
void skipRowStartDelimiter() override;
|
||||
|
@ -15,11 +15,11 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_)
|
||||
: JSONAsRowInputFormat(header_, std::make_unique<PeekableReadBuffer>(in_), params_) {}
|
||||
JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_)
|
||||
: JSONAsRowInputFormat(header_, std::make_unique<PeekableReadBuffer>(in_), params_, format_settings_) {}
|
||||
|
||||
JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, Params params_) :
|
||||
IRowInputFormat(header_, *buf_, std::move(params_)), buf(std::move(buf_))
|
||||
JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, Params params_, const FormatSettings & format_settings_) :
|
||||
JSONEachRowRowInputFormat(*buf_, header_, std::move(params_), format_settings_, false), buf(std::move(buf_))
|
||||
{
|
||||
if (header_.columns() > 1)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
@ -29,39 +29,10 @@ JSONAsRowInputFormat::JSONAsRowInputFormat(const Block & header_, std::unique_pt
|
||||
|
||||
void JSONAsRowInputFormat::resetParser()
|
||||
{
|
||||
IRowInputFormat::resetParser();
|
||||
JSONEachRowRowInputFormat::resetParser();
|
||||
buf->reset();
|
||||
}
|
||||
|
||||
void JSONAsRowInputFormat::readPrefix()
|
||||
{
|
||||
/// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it.
|
||||
skipBOMIfExists(*buf);
|
||||
|
||||
skipWhitespaceIfAny(*buf);
|
||||
if (!buf->eof() && *buf->position() == '[')
|
||||
{
|
||||
++buf->position();
|
||||
data_in_square_brackets = true;
|
||||
}
|
||||
}
|
||||
|
||||
void JSONAsRowInputFormat::readSuffix()
|
||||
{
|
||||
skipWhitespaceIfAny(*buf);
|
||||
if (data_in_square_brackets)
|
||||
{
|
||||
assertChar(']', *buf);
|
||||
skipWhitespaceIfAny(*buf);
|
||||
data_in_square_brackets = false;
|
||||
}
|
||||
if (!buf->eof() && *buf->position() == ';')
|
||||
{
|
||||
++buf->position();
|
||||
skipWhitespaceIfAny(*buf);
|
||||
}
|
||||
assertEOF(*buf);
|
||||
}
|
||||
|
||||
bool JSONAsRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &)
|
||||
{
|
||||
@ -104,8 +75,8 @@ void JSONAsRowInputFormat::setReadBuffer(ReadBuffer & in_)
|
||||
|
||||
|
||||
JSONAsStringRowInputFormat::JSONAsStringRowInputFormat(
|
||||
const Block & header_, ReadBuffer & in_, Params params_)
|
||||
: JSONAsRowInputFormat(header_, in_, params_)
|
||||
const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_)
|
||||
: JSONAsRowInputFormat(header_, in_, params_, format_settings_)
|
||||
{
|
||||
if (!isString(removeNullable(removeLowCardinality(header_.getByPosition(0).type))))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
@ -193,8 +164,7 @@ void JSONAsStringRowInputFormat::readJSONObject(IColumn & column)
|
||||
|
||||
JSONAsObjectRowInputFormat::JSONAsObjectRowInputFormat(
|
||||
const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_)
|
||||
: JSONAsRowInputFormat(header_, in_, params_)
|
||||
, format_settings(format_settings_)
|
||||
: JSONAsRowInputFormat(header_, in_, params_, format_settings_)
|
||||
{
|
||||
if (!isObject(header_.getByPosition(0).type))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
@ -207,6 +177,13 @@ void JSONAsObjectRowInputFormat::readJSONObject(IColumn & column)
|
||||
serializations[0]->deserializeTextJSON(column, *buf, format_settings);
|
||||
}
|
||||
|
||||
Chunk JSONAsObjectRowInputFormat::getChunkForCount(size_t rows)
|
||||
{
|
||||
auto object_type = getPort().getHeader().getDataTypes()[0];
|
||||
ColumnPtr column = object_type->createColumnConst(rows, Field(Object()));
|
||||
return Chunk({std::move(column)}, rows);
|
||||
}
|
||||
|
||||
JSONAsObjectExternalSchemaReader::JSONAsObjectExternalSchemaReader(const FormatSettings & settings)
|
||||
{
|
||||
if (!settings.json.allow_object_type)
|
||||
@ -222,9 +199,9 @@ void registerInputFormatJSONAsString(FormatFactory & factory)
|
||||
ReadBuffer & buf,
|
||||
const Block & sample,
|
||||
const RowInputFormatParams & params,
|
||||
const FormatSettings &)
|
||||
const FormatSettings & format_settings)
|
||||
{
|
||||
return std::make_shared<JSONAsStringRowInputFormat>(sample, buf, params);
|
||||
return std::make_shared<JSONAsStringRowInputFormat>(sample, buf, params, format_settings);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Formats/IRowInputFormat.h>
|
||||
#include <Processors/Formats/Impl/JSONEachRowRowInputFormat.h>
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
@ -13,30 +13,22 @@ namespace DB
|
||||
class ReadBuffer;
|
||||
|
||||
/// This format parses a sequence of JSON objects separated by newlines, spaces and/or comma.
|
||||
class JSONAsRowInputFormat : public IRowInputFormat
|
||||
class JSONAsRowInputFormat : public JSONEachRowRowInputFormat
|
||||
{
|
||||
public:
|
||||
JSONAsRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_);
|
||||
JSONAsRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings);
|
||||
|
||||
void resetParser() override;
|
||||
void setReadBuffer(ReadBuffer & in_) override;
|
||||
|
||||
private:
|
||||
JSONAsRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, Params params_);
|
||||
JSONAsRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, Params params_, const FormatSettings & format_settings);
|
||||
|
||||
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
|
||||
|
||||
void readPrefix() override;
|
||||
void readSuffix() override;
|
||||
|
||||
protected:
|
||||
virtual void readJSONObject(IColumn & column) = 0;
|
||||
std::unique_ptr<PeekableReadBuffer> buf;
|
||||
|
||||
private:
|
||||
/// This flag is needed to know if data is in square brackets.
|
||||
bool data_in_square_brackets = false;
|
||||
bool allow_new_rows = true;
|
||||
};
|
||||
|
||||
/// Each JSON object is parsed as a whole to string.
|
||||
@ -44,7 +36,7 @@ private:
|
||||
class JSONAsStringRowInputFormat final : public JSONAsRowInputFormat
|
||||
{
|
||||
public:
|
||||
JSONAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_);
|
||||
JSONAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings);
|
||||
String getName() const override { return "JSONAsStringRowInputFormat"; }
|
||||
|
||||
private:
|
||||
@ -61,8 +53,8 @@ public:
|
||||
String getName() const override { return "JSONAsObjectRowInputFormat"; }
|
||||
|
||||
private:
|
||||
Chunk getChunkForCount(size_t rows) override;
|
||||
void readJSONObject(IColumn & column) override;
|
||||
const FormatSettings format_settings;
|
||||
};
|
||||
|
||||
class JSONAsStringExternalSchemaReader : public IExternalSchemaReader
|
||||
|
@ -1,12 +1,10 @@
|
||||
#include <Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h>
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Formats/JSONUtils.h>
|
||||
#include <Formats/EscapingRuleUtils.h>
|
||||
#include <Formats/SchemaInferenceUtils.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <base/find_symbols.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -52,14 +50,22 @@ void JSONColumnsReaderBase::skipColumn()
|
||||
while (!in->eof() && balance)
|
||||
{
|
||||
if (inside_quotes)
|
||||
pos = find_first_symbols<'"'>(in->position(), in->buffer().end());
|
||||
pos = find_first_symbols<'\\', '"'>(in->position(), in->buffer().end());
|
||||
else
|
||||
pos = find_first_symbols<'[', ']', '"'>(in->position(), in->buffer().end());
|
||||
pos = find_first_symbols<'[', ']', '"', '\\'>(in->position(), in->buffer().end());
|
||||
|
||||
in->position() = pos;
|
||||
if (in->position() == in->buffer().end())
|
||||
continue;
|
||||
|
||||
if (*in->position() == '\\')
|
||||
{
|
||||
++in->position();
|
||||
if (!in->eof())
|
||||
++in->position();
|
||||
continue;
|
||||
}
|
||||
|
||||
if (*in->position() == '"')
|
||||
inside_quotes = !inside_quotes;
|
||||
else if (*in->position() == ']')
|
||||
@ -118,6 +124,31 @@ Chunk JSONColumnsBlockInputFormatBase::generate()
|
||||
return Chunk(std::move(columns), 0);
|
||||
|
||||
size_t chunk_start = getDataOffsetMaybeCompressed(*in);
|
||||
|
||||
if (need_only_count)
|
||||
{
|
||||
/// Count rows in first column and skip the rest columns.
|
||||
reader->readColumnStart();
|
||||
size_t num_rows = 0;
|
||||
if (!reader->checkColumnEnd())
|
||||
{
|
||||
do
|
||||
{
|
||||
skipJSONField(*in, "skip_field");
|
||||
++num_rows;
|
||||
} while (!reader->checkColumnEndOrSkipFieldDelimiter());
|
||||
}
|
||||
|
||||
while (!reader->checkChunkEndOrSkipColumnDelimiter())
|
||||
{
|
||||
reader->readColumnStart();
|
||||
reader->skipColumn();
|
||||
}
|
||||
|
||||
approx_bytes_read_for_chunk = getDataOffsetMaybeCompressed(*in) - chunk_start;
|
||||
return getChunkForCount(num_rows);
|
||||
}
|
||||
|
||||
std::vector<UInt8> seen_columns(columns.size(), 0);
|
||||
Int64 rows = -1;
|
||||
size_t iteration = 0;
|
||||
|
@ -88,6 +88,17 @@ void JSONCompactEachRowFormatReader::skipHeaderRow()
|
||||
skipRowEndDelimiter();
|
||||
}
|
||||
|
||||
bool JSONCompactEachRowFormatReader::checkForSuffix()
|
||||
{
|
||||
skipWhitespaceIfAny(*in);
|
||||
return in->eof();
|
||||
}
|
||||
|
||||
void JSONCompactEachRowFormatReader::skipRow()
|
||||
{
|
||||
JSONUtils::skipRowForJSONCompactEachRow(*in);
|
||||
}
|
||||
|
||||
std::vector<String> JSONCompactEachRowFormatReader::readHeaderRow()
|
||||
{
|
||||
skipRowStartDelimiter();
|
||||
|
@ -37,6 +37,7 @@ public:
|
||||
private:
|
||||
bool allowSyncAfterError() const override { return true; }
|
||||
void syncAfterError() override;
|
||||
bool supportsCountRows() const override { return true; }
|
||||
};
|
||||
|
||||
class JSONCompactEachRowFormatReader : public FormatWithNamesAndTypesReader
|
||||
@ -64,6 +65,10 @@ public:
|
||||
void skipFieldDelimiter() override;
|
||||
void skipRowEndDelimiter() override;
|
||||
|
||||
void skipRow() override;
|
||||
|
||||
bool checkForSuffix() override;
|
||||
|
||||
std::vector<String> readHeaderRow();
|
||||
std::vector<String> readNames() override { return readHeaderRow(); }
|
||||
std::vector<String> readTypes() override { return readHeaderRow(); }
|
||||
|
@ -16,6 +16,7 @@ public:
|
||||
private:
|
||||
bool allowSyncAfterError() const override { return true; }
|
||||
void syncAfterError() override;
|
||||
bool supportsCountRows() const override { return true; }
|
||||
|
||||
void readPrefix() override;
|
||||
void readSuffix() override;
|
||||
|
@ -39,9 +39,9 @@ JSONEachRowRowInputFormat::JSONEachRowRowInputFormat(
|
||||
const FormatSettings & format_settings_,
|
||||
bool yield_strings_)
|
||||
: IRowInputFormat(header_, in_, std::move(params_))
|
||||
, format_settings(format_settings_)
|
||||
, prev_positions(header_.columns())
|
||||
, yield_strings(yield_strings_)
|
||||
, format_settings(format_settings_)
|
||||
{
|
||||
const auto & header = getPort().getHeader();
|
||||
name_map = header.getNamesToIndexesMap();
|
||||
@ -302,6 +302,26 @@ void JSONEachRowRowInputFormat::readSuffix()
|
||||
assertEOF(*in);
|
||||
}
|
||||
|
||||
size_t JSONEachRowRowInputFormat::countRows(size_t max_block_size)
|
||||
{
|
||||
if (unlikely(!allow_new_rows))
|
||||
return 0;
|
||||
|
||||
size_t num_rows = 0;
|
||||
bool is_first_row = getCurrentUnitNumber() == 0 && getTotalRows() == 0;
|
||||
skipWhitespaceIfAny(*in);
|
||||
while (num_rows < max_block_size && !checkEndOfData(is_first_row))
|
||||
{
|
||||
skipRowStart();
|
||||
JSONUtils::skipRowForJSONEachRow(*in);
|
||||
++num_rows;
|
||||
is_first_row = false;
|
||||
skipWhitespaceIfAny(*in);
|
||||
}
|
||||
|
||||
return num_rows;
|
||||
}
|
||||
|
||||
JSONEachRowSchemaReader::JSONEachRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
|
||||
: IRowWithNamesSchemaReader(in_, format_settings_)
|
||||
{
|
||||
|
@ -40,6 +40,9 @@ private:
|
||||
bool allowSyncAfterError() const override { return true; }
|
||||
void syncAfterError() override;
|
||||
|
||||
size_t countRows(size_t max_block_size) override;
|
||||
bool supportsCountRows() const override { return true; }
|
||||
|
||||
const String & columnName(size_t i) const;
|
||||
size_t columnIndex(StringRef name, size_t key_index);
|
||||
bool advanceToNextKey(size_t key_index);
|
||||
@ -50,9 +53,7 @@ private:
|
||||
void readNestedData(const String & name, MutableColumns & columns);
|
||||
|
||||
virtual void readRowStart(MutableColumns &) {}
|
||||
virtual bool checkEndOfData(bool is_first_row);
|
||||
|
||||
const FormatSettings format_settings;
|
||||
virtual void skipRowStart() {}
|
||||
|
||||
/// Buffer for the read from the stream field name. Used when you have to copy it.
|
||||
/// Also, if processing of Nested data is in progress, it holds the common prefix
|
||||
@ -76,11 +77,12 @@ private:
|
||||
/// Cached search results for previous row (keyed as index in JSON object) - used as a hint.
|
||||
std::vector<Block::NameMap::const_iterator> prev_positions;
|
||||
|
||||
bool allow_new_rows = true;
|
||||
|
||||
bool yield_strings;
|
||||
|
||||
protected:
|
||||
virtual bool checkEndOfData(bool is_first_row);
|
||||
|
||||
const FormatSettings format_settings;
|
||||
|
||||
/// Set of columns for which the values were read. The rest will be filled with default values.
|
||||
std::vector<UInt8> read_columns;
|
||||
@ -89,6 +91,8 @@ protected:
|
||||
|
||||
/// This flag is needed to know if data is in square brackets.
|
||||
bool data_in_square_brackets = false;
|
||||
|
||||
bool allow_new_rows = true;
|
||||
};
|
||||
|
||||
class JSONEachRowSchemaReader : public IRowWithNamesSchemaReader
|
||||
|
@ -55,6 +55,11 @@ void JSONObjectEachRowInputFormat::readRowStart(MutableColumns & columns)
|
||||
}
|
||||
}
|
||||
|
||||
void JSONObjectEachRowInputFormat::skipRowStart()
|
||||
{
|
||||
JSONUtils::readFieldName(*in);
|
||||
}
|
||||
|
||||
bool JSONObjectEachRowInputFormat::checkEndOfData(bool is_first_row)
|
||||
{
|
||||
if (in->eof() || JSONUtils::checkAndSkipObjectEnd(*in))
|
||||
|
@ -29,6 +29,7 @@ private:
|
||||
void readPrefix() override;
|
||||
void readSuffix() override {}
|
||||
void readRowStart(MutableColumns & columns) override;
|
||||
void skipRowStart() override;
|
||||
bool checkEndOfData(bool is_first_row) override;
|
||||
|
||||
std::optional<size_t> field_index_for_object_name;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Processors/Formats/Impl/JSONRowInputFormat.h>
|
||||
#include <Formats/JSONUtils.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -51,6 +51,18 @@ bool LineAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtens
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t LineAsStringRowInputFormat::countRows(size_t max_block_size)
|
||||
{
|
||||
size_t num_rows = 0;
|
||||
while (!in->eof() && num_rows < max_block_size)
|
||||
{
|
||||
skipToNextLineOrEOF(*in);
|
||||
++num_rows;
|
||||
}
|
||||
|
||||
return num_rows;
|
||||
}
|
||||
|
||||
void registerInputFormatLineAsString(FormatFactory & factory)
|
||||
{
|
||||
factory.registerInputFormat("LineAsString", [](
|
||||
|
@ -26,6 +26,9 @@ private:
|
||||
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
|
||||
|
||||
void readLineObject(IColumn & column);
|
||||
|
||||
size_t countRows(size_t max_block_size) override;
|
||||
bool supportsCountRows() const override { return true; }
|
||||
};
|
||||
|
||||
class LinaAsStringSchemaReader : public IExternalSchemaReader
|
||||
|
@ -482,14 +482,15 @@ void MsgPackVisitor::parse_error(size_t, size_t) // NOLINT
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Error occurred while parsing msgpack data.");
|
||||
}
|
||||
|
||||
bool MsgPackRowInputFormat::readObject()
|
||||
template <typename Parser>
|
||||
bool MsgPackRowInputFormat::readObject(Parser & msgpack_parser)
|
||||
{
|
||||
if (buf->eof())
|
||||
return false;
|
||||
|
||||
PeekableReadBufferCheckpoint checkpoint{*buf};
|
||||
size_t offset = 0;
|
||||
while (!parser.execute(buf->position(), buf->available(), offset))
|
||||
while (!msgpack_parser.execute(buf->position(), buf->available(), offset))
|
||||
{
|
||||
buf->position() = buf->buffer().end();
|
||||
if (buf->eof())
|
||||
@ -502,6 +503,24 @@ bool MsgPackRowInputFormat::readObject()
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t MsgPackRowInputFormat::countRows(size_t max_block_size)
|
||||
{
|
||||
size_t num_rows = 0;
|
||||
msgpack::null_visitor null_visitor;
|
||||
msgpack::detail::parse_helper<msgpack::null_visitor> null_parser(null_visitor);
|
||||
|
||||
size_t columns = getPort().getHeader().columns();
|
||||
|
||||
while (!buf->eof() && num_rows < max_block_size)
|
||||
{
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
readObject(null_parser);
|
||||
++num_rows;
|
||||
}
|
||||
|
||||
return num_rows;
|
||||
}
|
||||
|
||||
bool MsgPackRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext)
|
||||
{
|
||||
size_t column_index = 0;
|
||||
@ -510,7 +529,7 @@ bool MsgPackRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &
|
||||
for (; column_index != columns.size(); ++column_index)
|
||||
{
|
||||
visitor.set_info(*columns[column_index], data_types[column_index], ext.read_columns[column_index]);
|
||||
has_more_data = readObject();
|
||||
has_more_data = readObject(parser);
|
||||
if (!has_more_data)
|
||||
break;
|
||||
}
|
||||
|
@ -75,7 +75,11 @@ private:
|
||||
|
||||
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
|
||||
|
||||
bool readObject();
|
||||
template <typename Parser>
|
||||
bool readObject(Parser & msgpack_parser);
|
||||
|
||||
size_t countRows(size_t max_block_size) override;
|
||||
bool supportsCountRows() const override { return true; }
|
||||
|
||||
std::unique_ptr<PeekableReadBuffer> buf;
|
||||
MsgPackVisitor visitor;
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include "MySQLDumpRowInputFormat.h"
|
||||
#include <Processors/Formats/Impl/MySQLDumpRowInputFormat.h>
|
||||
#include <Processors/Formats/Impl/ValuesBlockInputFormat.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <DataTypes/Serializations/SerializationNullable.h>
|
||||
@ -303,15 +304,8 @@ static void skipFieldDelimiter(ReadBuffer & in)
|
||||
skipWhitespaceIfAny(in);
|
||||
}
|
||||
|
||||
static void skipEndOfRow(ReadBuffer & in, String & table_name)
|
||||
static void skipEndOfInsertQueryIfNeeded(ReadBuffer & in, String & table_name)
|
||||
{
|
||||
skipWhitespaceIfAny(in);
|
||||
assertChar(')', in);
|
||||
|
||||
skipWhitespaceIfAny(in);
|
||||
if (!in.eof() && *in.position() == ',')
|
||||
++in.position();
|
||||
|
||||
skipWhitespaceIfAny(in);
|
||||
if (!in.eof() && *in.position() == ';')
|
||||
{
|
||||
@ -323,6 +317,18 @@ static void skipEndOfRow(ReadBuffer & in, String & table_name)
|
||||
}
|
||||
}
|
||||
|
||||
static void skipEndOfRow(ReadBuffer & in, String & table_name)
|
||||
{
|
||||
skipWhitespaceIfAny(in);
|
||||
assertChar(')', in);
|
||||
|
||||
skipWhitespaceIfAny(in);
|
||||
if (!in.eof() && *in.position() == ',')
|
||||
++in.position();
|
||||
|
||||
skipEndOfInsertQueryIfNeeded(in, table_name);
|
||||
}
|
||||
|
||||
static void readFirstCreateAndInsertQueries(ReadBuffer & in, String & table_name, NamesAndTypesList & structure_from_create, Names & column_names)
|
||||
{
|
||||
auto type = skipToInsertOrCreateQuery(table_name, in);
|
||||
@ -385,6 +391,19 @@ bool MySQLDumpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t MySQLDumpRowInputFormat::countRows(size_t max_block_size)
|
||||
{
|
||||
size_t num_rows = 0;
|
||||
while (!in->eof() && num_rows < max_block_size)
|
||||
{
|
||||
ValuesBlockInputFormat::skipToNextRow(in, 1, 0);
|
||||
skipEndOfInsertQueryIfNeeded(*in, table_name);
|
||||
++num_rows;
|
||||
}
|
||||
|
||||
return num_rows;
|
||||
}
|
||||
|
||||
bool MySQLDumpRowInputFormat::readField(IColumn & column, size_t column_idx)
|
||||
{
|
||||
const auto & type = types[column_idx];
|
||||
|
@ -20,6 +20,9 @@ private:
|
||||
bool readField(IColumn & column, size_t column_idx);
|
||||
void skipField();
|
||||
|
||||
bool supportsCountRows() const override { return true; }
|
||||
size_t countRows(size_t max_block_size) override;
|
||||
|
||||
String table_name;
|
||||
DataTypes types;
|
||||
Block::NameMap column_indexes_by_names;
|
||||
|
@ -291,6 +291,18 @@ Chunk NativeORCBlockInputFormat::generate()
|
||||
if (!file_reader)
|
||||
prepareFileReader();
|
||||
|
||||
if (need_only_count)
|
||||
{
|
||||
++current_stripe;
|
||||
for (; current_stripe < total_stripes && skip_stripes.contains(current_stripe); ++current_stripe)
|
||||
;
|
||||
|
||||
if (current_stripe >= total_stripes)
|
||||
return {};
|
||||
|
||||
return getChunkForCount(file_reader->getStripe(current_stripe)->getNumberOfRows());
|
||||
}
|
||||
|
||||
if (!stripe_reader)
|
||||
{
|
||||
if (!prepareStripeReader())
|
||||
|
@ -75,7 +75,7 @@ private:
|
||||
std::list<UInt64> include_indices;
|
||||
|
||||
BlockMissingValues block_missing_values;
|
||||
size_t approx_bytes_read_for_chunk;
|
||||
size_t approx_bytes_read_for_chunk = 0;
|
||||
|
||||
const FormatSettings format_settings;
|
||||
const std::unordered_set<int> & skip_stripes;
|
||||
|
@ -43,6 +43,9 @@ Chunk ORCBlockInputFormat::generate()
|
||||
if (stripe_current >= stripe_total)
|
||||
return {};
|
||||
|
||||
if (need_only_count)
|
||||
return getChunkForCount(file_reader->GetRawORCReader()->getStripe(stripe_current++)->getNumberOfRows());
|
||||
|
||||
auto batch_result = file_reader->ReadStripe(stripe_current, include_indices);
|
||||
if (!batch_result.ok())
|
||||
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to create batch reader: {}", batch_result.status().ToString());
|
||||
|
@ -641,6 +641,12 @@ Chunk ParquetBlockInputFormat::generate()
|
||||
{
|
||||
initializeIfNeeded();
|
||||
|
||||
if (is_stopped || row_group_batches_completed == row_group_batches.size())
|
||||
return {};
|
||||
|
||||
if (need_only_count)
|
||||
return getChunkForCount(row_group_batches[row_group_batches_completed++].total_rows);
|
||||
|
||||
std::unique_lock lock(mutex);
|
||||
|
||||
while (true)
|
||||
|
@ -57,6 +57,30 @@ bool ProtobufListInputFormat::readRow(MutableColumns & columns, RowReadExtension
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t ProtobufListInputFormat::countRows(size_t max_block_size)
|
||||
{
|
||||
if (getTotalRows() == 0)
|
||||
reader->startMessage(true);
|
||||
|
||||
if (reader->eof())
|
||||
{
|
||||
reader->endMessage(false);
|
||||
return 0;
|
||||
}
|
||||
|
||||
size_t num_rows = 0;
|
||||
while (!reader->eof() && num_rows < max_block_size)
|
||||
{
|
||||
int tag;
|
||||
reader->readFieldNumber(tag);
|
||||
reader->startNestedMessage();
|
||||
reader->endNestedMessage();
|
||||
++num_rows;
|
||||
}
|
||||
|
||||
return num_rows;
|
||||
}
|
||||
|
||||
ProtobufListSchemaReader::ProtobufListSchemaReader(const FormatSettings & format_settings)
|
||||
: schema_info(
|
||||
format_settings.schema.format_schema,
|
||||
|
@ -38,6 +38,9 @@ public:
|
||||
private:
|
||||
bool readRow(MutableColumns & columns, RowReadExtension & row_read_extension) override;
|
||||
|
||||
bool supportsCountRows() const override { return true; }
|
||||
size_t countRows(size_t max_block_size) override;
|
||||
|
||||
std::unique_ptr<ProtobufReader> reader;
|
||||
std::vector<size_t> missing_column_indices;
|
||||
std::unique_ptr<ProtobufSerializer> serializer;
|
||||
|
@ -78,6 +78,22 @@ void ProtobufRowInputFormat::resetParser()
|
||||
reader.reset();
|
||||
}
|
||||
|
||||
size_t ProtobufRowInputFormat::countRows(size_t max_block_size)
|
||||
{
|
||||
if (!reader)
|
||||
createReaderAndSerializer();
|
||||
|
||||
size_t num_rows = 0;
|
||||
while (!reader->eof() && num_rows < max_block_size)
|
||||
{
|
||||
reader->startMessage(with_length_delimiter);
|
||||
reader->endMessage(false);
|
||||
++num_rows;
|
||||
}
|
||||
|
||||
return num_rows;
|
||||
}
|
||||
|
||||
void registerInputFormatProtobuf(FormatFactory & factory)
|
||||
{
|
||||
for (bool with_length_delimiter : {false, true})
|
||||
|
@ -47,6 +47,9 @@ private:
|
||||
bool allowSyncAfterError() const override;
|
||||
void syncAfterError() override;
|
||||
|
||||
bool supportsCountRows() const override { return true; }
|
||||
size_t countRows(size_t max_block_size) override;
|
||||
|
||||
void createReaderAndSerializer();
|
||||
|
||||
std::unique_ptr<ProtobufReader> reader;
|
||||
|
@ -39,6 +39,15 @@ bool RawBLOBRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &
|
||||
return false;
|
||||
}
|
||||
|
||||
size_t RawBLOBRowInputFormat::countRows(size_t)
|
||||
{
|
||||
if (done_count_rows)
|
||||
return 0;
|
||||
|
||||
done_count_rows = true;
|
||||
return 1;
|
||||
}
|
||||
|
||||
void registerInputFormatRawBLOB(FormatFactory & factory)
|
||||
{
|
||||
factory.registerInputFormat("RawBLOB", [](
|
||||
|
@ -22,6 +22,11 @@ public:
|
||||
|
||||
private:
|
||||
bool readRow(MutableColumns & columns, RowReadExtension &) override;
|
||||
|
||||
bool supportsCountRows() const override { return true; }
|
||||
size_t countRows(size_t max_block_size) override;
|
||||
|
||||
bool done_count_rows = false;
|
||||
};
|
||||
|
||||
class RawBLOBSchemaReader: public IExternalSchemaReader
|
||||
|
@ -216,6 +216,18 @@ void TSKVRowInputFormat::resetParser()
|
||||
name_buf.clear();
|
||||
}
|
||||
|
||||
size_t TSKVRowInputFormat::countRows(size_t max_block_size)
|
||||
{
|
||||
size_t num_rows = 0;
|
||||
while (!in->eof() && num_rows < max_block_size)
|
||||
{
|
||||
skipToUnescapedNextLineOrEOF(*in);
|
||||
++num_rows;
|
||||
}
|
||||
|
||||
return num_rows;
|
||||
}
|
||||
|
||||
TSKVSchemaReader::TSKVSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
|
||||
: IRowWithNamesSchemaReader(in_, format_settings_, getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule::Escaped))
|
||||
{
|
||||
|
@ -36,6 +36,9 @@ private:
|
||||
bool allowSyncAfterError() const override { return true; }
|
||||
void syncAfterError() override;
|
||||
|
||||
bool supportsCountRows() const override { return true; }
|
||||
size_t countRows(size_t max_block_size) override;
|
||||
|
||||
const FormatSettings format_settings;
|
||||
|
||||
/// Buffer for the read from the stream the field name. Used when you have to copy it.
|
||||
|
@ -300,6 +300,51 @@ bool TabSeparatedFormatReader::checkForSuffix()
|
||||
return false;
|
||||
}
|
||||
|
||||
void TabSeparatedFormatReader::skipRow()
|
||||
{
|
||||
ReadBuffer & istr = *buf;
|
||||
while (!istr.eof())
|
||||
{
|
||||
char * pos;
|
||||
if (is_raw)
|
||||
pos = find_first_symbols<'\r', '\n'>(istr.position(), istr.buffer().end());
|
||||
else
|
||||
pos = find_first_symbols<'\\', '\r', '\n'>(istr.position(), istr.buffer().end());
|
||||
|
||||
istr.position() = pos;
|
||||
|
||||
if (istr.position() > istr.buffer().end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Position in buffer is out of bounds. There must be a bug.");
|
||||
else if (pos == istr.buffer().end())
|
||||
continue;
|
||||
|
||||
if (!is_raw && *istr.position() == '\\')
|
||||
{
|
||||
++istr.position();
|
||||
if (!istr.eof())
|
||||
++istr.position();
|
||||
continue;
|
||||
}
|
||||
|
||||
if (*istr.position() == '\n')
|
||||
{
|
||||
++istr.position();
|
||||
if (!istr.eof() && *istr.position() == '\r')
|
||||
++istr.position();
|
||||
return;
|
||||
}
|
||||
else if (*istr.position() == '\r')
|
||||
{
|
||||
++istr.position();
|
||||
if (!istr.eof() && *istr.position() == '\n')
|
||||
{
|
||||
++istr.position();
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
bool TabSeparatedFormatReader::checkForEndOfRow()
|
||||
{
|
||||
return buf->eof() || *buf->position() == '\n';
|
||||
|
@ -33,6 +33,7 @@ private:
|
||||
void syncAfterError() override;
|
||||
bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override { return *pos != '\n' && *pos != '\t'; }
|
||||
|
||||
bool supportsCountRows() const override { return true; }
|
||||
|
||||
std::unique_ptr<PeekableReadBuffer> buf;
|
||||
};
|
||||
@ -59,6 +60,8 @@ public:
|
||||
std::vector<String> readTypes() override { return readHeaderRow(); }
|
||||
std::vector<String> readHeaderRow() { return readRowImpl<true>(); }
|
||||
|
||||
void skipRow() override;
|
||||
|
||||
template <bool read_string>
|
||||
String readFieldIntoString();
|
||||
|
||||
|
@ -54,66 +54,8 @@ ValuesBlockInputFormat::ValuesBlockInputFormat(
|
||||
{
|
||||
}
|
||||
|
||||
Chunk ValuesBlockInputFormat::generate()
|
||||
{
|
||||
if (total_rows == 0)
|
||||
readPrefix();
|
||||
|
||||
const Block & header = getPort().getHeader();
|
||||
MutableColumns columns = header.cloneEmptyColumns();
|
||||
block_missing_values.clear();
|
||||
size_t chunk_start = getDataOffsetMaybeCompressed(*buf);
|
||||
|
||||
for (size_t rows_in_block = 0; rows_in_block < params.max_block_size; ++rows_in_block)
|
||||
{
|
||||
try
|
||||
{
|
||||
skipWhitespaceIfAny(*buf);
|
||||
if (buf->eof() || *buf->position() == ';')
|
||||
break;
|
||||
readRow(columns, rows_in_block);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
if (isParseError(e.code()))
|
||||
e.addMessage(" at row " + std::to_string(total_rows));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
approx_bytes_read_for_chunk = getDataOffsetMaybeCompressed(*buf) - chunk_start;
|
||||
|
||||
/// Evaluate expressions, which were parsed using templates, if any
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
if (!templates[i] || !templates[i]->rowsCount())
|
||||
continue;
|
||||
|
||||
const auto & expected_type = header.getByPosition(i).type;
|
||||
if (columns[i]->empty())
|
||||
columns[i] = IColumn::mutate(templates[i]->evaluateAll(block_missing_values, i, expected_type));
|
||||
else
|
||||
{
|
||||
ColumnPtr evaluated = templates[i]->evaluateAll(block_missing_values, i, expected_type, columns[i]->size());
|
||||
columns[i]->insertRangeFrom(*evaluated, 0, evaluated->size());
|
||||
}
|
||||
}
|
||||
|
||||
if (columns.empty() || columns[0]->empty())
|
||||
{
|
||||
readSuffix();
|
||||
return {};
|
||||
}
|
||||
|
||||
for (const auto & column : columns)
|
||||
column->finalize();
|
||||
|
||||
size_t rows_in_block = columns[0]->size();
|
||||
return Chunk{std::move(columns), rows_in_block};
|
||||
}
|
||||
|
||||
/// Can be used in fileSegmentationEngine for parallel parsing of Values
|
||||
static bool skipToNextRow(PeekableReadBuffer * buf, size_t min_chunk_bytes, int balance)
|
||||
bool ValuesBlockInputFormat::skipToNextRow(ReadBuffer * buf, size_t min_chunk_bytes, int balance)
|
||||
{
|
||||
skipWhitespaceIfAny(*buf);
|
||||
if (buf->eof() || *buf->position() == ';')
|
||||
@ -156,6 +98,80 @@ static bool skipToNextRow(PeekableReadBuffer * buf, size_t min_chunk_bytes, int
|
||||
return true;
|
||||
}
|
||||
|
||||
Chunk ValuesBlockInputFormat::generate()
|
||||
{
|
||||
if (total_rows == 0)
|
||||
readPrefix();
|
||||
|
||||
const Block & header = getPort().getHeader();
|
||||
MutableColumns columns = header.cloneEmptyColumns();
|
||||
block_missing_values.clear();
|
||||
size_t chunk_start = getDataOffsetMaybeCompressed(*buf);
|
||||
|
||||
size_t rows_in_block = 0;
|
||||
for (; rows_in_block < params.max_block_size; ++rows_in_block)
|
||||
{
|
||||
try
|
||||
{
|
||||
skipWhitespaceIfAny(*buf);
|
||||
if (buf->eof() || *buf->position() == ';')
|
||||
break;
|
||||
if (need_only_count)
|
||||
skipToNextRow(buf.get(), 1, 0);
|
||||
else
|
||||
readRow(columns, rows_in_block);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
if (isParseError(e.code()))
|
||||
e.addMessage(" at row " + std::to_string(total_rows));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
approx_bytes_read_for_chunk = getDataOffsetMaybeCompressed(*buf) - chunk_start;
|
||||
|
||||
if (need_only_count)
|
||||
{
|
||||
if (!rows_in_block)
|
||||
{
|
||||
readSuffix();
|
||||
return {};
|
||||
}
|
||||
|
||||
total_rows += rows_in_block;
|
||||
return getChunkForCount(rows_in_block);
|
||||
}
|
||||
|
||||
/// Evaluate expressions, which were parsed using templates, if any
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
if (!templates[i] || !templates[i]->rowsCount())
|
||||
continue;
|
||||
|
||||
const auto & expected_type = header.getByPosition(i).type;
|
||||
if (columns[i]->empty())
|
||||
columns[i] = IColumn::mutate(templates[i]->evaluateAll(block_missing_values, i, expected_type));
|
||||
else
|
||||
{
|
||||
ColumnPtr evaluated = templates[i]->evaluateAll(block_missing_values, i, expected_type, columns[i]->size());
|
||||
columns[i]->insertRangeFrom(*evaluated, 0, evaluated->size());
|
||||
}
|
||||
}
|
||||
|
||||
if (columns.empty() || columns[0]->empty())
|
||||
{
|
||||
readSuffix();
|
||||
return {};
|
||||
}
|
||||
|
||||
for (const auto & column : columns)
|
||||
column->finalize();
|
||||
|
||||
size_t rows = columns[0]->size();
|
||||
return Chunk{std::move(columns), rows};
|
||||
}
|
||||
|
||||
/// We need continuous memory containing the expression to use Lexer
|
||||
/// Note that this is both reading and tokenizing until the end of the row
|
||||
/// This is doing unnecessary work if the rest of the columns can be read with tryReadValue (which doesn't require tokens)
|
||||
|
@ -41,6 +41,9 @@ public:
|
||||
const BlockMissingValues & getMissingValues() const override { return block_missing_values; }
|
||||
|
||||
size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; }
|
||||
|
||||
static bool skipToNextRow(ReadBuffer * buf, size_t min_chunk_bytes, int balance);
|
||||
|
||||
private:
|
||||
ValuesBlockInputFormat(std::unique_ptr<PeekableReadBuffer> buf_, const Block & header_, const RowInputFormatParams & params_,
|
||||
const FormatSettings & format_settings_);
|
||||
@ -71,6 +74,8 @@ private:
|
||||
void readPrefix();
|
||||
void readSuffix();
|
||||
|
||||
size_t countRows(size_t max_block_size);
|
||||
|
||||
std::unique_ptr<PeekableReadBuffer> buf;
|
||||
std::optional<IParser::Pos> token_iterator{};
|
||||
std::optional<Tokens> tokens{};
|
||||
|
@ -58,8 +58,8 @@ RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes(
|
||||
, is_binary(is_binary_)
|
||||
, with_names(with_names_)
|
||||
, with_types(with_types_)
|
||||
, format_reader(std::move(format_reader_))
|
||||
, try_detect_header(try_detect_header_)
|
||||
, format_reader(std::move(format_reader_))
|
||||
{
|
||||
column_indexes_by_names = getPort().getHeader().getNamesToIndexesMap();
|
||||
}
|
||||
@ -264,6 +264,30 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t RowInputFormatWithNamesAndTypes::countRows(size_t max_block_size)
|
||||
{
|
||||
if (unlikely(end_of_stream))
|
||||
return 0;
|
||||
|
||||
size_t num_rows = 0;
|
||||
bool is_first_row = getTotalRows() == 0 && !with_names && !with_types && !is_header_detected;
|
||||
while (!format_reader->checkForSuffix() && num_rows < max_block_size)
|
||||
{
|
||||
if (likely(!is_first_row))
|
||||
format_reader->skipRowBetweenDelimiter();
|
||||
else
|
||||
is_first_row = false;
|
||||
|
||||
format_reader->skipRow();
|
||||
++num_rows;
|
||||
}
|
||||
|
||||
if (num_rows == 0 || num_rows < max_block_size)
|
||||
end_of_stream = true;
|
||||
|
||||
return num_rows;
|
||||
}
|
||||
|
||||
void RowInputFormatWithNamesAndTypes::resetParser()
|
||||
{
|
||||
RowInputFormatWithDiagnosticInfo::resetParser();
|
||||
|
@ -56,6 +56,8 @@ protected:
|
||||
private:
|
||||
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
|
||||
|
||||
size_t countRows(size_t max_block_size) override;
|
||||
|
||||
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override;
|
||||
void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override;
|
||||
|
||||
@ -64,11 +66,11 @@ private:
|
||||
bool is_binary;
|
||||
bool with_names;
|
||||
bool with_types;
|
||||
std::unique_ptr<FormatWithNamesAndTypesReader> format_reader;
|
||||
bool try_detect_header;
|
||||
bool is_header_detected = false;
|
||||
|
||||
protected:
|
||||
std::unique_ptr<FormatWithNamesAndTypesReader> format_reader;
|
||||
Block::NameMap column_indexes_by_names;
|
||||
};
|
||||
|
||||
@ -109,6 +111,16 @@ public:
|
||||
/// Skip the whole row with types.
|
||||
virtual void skipTypes() = 0;
|
||||
|
||||
virtual size_t countRows(size_t /*max_block_size*/)
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method countRows is not implemented for format reader");
|
||||
}
|
||||
|
||||
virtual void skipRow()
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method skipRow is not implemented for format reader");
|
||||
}
|
||||
|
||||
/// Skip delimiters, if any.
|
||||
virtual void skipPrefixBeforeHeader() {}
|
||||
virtual void skipRowStartDelimiter() {}
|
||||
|
@ -140,6 +140,17 @@ void ColumnDescription::readText(ReadBuffer & buf)
|
||||
}
|
||||
}
|
||||
|
||||
ColumnsDescription::ColumnsDescription(std::initializer_list<NameAndTypePair> ordinary)
|
||||
{
|
||||
for (const auto & elem : ordinary)
|
||||
add(ColumnDescription(elem.name, elem.type));
|
||||
}
|
||||
|
||||
ColumnsDescription::ColumnsDescription(NamesAndTypes ordinary)
|
||||
{
|
||||
for (auto & elem : ordinary)
|
||||
add(ColumnDescription(std::move(elem.name), std::move(elem.type)));
|
||||
}
|
||||
|
||||
ColumnsDescription::ColumnsDescription(NamesAndTypesList ordinary)
|
||||
{
|
||||
|
@ -102,6 +102,11 @@ class ColumnsDescription : public IHints<1, ColumnsDescription>
|
||||
{
|
||||
public:
|
||||
ColumnsDescription() = default;
|
||||
|
||||
ColumnsDescription(std::initializer_list<NameAndTypePair> ordinary);
|
||||
|
||||
explicit ColumnsDescription(NamesAndTypes ordinary);
|
||||
|
||||
explicit ColumnsDescription(NamesAndTypesList ordinary);
|
||||
|
||||
explicit ColumnsDescription(NamesAndTypesList ordinary, NamesAndAliases aliases);
|
||||
|
@ -29,7 +29,7 @@
|
||||
#include <Storages/HDFS/ReadBufferFromHDFS.h>
|
||||
#include <Storages/HDFS/WriteBufferFromHDFS.h>
|
||||
#include <Storages/PartitionedSink.h>
|
||||
#include <Storages/getVirtualsForStorage.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
@ -50,6 +50,11 @@
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event EngineFileLikeReadFiles;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
@ -291,12 +296,7 @@ StorageHDFS::StorageHDFS(
|
||||
storage_metadata.setComment(comment);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
|
||||
auto default_virtuals = NamesAndTypesList{
|
||||
{"_path", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
|
||||
{"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())}};
|
||||
|
||||
auto columns = storage_metadata.getSampleBlock().getNamesAndTypesList();
|
||||
virtual_columns = getVirtualsForStorage(columns, default_virtuals);
|
||||
virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList());
|
||||
}
|
||||
|
||||
ColumnsDescription StorageHDFS::getTableStructureFromData(
|
||||
@ -363,11 +363,25 @@ ColumnsDescription StorageHDFS::getTableStructureFromData(
|
||||
class HDFSSource::DisclosedGlobIterator::Impl
|
||||
{
|
||||
public:
|
||||
Impl(ContextPtr context_, const String & uri)
|
||||
Impl(const String & uri, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
{
|
||||
const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri);
|
||||
uris = getPathsList(path_from_uri, uri_without_path, context_);
|
||||
auto file_progress_callback = context_->getFileProgressCallback();
|
||||
uris = getPathsList(path_from_uri, uri_without_path, context);
|
||||
ASTPtr filter_ast;
|
||||
if (!uris.empty())
|
||||
filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, uris[0].path, context);
|
||||
|
||||
if (filter_ast)
|
||||
{
|
||||
std::vector<String> paths;
|
||||
paths.reserve(uris.size());
|
||||
for (const auto & path_with_info : uris)
|
||||
paths.push_back(path_with_info.path);
|
||||
|
||||
VirtualColumnUtils::filterByPathOrFile(uris, paths, query, virtual_columns, context, filter_ast);
|
||||
}
|
||||
auto file_progress_callback = context->getFileProgressCallback();
|
||||
|
||||
for (auto & elem : uris)
|
||||
{
|
||||
elem.path = uri_without_path + elem.path;
|
||||
@ -397,9 +411,23 @@ private:
|
||||
class HDFSSource::URISIterator::Impl : WithContext
|
||||
{
|
||||
public:
|
||||
explicit Impl(const std::vector<String> & uris_, ContextPtr context_)
|
||||
explicit Impl(const std::vector<String> & uris_, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context_)
|
||||
: WithContext(context_), uris(uris_), file_progress_callback(context_->getFileProgressCallback())
|
||||
{
|
||||
ASTPtr filter_ast;
|
||||
if (!uris.empty())
|
||||
filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, getPathFromUriAndUriWithoutPath(uris[0]).first, getContext());
|
||||
|
||||
if (filter_ast)
|
||||
{
|
||||
std::vector<String> paths;
|
||||
paths.reserve(uris.size());
|
||||
for (const auto & uri : uris)
|
||||
paths.push_back(getPathFromUriAndUriWithoutPath(uri).first);
|
||||
|
||||
VirtualColumnUtils::filterByPathOrFile(uris, paths, query, virtual_columns, getContext(), filter_ast);
|
||||
}
|
||||
|
||||
if (!uris.empty())
|
||||
{
|
||||
auto path_and_uri = getPathFromUriAndUriWithoutPath(uris[0]);
|
||||
@ -444,16 +472,16 @@ private:
|
||||
std::function<void(FileProgress)> file_progress_callback;
|
||||
};
|
||||
|
||||
HDFSSource::DisclosedGlobIterator::DisclosedGlobIterator(ContextPtr context_, const String & uri)
|
||||
: pimpl(std::make_shared<HDFSSource::DisclosedGlobIterator::Impl>(context_, uri)) {}
|
||||
HDFSSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
: pimpl(std::make_shared<HDFSSource::DisclosedGlobIterator::Impl>(uri, query, virtual_columns, context)) {}
|
||||
|
||||
StorageHDFS::PathWithInfo HDFSSource::DisclosedGlobIterator::next()
|
||||
{
|
||||
return pimpl->next();
|
||||
}
|
||||
|
||||
HDFSSource::URISIterator::URISIterator(const std::vector<String> & uris_, ContextPtr context)
|
||||
: pimpl(std::make_shared<HDFSSource::URISIterator::Impl>(uris_, context))
|
||||
HDFSSource::URISIterator::URISIterator(const std::vector<String> & uris_, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
|
||||
: pimpl(std::make_shared<HDFSSource::URISIterator::Impl>(uris_, query, virtual_columns, context))
|
||||
{
|
||||
}
|
||||
|
||||
@ -468,6 +496,7 @@ HDFSSource::HDFSSource(
|
||||
ContextPtr context_,
|
||||
UInt64 max_block_size_,
|
||||
std::shared_ptr<IteratorWrapper> file_iterator_,
|
||||
bool need_only_count_,
|
||||
const SelectQueryInfo & query_info_)
|
||||
: ISource(info.source_header, false)
|
||||
, WithContext(context_)
|
||||
@ -478,6 +507,7 @@ HDFSSource::HDFSSource(
|
||||
, max_block_size(max_block_size_)
|
||||
, file_iterator(file_iterator_)
|
||||
, columns_description(info.columns_description)
|
||||
, need_only_count(need_only_count_)
|
||||
, query_info(query_info_)
|
||||
{
|
||||
initialize();
|
||||
@ -516,9 +546,16 @@ bool HDFSSource::initialize()
|
||||
|
||||
current_path = path_with_info.path;
|
||||
|
||||
input_format = getContext()->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size);
|
||||
std::optional<size_t> max_parsing_threads;
|
||||
if (need_only_count)
|
||||
max_parsing_threads = 1;
|
||||
|
||||
input_format = getContext()->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, std::nullopt, max_parsing_threads);
|
||||
input_format->setQueryInfo(query_info, getContext());
|
||||
|
||||
if (need_only_count)
|
||||
input_format->needOnlyCount();
|
||||
|
||||
QueryPipelineBuilder builder;
|
||||
builder.init(Pipe(input_format));
|
||||
if (columns_description.hasDefaults())
|
||||
@ -538,6 +575,8 @@ bool HDFSSource::initialize()
|
||||
|
||||
pipeline = std::make_unique<QueryPipeline>(QueryPipelineBuilder::getPipeline(std::move(builder)));
|
||||
reader = std::make_unique<PullingPipelineExecutor>(*pipeline);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -560,29 +599,11 @@ Chunk HDFSSource::generate()
|
||||
Chunk chunk;
|
||||
if (reader->pull(chunk))
|
||||
{
|
||||
Columns columns = chunk.getColumns();
|
||||
UInt64 num_rows = chunk.getNumRows();
|
||||
size_t chunk_size = input_format->getApproxBytesReadForChunk();
|
||||
progress(num_rows, chunk_size ? chunk_size : chunk.bytes());
|
||||
|
||||
for (const auto & virtual_column : requested_virtual_columns)
|
||||
{
|
||||
if (virtual_column.name == "_path")
|
||||
{
|
||||
auto column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumnConst(num_rows, current_path);
|
||||
columns.push_back(column->convertToFullColumnIfConst());
|
||||
}
|
||||
else if (virtual_column.name == "_file")
|
||||
{
|
||||
size_t last_slash_pos = current_path.find_last_of('/');
|
||||
auto file_name = current_path.substr(last_slash_pos + 1);
|
||||
|
||||
auto column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumnConst(num_rows, std::move(file_name));
|
||||
columns.push_back(column->convertToFullColumnIfConst());
|
||||
}
|
||||
}
|
||||
|
||||
return Chunk(std::move(columns), num_rows);
|
||||
VirtualColumnUtils::addRequestedPathAndFileVirtualsToChunk(chunk, requested_virtual_columns, current_path);
|
||||
return chunk;
|
||||
}
|
||||
|
||||
reader.reset();
|
||||
@ -747,7 +768,7 @@ Pipe StorageHDFS::read(
|
||||
else if (is_path_with_globs)
|
||||
{
|
||||
/// Iterate through disclosed globs and make a source for each file
|
||||
auto glob_iterator = std::make_shared<HDFSSource::DisclosedGlobIterator>(context_, uris[0]);
|
||||
auto glob_iterator = std::make_shared<HDFSSource::DisclosedGlobIterator>(uris[0], query_info.query, virtual_columns, context_);
|
||||
iterator_wrapper = std::make_shared<HDFSSource::IteratorWrapper>([glob_iterator]()
|
||||
{
|
||||
return glob_iterator->next();
|
||||
@ -755,7 +776,7 @@ Pipe StorageHDFS::read(
|
||||
}
|
||||
else
|
||||
{
|
||||
auto uris_iterator = std::make_shared<HDFSSource::URISIterator>(uris, context_);
|
||||
auto uris_iterator = std::make_shared<HDFSSource::URISIterator>(uris, query_info.query, virtual_columns, context_);
|
||||
iterator_wrapper = std::make_shared<HDFSSource::IteratorWrapper>([uris_iterator]()
|
||||
{
|
||||
return uris_iterator->next();
|
||||
@ -763,6 +784,9 @@ Pipe StorageHDFS::read(
|
||||
}
|
||||
|
||||
auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals());
|
||||
bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty())
|
||||
&& context_->getSettingsRef().optimize_count_from_files;
|
||||
|
||||
Pipes pipes;
|
||||
auto this_ptr = std::static_pointer_cast<StorageHDFS>(shared_from_this());
|
||||
for (size_t i = 0; i < num_streams; ++i)
|
||||
@ -773,6 +797,7 @@ Pipe StorageHDFS::read(
|
||||
context_,
|
||||
max_block_size,
|
||||
iterator_wrapper,
|
||||
need_only_count,
|
||||
query_info));
|
||||
}
|
||||
return Pipe::unitePipes(std::move(pipes));
|
||||
|
@ -88,6 +88,8 @@ public:
|
||||
|
||||
static SchemaCache & getSchemaCache(const ContextPtr & ctx);
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return true; }
|
||||
|
||||
protected:
|
||||
friend class HDFSSource;
|
||||
|
||||
@ -124,7 +126,7 @@ public:
|
||||
class DisclosedGlobIterator
|
||||
{
|
||||
public:
|
||||
DisclosedGlobIterator(ContextPtr context_, const String & uri_);
|
||||
DisclosedGlobIterator(const String & uri_, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context);
|
||||
StorageHDFS::PathWithInfo next();
|
||||
private:
|
||||
class Impl;
|
||||
@ -135,7 +137,7 @@ public:
|
||||
class URISIterator
|
||||
{
|
||||
public:
|
||||
URISIterator(const std::vector<String> & uris_, ContextPtr context);
|
||||
URISIterator(const std::vector<String> & uris_, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context);
|
||||
StorageHDFS::PathWithInfo next();
|
||||
private:
|
||||
class Impl;
|
||||
@ -152,6 +154,7 @@ public:
|
||||
ContextPtr context_,
|
||||
UInt64 max_block_size_,
|
||||
std::shared_ptr<IteratorWrapper> file_iterator_,
|
||||
bool need_only_count_,
|
||||
const SelectQueryInfo & query_info_);
|
||||
|
||||
String getName() const override;
|
||||
@ -166,6 +169,7 @@ private:
|
||||
UInt64 max_block_size;
|
||||
std::shared_ptr<IteratorWrapper> file_iterator;
|
||||
ColumnsDescription columns_description;
|
||||
bool need_only_count;
|
||||
SelectQueryInfo query_info;
|
||||
|
||||
std::unique_ptr<ReadBuffer> read_buf;
|
||||
|
@ -21,6 +21,7 @@
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/extractTableFunctionArgumentsFromSelectQuery.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
|
||||
#include <TableFunctions/TableFunctionHDFSCluster.h>
|
||||
#include <memory>
|
||||
@ -64,6 +65,8 @@ StorageHDFSCluster::StorageHDFSCluster(
|
||||
|
||||
storage_metadata.setConstraints(constraints_);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
|
||||
virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList());
|
||||
}
|
||||
|
||||
void StorageHDFSCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context)
|
||||
@ -76,9 +79,9 @@ void StorageHDFSCluster::addColumnsStructureToQuery(ASTPtr & query, const String
|
||||
}
|
||||
|
||||
|
||||
RemoteQueryExecutor::Extension StorageHDFSCluster::getTaskIteratorExtension(ASTPtr, const ContextPtr & context) const
|
||||
RemoteQueryExecutor::Extension StorageHDFSCluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const
|
||||
{
|
||||
auto iterator = std::make_shared<HDFSSource::DisclosedGlobIterator>(context, uri);
|
||||
auto iterator = std::make_shared<HDFSSource::DisclosedGlobIterator>(uri, query, virtual_columns, context);
|
||||
auto callback = std::make_shared<std::function<String()>>([iter = std::move(iterator)]() mutable -> String { return iter->next().path; });
|
||||
return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)};
|
||||
}
|
||||
|
@ -39,12 +39,15 @@ public:
|
||||
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
|
||||
bool supportsTrivialCountOptimization() const override { return true; }
|
||||
|
||||
private:
|
||||
void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override;
|
||||
|
||||
String uri;
|
||||
String format_name;
|
||||
String compression_method;
|
||||
NamesAndTypesList virtual_columns;
|
||||
};
|
||||
|
||||
|
||||
|
@ -254,7 +254,8 @@ public:
|
||||
/// because those are internally translated into 'ALTER UDPATE' mutations.
|
||||
virtual bool supportsDelete() const { return false; }
|
||||
|
||||
/// Return true if the trivial count query could be optimized without reading the data at all.
|
||||
/// Return true if the trivial count query could be optimized without reading the data at all
|
||||
/// in totalRows() or totalRowsByPartitionPredicate() methods or with optimized reading in read() method.
|
||||
virtual bool supportsTrivialCountOptimization() const { return false; }
|
||||
|
||||
private:
|
||||
|
@ -41,11 +41,8 @@ private:
|
||||
using Base = CacheBase<UInt128, MarksInCompressedFile, UInt128TrivialHash, MarksWeightFunction>;
|
||||
|
||||
public:
|
||||
explicit MarkCache(size_t max_size_in_bytes)
|
||||
: Base(max_size_in_bytes) {}
|
||||
|
||||
MarkCache(const String & mark_cache_policy, size_t max_size_in_bytes)
|
||||
: Base(mark_cache_policy, max_size_in_bytes) {}
|
||||
MarkCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio)
|
||||
: Base(cache_policy, max_size_in_bytes, 0, size_ratio) {}
|
||||
|
||||
/// Calculate key from path to file and offset.
|
||||
static UInt128 hash(const String & path_to_file)
|
||||
|
@ -1304,6 +1304,10 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried);
|
||||
|
||||
/// Do not keep data parts in snapshot.
|
||||
/// They are stored separately, and some could be released after PK analysis.
|
||||
auto storage_snapshot_copy = storage_snapshot->clone(std::make_unique<MergeTreeData::SnapshotData>());
|
||||
|
||||
return std::make_unique<ReadFromMergeTree>(
|
||||
std::move(parts),
|
||||
std::move(alter_conversions),
|
||||
@ -1311,7 +1315,7 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts(
|
||||
virt_column_names,
|
||||
data,
|
||||
query_info,
|
||||
storage_snapshot,
|
||||
storage_snapshot_copy,
|
||||
context,
|
||||
max_block_size,
|
||||
num_streams,
|
||||
@ -1642,7 +1646,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
|
||||
{
|
||||
if (index_mark != index_range.begin || !granule || last_index_mark != index_range.begin)
|
||||
granule = reader.read();
|
||||
// Cast to Ann condition
|
||||
|
||||
auto ann_condition = std::dynamic_pointer_cast<IMergeTreeIndexConditionApproximateNearestNeighbor>(condition);
|
||||
if (ann_condition != nullptr)
|
||||
{
|
||||
|
@ -166,7 +166,8 @@ void MergeTreeIndexAggregatorAnnoy<Distance>::update(const Block & block, size_t
|
||||
if (offsets[i + 1] - offsets[i] != size)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name);
|
||||
|
||||
index = std::make_shared<AnnoyIndexWithSerialization<Distance>>(size);
|
||||
if (!index)
|
||||
index = std::make_shared<AnnoyIndexWithSerialization<Distance>>(size);
|
||||
|
||||
/// Add all rows of block
|
||||
index->add_item(index->get_n_items(), array.data());
|
||||
@ -189,7 +190,8 @@ void MergeTreeIndexAggregatorAnnoy<Distance>::update(const Block & block, size_t
|
||||
if (data.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Tuple has 0 rows, {} rows expected", rows_read);
|
||||
|
||||
index = std::make_shared<AnnoyIndexWithSerialization<Distance>>(data[0].size());
|
||||
if (!index)
|
||||
index = std::make_shared<AnnoyIndexWithSerialization<Distance>>(data[0].size());
|
||||
|
||||
for (const auto & item : data)
|
||||
index->add_item(index->get_n_items(), item.data());
|
||||
|
@ -160,8 +160,8 @@ void MergeTreeIndexAggregatorUSearch<Metric>::update(const Block & block, size_t
|
||||
if (offsets[i + 1] - offsets[i] != size)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name);
|
||||
|
||||
|
||||
index = std::make_shared<USearchIndexWithSerialization<Metric>>(size);
|
||||
if (!index)
|
||||
index = std::make_shared<USearchIndexWithSerialization<Metric>>(size);
|
||||
|
||||
/// Add all rows of block
|
||||
if (!index->reserve(unum::usearch::ceil2(index->size() + num_rows)))
|
||||
@ -188,7 +188,8 @@ void MergeTreeIndexAggregatorUSearch<Metric>::update(const Block & block, size_t
|
||||
if (data.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Tuple has 0 rows, {} rows expected", rows_read);
|
||||
|
||||
index = std::make_shared<USearchIndexWithSerialization<Metric>>(data[0].size());
|
||||
if (!index)
|
||||
index = std::make_shared<USearchIndexWithSerialization<Metric>>(data[0].size());
|
||||
|
||||
if (!index->reserve(unum::usearch::ceil2(index->size() + data.size())))
|
||||
throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for usearch index");
|
||||
|
@ -25,7 +25,6 @@
|
||||
# include <Storages/StorageS3.h>
|
||||
# include <Storages/StorageS3Settings.h>
|
||||
# include <Storages/VirtualColumnUtils.h>
|
||||
# include <Storages/getVirtualsForStorage.h>
|
||||
|
||||
# include <Formats/FormatFactory.h>
|
||||
|
||||
@ -70,13 +69,13 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator(
|
||||
const S3::Client & client_,
|
||||
const S3::URI & globbed_uri_,
|
||||
ASTPtr query,
|
||||
const Block & virtual_header,
|
||||
const NamesAndTypesList & virtual_columns,
|
||||
ContextPtr context,
|
||||
UInt64 & max_poll_size_,
|
||||
const S3Settings::RequestSettings & request_settings_)
|
||||
: max_poll_size(max_poll_size_)
|
||||
, glob_iterator(std::make_unique<StorageS3QueueSource::DisclosedGlobIterator>(
|
||||
client_, globbed_uri_, query, virtual_header, context, nullptr, request_settings_))
|
||||
client_, globbed_uri_, query, virtual_columns, context, nullptr, request_settings_))
|
||||
{
|
||||
/// todo(kssenii): remove this loop, it should not be here
|
||||
while (true)
|
||||
@ -193,6 +192,7 @@ StorageS3QueueSource::StorageS3QueueSource(
|
||||
version_id_,
|
||||
file_iterator,
|
||||
download_thread_num_,
|
||||
false,
|
||||
/* query_info */ std::nullopt);
|
||||
reader = std::move(internal_source->reader);
|
||||
if (reader)
|
||||
|
@ -44,7 +44,7 @@ public:
|
||||
const S3::Client & client_,
|
||||
const S3::URI & globbed_uri_,
|
||||
ASTPtr query,
|
||||
const Block & virtual_header,
|
||||
const NamesAndTypesList & virtual_columns,
|
||||
ContextPtr context,
|
||||
UInt64 & max_poll_size_,
|
||||
const S3Settings::RequestSettings & request_settings_ = {});
|
||||
|
@ -32,7 +32,6 @@
|
||||
# include <Storages/StorageS3.h>
|
||||
# include <Storages/StorageSnapshot.h>
|
||||
# include <Storages/VirtualColumnUtils.h>
|
||||
# include <Storages/getVirtualsForStorage.h>
|
||||
# include <Storages/prepareReadingFromFormat.h>
|
||||
# include <Common/NamedCollections/NamedCollections.h>
|
||||
|
||||
@ -171,15 +170,7 @@ StorageS3Queue::StorageS3Queue(
|
||||
}
|
||||
|
||||
files_metadata = std::make_shared<S3QueueFilesMetadata>(this, *s3queue_settings);
|
||||
|
||||
auto default_virtuals = NamesAndTypesList{
|
||||
{"_path", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
|
||||
{"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())}};
|
||||
|
||||
auto columns = storage_metadata.getSampleBlock().getNamesAndTypesList();
|
||||
virtual_columns = getVirtualsForStorage(columns, default_virtuals);
|
||||
for (const auto & column : virtual_columns)
|
||||
virtual_block.insert({column.type->createColumn(), column.type, column.name});
|
||||
virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList());
|
||||
|
||||
auto poll_thread = getContext()->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); });
|
||||
task = std::make_shared<TaskContext>(std::move(poll_thread));
|
||||
@ -527,7 +518,7 @@ StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query)
|
||||
*configuration.client,
|
||||
configuration.url,
|
||||
query,
|
||||
virtual_block,
|
||||
virtual_columns,
|
||||
local_context,
|
||||
s3queue_settings->s3queue_polling_size.value,
|
||||
configuration.request_settings);
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user