mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge branch 'master' into custom-key-parallel-replicas
This commit is contained in:
commit
fc9c53f963
2
.github/workflows/debug.yml
vendored
2
.github/workflows/debug.yml
vendored
@ -8,4 +8,4 @@ jobs:
|
||||
DebugInfo:
|
||||
runs-on: ubuntu-latest
|
||||
steps:
|
||||
- uses: hmarr/debug-action@1201a20fc9d278ddddd5f0f46922d06513892491
|
||||
- uses: hmarr/debug-action@a701ed95a46e6f2fb0df25e1a558c16356fae35a
|
||||
|
2
.gitignore
vendored
2
.gitignore
vendored
@ -154,6 +154,8 @@ website/package-lock.json
|
||||
/programs/server/data
|
||||
/programs/server/metadata
|
||||
/programs/server/store
|
||||
/programs/server/uuid
|
||||
/programs/server/coordination
|
||||
|
||||
# temporary test files
|
||||
tests/queries/0_stateless/test_*
|
||||
|
@ -58,7 +58,7 @@ echo 'SELECT version()' | curl 'http://localhost:18123/' --data-binary @-
|
||||
22.6.3.35
|
||||
```
|
||||
|
||||
or by allowing the container to use [host ports directly](https://docs.docker.com/network/host/) using `--network=host` (also allows archiving better network performance):
|
||||
or by allowing the container to use [host ports directly](https://docs.docker.com/network/host/) using `--network=host` (also allows achieving better network performance):
|
||||
|
||||
```bash
|
||||
docker run -d --network=host --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server
|
||||
|
@ -147,6 +147,14 @@ hash cmake
|
||||
|
||||
ClickHouse is available in pre-built binaries and packages. Binaries are portable and can be run on any Linux flavour.
|
||||
|
||||
Binaries are built for stable and LTS releases and also every commit to `master` for each pull request.
|
||||
The CI checks build the binaries on each commit to [ClickHouse](https://github.com/clickhouse/clickhouse/). To download them:
|
||||
|
||||
1. Open the [commits list](https://github.com/ClickHouse/ClickHouse/commits/master)
|
||||
1. Choose a **Merge pull request** commit that includes the new feature, or was added after the new feature
|
||||
1. Click the status symbol (yellow dot, red x, green check) to open the CI check list
|
||||
1. Scroll through the list until you find **ClickHouse build check x/x artifact groups are OK**
|
||||
1. Click **Details**
|
||||
1. Find the type of package for your operating system that you need and download the files.
|
||||
|
||||
![build artifact check](images/find-build-artifact.png)
|
||||
|
||||
To find the freshest build from `master`, go to [commits page](https://github.com/ClickHouse/ClickHouse/commits/master), click on the first green check mark or red cross near commit, and click to the “Details” link right after “ClickHouse Build Check”.
|
||||
|
BIN
docs/en/development/images/find-build-artifact.png
Normal file
BIN
docs/en/development/images/find-build-artifact.png
Normal file
Binary file not shown.
After Width: | Height: | Size: 122 KiB |
@ -390,40 +390,46 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234
|
||||
|
||||
### Available Types of Indices {#available-types-of-indices}
|
||||
|
||||
#### `minmax`
|
||||
#### MinMax
|
||||
|
||||
Stores extremes of the specified expression (if the expression is `tuple`, then it stores extremes for each element of `tuple`), uses stored info for skipping blocks of data like the primary key.
|
||||
|
||||
#### `set(max_rows)`
|
||||
Syntax: `minmax`
|
||||
|
||||
#### Set
|
||||
|
||||
Stores unique values of the specified expression (no more than `max_rows` rows, `max_rows=0` means “no limits”). Uses the values to check if the `WHERE` expression is not satisfiable on a block of data.
|
||||
|
||||
#### `ngrambf_v1(n, size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)`
|
||||
Syntax: `set(max_rows)`
|
||||
|
||||
Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) that contains all ngrams from a block of data. Works only with datatypes: [String](/docs/en/sql-reference/data-types/string.md), [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) and [Map](/docs/en/sql-reference/data-types/map.md). Can be used for optimization of `EQUALS`, `LIKE` and `IN` expressions.
|
||||
#### Bloom Filter
|
||||
|
||||
Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) for the specified columns. An optional `false_positive` parameter with possible values between 0 and 1 specifies the probability of receiving a false positive response from the filter. Default value: 0.025. Supported data types: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`, `UUID` and `Map`. For the `Map` data type, the client can specify if the index should be created for keys or values using [mapKeys](/docs/en/sql-reference/functions/tuple-map-functions.md/#mapkeys) or [mapValues](/docs/en/sql-reference/functions/tuple-map-functions.md/#mapvalues) function.
|
||||
|
||||
Syntax: `bloom_filter([false_positive])`
|
||||
|
||||
#### N-gram Bloom Filter
|
||||
|
||||
Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) that contains all n-grams from a block of data. Only works with datatypes: [String](/docs/en/sql-reference/data-types/string.md), [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) and [Map](/docs/en/sql-reference/data-types/map.md). Can be used for optimization of `EQUALS`, `LIKE` and `IN` expressions.
|
||||
|
||||
Syntax: `ngrambf_v1(n, size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)`
|
||||
|
||||
- `n` — ngram size,
|
||||
- `size_of_bloom_filter_in_bytes` — Bloom filter size in bytes (you can use large values here, for example, 256 or 512, because it can be compressed well).
|
||||
- `number_of_hash_functions` — The number of hash functions used in the Bloom filter.
|
||||
- `random_seed` — The seed for Bloom filter hash functions.
|
||||
|
||||
#### `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)`
|
||||
#### Token Bloom Filter
|
||||
|
||||
The same as `ngrambf_v1`, but stores tokens instead of ngrams. Tokens are sequences separated by non-alphanumeric characters.
|
||||
|
||||
#### `bloom_filter([false_positive])` — Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) for the specified columns.
|
||||
Syntax: `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)`
|
||||
|
||||
The optional `false_positive` parameter is the probability of receiving a false positive response from the filter. Possible values: (0, 1). Default value: 0.025.
|
||||
#### Special-purpose
|
||||
|
||||
Supported data types: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`, `UUID`, `Map`.
|
||||
- An experimental index to support approximate nearest neighbor (ANN) search. See [here](annindexes.md) for details.
|
||||
|
||||
For `Map` data type client can specify if index should be created for keys or values using [mapKeys](/docs/en/sql-reference/functions/tuple-map-functions.md/#mapkeys) or [mapValues](/docs/en/sql-reference/functions/tuple-map-functions.md/#mapvalues) function.
|
||||
|
||||
There are also special-purpose and experimental indexes to support approximate nearest neighbor (ANN) queries. See [here](annindexes.md) for details.
|
||||
|
||||
The following functions can use the filter: [equals](/docs/en/sql-reference/functions/comparison-functions.md), [notEquals](/docs/en/sql-reference/functions/comparison-functions.md), [in](/docs/en/sql-reference/functions/in-functions), [notIn](/docs/en/sql-reference/functions/in-functions), [has](/docs/en/sql-reference/functions/array-functions#hasarr-elem), [hasAny](/docs/en/sql-reference/functions/array-functions#hasany), [hasAll](/docs/en/sql-reference/functions/array-functions#hasall).
|
||||
|
||||
Example of index creation for `Map` data type
|
||||
## Example of index creation for Map data type
|
||||
|
||||
```
|
||||
INDEX map_key_index mapKeys(map_column) TYPE bloom_filter GRANULARITY 1
|
||||
@ -484,9 +490,6 @@ For example:
|
||||
:::
|
||||
|
||||
|
||||
## Approximate Nearest Neighbor Search Indexes [experimental] {#table_engines-ANNIndex}
|
||||
In addition to skip indices, there are also [Approximate Nearest Neighbor Search Indexes](/docs/en/engines/table-engines/mergetree-family/annindexes.md).
|
||||
|
||||
## Projections {#projections}
|
||||
Projections are like [materialized views](/docs/en/sql-reference/statements/create/view.md/#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries.
|
||||
|
||||
@ -885,6 +888,10 @@ User can assign new big parts to different disks of a [JBOD](https://en.wikipedi
|
||||
|
||||
## Using S3 for Data Storage {#table_engine-mergetree-s3}
|
||||
|
||||
:::note
|
||||
Google Cloud Storage (GCS) is also supported using the type `s3`. See [GCS backed MergeTree](/docs/en/integrations/data-ingestion/s3/gcs-merge-tree.md).
|
||||
:::
|
||||
|
||||
`MergeTree` family table engines can store data to [S3](https://aws.amazon.com/s3/) using a disk with type `s3`.
|
||||
|
||||
Configuration markup:
|
||||
@ -894,6 +901,7 @@ Configuration markup:
|
||||
<disks>
|
||||
<s3>
|
||||
<type>s3</type>
|
||||
<support_batch_delete>true</support_batch_delete>
|
||||
<endpoint>https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/root-path/</endpoint>
|
||||
<access_key_id>your_access_key_id</access_key_id>
|
||||
<secret_access_key>your_secret_access_key</secret_access_key>
|
||||
@ -927,6 +935,7 @@ Required parameters:
|
||||
Optional parameters:
|
||||
|
||||
- `region` — S3 region name.
|
||||
- `support_batch_delete` — This controls the check to see if batch deletes are supported. Set this to `false` when using Google Cloud Storage (GCS) as GCS does not support batch deletes and preventing the checks will prevent error messages in the logs.
|
||||
- `use_environment_credentials` — Reads AWS credentials from the Environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY and AWS_SESSION_TOKEN if they exist. Default value is `false`.
|
||||
- `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Default value is `false`.
|
||||
- `proxy` — Proxy configuration for S3 endpoint. Each `uri` element inside `proxy` block should contain a proxy URL.
|
||||
|
@ -271,6 +271,9 @@ You’ll need to create data and metadata folders manually and `chown` them for
|
||||
|
||||
On Gentoo, you can just use `emerge clickhouse` to install ClickHouse from sources.
|
||||
|
||||
### From CI checks pre-built binaries
|
||||
ClickHouse binaries are built for each [commit](/docs/en/development/build.md#you-dont-have-to-build-clickhouse).
|
||||
|
||||
## Launch {#launch}
|
||||
|
||||
To start the server as a daemon, run:
|
||||
|
@ -1191,6 +1191,7 @@ Use the following parameters to configure logging:
|
||||
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
|
||||
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
|
||||
- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table.
|
||||
- `storage_policy` – Name of storage policy to use for the table (optional)
|
||||
|
||||
**Example**
|
||||
|
||||
@ -1254,6 +1255,7 @@ Use the following parameters to configure logging:
|
||||
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
|
||||
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
|
||||
- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table.
|
||||
- `storage_policy` – Name of storage policy to use for the table (optional)
|
||||
|
||||
If the table does not exist, ClickHouse will create it. If the structure of the query log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically.
|
||||
|
||||
@ -1281,6 +1283,7 @@ Use the following parameters to configure logging:
|
||||
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
|
||||
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
|
||||
- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table.
|
||||
- `storage_policy` – Name of storage policy to use for the table (optional)
|
||||
|
||||
If the table does not exist, ClickHouse will create it. If the structure of the query thread log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically.
|
||||
|
||||
@ -1308,6 +1311,7 @@ Use the following parameters to configure logging:
|
||||
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
|
||||
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
|
||||
- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table.
|
||||
- `storage_policy` – Name of storage policy to use for the table (optional)
|
||||
|
||||
If the table does not exist, ClickHouse will create it. If the structure of the query views log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically.
|
||||
|
||||
@ -1334,6 +1338,7 @@ Parameters:
|
||||
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
|
||||
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined.
|
||||
- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table.
|
||||
- `storage_policy` – Name of storage policy to use for the table (optional)
|
||||
|
||||
**Example**
|
||||
```xml
|
||||
@ -1361,6 +1366,7 @@ Parameters:
|
||||
- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined.
|
||||
- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` defined.
|
||||
- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table.
|
||||
- `storage_policy` – Name of storage policy to use for the table (optional)
|
||||
|
||||
The default server configuration file `config.xml` contains the following settings section:
|
||||
|
||||
|
@ -1073,6 +1073,9 @@
|
||||
|
||||
<!-- Interval of flushing data. -->
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
|
||||
<!-- example of using a different storage policy for a system table -->
|
||||
<!-- storage_policy>local_ssd</storage_policy -->
|
||||
</query_log>
|
||||
|
||||
<!-- Trace log. Stores stack traces collected by query profilers.
|
||||
|
@ -24,11 +24,13 @@ class ConvertOrLikeChainVisitor : public InDepthQueryTreeVisitor<ConvertOrLikeCh
|
||||
using FunctionNodes = std::vector<std::shared_ptr<FunctionNode>>;
|
||||
|
||||
const FunctionOverloadResolverPtr match_function_ref;
|
||||
const FunctionOverloadResolverPtr or_function_resolver;
|
||||
public:
|
||||
|
||||
explicit ConvertOrLikeChainVisitor(FunctionOverloadResolverPtr _match_function_ref)
|
||||
explicit ConvertOrLikeChainVisitor(ContextPtr context)
|
||||
: InDepthQueryTreeVisitor<ConvertOrLikeChainVisitor>()
|
||||
, match_function_ref(_match_function_ref)
|
||||
, match_function_ref(FunctionFactory::instance().get("multiMatchAny", context))
|
||||
, or_function_resolver(FunctionFactory::instance().get("or", context))
|
||||
{}
|
||||
|
||||
static bool needChildVisit(VisitQueryTreeNodeType & parent, VisitQueryTreeNodeType &)
|
||||
@ -117,6 +119,7 @@ public:
|
||||
unique_elems.push_back(std::make_shared<ConstantNode>(false));
|
||||
|
||||
function_node->getArguments().getNodes() = std::move(unique_elems);
|
||||
function_node->resolveAsFunction(or_function_resolver);
|
||||
}
|
||||
};
|
||||
|
||||
@ -124,7 +127,7 @@ public:
|
||||
|
||||
void ConvertOrLikeChainPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
|
||||
{
|
||||
ConvertOrLikeChainVisitor visitor(FunctionFactory::instance().get("multiMatchAny", context));
|
||||
ConvertOrLikeChainVisitor visitor(context);
|
||||
visitor.visit(query_tree_node);
|
||||
}
|
||||
|
||||
|
@ -1058,6 +1058,8 @@ private:
|
||||
|
||||
static bool isTableExpressionNodeType(QueryTreeNodeType node_type);
|
||||
|
||||
static DataTypePtr getExpressionNodeResultTypeOrNull(const QueryTreeNodePtr & query_tree_node);
|
||||
|
||||
static ProjectionName calculateFunctionProjectionName(const QueryTreeNodePtr & function_node,
|
||||
const ProjectionNames & parameters_projection_names,
|
||||
const ProjectionNames & arguments_projection_names);
|
||||
@ -1241,6 +1243,34 @@ bool QueryAnalyzer::isTableExpressionNodeType(QueryTreeNodeType node_type)
|
||||
node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION;
|
||||
}
|
||||
|
||||
DataTypePtr QueryAnalyzer::getExpressionNodeResultTypeOrNull(const QueryTreeNodePtr & query_tree_node)
|
||||
{
|
||||
auto node_type = query_tree_node->getNodeType();
|
||||
|
||||
switch (node_type)
|
||||
{
|
||||
case QueryTreeNodeType::CONSTANT:
|
||||
[[fallthrough]];
|
||||
case QueryTreeNodeType::COLUMN:
|
||||
{
|
||||
return query_tree_node->getResultType();
|
||||
}
|
||||
case QueryTreeNodeType::FUNCTION:
|
||||
{
|
||||
auto & function_node = query_tree_node->as<FunctionNode &>();
|
||||
if (function_node.isResolved())
|
||||
return function_node.getResultType();
|
||||
break;
|
||||
}
|
||||
default:
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
ProjectionName QueryAnalyzer::calculateFunctionProjectionName(const QueryTreeNodePtr & function_node, const ProjectionNames & parameters_projection_names,
|
||||
const ProjectionNames & arguments_projection_names)
|
||||
{
|
||||
@ -1546,12 +1576,12 @@ void QueryAnalyzer::collectScopeValidIdentifiersForTypoCorrection(
|
||||
auto expression_identifier = Identifier(name);
|
||||
valid_identifiers_result.insert(expression_identifier);
|
||||
|
||||
auto expression_node_type = expression->getNodeType();
|
||||
auto result_type = getExpressionNodeResultTypeOrNull(expression);
|
||||
|
||||
if (identifier_is_compound && isExpressionNodeType(expression_node_type))
|
||||
if (identifier_is_compound && result_type)
|
||||
{
|
||||
collectCompoundExpressionValidIdentifiersForTypoCorrection(unresolved_identifier,
|
||||
expression->getResultType(),
|
||||
result_type,
|
||||
expression_identifier,
|
||||
valid_identifiers_result);
|
||||
}
|
||||
@ -1583,21 +1613,23 @@ void QueryAnalyzer::collectScopeValidIdentifiersForTypoCorrection(
|
||||
|
||||
for (const auto & [argument_name, expression] : scope.expression_argument_name_to_node)
|
||||
{
|
||||
assert(expression);
|
||||
auto expression_node_type = expression->getNodeType();
|
||||
|
||||
if (allow_expression_identifiers && isExpressionNodeType(expression_node_type))
|
||||
{
|
||||
auto expression_identifier = Identifier(argument_name);
|
||||
valid_identifiers_result.insert(expression_identifier);
|
||||
|
||||
if (identifier_is_compound)
|
||||
auto result_type = getExpressionNodeResultTypeOrNull(expression);
|
||||
|
||||
if (identifier_is_compound && result_type)
|
||||
{
|
||||
collectCompoundExpressionValidIdentifiersForTypoCorrection(unresolved_identifier,
|
||||
expression->getResultType(),
|
||||
result_type,
|
||||
expression_identifier,
|
||||
valid_identifiers_result);
|
||||
}
|
||||
|
||||
valid_identifiers_result.insert(expression_identifier);
|
||||
}
|
||||
else if (identifier_is_short && allow_function_identifiers && isFunctionExpressionNodeType(expression_node_type))
|
||||
{
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
#include <Backups/SettingsFieldOptionalUUID.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -16,48 +16,6 @@ namespace ErrorCodes
|
||||
extern const int WRONG_BACKUP_SETTINGS;
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
struct SettingFieldOptionalUUID
|
||||
{
|
||||
std::optional<UUID> value;
|
||||
|
||||
explicit SettingFieldOptionalUUID(const std::optional<UUID> & value_) : value(value_) {}
|
||||
|
||||
explicit SettingFieldOptionalUUID(const Field & field)
|
||||
{
|
||||
if (field.getType() == Field::Types::Null)
|
||||
{
|
||||
value = std::nullopt;
|
||||
return;
|
||||
}
|
||||
|
||||
if (field.getType() == Field::Types::String)
|
||||
{
|
||||
const String & str = field.get<const String &>();
|
||||
if (str.empty())
|
||||
{
|
||||
value = std::nullopt;
|
||||
return;
|
||||
}
|
||||
|
||||
UUID id;
|
||||
if (tryParse(id, str))
|
||||
{
|
||||
value = id;
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Cannot parse uuid from {}", field);
|
||||
}
|
||||
|
||||
explicit operator Field() const { return Field(value ? toString(*value) : ""); }
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
/// List of backup settings except base_backup_name and cluster_host_ids.
|
||||
#define LIST_OF_BACKUP_SETTINGS(M) \
|
||||
M(String, id) \
|
||||
|
@ -30,6 +30,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CONCURRENT_ACCESS_NOT_SUPPORTED;
|
||||
}
|
||||
|
||||
using OperationID = BackupsWorker::OperationID;
|
||||
@ -121,10 +122,12 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
BackupsWorker::BackupsWorker(size_t num_backup_threads, size_t num_restore_threads)
|
||||
BackupsWorker::BackupsWorker(size_t num_backup_threads, size_t num_restore_threads, bool allow_concurrent_backups_, bool allow_concurrent_restores_)
|
||||
: backups_thread_pool(num_backup_threads, /* max_free_threads = */ 0, num_backup_threads)
|
||||
, restores_thread_pool(num_restore_threads, /* max_free_threads = */ 0, num_restore_threads)
|
||||
, log(&Poco::Logger::get("BackupsWorker"))
|
||||
, allow_concurrent_backups(allow_concurrent_backups_)
|
||||
, allow_concurrent_restores(allow_concurrent_restores_)
|
||||
{
|
||||
/// We set max_free_threads = 0 because we don't want to keep any threads if there is no BACKUP or RESTORE query running right now.
|
||||
}
|
||||
@ -157,6 +160,16 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
|
||||
else
|
||||
backup_id = toString(*backup_settings.backup_uuid);
|
||||
|
||||
/// Check if there are no concurrent backups
|
||||
if (num_active_backups && !allow_concurrent_backups)
|
||||
{
|
||||
/// If its an internal backup and we currently have 1 active backup, it could be the original query, validate using backup_uuid
|
||||
if (!(num_active_backups == 1 && backup_settings.internal && getAllActiveBackupInfos().at(0).id == toString(*backup_settings.backup_uuid)))
|
||||
{
|
||||
throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent backups not supported, turn on setting 'allow_concurrent_backups'");
|
||||
}
|
||||
}
|
||||
|
||||
std::shared_ptr<IBackupCoordination> backup_coordination;
|
||||
if (backup_settings.internal)
|
||||
{
|
||||
@ -370,6 +383,9 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
|
||||
auto restore_query = std::static_pointer_cast<ASTBackupQuery>(query->clone());
|
||||
auto restore_settings = RestoreSettings::fromRestoreQuery(*restore_query);
|
||||
|
||||
if (!restore_settings.backup_uuid)
|
||||
restore_settings.backup_uuid = UUIDHelpers::generateV4();
|
||||
|
||||
/// `restore_id` will be used as a key to the `infos` map, so it should be unique.
|
||||
OperationID restore_id;
|
||||
if (restore_settings.internal)
|
||||
@ -377,7 +393,17 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
|
||||
else if (!restore_settings.id.empty())
|
||||
restore_id = restore_settings.id;
|
||||
else
|
||||
restore_id = toString(UUIDHelpers::generateV4());
|
||||
restore_id = toString(*restore_settings.backup_uuid);
|
||||
|
||||
/// Check if there are no concurrent restores
|
||||
if (num_active_restores && !allow_concurrent_restores)
|
||||
{
|
||||
/// If its an internal restore and we currently have 1 active restore, it could be the original query, validate using iz
|
||||
if (!(num_active_restores == 1 && restore_settings.internal && getAllActiveRestoreInfos().at(0).id == toString(*restore_settings.backup_uuid)))
|
||||
{
|
||||
throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent restores not supported, turn on setting 'allow_concurrent_restores'");
|
||||
}
|
||||
}
|
||||
|
||||
std::shared_ptr<IRestoreCoordination> restore_coordination;
|
||||
if (restore_settings.internal)
|
||||
@ -471,6 +497,7 @@ void BackupsWorker::doRestore(
|
||||
backup_open_params.context = context;
|
||||
backup_open_params.backup_info = backup_info;
|
||||
backup_open_params.base_backup_info = restore_settings.base_backup_info;
|
||||
backup_open_params.backup_uuid = restore_settings.backup_uuid;
|
||||
backup_open_params.password = restore_settings.password;
|
||||
BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params);
|
||||
|
||||
@ -687,6 +714,30 @@ std::vector<BackupsWorker::Info> BackupsWorker::getAllInfos() const
|
||||
return res_infos;
|
||||
}
|
||||
|
||||
std::vector<BackupsWorker::Info> BackupsWorker::getAllActiveBackupInfos() const
|
||||
{
|
||||
std::vector<Info> res_infos;
|
||||
std::lock_guard lock{infos_mutex};
|
||||
for (const auto & info : infos | boost::adaptors::map_values)
|
||||
{
|
||||
if (info.status==BackupStatus::CREATING_BACKUP)
|
||||
res_infos.push_back(info);
|
||||
}
|
||||
return res_infos;
|
||||
}
|
||||
|
||||
std::vector<BackupsWorker::Info> BackupsWorker::getAllActiveRestoreInfos() const
|
||||
{
|
||||
std::vector<Info> res_infos;
|
||||
std::lock_guard lock{infos_mutex};
|
||||
for (const auto & info : infos | boost::adaptors::map_values)
|
||||
{
|
||||
if (info.status==BackupStatus::RESTORING)
|
||||
res_infos.push_back(info);
|
||||
}
|
||||
return res_infos;
|
||||
}
|
||||
|
||||
void BackupsWorker::shutdown()
|
||||
{
|
||||
bool has_active_backups_and_restores = (num_active_backups || num_active_restores);
|
||||
|
@ -23,7 +23,7 @@ class IRestoreCoordination;
|
||||
class BackupsWorker
|
||||
{
|
||||
public:
|
||||
BackupsWorker(size_t num_backup_threads, size_t num_restore_threads);
|
||||
BackupsWorker(size_t num_backup_threads, size_t num_restore_threads, bool allow_concurrent_backups_, bool allow_concurrent_restores_);
|
||||
|
||||
/// Waits until all tasks have been completed.
|
||||
void shutdown();
|
||||
@ -103,6 +103,8 @@ private:
|
||||
void setStatus(const OperationID & id, BackupStatus status, bool throw_if_error = true);
|
||||
void setStatusSafe(const String & id, BackupStatus status) { setStatus(id, status, false); }
|
||||
void setNumFilesAndSize(const OperationID & id, size_t num_files, UInt64 uncompressed_size, UInt64 compressed_size);
|
||||
std::vector<Info> getAllActiveBackupInfos() const;
|
||||
std::vector<Info> getAllActiveRestoreInfos() const;
|
||||
|
||||
ThreadPool backups_thread_pool;
|
||||
ThreadPool restores_thread_pool;
|
||||
@ -113,6 +115,8 @@ private:
|
||||
std::atomic<size_t> num_active_restores = 0;
|
||||
mutable std::mutex infos_mutex;
|
||||
Poco::Logger * log;
|
||||
const bool allow_concurrent_backups;
|
||||
const bool allow_concurrent_restores;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <boost/algorithm/string/predicate.hpp>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Backups/SettingsFieldOptionalUUID.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -162,7 +163,9 @@ namespace
|
||||
M(RestoreUDFCreationMode, create_function) \
|
||||
M(Bool, internal) \
|
||||
M(String, host_id) \
|
||||
M(String, coordination_zk_path)
|
||||
M(String, coordination_zk_path) \
|
||||
M(OptionalUUID, backup_uuid)
|
||||
|
||||
|
||||
RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query)
|
||||
{
|
||||
|
@ -122,6 +122,11 @@ struct RestoreSettings
|
||||
/// Path in Zookeeper used to coordinate restoring process while executing by RESTORE ON CLUSTER.
|
||||
String coordination_zk_path;
|
||||
|
||||
/// Internal, should not be specified by user.
|
||||
/// UUID of the backup. If it's not set it will be generated randomly.
|
||||
/// This is used to validate internal restores when allow_concurrent_restores is turned off
|
||||
std::optional<UUID> backup_uuid;
|
||||
|
||||
static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query);
|
||||
void copySettingsToQuery(ASTBackupQuery & query) const;
|
||||
};
|
||||
|
43
src/Backups/SettingsFieldOptionalUUID.cpp
Normal file
43
src/Backups/SettingsFieldOptionalUUID.cpp
Normal file
@ -0,0 +1,43 @@
|
||||
#include <Backups/SettingsFieldOptionalUUID.h>
|
||||
#include <Common/ErrorCodes.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_PARSE_BACKUP_SETTINGS;
|
||||
}
|
||||
|
||||
|
||||
SettingFieldOptionalUUID::SettingFieldOptionalUUID(const Field & field)
|
||||
{
|
||||
if (field.getType() == Field::Types::Null)
|
||||
{
|
||||
value = std::nullopt;
|
||||
return;
|
||||
}
|
||||
|
||||
if (field.getType() == Field::Types::String)
|
||||
{
|
||||
const String & str = field.get<const String &>();
|
||||
if (str.empty())
|
||||
{
|
||||
value = std::nullopt;
|
||||
return;
|
||||
}
|
||||
|
||||
UUID id;
|
||||
if (tryParse(id, str))
|
||||
{
|
||||
value = id;
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Cannot parse uuid from {}", field);
|
||||
}
|
||||
|
||||
}
|
18
src/Backups/SettingsFieldOptionalUUID.h
Normal file
18
src/Backups/SettingsFieldOptionalUUID.h
Normal file
@ -0,0 +1,18 @@
|
||||
#pragma once
|
||||
|
||||
#include <optional>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct SettingFieldOptionalUUID
|
||||
{
|
||||
std::optional<UUID> value;
|
||||
|
||||
explicit SettingFieldOptionalUUID(const std::optional<UUID> & value_) : value(value_) {}
|
||||
|
||||
explicit SettingFieldOptionalUUID(const Field & field);
|
||||
|
||||
explicit operator Field() const { return Field(value ? toString(*value) : ""); }
|
||||
};
|
||||
}
|
@ -100,6 +100,7 @@
|
||||
M(CacheDetachedFileSegments, "Number of existing detached cache file segments") \
|
||||
M(FilesystemCacheSize, "Filesystem cache size in bytes") \
|
||||
M(FilesystemCacheElements, "Filesystem cache elements (file segments)") \
|
||||
M(AsyncInsertCacheSize, "Number of async insert hash id in cache") \
|
||||
M(S3Requests, "S3 requests") \
|
||||
M(KeeperAliveConnections, "Number of alive connections") \
|
||||
M(KeeperOutstandingRequets, "Number of outstanding requests") \
|
||||
|
480
src/Common/FST.cpp
Normal file
480
src/Common/FST.cpp
Normal file
@ -0,0 +1,480 @@
|
||||
#include "FST.h"
|
||||
#include <algorithm>
|
||||
#include <cassert>
|
||||
#include <iostream>
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
#include <Common/Exception.h>
|
||||
#include <city.h>
|
||||
|
||||
/// "paper" in the comments in this file refers to:
|
||||
/// [Direct Construction of Minimal Acyclic Subsequential Transduers] by Stoyan Mihov and Denis Maurel, University of Tours, France
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
};
|
||||
|
||||
namespace FST
|
||||
{
|
||||
|
||||
UInt64 Arc::serialize(WriteBuffer& write_buffer) const
|
||||
{
|
||||
UInt64 written_bytes = 0;
|
||||
bool has_output = output != 0;
|
||||
|
||||
/// First UInt64 is target_index << 1 + has_output
|
||||
assert(target != nullptr);
|
||||
UInt64 first = ((target->state_index) << 1) + has_output;
|
||||
writeVarUInt(first, write_buffer);
|
||||
written_bytes += getLengthOfVarUInt(first);
|
||||
|
||||
/// Second UInt64 is output (optional based on whether has_output is not zero)
|
||||
if (has_output)
|
||||
{
|
||||
writeVarUInt(output, write_buffer);
|
||||
written_bytes += getLengthOfVarUInt(output);
|
||||
}
|
||||
return written_bytes;
|
||||
}
|
||||
|
||||
bool operator==(const Arc & arc1, const Arc & arc2)
|
||||
{
|
||||
assert(arc1.target != nullptr && arc2.target != nullptr);
|
||||
return (arc1.output == arc2.output && arc1.target->id == arc2.target->id);
|
||||
}
|
||||
|
||||
void LabelsAsBitmap::addLabel(char label)
|
||||
{
|
||||
UInt8 index = label;
|
||||
UInt256 bit_label = 1;
|
||||
bit_label <<= index;
|
||||
|
||||
data |= bit_label;
|
||||
}
|
||||
|
||||
UInt64 LabelsAsBitmap::getIndex(char label) const
|
||||
{
|
||||
UInt64 bit_count = 0;
|
||||
|
||||
UInt8 index = label;
|
||||
int which_int64 = 0;
|
||||
while (true)
|
||||
{
|
||||
if (index < 64)
|
||||
{
|
||||
UInt64 mask = index == 63 ? (-1) : (1ULL << (index + 1)) - 1;
|
||||
|
||||
bit_count += std::popcount(mask & data.items[which_int64]);
|
||||
break;
|
||||
}
|
||||
index -= 64;
|
||||
bit_count += std::popcount(data.items[which_int64]);
|
||||
|
||||
which_int64++;
|
||||
}
|
||||
return bit_count;
|
||||
}
|
||||
|
||||
UInt64 LabelsAsBitmap::serialize(WriteBuffer& write_buffer)
|
||||
{
|
||||
writeVarUInt(data.items[0], write_buffer);
|
||||
writeVarUInt(data.items[1], write_buffer);
|
||||
writeVarUInt(data.items[2], write_buffer);
|
||||
writeVarUInt(data.items[3], write_buffer);
|
||||
|
||||
return getLengthOfVarUInt(data.items[0])
|
||||
+ getLengthOfVarUInt(data.items[1])
|
||||
+ getLengthOfVarUInt(data.items[2])
|
||||
+ getLengthOfVarUInt(data.items[3]);
|
||||
}
|
||||
|
||||
bool LabelsAsBitmap::hasLabel(char label) const
|
||||
{
|
||||
UInt8 index = label;
|
||||
UInt256 bit_label = 1;
|
||||
bit_label <<= index;
|
||||
|
||||
return ((data & bit_label) != 0);
|
||||
}
|
||||
|
||||
Arc* State::getArc(char label) const
|
||||
{
|
||||
auto it = arcs.find(label);
|
||||
if (it == arcs.cend())
|
||||
return nullptr;
|
||||
|
||||
return const_cast<Arc *>(&it->second);
|
||||
}
|
||||
|
||||
void State::addArc(char label, Output output, StatePtr target)
|
||||
{
|
||||
arcs[label] = Arc(output, target);
|
||||
}
|
||||
|
||||
void State::clear()
|
||||
{
|
||||
id = 0;
|
||||
state_index = 0;
|
||||
flag = 0;
|
||||
|
||||
arcs.clear();
|
||||
}
|
||||
|
||||
UInt64 State::hash() const
|
||||
{
|
||||
std::vector<char> values;
|
||||
values.reserve(arcs.size() * (sizeof(Output) + sizeof(UInt64) + 1));
|
||||
for (const auto & [label, arc] : arcs)
|
||||
{
|
||||
values.push_back(label);
|
||||
const auto * ptr = reinterpret_cast<const char*>(&arc.output);
|
||||
std::copy(ptr, ptr + sizeof(Output), std::back_inserter(values));
|
||||
|
||||
ptr = reinterpret_cast<const char*>(&arc.target->id);
|
||||
std::copy(ptr, ptr + sizeof(UInt64), std::back_inserter(values));
|
||||
}
|
||||
|
||||
return CityHash_v1_0_2::CityHash64(values.data(), values.size());
|
||||
}
|
||||
|
||||
bool operator== (const State & state1, const State & state2)
|
||||
{
|
||||
if (state1.arcs.size() != state2.arcs.size())
|
||||
return false;
|
||||
|
||||
for (const auto & [label, arc] : state1.arcs)
|
||||
{
|
||||
const auto it = state2.arcs.find(label);
|
||||
if (it == state2.arcs.cend())
|
||||
return false;
|
||||
|
||||
if (it->second != arc)
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
UInt64 State::serialize(WriteBuffer& write_buffer)
|
||||
{
|
||||
UInt64 written_bytes = 0;
|
||||
|
||||
/// Serialize flag
|
||||
write_buffer.write(flag);
|
||||
written_bytes += 1;
|
||||
|
||||
if (getEncodingMethod() == EncodingMethod::Sequential)
|
||||
{
|
||||
/// Serialize all labels
|
||||
std::vector<char> labels;
|
||||
labels.reserve(arcs.size());
|
||||
|
||||
for (auto& [label, state] : arcs)
|
||||
{
|
||||
labels.push_back(label);
|
||||
}
|
||||
|
||||
UInt8 label_size = labels.size();
|
||||
write_buffer.write(label_size);
|
||||
written_bytes += 1;
|
||||
|
||||
write_buffer.write(labels.data(), labels.size());
|
||||
written_bytes += labels.size();
|
||||
|
||||
/// Serialize all arcs
|
||||
for (char label : labels)
|
||||
{
|
||||
Arc* arc = getArc(label);
|
||||
assert(arc != nullptr);
|
||||
written_bytes += arc->serialize(write_buffer);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Serialize bitmap
|
||||
LabelsAsBitmap bmp;
|
||||
for (auto & [label, state] : arcs)
|
||||
{
|
||||
bmp.addLabel(label);
|
||||
}
|
||||
written_bytes += bmp.serialize(write_buffer);
|
||||
|
||||
/// Serialize all arcs
|
||||
for (auto & [label, state] : arcs)
|
||||
{
|
||||
Arc* arc = getArc(label);
|
||||
assert(arc != nullptr);
|
||||
written_bytes += arc->serialize(write_buffer);
|
||||
}
|
||||
}
|
||||
|
||||
return written_bytes;
|
||||
}
|
||||
|
||||
FSTBuilder::FSTBuilder(WriteBuffer& write_buffer_) : write_buffer(write_buffer_)
|
||||
{
|
||||
for (auto & temp_state : temp_states)
|
||||
{
|
||||
temp_state = std::make_shared<State>();
|
||||
}
|
||||
}
|
||||
|
||||
/// See FindMinimized in the paper pseudo code l11-l21.
|
||||
StatePtr FSTBuilder::findMinimized(const State & state, bool & found)
|
||||
{
|
||||
found = false;
|
||||
auto hash = state.hash();
|
||||
|
||||
/// MEMBER: in the paper pseudo code l15
|
||||
auto it = minimized_states.find(hash);
|
||||
|
||||
if (it != minimized_states.cend() && *it->second == state)
|
||||
{
|
||||
found = true;
|
||||
return it->second;
|
||||
}
|
||||
|
||||
/// COPY_STATE: in the paper pseudo code l17
|
||||
StatePtr p = std::make_shared<State>(state);
|
||||
|
||||
/// INSERT: in the paper pseudo code l18
|
||||
minimized_states[hash] = p;
|
||||
return p;
|
||||
}
|
||||
|
||||
/// See the paper pseudo code l33-34.
|
||||
size_t FSTBuilder::getCommonPrefixLength(const String & word1, const String & word2)
|
||||
{
|
||||
size_t i = 0;
|
||||
while (i < word1.size() && i < word2.size() && word1[i] == word2[i])
|
||||
i++;
|
||||
return i;
|
||||
}
|
||||
|
||||
/// See the paper pseudo code l33-39 and l70-72(when down_to is 0).
|
||||
void FSTBuilder::minimizePreviousWordSuffix(Int64 down_to)
|
||||
{
|
||||
for (Int64 i = static_cast<Int64>(previous_word.size()); i >= down_to; --i)
|
||||
{
|
||||
bool found = false;
|
||||
auto minimized_state = findMinimized(*temp_states[i], found);
|
||||
|
||||
if (i != 0)
|
||||
{
|
||||
Output output = 0;
|
||||
Arc* arc = temp_states[i - 1]->getArc(previous_word[i - 1]);
|
||||
if (arc)
|
||||
output = arc->output;
|
||||
|
||||
/// SET_TRANSITION
|
||||
temp_states[i - 1]->addArc(previous_word[i - 1], output, minimized_state);
|
||||
}
|
||||
if (minimized_state->id == 0)
|
||||
minimized_state->id = next_id++;
|
||||
|
||||
if (i > 0 && temp_states[i - 1]->id == 0)
|
||||
temp_states[i - 1]->id = next_id++;
|
||||
|
||||
if (!found)
|
||||
{
|
||||
minimized_state->state_index = previous_state_index;
|
||||
|
||||
previous_written_bytes = minimized_state->serialize(write_buffer);
|
||||
previous_state_index += previous_written_bytes;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void FSTBuilder::add(const std::string & current_word, Output current_output)
|
||||
{
|
||||
/// We assume word size is no greater than MAX_TERM_LENGTH(256).
|
||||
/// FSTs without word size limitation would be inefficient and easy to cause memory bloat
|
||||
/// Note that when using "split" tokenizer, if a granule has tokens which are longer than
|
||||
/// MAX_TERM_LENGTH, the granule cannot be dropped and will be fully-scanned. It doesn't affect "ngram" tokenizers.
|
||||
/// Another limitation is that if the query string has tokens which exceed this length
|
||||
/// it will fallback to default searching when using "split" tokenizers.
|
||||
auto current_word_len = current_word.size();
|
||||
|
||||
if (current_word_len > MAX_TERM_LENGTH)
|
||||
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Too long term ({}) passed to FST builder.", current_word_len);
|
||||
|
||||
size_t prefix_length_plus1 = getCommonPrefixLength(current_word, previous_word) + 1;
|
||||
|
||||
minimizePreviousWordSuffix(prefix_length_plus1);
|
||||
|
||||
/// Initialize the tail state, see paper pseudo code l39-43
|
||||
for (size_t i = prefix_length_plus1; i <= current_word.size(); ++i)
|
||||
{
|
||||
/// CLEAR_STATE: l41
|
||||
temp_states[i]->clear();
|
||||
|
||||
/// SET_TRANSITION: l42
|
||||
temp_states[i - 1]->addArc(current_word[i - 1], 0, temp_states[i]);
|
||||
}
|
||||
|
||||
/// We assume the current word is different with previous word
|
||||
/// See paper pseudo code l44-47
|
||||
temp_states[current_word_len]->setFinal(true);
|
||||
|
||||
/// Adjust outputs on the arcs
|
||||
/// See paper pseudo code l48-63
|
||||
for (size_t i = 1; i <= prefix_length_plus1 - 1; ++i)
|
||||
{
|
||||
Arc * arc_ptr = temp_states[i - 1]->getArc(current_word[i - 1]);
|
||||
assert(arc_ptr != nullptr);
|
||||
|
||||
Output common_prefix = std::min(arc_ptr->output, current_output);
|
||||
Output word_suffix = arc_ptr->output - common_prefix;
|
||||
arc_ptr->output = common_prefix;
|
||||
|
||||
/// For each arc, adjust its output
|
||||
if (word_suffix != 0)
|
||||
{
|
||||
for (auto & [label, arc] : temp_states[i]->arcs)
|
||||
{
|
||||
arc.output += word_suffix;
|
||||
}
|
||||
}
|
||||
/// Reduce current_output
|
||||
current_output -= common_prefix;
|
||||
}
|
||||
|
||||
/// Set last temp state's output
|
||||
/// paper pseudo code l66-67 (assuming CurrentWord != PreviousWorld)
|
||||
Arc * arc = temp_states[prefix_length_plus1 - 1]->getArc(current_word[prefix_length_plus1 - 1]);
|
||||
assert(arc != nullptr);
|
||||
arc->output = current_output;
|
||||
|
||||
previous_word = current_word;
|
||||
}
|
||||
|
||||
UInt64 FSTBuilder::build()
|
||||
{
|
||||
minimizePreviousWordSuffix(0);
|
||||
|
||||
/// Save initial state index
|
||||
|
||||
previous_state_index -= previous_written_bytes;
|
||||
UInt8 length = getLengthOfVarUInt(previous_state_index);
|
||||
writeVarUInt(previous_state_index, write_buffer);
|
||||
write_buffer.write(length);
|
||||
|
||||
return previous_state_index + previous_written_bytes + length + 1;
|
||||
}
|
||||
|
||||
FiniteStateTransducer::FiniteStateTransducer(std::vector<UInt8> data_) : data(std::move(data_))
|
||||
{
|
||||
}
|
||||
|
||||
void FiniteStateTransducer::clear()
|
||||
{
|
||||
data.clear();
|
||||
}
|
||||
|
||||
std::pair<UInt64, bool> FiniteStateTransducer::getOutput(const String & term)
|
||||
{
|
||||
std::pair<UInt64, bool> result{ 0, false };
|
||||
|
||||
/// Read index of initial state
|
||||
ReadBufferFromMemory read_buffer(data.data(), data.size());
|
||||
read_buffer.seek(data.size()-1, SEEK_SET);
|
||||
|
||||
UInt8 length{ 0 };
|
||||
read_buffer.readStrict(reinterpret_cast<char&>(length));
|
||||
|
||||
/// FST contains no terms
|
||||
if (length == 0)
|
||||
return { 0, false };
|
||||
|
||||
read_buffer.seek(data.size() - 1 - length, SEEK_SET);
|
||||
UInt64 state_index{ 0 };
|
||||
readVarUInt(state_index, read_buffer);
|
||||
|
||||
for (size_t i = 0; i <= term.size(); ++i)
|
||||
{
|
||||
UInt64 arc_output{ 0 };
|
||||
|
||||
/// Read flag
|
||||
State temp_state;
|
||||
|
||||
read_buffer.seek(state_index, SEEK_SET);
|
||||
temp_state.readFlag(read_buffer);
|
||||
if (i == term.size())
|
||||
{
|
||||
result.second = temp_state.isFinal();
|
||||
break;
|
||||
}
|
||||
|
||||
UInt8 label = term[i];
|
||||
if (temp_state.getEncodingMethod() == State::EncodingMethod::Sequential)
|
||||
{
|
||||
/// Read number of labels
|
||||
UInt8 label_num{ 0 };
|
||||
read_buffer.readStrict(reinterpret_cast<char&>(label_num));
|
||||
|
||||
if (label_num == 0)
|
||||
return { 0, false };
|
||||
|
||||
auto labels_position = read_buffer.getPosition();
|
||||
|
||||
/// Find the index of the label from "labels" bytes
|
||||
auto begin_it{ data.begin() + labels_position };
|
||||
auto end_it{ data.begin() + labels_position + label_num };
|
||||
|
||||
auto pos = std::find(begin_it, end_it, label);
|
||||
|
||||
if (pos == end_it)
|
||||
return { 0, false };
|
||||
|
||||
/// Read the arc for the label
|
||||
UInt64 arc_index = (pos - begin_it);
|
||||
auto arcs_start_postion = labels_position + label_num;
|
||||
|
||||
read_buffer.seek(arcs_start_postion, SEEK_SET);
|
||||
for (size_t j = 0; j <= arc_index; j++)
|
||||
{
|
||||
state_index = 0;
|
||||
arc_output = 0;
|
||||
readVarUInt(state_index, read_buffer);
|
||||
if (state_index & 0x1) // output is followed
|
||||
{
|
||||
readVarUInt(arc_output, read_buffer);
|
||||
}
|
||||
state_index >>= 1;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
LabelsAsBitmap bmp;
|
||||
|
||||
readVarUInt(bmp.data.items[0], read_buffer);
|
||||
readVarUInt(bmp.data.items[1], read_buffer);
|
||||
readVarUInt(bmp.data.items[2], read_buffer);
|
||||
readVarUInt(bmp.data.items[3], read_buffer);
|
||||
|
||||
if (!bmp.hasLabel(label))
|
||||
return { 0, false };
|
||||
|
||||
/// Read the arc for the label
|
||||
size_t arc_index = bmp.getIndex(label);
|
||||
for (size_t j = 0; j < arc_index; j++)
|
||||
{
|
||||
state_index = 0;
|
||||
arc_output = 0;
|
||||
readVarUInt(state_index, read_buffer);
|
||||
if (state_index & 0x1) // output is followed
|
||||
{
|
||||
readVarUInt(arc_output, read_buffer);
|
||||
}
|
||||
state_index >>= 1;
|
||||
}
|
||||
}
|
||||
/// Accumulate the output value
|
||||
result.first += arc_output;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
}
|
182
src/Common/FST.h
Normal file
182
src/Common/FST.h
Normal file
@ -0,0 +1,182 @@
|
||||
#pragma once
|
||||
#include <array>
|
||||
#include <map>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <unordered_map>
|
||||
#include <unordered_set>
|
||||
#include <vector>
|
||||
#include <Core/Types.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <base/types.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/// Finite State Transducer is an efficient way to represent term dictionary.
|
||||
/// It can be viewed as a map of <term, output> where output is an integer.
|
||||
/// Detailed explanation can be found in the following paper
|
||||
/// [Direct Construction of Minimal Acyclic Subsequential Transduers] by Stoyan Mihov and Denis Maurel, University of Tours, France
|
||||
namespace FST
|
||||
{
|
||||
using Output = UInt64;
|
||||
|
||||
class State;
|
||||
using StatePtr = std::shared_ptr<State>;
|
||||
|
||||
/// Arc represents a transition from one state to another
|
||||
/// It includes the target state to which the arc points and the arc's output.
|
||||
struct Arc
|
||||
{
|
||||
Arc() = default;
|
||||
|
||||
explicit Arc(Output output_, const StatePtr & target_) : output{output_}, target{target_} { }
|
||||
|
||||
/// 0 means the arc has no output
|
||||
Output output = 0;
|
||||
|
||||
StatePtr target;
|
||||
|
||||
UInt64 serialize(WriteBuffer & write_buffer) const;
|
||||
};
|
||||
|
||||
bool operator==(const Arc & arc1, const Arc & arc2);
|
||||
|
||||
/// LabelsAsBitmap implements a 256-bit bitmap for all labels of a state. Each bit represents
|
||||
/// a label's presence and the index value of the bit represents the corresponding label
|
||||
class LabelsAsBitmap
|
||||
{
|
||||
public:
|
||||
void addLabel(char label);
|
||||
bool hasLabel(char label) const;
|
||||
|
||||
/// computes the rank
|
||||
UInt64 getIndex(char label) const;
|
||||
|
||||
UInt64 serialize(WriteBuffer& write_buffer);
|
||||
private:
|
||||
friend class State;
|
||||
friend class FiniteStateTransducer;
|
||||
/// data holds a 256-bit bitmap for all labels of a state. Its 256 bits correspond to 256
|
||||
/// possible label values.
|
||||
UInt256 data{ 0 };
|
||||
};
|
||||
|
||||
/// State implements the State in Finite State Transducer
|
||||
/// Each state contains all its arcs and a flag indicating if it is final state
|
||||
class State
|
||||
{
|
||||
public:
|
||||
static constexpr size_t MAX_ARCS_IN_SEQUENTIAL_METHOD = 32;
|
||||
enum class EncodingMethod
|
||||
{
|
||||
/// Serialize arcs sequentially
|
||||
Sequential = 0,
|
||||
|
||||
/// Serialize arcs by using bitmap
|
||||
/// Note this is NOT enabled for now since it is experimental
|
||||
Bitmap,
|
||||
};
|
||||
State() = default;
|
||||
|
||||
State(const State & state) = default;
|
||||
|
||||
UInt64 hash() const;
|
||||
|
||||
Arc * getArc(char label) const;
|
||||
|
||||
void addArc(char label, Output output, StatePtr target);
|
||||
|
||||
void clear();
|
||||
|
||||
UInt64 serialize(WriteBuffer & write_buffer);
|
||||
|
||||
bool isFinal() const
|
||||
{
|
||||
return flag_values.is_final == 1;
|
||||
}
|
||||
void setFinal(bool value)
|
||||
{
|
||||
flag_values.is_final = value;
|
||||
}
|
||||
EncodingMethod getEncodingMethod() const
|
||||
{
|
||||
return flag_values.encoding_method;
|
||||
}
|
||||
void readFlag(ReadBuffer & read_buffer)
|
||||
{
|
||||
read_buffer.readStrict(reinterpret_cast<char&>(flag));
|
||||
}
|
||||
|
||||
/// Transient ID of the state which is used for building FST. It won't be serialized
|
||||
UInt64 id = 0;
|
||||
|
||||
/// State index which indicates location of state in FST
|
||||
UInt64 state_index = 0;
|
||||
|
||||
/// Arcs which are started from state, the 'char' is the label on the arc
|
||||
std::unordered_map<char, Arc> arcs;
|
||||
private:
|
||||
struct FlagValues
|
||||
{
|
||||
unsigned int is_final : 1;
|
||||
EncodingMethod encoding_method : 3;
|
||||
};
|
||||
|
||||
union
|
||||
{
|
||||
FlagValues flag_values;
|
||||
uint8_t flag = 0;
|
||||
};
|
||||
};
|
||||
|
||||
bool operator==(const State & state1, const State & state2);
|
||||
|
||||
inline constexpr size_t MAX_TERM_LENGTH = 256;
|
||||
|
||||
/// FSTBuilder is used to build Finite State Transducer by adding words incrementally.
|
||||
/// Note that all the words have to be added in sorted order in order to achieve minimized result.
|
||||
/// In the end, the caller should call build() to serialize minimized FST to WriteBuffer
|
||||
class FSTBuilder
|
||||
{
|
||||
public:
|
||||
explicit FSTBuilder(WriteBuffer & write_buffer_);
|
||||
|
||||
void add(const std::string & word, Output output);
|
||||
UInt64 build();
|
||||
private:
|
||||
StatePtr findMinimized(const State & s, bool & found);
|
||||
void minimizePreviousWordSuffix(Int64 down_to);
|
||||
static size_t getCommonPrefixLength(const String & word1, const String & word2);
|
||||
|
||||
std::array<StatePtr, MAX_TERM_LENGTH + 1> temp_states;
|
||||
String previous_word;
|
||||
StatePtr initial_state;
|
||||
|
||||
/// map of (state_hash, StatePtr)
|
||||
std::unordered_map<UInt64, StatePtr> minimized_states;
|
||||
|
||||
/// Next available ID of state
|
||||
UInt64 next_id = 1;
|
||||
|
||||
WriteBuffer & write_buffer;
|
||||
UInt64 previous_written_bytes = 0;
|
||||
UInt64 previous_state_index = 0;
|
||||
};
|
||||
|
||||
//FiniteStateTransducer is constructed by using minimized FST blob(which is loaded from index storage)
|
||||
// It is used to retrieve output by given term
|
||||
class FiniteStateTransducer
|
||||
{
|
||||
public:
|
||||
FiniteStateTransducer() = default;
|
||||
explicit FiniteStateTransducer(std::vector<UInt8> data_);
|
||||
std::pair<UInt64, bool> getOutput(const String & term);
|
||||
void clear();
|
||||
std::vector<UInt8> & getData() { return data; }
|
||||
|
||||
private:
|
||||
std::vector<UInt8> data;
|
||||
};
|
||||
}
|
||||
}
|
@ -10,6 +10,7 @@
|
||||
M(InsertQuery, "Same as Query, but only for INSERT queries.") \
|
||||
M(AsyncInsertQuery, "Same as InsertQuery, but only for asynchronous INSERT queries.") \
|
||||
M(AsyncInsertBytes, "Data size in bytes of asynchronous INSERT queries.") \
|
||||
M(AsyncInsertCacheHits, "Number of times a duplicate hash id has been found in asynchronous INSERT hash id cache.") \
|
||||
M(FailedQuery, "Number of failed queries.") \
|
||||
M(FailedSelectQuery, "Same as FailedQuery, but only for SELECT queries.") \
|
||||
M(FailedInsertQuery, "Same as FailedQuery, but only for INSERT queries.") \
|
||||
|
94
src/Common/tests/gtest_fst.cpp
Normal file
94
src/Common/tests/gtest_fst.cpp
Normal file
@ -0,0 +1,94 @@
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
#include <IO/WriteBufferFromVector.h>
|
||||
#include <Common/FST.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
TEST(FST, SimpleTest)
|
||||
{
|
||||
std::vector<std::pair<std::string, DB::FST::Output>> indexed_data
|
||||
{
|
||||
{"mop", 100},
|
||||
{"moth", 91},
|
||||
{"pop", 72},
|
||||
{"star", 83},
|
||||
{"stop", 54},
|
||||
{"top", 55},
|
||||
};
|
||||
|
||||
std::vector<std::pair<std::string, DB::FST::Output>> not_indexed_data
|
||||
{
|
||||
{"mo", 100},
|
||||
{"moth1", 91},
|
||||
{"po", 72},
|
||||
{"star2", 83},
|
||||
{"sto", 54},
|
||||
{"top33", 55},
|
||||
};
|
||||
|
||||
std::vector<UInt8> buffer;
|
||||
DB::WriteBufferFromVector<std::vector<UInt8>> wbuf(buffer);
|
||||
DB::FST::FSTBuilder builder(wbuf);
|
||||
|
||||
for (auto& [term, output] : indexed_data)
|
||||
{
|
||||
builder.add(term, output);
|
||||
}
|
||||
builder.build();
|
||||
wbuf.finalize();
|
||||
|
||||
DB::FST::FiniteStateTransducer fst(buffer);
|
||||
for (auto& [term, output] : indexed_data)
|
||||
{
|
||||
auto [result, found] = fst.getOutput(term);
|
||||
ASSERT_EQ(found, true);
|
||||
ASSERT_EQ(result, output);
|
||||
}
|
||||
|
||||
for (auto& [term, output] : not_indexed_data)
|
||||
{
|
||||
auto [result, found] = fst.getOutput(term);
|
||||
ASSERT_EQ(found, false);
|
||||
}
|
||||
}
|
||||
|
||||
TEST(FST, TestForLongTerms)
|
||||
{
|
||||
/// Test long terms within limitation
|
||||
std::string term1(DB::FST::MAX_TERM_LENGTH - 1, 'A');
|
||||
std::string term2(DB::FST::MAX_TERM_LENGTH, 'B');
|
||||
|
||||
DB::FST::Output output1 = 100;
|
||||
DB::FST::Output output2 = 200;
|
||||
|
||||
std::vector<UInt8> buffer;
|
||||
DB::WriteBufferFromVector<std::vector<UInt8>> wbuf(buffer);
|
||||
DB::FST::FSTBuilder builder(wbuf);
|
||||
|
||||
builder.add(term1, output1);
|
||||
builder.add(term2, output2);
|
||||
|
||||
builder.build();
|
||||
wbuf.finalize();
|
||||
|
||||
DB::FST::FiniteStateTransducer fst(buffer);
|
||||
|
||||
auto [result1, found1] = fst.getOutput(term1);
|
||||
ASSERT_EQ(found1, true);
|
||||
ASSERT_EQ(result1, output1);
|
||||
|
||||
auto [result2, found2] = fst.getOutput(term2);
|
||||
ASSERT_EQ(found2, true);
|
||||
ASSERT_EQ(result2, output2);
|
||||
|
||||
/// Test exception case when term length exceeds limitation
|
||||
std::string term3(DB::FST::MAX_TERM_LENGTH + 1, 'C');
|
||||
DB::FST::Output output3 = 300;
|
||||
|
||||
std::vector<UInt8> buffer3;
|
||||
DB::WriteBufferFromVector<std::vector<UInt8>> wbuf3(buffer3);
|
||||
DB::FST::FSTBuilder builder3(wbuf3);
|
||||
|
||||
EXPECT_THROW(builder3.add(term3, output3), DB::Exception);
|
||||
}
|
@ -185,6 +185,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
|
||||
M(UInt64, merge_tree_max_rows_to_use_cache, (128 * 8192), "The maximum number of rows per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \
|
||||
M(UInt64, merge_tree_max_bytes_to_use_cache, (192 * 10 * 1024 * 1024), "The maximum number of bytes per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \
|
||||
M(Bool, do_not_merge_across_partitions_select_final, false, "Merge parts only in one partition in select final", 0) \
|
||||
M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \
|
||||
\
|
||||
M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \
|
||||
\
|
||||
|
@ -1,9 +1,10 @@
|
||||
#include <DataTypes/Serializations/SerializationUUID.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/Serializations/SerializationUUID.h>
|
||||
#include <Formats/ProtobufReader.h>
|
||||
#include <Formats/ProtobufWriter.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
|
||||
@ -44,11 +45,44 @@ void SerializationUUID::serializeTextQuoted(const IColumn & column, size_t row_n
|
||||
|
||||
void SerializationUUID::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
UUID x;
|
||||
assertChar('\'', istr);
|
||||
readText(x, istr);
|
||||
assertChar('\'', istr);
|
||||
assert_cast<ColumnUUID &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
|
||||
UUID uuid;
|
||||
bool fast = false;
|
||||
if (istr.available() >= 38)
|
||||
{
|
||||
assertChar('\'', istr);
|
||||
char * next_pos = find_first_symbols<'\\', '\''>(istr.position(), istr.buffer().end());
|
||||
size_t len = next_pos - istr.position();
|
||||
if ((len == 32) && (istr.position()[32] == '\''))
|
||||
{
|
||||
parseUUIDWithoutSeparator(
|
||||
reinterpret_cast<const UInt8 *>(istr.position()), std::reverse_iterator<UInt8 *>(reinterpret_cast<UInt8 *>(&uuid) + 16));
|
||||
istr.ignore(33);
|
||||
fast = true;
|
||||
}
|
||||
else if ((len == 36) && (istr.position()[36] == '\''))
|
||||
{
|
||||
parseUUID(
|
||||
reinterpret_cast<const UInt8 *>(istr.position()), std::reverse_iterator<UInt8 *>(reinterpret_cast<UInt8 *>(&uuid) + 16));
|
||||
istr.ignore(37);
|
||||
fast = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
// It's ok to go back in the position because we haven't read from the buffer except the first char
|
||||
// and we know there were at least 38 bytes available (so no new read has been triggered)
|
||||
istr.position()--;
|
||||
}
|
||||
}
|
||||
|
||||
if (!fast)
|
||||
{
|
||||
String quoted_chars;
|
||||
readQuotedStringInto<false>(quoted_chars, istr);
|
||||
ReadBufferFromString parsed_quoted_buffer(quoted_chars);
|
||||
readText(uuid, parsed_quoted_buffer);
|
||||
}
|
||||
|
||||
assert_cast<ColumnUUID &>(column).getData().push_back(std::move(uuid)); /// It's important to do this at the end - for exception safety.
|
||||
}
|
||||
|
||||
void SerializationUUID::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
|
@ -243,13 +243,23 @@ void RegExpTreeDictionary::loadData()
|
||||
initRegexNodes(block);
|
||||
}
|
||||
initGraph();
|
||||
if (regexps.empty())
|
||||
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "There are no available regular expression. Please check your config");
|
||||
#if USE_VECTORSCAN
|
||||
std::vector<std::string_view> regexps_views(regexps.begin(), regexps.end());
|
||||
hyperscan_regex = MultiRegexps::getOrSet<true, false>(regexps_views, std::nullopt);
|
||||
/// TODO: fallback when exceptions occur.
|
||||
hyperscan_regex->get();
|
||||
try
|
||||
{
|
||||
std::vector<std::string_view> regexps_views(regexps.begin(), regexps.end());
|
||||
hyperscan_regex = MultiRegexps::getOrSet<true, false>(regexps_views, std::nullopt);
|
||||
hyperscan_regex->get();
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
/// Some compile errors will be thrown as LOGICAL ERROR and cause crash, e.g. empty expression or expressions are too large.
|
||||
/// We catch the error here and rethrow again.
|
||||
/// TODO: fallback to other engine, like re2, when exceptions occur.
|
||||
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Error occurs when compiling regular expressions, reason: {}", e.message());
|
||||
}
|
||||
#endif
|
||||
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -289,6 +289,12 @@ std::unique_ptr<ReadBufferFromFileBase> DiskEncrypted::readFile(
|
||||
std::optional<size_t> read_hint,
|
||||
std::optional<size_t> file_size) const
|
||||
{
|
||||
if (read_hint && *read_hint > 0)
|
||||
read_hint = *read_hint + FileEncryption::Header::kSize;
|
||||
|
||||
if (file_size && *file_size > 0)
|
||||
file_size = *file_size + FileEncryption::Header::kSize;
|
||||
|
||||
auto wrapped_path = wrappedPath(path);
|
||||
auto buffer = delegate->readFile(wrapped_path, settings, read_hint, file_size);
|
||||
if (buffer->eof())
|
||||
|
@ -55,9 +55,9 @@ protected:
|
||||
return temp_dir->path() + "/";
|
||||
}
|
||||
|
||||
String getFileContents(const String & file_name)
|
||||
String getFileContents(const String & file_name, std::optional<size_t> file_size = {})
|
||||
{
|
||||
auto buf = encrypted_disk->readFile(file_name, /* settings= */ {}, /* read_hint= */ {}, /* file_size= */ {});
|
||||
auto buf = encrypted_disk->readFile(file_name, /* settings= */ {}, /* read_hint= */ {}, file_size);
|
||||
String str;
|
||||
readStringUntilEOF(str, *buf);
|
||||
return str;
|
||||
@ -108,6 +108,10 @@ TEST_F(DiskEncryptedTest, WriteAndRead)
|
||||
EXPECT_EQ(getFileContents("a.txt"), "Some text");
|
||||
checkBinaryRepresentation(getDirectory() + "a.txt", kHeaderSize + 9);
|
||||
|
||||
/// Read the file with specified file size.
|
||||
EXPECT_EQ(getFileContents("a.txt", 9), "Some text");
|
||||
checkBinaryRepresentation(getDirectory() + "a.txt", kHeaderSize + 9);
|
||||
|
||||
/// Remove the file.
|
||||
encrypted_disk->removeFile("a.txt");
|
||||
|
||||
|
@ -39,8 +39,8 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
template <typename IteratorSrc, typename IteratorDst>
|
||||
void parseHex(IteratorSrc src, IteratorDst dst, const size_t num_bytes)
|
||||
template <size_t num_bytes, typename IteratorSrc, typename IteratorDst>
|
||||
inline void parseHex(IteratorSrc src, IteratorDst dst)
|
||||
{
|
||||
size_t src_pos = 0;
|
||||
size_t dst_pos = 0;
|
||||
@ -52,18 +52,18 @@ void parseUUID(const UInt8 * src36, UInt8 * dst16)
|
||||
{
|
||||
/// If string is not like UUID - implementation specific behaviour.
|
||||
|
||||
parseHex(&src36[0], &dst16[0], 4);
|
||||
parseHex(&src36[9], &dst16[4], 2);
|
||||
parseHex(&src36[14], &dst16[6], 2);
|
||||
parseHex(&src36[19], &dst16[8], 2);
|
||||
parseHex(&src36[24], &dst16[10], 6);
|
||||
parseHex<4>(&src36[0], &dst16[0]);
|
||||
parseHex<2>(&src36[9], &dst16[4]);
|
||||
parseHex<2>(&src36[14], &dst16[6]);
|
||||
parseHex<2>(&src36[19], &dst16[8]);
|
||||
parseHex<6>(&src36[24], &dst16[10]);
|
||||
}
|
||||
|
||||
void parseUUIDWithoutSeparator(const UInt8 * src36, UInt8 * dst16)
|
||||
{
|
||||
/// If string is not like UUID - implementation specific behaviour.
|
||||
|
||||
parseHex(&src36[0], &dst16[0], 16);
|
||||
parseHex<16>(&src36[0], &dst16[0]);
|
||||
}
|
||||
|
||||
/** Function used when byte ordering is important when parsing uuid
|
||||
@ -74,11 +74,11 @@ void parseUUID(const UInt8 * src36, std::reverse_iterator<UInt8 *> dst16)
|
||||
/// If string is not like UUID - implementation specific behaviour.
|
||||
|
||||
/// FIXME This code looks like trash.
|
||||
parseHex(&src36[0], dst16 + 8, 4);
|
||||
parseHex(&src36[9], dst16 + 12, 2);
|
||||
parseHex(&src36[14], dst16 + 14, 2);
|
||||
parseHex(&src36[19], dst16, 2);
|
||||
parseHex(&src36[24], dst16 + 2, 6);
|
||||
parseHex<4>(&src36[0], dst16 + 8);
|
||||
parseHex<2>(&src36[9], dst16 + 12);
|
||||
parseHex<2>(&src36[14], dst16 + 14);
|
||||
parseHex<2>(&src36[19], dst16);
|
||||
parseHex<6>(&src36[24], dst16 + 2);
|
||||
}
|
||||
|
||||
/** Function used when byte ordering is important when parsing uuid
|
||||
@ -88,8 +88,8 @@ void parseUUIDWithoutSeparator(const UInt8 * src36, std::reverse_iterator<UInt8
|
||||
{
|
||||
/// If string is not like UUID - implementation specific behaviour.
|
||||
|
||||
parseHex(&src36[0], dst16 + 8, 8);
|
||||
parseHex(&src36[16], dst16, 8);
|
||||
parseHex<8>(&src36[0], dst16 + 8);
|
||||
parseHex<8>(&src36[16], dst16);
|
||||
}
|
||||
|
||||
void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf)
|
||||
|
@ -1888,8 +1888,11 @@ BackupsWorker & Context::getBackupsWorker() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
const bool allow_concurrent_backups = this->getConfigRef().getBool("allow_concurrent_backups", true);
|
||||
const bool allow_concurrent_restores = this->getConfigRef().getBool("allow_concurrent_restores", true);
|
||||
|
||||
if (!shared->backups_worker)
|
||||
shared->backups_worker.emplace(getSettingsRef().backup_threads, getSettingsRef().restore_threads);
|
||||
shared->backups_worker.emplace(getSettingsRef().backup_threads, getSettingsRef().restore_threads, allow_concurrent_backups, allow_concurrent_restores);
|
||||
|
||||
return *shared->backups_worker;
|
||||
}
|
||||
|
185
src/Interpreters/GinFilter.cpp
Normal file
185
src/Interpreters/GinFilter.cpp
Normal file
@ -0,0 +1,185 @@
|
||||
#include <string>
|
||||
#include <algorithm>
|
||||
#include <city.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexGin.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexFullText.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Storages/MergeTree/GinIndexStore.h>
|
||||
#include <Interpreters/GinFilter.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
GinFilterParameters::GinFilterParameters(size_t ngrams_, Float64 density_)
|
||||
: ngrams(ngrams_), density(density_)
|
||||
{
|
||||
if (ngrams > 8)
|
||||
throw Exception("The size of gin filter cannot be greater than 8", ErrorCodes::BAD_ARGUMENTS);
|
||||
if (density <= 0 || density > 1)
|
||||
throw Exception("The density of gin filter must be between 0 and 1", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
GinFilter::GinFilter(const GinFilterParameters & params_)
|
||||
: params(params_)
|
||||
{
|
||||
}
|
||||
|
||||
void GinFilter::add(const char* data, size_t len, UInt32 rowID, GinIndexStorePtr& store, UInt64 limit) const
|
||||
{
|
||||
if (len > FST::MAX_TERM_LENGTH)
|
||||
return;
|
||||
|
||||
String term(data, len);
|
||||
auto it = store->getPostings().find(term);
|
||||
|
||||
if (it != store->getPostings().end())
|
||||
{
|
||||
if (!it->second->contains(rowID))
|
||||
it->second->add(rowID);
|
||||
}
|
||||
else
|
||||
{
|
||||
UInt64 threshold = std::lround(limit * params.density);
|
||||
GinIndexStore::GinIndexPostingsBuilderPtr builder = std::make_shared<GinIndexPostingsBuilder>(threshold);
|
||||
builder->add(rowID);
|
||||
|
||||
store->setPostingsBuilder(term, builder);
|
||||
}
|
||||
}
|
||||
|
||||
/// This method assumes segmentIDs are in increasing order, which is true since rows are
|
||||
/// digested sequentially and segments are created sequentially too.
|
||||
void GinFilter::addRowRangeToGinFilter(UInt32 segmentID, UInt32 rowIDStart, UInt32 rowIDEnd)
|
||||
{
|
||||
/// check segment ids are monotonic increasing
|
||||
assert(rowid_ranges.empty() || rowid_ranges.back().segment_id <= segmentID);
|
||||
|
||||
if (!rowid_ranges.empty())
|
||||
{
|
||||
/// Try to merge the rowID range with the last one in the container
|
||||
GinSegmentWithRowIDRange & last_rowid_range = rowid_ranges.back();
|
||||
|
||||
if (last_rowid_range.segment_id == segmentID &&
|
||||
last_rowid_range.range_end+1 == rowIDStart)
|
||||
{
|
||||
last_rowid_range.range_end = rowIDEnd;
|
||||
return;
|
||||
}
|
||||
}
|
||||
rowid_ranges.push_back({segmentID, rowIDStart, rowIDEnd});
|
||||
}
|
||||
|
||||
void GinFilter::clear()
|
||||
{
|
||||
terms.clear();
|
||||
rowid_ranges.clear();
|
||||
query_string.clear();
|
||||
}
|
||||
|
||||
bool GinFilter::hasEmptyPostingsList(const PostingsCache& postings_cache)
|
||||
{
|
||||
if (postings_cache.empty())
|
||||
return true;
|
||||
|
||||
for (const auto& term_postings : postings_cache)
|
||||
{
|
||||
const SegmentedPostingsListContainer& container = term_postings.second;
|
||||
if (container.empty())
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
bool GinFilter::matchInRange(const PostingsCache& postings_cache, UInt32 segment_id, UInt32 range_start, UInt32 range_end)
|
||||
{
|
||||
/// Check for each terms
|
||||
GinIndexPostingsList intersection_result;
|
||||
bool intersection_result_init = false;
|
||||
|
||||
for (const auto& term_postings : postings_cache)
|
||||
{
|
||||
/// Check if it is in the same segment by searching for segment_id
|
||||
const SegmentedPostingsListContainer& container = term_postings.second;
|
||||
auto container_it = container.find(segment_id);
|
||||
if (container_it == container.cend())
|
||||
{
|
||||
return false;
|
||||
}
|
||||
auto min_in_container = container_it->second->minimum();
|
||||
auto max_in_container = container_it->second->maximum();
|
||||
|
||||
//check if the postings list has always match flag
|
||||
if (container_it->second->cardinality() == 1 && UINT32_MAX == min_in_container)
|
||||
{
|
||||
continue; //always match
|
||||
}
|
||||
|
||||
if (range_start > max_in_container || min_in_container > range_end)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Delay initialization as late as possible
|
||||
if (!intersection_result_init)
|
||||
{
|
||||
intersection_result_init = true;
|
||||
intersection_result.addRange(range_start, range_end+1);
|
||||
}
|
||||
intersection_result &= *container_it->second;
|
||||
if (intersection_result.cardinality() == 0)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool GinFilter::match(const PostingsCache& postings_cache) const
|
||||
{
|
||||
if (hasEmptyPostingsList(postings_cache))
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Check for each row ID ranges
|
||||
for (const auto &rowid_range: rowid_ranges)
|
||||
{
|
||||
if (matchInRange(postings_cache, rowid_range.segment_id, rowid_range.range_start, rowid_range.range_end))
|
||||
{
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
bool GinFilter::contains(const GinFilter & filter, PostingsCacheForStore &cache_store) const
|
||||
{
|
||||
if (filter.getTerms().empty())
|
||||
return true;
|
||||
|
||||
PostingsCachePtr postings_cache = cache_store.getPostings(filter.getQueryString());
|
||||
if (postings_cache == nullptr)
|
||||
{
|
||||
GinIndexStoreDeserializer reader(cache_store.store);
|
||||
postings_cache = reader.createPostingsCacheFromTerms(filter.getTerms());
|
||||
cache_store.cache[filter.getQueryString()] = postings_cache;
|
||||
}
|
||||
|
||||
return match(*postings_cache);
|
||||
}
|
||||
|
||||
String GinFilter::getName()
|
||||
{
|
||||
return FilterName;
|
||||
}
|
||||
|
||||
}
|
108
src/Interpreters/GinFilter.h
Normal file
108
src/Interpreters/GinFilter.h
Normal file
@ -0,0 +1,108 @@
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
#include <memory>
|
||||
#include <Storages/MergeTree/GinIndexStore.h>
|
||||
namespace DB
|
||||
{
|
||||
struct GinFilterParameters
|
||||
{
|
||||
explicit GinFilterParameters(size_t ngrams_, Float64 density_);
|
||||
|
||||
size_t ngrams;
|
||||
Float64 density;
|
||||
};
|
||||
|
||||
struct GinSegmentWithRowIDRange
|
||||
{
|
||||
/// Segment ID of the row ID range
|
||||
UInt32 segment_id;
|
||||
|
||||
/// First row ID in the range
|
||||
UInt32 range_start;
|
||||
|
||||
/// Last row ID in the range (inclusive)
|
||||
UInt32 range_end;
|
||||
};
|
||||
|
||||
/// GinFilter provides underlying functionalities for building inverted index and also
|
||||
/// it does filtering the unmatched rows according to its query string.
|
||||
/// It also builds and uses skipping index which stores (segmentID, RowIDStart, RowIDEnd) triples.
|
||||
class GinFilter
|
||||
{
|
||||
public:
|
||||
using GinSegmentWithRowIDRanges = std::vector<GinSegmentWithRowIDRange>;
|
||||
|
||||
explicit GinFilter(const GinFilterParameters& params_);
|
||||
|
||||
/// Add term(which length is 'len' and located at 'data') and its row ID to
|
||||
/// the postings list builder for building inverted index for the given store.
|
||||
void add(const char* data, size_t len, UInt32 rowID, GinIndexStorePtr& store, UInt64 limit) const;
|
||||
|
||||
/// Accumulate (segmentID, RowIDStart, RowIDEnd) for building skipping index
|
||||
void addRowRangeToGinFilter(UInt32 segmentID, UInt32 rowIDStart, UInt32 rowIDEnd);
|
||||
|
||||
/// Clear the content
|
||||
void clear();
|
||||
|
||||
/// Check if the filter(built from query string) contains any rows in given filter 'af' by using
|
||||
/// given postings list cache
|
||||
bool contains(const GinFilter & filter, PostingsCacheForStore &cache_store) const;
|
||||
|
||||
/// Const getter for the row ID ranges
|
||||
const GinSegmentWithRowIDRanges& getFilter() const { return rowid_ranges; }
|
||||
|
||||
/// Mutable getter for the row ID ranges
|
||||
GinSegmentWithRowIDRanges& getFilter() { return rowid_ranges; }
|
||||
|
||||
/// Set the query string of the filter
|
||||
void setQueryString(const char* data, size_t len)
|
||||
{
|
||||
query_string = String(data, len);
|
||||
}
|
||||
|
||||
/// Const getter of the query string
|
||||
const String &getQueryString() const { return query_string; }
|
||||
|
||||
/// Add term which are tokens generated from the query string
|
||||
void addTerm(const char* data, size_t len)
|
||||
{
|
||||
if (len > FST::MAX_TERM_LENGTH)
|
||||
return;
|
||||
terms.push_back(String(data, len));
|
||||
}
|
||||
|
||||
/// Const getter of terms(generated from the query string)
|
||||
const std::vector<String>& getTerms() const { return terms;}
|
||||
|
||||
/// Check if the given postings list cache has matched rows by using the filter
|
||||
bool match(const PostingsCache& postings_cache) const;
|
||||
|
||||
/// Get filter name ("inverted")
|
||||
static String getName();
|
||||
|
||||
/// Constant of filter name
|
||||
static constexpr auto FilterName = "inverted";
|
||||
private:
|
||||
/// Filter parameters
|
||||
const GinFilterParameters& params;
|
||||
|
||||
/// Query string of the filter
|
||||
String query_string;
|
||||
|
||||
/// Tokenized terms from query string
|
||||
std::vector<String> terms;
|
||||
|
||||
/// Row ID ranges which are (segmentID, RowIDStart, RowIDEnd)
|
||||
GinSegmentWithRowIDRanges rowid_ranges;
|
||||
|
||||
/// Helper method for checking if postings list cache is empty
|
||||
static bool hasEmptyPostingsList(const PostingsCache& postings_cache);
|
||||
|
||||
/// Helper method to check if the postings list cache has intersection with given row ID range
|
||||
static bool matchInRange(const PostingsCache& postings_cache, UInt32 segment_id, UInt32 range_start, UInt32 range_end);
|
||||
};
|
||||
|
||||
using GinFilterPtr = std::shared_ptr<GinFilter>;
|
||||
|
||||
}
|
@ -3,7 +3,7 @@
|
||||
#include <base/types.h>
|
||||
|
||||
#include <Interpreters/BloomFilter.h>
|
||||
|
||||
#include <Interpreters/GinFilter.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -37,6 +37,15 @@ struct ITokenExtractor
|
||||
|
||||
virtual void stringLikeToBloomFilter(const char * data, size_t length, BloomFilter & bloom_filter) const = 0;
|
||||
|
||||
virtual void stringToGinFilter(const char * data, size_t length, GinFilter & gin_filter) const = 0;
|
||||
|
||||
virtual void stringPaddedToGinFilter(const char * data, size_t length, GinFilter & gin_filter) const
|
||||
{
|
||||
return stringToGinFilter(data, length, gin_filter);
|
||||
}
|
||||
|
||||
virtual void stringLikeToGinFilter(const char * data, size_t length, GinFilter & gin_filter) const = 0;
|
||||
|
||||
};
|
||||
|
||||
using TokenExtractorPtr = const ITokenExtractor *;
|
||||
@ -71,6 +80,36 @@ class ITokenExtractorHelper : public ITokenExtractor
|
||||
while (cur < length && static_cast<const Derived *>(this)->nextInStringLike(data, length, &cur, token))
|
||||
bloom_filter.add(token.c_str(), token.size());
|
||||
}
|
||||
void stringToGinFilter(const char * data, size_t length, GinFilter & gin_filter) const override
|
||||
{
|
||||
gin_filter.setQueryString(data, length);
|
||||
size_t cur = 0;
|
||||
size_t token_start = 0;
|
||||
size_t token_len = 0;
|
||||
|
||||
while (cur < length && static_cast<const Derived *>(this)->nextInString(data, length, &cur, &token_start, &token_len))
|
||||
gin_filter.addTerm(data + token_start, token_len);
|
||||
}
|
||||
|
||||
void stringPaddedToGinFilter(const char * data, size_t length, GinFilter & gin_filter) const override
|
||||
{
|
||||
gin_filter.setQueryString(data, length);
|
||||
size_t cur = 0;
|
||||
size_t token_start = 0;
|
||||
size_t token_len = 0;
|
||||
|
||||
while (cur < length && static_cast<const Derived *>(this)->nextInStringPadded(data, length, &cur, &token_start, &token_len))
|
||||
gin_filter.addTerm(data + token_start, token_len);
|
||||
}
|
||||
|
||||
void stringLikeToGinFilter(const char * data, size_t length, GinFilter & gin_filter) const override
|
||||
{
|
||||
gin_filter.setQueryString(data, length);
|
||||
size_t cur = 0;
|
||||
String token;
|
||||
while (cur < length && static_cast<const Derived *>(this)->nextInStringLike(data, length, &cur, token))
|
||||
gin_filter.addTerm(token.c_str(), token.size());
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
@ -41,6 +41,7 @@
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Interpreters/InterpreterRenameQuery.h>
|
||||
#include <Interpreters/AddDefaultDatabaseVisitor.h>
|
||||
#include <Interpreters/GinFilter.h>
|
||||
|
||||
#include <Access/Common/AccessRightsElement.h>
|
||||
|
||||
@ -100,6 +101,7 @@ namespace ErrorCodes
|
||||
extern const int ENGINE_REQUIRED;
|
||||
extern const int UNKNOWN_STORAGE;
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int SUPPORT_IS_DISABLED;
|
||||
}
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
@ -677,12 +679,18 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
|
||||
if (create.columns_list->indices)
|
||||
for (const auto & index : create.columns_list->indices->children)
|
||||
{
|
||||
properties.indices.push_back(
|
||||
IndexDescription::getIndexFromAST(index->clone(), properties.columns, getContext()));
|
||||
if (properties.indices.back().type == "annoy" && !getContext()->getSettingsRef().allow_experimental_annoy_index)
|
||||
throw Exception("Annoy index is disabled. Turn on allow_experimental_annoy_index", ErrorCodes::INCORRECT_QUERY);
|
||||
}
|
||||
IndexDescription index_desc = IndexDescription::getIndexFromAST(index->clone(), properties.columns, getContext());
|
||||
if (index_desc.type == GinFilter::FilterName && getContext()->getSettingsRef().allow_experimental_inverted_index == false)
|
||||
{
|
||||
throw Exception(
|
||||
"Experimental Inverted Index feature is not enabled (the setting 'allow_experimental_inverted_index')",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED);
|
||||
}
|
||||
if (index_desc.type == "annoy" && !getContext()->getSettingsRef().allow_experimental_annoy_index)
|
||||
throw Exception("Annoy index is disabled. Turn on allow_experimental_annoy_index", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
properties.indices.push_back(index_desc);
|
||||
}
|
||||
if (create.columns_list->projections)
|
||||
for (const auto & projection_ast : create.columns_list->projections->children)
|
||||
{
|
||||
|
@ -1,41 +1,40 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/AsynchronousInsertLog.h>
|
||||
#include <Interpreters/AsynchronousMetricLog.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/CrashLog.h>
|
||||
#include <Interpreters/FilesystemCacheLog.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Interpreters/InterpreterRenameQuery.h>
|
||||
#include <Interpreters/MetricLog.h>
|
||||
#include <Interpreters/OpenTelemetrySpanLog.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/ProcessorsProfileLog.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Interpreters/QueryThreadLog.h>
|
||||
#include <Interpreters/QueryViewsLog.h>
|
||||
#include <Interpreters/SessionLog.h>
|
||||
#include <Interpreters/TextLog.h>
|
||||
#include <Interpreters/TraceLog.h>
|
||||
#include <Interpreters/ProcessorsProfileLog.h>
|
||||
#include <Interpreters/ZooKeeperLog.h>
|
||||
#include <Interpreters/TransactionsInfoLog.h>
|
||||
#include <Interpreters/FilesystemCacheLog.h>
|
||||
#include <Interpreters/AsynchronousInsertLog.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/InterpreterRenameQuery.h>
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Processors/Executors/PushingPipelineExecutor.h>
|
||||
#include <Interpreters/ZooKeeperLog.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/ASTRenameQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTRenameQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Processors/Executors/PushingPipelineExecutor.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/MergeTree/MergeTreeSettings.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/MemoryTrackerBlockerInThread.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Common/MemoryTrackerBlockerInThread.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/setThreadName.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -144,6 +143,10 @@ std::shared_ptr<TSystemLog> createSystemLog(
|
||||
throw Exception("If 'engine' is specified for system table, "
|
||||
"TTL parameters should be specified directly inside 'engine' and 'ttl' setting doesn't make sense",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
if (config.has(config_prefix + ".storage_policy"))
|
||||
throw Exception("If 'engine' is specified for system table, SETTINGS storage_policy = '...' "
|
||||
"should be specified directly inside 'engine' and 'storage_policy' setting doesn't make sense",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
engine = config.getString(config_prefix + ".engine");
|
||||
}
|
||||
else
|
||||
@ -158,6 +161,9 @@ std::shared_ptr<TSystemLog> createSystemLog(
|
||||
|
||||
engine += " ORDER BY ";
|
||||
engine += TSystemLog::getDefaultOrderBy();
|
||||
String storage_policy = config.getString(config_prefix + ".storage_policy", "");
|
||||
if (!storage_policy.empty())
|
||||
engine += " SETTINGS storage_policy = " + quoteString(storage_policy);
|
||||
}
|
||||
|
||||
/// Validate engine definition syntax to prevent some configuration errors.
|
||||
|
@ -53,10 +53,10 @@ BSONEachRowRowInputFormat::BSONEachRowRowInputFormat(
|
||||
ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_)
|
||||
: IRowInputFormat(header_, in_, std::move(params_))
|
||||
, format_settings(format_settings_)
|
||||
, name_map(header_.getNamesToIndexesMap())
|
||||
, prev_positions(header_.columns())
|
||||
, types(header_.getDataTypes())
|
||||
{
|
||||
name_map = getPort().getHeader().getNamesToIndexesMap();
|
||||
}
|
||||
|
||||
inline size_t BSONEachRowRowInputFormat::columnIndex(const StringRef & name, size_t key_index)
|
||||
|
@ -43,12 +43,13 @@ JSONEachRowRowInputFormat::JSONEachRowRowInputFormat(
|
||||
, prev_positions(header_.columns())
|
||||
, yield_strings(yield_strings_)
|
||||
{
|
||||
name_map = getPort().getHeader().getNamesToIndexesMap();
|
||||
const auto & header = getPort().getHeader();
|
||||
name_map = header.getNamesToIndexesMap();
|
||||
if (format_settings_.import_nested_json)
|
||||
{
|
||||
for (size_t i = 0; i != header_.columns(); ++i)
|
||||
for (size_t i = 0; i != header.columns(); ++i)
|
||||
{
|
||||
const StringRef column_name = header_.getByPosition(i).name;
|
||||
const StringRef column_name = header.getByPosition(i).name;
|
||||
const auto split = Nested::splitName(column_name.toView());
|
||||
if (!split.second.empty())
|
||||
{
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/RenameColumnVisitor.h>
|
||||
#include <Interpreters/GinFilter.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
@ -904,7 +905,26 @@ std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(Storage
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
bool AlterCommands::hasInvertedIndex(const StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
{
|
||||
for (const auto & index : metadata.secondary_indices)
|
||||
{
|
||||
IndexDescription index_desc;
|
||||
try
|
||||
{
|
||||
index_desc = IndexDescription::getIndexFromAST(index.definition_ast, metadata.columns, context);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
if (index.type == GinFilter::FilterName)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
void AlterCommands::apply(StorageInMemoryMetadata & metadata, ContextPtr context) const
|
||||
{
|
||||
if (!prepared)
|
||||
|
@ -210,6 +210,9 @@ public:
|
||||
/// empty. If some TTL changes happened than, depending on materialize_ttl
|
||||
/// additional mutation command (MATERIALIZE_TTL) will be returned.
|
||||
MutationCommands getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, ContextPtr context, bool with_alters=false) const;
|
||||
|
||||
/// Check if commands have any inverted index
|
||||
static bool hasInvertedIndex(const StorageInMemoryMetadata & metadata, ContextPtr context);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -81,6 +81,10 @@ bool traverseASTFilter(
|
||||
|
||||
set->checkColumnsNumber(1);
|
||||
const auto & set_column = *set->getSetElements()[0];
|
||||
|
||||
if (set_column.getDataType() != primary_key_type->getTypeId())
|
||||
return false;
|
||||
|
||||
for (size_t row = 0; row < set_column.size(); ++row)
|
||||
res->push_back(set_column[row]);
|
||||
return true;
|
||||
|
130
src/Storages/MergeTree/AsyncBlockIDsCache.cpp
Normal file
130
src/Storages/MergeTree/AsyncBlockIDsCache.cpp
Normal file
@ -0,0 +1,130 @@
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Storages/MergeTree/AsyncBlockIDsCache.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
|
||||
#include <unordered_set>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event AsyncInsertCacheHits;
|
||||
}
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric AsyncInsertCacheSize;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct AsyncBlockIDsCache::Cache : public std::unordered_set<String>
|
||||
{
|
||||
CurrentMetrics::Increment cache_size_increment;
|
||||
explicit Cache(std::unordered_set<String> && set_)
|
||||
: std::unordered_set<String>(std::move(set_))
|
||||
, cache_size_increment(CurrentMetrics::AsyncInsertCacheSize, size())
|
||||
{}
|
||||
};
|
||||
|
||||
std::vector<String> AsyncBlockIDsCache::getChildren()
|
||||
{
|
||||
auto zookeeper = storage.getZooKeeper();
|
||||
|
||||
auto watch_callback = [&](const Coordination::WatchResponse &)
|
||||
{
|
||||
auto now = std::chrono::steady_clock::now();
|
||||
auto last_time = last_updatetime.load();
|
||||
if (now - last_time < update_min_interval)
|
||||
{
|
||||
std::chrono::milliseconds sleep_time = std::chrono::duration_cast<std::chrono::milliseconds>(update_min_interval - (now - last_time));
|
||||
task->scheduleAfter(sleep_time.count());
|
||||
}
|
||||
else
|
||||
task->schedule();
|
||||
};
|
||||
std::vector<String> children;
|
||||
Coordination::Stat stat;
|
||||
zookeeper->tryGetChildrenWatch(path, children, &stat, watch_callback);
|
||||
return children;
|
||||
}
|
||||
|
||||
void AsyncBlockIDsCache::update()
|
||||
try
|
||||
{
|
||||
std::vector<String> paths = getChildren();
|
||||
std::unordered_set<String> set;
|
||||
for (String & p : paths)
|
||||
{
|
||||
set.insert(std::move(p));
|
||||
}
|
||||
{
|
||||
std::lock_guard lock(mu);
|
||||
cache_ptr = std::make_shared<Cache>(std::move(set));
|
||||
++version;
|
||||
}
|
||||
cv.notify_all();
|
||||
last_updatetime = std::chrono::steady_clock::now();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_INFO(log, "Updating async block ids cache failed. Reason: {}", getCurrentExceptionMessage(false));
|
||||
task->scheduleAfter(update_min_interval.count());
|
||||
}
|
||||
|
||||
AsyncBlockIDsCache::AsyncBlockIDsCache(StorageReplicatedMergeTree & storage_)
|
||||
: storage(storage_),
|
||||
update_min_interval(storage.getSettings()->async_block_ids_cache_min_update_interval_ms),
|
||||
path(storage.zookeeper_path + "/async_blocks"),
|
||||
log_name(storage.getStorageID().getFullTableName() + " (AsyncBlockIDsCache)"),
|
||||
log(&Poco::Logger::get(log_name))
|
||||
{
|
||||
task = storage.getContext()->getSchedulePool().createTask(log_name, [this]{ update(); });
|
||||
}
|
||||
|
||||
void AsyncBlockIDsCache::start()
|
||||
{
|
||||
if (storage.getSettings()->use_async_block_ids_cache)
|
||||
task->activateAndSchedule();
|
||||
}
|
||||
|
||||
/// Caller will keep the version of last call. When the caller calls again, it will wait util gets a newer version.
|
||||
Strings AsyncBlockIDsCache::detectConflicts(const Strings & paths, UInt64 & last_version)
|
||||
{
|
||||
if (!storage.getSettings()->use_async_block_ids_cache)
|
||||
return {};
|
||||
|
||||
std::unique_lock lk(mu);
|
||||
/// For first time access of this cache, the `last_version` is zero, so it will not block here.
|
||||
/// For retrying request, We compare the request version and cache version, because zk only returns
|
||||
/// incomplete information of duplication, we need to update the cache to find out more duplication.
|
||||
/// The timeout here is to prevent deadlock, just in case.
|
||||
cv.wait_for(lk, update_min_interval * 2, [&]{return version != last_version;});
|
||||
|
||||
if (version == last_version)
|
||||
LOG_INFO(log, "Read cache with a old version {}", last_version);
|
||||
|
||||
CachePtr cur_cache;
|
||||
cur_cache = cache_ptr;
|
||||
last_version = version;
|
||||
|
||||
lk.unlock();
|
||||
|
||||
if (cur_cache == nullptr)
|
||||
return {};
|
||||
|
||||
Strings conflicts;
|
||||
for (const String & p : paths)
|
||||
{
|
||||
if (cur_cache->contains(p))
|
||||
{
|
||||
conflicts.push_back(p);
|
||||
}
|
||||
}
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::AsyncInsertCacheHits, !conflicts.empty());
|
||||
|
||||
return conflicts;
|
||||
}
|
||||
|
||||
}
|
54
src/Storages/MergeTree/AsyncBlockIDsCache.h
Normal file
54
src/Storages/MergeTree/AsyncBlockIDsCache.h
Normal file
@ -0,0 +1,54 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
|
||||
#include <chrono>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class StorageReplicatedMergeTree;
|
||||
|
||||
class AsyncBlockIDsCache
|
||||
{
|
||||
struct Cache;
|
||||
using CachePtr = std::shared_ptr<Cache>;
|
||||
|
||||
std::vector<String> getChildren();
|
||||
|
||||
void update();
|
||||
|
||||
public:
|
||||
explicit AsyncBlockIDsCache(StorageReplicatedMergeTree & storage_);
|
||||
|
||||
void start();
|
||||
|
||||
void stop() { task->deactivate(); }
|
||||
|
||||
Strings detectConflicts(const Strings & paths, UInt64 & last_version);
|
||||
|
||||
private:
|
||||
|
||||
StorageReplicatedMergeTree & storage;
|
||||
|
||||
std::atomic<std::chrono::steady_clock::time_point> last_updatetime;
|
||||
const std::chrono::milliseconds update_min_interval;
|
||||
|
||||
std::mutex mu;
|
||||
CachePtr cache_ptr;
|
||||
std::condition_variable cv;
|
||||
UInt64 version = 0;
|
||||
|
||||
const String path;
|
||||
|
||||
BackgroundSchedulePool::TaskHolder task;
|
||||
|
||||
const String log_name;
|
||||
Poco::Logger * log;
|
||||
};
|
||||
|
||||
using AsyncBlockIDsCachePtr = std::shared_ptr<AsyncBlockIDsCache>;
|
||||
|
||||
}
|
@ -741,6 +741,17 @@ std::unique_ptr<WriteBufferFromFileBase> DataPartStorageOnDisk::writeFile(
|
||||
return volume->getDisk()->writeFile(fs::path(root_path) / part_dir / name, buf_size, WriteMode::Rewrite, settings);
|
||||
}
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DataPartStorageOnDisk::writeFile(
|
||||
const String & name,
|
||||
size_t buf_size,
|
||||
WriteMode mode,
|
||||
const WriteSettings & settings)
|
||||
{
|
||||
if (transaction)
|
||||
return transaction->writeFile(fs::path(root_path) / part_dir / name, buf_size, mode, settings, /* autocommit = */ false);
|
||||
return volume->getDisk()->writeFile(fs::path(root_path) / part_dir / name, buf_size, mode, settings);
|
||||
}
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DataPartStorageOnDisk::writeTransactionFile(WriteMode mode) const
|
||||
{
|
||||
return volume->getDisk()->writeFile(fs::path(root_path) / part_dir / "txn_version.txt", 256, mode);
|
||||
|
@ -112,6 +112,11 @@ public:
|
||||
void createFile(const String & name) override;
|
||||
void moveFile(const String & from_name, const String & to_name) override;
|
||||
void replaceFile(const String & from_name, const String & to_name) override;
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeFile(
|
||||
const String & name,
|
||||
size_t buf_size,
|
||||
DB::WriteMode mode,
|
||||
const WriteSettings & settings) override;
|
||||
|
||||
void removeFile(const String & name) override;
|
||||
void removeFileIfExists(const String & name) override;
|
||||
|
487
src/Storages/MergeTree/GinIndexStore.cpp
Normal file
487
src/Storages/MergeTree/GinIndexStore.cpp
Normal file
@ -0,0 +1,487 @@
|
||||
#include <Storages/MergeTree/GinIndexStore.h>
|
||||
#include <vector>
|
||||
#include <unordered_map>
|
||||
#include <iostream>
|
||||
#include <numeric>
|
||||
#include <algorithm>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromVector.h>
|
||||
#include <Common/FST.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using TokenPostingsBuilderPair = std::pair<std::string_view, GinIndexStore::GinIndexPostingsBuilderPtr>;
|
||||
using TokenPostingsBuilderPairs = std::vector<TokenPostingsBuilderPair>;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int UNKNOWN_FORMAT_VERSION;
|
||||
};
|
||||
|
||||
GinIndexStore::GinIndexStore(const String & name_, DataPartStoragePtr storage_)
|
||||
: name(name_)
|
||||
, storage(storage_)
|
||||
{
|
||||
}
|
||||
GinIndexStore::GinIndexStore(const String& name_, DataPartStoragePtr storage_, MutableDataPartStoragePtr data_part_storage_builder_, UInt64 max_digestion_size_)
|
||||
: name(name_)
|
||||
, storage(storage_)
|
||||
, data_part_storage_builder(data_part_storage_builder_)
|
||||
, max_digestion_size(max_digestion_size_)
|
||||
{
|
||||
}
|
||||
|
||||
GinIndexPostingsBuilder::GinIndexPostingsBuilder(UInt64 limit) : rowid_lst{}, size_limit(limit)
|
||||
{}
|
||||
|
||||
bool GinIndexPostingsBuilder::contains(UInt32 row_id) const
|
||||
{
|
||||
if (useRoaring())
|
||||
return rowid_bitmap.contains(row_id);
|
||||
|
||||
const auto * const it = std::find(rowid_lst.begin(), rowid_lst.begin()+rowid_lst_length, row_id);
|
||||
return it != rowid_lst.begin() + rowid_lst_length;
|
||||
}
|
||||
|
||||
void GinIndexPostingsBuilder::add(UInt32 row_id)
|
||||
{
|
||||
if (containsAllRows())
|
||||
{
|
||||
return;
|
||||
}
|
||||
if (useRoaring())
|
||||
{
|
||||
if (rowid_bitmap.cardinality() == size_limit)
|
||||
{
|
||||
//reset the postings list with MATCH ALWAYS;
|
||||
rowid_lst_length = 1; //makes sure useRoaring() returns false;
|
||||
rowid_lst[0] = UINT32_MAX; //set CONTAINS ALL flag;
|
||||
}
|
||||
else
|
||||
{
|
||||
rowid_bitmap.add(row_id);
|
||||
}
|
||||
return;
|
||||
}
|
||||
assert(rowid_lst_length < MIN_SIZE_FOR_ROARING_ENCODING);
|
||||
rowid_lst[rowid_lst_length] = row_id;
|
||||
rowid_lst_length++;
|
||||
|
||||
if (rowid_lst_length == MIN_SIZE_FOR_ROARING_ENCODING)
|
||||
{
|
||||
for (size_t i = 0; i < rowid_lst_length; i++)
|
||||
rowid_bitmap.add(rowid_lst[i]);
|
||||
|
||||
rowid_lst_length = UsesBitMap;
|
||||
}
|
||||
}
|
||||
|
||||
bool GinIndexPostingsBuilder::useRoaring() const
|
||||
{
|
||||
return rowid_lst_length == UsesBitMap;
|
||||
}
|
||||
|
||||
bool GinIndexPostingsBuilder::containsAllRows() const
|
||||
{
|
||||
return rowid_lst[0] == UINT32_MAX;
|
||||
}
|
||||
|
||||
UInt64 GinIndexPostingsBuilder::serialize(WriteBuffer &buffer) const
|
||||
{
|
||||
UInt64 written_bytes = 0;
|
||||
buffer.write(rowid_lst_length);
|
||||
written_bytes += 1;
|
||||
|
||||
if (!useRoaring())
|
||||
{
|
||||
for (size_t i = 0; i < rowid_lst_length; ++i)
|
||||
{
|
||||
writeVarUInt(rowid_lst[i], buffer);
|
||||
written_bytes += getLengthOfVarUInt(rowid_lst[i]);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
auto size = rowid_bitmap.getSizeInBytes();
|
||||
|
||||
writeVarUInt(size, buffer);
|
||||
written_bytes += getLengthOfVarUInt(size);
|
||||
|
||||
auto buf = std::make_unique<char[]>(size);
|
||||
rowid_bitmap.write(buf.get());
|
||||
buffer.write(buf.get(), size);
|
||||
written_bytes += size;
|
||||
}
|
||||
return written_bytes;
|
||||
}
|
||||
|
||||
GinIndexPostingsListPtr GinIndexPostingsBuilder::deserialize(ReadBuffer &buffer)
|
||||
{
|
||||
UInt8 postings_list_size = 0;
|
||||
buffer.readStrict(reinterpret_cast<char&>(postings_list_size));
|
||||
|
||||
if (postings_list_size != UsesBitMap)
|
||||
{
|
||||
assert(postings_list_size < MIN_SIZE_FOR_ROARING_ENCODING);
|
||||
GinIndexPostingsListPtr postings_list = std::make_shared<GinIndexPostingsList>();
|
||||
UInt32 row_ids[MIN_SIZE_FOR_ROARING_ENCODING];
|
||||
|
||||
for (auto i = 0; i < postings_list_size; ++i)
|
||||
{
|
||||
readVarUInt(row_ids[i], buffer);
|
||||
}
|
||||
postings_list->addMany(postings_list_size, row_ids);
|
||||
return postings_list;
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t size{0};
|
||||
readVarUInt(size, buffer);
|
||||
auto buf = std::make_unique<char[]>(size);
|
||||
buffer.readStrict(reinterpret_cast<char*>(buf.get()), size);
|
||||
|
||||
GinIndexPostingsListPtr postings_list = std::make_shared<GinIndexPostingsList>(GinIndexPostingsList::read(buf.get()));
|
||||
|
||||
return postings_list;
|
||||
}
|
||||
}
|
||||
|
||||
bool GinIndexStore::exists() const
|
||||
{
|
||||
String id_file_name = getName() + GIN_SEGMENT_ID_FILE_TYPE;
|
||||
return storage->exists(id_file_name);
|
||||
}
|
||||
|
||||
UInt32 GinIndexStore::getNextSegmentIDRange(const String& file_name, size_t n)
|
||||
{
|
||||
std::lock_guard guard(gin_index_store_mutex);
|
||||
|
||||
/// When the method is called for the first time, the file doesn't exist yet, need to create it
|
||||
/// and write segment ID 1.
|
||||
if (!storage->exists(file_name))
|
||||
{
|
||||
/// Create file and write initial segment id = 1
|
||||
std::unique_ptr<DB::WriteBufferFromFileBase> ostr = this->data_part_storage_builder->writeFile(file_name, DBMS_DEFAULT_BUFFER_SIZE, {});
|
||||
|
||||
/// Write version
|
||||
writeChar(static_cast<char>(CURRENT_GIN_FILE_FORMAT_VERSION), *ostr);
|
||||
|
||||
writeVarUInt(1, *ostr);
|
||||
ostr->sync();
|
||||
}
|
||||
|
||||
/// read id in file
|
||||
UInt32 result = 0;
|
||||
{
|
||||
std::unique_ptr<DB::ReadBufferFromFileBase> istr = this->storage->readFile(file_name, {}, std::nullopt, std::nullopt);
|
||||
|
||||
/// Skip version
|
||||
istr->seek(1, SEEK_SET);
|
||||
|
||||
readVarUInt(result, *istr);
|
||||
}
|
||||
//save result+n
|
||||
{
|
||||
std::unique_ptr<DB::WriteBufferFromFileBase> ostr = this->data_part_storage_builder->writeFile(file_name, DBMS_DEFAULT_BUFFER_SIZE, {});
|
||||
|
||||
/// Write version
|
||||
writeChar(static_cast<char>(CURRENT_GIN_FILE_FORMAT_VERSION), *ostr);
|
||||
|
||||
writeVarUInt(result + n, *ostr);
|
||||
ostr->sync();
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
UInt32 GinIndexStore::getNextRowIDRange(size_t numIDs)
|
||||
{
|
||||
UInt32 result =current_segment.next_row_id;
|
||||
current_segment.next_row_id += numIDs;
|
||||
return result;
|
||||
}
|
||||
|
||||
UInt32 GinIndexStore::getNextSegmentID()
|
||||
{
|
||||
String sid_file_name = getName() + GIN_SEGMENT_ID_FILE_TYPE;
|
||||
return getNextSegmentIDRange(sid_file_name, 1);
|
||||
}
|
||||
|
||||
UInt32 GinIndexStore::getNumOfSegments()
|
||||
{
|
||||
if (cached_segment_num)
|
||||
return cached_segment_num;
|
||||
|
||||
String sid_file_name = getName() + GIN_SEGMENT_ID_FILE_TYPE;
|
||||
if (!storage->exists(sid_file_name))
|
||||
return 0;
|
||||
|
||||
UInt32 result = 0;
|
||||
{
|
||||
std::unique_ptr<DB::ReadBufferFromFileBase> istr = this->storage->readFile(sid_file_name, {}, std::nullopt, std::nullopt);
|
||||
|
||||
uint8_t version = 0;
|
||||
readBinary(version, *istr);
|
||||
|
||||
if (version > CURRENT_GIN_FILE_FORMAT_VERSION)
|
||||
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported inverted index version {}", version);
|
||||
|
||||
readVarUInt(result, *istr);
|
||||
}
|
||||
|
||||
cached_segment_num = result - 1;
|
||||
return cached_segment_num;
|
||||
}
|
||||
|
||||
bool GinIndexStore::needToWrite() const
|
||||
{
|
||||
assert(max_digestion_size > 0);
|
||||
return current_size > max_digestion_size;
|
||||
}
|
||||
|
||||
void GinIndexStore::finalize()
|
||||
{
|
||||
if (!current_postings.empty())
|
||||
{
|
||||
writeSegment();
|
||||
}
|
||||
}
|
||||
|
||||
void GinIndexStore::initFileStreams()
|
||||
{
|
||||
String segment_file_name = getName() + GIN_SEGMENT_FILE_TYPE;
|
||||
String term_dict_file_name = getName() + GIN_DICTIONARY_FILE_TYPE;
|
||||
String postings_file_name = getName() + GIN_POSTINGS_FILE_TYPE;
|
||||
|
||||
segment_file_stream = data_part_storage_builder->writeFile(segment_file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {});
|
||||
term_dict_file_stream = data_part_storage_builder->writeFile(term_dict_file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {});
|
||||
postings_file_stream = data_part_storage_builder->writeFile(postings_file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {});
|
||||
}
|
||||
|
||||
void GinIndexStore::writeSegment()
|
||||
{
|
||||
if (segment_file_stream == nullptr)
|
||||
{
|
||||
initFileStreams();
|
||||
}
|
||||
|
||||
/// Write segment
|
||||
segment_file_stream->write(reinterpret_cast<char*>(¤t_segment), sizeof(GinIndexSegment));
|
||||
TokenPostingsBuilderPairs token_postings_list_pairs;
|
||||
token_postings_list_pairs.reserve(current_postings.size());
|
||||
|
||||
for (const auto& [token, postings_list] : current_postings)
|
||||
{
|
||||
token_postings_list_pairs.push_back({token, postings_list});
|
||||
}
|
||||
|
||||
/// Sort token-postings list pairs since all tokens have to be added in FST in sorted order
|
||||
std::sort(token_postings_list_pairs.begin(), token_postings_list_pairs.end(),
|
||||
[](const TokenPostingsBuilderPair& a, const TokenPostingsBuilderPair & b)
|
||||
{
|
||||
return a.first < b.first;
|
||||
});
|
||||
|
||||
///write postings
|
||||
std::vector<UInt64> posting_list_byte_sizes(current_postings.size(), 0);
|
||||
|
||||
for (size_t current_index = 0; const auto& [token, postings_list] : token_postings_list_pairs)
|
||||
{
|
||||
auto posting_list_byte_size = postings_list->serialize(*postings_file_stream);
|
||||
|
||||
posting_list_byte_sizes[current_index] = posting_list_byte_size;
|
||||
current_index++;
|
||||
current_segment.postings_start_offset += posting_list_byte_size;
|
||||
}
|
||||
///write item dictionary
|
||||
std::vector<UInt8> buffer;
|
||||
WriteBufferFromVector<std::vector<UInt8>> write_buf(buffer);
|
||||
FST::FSTBuilder builder(write_buf);
|
||||
|
||||
UInt64 offset = 0;
|
||||
for (size_t current_index = 0; const auto& [token, postings_list] : token_postings_list_pairs)
|
||||
{
|
||||
String str_token{token};
|
||||
builder.add(str_token, offset);
|
||||
offset += posting_list_byte_sizes[current_index];
|
||||
current_index++;
|
||||
}
|
||||
|
||||
builder.build();
|
||||
write_buf.finalize();
|
||||
|
||||
/// Write FST size
|
||||
writeVarUInt(buffer.size(), *term_dict_file_stream);
|
||||
current_segment.term_dict_start_offset += getLengthOfVarUInt(buffer.size());
|
||||
|
||||
/// Write FST content
|
||||
term_dict_file_stream->write(reinterpret_cast<char*>(buffer.data()), buffer.size());
|
||||
current_segment.term_dict_start_offset += buffer.size();
|
||||
|
||||
current_size = 0;
|
||||
current_postings.clear();
|
||||
current_segment.segment_id = getNextSegmentID();
|
||||
|
||||
segment_file_stream->sync();
|
||||
term_dict_file_stream->sync();
|
||||
postings_file_stream->sync();
|
||||
}
|
||||
|
||||
GinIndexStoreDeserializer::GinIndexStoreDeserializer(const GinIndexStorePtr & store_)
|
||||
: store(store_)
|
||||
{
|
||||
initFileStreams();
|
||||
}
|
||||
|
||||
void GinIndexStoreDeserializer::initFileStreams()
|
||||
{
|
||||
String segment_file_name = store->getName() + GinIndexStore::GIN_SEGMENT_FILE_TYPE;
|
||||
String term_dict_file_name = store->getName() + GinIndexStore::GIN_DICTIONARY_FILE_TYPE;
|
||||
String postings_file_name = store->getName() + GinIndexStore::GIN_POSTINGS_FILE_TYPE;
|
||||
|
||||
segment_file_stream = store->storage->readFile(segment_file_name, {}, std::nullopt, std::nullopt);
|
||||
term_dict_file_stream = store->storage->readFile(term_dict_file_name, {}, std::nullopt, std::nullopt);
|
||||
postings_file_stream = store->storage->readFile(postings_file_name, {}, std::nullopt, std::nullopt);
|
||||
}
|
||||
void GinIndexStoreDeserializer::readSegments()
|
||||
{
|
||||
auto num_segments = store->getNumOfSegments();
|
||||
if (num_segments == 0)
|
||||
return;
|
||||
|
||||
GinIndexSegments segments (num_segments);
|
||||
|
||||
assert(segment_file_stream != nullptr);
|
||||
|
||||
segment_file_stream->readStrict(reinterpret_cast<char*>(segments.data()), num_segments * sizeof(GinIndexSegment));
|
||||
for (size_t i = 0; i < num_segments; ++i)
|
||||
{
|
||||
auto seg_id = segments[i].segment_id;
|
||||
auto term_dict = std::make_shared<SegmentTermDictionary>();
|
||||
term_dict->postings_start_offset = segments[i].postings_start_offset;
|
||||
term_dict->term_dict_start_offset = segments[i].term_dict_start_offset;
|
||||
store->term_dicts[seg_id] = term_dict;
|
||||
}
|
||||
}
|
||||
|
||||
void GinIndexStoreDeserializer::readSegmentTermDictionaries()
|
||||
{
|
||||
for (UInt32 seg_index = 0; seg_index < store->getNumOfSegments(); ++seg_index)
|
||||
{
|
||||
readSegmentTermDictionary(seg_index);
|
||||
}
|
||||
}
|
||||
|
||||
void GinIndexStoreDeserializer::readSegmentTermDictionary(UInt32 segment_id)
|
||||
{
|
||||
/// Check validity of segment_id
|
||||
auto it = store->term_dicts.find(segment_id);
|
||||
if (it == store->term_dicts.end())
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid segment id {}", segment_id);
|
||||
}
|
||||
|
||||
assert(term_dict_file_stream != nullptr);
|
||||
|
||||
/// Set file pointer of term dictionary file
|
||||
term_dict_file_stream->seek(it->second->term_dict_start_offset, SEEK_SET);
|
||||
|
||||
it->second->offsets.getData().clear();
|
||||
/// Read FST size
|
||||
size_t fst_size{0};
|
||||
readVarUInt(fst_size, *term_dict_file_stream);
|
||||
|
||||
/// Read FST content
|
||||
it->second->offsets.getData().resize(fst_size);
|
||||
term_dict_file_stream->readStrict(reinterpret_cast<char*>(it->second->offsets.getData().data()), fst_size);
|
||||
}
|
||||
|
||||
SegmentedPostingsListContainer GinIndexStoreDeserializer::readSegmentedPostingsLists(const String& term)
|
||||
{
|
||||
assert(postings_file_stream != nullptr);
|
||||
|
||||
SegmentedPostingsListContainer container;
|
||||
for (auto const& seg_term_dict : store->term_dicts)
|
||||
{
|
||||
auto segment_id = seg_term_dict.first;
|
||||
|
||||
auto [offset, found] = seg_term_dict.second->offsets.getOutput(term);
|
||||
if (!found)
|
||||
continue;
|
||||
|
||||
// Set postings file pointer for reading postings list
|
||||
postings_file_stream->seek(seg_term_dict.second->postings_start_offset + offset, SEEK_SET);
|
||||
|
||||
// Read posting list
|
||||
auto postings_list = GinIndexPostingsBuilder::deserialize(*postings_file_stream);
|
||||
container[segment_id] = postings_list;
|
||||
}
|
||||
return container;
|
||||
}
|
||||
|
||||
PostingsCachePtr GinIndexStoreDeserializer::createPostingsCacheFromTerms(const std::vector<String>& terms)
|
||||
{
|
||||
auto postings_cache = std::make_shared<PostingsCache>();
|
||||
for (const auto& term : terms)
|
||||
{
|
||||
// Make sure don't read for duplicated terms
|
||||
if (postings_cache->find(term) != postings_cache->end())
|
||||
continue;
|
||||
|
||||
auto container = readSegmentedPostingsLists(term);
|
||||
(*postings_cache)[term] = container;
|
||||
}
|
||||
return postings_cache;
|
||||
}
|
||||
|
||||
GinIndexStoreFactory& GinIndexStoreFactory::instance()
|
||||
{
|
||||
static GinIndexStoreFactory instance;
|
||||
return instance;
|
||||
}
|
||||
|
||||
GinIndexStorePtr GinIndexStoreFactory::get(const String& name, DataPartStoragePtr storage)
|
||||
{
|
||||
const String& part_path = storage->getRelativePath();
|
||||
String key = name + ":" + part_path;
|
||||
|
||||
std::lock_guard lock(stores_mutex);
|
||||
GinIndexStores::const_iterator it = stores.find(key);
|
||||
|
||||
if (it == stores.end())
|
||||
{
|
||||
GinIndexStorePtr store = std::make_shared<GinIndexStore>(name, storage);
|
||||
if (!store->exists())
|
||||
return nullptr;
|
||||
|
||||
GinIndexStoreDeserializer deserializer(store);
|
||||
deserializer.readSegments();
|
||||
deserializer.readSegmentTermDictionaries();
|
||||
|
||||
stores[key] = store;
|
||||
|
||||
return store;
|
||||
}
|
||||
return it->second;
|
||||
}
|
||||
|
||||
void GinIndexStoreFactory::remove(const String& part_path)
|
||||
{
|
||||
std::lock_guard lock(stores_mutex);
|
||||
for (auto it = stores.begin(); it != stores.end();)
|
||||
{
|
||||
if (it->first.find(part_path) != String::npos)
|
||||
it = stores.erase(it);
|
||||
else
|
||||
++it;
|
||||
}
|
||||
}
|
||||
}
|
304
src/Storages/MergeTree/GinIndexStore.h
Normal file
304
src/Storages/MergeTree/GinIndexStore.h
Normal file
@ -0,0 +1,304 @@
|
||||
#pragma once
|
||||
|
||||
#include <array>
|
||||
#include <vector>
|
||||
#include <unordered_map>
|
||||
#include <mutex>
|
||||
#include <Core/Block.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <IO/ReadBufferFromFileBase.h>
|
||||
#include <IO/WriteBufferFromFileBase.h>
|
||||
#include <roaring.hh>
|
||||
#include <Common/FST.h>
|
||||
#include <Storages/MergeTree/IDataPartStorage.h>
|
||||
|
||||
/// GinIndexStore manages the inverted index for a data part, and it is made up of one or more immutable
|
||||
/// index segments.
|
||||
///
|
||||
/// There are 4 types of index files in a store:
|
||||
/// 1. Segment ID file(.gin_sid): it contains one byte for version followed by the next available segment ID.
|
||||
/// 2. Segment Metadata file(.gin_seg): it contains index segment metadata.
|
||||
/// - Its file format is an array of GinIndexSegment as defined in this file.
|
||||
/// - postings_start_offset points to the file(.gin_post) starting position for the segment's postings list.
|
||||
/// - term_dict_start_offset points to the file(.gin_dict) starting position for the segment's term dictionaries.
|
||||
/// 3. Term Dictionary file(.gin_dict): it contains term dictionaries.
|
||||
/// - It contains an array of (FST_size, FST_blob) which has size and actual data of FST.
|
||||
/// 4. Postings Lists(.gin_post): it contains postings lists data.
|
||||
/// - It contains an array of serialized postings lists.
|
||||
///
|
||||
/// During the searching in the segment, the segment's meta data can be found in .gin_seg file. From the meta data,
|
||||
/// the starting position of its term dictionary is used to locate its FST. Then FST is read into memory.
|
||||
/// By using the term and FST, the offset("output" in FST) of the postings list for the term
|
||||
/// in FST is found. The offset plus the postings_start_offset is the file location in .gin_post file
|
||||
/// for its postings list.
|
||||
|
||||
namespace DB
|
||||
{
|
||||
enum : uint8_t
|
||||
{
|
||||
GIN_VERSION_0 = 0,
|
||||
GIN_VERSION_1 = 1, /// Initial version
|
||||
};
|
||||
|
||||
static constexpr auto CURRENT_GIN_FILE_FORMAT_VERSION = GIN_VERSION_1;
|
||||
|
||||
/// GinIndexPostingsList which uses 32-bit Roaring
|
||||
using GinIndexPostingsList = roaring::Roaring;
|
||||
|
||||
using GinIndexPostingsListPtr = std::shared_ptr<GinIndexPostingsList>;
|
||||
|
||||
/// Gin Index Postings List Builder.
|
||||
class GinIndexPostingsBuilder
|
||||
{
|
||||
public:
|
||||
constexpr static int MIN_SIZE_FOR_ROARING_ENCODING = 16;
|
||||
|
||||
GinIndexPostingsBuilder(UInt64 limit);
|
||||
|
||||
/// Check whether a row_id is already added
|
||||
bool contains(UInt32 row_id) const;
|
||||
|
||||
/// Add a row_id into the builder
|
||||
void add(UInt32 row_id);
|
||||
|
||||
/// Check whether the builder is using roaring bitmap
|
||||
bool useRoaring() const;
|
||||
|
||||
/// Check whether the postings list has been flagged to contain all row ids
|
||||
bool containsAllRows() const;
|
||||
|
||||
/// Serialize the content of builder to given WriteBuffer, returns the bytes of serialized data
|
||||
UInt64 serialize(WriteBuffer &buffer) const;
|
||||
|
||||
/// Deserialize the postings list data from given ReadBuffer, return a pointer to the GinIndexPostingsList created by deserialization
|
||||
static GinIndexPostingsListPtr deserialize(ReadBuffer &buffer);
|
||||
private:
|
||||
/// When the list length is no greater than MIN_SIZE_FOR_ROARING_ENCODING, array 'rowid_lst' is used
|
||||
std::array<UInt32, MIN_SIZE_FOR_ROARING_ENCODING> rowid_lst;
|
||||
|
||||
/// When the list length is greater than MIN_SIZE_FOR_ROARING_ENCODING, Roaring bitmap 'rowid_bitmap' is used
|
||||
roaring::Roaring rowid_bitmap;
|
||||
|
||||
/// rowid_lst_length stores the number of row IDs in 'rowid_lst' array, can also be a flag(0xFF) indicating that roaring bitmap is used
|
||||
UInt8 rowid_lst_length{0};
|
||||
|
||||
static constexpr UInt8 UsesBitMap = 0xFF;
|
||||
/// Clear the postings list and reset it with MATCHALL flags when the size of the postings list is beyond the limit
|
||||
UInt64 size_limit;
|
||||
};
|
||||
|
||||
/// Container for postings lists for each segment
|
||||
using SegmentedPostingsListContainer = std::unordered_map<UInt32, GinIndexPostingsListPtr>;
|
||||
|
||||
/// Postings lists and terms built from query string
|
||||
using PostingsCache = std::unordered_map<std::string, SegmentedPostingsListContainer>;
|
||||
using PostingsCachePtr = std::shared_ptr<PostingsCache>;
|
||||
|
||||
/// Gin Index Segment information, which contains:
|
||||
struct GinIndexSegment
|
||||
{
|
||||
/// Segment ID retrieved from next available ID from file .gin_sid
|
||||
UInt32 segment_id = 0;
|
||||
|
||||
/// Next row ID for this segment
|
||||
UInt32 next_row_id = 1;
|
||||
|
||||
/// .gin_post file offset of this segment's postings lists
|
||||
UInt64 postings_start_offset = 0;
|
||||
|
||||
/// .term_dict file offset of this segment's term dictionaries
|
||||
UInt64 term_dict_start_offset = 0;
|
||||
};
|
||||
|
||||
using GinIndexSegments = std::vector<GinIndexSegment>;
|
||||
|
||||
struct SegmentTermDictionary
|
||||
{
|
||||
/// .gin_post file offset of this segment's postings lists
|
||||
UInt64 postings_start_offset;
|
||||
|
||||
/// .gin_dict file offset of this segment's term dictionaries
|
||||
UInt64 term_dict_start_offset;
|
||||
|
||||
/// Finite State Transducer, which can be viewed as a map of <term, offset>, where offset is the
|
||||
/// offset to the term's posting list in postings list file
|
||||
FST::FiniteStateTransducer offsets;
|
||||
};
|
||||
|
||||
using SegmentTermDictionaryPtr = std::shared_ptr<SegmentTermDictionary>;
|
||||
|
||||
/// Term dictionaries indexed by segment ID
|
||||
using SegmentTermDictionaries = std::unordered_map<UInt32, SegmentTermDictionaryPtr>;
|
||||
|
||||
/// Gin Index Store which has Gin Index meta data for the corresponding Data Part
|
||||
class GinIndexStore
|
||||
{
|
||||
public:
|
||||
using GinIndexPostingsBuilderPtr = std::shared_ptr<GinIndexPostingsBuilder>;
|
||||
/// Container for all term's Gin Index Postings List Builder
|
||||
using GinIndexPostingsBuilderContainer = std::unordered_map<std::string, GinIndexPostingsBuilderPtr>;
|
||||
|
||||
explicit GinIndexStore(const String & name_, DataPartStoragePtr storage_);
|
||||
|
||||
GinIndexStore(const String& name_, DataPartStoragePtr storage_, MutableDataPartStoragePtr data_part_storage_builder_, UInt64 max_digestion_size_);
|
||||
|
||||
/// Check existence by checking the existence of file .gin_sid
|
||||
bool exists() const;
|
||||
|
||||
/// Get a range of next 'numIDs' available row IDs
|
||||
UInt32 getNextRowIDRange(size_t numIDs);
|
||||
|
||||
/// Get next available segment ID by updating file .gin_sid
|
||||
UInt32 getNextSegmentID();
|
||||
|
||||
/// Get total number of segments in the store
|
||||
UInt32 getNumOfSegments();
|
||||
|
||||
/// Get current postings list builder
|
||||
const GinIndexPostingsBuilderContainer& getPostings() const { return current_postings; }
|
||||
|
||||
/// Set postings list builder for given term
|
||||
void setPostingsBuilder(const String & term, GinIndexPostingsBuilderPtr builder) { current_postings[term] = builder; }
|
||||
/// Check if we need to write segment to Gin index files
|
||||
bool needToWrite() const;
|
||||
|
||||
/// Accumulate the size of text data which has been digested
|
||||
void incrementCurrentSizeBy(UInt64 sz) { current_size += sz; }
|
||||
|
||||
UInt32 getCurrentSegmentID() const { return current_segment.segment_id;}
|
||||
|
||||
/// Do last segment writing
|
||||
void finalize();
|
||||
|
||||
/// method for writing segment data to Gin index files
|
||||
void writeSegment();
|
||||
|
||||
const String & getName() const {return name;}
|
||||
|
||||
private:
|
||||
friend class GinIndexStoreDeserializer;
|
||||
|
||||
/// Initialize all indexing files for this store
|
||||
void initFileStreams();
|
||||
|
||||
/// Get a range of next available segment IDs by updating file .gin_sid
|
||||
UInt32 getNextSegmentIDRange(const String &file_name, size_t n);
|
||||
|
||||
String name;
|
||||
DataPartStoragePtr storage;
|
||||
MutableDataPartStoragePtr data_part_storage_builder;
|
||||
|
||||
UInt32 cached_segment_num = 0;
|
||||
|
||||
std::mutex gin_index_store_mutex;
|
||||
|
||||
/// Terms dictionaries which are loaded from .gin_dict files
|
||||
SegmentTermDictionaries term_dicts;
|
||||
|
||||
/// container for building postings lists during index construction
|
||||
GinIndexPostingsBuilderContainer current_postings;
|
||||
|
||||
/// The following is for segmentation of Gin index
|
||||
GinIndexSegment current_segment{};
|
||||
UInt64 current_size = 0;
|
||||
const UInt64 max_digestion_size = 0;
|
||||
|
||||
/// File streams for segment, term dictionaries and postings lists
|
||||
std::unique_ptr<WriteBufferFromFileBase> segment_file_stream;
|
||||
std::unique_ptr<WriteBufferFromFileBase> term_dict_file_stream;
|
||||
std::unique_ptr<WriteBufferFromFileBase> postings_file_stream;
|
||||
|
||||
static constexpr auto GIN_SEGMENT_ID_FILE_TYPE = ".gin_sid";
|
||||
static constexpr auto GIN_SEGMENT_FILE_TYPE = ".gin_seg";
|
||||
static constexpr auto GIN_DICTIONARY_FILE_TYPE = ".gin_dict";
|
||||
static constexpr auto GIN_POSTINGS_FILE_TYPE = ".gin_post";
|
||||
};
|
||||
|
||||
using GinIndexStorePtr = std::shared_ptr<GinIndexStore>;
|
||||
|
||||
/// GinIndexStores indexed by part file path
|
||||
using GinIndexStores = std::unordered_map<std::string, GinIndexStorePtr>;
|
||||
|
||||
/// PostingsCacheForStore contains postings lists from 'store' which are retrieved from Gin index files for the terms in query strings
|
||||
/// PostingsCache is per query string(one query can have multiple query strings): when skipping index(row ID ranges) is used for the part during the
|
||||
/// query, the postings cache is created and associated with the store where postings lists are read
|
||||
/// for the tokenized query string. The postings caches are released automatically when the query is done.
|
||||
struct PostingsCacheForStore
|
||||
{
|
||||
/// Which store to retrieve postings lists
|
||||
GinIndexStorePtr store;
|
||||
|
||||
/// map of <query, postings lists>
|
||||
std::unordered_map<String, PostingsCachePtr> cache;
|
||||
|
||||
/// Get postings lists for query string, return nullptr if not found
|
||||
PostingsCachePtr getPostings(const String &query_string) const
|
||||
{
|
||||
auto it {cache.find(query_string)};
|
||||
|
||||
if (it == cache.cend())
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
return it->second;
|
||||
}
|
||||
};
|
||||
|
||||
/// GinIndexStore Factory, which is a singleton for storing GinIndexStores
|
||||
class GinIndexStoreFactory : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
/// Get singleton of GinIndexStoreFactory
|
||||
static GinIndexStoreFactory& instance();
|
||||
|
||||
/// Get GinIndexStore by using index name, disk and part_path (which are combined to create key in stores)
|
||||
GinIndexStorePtr get(const String& name, DataPartStoragePtr storage);
|
||||
|
||||
/// Remove all Gin index files which are under the same part_path
|
||||
void remove(const String& part_path);
|
||||
|
||||
private:
|
||||
GinIndexStores stores;
|
||||
std::mutex stores_mutex;
|
||||
};
|
||||
|
||||
/// Term dictionary information, which contains:
|
||||
|
||||
/// Gin Index Store Reader which helps to read segments, term dictionaries and postings list
|
||||
class GinIndexStoreDeserializer : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
explicit GinIndexStoreDeserializer(const GinIndexStorePtr & store_);
|
||||
|
||||
/// Read all segment information from .gin_seg files
|
||||
void readSegments();
|
||||
|
||||
/// Read all term dictionaries from .gin_dict files
|
||||
void readSegmentTermDictionaries();
|
||||
|
||||
/// Read term dictionary for given segment id
|
||||
void readSegmentTermDictionary(UInt32 segment_id);
|
||||
|
||||
/// Read postings lists for the term
|
||||
SegmentedPostingsListContainer readSegmentedPostingsLists(const String& term);
|
||||
|
||||
/// Read postings lists for terms(which are created by tokenzing query string)
|
||||
PostingsCachePtr createPostingsCacheFromTerms(const std::vector<String>& terms);
|
||||
|
||||
private:
|
||||
/// Initialize Gin index files
|
||||
void initFileStreams();
|
||||
|
||||
/// The store for the reader
|
||||
GinIndexStorePtr store;
|
||||
|
||||
/// File streams for reading Gin Index
|
||||
std::unique_ptr<ReadBufferFromFileBase> segment_file_stream;
|
||||
std::unique_ptr<ReadBufferFromFileBase> term_dict_file_stream;
|
||||
std::unique_ptr<ReadBufferFromFileBase> postings_file_stream;
|
||||
|
||||
/// Current segment, used in building index
|
||||
GinIndexSegment current_segment;
|
||||
};
|
||||
|
||||
}
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <IO/WriteSettings.h>
|
||||
#include <base/types.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Interpreters/TransactionVersionMetadata.h>
|
||||
@ -215,6 +216,7 @@ public:
|
||||
const String & name,
|
||||
size_t buf_size,
|
||||
const WriteSettings & settings) = 0;
|
||||
virtual std::unique_ptr<WriteBufferFromFileBase> writeFile(const String & name, size_t buf_size, WriteMode mode, const WriteSettings & settings) = 0;
|
||||
|
||||
/// A special const method to write transaction file.
|
||||
/// It's const, because file with transaction metadata
|
||||
|
@ -1673,6 +1673,8 @@ void IMergeTreeDataPart::remove()
|
||||
metadata_manager->deleteAll(false);
|
||||
metadata_manager->assertAllDeleted(false);
|
||||
|
||||
GinIndexStoreFactory::instance().remove(getDataPartStoragePtr()->getRelativePath());
|
||||
|
||||
std::list<IDataPartStorage::ProjectionChecksums> projection_checksums;
|
||||
|
||||
for (const auto & [p_name, projection_part] : projection_parts)
|
||||
|
@ -2774,6 +2774,13 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
|
||||
if (!mutation_commands.empty())
|
||||
throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, "The following alter commands: '{}' will modify data on disk, but setting `allow_non_metadata_alters` is disabled", queryToString(mutation_commands.ast()));
|
||||
}
|
||||
|
||||
if (commands.hasInvertedIndex(new_metadata, getContext()) && !settings.allow_experimental_inverted_index)
|
||||
{
|
||||
throw Exception(
|
||||
"Experimental Inverted Index feature is not enabled (the setting 'allow_experimental_inverted_index')",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED);
|
||||
}
|
||||
commands.apply(new_metadata, getContext());
|
||||
|
||||
/// Set of columns that shouldn't be altered.
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexGin.h>
|
||||
#include <Common/MemoryTrackerBlockerInThread.h>
|
||||
|
||||
#include <utility>
|
||||
#include "IO/WriteBufferFromFileDecorator.h"
|
||||
|
||||
@ -212,7 +212,14 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices()
|
||||
default_codec, settings.max_compress_block_size,
|
||||
marks_compression_codec, settings.marks_compress_block_size,
|
||||
settings.query_write_settings));
|
||||
skip_indices_aggregators.push_back(index_helper->createIndexAggregator());
|
||||
|
||||
GinIndexStorePtr store = nullptr;
|
||||
if (dynamic_cast<const MergeTreeIndexGinFilter *>(&*index_helper) != nullptr)
|
||||
{
|
||||
store = std::make_shared<GinIndexStore>(stream_name, data_part->getDataPartStoragePtr(), data_part->getDataPartStoragePtr(), storage.getSettings()->max_digestion_size_per_segment);
|
||||
gin_index_stores[stream_name] = store;
|
||||
}
|
||||
skip_indices_aggregators.push_back(index_helper->createIndexAggregatorForPart(store));
|
||||
skip_index_accumulated_marks.push_back(0);
|
||||
}
|
||||
}
|
||||
@ -268,6 +275,18 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block
|
||||
auto & stream = *skip_indices_streams[i];
|
||||
WriteBuffer & marks_out = stream.compress_marks ? stream.marks_compressed_hashing : stream.marks_hashing;
|
||||
|
||||
GinIndexStorePtr store = nullptr;
|
||||
if (dynamic_cast<const MergeTreeIndexGinFilter *>(&*index_helper) != nullptr)
|
||||
{
|
||||
String stream_name = index_helper->getFileName();
|
||||
auto it = gin_index_stores.find(stream_name);
|
||||
if (it == gin_index_stores.cend())
|
||||
{
|
||||
throw Exception("Index '" + stream_name + "' does not exist", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
store = it->second;
|
||||
}
|
||||
|
||||
for (const auto & granule : granules_to_write)
|
||||
{
|
||||
if (skip_index_accumulated_marks[i] == index_helper->index.granularity)
|
||||
@ -278,7 +297,7 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block
|
||||
|
||||
if (skip_indices_aggregators[i]->empty() && granule.mark_on_start)
|
||||
{
|
||||
skip_indices_aggregators[i] = index_helper->createIndexAggregator();
|
||||
skip_indices_aggregators[i] = index_helper->createIndexAggregatorForPart(store);
|
||||
|
||||
if (stream.compressed_hashing.offset() >= settings.min_compress_block_size)
|
||||
stream.compressed_hashing.next();
|
||||
@ -380,7 +399,11 @@ void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(bool sync)
|
||||
if (sync)
|
||||
stream->sync();
|
||||
}
|
||||
|
||||
for (auto & store: gin_index_stores)
|
||||
{
|
||||
store.second->finalize();
|
||||
}
|
||||
gin_index_stores.clear();
|
||||
skip_indices_streams.clear();
|
||||
skip_indices_aggregators.clear();
|
||||
skip_index_accumulated_marks.clear();
|
||||
|
@ -162,6 +162,7 @@ protected:
|
||||
/// Data is already written up to this mark.
|
||||
size_t current_mark = 0;
|
||||
|
||||
GinIndexStores gin_index_stores;
|
||||
private:
|
||||
void initSkipIndices();
|
||||
void initPrimaryIndex();
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Storages/MergeTree/KeyCondition.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
|
||||
#include <Storages/MergeTree/StorageFromMergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexGin.h>
|
||||
#include <Storages/ReadInOrderOptimizer.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
@ -1692,6 +1693,14 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
|
||||
/// this variable is stored to avoid reading the same granule twice.
|
||||
MergeTreeIndexGranulePtr granule = nullptr;
|
||||
size_t last_index_mark = 0;
|
||||
|
||||
PostingsCacheForStore cache_in_store;
|
||||
|
||||
if (dynamic_cast<const MergeTreeIndexGinFilter *>(&*index_helper) != nullptr)
|
||||
{
|
||||
cache_in_store.store = GinIndexStoreFactory::instance().get(index_helper->getFileName(), part->getDataPartStoragePtr());
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < ranges.size(); ++i)
|
||||
{
|
||||
const MarkRange & index_range = index_ranges[i];
|
||||
@ -1705,6 +1714,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
|
||||
{
|
||||
if (index_mark != index_range.begin || !granule || last_index_mark != index_range.begin)
|
||||
granule = reader.read();
|
||||
const auto * gin_filter_condition = dynamic_cast<const MergeTreeConditionGinFilter *>(&*condition);
|
||||
// Cast to Ann condition
|
||||
auto ann_condition = std::dynamic_pointer_cast<ApproximateNearestNeighbour::IMergeTreeIndexConditionAnn>(condition);
|
||||
if (ann_condition != nullptr)
|
||||
@ -1731,7 +1741,13 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
|
||||
continue;
|
||||
}
|
||||
|
||||
if (!condition->mayBeTrueOnGranule(granule))
|
||||
bool result{false};
|
||||
if (!gin_filter_condition)
|
||||
result = condition->mayBeTrueOnGranule(granule);
|
||||
else
|
||||
result = cache_in_store.store ? gin_filter_condition->mayBeTrueOnGranuleInPart(granule, cache_in_store) : true;
|
||||
|
||||
if (!result)
|
||||
{
|
||||
++granules_dropped;
|
||||
continue;
|
||||
|
@ -183,6 +183,7 @@ MergeTreeConditionFullText::MergeTreeConditionFullText(
|
||||
rpn = std::move(builder).extractRPN();
|
||||
}
|
||||
|
||||
/// Keep in-sync with MergeTreeConditionGinFilter::alwaysUnknownOrTrue
|
||||
bool MergeTreeConditionFullText::alwaysUnknownOrTrue() const
|
||||
{
|
||||
/// Check like in KeyCondition.
|
||||
|
786
src/Storages/MergeTree/MergeTreeIndexGin.cpp
Normal file
786
src/Storages/MergeTree/MergeTreeIndexGin.cpp
Normal file
@ -0,0 +1,786 @@
|
||||
|
||||
#include <algorithm>
|
||||
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Interpreters/GinFilter.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/RPNBuilder.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexGin.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexUtils.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Core/Defines.h>
|
||||
|
||||
#include <Poco/Logger.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int INCORRECT_QUERY;
|
||||
}
|
||||
|
||||
MergeTreeIndexGranuleGinFilter::MergeTreeIndexGranuleGinFilter(
|
||||
const String & index_name_,
|
||||
size_t columns_number,
|
||||
const GinFilterParameters & params_)
|
||||
: index_name(index_name_)
|
||||
, params(params_)
|
||||
, gin_filters(
|
||||
columns_number, GinFilter(params))
|
||||
, has_elems(false)
|
||||
{
|
||||
}
|
||||
|
||||
void MergeTreeIndexGranuleGinFilter::serializeBinary(WriteBuffer & ostr) const
|
||||
{
|
||||
if (empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty fulltext index {}.", backQuote(index_name));
|
||||
|
||||
const auto & size_type = std::make_shared<DataTypeUInt32>();
|
||||
auto size_serialization = size_type->getDefaultSerialization();
|
||||
|
||||
for (const auto & gin_filter : gin_filters)
|
||||
{
|
||||
size_t filter_size = gin_filter.getFilter().size();
|
||||
size_serialization->serializeBinary(filter_size, ostr, {});
|
||||
ostr.write(reinterpret_cast<const char*>(gin_filter.getFilter().data()), filter_size * sizeof(GinFilter::GinSegmentWithRowIDRanges::value_type));
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeIndexGranuleGinFilter::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version)
|
||||
{
|
||||
if (version != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version);
|
||||
|
||||
Field field_rows;
|
||||
const auto & size_type = std::make_shared<DataTypeUInt32>();
|
||||
|
||||
auto size_serialization = size_type->getDefaultSerialization();
|
||||
for (auto & gin_filter : gin_filters)
|
||||
{
|
||||
size_serialization->deserializeBinary(field_rows, istr, {});
|
||||
size_t filter_size = field_rows.get<size_t>();
|
||||
|
||||
if (filter_size == 0)
|
||||
continue;
|
||||
|
||||
gin_filter.getFilter().assign(filter_size, {});
|
||||
istr.readStrict(reinterpret_cast<char*>(gin_filter.getFilter().data()), filter_size * sizeof(GinFilter::GinSegmentWithRowIDRanges::value_type));
|
||||
}
|
||||
has_elems = true;
|
||||
}
|
||||
|
||||
|
||||
MergeTreeIndexAggregatorGinFilter::MergeTreeIndexAggregatorGinFilter(
|
||||
GinIndexStorePtr store_,
|
||||
const Names & index_columns_,
|
||||
const String & index_name_,
|
||||
const GinFilterParameters & params_,
|
||||
TokenExtractorPtr token_extractor_)
|
||||
: store(store_)
|
||||
, index_columns(index_columns_)
|
||||
, index_name (index_name_)
|
||||
, params(params_)
|
||||
, token_extractor(token_extractor_)
|
||||
, granule(
|
||||
std::make_shared<MergeTreeIndexGranuleGinFilter>(
|
||||
index_name, index_columns.size(), params))
|
||||
{
|
||||
}
|
||||
|
||||
MergeTreeIndexGranulePtr MergeTreeIndexAggregatorGinFilter::getGranuleAndReset()
|
||||
{
|
||||
auto new_granule = std::make_shared<MergeTreeIndexGranuleGinFilter>(
|
||||
index_name, index_columns.size(), params);
|
||||
new_granule.swap(granule);
|
||||
return new_granule;
|
||||
}
|
||||
|
||||
void MergeTreeIndexAggregatorGinFilter::addToGinFilter(UInt32 rowID, const char* data, size_t length, GinFilter& gin_filter, UInt64 limit)
|
||||
{
|
||||
size_t cur = 0;
|
||||
size_t token_start = 0;
|
||||
size_t token_len = 0;
|
||||
|
||||
while (cur < length && token_extractor->nextInStringPadded(data, length, &cur, &token_start, &token_len))
|
||||
{
|
||||
gin_filter.add(data + token_start, token_len, rowID, store, limit);
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeIndexAggregatorGinFilter::update(const Block & block, size_t * pos, size_t limit)
|
||||
{
|
||||
if (*pos >= block.rows())
|
||||
throw Exception(
|
||||
"The provided position is not less than the number of block rows. Position: "
|
||||
+ toString(*pos) + ", Block rows: " + toString(block.rows()) + ".", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
size_t rows_read = std::min(limit, block.rows() - *pos);
|
||||
auto row_id = store->getNextRowIDRange(rows_read);
|
||||
auto start_row_id = row_id;
|
||||
|
||||
for (size_t col = 0; col < index_columns.size(); ++col)
|
||||
{
|
||||
const auto & column_with_type = block.getByName(index_columns[col]);
|
||||
const auto & column = column_with_type.column;
|
||||
size_t current_position = *pos;
|
||||
|
||||
bool need_to_write = false;
|
||||
if (isArray(column_with_type.type))
|
||||
{
|
||||
const auto & column_array = assert_cast<const ColumnArray &>(*column);
|
||||
const auto & column_offsets = column_array.getOffsets();
|
||||
const auto & column_key = column_array.getData();
|
||||
|
||||
for (size_t i = 0; i < rows_read; ++i)
|
||||
{
|
||||
size_t element_start_row = column_offsets[current_position - 1];
|
||||
size_t elements_size = column_offsets[current_position] - element_start_row;
|
||||
|
||||
for (size_t row_num = 0; row_num < elements_size; ++row_num)
|
||||
{
|
||||
auto ref = column_key.getDataAt(element_start_row + row_num);
|
||||
addToGinFilter(row_id, ref.data, ref.size, granule->gin_filters[col], rows_read);
|
||||
store->incrementCurrentSizeBy(ref.size);
|
||||
}
|
||||
current_position += 1;
|
||||
row_id++;
|
||||
|
||||
if (store->needToWrite())
|
||||
need_to_write = true;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < rows_read; ++i)
|
||||
{
|
||||
auto ref = column->getDataAt(current_position + i);
|
||||
addToGinFilter(row_id, ref.data, ref.size, granule->gin_filters[col], rows_read);
|
||||
store->incrementCurrentSizeBy(ref.size);
|
||||
row_id++;
|
||||
if (store->needToWrite())
|
||||
need_to_write = true;
|
||||
}
|
||||
}
|
||||
granule->gin_filters[col].addRowRangeToGinFilter(store->getCurrentSegmentID(), start_row_id, static_cast<UInt32>(start_row_id + rows_read - 1));
|
||||
if (need_to_write)
|
||||
{
|
||||
store->writeSegment();
|
||||
}
|
||||
}
|
||||
|
||||
granule->has_elems = true;
|
||||
*pos += rows_read;
|
||||
}
|
||||
|
||||
MergeTreeConditionGinFilter::MergeTreeConditionGinFilter(
|
||||
const SelectQueryInfo & query_info,
|
||||
ContextPtr context_,
|
||||
const Block & index_sample_block,
|
||||
const GinFilterParameters & params_,
|
||||
TokenExtractorPtr token_extactor_)
|
||||
: WithContext(context_), header(index_sample_block)
|
||||
, params(params_)
|
||||
, token_extractor(token_extactor_)
|
||||
, prepared_sets(query_info.prepared_sets)
|
||||
{
|
||||
if (context_->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
if (!query_info.filter_actions_dag)
|
||||
{
|
||||
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
|
||||
return;
|
||||
}
|
||||
rpn = std::move(
|
||||
RPNBuilder<RPNElement>(
|
||||
query_info.filter_actions_dag->getOutputs().at(0), context_,
|
||||
[&](const RPNBuilderTreeNode & node, RPNElement & out)
|
||||
{
|
||||
return this->traverseAtomAST(node, out);
|
||||
}).extractRPN());
|
||||
}
|
||||
|
||||
ASTPtr filter_node = buildFilterNode(query_info.query);
|
||||
|
||||
if (!filter_node)
|
||||
{
|
||||
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
|
||||
return;
|
||||
}
|
||||
|
||||
auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context_);
|
||||
RPNBuilder<RPNElement> builder(
|
||||
filter_node,
|
||||
context_,
|
||||
std::move(block_with_constants),
|
||||
query_info.prepared_sets,
|
||||
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return traverseAtomAST(node, out); });
|
||||
rpn = std::move(builder).extractRPN();
|
||||
|
||||
}
|
||||
|
||||
/// Keep in-sync with MergeTreeConditionFullText::alwaysUnknownOrTrue
|
||||
bool MergeTreeConditionGinFilter::alwaysUnknownOrTrue() const
|
||||
{
|
||||
/// Check like in KeyCondition.
|
||||
std::vector<bool> rpn_stack;
|
||||
|
||||
for (const auto & element : rpn)
|
||||
{
|
||||
if (element.function == RPNElement::FUNCTION_UNKNOWN
|
||||
|| element.function == RPNElement::ALWAYS_TRUE)
|
||||
{
|
||||
rpn_stack.push_back(true);
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_EQUALS
|
||||
|| element.function == RPNElement::FUNCTION_NOT_EQUALS
|
||||
|| element.function == RPNElement::FUNCTION_HAS
|
||||
|| element.function == RPNElement::FUNCTION_IN
|
||||
|| element.function == RPNElement::FUNCTION_NOT_IN
|
||||
|| element.function == RPNElement::FUNCTION_MULTI_SEARCH
|
||||
|| element.function == RPNElement::ALWAYS_FALSE)
|
||||
{
|
||||
rpn_stack.push_back(false);
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_NOT)
|
||||
{
|
||||
// do nothing
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_AND)
|
||||
{
|
||||
auto arg1 = rpn_stack.back();
|
||||
rpn_stack.pop_back();
|
||||
auto arg2 = rpn_stack.back();
|
||||
rpn_stack.back() = arg1 && arg2;
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_OR)
|
||||
{
|
||||
auto arg1 = rpn_stack.back();
|
||||
rpn_stack.pop_back();
|
||||
auto arg2 = rpn_stack.back();
|
||||
rpn_stack.back() = arg1 || arg2;
|
||||
}
|
||||
else
|
||||
throw Exception("Unexpected function type in KeyCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
return rpn_stack[0];
|
||||
}
|
||||
|
||||
bool MergeTreeConditionGinFilter::mayBeTrueOnGranuleInPart(MergeTreeIndexGranulePtr idx_granule,[[maybe_unused]] PostingsCacheForStore &cache_store) const
|
||||
{
|
||||
std::shared_ptr<MergeTreeIndexGranuleGinFilter> granule
|
||||
= std::dynamic_pointer_cast<MergeTreeIndexGranuleGinFilter>(idx_granule);
|
||||
if (!granule)
|
||||
throw Exception(
|
||||
"GinFilter index condition got a granule with the wrong type.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/// Check like in KeyCondition.
|
||||
std::vector<BoolMask> rpn_stack;
|
||||
for (const auto & element : rpn)
|
||||
{
|
||||
if (element.function == RPNElement::FUNCTION_UNKNOWN)
|
||||
{
|
||||
rpn_stack.emplace_back(true, true);
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_EQUALS
|
||||
|| element.function == RPNElement::FUNCTION_NOT_EQUALS
|
||||
|| element.function == RPNElement::FUNCTION_HAS)
|
||||
{
|
||||
rpn_stack.emplace_back(granule->gin_filters[element.key_column].contains(*element.gin_filter, cache_store), true);
|
||||
|
||||
if (element.function == RPNElement::FUNCTION_NOT_EQUALS)
|
||||
rpn_stack.back() = !rpn_stack.back();
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_IN
|
||||
|| element.function == RPNElement::FUNCTION_NOT_IN)
|
||||
{
|
||||
std::vector<bool> result(element.set_gin_filters.back().size(), true);
|
||||
|
||||
for (size_t column = 0; column < element.set_key_position.size(); ++column)
|
||||
{
|
||||
const size_t key_idx = element.set_key_position[column];
|
||||
|
||||
const auto & gin_filters = element.set_gin_filters[column];
|
||||
for (size_t row = 0; row < gin_filters.size(); ++row)
|
||||
result[row] = result[row] && granule->gin_filters[key_idx].contains(gin_filters[row], cache_store);
|
||||
}
|
||||
|
||||
rpn_stack.emplace_back(
|
||||
std::find(std::cbegin(result), std::cend(result), true) != std::end(result), true);
|
||||
if (element.function == RPNElement::FUNCTION_NOT_IN)
|
||||
rpn_stack.back() = !rpn_stack.back();
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_MULTI_SEARCH)
|
||||
{
|
||||
std::vector<bool> result(element.set_gin_filters.back().size(), true);
|
||||
|
||||
const auto & gin_filters = element.set_gin_filters[0];
|
||||
|
||||
for (size_t row = 0; row < gin_filters.size(); ++row)
|
||||
result[row] = result[row] && granule->gin_filters[element.key_column].contains(gin_filters[row], cache_store);
|
||||
|
||||
rpn_stack.emplace_back(
|
||||
std::find(std::cbegin(result), std::cend(result), true) != std::end(result), true);
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_NOT)
|
||||
{
|
||||
rpn_stack.back() = !rpn_stack.back();
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_AND)
|
||||
{
|
||||
auto arg1 = rpn_stack.back();
|
||||
rpn_stack.pop_back();
|
||||
auto arg2 = rpn_stack.back();
|
||||
rpn_stack.back() = arg1 & arg2;
|
||||
}
|
||||
else if (element.function == RPNElement::FUNCTION_OR)
|
||||
{
|
||||
auto arg1 = rpn_stack.back();
|
||||
rpn_stack.pop_back();
|
||||
auto arg2 = rpn_stack.back();
|
||||
rpn_stack.back() = arg1 | arg2;
|
||||
}
|
||||
else if (element.function == RPNElement::ALWAYS_FALSE)
|
||||
{
|
||||
rpn_stack.emplace_back(false, true);
|
||||
}
|
||||
else if (element.function == RPNElement::ALWAYS_TRUE)
|
||||
{
|
||||
rpn_stack.emplace_back(true, false);
|
||||
}
|
||||
else
|
||||
throw Exception("Unexpected function type in GinFilterCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
if (rpn_stack.size() != 1)
|
||||
throw Exception("Unexpected stack size in GinFilterCondition::mayBeTrueOnGranule", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return rpn_stack[0].can_be_true;
|
||||
}
|
||||
|
||||
bool MergeTreeConditionGinFilter::traverseAtomAST(const RPNBuilderTreeNode & node, RPNElement & out)
|
||||
{
|
||||
{
|
||||
Field const_value;
|
||||
DataTypePtr const_type;
|
||||
|
||||
if (node.tryGetConstant(const_value, const_type))
|
||||
{
|
||||
/// Check constant like in KeyCondition
|
||||
if (const_value.getType() == Field::Types::UInt64
|
||||
|| const_value.getType() == Field::Types::Int64
|
||||
|| const_value.getType() == Field::Types::Float64)
|
||||
{
|
||||
/// Zero in all types is represented in memory the same way as in UInt64.
|
||||
out.function = const_value.get<UInt64>()
|
||||
? RPNElement::ALWAYS_TRUE
|
||||
: RPNElement::ALWAYS_FALSE;
|
||||
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (node.isFunction())
|
||||
{
|
||||
const auto function = node.toFunctionNode();
|
||||
// auto arguments_size = function.getArgumentsSize();
|
||||
auto function_name = function.getFunctionName();
|
||||
|
||||
size_t function_arguments_size = function.getArgumentsSize();
|
||||
if (function_arguments_size != 2)
|
||||
return false;
|
||||
auto lhs_argument = function.getArgumentAt(0);
|
||||
auto rhs_argument = function.getArgumentAt(1);
|
||||
|
||||
if (functionIsInOrGlobalInOperator(function_name))
|
||||
{
|
||||
if (tryPrepareSetGinFilter(lhs_argument, rhs_argument, out))
|
||||
{
|
||||
if (function_name == "notIn")
|
||||
{
|
||||
out.function = RPNElement::FUNCTION_NOT_IN;
|
||||
return true;
|
||||
}
|
||||
else if (function_name == "in")
|
||||
{
|
||||
out.function = RPNElement::FUNCTION_IN;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (function_name == "equals" ||
|
||||
function_name == "notEquals" ||
|
||||
function_name == "has" ||
|
||||
function_name == "mapContains" ||
|
||||
function_name == "like" ||
|
||||
function_name == "notLike" ||
|
||||
function_name == "hasToken" ||
|
||||
function_name == "startsWith" ||
|
||||
function_name == "endsWith" ||
|
||||
function_name == "multiSearchAny")
|
||||
{
|
||||
Field const_value;
|
||||
DataTypePtr const_type;
|
||||
if (rhs_argument.tryGetConstant(const_value, const_type))
|
||||
{
|
||||
if (traverseASTEquals(function_name, lhs_argument, const_type, const_value, out))
|
||||
return true;
|
||||
}
|
||||
else if (lhs_argument.tryGetConstant(const_value, const_type) && (function_name == "equals" || function_name == "notEquals"))
|
||||
{
|
||||
if (traverseASTEquals(function_name, rhs_argument, const_type, const_value, out))
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
bool MergeTreeConditionGinFilter::traverseASTEquals(
|
||||
const String & function_name,
|
||||
const RPNBuilderTreeNode & key_ast,
|
||||
const DataTypePtr & value_type,
|
||||
const Field & value_field,
|
||||
RPNElement & out)
|
||||
{
|
||||
auto value_data_type = WhichDataType(value_type);
|
||||
if (!value_data_type.isStringOrFixedString() && !value_data_type.isArray())
|
||||
return false;
|
||||
|
||||
Field const_value = value_field;
|
||||
size_t key_column_num = 0;
|
||||
bool key_exists = header.has(key_ast.getColumnName());
|
||||
bool map_key_exists = header.has(fmt::format("mapKeys({})", key_ast.getColumnName()));
|
||||
|
||||
if (key_ast.isFunction())
|
||||
{
|
||||
const auto function = key_ast.toFunctionNode();
|
||||
if (function.getFunctionName() == "arrayElement")
|
||||
{
|
||||
/** Try to parse arrayElement for mapKeys index.
|
||||
* It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map
|
||||
* we return default value for arrayElement.
|
||||
*
|
||||
* We cannot skip keys that does not exist in map if comparison is with default type value because
|
||||
* that way we skip necessary granules where map key does not exists.
|
||||
*/
|
||||
if (value_field == value_type->getDefault())
|
||||
return false;
|
||||
|
||||
auto first_argument = function.getArgumentAt(0);
|
||||
const auto map_column_name = first_argument.getColumnName();
|
||||
auto map_keys_index_column_name = fmt::format("mapKeys({})", map_column_name);
|
||||
auto map_values_index_column_name = fmt::format("mapValues({})", map_column_name);
|
||||
|
||||
if (header.has(map_keys_index_column_name))
|
||||
{
|
||||
auto argument = function.getArgumentAt(1);
|
||||
DataTypePtr const_type;
|
||||
if (argument.tryGetConstant(const_value, const_type))
|
||||
{
|
||||
key_column_num = header.getPositionByName(map_keys_index_column_name);
|
||||
key_exists = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else if (header.has(map_values_index_column_name))
|
||||
{
|
||||
key_column_num = header.getPositionByName(map_values_index_column_name);
|
||||
key_exists = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!key_exists && !map_key_exists)
|
||||
return false;
|
||||
|
||||
if (map_key_exists && (function_name == "has" || function_name == "mapContains"))
|
||||
{
|
||||
out.key_column = key_column_num;
|
||||
out.function = RPNElement::FUNCTION_HAS;
|
||||
out.gin_filter = std::make_unique<GinFilter>(params);
|
||||
auto & value = const_value.get<String>();
|
||||
token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter);
|
||||
return true;
|
||||
}
|
||||
else if (function_name == "has")
|
||||
{
|
||||
out.key_column = key_column_num;
|
||||
out.function = RPNElement::FUNCTION_HAS;
|
||||
out.gin_filter = std::make_unique<GinFilter>(params);
|
||||
auto & value = const_value.get<String>();
|
||||
token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter);
|
||||
return true;
|
||||
}
|
||||
|
||||
if (function_name == "notEquals")
|
||||
{
|
||||
out.key_column = key_column_num;
|
||||
out.function = RPNElement::FUNCTION_NOT_EQUALS;
|
||||
out.gin_filter = std::make_unique<GinFilter>(params);
|
||||
const auto & value = const_value.get<String>();
|
||||
token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter);
|
||||
return true;
|
||||
}
|
||||
else if (function_name == "equals")
|
||||
{
|
||||
out.key_column = key_column_num;
|
||||
out.function = RPNElement::FUNCTION_EQUALS;
|
||||
out.gin_filter = std::make_unique<GinFilter>(params);
|
||||
const auto & value = const_value.get<String>();
|
||||
token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter);
|
||||
return true;
|
||||
}
|
||||
else if (function_name == "like")
|
||||
{
|
||||
out.key_column = key_column_num;
|
||||
out.function = RPNElement::FUNCTION_EQUALS;
|
||||
out.gin_filter = std::make_unique<GinFilter>(params);
|
||||
const auto & value = const_value.get<String>();
|
||||
token_extractor->stringLikeToGinFilter(value.data(), value.size(), *out.gin_filter);
|
||||
return true;
|
||||
}
|
||||
else if (function_name == "notLike")
|
||||
{
|
||||
out.key_column = key_column_num;
|
||||
out.function = RPNElement::FUNCTION_NOT_EQUALS;
|
||||
out.gin_filter = std::make_unique<GinFilter>(params);
|
||||
const auto & value = const_value.get<String>();
|
||||
token_extractor->stringLikeToGinFilter(value.data(), value.size(), *out.gin_filter);
|
||||
return true;
|
||||
}
|
||||
else if (function_name == "hasToken")
|
||||
{
|
||||
out.key_column = key_column_num;
|
||||
out.function = RPNElement::FUNCTION_EQUALS;
|
||||
out.gin_filter = std::make_unique<GinFilter>(params);
|
||||
const auto & value = const_value.get<String>();
|
||||
token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter);
|
||||
return true;
|
||||
}
|
||||
else if (function_name == "startsWith")
|
||||
{
|
||||
out.key_column = key_column_num;
|
||||
out.function = RPNElement::FUNCTION_EQUALS;
|
||||
out.gin_filter = std::make_unique<GinFilter>(params);
|
||||
const auto & value = const_value.get<String>();
|
||||
token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter);
|
||||
return true;
|
||||
}
|
||||
else if (function_name == "endsWith")
|
||||
{
|
||||
out.key_column = key_column_num;
|
||||
out.function = RPNElement::FUNCTION_EQUALS;
|
||||
out.gin_filter = std::make_unique<GinFilter>(params);
|
||||
const auto & value = const_value.get<String>();
|
||||
token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter);
|
||||
return true;
|
||||
}
|
||||
else if (function_name == "multiSearchAny")
|
||||
{
|
||||
out.key_column = key_column_num;
|
||||
out.function = RPNElement::FUNCTION_MULTI_SEARCH;
|
||||
|
||||
/// 2d vector is not needed here but is used because already exists for FUNCTION_IN
|
||||
std::vector<std::vector<GinFilter>> gin_filters;
|
||||
gin_filters.emplace_back();
|
||||
for (const auto & element : const_value.get<Array>())
|
||||
{
|
||||
if (element.getType() != Field::Types::String)
|
||||
return false;
|
||||
|
||||
gin_filters.back().emplace_back(params);
|
||||
const auto & value = element.get<String>();
|
||||
token_extractor->stringToGinFilter(value.data(), value.size(), gin_filters.back().back());
|
||||
}
|
||||
out.set_gin_filters = std::move(gin_filters);
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
bool MergeTreeConditionGinFilter::tryPrepareSetGinFilter(
|
||||
const RPNBuilderTreeNode & lhs,
|
||||
const RPNBuilderTreeNode & rhs,
|
||||
RPNElement & out)
|
||||
{
|
||||
std::vector<KeyTuplePositionMapping> key_tuple_mapping;
|
||||
DataTypes data_types;
|
||||
|
||||
if (lhs.isFunction() && lhs.toFunctionNode().getFunctionName() == "tuple")
|
||||
{
|
||||
const auto function = lhs.toFunctionNode();
|
||||
auto arguments_size = function.getArgumentsSize();
|
||||
for (size_t i = 0; i < arguments_size; ++i)
|
||||
{
|
||||
if (header.has(function.getArgumentAt(i).getColumnName()))
|
||||
{
|
||||
auto key = header.getPositionByName(function.getArgumentAt(i).getColumnName());
|
||||
key_tuple_mapping.emplace_back(i, key);
|
||||
data_types.push_back(header.getByPosition(key).type);
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
if (header.has(lhs.getColumnName()))
|
||||
{
|
||||
auto key = header.getPositionByName(lhs.getColumnName());
|
||||
key_tuple_mapping.emplace_back(0, key);
|
||||
data_types.push_back(header.getByPosition(key).type);
|
||||
}
|
||||
}
|
||||
|
||||
if (key_tuple_mapping.empty())
|
||||
return false;
|
||||
|
||||
ConstSetPtr prepared_set = rhs.tryGetPreparedSet();
|
||||
if (!prepared_set && !prepared_set->hasExplicitSetElements())
|
||||
return false;
|
||||
|
||||
for (const auto & data_type : prepared_set->getDataTypes())
|
||||
if (data_type->getTypeId() != TypeIndex::String && data_type->getTypeId() != TypeIndex::FixedString)
|
||||
return false;
|
||||
|
||||
std::vector<std::vector<GinFilter>> gin_filters;
|
||||
std::vector<size_t> key_position;
|
||||
|
||||
Columns columns = prepared_set->getSetElements();
|
||||
for (const auto & elem : key_tuple_mapping)
|
||||
{
|
||||
gin_filters.emplace_back();
|
||||
gin_filters.back().reserve(prepared_set->getTotalRowCount());
|
||||
key_position.push_back(elem.key_index);
|
||||
|
||||
size_t tuple_idx = elem.tuple_index;
|
||||
const auto & column = columns[tuple_idx];
|
||||
for (size_t row = 0; row < prepared_set->getTotalRowCount(); ++row)
|
||||
{
|
||||
gin_filters.back().emplace_back(params);
|
||||
auto ref = column->getDataAt(row);
|
||||
token_extractor->stringToGinFilter(ref.data, ref.size, gin_filters.back().back());
|
||||
}
|
||||
}
|
||||
|
||||
out.set_key_position = std::move(key_position);
|
||||
out.set_gin_filters = std::move(gin_filters);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
MergeTreeIndexGranulePtr MergeTreeIndexGinFilter::createIndexGranule() const
|
||||
{
|
||||
return std::make_shared<MergeTreeIndexGranuleGinFilter>(index.name, index.column_names.size(), params);
|
||||
}
|
||||
|
||||
MergeTreeIndexAggregatorPtr MergeTreeIndexGinFilter::createIndexAggregator() const
|
||||
{
|
||||
/// should not be called: createIndexAggregatorForPart should be used
|
||||
assert(false);
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
MergeTreeIndexAggregatorPtr MergeTreeIndexGinFilter::createIndexAggregatorForPart(const GinIndexStorePtr &store) const
|
||||
{
|
||||
return std::make_shared<MergeTreeIndexAggregatorGinFilter>(store, index.column_names, index.name, params, token_extractor.get());
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexGinFilter::createIndexCondition(
|
||||
const SelectQueryInfo & query, ContextPtr context) const
|
||||
{
|
||||
return std::make_shared<MergeTreeConditionGinFilter>(query, context, index.sample_block, params, token_extractor.get());
|
||||
};
|
||||
|
||||
bool MergeTreeIndexGinFilter::mayBenefitFromIndexForIn(const ASTPtr & node) const
|
||||
{
|
||||
return std::find(std::cbegin(index.column_names), std::cend(index.column_names), node->getColumnName()) != std::cend(index.column_names);
|
||||
}
|
||||
|
||||
MergeTreeIndexPtr ginIndexCreator(
|
||||
const IndexDescription & index)
|
||||
{
|
||||
size_t n = index.arguments.empty() ? 0 : index.arguments[0].get<size_t>();
|
||||
Float64 density = index.arguments.size() < 2 ? 1.0f : index.arguments[1].get<Float64>();
|
||||
GinFilterParameters params(n, density);
|
||||
|
||||
/// Use SplitTokenExtractor when n is 0, otherwise use NgramTokenExtractor
|
||||
if (n > 0)
|
||||
{
|
||||
auto tokenizer = std::make_unique<NgramTokenExtractor>(n);
|
||||
return std::make_shared<MergeTreeIndexGinFilter>(index, params, std::move(tokenizer));
|
||||
}
|
||||
else
|
||||
{
|
||||
auto tokenizer = std::make_unique<SplitTokenExtractor>();
|
||||
return std::make_shared<MergeTreeIndexGinFilter>(index, params, std::move(tokenizer));
|
||||
}
|
||||
}
|
||||
|
||||
void ginIndexValidator(const IndexDescription & index, bool /*attach*/)
|
||||
{
|
||||
for (const auto & index_data_type : index.data_types)
|
||||
{
|
||||
WhichDataType data_type(index_data_type);
|
||||
|
||||
if (data_type.isArray())
|
||||
{
|
||||
const auto & gin_type = assert_cast<const DataTypeArray &>(*index_data_type);
|
||||
data_type = WhichDataType(gin_type.getNestedType());
|
||||
}
|
||||
else if (data_type.isLowCarnality())
|
||||
{
|
||||
const auto & low_cardinality = assert_cast<const DataTypeLowCardinality &>(*index_data_type);
|
||||
data_type = WhichDataType(low_cardinality.getDictionaryType());
|
||||
}
|
||||
|
||||
if (!data_type.isString() && !data_type.isFixedString())
|
||||
throw Exception("Inverted index can be used only with `String`, `FixedString`, `LowCardinality(String)`, `LowCardinality(FixedString)` column or Array with `String` or `FixedString` values column.", ErrorCodes::INCORRECT_QUERY);
|
||||
}
|
||||
|
||||
if (index.type != GinFilter::getName())
|
||||
throw Exception("Unknown index type: " + backQuote(index.name), ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (index.arguments.size() > 2)
|
||||
throw Exception("Inverted index must have less than two arguments.", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64)
|
||||
throw Exception("The first Inverted index argument must be positive integer.", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
if (index.arguments.size() == 2 && (index.arguments[1].getType() != Field::Types::Float64 || index.arguments[1].get<Float64>() <= 0 || index.arguments[1].get<Float64>() > 1))
|
||||
throw Exception("The second Inverted index argument must be a float between 0 and 1.", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
size_t ngrams = index.arguments.empty() ? 0 : index.arguments[0].get<size_t>();
|
||||
Float64 density = index.arguments.size() < 2 ? 1.0f : index.arguments[1].get<Float64>();
|
||||
|
||||
/// Just validate
|
||||
GinFilterParameters params(ngrams, density);
|
||||
}
|
||||
|
||||
}
|
183
src/Storages/MergeTree/MergeTreeIndexGin.h
Normal file
183
src/Storages/MergeTree/MergeTreeIndexGin.h
Normal file
@ -0,0 +1,183 @@
|
||||
#pragma once
|
||||
#include <atomic>
|
||||
#include <base/types.h>
|
||||
|
||||
#include <memory>
|
||||
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/KeyCondition.h>
|
||||
#include <Interpreters/ITokenExtractor.h>
|
||||
#include <Interpreters/GinFilter.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct MergeTreeIndexGranuleGinFilter final : public IMergeTreeIndexGranule
|
||||
{
|
||||
explicit MergeTreeIndexGranuleGinFilter(
|
||||
const String & index_name_,
|
||||
size_t columns_number,
|
||||
const GinFilterParameters & params_);
|
||||
|
||||
~MergeTreeIndexGranuleGinFilter() override = default;
|
||||
|
||||
void serializeBinary(WriteBuffer & ostr) const override;
|
||||
void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override;
|
||||
|
||||
bool empty() const override { return !has_elems; }
|
||||
|
||||
String index_name;
|
||||
GinFilterParameters params;
|
||||
|
||||
std::vector<GinFilter> gin_filters;
|
||||
bool has_elems;
|
||||
};
|
||||
|
||||
using MergeTreeIndexGranuleGinFilterPtr = std::shared_ptr<MergeTreeIndexGranuleGinFilter>;
|
||||
|
||||
struct MergeTreeIndexAggregatorGinFilter final : IMergeTreeIndexAggregator
|
||||
{
|
||||
explicit MergeTreeIndexAggregatorGinFilter(
|
||||
GinIndexStorePtr store_,
|
||||
const Names & index_columns_,
|
||||
const String & index_name_,
|
||||
const GinFilterParameters & params_,
|
||||
TokenExtractorPtr token_extractor_);
|
||||
|
||||
~MergeTreeIndexAggregatorGinFilter() override = default;
|
||||
|
||||
bool empty() const override { return !granule || granule->empty(); }
|
||||
MergeTreeIndexGranulePtr getGranuleAndReset() override;
|
||||
|
||||
void update(const Block & block, size_t * pos, size_t limit) override;
|
||||
|
||||
void addToGinFilter(UInt32 rowID, const char* data, size_t length, GinFilter& gin_filter, UInt64 limit);
|
||||
|
||||
GinIndexStorePtr store;
|
||||
Names index_columns;
|
||||
const String index_name;
|
||||
const GinFilterParameters params;
|
||||
TokenExtractorPtr token_extractor;
|
||||
|
||||
MergeTreeIndexGranuleGinFilterPtr granule;
|
||||
};
|
||||
|
||||
|
||||
class MergeTreeConditionGinFilter final : public IMergeTreeIndexCondition, WithContext
|
||||
{
|
||||
public:
|
||||
MergeTreeConditionGinFilter(
|
||||
const SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
const Block & index_sample_block,
|
||||
const GinFilterParameters & params_,
|
||||
TokenExtractorPtr token_extactor_);
|
||||
|
||||
~MergeTreeConditionGinFilter() override = default;
|
||||
|
||||
bool alwaysUnknownOrTrue() const override;
|
||||
bool mayBeTrueOnGranule([[maybe_unused]]MergeTreeIndexGranulePtr idx_granule) const override
|
||||
{
|
||||
/// should call mayBeTrueOnGranuleInPart instead
|
||||
assert(false);
|
||||
return false;
|
||||
}
|
||||
bool mayBeTrueOnGranuleInPart(MergeTreeIndexGranulePtr idx_granule, [[maybe_unused]] PostingsCacheForStore& cache_store) const;
|
||||
private:
|
||||
struct KeyTuplePositionMapping
|
||||
{
|
||||
KeyTuplePositionMapping(size_t tuple_index_, size_t key_index_) : tuple_index(tuple_index_), key_index(key_index_) {}
|
||||
|
||||
size_t tuple_index;
|
||||
size_t key_index;
|
||||
};
|
||||
/// Uses RPN like KeyCondition
|
||||
struct RPNElement
|
||||
{
|
||||
enum Function
|
||||
{
|
||||
/// Atoms of a Boolean expression.
|
||||
FUNCTION_EQUALS,
|
||||
FUNCTION_NOT_EQUALS,
|
||||
FUNCTION_HAS,
|
||||
FUNCTION_IN,
|
||||
FUNCTION_NOT_IN,
|
||||
FUNCTION_MULTI_SEARCH,
|
||||
FUNCTION_UNKNOWN, /// Can take any value.
|
||||
/// Operators of the logical expression.
|
||||
FUNCTION_NOT,
|
||||
FUNCTION_AND,
|
||||
FUNCTION_OR,
|
||||
/// Constants
|
||||
ALWAYS_FALSE,
|
||||
ALWAYS_TRUE,
|
||||
};
|
||||
|
||||
RPNElement( /// NOLINT
|
||||
Function function_ = FUNCTION_UNKNOWN, size_t key_column_ = 0, std::unique_ptr<GinFilter> && const_gin_filter_ = nullptr)
|
||||
: function(function_), key_column(key_column_), gin_filter(std::move(const_gin_filter_)) {}
|
||||
|
||||
Function function = FUNCTION_UNKNOWN;
|
||||
/// For FUNCTION_EQUALS, FUNCTION_NOT_EQUALS and FUNCTION_MULTI_SEARCH
|
||||
size_t key_column;
|
||||
|
||||
/// For FUNCTION_EQUALS, FUNCTION_NOT_EQUALS
|
||||
std::unique_ptr<GinFilter> gin_filter;
|
||||
|
||||
/// For FUNCTION_IN, FUNCTION_NOT_IN and FUNCTION_MULTI_SEARCH
|
||||
std::vector<std::vector<GinFilter>> set_gin_filters;
|
||||
|
||||
/// For FUNCTION_IN and FUNCTION_NOT_IN
|
||||
std::vector<size_t> set_key_position;
|
||||
};
|
||||
|
||||
using RPN = std::vector<RPNElement>;
|
||||
|
||||
bool traverseAtomAST(const RPNBuilderTreeNode & node, RPNElement & out);
|
||||
|
||||
bool traverseASTEquals(
|
||||
const String & function_name,
|
||||
const RPNBuilderTreeNode & key_ast,
|
||||
const DataTypePtr & value_type,
|
||||
const Field & value_field,
|
||||
RPNElement & out);
|
||||
|
||||
bool tryPrepareSetGinFilter(const RPNBuilderTreeNode & lhs, const RPNBuilderTreeNode & rhs, RPNElement & out);
|
||||
|
||||
static bool createFunctionEqualsCondition(
|
||||
RPNElement & out, const Field & value, const GinFilterParameters & params, TokenExtractorPtr token_extractor);
|
||||
|
||||
const Block & header;
|
||||
GinFilterParameters params;
|
||||
TokenExtractorPtr token_extractor;
|
||||
RPN rpn;
|
||||
/// Sets from syntax analyzer.
|
||||
PreparedSetsPtr prepared_sets;
|
||||
};
|
||||
|
||||
class MergeTreeIndexGinFilter final : public IMergeTreeIndex
|
||||
{
|
||||
public:
|
||||
MergeTreeIndexGinFilter(
|
||||
const IndexDescription & index_,
|
||||
const GinFilterParameters & params_,
|
||||
std::unique_ptr<ITokenExtractor> && token_extractor_)
|
||||
: IMergeTreeIndex(index_)
|
||||
, params(params_)
|
||||
, token_extractor(std::move(token_extractor_)) {}
|
||||
|
||||
~MergeTreeIndexGinFilter() override = default;
|
||||
|
||||
MergeTreeIndexGranulePtr createIndexGranule() const override;
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregator() const override;
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregatorForPart(const GinIndexStorePtr &store) const override;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(
|
||||
const SelectQueryInfo & query, ContextPtr context) const override;
|
||||
|
||||
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
||||
|
||||
GinFilterParameters params;
|
||||
/// Function for selecting next token.
|
||||
std::unique_ptr<ITokenExtractor> token_extractor;
|
||||
};
|
||||
|
||||
}
|
@ -3,7 +3,6 @@
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
#include <numeric>
|
||||
|
||||
#include <boost/algorithm/string.hpp>
|
||||
@ -106,6 +105,9 @@ MergeTreeIndexFactory::MergeTreeIndexFactory()
|
||||
registerCreator("annoy", annoyIndexCreator);
|
||||
registerValidator("annoy", annoyIndexValidator);
|
||||
#endif
|
||||
registerCreator("inverted", ginIndexCreator);
|
||||
registerValidator("inverted", ginIndexValidator);
|
||||
|
||||
}
|
||||
|
||||
MergeTreeIndexFactory & MergeTreeIndexFactory::instance()
|
||||
|
@ -1,10 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <string>
|
||||
#include <map>
|
||||
#include <unordered_map>
|
||||
#include <vector>
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
#include <mutex>
|
||||
#include <Core/Block.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
|
||||
@ -14,6 +16,8 @@
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
|
||||
#include <Storages/MergeTree/GinIndexStore.h>
|
||||
|
||||
constexpr auto INDEX_FILE_PREFIX = "skp_idx_";
|
||||
|
||||
namespace DB
|
||||
@ -162,6 +166,11 @@ struct IMergeTreeIndex
|
||||
|
||||
virtual MergeTreeIndexAggregatorPtr createIndexAggregator() const = 0;
|
||||
|
||||
virtual MergeTreeIndexAggregatorPtr createIndexAggregatorForPart([[maybe_unused]]const GinIndexStorePtr &store) const
|
||||
{
|
||||
return createIndexAggregator();
|
||||
}
|
||||
|
||||
virtual MergeTreeIndexConditionPtr createIndexCondition(
|
||||
const SelectQueryInfo & query_info, ContextPtr context) const = 0;
|
||||
|
||||
@ -228,5 +237,7 @@ void hypothesisIndexValidator(const IndexDescription & index, bool attach);
|
||||
MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index);
|
||||
void annoyIndexValidator(const IndexDescription & index, bool attach);
|
||||
#endif
|
||||
MergeTreeIndexPtr ginIndexCreator(const IndexDescription& index);
|
||||
void ginIndexValidator(const IndexDescription& index, bool attach);
|
||||
|
||||
}
|
||||
|
@ -25,6 +25,7 @@ struct Settings;
|
||||
M(UInt64, min_compress_block_size, 0, "When granule is written, compress the data in buffer if the size of pending uncompressed data is larger or equal than the specified threshold. If this setting is not set, the corresponding global setting is used.", 0) \
|
||||
M(UInt64, max_compress_block_size, 0, "Compress the pending uncompressed data in buffer if its size is larger or equal than the specified threshold. Block of data will be compressed even if the current granule is not finished. If this setting is not set, the corresponding global setting is used.", 0) \
|
||||
M(UInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \
|
||||
M(UInt64, max_digestion_size_per_segment, 1024 * 1024 * 256, "Max number of bytes to digest per segment to build GIN index.", 0) \
|
||||
\
|
||||
/** Data storing format settings. */ \
|
||||
M(UInt64, min_bytes_for_wide_part, 10485760, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \
|
||||
@ -85,6 +86,8 @@ struct Settings;
|
||||
M(UInt64, replicated_deduplication_window_seconds, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \
|
||||
M(UInt64, replicated_deduplication_window_for_async_inserts, 10000, "How many last hash values of async_insert blocks should be kept in ZooKeeper (old blocks will be deleted).", 0) \
|
||||
M(UInt64, replicated_deduplication_window_seconds_for_async_inserts, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window_for_async_inserts\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \
|
||||
M(Milliseconds, async_block_ids_cache_min_update_interval_ms, 100, "minimum interval between updates of async_block_ids_cache", 0) \
|
||||
M(Bool, use_async_block_ids_cache, false, "use in-memory cache to filter duplicated async inserts based on block ids", 0) \
|
||||
M(UInt64, max_replicated_logs_to_keep, 1000, "How many records may be in log, if there is inactive replica. Inactive replica becomes lost when when this number exceed.", 0) \
|
||||
M(UInt64, min_replicated_logs_to_keep, 10, "Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.", 0) \
|
||||
M(Seconds, prefer_fetch_merged_part_time_threshold, 3600, "If time passed after replication log entry creation exceeds this threshold and sum size of parts is greater than \"prefer_fetch_merged_part_size_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \
|
||||
|
@ -144,6 +144,7 @@ bool ReplicatedMergeTreeRestartingThread::runImpl()
|
||||
storage.mutations_finalizing_task->activateAndSchedule();
|
||||
storage.merge_selecting_task->activateAndSchedule();
|
||||
storage.cleanup_thread.start();
|
||||
storage.async_block_ids_cache.start();
|
||||
storage.part_check_thread.start();
|
||||
|
||||
LOG_DEBUG(log, "Table started successfully");
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Common/ThreadFuzzer.h>
|
||||
#include <Storages/MergeTree/AsyncBlockIDsCache.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <Core/Block.h>
|
||||
#include <IO/Operators.h>
|
||||
@ -105,7 +106,7 @@ struct ReplicatedMergeTreeSinkImpl<async_insert>::DelayedChunk
|
||||
String conflict_block_id = p.filename();
|
||||
auto it = block_id_to_offset_idx.find(conflict_block_id);
|
||||
if (it == block_id_to_offset_idx.end())
|
||||
throw Exception("Unknown conflict path " + conflict_block_id, ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown conflict path {}", conflict_block_id);
|
||||
/// if this filter is for self_dedup, that means the block paths is selected by `filterSelfDuplicate`, which is a self purge.
|
||||
/// in this case, we don't know if zk has this insert, then we should keep one insert, to avoid missing this insert.
|
||||
offset_idx.insert(std::end(offset_idx), std::begin(it->second) + self_dedup, std::end(it->second));
|
||||
@ -544,6 +545,8 @@ void ReplicatedMergeTreeSinkImpl<true>::finishDelayedChunk(const ZooKeeperWithFa
|
||||
partition.temp_part = storage.writer.writeTempPart(partition.block_with_partition, metadata_snapshot, context);
|
||||
}
|
||||
|
||||
/// reset the cache version to zero for every partition write.
|
||||
cache_version = 0;
|
||||
while (true)
|
||||
{
|
||||
partition.temp_part.finalize();
|
||||
@ -676,6 +679,13 @@ std::vector<String> ReplicatedMergeTreeSinkImpl<async_insert>::commitPart(
|
||||
BlockIDsType block_id_path ;
|
||||
if constexpr (async_insert)
|
||||
{
|
||||
/// prefilter by cache
|
||||
conflict_block_ids = storage.async_block_ids_cache.detectConflicts(block_id, cache_version);
|
||||
if (!conflict_block_ids.empty())
|
||||
{
|
||||
cache_version = 0;
|
||||
return;
|
||||
}
|
||||
for (const auto & single_block_id : block_id)
|
||||
block_id_path.push_back(storage.zookeeper_path + "/async_blocks/" + single_block_id);
|
||||
}
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <base/types.h>
|
||||
#include <Storages/MergeTree/ZooKeeperRetries.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperWithFaultInjection.h>
|
||||
#include <Storages/MergeTree/AsyncBlockIDsCache.h>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
@ -115,6 +116,8 @@ private:
|
||||
size_t quorum_timeout_ms;
|
||||
size_t max_parts_per_block;
|
||||
|
||||
UInt64 cache_version = 0;
|
||||
|
||||
bool is_attach = false;
|
||||
bool quorum_parallel = false;
|
||||
const bool deduplicate = true;
|
||||
|
@ -91,6 +91,7 @@
|
||||
|
||||
#include <base/scope_guard.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Storages/MergeTree/AsyncBlockIDsCache.h>
|
||||
|
||||
#include <boost/algorithm/string/join.hpp>
|
||||
#include <boost/algorithm/string/replace.hpp>
|
||||
@ -283,6 +284,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
||||
, queue(*this, merge_strategy_picker)
|
||||
, fetcher(*this)
|
||||
, cleanup_thread(*this)
|
||||
, async_block_ids_cache(*this)
|
||||
, part_check_thread(*this)
|
||||
, restarting_thread(*this)
|
||||
, part_moves_between_shards_orchestrator(*this)
|
||||
@ -4404,6 +4406,7 @@ void StorageReplicatedMergeTree::partialShutdown()
|
||||
mutations_finalizing_task->deactivate();
|
||||
|
||||
cleanup_thread.stop();
|
||||
async_block_ids_cache.stop();
|
||||
part_check_thread.stop();
|
||||
|
||||
/// Stop queue processing
|
||||
@ -8903,16 +8906,20 @@ void StorageReplicatedMergeTree::backupData(
|
||||
|
||||
/// Send a list of mutations to the coordination too (we need to find the mutations which are not finished for added part names).
|
||||
{
|
||||
std::vector<IBackupCoordination::MutationInfo> mutation_infos;
|
||||
auto zookeeper = getZooKeeper();
|
||||
Strings mutation_ids = zookeeper->getChildren(fs::path(zookeeper_path) / "mutations");
|
||||
mutation_infos.reserve(mutation_ids.size());
|
||||
for (const auto & mutation_id : mutation_ids)
|
||||
Strings mutation_ids;
|
||||
if (zookeeper->tryGetChildren(fs::path(zookeeper_path) / "mutations", mutation_ids) == Coordination::Error::ZOK)
|
||||
{
|
||||
mutation_infos.emplace_back(
|
||||
IBackupCoordination::MutationInfo{mutation_id, zookeeper->get(fs::path(zookeeper_path) / "mutations" / mutation_id)});
|
||||
std::vector<IBackupCoordination::MutationInfo> mutation_infos;
|
||||
mutation_infos.reserve(mutation_ids.size());
|
||||
for (const auto & mutation_id : mutation_ids)
|
||||
{
|
||||
String mutation;
|
||||
if (zookeeper->tryGet(fs::path(zookeeper_path) / "mutations" / mutation_id, mutation))
|
||||
mutation_infos.emplace_back(IBackupCoordination::MutationInfo{mutation_id, mutation});
|
||||
}
|
||||
coordination->addReplicatedMutations(shared_id, getStorageID().getFullTableName(), getReplicaName(), mutation_infos);
|
||||
}
|
||||
coordination->addReplicatedMutations(shared_id, getStorageID().getFullTableName(), getReplicaName(), mutation_infos);
|
||||
}
|
||||
|
||||
/// This task will be executed after all replicas have collected their parts and the coordination is ready to
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <atomic>
|
||||
#include <pcg_random.hpp>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/MergeTree/AsyncBlockIDsCache.h>
|
||||
#include <Storages/IStorageCluster.h>
|
||||
#include <Storages/MergeTree/DataPartsExchange.h>
|
||||
#include <Storages/MergeTree/EphemeralLockInZooKeeper.h>
|
||||
@ -335,6 +336,7 @@ private:
|
||||
friend class ReplicatedMergeTreeSinkImpl;
|
||||
friend class ReplicatedMergeTreePartCheckThread;
|
||||
friend class ReplicatedMergeTreeCleanupThread;
|
||||
friend class AsyncBlockIDsCache;
|
||||
friend class ReplicatedMergeTreeAlterThread;
|
||||
friend class ReplicatedMergeTreeRestartingThread;
|
||||
friend class ReplicatedMergeTreeAttachThread;
|
||||
@ -443,6 +445,8 @@ private:
|
||||
/// A thread that removes old parts, log entries, and blocks.
|
||||
ReplicatedMergeTreeCleanupThread cleanup_thread;
|
||||
|
||||
AsyncBlockIDsCache async_block_ids_cache;
|
||||
|
||||
/// A thread that checks the data of the parts, as well as the queue of the parts to be checked.
|
||||
ReplicatedMergeTreePartCheckThread part_check_thread;
|
||||
|
||||
|
@ -46,13 +46,15 @@ StorageS3Cluster::StorageS3Cluster(
|
||||
const StorageID & table_id_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
ContextPtr context_)
|
||||
ContextPtr context_,
|
||||
bool structure_argument_was_provided_)
|
||||
: IStorageCluster(table_id_)
|
||||
, s3_configuration{configuration_.url, configuration_.auth_settings, configuration_.request_settings, configuration_.headers}
|
||||
, filename(configuration_.url)
|
||||
, cluster_name(configuration_.cluster_name)
|
||||
, format_name(configuration_.format)
|
||||
, compression_method(configuration_.compression_method)
|
||||
, structure_argument_was_provided(structure_argument_was_provided_)
|
||||
{
|
||||
context_->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI{filename});
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
@ -68,7 +70,6 @@ StorageS3Cluster::StorageS3Cluster(
|
||||
auto columns = StorageS3::getTableStructureFromDataImpl(format_name, s3_configuration, compression_method,
|
||||
/*distributed_processing_*/false, is_key_with_globs, /*format_settings=*/std::nullopt, context_);
|
||||
storage_metadata.setColumns(columns);
|
||||
add_columns_structure_to_query = true;
|
||||
}
|
||||
else
|
||||
storage_metadata.setColumns(columns_);
|
||||
@ -111,7 +112,7 @@ Pipe StorageS3Cluster::read(
|
||||
const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState;
|
||||
|
||||
ASTPtr query_to_send = interpreter.getQueryInfo().query->clone();
|
||||
if (add_columns_structure_to_query)
|
||||
if (!structure_argument_was_provided)
|
||||
addColumnsStructureToQueryWithClusterEngine(
|
||||
query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 5, getName());
|
||||
|
||||
|
@ -26,7 +26,8 @@ public:
|
||||
const StorageID & table_id_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
ContextPtr context_);
|
||||
ContextPtr context_,
|
||||
bool structure_argument_was_provided_);
|
||||
|
||||
std::string getName() const override { return "S3Cluster"; }
|
||||
|
||||
@ -49,7 +50,7 @@ private:
|
||||
String compression_method;
|
||||
NamesAndTypesList virtual_columns;
|
||||
Block virtual_block;
|
||||
bool add_columns_structure_to_query = false;
|
||||
bool structure_argument_was_provided;
|
||||
};
|
||||
|
||||
|
||||
|
@ -20,6 +20,37 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace
|
||||
{
|
||||
|
||||
StorageID getDictionaryID(const ExternalLoader::LoadResult & load_result, const std::shared_ptr<const IDictionary> & dict_ptr)
|
||||
{
|
||||
StorageID dict_id = StorageID::createEmpty();
|
||||
|
||||
if (dict_ptr)
|
||||
dict_id = dict_ptr->getDictionaryID();
|
||||
else if (load_result.config)
|
||||
dict_id = StorageID::fromDictionaryConfig(*load_result.config->config, load_result.config->key_in_config);
|
||||
else
|
||||
dict_id.table_name = load_result.name;
|
||||
|
||||
return dict_id;
|
||||
}
|
||||
|
||||
std::optional<DictionaryStructure>
|
||||
getDictionaryStructure(const ExternalLoader::LoadResult & load_result, std::exception_ptr & last_exception)
|
||||
try
|
||||
{
|
||||
return ExternalDictionariesLoader::getDictionaryStructure(*load_result.config);
|
||||
}
|
||||
catch (const DB::Exception &)
|
||||
{
|
||||
if (!last_exception)
|
||||
last_exception = std::current_exception();
|
||||
return {};
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes()
|
||||
{
|
||||
@ -47,7 +78,6 @@ NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes()
|
||||
{"loading_start_time", std::make_shared<DataTypeDateTime>()},
|
||||
{"last_successful_update_time", std::make_shared<DataTypeDateTime>()},
|
||||
{"loading_duration", std::make_shared<DataTypeFloat32>()},
|
||||
//{ "creation_time", std::make_shared<DataTypeDateTime>() },
|
||||
{"last_exception", std::make_shared<DataTypeString>()},
|
||||
{"comment", std::make_shared<DataTypeString>()}
|
||||
};
|
||||
@ -73,15 +103,11 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, ContextPt
|
||||
for (const auto & load_result : external_dictionaries.getLoadResults())
|
||||
{
|
||||
const auto dict_ptr = std::dynamic_pointer_cast<const IDictionary>(load_result.object);
|
||||
DictionaryStructure dictionary_structure = ExternalDictionariesLoader::getDictionaryStructure(*load_result.config);
|
||||
|
||||
StorageID dict_id = StorageID::createEmpty();
|
||||
if (dict_ptr)
|
||||
dict_id = dict_ptr->getDictionaryID();
|
||||
else if (load_result.config)
|
||||
dict_id = StorageID::fromDictionaryConfig(*load_result.config->config, load_result.config->key_in_config);
|
||||
else
|
||||
dict_id.table_name = load_result.name;
|
||||
std::exception_ptr last_exception = load_result.exception;
|
||||
auto dict_structure = getDictionaryStructure(load_result, last_exception);
|
||||
|
||||
StorageID dict_id = getDictionaryID(load_result, dict_ptr);
|
||||
|
||||
String db_or_tag = dict_id.database_name.empty() ? IDictionary::NO_DATABASE_TAG : dict_id.database_name;
|
||||
if (!access->isGranted(AccessType::SHOW_DICTIONARIES, db_or_tag, dict_id.table_name))
|
||||
@ -94,22 +120,28 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, ContextPt
|
||||
res_columns[i++]->insert(static_cast<Int8>(load_result.status));
|
||||
res_columns[i++]->insert(load_result.config ? load_result.config->path : "");
|
||||
|
||||
std::exception_ptr last_exception = load_result.exception;
|
||||
|
||||
if (dict_ptr)
|
||||
res_columns[i++]->insert(dict_ptr->getTypeName());
|
||||
else
|
||||
res_columns[i++]->insertDefault();
|
||||
|
||||
res_columns[i++]->insert(collections::map<Array>(dictionary_structure.getKeysNames(), [] (auto & name) { return name; }));
|
||||
if (dict_structure)
|
||||
{
|
||||
res_columns[i++]->insert(collections::map<Array>(dict_structure->getKeysNames(), [](auto & name) { return name; }));
|
||||
|
||||
if (dictionary_structure.id)
|
||||
res_columns[i++]->insert(Array({"UInt64"}));
|
||||
if (dict_structure->id)
|
||||
res_columns[i++]->insert(Array({"UInt64"}));
|
||||
else
|
||||
res_columns[i++]->insert(collections::map<Array>(*dict_structure->key, [](auto & attr) { return attr.type->getName(); }));
|
||||
|
||||
res_columns[i++]->insert(collections::map<Array>(dict_structure->attributes, [](auto & attr) { return attr.name; }));
|
||||
res_columns[i++]->insert(collections::map<Array>(dict_structure->attributes, [](auto & attr) { return attr.type->getName(); }));
|
||||
}
|
||||
else
|
||||
res_columns[i++]->insert(collections::map<Array>(*dictionary_structure.key, [] (auto & attr) { return attr.type->getName(); }));
|
||||
|
||||
res_columns[i++]->insert(collections::map<Array>(dictionary_structure.attributes, [] (auto & attr) { return attr.name; }));
|
||||
res_columns[i++]->insert(collections::map<Array>(dictionary_structure.attributes, [] (auto & attr) { return attr.type->getName(); }));
|
||||
{
|
||||
for (size_t j = 0; j != 4; ++j) // Number of empty fields if dict_structure is null
|
||||
res_columns[i++]->insertDefault();
|
||||
}
|
||||
|
||||
if (dict_ptr)
|
||||
{
|
||||
@ -157,9 +189,11 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, ContextPt
|
||||
|
||||
/// Start fill virtual columns
|
||||
|
||||
res_columns[i++]->insert(dictionary_structure.getKeyDescription());
|
||||
if (dict_structure)
|
||||
res_columns[i++]->insert(dict_structure->getKeyDescription());
|
||||
else
|
||||
res_columns[i++]->insertDefault();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
@ -96,8 +96,9 @@ StoragePtr TableFunctionS3Cluster::executeImpl(
|
||||
{
|
||||
StoragePtr storage;
|
||||
ColumnsDescription columns;
|
||||
bool structure_argument_was_provided = configuration.structure != "auto";
|
||||
|
||||
if (configuration.structure != "auto")
|
||||
if (structure_argument_was_provided)
|
||||
{
|
||||
columns = parseColumnsListFromString(configuration.structure, context);
|
||||
}
|
||||
@ -126,7 +127,8 @@ StoragePtr TableFunctionS3Cluster::executeImpl(
|
||||
StorageID(getDatabaseName(), table_name),
|
||||
columns,
|
||||
ConstraintsDescription{},
|
||||
context);
|
||||
context,
|
||||
structure_argument_was_provided);
|
||||
}
|
||||
|
||||
storage->startup();
|
||||
|
@ -3,5 +3,6 @@
|
||||
<database>system</database>
|
||||
<table>zookeeper_log</table>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
<ttl>event_date + INTERVAL 1 WEEK DELETE</ttl>
|
||||
</zookeeper_log>
|
||||
</clickhouse>
|
||||
|
@ -467,6 +467,7 @@ class ClickHouseCluster:
|
||||
|
||||
# available when with_kafka == True
|
||||
self.kafka_host = "kafka1"
|
||||
self.kafka_dir = os.path.join(self.instances_dir, "kafka")
|
||||
self.kafka_port = get_free_port()
|
||||
self.kafka_docker_id = None
|
||||
self.schema_registry_host = "schema-registry"
|
||||
@ -476,6 +477,7 @@ class ClickHouseCluster:
|
||||
self.coredns_host = "coredns"
|
||||
|
||||
# available when with_kerberozed_kafka == True
|
||||
# reuses kafka_dir
|
||||
self.kerberized_kafka_host = "kerberized_kafka1"
|
||||
self.kerberized_kafka_port = get_free_port()
|
||||
self.kerberized_kafka_docker_id = self.get_instance_docker_id(
|
||||
@ -2235,14 +2237,23 @@ class ClickHouseCluster:
|
||||
retries = 0
|
||||
while True:
|
||||
if check_kafka_is_available(kafka_docker_id, kafka_port):
|
||||
break
|
||||
return
|
||||
else:
|
||||
retries += 1
|
||||
if retries > max_retries:
|
||||
raise Exception("Kafka is not available")
|
||||
break
|
||||
logging.debug("Waiting for Kafka to start up")
|
||||
time.sleep(1)
|
||||
|
||||
try:
|
||||
with open(os.path.join(self.kafka_dir, "docker.log"), "w+") as f:
|
||||
subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL
|
||||
self.base_kafka_cmd + ["logs"], stdout=f
|
||||
)
|
||||
except Exception as e:
|
||||
logging.debug("Unable to get logs from docker.")
|
||||
raise Exception("Kafka is not available")
|
||||
|
||||
def wait_kerberos_kdc_is_available(self, kerberos_kdc_docker_id, max_retries=50):
|
||||
retries = 0
|
||||
while True:
|
||||
@ -2601,6 +2612,7 @@ class ClickHouseCluster:
|
||||
|
||||
if self.with_kafka and self.base_kafka_cmd:
|
||||
logging.debug("Setup Kafka")
|
||||
os.mkdir(self.kafka_dir)
|
||||
subprocess_check_call(
|
||||
self.base_kafka_cmd + common_opts + ["--renew-anon-volumes"]
|
||||
)
|
||||
@ -2610,6 +2622,7 @@ class ClickHouseCluster:
|
||||
|
||||
if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd:
|
||||
logging.debug("Setup kerberized kafka")
|
||||
os.mkdir(self.kafka_dir)
|
||||
run_and_check(
|
||||
self.base_kerberized_kafka_cmd
|
||||
+ common_opts
|
||||
|
@ -0,0 +1,15 @@
|
||||
<clickhouse>
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<backups>
|
||||
<type>local</type>
|
||||
<path>/backups/</path>
|
||||
</backups>
|
||||
</disks>
|
||||
</storage_configuration>
|
||||
<backups>
|
||||
<allowed_disk>backups</allowed_disk>
|
||||
</backups>
|
||||
<allow_concurrent_backups>false</allow_concurrent_backups>
|
||||
<allow_concurrent_restores>false</allow_concurrent_restores>
|
||||
</clickhouse>
|
@ -1,4 +1,4 @@
|
||||
from random import randint
|
||||
from random import random, randint
|
||||
import pytest
|
||||
import os.path
|
||||
import time
|
||||
@ -79,7 +79,7 @@ def create_and_fill_table():
|
||||
"CREATE TABLE tbl ON CLUSTER 'cluster' ("
|
||||
"x Int32"
|
||||
") ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}')"
|
||||
"ORDER BY x"
|
||||
"ORDER BY tuple()"
|
||||
)
|
||||
for i in range(num_nodes):
|
||||
nodes[i].query(f"INSERT INTO tbl VALUES ({i})")
|
||||
@ -285,3 +285,42 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine):
|
||||
node0.query(
|
||||
f"RESTORE DATABASE mydb ON CLUSTER 'cluster' FROM {backup_names[id]}"
|
||||
)
|
||||
|
||||
|
||||
def test_kill_mutation_during_backup():
|
||||
repeat_count = 1
|
||||
|
||||
for n in range(repeat_count):
|
||||
create_and_fill_table()
|
||||
|
||||
node0.query("ALTER TABLE tbl UPDATE x=x+1 WHERE 1")
|
||||
node0.query("ALTER TABLE tbl UPDATE x=x+1+sleep(3) WHERE 1")
|
||||
node0.query("ALTER TABLE tbl UPDATE x=x+1+sleep(3) WHERE 1")
|
||||
|
||||
backup_name = new_backup_name()
|
||||
|
||||
id = node0.query(
|
||||
f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name} ASYNC"
|
||||
).split("\t")[0]
|
||||
|
||||
time.sleep(random())
|
||||
node0.query(
|
||||
"KILL MUTATION WHERE database = 'default' AND table = 'tbl' AND mutation_id = '0000000001'"
|
||||
)
|
||||
|
||||
time.sleep(random())
|
||||
node0.query(
|
||||
"KILL MUTATION WHERE database = 'default' AND table = 'tbl' AND mutation_id = '0000000002'"
|
||||
)
|
||||
|
||||
assert_eq_with_retry(
|
||||
node0,
|
||||
f"SELECT status, error FROM system.backups WHERE id='{id}'",
|
||||
TSV([["BACKUP_CREATED", ""]]),
|
||||
)
|
||||
|
||||
node0.query(f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY")
|
||||
node0.query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}")
|
||||
|
||||
if n != repeat_count - 1:
|
||||
node0.query(f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY")
|
||||
|
@ -0,0 +1,227 @@
|
||||
from random import randint
|
||||
import pytest
|
||||
import os.path
|
||||
import time
|
||||
import concurrent
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import TSV, assert_eq_with_retry
|
||||
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
num_nodes = 10
|
||||
|
||||
|
||||
def generate_cluster_def():
|
||||
path = os.path.join(
|
||||
os.path.dirname(os.path.realpath(__file__)),
|
||||
"./_gen/cluster_for_concurrency_test.xml",
|
||||
)
|
||||
os.makedirs(os.path.dirname(path), exist_ok=True)
|
||||
with open(path, "w") as f:
|
||||
f.write(
|
||||
"""
|
||||
<clickhouse>
|
||||
<remote_servers>
|
||||
<cluster>
|
||||
<shard>
|
||||
"""
|
||||
)
|
||||
for i in range(num_nodes):
|
||||
f.write(
|
||||
"""
|
||||
<replica>
|
||||
<host>node"""
|
||||
+ str(i)
|
||||
+ """</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
"""
|
||||
)
|
||||
f.write(
|
||||
"""
|
||||
</shard>
|
||||
</cluster>
|
||||
</remote_servers>
|
||||
</clickhouse>
|
||||
"""
|
||||
)
|
||||
return path
|
||||
|
||||
|
||||
main_configs = ["configs/disallow_concurrency.xml", generate_cluster_def()]
|
||||
user_configs = ["configs/allow_database_types.xml"]
|
||||
|
||||
nodes = []
|
||||
for i in range(num_nodes):
|
||||
nodes.append(
|
||||
cluster.add_instance(
|
||||
f"node{i}",
|
||||
main_configs=main_configs,
|
||||
user_configs=user_configs,
|
||||
external_dirs=["/backups/"],
|
||||
macros={"replica": f"node{i}", "shard": "shard1"},
|
||||
with_zookeeper=True,
|
||||
)
|
||||
)
|
||||
|
||||
node0 = nodes[0]
|
||||
|
||||
|
||||
@pytest.fixture(scope="module", autouse=True)
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
@pytest.fixture(autouse=True)
|
||||
def drop_after_test():
|
||||
try:
|
||||
yield
|
||||
finally:
|
||||
node0.query("DROP TABLE IF EXISTS tbl ON CLUSTER 'cluster' NO DELAY")
|
||||
node0.query("DROP DATABASE IF EXISTS mydb ON CLUSTER 'cluster' NO DELAY")
|
||||
|
||||
|
||||
backup_id_counter = 0
|
||||
|
||||
|
||||
def new_backup_name():
|
||||
global backup_id_counter
|
||||
backup_id_counter += 1
|
||||
return f"Disk('backups', '{backup_id_counter}')"
|
||||
|
||||
|
||||
def create_and_fill_table():
|
||||
node0.query(
|
||||
"CREATE TABLE tbl ON CLUSTER 'cluster' ("
|
||||
"x UInt64"
|
||||
") ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}')"
|
||||
"ORDER BY x"
|
||||
)
|
||||
for i in range(num_nodes):
|
||||
nodes[i].query(f"INSERT INTO tbl SELECT number FROM numbers(40000000)")
|
||||
|
||||
|
||||
# All the tests have concurrent backup/restores with same backup names
|
||||
# The same works with different backup names too. Since concurrency
|
||||
# check comes before backup name check, separate tests are not added for different names
|
||||
|
||||
|
||||
def test_concurrent_backups_on_same_node():
|
||||
create_and_fill_table()
|
||||
|
||||
backup_name = new_backup_name()
|
||||
|
||||
id = (
|
||||
nodes[0]
|
||||
.query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name} ASYNC")
|
||||
.split("\t")[0]
|
||||
)
|
||||
assert_eq_with_retry(
|
||||
nodes[0],
|
||||
f"SELECT status FROM system.backups WHERE status == 'CREATING_BACKUP' AND id = '{id}'",
|
||||
"CREATING_BACKUP",
|
||||
)
|
||||
assert "Concurrent backups not supported" in nodes[0].query_and_get_error(
|
||||
f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}"
|
||||
)
|
||||
|
||||
assert_eq_with_retry(
|
||||
nodes[0],
|
||||
f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'",
|
||||
"BACKUP_CREATED",
|
||||
)
|
||||
|
||||
# This restore part is added to confirm creating an internal backup & restore work
|
||||
# even when a concurrent backup is stopped
|
||||
nodes[0].query(f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY")
|
||||
nodes[0].query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}")
|
||||
nodes[0].query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl")
|
||||
|
||||
|
||||
def test_concurrent_backups_on_different_nodes():
|
||||
create_and_fill_table()
|
||||
|
||||
backup_name = new_backup_name()
|
||||
|
||||
nodes[1].query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name} ASYNC")
|
||||
assert_eq_with_retry(
|
||||
nodes[1],
|
||||
f"SELECT status FROM system.backups WHERE status == 'CREATING_BACKUP'",
|
||||
"CREATING_BACKUP",
|
||||
)
|
||||
assert "Concurrent backups not supported" in nodes[2].query_and_get_error(
|
||||
f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}"
|
||||
)
|
||||
|
||||
|
||||
def test_concurrent_restores_on_same_node():
|
||||
create_and_fill_table()
|
||||
|
||||
backup_name = new_backup_name()
|
||||
|
||||
id = (
|
||||
nodes[0]
|
||||
.query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name} ASYNC")
|
||||
.split("\t")[0]
|
||||
)
|
||||
assert_eq_with_retry(
|
||||
nodes[0],
|
||||
f"SELECT status FROM system.backups WHERE status == 'CREATING_BACKUP' AND id = '{id}'",
|
||||
"CREATING_BACKUP",
|
||||
)
|
||||
|
||||
assert_eq_with_retry(
|
||||
nodes[0],
|
||||
f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'",
|
||||
"BACKUP_CREATED",
|
||||
)
|
||||
|
||||
nodes[0].query(f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY")
|
||||
nodes[0].query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name} ASYNC")
|
||||
assert_eq_with_retry(
|
||||
nodes[0],
|
||||
f"SELECT status FROM system.backups WHERE status == 'RESTORING'",
|
||||
"RESTORING",
|
||||
)
|
||||
assert "Concurrent restores not supported" in nodes[0].query_and_get_error(
|
||||
f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}"
|
||||
)
|
||||
|
||||
|
||||
def test_concurrent_restores_on_different_node():
|
||||
create_and_fill_table()
|
||||
|
||||
backup_name = new_backup_name()
|
||||
|
||||
id = (
|
||||
nodes[0]
|
||||
.query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name} ASYNC")
|
||||
.split("\t")[0]
|
||||
)
|
||||
assert_eq_with_retry(
|
||||
nodes[0],
|
||||
f"SELECT status FROM system.backups WHERE status == 'CREATING_BACKUP' AND id = '{id}'",
|
||||
"CREATING_BACKUP",
|
||||
)
|
||||
|
||||
assert_eq_with_retry(
|
||||
nodes[0],
|
||||
f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'",
|
||||
"BACKUP_CREATED",
|
||||
)
|
||||
|
||||
nodes[0].query(f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY")
|
||||
nodes[0].query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name} ASYNC")
|
||||
assert_eq_with_retry(
|
||||
nodes[0],
|
||||
f"SELECT status FROM system.backups WHERE status == 'RESTORING'",
|
||||
"RESTORING",
|
||||
)
|
||||
assert "Concurrent restores not supported" in nodes[1].query_and_get_error(
|
||||
f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}"
|
||||
)
|
@ -0,0 +1,28 @@
|
||||
<clickhouse>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
|
||||
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
|
||||
<size>1000M</size>
|
||||
<count>10</count>
|
||||
</logger>
|
||||
|
||||
<tcp_port>9000</tcp_port>
|
||||
<listen_host>127.0.0.1</listen_host>
|
||||
|
||||
<openSSL>
|
||||
<client>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
<verificationMode>none</verificationMode>
|
||||
<invalidCertificateHandler>
|
||||
<name>AcceptCertificateHandler</name>
|
||||
</invalidCertificateHandler>
|
||||
</client>
|
||||
</openSSL>
|
||||
|
||||
<max_concurrent_queries>500</max_concurrent_queries>
|
||||
<path>./clickhouse/</path>
|
||||
<users_config>users.xml</users_config>
|
||||
|
||||
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
|
||||
</clickhouse>
|
@ -0,0 +1,37 @@
|
||||
<clickhouse>
|
||||
<dictionary>
|
||||
<name>invalid_dict</name>
|
||||
|
||||
<source>
|
||||
<clickhouse>
|
||||
<host>localhost</host>
|
||||
<port>9000</port>
|
||||
<user>default</user>
|
||||
<password></password>
|
||||
<db>test</db>
|
||||
<table>source</table>
|
||||
</clickhouse>
|
||||
</source>
|
||||
|
||||
<lifetime>0</lifetime>
|
||||
|
||||
<layout>
|
||||
<cache><size_in_cells>128</size_in_cells></cache>
|
||||
</layout>
|
||||
|
||||
<structure>
|
||||
<id>
|
||||
<name>id</name>
|
||||
</id>
|
||||
|
||||
<attribute>
|
||||
<name>invalid_attr</name>
|
||||
<type>invalid_type</type>
|
||||
<null_value></null_value>
|
||||
<hierarchical>false</hierarchical>
|
||||
<injective>false</injective>
|
||||
</attribute>
|
||||
|
||||
</structure>
|
||||
</dictionary>
|
||||
</clickhouse>
|
@ -0,0 +1,37 @@
|
||||
<clickhouse>
|
||||
<dictionary>
|
||||
<name>valid_dict</name>
|
||||
|
||||
<source>
|
||||
<clickhouse>
|
||||
<host>localhost</host>
|
||||
<port>9000</port>
|
||||
<user>default</user>
|
||||
<password></password>
|
||||
<db>test</db>
|
||||
<table>source</table>
|
||||
</clickhouse>
|
||||
</source>
|
||||
|
||||
<lifetime>0</lifetime>
|
||||
|
||||
<layout>
|
||||
<cache><size_in_cells>128</size_in_cells></cache>
|
||||
</layout>
|
||||
|
||||
<structure>
|
||||
<id>
|
||||
<name>id</name>
|
||||
</id>
|
||||
|
||||
<attribute>
|
||||
<name>UInt64_attr</name>
|
||||
<type>UInt64</type>
|
||||
<null_value></null_value>
|
||||
<hierarchical>false</hierarchical>
|
||||
<injective>false</injective>
|
||||
</attribute>
|
||||
|
||||
</structure>
|
||||
</dictionary>
|
||||
</clickhouse>
|
@ -0,0 +1,22 @@
|
||||
<clickhouse>
|
||||
<profiles>
|
||||
<default>
|
||||
</default>
|
||||
</profiles>
|
||||
|
||||
<users>
|
||||
<default>
|
||||
<password></password>
|
||||
<networks incl="networks" replace="replace">
|
||||
<ip>::/0</ip>
|
||||
</networks>
|
||||
<profile>default</profile>
|
||||
<quota>default</quota>
|
||||
</default>
|
||||
</users>
|
||||
|
||||
<quotas>
|
||||
<default>
|
||||
</default>
|
||||
</quotas>
|
||||
</clickhouse>
|
@ -0,0 +1,58 @@
|
||||
import pytest
|
||||
|
||||
from helpers.client import QueryRuntimeException
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
DICTIONARY_FILES = [
|
||||
"configs/dictionaries/invalid_dict.xml",
|
||||
"configs/dictionaries/valid_dict.xml",
|
||||
]
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance("instance", dictionaries=DICTIONARY_FILES)
|
||||
|
||||
VALID_DICT_NAME = "valid_dict"
|
||||
INVALID_DICT_NAME = "invalid_dict"
|
||||
UNKNOWN_DATA_TYPE_EXCEPTION_STR = "DB::Exception: Unknown data type"
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def test_select_from_system_dictionaries_with_invalid_dictionary(started_cluster):
|
||||
query = instance.query
|
||||
|
||||
assert query("SELECT name FROM system.dictionaries;").splitlines() == [
|
||||
VALID_DICT_NAME,
|
||||
INVALID_DICT_NAME,
|
||||
]
|
||||
|
||||
assert (
|
||||
query(
|
||||
f"select last_exception from system.dictionaries WHERE name='{VALID_DICT_NAME}';"
|
||||
).strip()
|
||||
== ""
|
||||
)
|
||||
|
||||
assert (
|
||||
UNKNOWN_DATA_TYPE_EXCEPTION_STR
|
||||
in query(
|
||||
f"select last_exception from system.dictionaries WHERE name='{INVALID_DICT_NAME}';"
|
||||
).strip()
|
||||
)
|
||||
|
||||
|
||||
def test_dictGet_func_for_invalid_dictionary(started_cluster):
|
||||
query = instance.query
|
||||
|
||||
with pytest.raises(QueryRuntimeException) as exc:
|
||||
query(
|
||||
f"SELECT dictGetString('{INVALID_DICT_NAME}', 'invalid_attr', toInt64(1));"
|
||||
)
|
||||
assert UNKNOWN_DATA_TYPE_EXCEPTION_STR in str(exc.value)
|
@ -177,10 +177,7 @@ def get_table_uuid(node, db_atomic, table):
|
||||
return uuid
|
||||
|
||||
|
||||
@pytest.fixture(autouse=True)
|
||||
def drop_table(cluster):
|
||||
yield
|
||||
|
||||
node_names = ["node1z", "node2z", "node1n", "node2n", "node_another_bucket"]
|
||||
|
||||
for node_name in node_names:
|
||||
@ -257,3 +254,5 @@ def test_restore_another_bucket_path(cluster, db_atomic, zero_copy):
|
||||
assert node_another_bucket.query(
|
||||
"SELECT count(*) FROM s3.test FORMAT Values"
|
||||
) == "({})".format(size * (keys - dropped_keys))
|
||||
|
||||
drop_table(cluster)
|
||||
|
@ -0,0 +1,19 @@
|
||||
<clickhouse>
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<disk1>
|
||||
<path>/disk1/</path>
|
||||
</disk1>
|
||||
</disks>
|
||||
|
||||
<policies>
|
||||
<system_tables>
|
||||
<volumes>
|
||||
<main>
|
||||
<disk>disk1</disk>
|
||||
</main>
|
||||
</volumes>
|
||||
</system_tables>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
</clickhouse>
|
@ -6,7 +6,11 @@ import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node = cluster.add_instance("node_default", stay_alive=True)
|
||||
node = cluster.add_instance(
|
||||
"node_default",
|
||||
main_configs=["configs/config.d/storage_configuration.xml"],
|
||||
stay_alive=True,
|
||||
)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module", autouse=True)
|
||||
@ -76,6 +80,45 @@ def test_system_logs_recreate():
|
||||
== 2
|
||||
)
|
||||
|
||||
# apply only storage_policy for all system tables
|
||||
for table in system_logs:
|
||||
node.exec_in_container(
|
||||
[
|
||||
"bash",
|
||||
"-c",
|
||||
f"""echo "
|
||||
<clickhouse>
|
||||
<{table}>
|
||||
<storage_policy>system_tables</storage_policy>
|
||||
</{table}>
|
||||
</clickhouse>
|
||||
" > /etc/clickhouse-server/config.d/zzz-override-{table}.xml
|
||||
""",
|
||||
]
|
||||
)
|
||||
node.restart_clickhouse()
|
||||
node.query("SYSTEM FLUSH LOGS")
|
||||
import logging
|
||||
|
||||
for table in system_logs:
|
||||
create_table_sql = node.query(f"SHOW CREATE TABLE system.{table} FORMAT TSVRaw")
|
||||
logging.debug(
|
||||
"With storage policy, SHOW CREATE TABLE system.%s is: %s",
|
||||
table,
|
||||
create_table_sql,
|
||||
)
|
||||
assert "ENGINE = MergeTree" in create_table_sql
|
||||
assert "ENGINE = Null" not in create_table_sql
|
||||
assert "SETTINGS storage_policy = 'system_tables'" in create_table_sql
|
||||
assert (
|
||||
len(
|
||||
node.query(f"SHOW TABLES FROM system LIKE '{table}%'")
|
||||
.strip()
|
||||
.split("\n")
|
||||
)
|
||||
== 3
|
||||
)
|
||||
|
||||
for table in system_logs:
|
||||
node.exec_in_container(
|
||||
["rm", f"/etc/clickhouse-server/config.d/zzz-override-{table}.xml"]
|
||||
@ -92,7 +135,7 @@ def test_system_logs_recreate():
|
||||
.strip()
|
||||
.split("\n")
|
||||
)
|
||||
== 3
|
||||
== 4
|
||||
)
|
||||
|
||||
node.query("SYSTEM FLUSH LOGS")
|
||||
@ -105,7 +148,7 @@ def test_system_logs_recreate():
|
||||
.strip()
|
||||
.split("\n")
|
||||
)
|
||||
== 3
|
||||
== 4
|
||||
)
|
||||
|
||||
|
||||
|
@ -3,6 +3,7 @@ import time
|
||||
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import assert_eq_with_retry
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
@ -20,6 +21,12 @@ node2 = cluster.add_instance(
|
||||
macros={"shard": 0, "replica": 2},
|
||||
)
|
||||
|
||||
settings = {
|
||||
"mutations_sync": 2,
|
||||
"replication_alter_partitions_sync": 2,
|
||||
"optimize_throw_if_noop": 1,
|
||||
}
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
@ -45,38 +52,34 @@ def split_tsv(data):
|
||||
|
||||
@pytest.mark.parametrize("replicated", ["", "replicated"])
|
||||
def test_merge_simple(started_cluster, replicated):
|
||||
try:
|
||||
clickhouse_path = "/var/lib/clickhouse"
|
||||
db_name = "test"
|
||||
table_name = "merge_simple"
|
||||
name = db_name + "." + table_name
|
||||
table_path = "data/" + db_name + "/" + table_name
|
||||
nodes = [node1, node2] if replicated else [node1]
|
||||
engine = (
|
||||
"ReplicatedMergeTree('/clickhouse/test_merge_simple', '{replica}')"
|
||||
if replicated
|
||||
else "MergeTree()"
|
||||
)
|
||||
node_check = nodes[-1]
|
||||
starting_block = 0 if replicated else 1
|
||||
clickhouse_path = "/var/lib/clickhouse"
|
||||
db_name = "test"
|
||||
table_name = "merge_simple"
|
||||
name = db_name + "." + table_name
|
||||
table_path = "data/" + db_name + "/" + table_name
|
||||
nodes = [node1, node2] if replicated else [node1]
|
||||
engine = (
|
||||
"ReplicatedMergeTree('/clickhouse/test_merge_simple', '{replica}')"
|
||||
if replicated
|
||||
else "MergeTree()"
|
||||
)
|
||||
node_check = nodes[-1]
|
||||
starting_block = 0 if replicated else 1
|
||||
|
||||
try:
|
||||
for node in nodes:
|
||||
node.query(
|
||||
"""
|
||||
CREATE TABLE {name}
|
||||
(
|
||||
`a` Int64
|
||||
)
|
||||
ENGINE = {engine}
|
||||
ORDER BY sleep(2)
|
||||
""".format(
|
||||
engine=engine, name=name
|
||||
)
|
||||
f"create table {name} (a Int64) engine={engine} order by tuple()"
|
||||
)
|
||||
|
||||
node1.query("INSERT INTO {name} VALUES (1)".format(name=name))
|
||||
node1.query("INSERT INTO {name} VALUES (2)".format(name=name))
|
||||
node1.query("INSERT INTO {name} VALUES (3)".format(name=name))
|
||||
node1.query(f"INSERT INTO {name} VALUES (1)")
|
||||
node1.query(f"INSERT INTO {name} VALUES (2)")
|
||||
node1.query(f"INSERT INTO {name} VALUES (3)")
|
||||
|
||||
node1.query(
|
||||
f"alter table {name} add column b int materialized sleepEachRow(3)",
|
||||
settings=settings,
|
||||
)
|
||||
|
||||
parts = [
|
||||
"all_{}_{}_0".format(x, x)
|
||||
@ -84,15 +87,22 @@ def test_merge_simple(started_cluster, replicated):
|
||||
]
|
||||
result_part = "all_{}_{}_1".format(starting_block, starting_block + 2)
|
||||
|
||||
# OPTIMIZE will sleep for 3s * 3 (parts) = 9s
|
||||
def optimize():
|
||||
node1.query("OPTIMIZE TABLE {name}".format(name=name))
|
||||
node1.query("OPTIMIZE TABLE {name}".format(name=name), settings=settings)
|
||||
|
||||
wait = threading.Thread(target=time.sleep, args=(5,))
|
||||
wait.start()
|
||||
t = threading.Thread(target=optimize)
|
||||
t.start()
|
||||
|
||||
time.sleep(1)
|
||||
# Wait for OPTIMIZE to actually start
|
||||
assert_eq_with_retry(
|
||||
node1,
|
||||
f"select count() from system.merges where table='{table_name}'",
|
||||
"1\n",
|
||||
retry_count=30,
|
||||
sleep_time=0.1,
|
||||
)
|
||||
|
||||
assert (
|
||||
split_tsv(
|
||||
node_check.query(
|
||||
@ -124,17 +134,21 @@ def test_merge_simple(started_cluster, replicated):
|
||||
]
|
||||
)
|
||||
t.join()
|
||||
wait.join()
|
||||
|
||||
# It still can show a row with progress=1, because OPTIMIZE returns before the entry is removed from MergeList
|
||||
assert (
|
||||
node_check.query(
|
||||
"SELECT * FROM system.merges WHERE table = '{name}' and progress < 1".format(
|
||||
name=table_name
|
||||
)
|
||||
f"SELECT * FROM system.merges WHERE table = '{table_name}' and progress < 1"
|
||||
)
|
||||
== ""
|
||||
)
|
||||
|
||||
# It will eventually disappear
|
||||
assert_eq_with_retry(
|
||||
node_check,
|
||||
f"SELECT * FROM system.merges WHERE table = '{table_name}' and progress < 1",
|
||||
"\n",
|
||||
)
|
||||
finally:
|
||||
for node in nodes:
|
||||
node.query("DROP TABLE {name}".format(name=name))
|
||||
@ -142,55 +156,53 @@ def test_merge_simple(started_cluster, replicated):
|
||||
|
||||
@pytest.mark.parametrize("replicated", ["", "replicated"])
|
||||
def test_mutation_simple(started_cluster, replicated):
|
||||
clickhouse_path = "/var/lib/clickhouse"
|
||||
db_name = "test"
|
||||
table_name = "mutation_simple"
|
||||
name = db_name + "." + table_name
|
||||
table_path = "data/" + db_name + "/" + table_name
|
||||
nodes = [node1, node2] if replicated else [node1]
|
||||
engine = (
|
||||
"ReplicatedMergeTree('/clickhouse/test_mutation_simple', '{replica}')"
|
||||
if replicated
|
||||
else "MergeTree()"
|
||||
)
|
||||
node_check = nodes[-1]
|
||||
starting_block = 0 if replicated else 1
|
||||
|
||||
try:
|
||||
clickhouse_path = "/var/lib/clickhouse"
|
||||
db_name = "test"
|
||||
table_name = "mutation_simple"
|
||||
name = db_name + "." + table_name
|
||||
table_path = "data/" + db_name + "/" + table_name
|
||||
nodes = [node1, node2] if replicated else [node1]
|
||||
engine = (
|
||||
"ReplicatedMergeTree('/clickhouse/test_mutation_simple', '{replica}')"
|
||||
if replicated
|
||||
else "MergeTree()"
|
||||
)
|
||||
node_check = nodes[-1]
|
||||
starting_block = 0 if replicated else 1
|
||||
|
||||
for node in nodes:
|
||||
node.query(
|
||||
"""
|
||||
CREATE TABLE {name}
|
||||
(
|
||||
`a` Int64
|
||||
)
|
||||
ENGINE = {engine}
|
||||
ORDER BY tuple()
|
||||
""".format(
|
||||
engine=engine, name=name
|
||||
)
|
||||
f"create table {name} (a Int64) engine={engine} order by tuple()"
|
||||
)
|
||||
|
||||
node1.query("INSERT INTO {name} VALUES (1)".format(name=name))
|
||||
node1.query(f"INSERT INTO {name} VALUES (1), (2), (3)")
|
||||
|
||||
part = "all_{}_{}_0".format(starting_block, starting_block)
|
||||
result_part = "all_{}_{}_0_{}".format(
|
||||
starting_block, starting_block, starting_block + 1
|
||||
)
|
||||
|
||||
# ALTER will sleep for 3s * 3 (rows) = 9s
|
||||
def alter():
|
||||
node1.query(
|
||||
"ALTER TABLE {name} UPDATE a = 42 WHERE sleep(2) OR 1".format(
|
||||
name=name
|
||||
),
|
||||
settings={
|
||||
"mutations_sync": 1,
|
||||
},
|
||||
f"ALTER TABLE {name} UPDATE a = 42 WHERE sleep(3) OR 1",
|
||||
settings=settings,
|
||||
)
|
||||
|
||||
t = threading.Thread(target=alter)
|
||||
t.start()
|
||||
|
||||
time.sleep(1)
|
||||
# Wait for the mutation to actually start
|
||||
assert_eq_with_retry(
|
||||
node1,
|
||||
f"select count() from system.merges where table='{table_name}'",
|
||||
"1\n",
|
||||
retry_count=30,
|
||||
sleep_time=0.1,
|
||||
)
|
||||
|
||||
assert (
|
||||
split_tsv(
|
||||
node_check.query(
|
||||
@ -225,13 +237,18 @@ def test_mutation_simple(started_cluster, replicated):
|
||||
|
||||
assert (
|
||||
node_check.query(
|
||||
"SELECT * FROM system.merges WHERE table = '{name}' and progress < 1".format(
|
||||
name=table_name
|
||||
)
|
||||
f"SELECT * FROM system.merges WHERE table = '{table_name}' and progress < 1"
|
||||
)
|
||||
== ""
|
||||
)
|
||||
|
||||
# It will eventually disappear
|
||||
assert_eq_with_retry(
|
||||
node_check,
|
||||
f"SELECT * FROM system.merges WHERE table = '{table_name}' and progress < 1",
|
||||
"\n",
|
||||
)
|
||||
|
||||
finally:
|
||||
for node in nodes:
|
||||
node.query("DROP TABLE {name}".format(name=name))
|
||||
|
@ -56,7 +56,7 @@ QUERY id: 0
|
||||
JOIN TREE
|
||||
TABLE id: 5, table_name: system.one
|
||||
WHERE
|
||||
FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8
|
||||
FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: Bool
|
||||
ARGUMENTS
|
||||
LIST id: 7, nodes: 2
|
||||
FUNCTION id: 8, function_name: multiMatchAny, function_type: ordinary, result_type: UInt8
|
||||
@ -102,6 +102,7 @@ SETTINGS optimize_or_like_chain = 1
|
||||
Привет, optimized World
|
||||
Привет, World
|
||||
Привет, World
|
||||
Привет, World
|
||||
SELECT
|
||||
(materialize(\'Привет, World\') AS s) LIKE \'hell%\' AS test,
|
||||
s
|
||||
|
@ -22,6 +22,8 @@ SELECT materialize('Привет, optimized World') AS s WHERE (s LIKE 'hell%')
|
||||
SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s LIKE 'world%') SETTINGS optimize_or_like_chain = 0;
|
||||
SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s LIKE 'world%') SETTINGS optimize_or_like_chain = 0, allow_experimental_analyzer = 1;
|
||||
|
||||
SELECT materialize('Привет, World') AS s WHERE (s LIKE 'hell%') OR (s ILIKE '%привет%') OR (s ILIKE 'world%') SETTINGS optimize_or_like_chain = 1, allow_experimental_analyzer = 1;
|
||||
|
||||
-- Aliases
|
||||
|
||||
EXPLAIN SYNTAX SELECT test, materialize('Привет, World') AS s WHERE ((s LIKE 'hell%') AS test) OR (s ILIKE '%привет%') OR (s ILIKE 'world%') SETTINGS optimize_or_like_chain = 1;
|
||||
|
52
tests/queries/0_stateless/02346_full_text_search.reference
Normal file
52
tests/queries/0_stateless/02346_full_text_search.reference
Normal file
@ -0,0 +1,52 @@
|
||||
af inverted
|
||||
101 Alick a01
|
||||
1
|
||||
101 Alick a01
|
||||
111 Alick b01
|
||||
1
|
||||
103 Click a03
|
||||
108 Click a08
|
||||
113 Click b03
|
||||
118 Click b08
|
||||
1
|
||||
af inverted
|
||||
101 Alick a01
|
||||
106 Alick a06
|
||||
111 Alick b01
|
||||
116 Alick b06
|
||||
101 Alick a01
|
||||
106 Alick a06
|
||||
1
|
||||
101 Alick a01
|
||||
111 Alick b01
|
||||
1
|
||||
af inverted
|
||||
3 ['Click a03','Click b03']
|
||||
1
|
||||
af inverted
|
||||
103 {'Click':'Click a03'}
|
||||
108 {'Click':'Click a08'}
|
||||
113 {'Click':'Click b03'}
|
||||
118 {'Click':'Click b08'}
|
||||
1
|
||||
103 {'Click':'Click a03'}
|
||||
1
|
||||
af inverted
|
||||
101 Alick a01
|
||||
111 Alick b01
|
||||
201 rick c01
|
||||
1
|
||||
af inverted
|
||||
102 clickhouse你好
|
||||
1
|
||||
af inverted
|
||||
BC614E,05397FB1,6969696969898240,CF3304
|
||||
1
|
||||
af inverted
|
||||
1
|
||||
1
|
||||
af inverted
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
261
tests/queries/0_stateless/02346_full_text_search.sql
Normal file
261
tests/queries/0_stateless/02346_full_text_search.sql
Normal file
@ -0,0 +1,261 @@
|
||||
SET log_queries = 1;
|
||||
SET allow_experimental_inverted_index = 1;
|
||||
|
||||
-- create table for inverted(2)
|
||||
DROP TABLE IF EXISTS simple1;
|
||||
CREATE TABLE simple1(k UInt64,s String,INDEX af (s) TYPE inverted(2) GRANULARITY 1)
|
||||
ENGINE = MergeTree() ORDER BY k
|
||||
SETTINGS index_granularity = 2;
|
||||
-- insert test data into table
|
||||
INSERT INTO simple1 VALUES (101, 'Alick a01'), (102, 'Blick a02'), (103, 'Click a03'),(104, 'Dlick a04'),(105, 'Elick a05'),(106, 'Alick a06'),(107, 'Blick a07'),(108, 'Click a08'),(109, 'Dlick a09'),(110, 'Elick a10'),(111, 'Alick b01'),(112, 'Blick b02'),(113, 'Click b03'),(114, 'Dlick b04'),(115, 'Elick b05'),(116, 'Alick b06'),(117, 'Blick b07'),(118, 'Click b08'),(119, 'Dlick b09'),(120, 'Elick b10');
|
||||
-- check inverted index was created
|
||||
SELECT name, type FROM system.data_skipping_indices where (table =='simple1') limit 1;
|
||||
|
||||
-- search inverted index with ==
|
||||
SELECT * FROM simple1 WHERE s == 'Alick a01';
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query only read 1 granules (2 rows total; each granule has 2 rows)
|
||||
SELECT read_rows==2 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT * FROM simple1 WHERE s == \'Alick a01\';')
|
||||
and type='QueryFinish'
|
||||
and result_rows==1
|
||||
limit 1;
|
||||
|
||||
-- search inverted index with LIKE
|
||||
SELECT * FROM simple1 WHERE s LIKE '%01%' ORDER BY k;
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query only read 2 granules (4 rows total; each granule has 2 rows)
|
||||
SELECT read_rows==4 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT * FROM simple1 WHERE s LIKE \'%01%\' ORDER BY k;')
|
||||
and type='QueryFinish'
|
||||
and result_rows==2
|
||||
limit 1;
|
||||
|
||||
-- search inverted index with hasToken
|
||||
SELECT * FROM simple1 WHERE hasToken(s, 'Click') ORDER BY k;
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query only read 4 granules (8 rows total; each granule has 2 rows)
|
||||
SELECT read_rows==8 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT * FROM simple1 WHERE hasToken(s, \'Click\') ORDER BY k;')
|
||||
and type='QueryFinish'
|
||||
and result_rows==4 limit 1;
|
||||
|
||||
-- create table for inverted()
|
||||
DROP TABLE IF EXISTS simple2;
|
||||
CREATE TABLE simple2(k UInt64,s String,INDEX af (s) TYPE inverted() GRANULARITY 1)
|
||||
ENGINE = MergeTree() ORDER BY k
|
||||
SETTINGS index_granularity = 2;
|
||||
|
||||
-- insert test data into table
|
||||
INSERT INTO simple2 VALUES (101, 'Alick a01'), (102, 'Blick a02'), (103, 'Click a03'),(104, 'Dlick a04'),(105, 'Elick a05'),(106, 'Alick a06'),(107, 'Blick a07'),(108, 'Click a08'),(109, 'Dlick a09'),(110, 'Elick a10'),(111, 'Alick b01'),(112, 'Blick b02'),(113, 'Click b03'),(114, 'Dlick b04'),(115, 'Elick b05'),(116, 'Alick b06'),(117, 'Blick b07'),(118, 'Click b08'),(119, 'Dlick b09'),(120, 'Elick b10');
|
||||
|
||||
-- check inverted index was created
|
||||
SELECT name, type FROM system.data_skipping_indices where (table =='simple2') limit 1;
|
||||
|
||||
-- search inverted index with hasToken
|
||||
SELECT * FROM simple2 WHERE hasToken(s, 'Alick') order by k;
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query only read 4 granules (8 rows total; each granule has 2 rows)
|
||||
SELECT read_rows==8 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT * FROM simple2 WHERE hasToken(s, \'Alick\');')
|
||||
and type='QueryFinish'
|
||||
and result_rows==4 limit 1;
|
||||
|
||||
-- search inverted index with IN operator
|
||||
SELECT * FROM simple2 WHERE s IN ('Alick a01', 'Alick a06') ORDER BY k;
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query only read 2 granules (4 rows total; each granule has 2 rows)
|
||||
SELECT read_rows==4 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT * FROM simple2 WHERE s IN (\'Alick a01\', \'Alick a06\') ORDER BY k;')
|
||||
and type='QueryFinish'
|
||||
and result_rows==2 limit 1;
|
||||
|
||||
-- search inverted index with multiSearch
|
||||
SELECT * FROM simple2 WHERE multiSearchAny(s, ['a01', 'b01']) ORDER BY k;
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query only read 2 granules (4 rows total; each granule has 2 rows)
|
||||
SELECT read_rows==4 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT * FROM simple2 WHERE multiSearchAny(s, [\'a01\', \'b01\']) ORDER BY k;')
|
||||
and type='QueryFinish'
|
||||
and result_rows==2 limit 1;
|
||||
|
||||
-- create table with an array column
|
||||
DROP TABLE IF EXISTS simple_array;
|
||||
create table simple_array (k UInt64, s Array(String), INDEX af (s) TYPE inverted(2) GRANULARITY 1)
|
||||
ENGINE = MergeTree() ORDER BY k
|
||||
SETTINGS index_granularity = 2;
|
||||
INSERT INTO simple_array SELECT rowNumberInBlock(), groupArray(s) FROM simple2 GROUP BY k%10;
|
||||
-- check inverted index was created
|
||||
SELECT name, type FROM system.data_skipping_indices where (table =='simple_array') limit 1;
|
||||
-- search inverted index with has
|
||||
SELECT * FROM simple_array WHERE has(s, 'Click a03') ORDER BY k;
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query must read all 10 granules (20 rows total; each granule has 2 rows)
|
||||
SELECT read_rows==2 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT * FROM simple_array WHERE has(s, \'Click a03\') ORDER BY k;')
|
||||
and type='QueryFinish'
|
||||
and result_rows==1 limit 1;
|
||||
|
||||
-- create table with a map column
|
||||
DROP TABLE IF EXISTS simple_map;
|
||||
CREATE TABLE simple_map (k UInt64, s Map(String,String), INDEX af (mapKeys(s)) TYPE inverted(2) GRANULARITY 1)
|
||||
ENGINE = MergeTree() ORDER BY k
|
||||
SETTINGS index_granularity = 2;
|
||||
INSERT INTO simple_map VALUES (101, {'Alick':'Alick a01'}), (102, {'Blick':'Blick a02'}), (103, {'Click':'Click a03'}),(104, {'Dlick':'Dlick a04'}),(105, {'Elick':'Elick a05'}),(106, {'Alick':'Alick a06'}),(107, {'Blick':'Blick a07'}),(108, {'Click':'Click a08'}),(109, {'Dlick':'Dlick a09'}),(110, {'Elick':'Elick a10'}),(111, {'Alick':'Alick b01'}),(112, {'Blick':'Blick b02'}),(113, {'Click':'Click b03'}),(114, {'Dlick':'Dlick b04'}),(115, {'Elick':'Elick b05'}),(116, {'Alick':'Alick b06'}),(117, {'Blick':'Blick b07'}),(118, {'Click':'Click b08'}),(119, {'Dlick':'Dlick b09'}),(120, {'Elick':'Elick b10'});
|
||||
-- check inverted index was created
|
||||
SELECT name, type FROM system.data_skipping_indices where (table =='simple_map') limit 1;
|
||||
-- search inverted index with mapContains
|
||||
SELECT * FROM simple_map WHERE mapContains(s, 'Click') ORDER BY k;
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query must read all 4 granules (8 rows total; each granule has 2 rows)
|
||||
SELECT read_rows==8 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT * FROM simple_map WHERE mapContains(s, \'Click\') ORDER BY k;')
|
||||
and type='QueryFinish'
|
||||
and result_rows==4 limit 1;
|
||||
|
||||
-- search inverted index with map key
|
||||
SELECT * FROM simple_map WHERE s['Click'] = 'Click a03';
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query must read all 4 granules (8 rows total; each granule has 2 rows)
|
||||
SELECT read_rows==8 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT * FROM simple_map WHERE s[\'Click\'] = \'Click a03\';')
|
||||
and type='QueryFinish'
|
||||
and result_rows==1 limit 1;
|
||||
|
||||
-- create table for inverted(2) with two parts
|
||||
DROP TABLE IF EXISTS simple3;
|
||||
CREATE TABLE simple3(k UInt64,s String,INDEX af (s) TYPE inverted(2) GRANULARITY 1)
|
||||
ENGINE = MergeTree() ORDER BY k
|
||||
SETTINGS index_granularity = 2;
|
||||
-- insert test data into table
|
||||
INSERT INTO simple3 VALUES (101, 'Alick a01'), (102, 'Blick a02'), (103, 'Click a03'),(104, 'Dlick a04'),(105, 'Elick a05'),(106, 'Alick a06'),(107, 'Blick a07'),(108, 'Click a08'),(109, 'Dlick a09'),(110, 'Elick b10'),(111, 'Alick b01'),(112, 'Blick b02'),(113, 'Click b03'),(114, 'Dlick b04'),(115, 'Elick b05'),(116, 'Alick b06'),(117, 'Blick b07'),(118, 'Click b08'),(119, 'Dlick b09'),(120, 'Elick b10');
|
||||
INSERT INTO simple3 VALUES (201, 'rick c01'), (202, 'mick c02'),(203, 'nick c03');
|
||||
-- check inverted index was created
|
||||
SELECT name, type FROM system.data_skipping_indices where (table =='simple3') limit 1;
|
||||
-- search inverted index
|
||||
SELECT * FROM simple3 WHERE s LIKE '%01%' order by k;
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query only read 3 granules (6 rows total; each granule has 2 rows)
|
||||
SELECT read_rows==6 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT * FROM simple3 WHERE s LIKE \'%01%\' order by k;')
|
||||
and type='QueryFinish'
|
||||
and result_rows==3 limit 1;
|
||||
|
||||
-- create table for inverted(2) for utf8 string test
|
||||
DROP TABLE IF EXISTS simple4;
|
||||
CREATE TABLE simple4(k UInt64,s String,INDEX af (s) TYPE inverted(2) GRANULARITY 1) ENGINE = MergeTree() ORDER BY k
|
||||
SETTINGS index_granularity = 2;
|
||||
-- insert test data into table
|
||||
INSERT INTO simple4 VALUES (101, 'Alick 好'),(102, 'clickhouse你好'), (103, 'Click 你'),(104, 'Dlick 你a好'),(105, 'Elick 好好你你'),(106, 'Alick 好a好a你a你');
|
||||
-- check inverted index was created
|
||||
SELECT name, type FROM system.data_skipping_indices where (table =='simple4') limit 1;
|
||||
-- search inverted index
|
||||
SELECT * FROM simple4 WHERE s LIKE '%你好%' order by k;
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query only read 1 granule (2 rows total; each granule has 2 rows)
|
||||
SELECT read_rows==2 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT * FROM simple4 WHERE s LIKE \'%你好%\' order by k;')
|
||||
and type='QueryFinish'
|
||||
and result_rows==1 limit 1;
|
||||
|
||||
-- create table for max_digestion_size_per_segment test
|
||||
DROP TABLE IF EXISTS simple5;
|
||||
CREATE TABLE simple5(k UInt64,s String,INDEX af(s) TYPE inverted(0) GRANULARITY 1)
|
||||
Engine=MergeTree
|
||||
ORDER BY (k)
|
||||
SETTINGS max_digestion_size_per_segment = 1024, index_granularity = 256
|
||||
AS
|
||||
SELECT
|
||||
number,
|
||||
format('{},{},{},{}', hex(12345678), hex(87654321), hex(number/17 + 5), hex(13579012)) as s
|
||||
FROM numbers(10240);
|
||||
|
||||
-- check inverted index was created
|
||||
SELECT name, type FROM system.data_skipping_indices where (table =='simple5') limit 1;
|
||||
-- search inverted index
|
||||
SELECT s FROM simple5 WHERE hasToken(s, '6969696969898240');
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query only read 1 granule (1 row total; each granule has 256 rows)
|
||||
SELECT read_rows==256 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT s FROM simple5 WHERE hasToken(s, \'6969696969898240\');')
|
||||
and type='QueryFinish'
|
||||
and result_rows==1 limit 1;
|
||||
|
||||
DROP TABLE IF EXISTS simple6;
|
||||
-- create inverted index with density==1
|
||||
CREATE TABLE simple6(k UInt64,s String,INDEX af(s) TYPE inverted(0, 1.0) GRANULARITY 1)
|
||||
Engine=MergeTree
|
||||
ORDER BY (k)
|
||||
SETTINGS max_digestion_size_per_segment = 1, index_granularity = 512
|
||||
AS
|
||||
SELECT number, if(number%2, format('happy {}', hex(number)), format('birthday {}', hex(number)))
|
||||
FROM numbers(1024);
|
||||
-- check inverted index was created
|
||||
SELECT name, type FROM system.data_skipping_indices where (table =='simple6') limit 1;
|
||||
-- search inverted index, no row has 'happy birthday'
|
||||
SELECT count()==0 FROM simple6 WHERE s=='happy birthday';
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query only skip all granules (0 row total; each granule has 512 rows)
|
||||
SELECT read_rows==0 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT count()==0 FROM simple6 WHERE s==\'happy birthday\';')
|
||||
and type='QueryFinish'
|
||||
and result_rows==1 limit 1;
|
||||
|
||||
DROP TABLE IF EXISTS simple7;
|
||||
-- create inverted index with density==0.1
|
||||
CREATE TABLE simple7(k UInt64,s String,INDEX af(s) TYPE inverted(0, 0.1) GRANULARITY 1)
|
||||
Engine=MergeTree
|
||||
ORDER BY (k)
|
||||
SETTINGS max_digestion_size_per_segment = 1, index_granularity = 512
|
||||
AS
|
||||
SELECT number, if(number==1023, 'happy new year', if(number%2, format('happy {}', hex(number)), format('birthday {}', hex(number))))
|
||||
FROM numbers(1024);
|
||||
-- check inverted index was created
|
||||
SELECT name, type FROM system.data_skipping_indices where (table =='simple7') limit 1;
|
||||
-- search inverted index, no row has 'happy birthday'
|
||||
SELECT count()==0 FROM simple7 WHERE s=='happy birthday';
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query does not skip any of the 2 granules(1024 rows total; each granule has 512 rows)
|
||||
SELECT read_rows==1024 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT count()==0 FROM simple7 WHERE s==\'happy birthday\';')
|
||||
and type='QueryFinish'
|
||||
and result_rows==1 limit 1;
|
||||
-- search inverted index, no row has 'happy new year'
|
||||
SELECT count()==1 FROM simple7 WHERE s=='happy new year';
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- check the query only read 1 granule because of density (1024 rows total; each granule has 512 rows)
|
||||
SELECT read_rows==512 from system.query_log
|
||||
where query_kind ='Select'
|
||||
and current_database = currentDatabase()
|
||||
and endsWith(trimRight(query), 'SELECT count()==1 FROM simple7 WHERE s==\'happy new year\';')
|
||||
and type='QueryFinish'
|
||||
and result_rows==1 limit 1;
|
||||
|
@ -81,7 +81,7 @@ EventDate DateTime,
|
||||
KeyID UInt32
|
||||
) Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{database}/t_async_insert_dedup', '{replica}')
|
||||
PARTITION BY toYYYYMM(EventDate)
|
||||
ORDER BY (KeyID, EventDate)
|
||||
ORDER BY (KeyID, EventDate) SETTINGS use_async_block_ids_cache = 1
|
||||
''')
|
||||
|
||||
q = queue.Queue(100)
|
||||
@ -98,7 +98,7 @@ gen.join()
|
||||
|
||||
retry = 0
|
||||
|
||||
while (True):
|
||||
while True:
|
||||
time.sleep(5)
|
||||
result = client.query("select KeyID from t_async_insert_dedup order by KeyID")
|
||||
result = result.split()
|
||||
@ -124,6 +124,16 @@ while (True):
|
||||
else:
|
||||
print(len(result), flush=True)
|
||||
break
|
||||
|
||||
result = client.query("SELECT value FROM system.metrics where metric = 'AsyncInsertCacheSize'")
|
||||
result = int(result.split()[0])
|
||||
if result <= 0:
|
||||
raise Exception(f"AsyncInsertCacheSize should > 0, but got {result}")
|
||||
result = client.query("SELECT value FROM system.events where event = 'AsyncInsertCacheHits'")
|
||||
result = int(result.split()[0])
|
||||
if result <= 0:
|
||||
raise Exception(f"AsyncInsertCacheHits should > 0, but got {result}")
|
||||
|
||||
client.query("DROP TABLE IF EXISTS t_async_insert_dedup NO DELAY")
|
||||
|
||||
os._exit(os.EX_OK)
|
||||
|
@ -76,6 +76,9 @@ INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '33/tclwebkit', ['versi
|
||||
SYSTEM RELOAD dictionary regexp_dict1;
|
||||
select dictGet(regexp_dict1, ('name', 'version', 'comment'), '33/tclwebkit');
|
||||
|
||||
truncate table regexp_dictionary_source_table;
|
||||
SYSTEM RELOAD dictionary regexp_dict1; -- { serverError 489 }
|
||||
|
||||
|
||||
DROP TABLE IF EXISTS regexp_dictionary_source_table;
|
||||
DROP TABLE IF EXISTS needle_table;
|
||||
|
2
tests/queries/0_stateless/02517_uuid_parsing.reference
Normal file
2
tests/queries/0_stateless/02517_uuid_parsing.reference
Normal file
@ -0,0 +1,2 @@
|
||||
1011 1d83904a-c31d-4a6c-bbf0-217656b46444 1d83904a-c31d-4a6c-bbf0-217656b46444 -200 0 FOO BAR 1d83904a-c31d-4a6c-bbf0-217656b46444 2022-12-18 03:14:56 \N dispatcher
|
||||
10112222334444 1d83904a-c31d-4a6c-bbf0-217656b46444 1d83904a-c31d-4a6c-bbf0-217656b46444 12300 0 FOO BAR 1d83904a-c31d-4a6c-bbf0-217656b46444 2022-12-17 23:37:18 1d83904a-c31d-4a6c-bbf0-217656b46444 other
|
26
tests/queries/0_stateless/02517_uuid_parsing.sql
Normal file
26
tests/queries/0_stateless/02517_uuid_parsing.sql
Normal file
@ -0,0 +1,26 @@
|
||||
-- https://github.com/ClickHouse/ClickHouse/issues/44668
|
||||
|
||||
CREATE TABLE temp
|
||||
(
|
||||
`id` UInt64,
|
||||
`field1` UUID,
|
||||
`field2` UUID,
|
||||
`field3` Int64,
|
||||
`field4` Int64,
|
||||
`field5` LowCardinality(String),
|
||||
`field6` FixedString(3),
|
||||
`field7` String,
|
||||
`field8` Nullable(UUID),
|
||||
`event_at` DateTime('UTC'),
|
||||
`order_id` Nullable(UUID),
|
||||
`identity` LowCardinality(String)
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
PARTITION BY toYYYYMM(event_at)
|
||||
ORDER BY (field1, event_at, field2, field5, id)
|
||||
SETTINGS index_granularity = 8192;
|
||||
|
||||
INSERT INTO temp (id, field1, field2, field3, field4, field5, field6, field7, field8, event_at, order_id, identity)
|
||||
VALUES ('1011','1d83904a-c31d-4a6c-bbf0-217656b46444','1d83904a-c31d-4a6c-bbf0-217656b46444',-200,0,'FOO','BAR','','1d83904a-c31d-4a6c-bbf0-217656b46444','2022-12-18 03:14:56','','dispatcher'),('10112222334444','1d83904a-c31d-4a6c-bbf0-217656b46444','1d83904a-c31d-4a6c-bbf0-217656b46444',12300,0,'FOO','BAR','','1d83904a-c31d-4a6c-bbf0-217656b46444','2022-12-17 23:37:18','1d83904a-c31d-4a6c-bbf0-217656b46444','other');
|
||||
|
||||
SELECT * FROM temp ORDER BY id;
|
@ -0,0 +1,16 @@
|
||||
-- Tags: no-ordinary-database, use-rocksdb
|
||||
|
||||
DROP TABLE IF EXISTS 02526_keeper_map;
|
||||
DROP TABLE IF EXISTS 02526_rocksdb;
|
||||
|
||||
CREATE TABLE 02526_keeper_map (`key` String, `value` UInt32) ENGINE = KeeperMap('/' || currentDatabase() || '/02526_kv_filter_types') PRIMARY KEY key;
|
||||
INSERT INTO 02526_keeper_map SELECT * FROM generateRandom('`key` String, `value` UInt32') LIMIT 100;
|
||||
SELECT * FROM 02526_keeper_map WHERE key in (SELECT number * 5 FROM numbers(1000)) FORMAT Null;
|
||||
|
||||
DROP TABLE 02526_keeper_map;
|
||||
|
||||
CREATE TABLE 02526_rocksdb (`key` String, `value` UInt32) ENGINE = EmbeddedRocksDB PRIMARY KEY key;
|
||||
INSERT INTO 02526_rocksdb SELECT * FROM generateRandom('`key` String, `value` UInt32') LIMIT 100;
|
||||
SELECT * FROM 02526_rocksdb WHERE key in (SELECT number * 5 FROM numbers(1000)) FORMAT Null;
|
||||
|
||||
DROP TABLE 02526_rocksdb;
|
@ -0,0 +1,4 @@
|
||||
1 2 3
|
||||
4 5 6
|
||||
7 8 9
|
||||
0 0 0
|
@ -0,0 +1,9 @@
|
||||
-- Tags: no-fasttest
|
||||
-- Tag no-fasttest: Depends on AWS
|
||||
|
||||
drop table if exists test;
|
||||
create table test (x UInt32, y UInt32, z UInt32) engine=Memory();
|
||||
insert into test select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/a.tsv');
|
||||
select * from test;
|
||||
drop table test;
|
||||
|
@ -0,0 +1,19 @@
|
||||
Row 1:
|
||||
──────
|
||||
x.a: 1
|
||||
x.b: 2
|
||||
|
||||
Row 2:
|
||||
──────
|
||||
x.a: 3
|
||||
x.b: 4
|
||||
|
||||
Row 3:
|
||||
──────
|
||||
x.a: 5
|
||||
x.b: 6
|
||||
|
||||
Row 4:
|
||||
──────
|
||||
x.a: 7
|
||||
x.b: 8
|
20
tests/queries/0_stateless/02535_json_bson_each_row_curl.sh
Executable file
20
tests/queries/0_stateless/02535_json_bson_each_row_curl.sh
Executable file
@ -0,0 +1,20 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT -q $'create table test (`x.a` UInt32, `x.b` UInt32) engine=Memory'
|
||||
|
||||
echo '{"x" : {"a" : 1, "b" : 2}}' | ${CLICKHOUSE_CURL} --data-binary @- "${CLICKHOUSE_URL}&query=INSERT+INTO+test+FORMAT+JSONEachRow&input_format_import_nested_json=1&max_threads=10&input_format_parallel_parsing=0"
|
||||
|
||||
echo '{"x" : {"a" : 3, "b" : 4}}' | ${CLICKHOUSE_CURL} --data-binary @- "${CLICKHOUSE_URL}&query=INSERT+INTO+test+FORMAT+JSONEachRow&input_format_import_nested_json=1&max_threads=10&input_format_parallel_parsing=1"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q $'select 5 as `x.a`, 6 as `x.b` format BSONEachRow' | ${CLICKHOUSE_CURL} --data-binary @- "${CLICKHOUSE_URL}&query=INSERT+INTO+test+FORMAT+BSONEachRow&max_threads=10&input_format_parallel_parsing=0"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q $'select 7 as `x.a`, 8 as `x.b` format BSONEachRow' | ${CLICKHOUSE_CURL} --data-binary @- "${CLICKHOUSE_URL}&query=INSERT+INTO+test+FORMAT+BSONEachRow&max_threads=10&input_format_parallel_parsing=1"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "select * from test order by 1 format Vertical";
|
||||
$CLICKHOUSE_CLIENT -q "drop table test";
|
||||
|
Loading…
Reference in New Issue
Block a user