mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-19 14:11:58 +00:00
Merge branch 'master' into keeper-use-fallocate
This commit is contained in:
commit
dfaed11c83
@ -146,6 +146,12 @@ def prepare_for_hung_check(drop_databases):
|
||||
"KILL QUERY WHERE query LIKE 'SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u %'"
|
||||
)
|
||||
)
|
||||
# Long query from 02136_kill_scalar_queries
|
||||
call_with_retry(
|
||||
make_query_command(
|
||||
"KILL QUERY WHERE query LIKE 'SELECT (SELECT number FROM system.numbers WHERE number = 1000000000000)%'"
|
||||
)
|
||||
)
|
||||
|
||||
if drop_databases:
|
||||
for i in range(5):
|
||||
|
@ -21,6 +21,13 @@ ENGINE = HDFS(URI, format)
|
||||
`SELECT` queries, the format must be supported for input, and to perform
|
||||
`INSERT` queries – for output. The available formats are listed in the
|
||||
[Formats](../../../interfaces/formats.md#formats) section.
|
||||
- [PARTITION BY expr]
|
||||
|
||||
### PARTITION BY
|
||||
|
||||
`PARTITION BY` — Optional. In most cases you don't need a partition key, and if it is needed you generally don't need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression).
|
||||
|
||||
For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format.
|
||||
|
||||
**Example:**
|
||||
|
||||
|
@ -13,6 +13,7 @@ This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ec
|
||||
``` sql
|
||||
CREATE TABLE s3_engine_table (name String, value UInt32)
|
||||
ENGINE = S3(path, [aws_access_key_id, aws_secret_access_key,] format, [compression])
|
||||
[PARTITION BY expr]
|
||||
[SETTINGS ...]
|
||||
```
|
||||
|
||||
@ -23,6 +24,12 @@ CREATE TABLE s3_engine_table (name String, value UInt32)
|
||||
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3).
|
||||
- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension.
|
||||
|
||||
### PARTITION BY
|
||||
|
||||
`PARTITION BY` — Optional. In most cases you don't need a partition key, and if it is needed you generally don't need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression).
|
||||
|
||||
For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format.
|
||||
|
||||
**Example**
|
||||
|
||||
``` sql
|
||||
|
@ -77,7 +77,7 @@ Use the `ORDER BY tuple()` syntax, if you do not need sorting. See [Selecting th
|
||||
|
||||
#### PARTITION BY
|
||||
|
||||
`PARTITION BY` — The [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md). Optional. In most cases you don't need partition key, and in most other cases you don't need partition key more granular than by months. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead make client identifier or name the first column in the ORDER BY expression).
|
||||
`PARTITION BY` — The [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md). Optional. In most cases, you don't need a partition key, and if you do need to partition, generally you do not need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression).
|
||||
|
||||
For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format.
|
||||
|
||||
|
@ -86,3 +86,9 @@ $ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64
|
||||
- `SELECT ... SAMPLE`
|
||||
- Indices
|
||||
- Replication
|
||||
|
||||
## PARTITION BY
|
||||
|
||||
`PARTITION BY` — Optional. It is possible to create separate files by partitioning the data on a partition key. In most cases, you don't need a partition key, and if it is needed you generally don't need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression).
|
||||
|
||||
For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format.
|
||||
|
@ -96,3 +96,9 @@ SELECT * FROM url_engine_table
|
||||
- `ALTER` and `SELECT...SAMPLE` operations.
|
||||
- Indexes.
|
||||
- Replication.
|
||||
|
||||
## PARTITION BY
|
||||
|
||||
`PARTITION BY` — Optional. It is possible to create separate files by partitioning the data on a partition key. In most cases, you don't need a partition key, and if it is needed you generally don't need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression).
|
||||
|
||||
For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format.
|
||||
|
@ -276,14 +276,12 @@ EXPLAIN json = 1, description = 0, header = 1 SELECT 1, 2 + dummy;
|
||||
|
||||
With `indexes` = 1, the `Indexes` key is added. It contains an array of used indexes. Each index is described as JSON with `Type` key (a string `MinMax`, `Partition`, `PrimaryKey` or `Skip`) and optional keys:
|
||||
|
||||
- `Name` — An index name (for now, is used only for `Skip` index).
|
||||
- `Keys` — An array of columns used by the index.
|
||||
- `Condition` — A string with condition used.
|
||||
- `Description` — An index (for now, is used only for `Skip` index).
|
||||
- `Initial Parts` — A number of parts before the index is applied.
|
||||
- `Selected Parts` — A number of parts after the index is applied.
|
||||
- `Initial Granules` — A number of granules before the index is applied.
|
||||
- `Selected Granulesis` — A number of granules after the index is applied.
|
||||
- `Name` — The index name (currently only used for `Skip` indexes).
|
||||
- `Keys` — The array of columns used by the index.
|
||||
- `Condition` — The used condition.
|
||||
- `Description` — The index description (currently only used for `Skip` indexes).
|
||||
- `Parts` — The number of parts before/after the index is applied.
|
||||
- `Granules` — The number of granules before/after the index is applied.
|
||||
|
||||
Example:
|
||||
|
||||
@ -294,46 +292,36 @@ Example:
|
||||
"Type": "MinMax",
|
||||
"Keys": ["y"],
|
||||
"Condition": "(y in [1, +inf))",
|
||||
"Initial Parts": 5,
|
||||
"Selected Parts": 4,
|
||||
"Initial Granules": 12,
|
||||
"Selected Granules": 11
|
||||
"Parts": 5/4,
|
||||
"Granules": 12/11
|
||||
},
|
||||
{
|
||||
"Type": "Partition",
|
||||
"Keys": ["y", "bitAnd(z, 3)"],
|
||||
"Condition": "and((bitAnd(z, 3) not in [1, 1]), and((y in [1, +inf)), (bitAnd(z, 3) not in [1, 1])))",
|
||||
"Initial Parts": 4,
|
||||
"Selected Parts": 3,
|
||||
"Initial Granules": 11,
|
||||
"Selected Granules": 10
|
||||
"Parts": 4/3,
|
||||
"Granules": 11/10
|
||||
},
|
||||
{
|
||||
"Type": "PrimaryKey",
|
||||
"Keys": ["x", "y"],
|
||||
"Condition": "and((x in [11, +inf)), (y in [1, +inf)))",
|
||||
"Initial Parts": 3,
|
||||
"Selected Parts": 2,
|
||||
"Initial Granules": 10,
|
||||
"Selected Granules": 6
|
||||
"Parts": 3/2,
|
||||
"Granules": 10/6
|
||||
},
|
||||
{
|
||||
"Type": "Skip",
|
||||
"Name": "t_minmax",
|
||||
"Description": "minmax GRANULARITY 2",
|
||||
"Initial Parts": 2,
|
||||
"Selected Parts": 1,
|
||||
"Initial Granules": 6,
|
||||
"Selected Granules": 2
|
||||
"Parts": 2/1,
|
||||
"Granules": 6/2
|
||||
},
|
||||
{
|
||||
"Type": "Skip",
|
||||
"Name": "t_set",
|
||||
"Description": "set GRANULARITY 2",
|
||||
"Initial Parts": 1,
|
||||
"Selected Parts": 1,
|
||||
"Initial Granules": 2,
|
||||
"Selected Granules": 1
|
||||
"": 1/1,
|
||||
"Granules": 2/1
|
||||
}
|
||||
]
|
||||
```
|
||||
|
@ -248,10 +248,8 @@ EXPLAIN json = 1, description = 0, header = 1 SELECT 1, 2 + dummy;
|
||||
- `Keys` — массив столбцов, используемых индексом.
|
||||
- `Condition` — строка с используемым условием.
|
||||
- `Description` — индекс (на данный момент используется только для индекса `Skip`).
|
||||
- `Initial Parts` — количество кусков до применения индекса.
|
||||
- `Selected Parts` — количество кусков после применения индекса.
|
||||
- `Initial Granules` — количество гранул до применения индекса.
|
||||
- `Selected Granulesis` — количество гранул после применения индекса.
|
||||
- `Parts` — количество кусков до/после применения индекса.
|
||||
- `Granules` — количество гранул до/после применения индекса.
|
||||
|
||||
Пример:
|
||||
|
||||
@ -262,46 +260,36 @@ EXPLAIN json = 1, description = 0, header = 1 SELECT 1, 2 + dummy;
|
||||
"Type": "MinMax",
|
||||
"Keys": ["y"],
|
||||
"Condition": "(y in [1, +inf))",
|
||||
"Initial Parts": 5,
|
||||
"Selected Parts": 4,
|
||||
"Initial Granules": 12,
|
||||
"Selected Granules": 11
|
||||
"Parts": 5/4,
|
||||
"Granules": 12/11
|
||||
},
|
||||
{
|
||||
"Type": "Partition",
|
||||
"Keys": ["y", "bitAnd(z, 3)"],
|
||||
"Condition": "and((bitAnd(z, 3) not in [1, 1]), and((y in [1, +inf)), (bitAnd(z, 3) not in [1, 1])))",
|
||||
"Initial Parts": 4,
|
||||
"Selected Parts": 3,
|
||||
"Initial Granules": 11,
|
||||
"Selected Granules": 10
|
||||
"Parts": 4/3,
|
||||
"Granules": 11/10
|
||||
},
|
||||
{
|
||||
"Type": "PrimaryKey",
|
||||
"Keys": ["x", "y"],
|
||||
"Condition": "and((x in [11, +inf)), (y in [1, +inf)))",
|
||||
"Initial Parts": 3,
|
||||
"Selected Parts": 2,
|
||||
"Initial Granules": 10,
|
||||
"Selected Granules": 6
|
||||
"Parts": 3/2,
|
||||
"Granules": 10/6
|
||||
},
|
||||
{
|
||||
"Type": "Skip",
|
||||
"Name": "t_minmax",
|
||||
"Description": "minmax GRANULARITY 2",
|
||||
"Initial Parts": 2,
|
||||
"Selected Parts": 1,
|
||||
"Initial Granules": 6,
|
||||
"Selected Granules": 2
|
||||
"Parts": 2/1,
|
||||
"Granules": 6/2
|
||||
},
|
||||
{
|
||||
"Type": "Skip",
|
||||
"Name": "t_set",
|
||||
"Description": "set GRANULARITY 2",
|
||||
"Initial Parts": 1,
|
||||
"Selected Parts": 1,
|
||||
"Initial Granules": 2,
|
||||
"Selected Granules": 1
|
||||
"": 1/1,
|
||||
"Granules": 2/1
|
||||
}
|
||||
]
|
||||
```
|
||||
|
@ -8,7 +8,7 @@
|
||||
#include <IO/ReadBufferFromS3.h>
|
||||
#include <IO/WriteBufferFromS3.h>
|
||||
#include <IO/HTTPHeaderEntries.h>
|
||||
#include <IO/S3/copyDataToS3.h>
|
||||
#include <IO/S3/copyS3File.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
#include <aws/core/auth/AWSCredentials.h>
|
||||
@ -167,16 +167,16 @@ void BackupWriterS3::copyFileNative(DiskPtr src_disk, const String & src_file_na
|
||||
auto object_storage = src_disk->getObjectStorage();
|
||||
std::string src_bucket = object_storage->getObjectsNamespace();
|
||||
auto file_path = fs::path(s3_uri.key) / dest_file_name;
|
||||
copyFileS3ToS3(client, src_bucket, objects[0].absolute_path, src_offset, src_size, s3_uri.bucket, file_path, request_settings, {},
|
||||
threadPoolCallbackRunner<void>(IOThreadPool::get(), "BackupWriterS3"));
|
||||
copyS3File(client, src_bucket, objects[0].absolute_path, src_offset, src_size, s3_uri.bucket, file_path, request_settings, {},
|
||||
threadPoolCallbackRunner<void>(IOThreadPool::get(), "BackupWriterS3"));
|
||||
}
|
||||
}
|
||||
|
||||
void BackupWriterS3::copyDataToFile(
|
||||
const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name)
|
||||
{
|
||||
copyDataToS3(create_read_buffer, offset, size, client, s3_uri.bucket, fs::path(s3_uri.key) / dest_file_name, request_settings, {},
|
||||
threadPoolCallbackRunner<void>(IOThreadPool::get(), "BackupWriterS3"));
|
||||
copyDataToS3File(create_read_buffer, offset, size, client, s3_uri.bucket, fs::path(s3_uri.key) / dest_file_name, request_settings, {},
|
||||
threadPoolCallbackRunner<void>(IOThreadPool::get(), "BackupWriterS3"));
|
||||
}
|
||||
|
||||
BackupWriterS3::~BackupWriterS3() = default;
|
||||
|
@ -98,9 +98,15 @@ void CancelableSharedMutex::lock_shared()
|
||||
bool CancelableSharedMutex::try_lock_shared()
|
||||
{
|
||||
UInt64 value = state.load();
|
||||
if (!(value & writers) && state.compare_exchange_strong(value, value + 1)) // overflow is not realistic
|
||||
return true;
|
||||
return false;
|
||||
while (true)
|
||||
{
|
||||
if (value & writers)
|
||||
return false;
|
||||
if (state.compare_exchange_strong(value, value + 1)) // overflow is not realistic
|
||||
break;
|
||||
// Concurrent try_lock_shared() should not fail, so we have to retry CAS, but avoid blocking wait
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
void CancelableSharedMutex::unlock_shared()
|
||||
|
@ -12,6 +12,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
@ -20,7 +21,12 @@ namespace ErrorCodes
|
||||
namespace FST
|
||||
{
|
||||
|
||||
UInt64 Arc::serialize(WriteBuffer& write_buffer) const
|
||||
Arc::Arc(Output output_, const StatePtr & target_)
|
||||
: output(output_)
|
||||
, target(target_)
|
||||
{}
|
||||
|
||||
UInt64 Arc::serialize(WriteBuffer & write_buffer) const
|
||||
{
|
||||
UInt64 written_bytes = 0;
|
||||
bool has_output = output != 0;
|
||||
@ -55,6 +61,14 @@ void LabelsAsBitmap::addLabel(char label)
|
||||
data |= bit_label;
|
||||
}
|
||||
|
||||
bool LabelsAsBitmap::hasLabel(char label) const
|
||||
{
|
||||
UInt8 index = label;
|
||||
UInt256 bit_label = 1;
|
||||
bit_label <<= index;
|
||||
return ((data & bit_label) != 0);
|
||||
}
|
||||
|
||||
UInt64 LabelsAsBitmap::getIndex(char label) const
|
||||
{
|
||||
UInt64 bit_count = 0;
|
||||
@ -78,7 +92,7 @@ UInt64 LabelsAsBitmap::getIndex(char label) const
|
||||
return bit_count;
|
||||
}
|
||||
|
||||
UInt64 LabelsAsBitmap::serialize(WriteBuffer& write_buffer)
|
||||
UInt64 LabelsAsBitmap::serialize(WriteBuffer & write_buffer)
|
||||
{
|
||||
writeVarUInt(data.items[0], write_buffer);
|
||||
writeVarUInt(data.items[1], write_buffer);
|
||||
@ -91,19 +105,28 @@ UInt64 LabelsAsBitmap::serialize(WriteBuffer& write_buffer)
|
||||
+ getLengthOfVarUInt(data.items[3]);
|
||||
}
|
||||
|
||||
bool LabelsAsBitmap::hasLabel(char label) const
|
||||
UInt64 State::hash() const
|
||||
{
|
||||
UInt8 index = label;
|
||||
UInt256 bit_label = 1;
|
||||
bit_label <<= index;
|
||||
std::vector<char> values;
|
||||
values.reserve(arcs.size() * (sizeof(Output) + sizeof(UInt64) + 1));
|
||||
|
||||
return ((data & bit_label) != 0);
|
||||
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());
|
||||
}
|
||||
|
||||
Arc* State::getArc(char label) const
|
||||
Arc * State::getArc(char label) const
|
||||
{
|
||||
auto it = arcs.find(label);
|
||||
if (it == arcs.cend())
|
||||
if (it == arcs.end())
|
||||
return nullptr;
|
||||
|
||||
return const_cast<Arc *>(&it->second);
|
||||
@ -118,46 +141,11 @@ void State::clear()
|
||||
{
|
||||
id = 0;
|
||||
state_index = 0;
|
||||
flag = 0;
|
||||
|
||||
arcs.clear();
|
||||
flag = 0;
|
||||
}
|
||||
|
||||
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 State::serialize(WriteBuffer & write_buffer)
|
||||
{
|
||||
UInt64 written_bytes = 0;
|
||||
|
||||
@ -171,10 +159,8 @@ UInt64 State::serialize(WriteBuffer& write_buffer)
|
||||
std::vector<char> labels;
|
||||
labels.reserve(arcs.size());
|
||||
|
||||
for (auto& [label, state] : arcs)
|
||||
{
|
||||
for (auto & [label, state] : arcs)
|
||||
labels.push_back(label);
|
||||
}
|
||||
|
||||
UInt8 label_size = labels.size();
|
||||
write_buffer.write(label_size);
|
||||
@ -186,7 +172,7 @@ UInt64 State::serialize(WriteBuffer& write_buffer)
|
||||
/// Serialize all arcs
|
||||
for (char label : labels)
|
||||
{
|
||||
Arc* arc = getArc(label);
|
||||
Arc * arc = getArc(label);
|
||||
assert(arc != nullptr);
|
||||
written_bytes += arc->serialize(write_buffer);
|
||||
}
|
||||
@ -196,15 +182,13 @@ UInt64 State::serialize(WriteBuffer& write_buffer)
|
||||
/// 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);
|
||||
Arc * arc = getArc(label);
|
||||
assert(arc != nullptr);
|
||||
written_bytes += arc->serialize(write_buffer);
|
||||
}
|
||||
@ -213,16 +197,36 @@ UInt64 State::serialize(WriteBuffer& write_buffer)
|
||||
return written_bytes;
|
||||
}
|
||||
|
||||
FSTBuilder::FSTBuilder(WriteBuffer& write_buffer_) : write_buffer(write_buffer_)
|
||||
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.end())
|
||||
return false;
|
||||
|
||||
if (it->second != arc)
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
void State::readFlag(ReadBuffer & read_buffer)
|
||||
{
|
||||
read_buffer.readStrict(reinterpret_cast<char &>(flag));
|
||||
}
|
||||
|
||||
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)
|
||||
StatePtr FstBuilder::findMinimized(const State & state, bool & found)
|
||||
{
|
||||
found = false;
|
||||
auto hash = state.hash();
|
||||
@ -230,7 +234,7 @@ StatePtr FSTBuilder::findMinimized(const State & state, bool & found)
|
||||
/// MEMBER: in the paper pseudo code l15
|
||||
auto it = minimized_states.find(hash);
|
||||
|
||||
if (it != minimized_states.cend() && *it->second == state)
|
||||
if (it != minimized_states.end() && *it->second == state)
|
||||
{
|
||||
found = true;
|
||||
return it->second;
|
||||
@ -244,8 +248,11 @@ StatePtr FSTBuilder::findMinimized(const State & state, bool & found)
|
||||
return p;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// See the paper pseudo code l33-34.
|
||||
size_t FSTBuilder::getCommonPrefixLength(const String & word1, const String & word2)
|
||||
size_t getCommonPrefixLength(std::string_view word1, std::string_view word2)
|
||||
{
|
||||
size_t i = 0;
|
||||
while (i < word1.size() && i < word2.size() && word1[i] == word2[i])
|
||||
@ -253,8 +260,10 @@ size_t FSTBuilder::getCommonPrefixLength(const String & word1, const String & wo
|
||||
return i;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/// See the paper pseudo code l33-39 and l70-72(when down_to is 0).
|
||||
void FSTBuilder::minimizePreviousWordSuffix(Int64 down_to)
|
||||
void FstBuilder::minimizePreviousWordSuffix(Int64 down_to)
|
||||
{
|
||||
for (Int64 i = static_cast<Int64>(previous_word.size()); i >= down_to; --i)
|
||||
{
|
||||
@ -264,7 +273,7 @@ void FSTBuilder::minimizePreviousWordSuffix(Int64 down_to)
|
||||
if (i != 0)
|
||||
{
|
||||
Output output = 0;
|
||||
Arc* arc = temp_states[i - 1]->getArc(previous_word[i - 1]);
|
||||
Arc * arc = temp_states[i - 1]->getArc(previous_word[i - 1]);
|
||||
if (arc)
|
||||
output = arc->output;
|
||||
|
||||
@ -287,7 +296,7 @@ void FSTBuilder::minimizePreviousWordSuffix(Int64 down_to)
|
||||
}
|
||||
}
|
||||
|
||||
void FSTBuilder::add(const std::string & current_word, Output current_output)
|
||||
void FstBuilder::add(std::string_view 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
|
||||
@ -295,10 +304,10 @@ void FSTBuilder::add(const std::string & current_word, Output current_output)
|
||||
/// 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();
|
||||
size_t 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);
|
||||
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot build inverted index: The maximum term length is {}, this is exceeded by term {}", MAX_TERM_LENGTH, current_word_len);
|
||||
|
||||
size_t prefix_length_plus1 = getCommonPrefixLength(current_word, previous_word) + 1;
|
||||
|
||||
@ -333,9 +342,7 @@ void FSTBuilder::add(const std::string & current_word, Output current_output)
|
||||
if (word_suffix != 0)
|
||||
{
|
||||
for (auto & [label, arc] : temp_states[i]->arcs)
|
||||
{
|
||||
arc.output += word_suffix;
|
||||
}
|
||||
}
|
||||
/// Reduce current_output
|
||||
current_output -= common_prefix;
|
||||
@ -350,7 +357,7 @@ void FSTBuilder::add(const std::string & current_word, Output current_output)
|
||||
previous_word = current_word;
|
||||
}
|
||||
|
||||
UInt64 FSTBuilder::build()
|
||||
UInt64 FstBuilder::build()
|
||||
{
|
||||
minimizePreviousWordSuffix(0);
|
||||
|
||||
@ -364,7 +371,8 @@ UInt64 FSTBuilder::build()
|
||||
return previous_state_index + previous_written_bytes + length + 1;
|
||||
}
|
||||
|
||||
FiniteStateTransducer::FiniteStateTransducer(std::vector<UInt8> data_) : data(std::move(data_))
|
||||
FiniteStateTransducer::FiniteStateTransducer(std::vector<UInt8> data_)
|
||||
: data(std::move(data_))
|
||||
{
|
||||
}
|
||||
|
||||
@ -373,28 +381,28 @@ void FiniteStateTransducer::clear()
|
||||
data.clear();
|
||||
}
|
||||
|
||||
std::pair<UInt64, bool> FiniteStateTransducer::getOutput(const String & term)
|
||||
std::pair<UInt64, bool> FiniteStateTransducer::getOutput(std::string_view term)
|
||||
{
|
||||
std::pair<UInt64, bool> result{ 0, false };
|
||||
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);
|
||||
read_buffer.seek(data.size() - 1, SEEK_SET);
|
||||
|
||||
UInt8 length{ 0 };
|
||||
read_buffer.readStrict(reinterpret_cast<char&>(length));
|
||||
UInt8 length = 0;
|
||||
read_buffer.readStrict(reinterpret_cast<char &>(length));
|
||||
|
||||
/// FST contains no terms
|
||||
if (length == 0)
|
||||
return { 0, false };
|
||||
return {0, false};
|
||||
|
||||
read_buffer.seek(data.size() - 1 - length, SEEK_SET);
|
||||
UInt64 state_index{ 0 };
|
||||
UInt64 state_index = 0;
|
||||
readVarUInt(state_index, read_buffer);
|
||||
|
||||
for (size_t i = 0; i <= term.size(); ++i)
|
||||
{
|
||||
UInt64 arc_output{ 0 };
|
||||
UInt64 arc_output = 0;
|
||||
|
||||
/// Read flag
|
||||
State temp_state;
|
||||
@ -411,22 +419,22 @@ std::pair<UInt64, bool> FiniteStateTransducer::getOutput(const String & term)
|
||||
if (temp_state.getEncodingMethod() == State::EncodingMethod::Sequential)
|
||||
{
|
||||
/// Read number of labels
|
||||
UInt8 label_num{ 0 };
|
||||
read_buffer.readStrict(reinterpret_cast<char&>(label_num));
|
||||
UInt8 label_num = 0;
|
||||
read_buffer.readStrict(reinterpret_cast<char &>(label_num));
|
||||
|
||||
if (label_num == 0)
|
||||
return { 0, false };
|
||||
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 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 };
|
||||
return {0, false};
|
||||
|
||||
/// Read the arc for the label
|
||||
UInt64 arc_index = (pos - begin_it);
|
||||
@ -439,9 +447,7 @@ std::pair<UInt64, bool> FiniteStateTransducer::getOutput(const String & term)
|
||||
arc_output = 0;
|
||||
readVarUInt(state_index, read_buffer);
|
||||
if (state_index & 0x1) // output is followed
|
||||
{
|
||||
readVarUInt(arc_output, read_buffer);
|
||||
}
|
||||
state_index >>= 1;
|
||||
}
|
||||
}
|
||||
@ -455,7 +461,7 @@ std::pair<UInt64, bool> FiniteStateTransducer::getOutput(const String & term)
|
||||
readVarUInt(bmp.data.items[3], read_buffer);
|
||||
|
||||
if (!bmp.hasLabel(label))
|
||||
return { 0, false };
|
||||
return {0, false};
|
||||
|
||||
/// Read the arc for the label
|
||||
size_t arc_index = bmp.getIndex(label);
|
||||
@ -465,9 +471,7 @@ std::pair<UInt64, bool> FiniteStateTransducer::getOutput(const String & term)
|
||||
arc_output = 0;
|
||||
readVarUInt(state_index, read_buffer);
|
||||
if (state_index & 0x1) // output is followed
|
||||
{
|
||||
readVarUInt(arc_output, read_buffer);
|
||||
}
|
||||
state_index >>= 1;
|
||||
}
|
||||
}
|
||||
@ -476,5 +480,7 @@ std::pair<UInt64, bool> FiniteStateTransducer::getOutput(const String & term)
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,18 +19,18 @@ namespace DB
|
||||
/// [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
|
||||
/// 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_} { }
|
||||
Arc(Output output_, const StatePtr & target_);
|
||||
|
||||
/// 0 means the arc has no output
|
||||
Output output = 0;
|
||||
@ -53,13 +53,15 @@ public:
|
||||
/// computes the rank
|
||||
UInt64 getIndex(char label) const;
|
||||
|
||||
UInt64 serialize(WriteBuffer& write_buffer);
|
||||
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 };
|
||||
UInt256 data = 0;
|
||||
|
||||
friend class State;
|
||||
friend class FiniteStateTransducer;
|
||||
};
|
||||
|
||||
/// State implements the State in Finite State Transducer
|
||||
@ -77,9 +79,9 @@ public:
|
||||
/// Note this is NOT enabled for now since it is experimental
|
||||
Bitmap,
|
||||
};
|
||||
State() = default;
|
||||
|
||||
State(const State & state) = default;
|
||||
State() = default;
|
||||
State(const State & State) = default;
|
||||
|
||||
UInt64 hash() const;
|
||||
|
||||
@ -91,22 +93,12 @@ public:
|
||||
|
||||
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));
|
||||
}
|
||||
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);
|
||||
|
||||
/// Transient ID of the state which is used for building FST. It won't be serialized
|
||||
UInt64 id = 0;
|
||||
@ -116,6 +108,7 @@ public:
|
||||
|
||||
/// Arcs which are started from state, the 'char' is the label on the arc
|
||||
std::unordered_map<char, Arc> arcs;
|
||||
|
||||
private:
|
||||
struct FlagValues
|
||||
{
|
||||
@ -132,22 +125,21 @@ private:
|
||||
|
||||
bool operator==(const State & state1, const State & state2);
|
||||
|
||||
inline constexpr size_t MAX_TERM_LENGTH = 256;
|
||||
static constexpr size_t MAX_TERM_LENGTH = 256;
|
||||
|
||||
/// FSTBuilder is used to build Finite State Transducer by adding words incrementally.
|
||||
/// 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
|
||||
/// In the end, the caller should call build() to serialize minimized FST to WriteBuffer.
|
||||
class FstBuilder
|
||||
{
|
||||
public:
|
||||
explicit FSTBuilder(WriteBuffer & write_buffer_);
|
||||
explicit FstBuilder(WriteBuffer & write_buffer_);
|
||||
|
||||
void add(const std::string & word, Output output);
|
||||
void add(std::string_view 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;
|
||||
@ -171,8 +163,8 @@ class FiniteStateTransducer
|
||||
public:
|
||||
FiniteStateTransducer() = default;
|
||||
explicit FiniteStateTransducer(std::vector<UInt8> data_);
|
||||
std::pair<UInt64, bool> getOutput(const String & term);
|
||||
void clear();
|
||||
std::pair<UInt64, bool> getOutput(std::string_view term);
|
||||
std::vector<UInt8> & getData() { return data; }
|
||||
|
||||
private:
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <shared_mutex>
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
|
||||
@ -12,7 +12,7 @@ class ProfilingScopedWriteRWLock
|
||||
{
|
||||
public:
|
||||
|
||||
ProfilingScopedWriteRWLock(std::shared_mutex & rwl_, ProfileEvents::Event event) :
|
||||
ProfilingScopedWriteRWLock(SharedMutex & rwl_, ProfileEvents::Event event) :
|
||||
scoped_write_lock(rwl_)
|
||||
{
|
||||
ProfileEvents::increment(event, watch.elapsed());
|
||||
@ -20,14 +20,14 @@ public:
|
||||
|
||||
private:
|
||||
Stopwatch watch;
|
||||
std::unique_lock<std::shared_mutex> scoped_write_lock;
|
||||
std::unique_lock<SharedMutex> scoped_write_lock;
|
||||
};
|
||||
|
||||
|
||||
class ProfilingScopedReadRWLock
|
||||
{
|
||||
public:
|
||||
ProfilingScopedReadRWLock(std::shared_mutex & rwl, ProfileEvents::Event event) :
|
||||
ProfilingScopedReadRWLock(SharedMutex & rwl, ProfileEvents::Event event) :
|
||||
scoped_read_lock(rwl)
|
||||
{
|
||||
ProfileEvents::increment(event, watch.elapsed());
|
||||
@ -35,7 +35,7 @@ public:
|
||||
|
||||
private:
|
||||
Stopwatch watch;
|
||||
std::shared_lock<std::shared_mutex> scoped_read_lock;
|
||||
std::shared_lock<SharedMutex> scoped_read_lock;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -37,9 +37,7 @@ void SharedMutex::lock()
|
||||
bool SharedMutex::try_lock()
|
||||
{
|
||||
UInt64 value = 0;
|
||||
if (state.compare_exchange_strong(value, writers))
|
||||
return true;
|
||||
return false;
|
||||
return state.compare_exchange_strong(value, writers);
|
||||
}
|
||||
|
||||
void SharedMutex::unlock()
|
||||
@ -68,9 +66,15 @@ void SharedMutex::lock_shared()
|
||||
bool SharedMutex::try_lock_shared()
|
||||
{
|
||||
UInt64 value = state.load();
|
||||
if (!(value & writers) && state.compare_exchange_strong(value, value + 1))
|
||||
return true;
|
||||
return false;
|
||||
while (true)
|
||||
{
|
||||
if (value & writers)
|
||||
return false;
|
||||
if (state.compare_exchange_strong(value, value + 1))
|
||||
break;
|
||||
// Concurrent try_lock_shared() should not fail, so we have to retry CAS, but avoid blocking wait
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
void SharedMutex::unlock_shared()
|
||||
|
@ -13,7 +13,6 @@
|
||||
#include <map>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <shared_mutex>
|
||||
#include <unordered_set>
|
||||
|
||||
|
||||
|
@ -7,7 +7,7 @@
|
||||
|
||||
TEST(FST, SimpleTest)
|
||||
{
|
||||
std::vector<std::pair<std::string, DB::FST::Output>> indexed_data
|
||||
std::vector<std::pair<String, DB::FST::Output>> indexed_data
|
||||
{
|
||||
{"mop", 100},
|
||||
{"moth", 91},
|
||||
@ -17,7 +17,7 @@ TEST(FST, SimpleTest)
|
||||
{"top", 55},
|
||||
};
|
||||
|
||||
std::vector<std::pair<std::string, DB::FST::Output>> not_indexed_data
|
||||
std::vector<std::pair<String, DB::FST::Output>> not_indexed_data
|
||||
{
|
||||
{"mo", 100},
|
||||
{"moth1", 91},
|
||||
@ -29,42 +29,40 @@ TEST(FST, SimpleTest)
|
||||
|
||||
std::vector<UInt8> buffer;
|
||||
DB::WriteBufferFromVector<std::vector<UInt8>> wbuf(buffer);
|
||||
DB::FST::FSTBuilder builder(wbuf);
|
||||
DB::FST::FstBuilder builder(wbuf);
|
||||
|
||||
for (auto& [term, output] : indexed_data)
|
||||
{
|
||||
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)
|
||||
for (auto & [term, output] : indexed_data)
|
||||
{
|
||||
auto [result, found] = fst.getOutput(term);
|
||||
ASSERT_EQ(found, true);
|
||||
ASSERT_TRUE(found);
|
||||
ASSERT_EQ(result, output);
|
||||
}
|
||||
|
||||
for (auto& [term, output] : not_indexed_data)
|
||||
for (auto & [term, output] : not_indexed_data)
|
||||
{
|
||||
auto [result, found] = fst.getOutput(term);
|
||||
ASSERT_EQ(found, false);
|
||||
ASSERT_FALSE(found);
|
||||
}
|
||||
}
|
||||
|
||||
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');
|
||||
String term1(DB::FST::MAX_TERM_LENGTH - 1, 'A');
|
||||
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);
|
||||
DB::FST::FstBuilder builder(wbuf);
|
||||
|
||||
builder.add(term1, output1);
|
||||
builder.add(term2, output2);
|
||||
@ -75,20 +73,20 @@ TEST(FST, TestForLongTerms)
|
||||
DB::FST::FiniteStateTransducer fst(buffer);
|
||||
|
||||
auto [result1, found1] = fst.getOutput(term1);
|
||||
ASSERT_EQ(found1, true);
|
||||
ASSERT_TRUE(found1);
|
||||
ASSERT_EQ(result1, output1);
|
||||
|
||||
auto [result2, found2] = fst.getOutput(term2);
|
||||
ASSERT_EQ(found2, true);
|
||||
ASSERT_TRUE(found2);
|
||||
ASSERT_EQ(result2, output2);
|
||||
|
||||
/// Test exception case when term length exceeds limitation
|
||||
std::string term3(DB::FST::MAX_TERM_LENGTH + 1, 'C');
|
||||
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);
|
||||
DB::FST::FstBuilder builder3(wbuf3);
|
||||
|
||||
EXPECT_THROW(builder3.add(term3, output3), DB::Exception);
|
||||
}
|
||||
|
@ -27,7 +27,7 @@ namespace DB
|
||||
struct NoCancel {};
|
||||
|
||||
// for all PerfTests
|
||||
static constexpr int requests = 512 * 1024;
|
||||
static constexpr int requests = 128 * 1024;
|
||||
static constexpr int max_threads = 16;
|
||||
|
||||
template <class T, class Status = NoCancel>
|
||||
@ -91,6 +91,49 @@ void TestSharedMutex()
|
||||
|
||||
ASSERT_EQ(test, writers);
|
||||
}
|
||||
|
||||
// Test multiple readers can acquire lock simultaneously using try_shared_lock
|
||||
for (int readers = 1; readers <= 128; readers *= 2)
|
||||
{
|
||||
T sm;
|
||||
std::atomic<int> test(0);
|
||||
std::barrier sync(readers + 1);
|
||||
|
||||
std::vector<std::thread> threads;
|
||||
threads.reserve(readers);
|
||||
auto reader = [&]
|
||||
{
|
||||
[[maybe_unused]] Status status;
|
||||
bool acquired = sm.try_lock_shared();
|
||||
ASSERT_TRUE(acquired);
|
||||
if (!acquired) return; // Just to make TSA happy
|
||||
sync.arrive_and_wait(); // (A) sync with writer
|
||||
test++;
|
||||
sync.arrive_and_wait(); // (B) wait for writer to call try_lock() while shared_lock is held
|
||||
sm.unlock_shared();
|
||||
sync.arrive_and_wait(); // (C) wait for writer to release lock, to ensure try_lock_shared() will see no writer
|
||||
};
|
||||
|
||||
for (int i = 0; i < readers; i++)
|
||||
threads.emplace_back(reader);
|
||||
|
||||
{ // writer
|
||||
[[maybe_unused]] Status status;
|
||||
sync.arrive_and_wait(); // (A) wait for all reader to acquire lock to avoid blocking them
|
||||
ASSERT_FALSE(sm.try_lock());
|
||||
sync.arrive_and_wait(); // (B) sync with readers
|
||||
{
|
||||
std::unique_lock lock(sm);
|
||||
test++;
|
||||
}
|
||||
sync.arrive_and_wait(); // (C) sync with readers
|
||||
}
|
||||
|
||||
for (auto & thread : threads)
|
||||
thread.join();
|
||||
|
||||
ASSERT_EQ(test, readers + 1);
|
||||
}
|
||||
}
|
||||
|
||||
template <class T, class Status = NoCancel>
|
||||
|
@ -528,8 +528,7 @@ class IColumn;
|
||||
M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \
|
||||
M(Bool, enable_global_with_statement, true, "Propagate WITH statements to UNION queries and all subqueries", 0) \
|
||||
M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) \
|
||||
M(Bool, optimize_syntax_fuse_functions, false, "Not ready for production, do not use. Allow apply syntax optimisation: fuse aggregate functions", 0) \
|
||||
M(Bool, optimize_fuse_sum_count_avg, false, "Replace calls of functions `sum`, `avg`, `count` with identical arguments into one `sumCount`", 0) \
|
||||
M(Bool, optimize_syntax_fuse_functions, false, "Allow apply fuse aggregating function. Available only with `allow_experimental_analyzer`", 0) \
|
||||
M(Bool, flatten_nested, true, "If true, columns of type Nested will be flatten to separate array columns instead of one array of tuples", 0) \
|
||||
M(Bool, asterisk_include_materialized_columns, false, "Include MATERIALIZED columns for wildcard query", 0) \
|
||||
M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \
|
||||
@ -727,6 +726,8 @@ class IColumn;
|
||||
MAKE_OBSOLETE(M, UInt64, max_pipeline_depth, 0) \
|
||||
MAKE_OBSOLETE(M, Seconds, temporary_live_view_timeout, 1) \
|
||||
MAKE_OBSOLETE(M, Milliseconds, async_insert_cleanup_timeout_ms, 1000) \
|
||||
MAKE_OBSOLETE(M, Bool, optimize_fuse_sum_count_avg, 0) \
|
||||
|
||||
|
||||
/** The section above is for obsolete settings. Do not add anything there. */
|
||||
|
||||
|
@ -379,14 +379,18 @@ void DatabaseOnDisk::renameTable(
|
||||
if (dictionary && table && !table->isDictionary())
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY, "Use RENAME/EXCHANGE TABLE (instead of RENAME/EXCHANGE DICTIONARY) for tables");
|
||||
|
||||
/// We have to lock the table before detaching, because otherwise lockExclusively will throw. But the table may not exist.
|
||||
bool need_lock = table != nullptr;
|
||||
if (need_lock)
|
||||
table_lock = table->lockExclusively(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout);
|
||||
|
||||
detachTable(local_context, table_name);
|
||||
if (!need_lock)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table was detached without locking, it's a bug");
|
||||
|
||||
UUID prev_uuid = UUIDHelpers::Nil;
|
||||
try
|
||||
{
|
||||
table_lock = table->lockExclusively(
|
||||
local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout);
|
||||
|
||||
table_metadata_path = getObjectMetadataPath(table_name);
|
||||
attach_query = parseQueryFromMetadata(log, local_context, table_metadata_path);
|
||||
auto & create = attach_query->as<ASTCreateQuery &>();
|
||||
|
@ -236,6 +236,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n
|
||||
backQuote(database_name), backQuote(table_name));
|
||||
res = it->second;
|
||||
tables.erase(it);
|
||||
res->is_detached = true;
|
||||
|
||||
auto table_id = res->getStorageID();
|
||||
if (table_id.hasUUID())
|
||||
@ -272,6 +273,10 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c
|
||||
DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid);
|
||||
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", table_id.getFullTableName());
|
||||
}
|
||||
|
||||
/// It is important to reset is_detached here since in case of RENAME in
|
||||
/// non-Atomic database the is_detached is set to true before RENAME.
|
||||
table->is_detached = false;
|
||||
}
|
||||
|
||||
void DatabaseWithOwnTablesBase::shutdown()
|
||||
|
@ -448,7 +448,7 @@ void DatabaseMySQL::detachTablePermanently(ContextPtr, const String & table_name
|
||||
remove_or_detach_tables.erase(table_name);
|
||||
throw;
|
||||
}
|
||||
table_iter->second.second->is_dropped = true;
|
||||
table_iter->second.second->is_detached = true;
|
||||
}
|
||||
|
||||
void DatabaseMySQL::dropTable(ContextPtr local_context, const String & table_name, bool /*sync*/)
|
||||
|
@ -14,6 +14,7 @@ namespace ErrorCodes
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
static std::mutex init_sqlite_db_mutex;
|
||||
|
||||
void processSQLiteError(const String & message, bool throw_on_error)
|
||||
{
|
||||
@ -52,7 +53,11 @@ SQLitePtr openSQLiteDB(const String & path, ContextPtr context, bool throw_on_er
|
||||
LOG_DEBUG(&Poco::Logger::get("SQLite"), "SQLite database path {} does not exist, will create an empty SQLite database", database_path);
|
||||
|
||||
sqlite3 * tmp_sqlite_db = nullptr;
|
||||
int status = sqlite3_open(database_path.c_str(), &tmp_sqlite_db);
|
||||
int status;
|
||||
{
|
||||
std::lock_guard lock(init_sqlite_db_mutex);
|
||||
status = sqlite3_open(database_path.c_str(), &tmp_sqlite_db);
|
||||
}
|
||||
|
||||
if (status != SQLITE_OK)
|
||||
{
|
||||
|
@ -4,7 +4,6 @@
|
||||
#include <chrono>
|
||||
#include <cmath>
|
||||
#include <mutex>
|
||||
#include <shared_mutex>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
@ -14,6 +13,7 @@
|
||||
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
|
||||
#include <Dictionaries/IDictionary.h>
|
||||
@ -206,7 +206,7 @@ private:
|
||||
/// This lock is used for the inner cache state update function lock it for
|
||||
/// write, when it need to update cache state all other functions just
|
||||
/// readers. Surprisingly this lock is also used for last_exception pointer.
|
||||
mutable std::shared_mutex rw_lock;
|
||||
mutable SharedMutex rw_lock;
|
||||
|
||||
mutable std::exception_ptr last_exception;
|
||||
mutable std::atomic<size_t> error_count {0};
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <atomic>
|
||||
#include <mutex>
|
||||
#include <shared_mutex>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
#include <functional>
|
||||
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
13
src/Dictionaries/RangeHashedDictionaryComplex.cpp
Normal file
13
src/Dictionaries/RangeHashedDictionaryComplex.cpp
Normal file
@ -0,0 +1,13 @@
|
||||
#include <Dictionaries/RangeHashedDictionary.h>
|
||||
|
||||
/// RangeHashedDictionary is instantiated from two files
|
||||
/// RangeHashedDictionarySimple.cpp and RangeHashedDictionaryComplex.cpp
|
||||
/// to better parallelize the build procedure and avoid MSan build failure
|
||||
/// due to excessive resource consumption.
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template class RangeHashedDictionary<DictionaryKeyType::Complex>;
|
||||
|
||||
}
|
13
src/Dictionaries/RangeHashedDictionarySimple.cpp
Normal file
13
src/Dictionaries/RangeHashedDictionarySimple.cpp
Normal file
@ -0,0 +1,13 @@
|
||||
#include <Dictionaries/RangeHashedDictionary.h>
|
||||
|
||||
/// RangeHashedDictionary is instantiated from two files
|
||||
/// RangeHashedDictionarySimple.cpp and RangeHashedDictionaryComplex.cpp
|
||||
/// to better parallelize the build procedure and avoid MSan build failure
|
||||
/// due to excessive resource consumption.
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template class RangeHashedDictionary<DictionaryKeyType::Simple>;
|
||||
|
||||
}
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Disks/ObjectStorages/IMetadataStorage.h>
|
||||
#include <Disks/ObjectStorages/MetadataFromDiskTransactionState.h>
|
||||
@ -15,7 +16,7 @@ class FakeMetadataStorageFromDisk final : public IMetadataStorage
|
||||
private:
|
||||
friend class FakeMetadataStorageFromDiskTransaction;
|
||||
|
||||
mutable std::shared_mutex metadata_mutex;
|
||||
mutable SharedMutex metadata_mutex;
|
||||
|
||||
DiskPtr disk;
|
||||
ObjectStoragePtr object_storage;
|
||||
|
@ -83,7 +83,7 @@ std::string MetadataStorageFromDisk::readInlineDataToString(const std::string &
|
||||
return readMetadata(path)->getInlineData();
|
||||
}
|
||||
|
||||
DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const std::string & path, std::shared_lock<std::shared_mutex> &) const
|
||||
DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const std::string & path, std::shared_lock<SharedMutex> &) const
|
||||
{
|
||||
auto metadata = std::make_unique<DiskObjectStorageMetadata>(disk->getPath(), object_storage_root_path, path);
|
||||
auto str = readFileToString(path);
|
||||
@ -91,7 +91,7 @@ DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const
|
||||
return metadata;
|
||||
}
|
||||
|
||||
DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const std::string & path, std::unique_lock<std::shared_mutex> &) const
|
||||
DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const std::string & path, std::unique_lock<SharedMutex> &) const
|
||||
{
|
||||
auto metadata = std::make_unique<DiskObjectStorageMetadata>(disk->getPath(), object_storage_root_path, path);
|
||||
auto str = readFileToString(path);
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <Disks/ObjectStorages/IMetadataStorage.h>
|
||||
|
||||
#include <Disks/IDisk.h>
|
||||
@ -17,7 +18,7 @@ class MetadataStorageFromDisk final : public IMetadataStorage
|
||||
private:
|
||||
friend class MetadataStorageFromDiskTransaction;
|
||||
|
||||
mutable std::shared_mutex metadata_mutex;
|
||||
mutable SharedMutex metadata_mutex;
|
||||
|
||||
DiskPtr disk;
|
||||
std::string object_storage_root_path;
|
||||
@ -67,8 +68,8 @@ public:
|
||||
|
||||
DiskObjectStorageMetadataPtr readMetadata(const std::string & path) const;
|
||||
|
||||
DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::unique_lock<std::shared_mutex> & lock) const;
|
||||
DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::shared_lock<std::shared_mutex> & lock) const;
|
||||
DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::unique_lock<SharedMutex> & lock) const;
|
||||
DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::shared_lock<SharedMutex> & lock) const;
|
||||
};
|
||||
|
||||
class MetadataStorageFromDiskTransaction final : public IMetadataTransaction
|
||||
|
@ -26,7 +26,7 @@ SetLastModifiedOperation::SetLastModifiedOperation(const std::string & path_, Po
|
||||
{
|
||||
}
|
||||
|
||||
void SetLastModifiedOperation::execute(std::unique_lock<std::shared_mutex> &)
|
||||
void SetLastModifiedOperation::execute(std::unique_lock<SharedMutex> &)
|
||||
{
|
||||
old_timestamp = disk.getLastModified(path);
|
||||
disk.setLastModified(path, new_timestamp);
|
||||
@ -44,7 +44,7 @@ ChmodOperation::ChmodOperation(const std::string & path_, mode_t mode_, IDisk &
|
||||
{
|
||||
}
|
||||
|
||||
void ChmodOperation::execute(std::unique_lock<std::shared_mutex> &)
|
||||
void ChmodOperation::execute(std::unique_lock<SharedMutex> &)
|
||||
{
|
||||
old_mode = disk.stat(path).st_mode;
|
||||
disk.chmod(path, mode);
|
||||
@ -61,7 +61,7 @@ UnlinkFileOperation::UnlinkFileOperation(const std::string & path_, IDisk & disk
|
||||
{
|
||||
}
|
||||
|
||||
void UnlinkFileOperation::execute(std::unique_lock<std::shared_mutex> &)
|
||||
void UnlinkFileOperation::execute(std::unique_lock<SharedMutex> &)
|
||||
{
|
||||
auto buf = disk.readFile(path, ReadSettings{}, std::nullopt, disk.getFileSize(path));
|
||||
readStringUntilEOF(prev_data, *buf);
|
||||
@ -81,7 +81,7 @@ CreateDirectoryOperation::CreateDirectoryOperation(const std::string & path_, ID
|
||||
{
|
||||
}
|
||||
|
||||
void CreateDirectoryOperation::execute(std::unique_lock<std::shared_mutex> &)
|
||||
void CreateDirectoryOperation::execute(std::unique_lock<SharedMutex> &)
|
||||
{
|
||||
disk.createDirectory(path);
|
||||
}
|
||||
@ -97,7 +97,7 @@ CreateDirectoryRecursiveOperation::CreateDirectoryRecursiveOperation(const std::
|
||||
{
|
||||
}
|
||||
|
||||
void CreateDirectoryRecursiveOperation::execute(std::unique_lock<std::shared_mutex> &)
|
||||
void CreateDirectoryRecursiveOperation::execute(std::unique_lock<SharedMutex> &)
|
||||
{
|
||||
namespace fs = std::filesystem;
|
||||
fs::path p(path);
|
||||
@ -124,7 +124,7 @@ RemoveDirectoryOperation::RemoveDirectoryOperation(const std::string & path_, ID
|
||||
{
|
||||
}
|
||||
|
||||
void RemoveDirectoryOperation::execute(std::unique_lock<std::shared_mutex> &)
|
||||
void RemoveDirectoryOperation::execute(std::unique_lock<SharedMutex> &)
|
||||
{
|
||||
disk.removeDirectory(path);
|
||||
}
|
||||
@ -141,7 +141,7 @@ RemoveRecursiveOperation::RemoveRecursiveOperation(const std::string & path_, ID
|
||||
{
|
||||
}
|
||||
|
||||
void RemoveRecursiveOperation::execute(std::unique_lock<std::shared_mutex> &)
|
||||
void RemoveRecursiveOperation::execute(std::unique_lock<SharedMutex> &)
|
||||
{
|
||||
if (disk.isFile(path))
|
||||
disk.moveFile(path, temp_path);
|
||||
@ -174,7 +174,7 @@ CreateHardlinkOperation::CreateHardlinkOperation(const std::string & path_from_,
|
||||
{
|
||||
}
|
||||
|
||||
void CreateHardlinkOperation::execute(std::unique_lock<std::shared_mutex> & lock)
|
||||
void CreateHardlinkOperation::execute(std::unique_lock<SharedMutex> & lock)
|
||||
{
|
||||
auto metadata = metadata_storage.readMetadataUnlocked(path_from, lock);
|
||||
|
||||
@ -201,7 +201,7 @@ MoveFileOperation::MoveFileOperation(const std::string & path_from_, const std::
|
||||
{
|
||||
}
|
||||
|
||||
void MoveFileOperation::execute(std::unique_lock<std::shared_mutex> &)
|
||||
void MoveFileOperation::execute(std::unique_lock<SharedMutex> &)
|
||||
{
|
||||
disk.moveFile(path_from, path_to);
|
||||
}
|
||||
@ -218,7 +218,7 @@ MoveDirectoryOperation::MoveDirectoryOperation(const std::string & path_from_, c
|
||||
{
|
||||
}
|
||||
|
||||
void MoveDirectoryOperation::execute(std::unique_lock<std::shared_mutex> &)
|
||||
void MoveDirectoryOperation::execute(std::unique_lock<SharedMutex> &)
|
||||
{
|
||||
disk.moveDirectory(path_from, path_to);
|
||||
}
|
||||
@ -236,7 +236,7 @@ ReplaceFileOperation::ReplaceFileOperation(const std::string & path_from_, const
|
||||
{
|
||||
}
|
||||
|
||||
void ReplaceFileOperation::execute(std::unique_lock<std::shared_mutex> &)
|
||||
void ReplaceFileOperation::execute(std::unique_lock<SharedMutex> &)
|
||||
{
|
||||
if (disk.exists(path_to))
|
||||
disk.moveFile(path_to, temp_path_to);
|
||||
@ -262,7 +262,7 @@ WriteFileOperation::WriteFileOperation(const std::string & path_, IDisk & disk_,
|
||||
{
|
||||
}
|
||||
|
||||
void WriteFileOperation::execute(std::unique_lock<std::shared_mutex> &)
|
||||
void WriteFileOperation::execute(std::unique_lock<SharedMutex> &)
|
||||
{
|
||||
if (disk.exists(path))
|
||||
{
|
||||
@ -288,7 +288,7 @@ void WriteFileOperation::undo()
|
||||
}
|
||||
}
|
||||
|
||||
void AddBlobOperation::execute(std::unique_lock<std::shared_mutex> & metadata_lock)
|
||||
void AddBlobOperation::execute(std::unique_lock<SharedMutex> & metadata_lock)
|
||||
{
|
||||
DiskObjectStorageMetadataPtr metadata;
|
||||
if (metadata_storage.exists(path))
|
||||
@ -309,7 +309,7 @@ void AddBlobOperation::undo()
|
||||
write_operation->undo();
|
||||
}
|
||||
|
||||
void UnlinkMetadataFileOperation::execute(std::unique_lock<std::shared_mutex> & metadata_lock)
|
||||
void UnlinkMetadataFileOperation::execute(std::unique_lock<SharedMutex> & metadata_lock)
|
||||
{
|
||||
auto metadata = metadata_storage.readMetadataUnlocked(path, metadata_lock);
|
||||
uint32_t ref_count = metadata->getRefCount();
|
||||
@ -336,7 +336,7 @@ void UnlinkMetadataFileOperation::undo()
|
||||
write_operation->undo();
|
||||
}
|
||||
|
||||
void SetReadonlyFileOperation::execute(std::unique_lock<std::shared_mutex> & metadata_lock)
|
||||
void SetReadonlyFileOperation::execute(std::unique_lock<SharedMutex> & metadata_lock)
|
||||
{
|
||||
auto metadata = metadata_storage.readMetadataUnlocked(path, metadata_lock);
|
||||
metadata->setReadOnly();
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <Disks/ObjectStorages/IMetadataStorage.h>
|
||||
|
||||
namespace DB
|
||||
@ -13,7 +14,7 @@ class IDisk;
|
||||
|
||||
struct IMetadataOperation
|
||||
{
|
||||
virtual void execute(std::unique_lock<std::shared_mutex> & metadata_lock) = 0;
|
||||
virtual void execute(std::unique_lock<SharedMutex> & metadata_lock) = 0;
|
||||
virtual void undo() = 0;
|
||||
virtual void finalize() {}
|
||||
virtual ~IMetadataOperation() = default;
|
||||
@ -26,7 +27,7 @@ struct SetLastModifiedOperation final : public IMetadataOperation
|
||||
{
|
||||
SetLastModifiedOperation(const std::string & path_, Poco::Timestamp new_timestamp_, IDisk & disk_);
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
|
||||
@ -41,7 +42,7 @@ struct ChmodOperation final : public IMetadataOperation
|
||||
{
|
||||
ChmodOperation(const std::string & path_, mode_t mode_, IDisk & disk_);
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
|
||||
@ -57,7 +58,7 @@ struct UnlinkFileOperation final : public IMetadataOperation
|
||||
{
|
||||
UnlinkFileOperation(const std::string & path_, IDisk & disk_);
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
|
||||
@ -72,7 +73,7 @@ struct CreateDirectoryOperation final : public IMetadataOperation
|
||||
{
|
||||
CreateDirectoryOperation(const std::string & path_, IDisk & disk_);
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
|
||||
@ -86,7 +87,7 @@ struct CreateDirectoryRecursiveOperation final : public IMetadataOperation
|
||||
{
|
||||
CreateDirectoryRecursiveOperation(const std::string & path_, IDisk & disk_);
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
|
||||
@ -101,7 +102,7 @@ struct RemoveDirectoryOperation final : public IMetadataOperation
|
||||
{
|
||||
RemoveDirectoryOperation(const std::string & path_, IDisk & disk_);
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
|
||||
@ -114,7 +115,7 @@ struct RemoveRecursiveOperation final : public IMetadataOperation
|
||||
{
|
||||
RemoveRecursiveOperation(const std::string & path_, IDisk & disk_);
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
|
||||
@ -130,7 +131,7 @@ struct WriteFileOperation final : public IMetadataOperation
|
||||
{
|
||||
WriteFileOperation(const std::string & path_, IDisk & disk_, const std::string & data_);
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
private:
|
||||
@ -149,7 +150,7 @@ struct CreateHardlinkOperation final : public IMetadataOperation
|
||||
IDisk & disk_,
|
||||
const MetadataStorageFromDisk & metadata_storage_);
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
|
||||
@ -166,7 +167,7 @@ struct MoveFileOperation final : public IMetadataOperation
|
||||
{
|
||||
MoveFileOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_);
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
|
||||
@ -181,7 +182,7 @@ struct MoveDirectoryOperation final : public IMetadataOperation
|
||||
{
|
||||
MoveDirectoryOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_);
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
|
||||
@ -196,7 +197,7 @@ struct ReplaceFileOperation final : public IMetadataOperation
|
||||
{
|
||||
ReplaceFileOperation(const std::string & path_from_, const std::string & path_to_, IDisk & disk_);
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
|
||||
@ -226,7 +227,7 @@ struct AddBlobOperation final : public IMetadataOperation
|
||||
, metadata_storage(metadata_storage_)
|
||||
{}
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
|
||||
@ -254,7 +255,7 @@ struct UnlinkMetadataFileOperation final : public IMetadataOperation
|
||||
{
|
||||
}
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
|
||||
@ -279,7 +280,7 @@ struct SetReadonlyFileOperation final : public IMetadataOperation
|
||||
{
|
||||
}
|
||||
|
||||
void execute(std::unique_lock<std::shared_mutex> & metadata_lock) override;
|
||||
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
|
||||
|
||||
void undo() override;
|
||||
|
||||
|
@ -16,18 +16,13 @@
|
||||
#include <IO/WriteBufferFromS3.h>
|
||||
#include <IO/ReadBufferFromS3.h>
|
||||
#include <IO/SeekAvoidingReadBuffer.h>
|
||||
#include <IO/S3/copyS3File.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Disks/ObjectStorages/S3/diskSettings.h>
|
||||
|
||||
#include <aws/s3/model/CopyObjectRequest.h>
|
||||
#include <aws/s3/model/ListObjectsV2Request.h>
|
||||
#include <aws/s3/model/HeadObjectRequest.h>
|
||||
#include <aws/s3/model/DeleteObjectRequest.h>
|
||||
#include <aws/s3/model/DeleteObjectsRequest.h>
|
||||
#include <aws/s3/model/CreateMultipartUploadRequest.h>
|
||||
#include <aws/s3/model/CompleteMultipartUploadRequest.h>
|
||||
#include <aws/s3/model/UploadPartCopyRequest.h>
|
||||
#include <aws/s3/model/AbortMultipartUploadRequest.h>
|
||||
|
||||
#include <Common/getRandomASCIIString.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
@ -39,22 +34,9 @@
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event S3DeleteObjects;
|
||||
extern const Event S3HeadObject;
|
||||
extern const Event S3ListObjects;
|
||||
extern const Event S3CopyObject;
|
||||
extern const Event S3CreateMultipartUpload;
|
||||
extern const Event S3UploadPartCopy;
|
||||
extern const Event S3AbortMultipartUpload;
|
||||
extern const Event S3CompleteMultipartUpload;
|
||||
|
||||
extern const Event DiskS3DeleteObjects;
|
||||
extern const Event DiskS3HeadObject;
|
||||
extern const Event DiskS3ListObjects;
|
||||
extern const Event DiskS3CopyObject;
|
||||
extern const Event DiskS3CreateMultipartUpload;
|
||||
extern const Event DiskS3UploadPartCopy;
|
||||
extern const Event DiskS3AbortMultipartUpload;
|
||||
extern const Event DiskS3CompleteMultipartUpload;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
@ -125,21 +107,11 @@ std::string S3ObjectStorage::generateBlobNameForPath(const std::string & /* path
|
||||
getRandomASCIIString(key_name_total_size - key_name_prefix_size));
|
||||
}
|
||||
|
||||
size_t S3ObjectStorage::getObjectSize(const std::string & bucket_from, const std::string & key) const
|
||||
{
|
||||
return S3::getObjectSize(*client.get(), bucket_from, key, {}, /* for_disk_s3= */ true);
|
||||
}
|
||||
|
||||
bool S3ObjectStorage::exists(const StoredObject & object) const
|
||||
{
|
||||
return S3::objectExists(*client.get(), bucket, object.absolute_path, {}, /* for_disk_s3= */ true);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::checkObjectExists(const std::string & bucket_from, const std::string & key, std::string_view description) const
|
||||
{
|
||||
return S3::checkObjectExists(*client.get(), bucket_from, key, {}, /* for_disk_s3= */ true, description);
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> S3ObjectStorage::readObjects( /// NOLINT
|
||||
const StoredObjects & objects,
|
||||
const ReadSettings & read_settings,
|
||||
@ -431,7 +403,12 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT
|
||||
/// Shortcut for S3
|
||||
if (auto * dest_s3 = dynamic_cast<S3ObjectStorage * >(&object_storage_to); dest_s3 != nullptr)
|
||||
{
|
||||
copyObjectImpl(bucket, object_from.absolute_path, dest_s3->bucket, object_to.absolute_path, {}, object_to_attributes);
|
||||
auto client_ptr = client.get();
|
||||
auto size = S3::getObjectSize(*client_ptr, bucket, object_from.absolute_path, {}, /* for_disk_s3= */ true);
|
||||
auto settings_ptr = s3_settings.get();
|
||||
auto scheduler = threadPoolCallbackRunner<void>(getThreadPoolWriter(), "S3ObjStor_copy");
|
||||
copyS3File(client_ptr, bucket, object_from.absolute_path, 0, size, dest_s3->bucket, object_to.absolute_path,
|
||||
settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -439,148 +416,15 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT
|
||||
}
|
||||
}
|
||||
|
||||
void S3ObjectStorage::copyObjectImpl(
|
||||
const String & src_bucket,
|
||||
const String & src_key,
|
||||
const String & dst_bucket,
|
||||
const String & dst_key,
|
||||
size_t size,
|
||||
std::optional<ObjectAttributes> metadata) const
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::S3CopyObject);
|
||||
ProfileEvents::increment(ProfileEvents::DiskS3CopyObject);
|
||||
Aws::S3::Model::CopyObjectRequest request;
|
||||
request.SetCopySource(src_bucket + "/" + src_key);
|
||||
request.SetBucket(dst_bucket);
|
||||
request.SetKey(dst_key);
|
||||
if (metadata)
|
||||
{
|
||||
request.SetMetadata(*metadata);
|
||||
request.SetMetadataDirective(Aws::S3::Model::MetadataDirective::REPLACE);
|
||||
}
|
||||
|
||||
auto outcome = client_ptr->CopyObject(request);
|
||||
|
||||
if (!outcome.IsSuccess() && (outcome.GetError().GetExceptionName() == "EntityTooLarge"
|
||||
|| outcome.GetError().GetExceptionName() == "InvalidRequest"))
|
||||
{ // Can't come here with MinIO, MinIO allows single part upload for large objects.
|
||||
copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, size, metadata);
|
||||
return;
|
||||
}
|
||||
|
||||
throwIfError(outcome);
|
||||
|
||||
auto settings_ptr = s3_settings.get();
|
||||
if (settings_ptr->request_settings.check_objects_after_upload)
|
||||
checkObjectExists(dst_bucket, dst_key, "Immediately after upload");
|
||||
}
|
||||
|
||||
void S3ObjectStorage::copyObjectMultipartImpl(
|
||||
const String & src_bucket,
|
||||
const String & src_key,
|
||||
const String & dst_bucket,
|
||||
const String & dst_key,
|
||||
size_t size,
|
||||
std::optional<ObjectAttributes> metadata) const
|
||||
{
|
||||
auto settings_ptr = s3_settings.get();
|
||||
auto client_ptr = client.get();
|
||||
|
||||
String multipart_upload_id;
|
||||
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::S3CreateMultipartUpload);
|
||||
ProfileEvents::increment(ProfileEvents::DiskS3CreateMultipartUpload);
|
||||
Aws::S3::Model::CreateMultipartUploadRequest request;
|
||||
request.SetBucket(dst_bucket);
|
||||
request.SetKey(dst_key);
|
||||
if (metadata)
|
||||
request.SetMetadata(*metadata);
|
||||
|
||||
auto outcome = client_ptr->CreateMultipartUpload(request);
|
||||
|
||||
throwIfError(outcome);
|
||||
|
||||
multipart_upload_id = outcome.GetResult().GetUploadId();
|
||||
}
|
||||
|
||||
std::vector<String> part_tags;
|
||||
|
||||
size_t upload_part_size = settings_ptr->request_settings.getUploadSettings().min_upload_part_size;
|
||||
for (size_t position = 0, part_number = 1; position < size; ++part_number, position += upload_part_size)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::S3UploadPartCopy);
|
||||
ProfileEvents::increment(ProfileEvents::DiskS3UploadPartCopy);
|
||||
Aws::S3::Model::UploadPartCopyRequest part_request;
|
||||
part_request.SetCopySource(src_bucket + "/" + src_key);
|
||||
part_request.SetBucket(dst_bucket);
|
||||
part_request.SetKey(dst_key);
|
||||
part_request.SetUploadId(multipart_upload_id);
|
||||
part_request.SetPartNumber(static_cast<int>(part_number));
|
||||
part_request.SetCopySourceRange(fmt::format("bytes={}-{}", position, std::min(size, position + upload_part_size) - 1));
|
||||
|
||||
auto outcome = client_ptr->UploadPartCopy(part_request);
|
||||
if (!outcome.IsSuccess())
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::S3AbortMultipartUpload);
|
||||
ProfileEvents::increment(ProfileEvents::DiskS3AbortMultipartUpload);
|
||||
Aws::S3::Model::AbortMultipartUploadRequest abort_request;
|
||||
abort_request.SetBucket(dst_bucket);
|
||||
abort_request.SetKey(dst_key);
|
||||
abort_request.SetUploadId(multipart_upload_id);
|
||||
client_ptr->AbortMultipartUpload(abort_request);
|
||||
// In error case we throw exception later with first error from UploadPartCopy
|
||||
}
|
||||
throwIfError(outcome);
|
||||
|
||||
auto etag = outcome.GetResult().GetCopyPartResult().GetETag();
|
||||
part_tags.push_back(etag);
|
||||
}
|
||||
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::S3CompleteMultipartUpload);
|
||||
ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload);
|
||||
Aws::S3::Model::CompleteMultipartUploadRequest req;
|
||||
req.SetBucket(dst_bucket);
|
||||
req.SetKey(dst_key);
|
||||
req.SetUploadId(multipart_upload_id);
|
||||
|
||||
Aws::S3::Model::CompletedMultipartUpload multipart_upload;
|
||||
for (size_t i = 0; i < part_tags.size(); ++i)
|
||||
{
|
||||
Aws::S3::Model::CompletedPart part;
|
||||
multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(static_cast<int>(i) + 1));
|
||||
}
|
||||
|
||||
req.SetMultipartUpload(multipart_upload);
|
||||
|
||||
auto outcome = client_ptr->CompleteMultipartUpload(req);
|
||||
|
||||
throwIfError(outcome);
|
||||
}
|
||||
|
||||
if (settings_ptr->request_settings.check_objects_after_upload)
|
||||
checkObjectExists(dst_bucket, dst_key, "Immediately after upload");
|
||||
}
|
||||
|
||||
void S3ObjectStorage::copyObject( // NOLINT
|
||||
const StoredObject & object_from, const StoredObject & object_to, std::optional<ObjectAttributes> object_to_attributes)
|
||||
{
|
||||
auto size = getObjectSize(bucket, object_from.absolute_path);
|
||||
static constexpr int64_t multipart_upload_threashold = 5UL * 1024 * 1024 * 1024;
|
||||
|
||||
if (size >= multipart_upload_threashold)
|
||||
{
|
||||
copyObjectMultipartImpl(
|
||||
bucket, object_from.absolute_path, bucket, object_to.absolute_path, size, object_to_attributes);
|
||||
}
|
||||
else
|
||||
{
|
||||
copyObjectImpl(
|
||||
bucket, object_from.absolute_path, bucket, object_to.absolute_path, size, object_to_attributes);
|
||||
}
|
||||
auto client_ptr = client.get();
|
||||
auto size = S3::getObjectSize(*client_ptr, bucket, object_from.absolute_path, {}, /* for_disk_s3= */ true);
|
||||
auto settings_ptr = s3_settings.get();
|
||||
auto scheduler = threadPoolCallbackRunner<void>(getThreadPoolWriter(), "S3ObjStor_copy");
|
||||
copyS3File(client_ptr, bucket, object_from.absolute_path, 0, size, bucket, object_to.absolute_path,
|
||||
settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::setNewSettings(std::unique_ptr<S3ObjectStorageSettings> && s3_settings_)
|
||||
|
@ -8,8 +8,6 @@
|
||||
#include <Disks/ObjectStorages/S3/S3Capabilities.h>
|
||||
#include <memory>
|
||||
#include <aws/s3/S3Client.h>
|
||||
#include <aws/s3/model/HeadObjectResult.h>
|
||||
#include <aws/s3/model/ListObjectsV2Result.h>
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
#include <Common/MultiVersion.h>
|
||||
#include <Common/logger_useful.h>
|
||||
@ -167,28 +165,9 @@ private:
|
||||
|
||||
void setNewClient(std::unique_ptr<Aws::S3::S3Client> && client_);
|
||||
|
||||
void copyObjectImpl(
|
||||
const String & src_bucket,
|
||||
const String & src_key,
|
||||
const String & dst_bucket,
|
||||
const String & dst_key,
|
||||
size_t size,
|
||||
std::optional<ObjectAttributes> metadata = std::nullopt) const;
|
||||
|
||||
void copyObjectMultipartImpl(
|
||||
const String & src_bucket,
|
||||
const String & src_key,
|
||||
const String & dst_bucket,
|
||||
const String & dst_key,
|
||||
size_t size,
|
||||
std::optional<ObjectAttributes> metadata = std::nullopt) const;
|
||||
|
||||
void removeObjectImpl(const StoredObject & object, bool if_exists);
|
||||
void removeObjectsImpl(const StoredObjects & objects, bool if_exists);
|
||||
|
||||
size_t getObjectSize(const std::string & bucket_from, const std::string & key) const;
|
||||
void checkObjectExists(const std::string & bucket_from, const std::string & key, std::string_view description) const;
|
||||
|
||||
std::string bucket;
|
||||
|
||||
MultiVersion<Aws::S3::S3Client> client;
|
||||
|
17
src/Functions/FunctionsHashingInt.cpp
Normal file
17
src/Functions/FunctionsHashingInt.cpp
Normal file
@ -0,0 +1,17 @@
|
||||
#include "FunctionsHashing.h"
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
/// FunctionsHashing instantiations are separated into files FunctionsHashing*.cpp
|
||||
/// to better parallelize the build procedure and avoid MSan build failure
|
||||
/// due to excessive resource consumption.
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
REGISTER_FUNCTION(HashingInt)
|
||||
{
|
||||
factory.registerFunction<FunctionIntHash32>();
|
||||
factory.registerFunction<FunctionIntHash64>();
|
||||
}
|
||||
}
|
@ -2,6 +2,9 @@
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
/// FunctionsHashing instantiations are separated into files FunctionsHashing*.cpp
|
||||
/// to better parallelize the build procedure and avoid MSan build failure
|
||||
/// due to excessive resource consumption.
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -14,18 +17,10 @@ REGISTER_FUNCTION(Hashing)
|
||||
factory.registerFunction<FunctionFarmFingerprint64>();
|
||||
factory.registerFunction<FunctionFarmHash64>();
|
||||
factory.registerFunction<FunctionMetroHash64>();
|
||||
factory.registerFunction<FunctionIntHash32>();
|
||||
factory.registerFunction<FunctionIntHash64>();
|
||||
factory.registerFunction<FunctionURLHash>();
|
||||
factory.registerFunction<FunctionJavaHash>();
|
||||
factory.registerFunction<FunctionJavaHashUTF16LE>();
|
||||
factory.registerFunction<FunctionHiveHash>();
|
||||
factory.registerFunction<FunctionMurmurHash2_32>();
|
||||
factory.registerFunction<FunctionMurmurHash2_64>();
|
||||
factory.registerFunction<FunctionMurmurHash3_32>();
|
||||
factory.registerFunction<FunctionMurmurHash3_64>();
|
||||
factory.registerFunction<FunctionMurmurHash3_128>();
|
||||
factory.registerFunction<FunctionGccMurmurHash>();
|
||||
|
||||
factory.registerFunction<FunctionXxHash32>();
|
||||
factory.registerFunction<FunctionXxHash64>();
|
21
src/Functions/FunctionsHashingMurmur.cpp
Normal file
21
src/Functions/FunctionsHashingMurmur.cpp
Normal file
@ -0,0 +1,21 @@
|
||||
#include "FunctionsHashing.h"
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
/// FunctionsHashing instantiations are separated into files FunctionsHashing*.cpp
|
||||
/// to better parallelize the build procedure and avoid MSan build failure
|
||||
/// due to excessive resource consumption.
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
REGISTER_FUNCTION(HashingMurmur)
|
||||
{
|
||||
factory.registerFunction<FunctionMurmurHash2_32>();
|
||||
factory.registerFunction<FunctionMurmurHash2_64>();
|
||||
factory.registerFunction<FunctionMurmurHash3_32>();
|
||||
factory.registerFunction<FunctionMurmurHash3_64>();
|
||||
factory.registerFunction<FunctionMurmurHash3_128>();
|
||||
factory.registerFunction<FunctionGccMurmurHash>();
|
||||
}
|
||||
}
|
@ -5,8 +5,9 @@
|
||||
#include "FunctionsHashing.h"
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
/// SSL functions are located in the separate FunctionsHashingSSL.cpp file
|
||||
/// to lower the compilation time of FunctionsHashing.cpp
|
||||
/// FunctionsHashing instantiations are separated into files FunctionsHashing*.cpp
|
||||
/// to better parallelize the build procedure and avoid MSan build failure
|
||||
/// due to excessive resource consumption.
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <IO/S3/copyDataToS3.h>
|
||||
#include <IO/S3/copyS3File.h>
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
@ -22,8 +22,17 @@ namespace ProfileEvents
|
||||
{
|
||||
extern const Event S3CreateMultipartUpload;
|
||||
extern const Event S3CompleteMultipartUpload;
|
||||
extern const Event S3UploadPart;
|
||||
extern const Event S3PutObject;
|
||||
extern const Event S3CopyObject;
|
||||
extern const Event S3UploadPart;
|
||||
extern const Event S3UploadPartCopy;
|
||||
|
||||
extern const Event DiskS3CreateMultipartUpload;
|
||||
extern const Event DiskS3CompleteMultipartUpload;
|
||||
extern const Event DiskS3PutObject;
|
||||
extern const Event DiskS3CopyObject;
|
||||
extern const Event DiskS3UploadPart;
|
||||
extern const Event DiskS3UploadPartCopy;
|
||||
}
|
||||
|
||||
|
||||
@ -50,6 +59,7 @@ namespace
|
||||
const S3Settings::RequestSettings & request_settings_,
|
||||
const std::optional<std::map<String, String>> & object_metadata_,
|
||||
ThreadPoolCallbackRunner<void> schedule_,
|
||||
bool for_disk_s3_,
|
||||
const Poco::Logger * log_)
|
||||
: client_ptr(client_ptr_)
|
||||
, dest_bucket(dest_bucket_)
|
||||
@ -59,6 +69,7 @@ namespace
|
||||
, max_unexpected_write_error_retries(request_settings_.max_unexpected_write_error_retries)
|
||||
, object_metadata(object_metadata_)
|
||||
, schedule(schedule_)
|
||||
, for_disk_s3(for_disk_s3_)
|
||||
, log(log_)
|
||||
{
|
||||
}
|
||||
@ -74,6 +85,7 @@ namespace
|
||||
size_t max_unexpected_write_error_retries;
|
||||
const std::optional<std::map<String, String>> & object_metadata;
|
||||
ThreadPoolCallbackRunner<void> schedule;
|
||||
bool for_disk_s3;
|
||||
const Poco::Logger * log;
|
||||
|
||||
struct UploadPartTask
|
||||
@ -111,6 +123,8 @@ namespace
|
||||
request.SetStorageClass(Aws::S3::Model::StorageClassMapper::GetStorageClassForName(settings.storage_class_name));
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::S3CreateMultipartUpload);
|
||||
if (for_disk_s3)
|
||||
ProfileEvents::increment(ProfileEvents::DiskS3CreateMultipartUpload);
|
||||
|
||||
auto outcome = client_ptr->CreateMultipartUpload(request);
|
||||
|
||||
@ -147,10 +161,12 @@ namespace
|
||||
|
||||
request.SetMultipartUpload(multipart_upload);
|
||||
|
||||
size_t max_retry = std::max(max_unexpected_write_error_retries, 1UL);
|
||||
for (size_t i = 0; i < max_retry; ++i)
|
||||
size_t max_retries = std::max(max_unexpected_write_error_retries, 1UL);
|
||||
for (size_t retries = 1;; ++retries)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::S3CompleteMultipartUpload);
|
||||
if (for_disk_s3)
|
||||
ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload);
|
||||
|
||||
auto outcome = client_ptr->CompleteMultipartUpload(request);
|
||||
|
||||
@ -159,20 +175,19 @@ namespace
|
||||
LOG_TRACE(log, "Multipart upload has completed. Bucket: {}, Key: {}, Upload_id: {}, Parts: {}", dest_bucket, dest_key, multipart_upload_id, part_tags.size());
|
||||
break;
|
||||
}
|
||||
else if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY)
|
||||
|
||||
if ((outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) && (retries < max_retries))
|
||||
{
|
||||
/// For unknown reason, at least MinIO can respond with NO_SUCH_KEY for put requests
|
||||
/// BTW, NO_SUCH_UPLOAD is expected error and we shouldn't retry it
|
||||
LOG_INFO(log, "Multipart upload failed with NO_SUCH_KEY error for Bucket: {}, Key: {}, Upload_id: {}, Parts: {}, will retry", dest_bucket, dest_key, multipart_upload_id, part_tags.size());
|
||||
/// will retry
|
||||
}
|
||||
else
|
||||
{
|
||||
throw S3Exception(
|
||||
outcome.GetError().GetErrorType(),
|
||||
"Message: {}, Key: {}, Bucket: {}, Tags: {}",
|
||||
outcome.GetError().GetMessage(), dest_key, dest_bucket, fmt::join(part_tags.begin(), part_tags.end(), " "));
|
||||
continue; /// will retry
|
||||
}
|
||||
|
||||
throw S3Exception(
|
||||
outcome.GetError().GetErrorType(),
|
||||
"Message: {}, Key: {}, Bucket: {}, Tags: {}",
|
||||
outcome.GetError().GetMessage(), dest_key, dest_bucket, fmt::join(part_tags.begin(), part_tags.end(), " "));
|
||||
}
|
||||
}
|
||||
|
||||
@ -379,11 +394,11 @@ namespace
|
||||
}
|
||||
};
|
||||
|
||||
/// Helper class to help implementing copyDataToS3().
|
||||
class CopyDataToS3Helper : public UploadHelper
|
||||
/// Helper class to help implementing copyDataToS3File().
|
||||
class CopyDataToFileHelper : public UploadHelper
|
||||
{
|
||||
public:
|
||||
CopyDataToS3Helper(
|
||||
CopyDataToFileHelper(
|
||||
const std::function<std::unique_ptr<SeekableReadBuffer>()> & create_read_buffer_,
|
||||
size_t offset_,
|
||||
size_t size_,
|
||||
@ -392,8 +407,9 @@ namespace
|
||||
const String & dest_key_,
|
||||
const S3Settings::RequestSettings & request_settings_,
|
||||
const std::optional<std::map<String, String>> & object_metadata_,
|
||||
ThreadPoolCallbackRunner<void> schedule_)
|
||||
: UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, &Poco::Logger::get("copyDataToS3"))
|
||||
ThreadPoolCallbackRunner<void> schedule_,
|
||||
bool for_disk_s3_)
|
||||
: UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyDataToS3File"))
|
||||
, create_read_buffer(create_read_buffer_)
|
||||
, offset(offset_)
|
||||
, size(size_)
|
||||
@ -444,10 +460,13 @@ namespace
|
||||
|
||||
void processPutRequest(const Aws::S3::Model::PutObjectRequest & request)
|
||||
{
|
||||
size_t max_retry = std::max(max_unexpected_write_error_retries, 1UL);
|
||||
for (size_t i = 0; i < max_retry; ++i)
|
||||
size_t max_retries = std::max(max_unexpected_write_error_retries, 1UL);
|
||||
for (size_t retries = 1;; ++retries)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::S3PutObject);
|
||||
if (for_disk_s3)
|
||||
ProfileEvents::increment(ProfileEvents::DiskS3PutObject);
|
||||
|
||||
auto outcome = client_ptr->PutObject(request);
|
||||
|
||||
if (outcome.IsSuccess())
|
||||
@ -460,7 +479,8 @@ namespace
|
||||
request.GetContentLength());
|
||||
break;
|
||||
}
|
||||
else if (outcome.GetError().GetExceptionName() == "EntityTooLarge" || outcome.GetError().GetExceptionName() == "InvalidRequest")
|
||||
|
||||
if (outcome.GetError().GetExceptionName() == "EntityTooLarge" || outcome.GetError().GetExceptionName() == "InvalidRequest")
|
||||
{
|
||||
// Can't come here with MinIO, MinIO allows single part upload for large objects.
|
||||
LOG_INFO(
|
||||
@ -473,7 +493,8 @@ namespace
|
||||
performMultipartUpload();
|
||||
break;
|
||||
}
|
||||
else if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY)
|
||||
|
||||
if ((outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) && (retries < max_retries))
|
||||
{
|
||||
/// For unknown reason, at least MinIO can respond with NO_SUCH_KEY for put requests
|
||||
LOG_INFO(
|
||||
@ -482,18 +503,16 @@ namespace
|
||||
dest_bucket,
|
||||
dest_key,
|
||||
request.GetContentLength());
|
||||
/// will retry
|
||||
}
|
||||
else
|
||||
{
|
||||
throw S3Exception(
|
||||
outcome.GetError().GetErrorType(),
|
||||
"Message: {}, Key: {}, Bucket: {}, Object size: {}",
|
||||
outcome.GetError().GetMessage(),
|
||||
dest_key,
|
||||
dest_bucket,
|
||||
request.GetContentLength());
|
||||
continue; /// will retry
|
||||
}
|
||||
|
||||
throw S3Exception(
|
||||
outcome.GetError().GetErrorType(),
|
||||
"Message: {}, Key: {}, Bucket: {}, Object size: {}",
|
||||
outcome.GetError().GetMessage(),
|
||||
dest_key,
|
||||
dest_bucket,
|
||||
request.GetContentLength());
|
||||
}
|
||||
}
|
||||
|
||||
@ -523,6 +542,8 @@ namespace
|
||||
auto & req = typeid_cast<Aws::S3::Model::UploadPartRequest &>(request);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::S3UploadPart);
|
||||
if (for_disk_s3)
|
||||
ProfileEvents::increment(ProfileEvents::DiskS3UploadPart);
|
||||
|
||||
auto outcome = client_ptr->UploadPart(req);
|
||||
if (!outcome.IsSuccess())
|
||||
@ -535,11 +556,11 @@ namespace
|
||||
}
|
||||
};
|
||||
|
||||
/// Helper class to help implementing copyFileS3ToS3().
|
||||
class CopyFileS3ToS3Helper : public UploadHelper
|
||||
/// Helper class to help implementing copyS3File().
|
||||
class CopyFileHelper : public UploadHelper
|
||||
{
|
||||
public:
|
||||
CopyFileS3ToS3Helper(
|
||||
CopyFileHelper(
|
||||
const std::shared_ptr<const Aws::S3::S3Client> & client_ptr_,
|
||||
const String & src_bucket_,
|
||||
const String & src_key_,
|
||||
@ -549,8 +570,9 @@ namespace
|
||||
const String & dest_key_,
|
||||
const S3Settings::RequestSettings & request_settings_,
|
||||
const std::optional<std::map<String, String>> & object_metadata_,
|
||||
ThreadPoolCallbackRunner<void> schedule_)
|
||||
: UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, &Poco::Logger::get("copyFileS3ToS3"))
|
||||
ThreadPoolCallbackRunner<void> schedule_,
|
||||
bool for_disk_s3_)
|
||||
: UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyS3File"))
|
||||
, src_bucket(src_bucket_)
|
||||
, src_key(src_key_)
|
||||
, offset(src_offset_)
|
||||
@ -603,9 +625,13 @@ namespace
|
||||
|
||||
void processCopyRequest(const Aws::S3::Model::CopyObjectRequest & request)
|
||||
{
|
||||
size_t max_retry = std::max(max_unexpected_write_error_retries, 1UL);
|
||||
for (size_t i = 0; i < max_retry; ++i)
|
||||
size_t max_retries = std::max(max_unexpected_write_error_retries, 1UL);
|
||||
for (size_t retries = 1;; ++retries)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::S3CopyObject);
|
||||
if (for_disk_s3)
|
||||
ProfileEvents::increment(ProfileEvents::DiskS3CopyObject);
|
||||
|
||||
auto outcome = client_ptr->CopyObject(request);
|
||||
if (outcome.IsSuccess())
|
||||
{
|
||||
@ -617,7 +643,8 @@ namespace
|
||||
size);
|
||||
break;
|
||||
}
|
||||
else if (outcome.GetError().GetExceptionName() == "EntityTooLarge" || outcome.GetError().GetExceptionName() == "InvalidRequest")
|
||||
|
||||
if (outcome.GetError().GetExceptionName() == "EntityTooLarge" || outcome.GetError().GetExceptionName() == "InvalidRequest")
|
||||
{
|
||||
// Can't come here with MinIO, MinIO allows single part upload for large objects.
|
||||
LOG_INFO(
|
||||
@ -630,7 +657,8 @@ namespace
|
||||
performMultipartUploadCopy();
|
||||
break;
|
||||
}
|
||||
else if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY)
|
||||
|
||||
if ((outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) && (retries < max_retries))
|
||||
{
|
||||
/// TODO: Is it true for copy requests?
|
||||
/// For unknown reason, at least MinIO can respond with NO_SUCH_KEY for put requests
|
||||
@ -640,18 +668,16 @@ namespace
|
||||
dest_bucket,
|
||||
dest_key,
|
||||
size);
|
||||
/// will retry
|
||||
}
|
||||
else
|
||||
{
|
||||
throw S3Exception(
|
||||
outcome.GetError().GetErrorType(),
|
||||
"Message: {}, Key: {}, Bucket: {}, Object size: {}",
|
||||
outcome.GetError().GetMessage(),
|
||||
dest_key,
|
||||
dest_bucket,
|
||||
size);
|
||||
continue; /// will retry
|
||||
}
|
||||
|
||||
throw S3Exception(
|
||||
outcome.GetError().GetErrorType(),
|
||||
"Message: {}, Key: {}, Bucket: {}, Object size: {}",
|
||||
outcome.GetError().GetMessage(),
|
||||
dest_key,
|
||||
dest_bucket,
|
||||
size);
|
||||
}
|
||||
}
|
||||
|
||||
@ -676,6 +702,10 @@ namespace
|
||||
{
|
||||
auto & req = typeid_cast<Aws::S3::Model::UploadPartCopyRequest &>(request);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::S3UploadPartCopy);
|
||||
if (for_disk_s3)
|
||||
ProfileEvents::increment(ProfileEvents::DiskS3UploadPartCopy);
|
||||
|
||||
auto outcome = client_ptr->UploadPartCopy(req);
|
||||
if (!outcome.IsSuccess())
|
||||
{
|
||||
@ -689,7 +719,7 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
void copyDataToS3(
|
||||
void copyDataToS3File(
|
||||
const std::function<std::unique_ptr<SeekableReadBuffer>()> & create_read_buffer,
|
||||
size_t offset,
|
||||
size_t size,
|
||||
@ -698,14 +728,15 @@ void copyDataToS3(
|
||||
const String & dest_key,
|
||||
const S3Settings::RequestSettings & settings,
|
||||
const std::optional<std::map<String, String>> & object_metadata,
|
||||
ThreadPoolCallbackRunner<void> schedule)
|
||||
ThreadPoolCallbackRunner<void> schedule,
|
||||
bool for_disk_s3)
|
||||
{
|
||||
CopyDataToS3Helper helper{create_read_buffer, offset, size, dest_s3_client, dest_bucket, dest_key, settings, object_metadata, schedule};
|
||||
CopyDataToFileHelper helper{create_read_buffer, offset, size, dest_s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3};
|
||||
helper.performCopy();
|
||||
}
|
||||
|
||||
|
||||
void copyFileS3ToS3(
|
||||
void copyS3File(
|
||||
const std::shared_ptr<const Aws::S3::S3Client> & s3_client,
|
||||
const String & src_bucket,
|
||||
const String & src_key,
|
||||
@ -715,9 +746,10 @@ void copyFileS3ToS3(
|
||||
const String & dest_key,
|
||||
const S3Settings::RequestSettings & settings,
|
||||
const std::optional<std::map<String, String>> & object_metadata,
|
||||
ThreadPoolCallbackRunner<void> schedule)
|
||||
ThreadPoolCallbackRunner<void> schedule,
|
||||
bool for_disk_s3)
|
||||
{
|
||||
CopyFileS3ToS3Helper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule};
|
||||
CopyFileHelper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3};
|
||||
helper.performCopy();
|
||||
}
|
||||
|
@ -16,27 +16,11 @@ namespace DB
|
||||
{
|
||||
class SeekableReadBuffer;
|
||||
|
||||
/// Copies data from any seekable source to S3.
|
||||
/// The same functionality can be done by using the function copyData() and the class WriteBufferFromS3
|
||||
/// however copyDataToS3() is faster and spends less memory.
|
||||
/// The callback `create_read_buffer` can be called from multiple threads in parallel, so that should be thread-safe.
|
||||
/// The parameters `offset` and `size` specify a part in the source to copy.
|
||||
void copyDataToS3(
|
||||
const std::function<std::unique_ptr<SeekableReadBuffer>()> & create_read_buffer,
|
||||
size_t offset,
|
||||
size_t size,
|
||||
const std::shared_ptr<const Aws::S3::S3Client> & dest_s3_client,
|
||||
const String & dest_bucket,
|
||||
const String & dest_key,
|
||||
const S3Settings::RequestSettings & settings,
|
||||
const std::optional<std::map<String, String>> & object_metadata = std::nullopt,
|
||||
ThreadPoolCallbackRunner<void> schedule_ = {});
|
||||
|
||||
/// Copies a file from S3 to S3.
|
||||
/// The same functionality can be done by using the function copyData() and the classes ReadBufferFromS3 and WriteBufferFromS3
|
||||
/// however copyFileS3ToS3() is faster and spends less network traffic and memory.
|
||||
/// however copyS3File() is faster and spends less network traffic and memory.
|
||||
/// The parameters `src_offset` and `src_size` specify a part in the source to copy.
|
||||
void copyFileS3ToS3(
|
||||
void copyS3File(
|
||||
const std::shared_ptr<const Aws::S3::S3Client> & s3_client,
|
||||
const String & src_bucket,
|
||||
const String & src_key,
|
||||
@ -46,7 +30,25 @@ void copyFileS3ToS3(
|
||||
const String & dest_key,
|
||||
const S3Settings::RequestSettings & settings,
|
||||
const std::optional<std::map<String, String>> & object_metadata = std::nullopt,
|
||||
ThreadPoolCallbackRunner<void> schedule_ = {});
|
||||
ThreadPoolCallbackRunner<void> schedule_ = {},
|
||||
bool for_disk_s3 = false);
|
||||
|
||||
/// Copies data from any seekable source to S3.
|
||||
/// The same functionality can be done by using the function copyData() and the class WriteBufferFromS3
|
||||
/// however copyDataToS3File() is faster and spends less memory.
|
||||
/// The callback `create_read_buffer` can be called from multiple threads in parallel, so that should be thread-safe.
|
||||
/// The parameters `offset` and `size` specify a part in the source to copy.
|
||||
void copyDataToS3File(
|
||||
const std::function<std::unique_ptr<SeekableReadBuffer>()> & create_read_buffer,
|
||||
size_t offset,
|
||||
size_t size,
|
||||
const std::shared_ptr<const Aws::S3::S3Client> & dest_s3_client,
|
||||
const String & dest_bucket,
|
||||
const String & dest_key,
|
||||
const S3Settings::RequestSettings & settings,
|
||||
const std::optional<std::map<String, String>> & object_metadata = std::nullopt,
|
||||
ThreadPoolCallbackRunner<void> schedule_ = {},
|
||||
bool for_disk_s3 = false);
|
||||
|
||||
}
|
||||
|
@ -726,7 +726,7 @@ DDLGuardPtr DatabaseCatalog::getDDLGuard(const String & database, const String &
|
||||
return std::make_unique<DDLGuard>(db_guard.first, db_guard.second, std::move(lock), table, database);
|
||||
}
|
||||
|
||||
std::unique_lock<std::shared_mutex> DatabaseCatalog::getExclusiveDDLGuardForDatabase(const String & database)
|
||||
std::unique_lock<SharedMutex> DatabaseCatalog::getExclusiveDDLGuardForDatabase(const String & database)
|
||||
{
|
||||
DDLGuards::iterator db_guard_iter;
|
||||
{
|
||||
@ -1279,7 +1279,7 @@ TemporaryLockForUUIDDirectory & TemporaryLockForUUIDDirectory::operator = (Tempo
|
||||
}
|
||||
|
||||
|
||||
DDLGuard::DDLGuard(Map & map_, std::shared_mutex & db_mutex_, std::unique_lock<std::mutex> guards_lock_, const String & elem, const String & database_name)
|
||||
DDLGuard::DDLGuard(Map & map_, SharedMutex & db_mutex_, std::unique_lock<std::mutex> guards_lock_, const String & elem, const String & database_name)
|
||||
: map(map_), db_mutex(db_mutex_), guards_lock(std::move(guards_lock_))
|
||||
{
|
||||
it = map.emplace(elem, Entry{std::make_unique<std::mutex>(), 0}).first;
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Databases/TablesDependencyGraph.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Common/SharedMutex.h>
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Poco/Logger.h>
|
||||
@ -17,7 +18,6 @@
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <set>
|
||||
#include <shared_mutex>
|
||||
#include <unordered_map>
|
||||
#include <unordered_set>
|
||||
#include <filesystem>
|
||||
@ -58,7 +58,7 @@ public:
|
||||
|
||||
DDLGuard(
|
||||
Map & map_,
|
||||
std::shared_mutex & db_mutex_,
|
||||
SharedMutex & db_mutex_,
|
||||
std::unique_lock<std::mutex> guards_lock_,
|
||||
const String & elem,
|
||||
const String & database_name);
|
||||
@ -69,7 +69,7 @@ public:
|
||||
|
||||
private:
|
||||
Map & map;
|
||||
std::shared_mutex & db_mutex;
|
||||
SharedMutex & db_mutex;
|
||||
Map::iterator it;
|
||||
std::unique_lock<std::mutex> guards_lock;
|
||||
std::unique_lock<std::mutex> table_lock;
|
||||
@ -142,7 +142,7 @@ public:
|
||||
/// Get an object that protects the table from concurrently executing multiple DDL operations.
|
||||
DDLGuardPtr getDDLGuard(const String & database, const String & table);
|
||||
/// Get an object that protects the database from concurrent DDL queries all tables in the database
|
||||
std::unique_lock<std::shared_mutex> getExclusiveDDLGuardForDatabase(const String & database);
|
||||
std::unique_lock<SharedMutex> getExclusiveDDLGuardForDatabase(const String & database);
|
||||
|
||||
|
||||
void assertDatabaseExists(const String & database_name) const;
|
||||
@ -298,7 +298,7 @@ private:
|
||||
/// For the duration of the operation, an element is placed here, and an object is returned,
|
||||
/// which deletes the element in the destructor when counter becomes zero.
|
||||
/// In case the element already exists, waits when query will be executed in other thread. See class DDLGuard below.
|
||||
using DatabaseGuard = std::pair<DDLGuard::Map, std::shared_mutex>;
|
||||
using DatabaseGuard = std::pair<DDLGuard::Map, SharedMutex>;
|
||||
using DDLGuards = std::map<String, DatabaseGuard>;
|
||||
DDLGuards ddl_guards TSA_GUARDED_BY(ddl_guards_mutex);
|
||||
/// If you capture mutex and ddl_guards_mutex, then you need to grab them strictly in this order.
|
||||
|
@ -1,31 +1,34 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Interpreters/GinFilter.h>
|
||||
#include <Storages/MergeTree/GinIndexStore.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexFullText.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexInverted.h>
|
||||
#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_)
|
||||
: ngrams(ngrams_)
|
||||
, density(density_)
|
||||
{
|
||||
if (ngrams > 8)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The size of gin filter cannot be greater than 8");
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The size of inverted index filter cannot be greater than 8");
|
||||
if (density <= 0 || density > 1)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The density of gin filter must be between 0 and 1");
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The density inverted index gin filter must be between 0 and 1");
|
||||
}
|
||||
|
||||
GinFilter::GinFilter(const GinFilterParameters & params_)
|
||||
@ -33,23 +36,23 @@ GinFilter::GinFilter(const GinFilterParameters & params_)
|
||||
{
|
||||
}
|
||||
|
||||
void GinFilter::add(const char* data, size_t len, UInt32 rowID, GinIndexStorePtr& store, UInt64 limit) const
|
||||
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);
|
||||
auto it = store->getPostingsListBuilder().find(term);
|
||||
|
||||
if (it != store->getPostings().end())
|
||||
if (it != store->getPostingsListBuilder().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);
|
||||
UInt64 size_limit = std::lround(limit * params.density);
|
||||
auto builder = std::make_shared<GinIndexPostingsBuilder>(size_limit);
|
||||
builder->add(rowID);
|
||||
|
||||
store->setPostingsBuilder(term, builder);
|
||||
@ -66,7 +69,7 @@ void GinFilter::addRowRangeToGinFilter(UInt32 segmentID, UInt32 rowIDStart, UInt
|
||||
if (!rowid_ranges.empty())
|
||||
{
|
||||
/// Try to merge the rowID range with the last one in the container
|
||||
GinSegmentWithRowIDRange & last_rowid_range = rowid_ranges.back();
|
||||
GinSegmentWithRowIdRange & last_rowid_range = rowid_ranges.back();
|
||||
|
||||
if (last_rowid_range.segment_id == segmentID &&
|
||||
last_rowid_range.range_end+1 == rowIDStart)
|
||||
@ -80,93 +83,17 @@ void GinFilter::addRowRangeToGinFilter(UInt32 segmentID, UInt32 rowIDStart, UInt
|
||||
|
||||
void GinFilter::clear()
|
||||
{
|
||||
query_string.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
|
||||
bool GinFilter::contains(const GinFilter & filter, PostingsCacheForStore & cache_store) const
|
||||
{
|
||||
if (filter.getTerms().empty())
|
||||
return true;
|
||||
|
||||
PostingsCachePtr postings_cache = cache_store.getPostings(filter.getQueryString());
|
||||
GinPostingsCachePtr postings_cache = cache_store.getPostings(filter.getQueryString());
|
||||
if (postings_cache == nullptr)
|
||||
{
|
||||
GinIndexStoreDeserializer reader(cache_store.store);
|
||||
@ -177,9 +104,73 @@ bool GinFilter::contains(const GinFilter & filter, PostingsCacheForStore &cache_
|
||||
return match(*postings_cache);
|
||||
}
|
||||
|
||||
String GinFilter::getName()
|
||||
namespace
|
||||
{
|
||||
return FilterName;
|
||||
|
||||
/// Helper method for checking if postings list cache is empty
|
||||
bool hasEmptyPostingsList(const GinPostingsCache & postings_cache)
|
||||
{
|
||||
if (postings_cache.empty())
|
||||
return true;
|
||||
|
||||
for (const auto & term_postings : postings_cache)
|
||||
{
|
||||
const GinSegmentedPostingsListContainer & container = term_postings.second;
|
||||
if (container.empty())
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Helper method to check if the postings list cache has intersection with given row ID range
|
||||
bool matchInRange(const GinPostingsCache & postings_cache, UInt32 segment_id, UInt32 range_start, UInt32 range_end)
|
||||
{
|
||||
/// Check for each term
|
||||
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 GinSegmentedPostingsListContainer & 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 GinPostingsCache & 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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,19 +1,23 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/MergeTree/GinIndexStore.h>
|
||||
#include <vector>
|
||||
#include <memory>
|
||||
#include <Storages/MergeTree/GinIndexStore.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static inline constexpr auto INVERTED_INDEX_NAME = "inverted";
|
||||
|
||||
struct GinFilterParameters
|
||||
{
|
||||
explicit GinFilterParameters(size_t ngrams_, Float64 density_);
|
||||
GinFilterParameters(size_t ngrams_, Float64 density_);
|
||||
|
||||
size_t ngrams;
|
||||
Float64 density;
|
||||
};
|
||||
|
||||
struct GinSegmentWithRowIDRange
|
||||
struct GinSegmentWithRowIdRange
|
||||
{
|
||||
/// Segment ID of the row ID range
|
||||
UInt32 segment_id;
|
||||
@ -25,19 +29,20 @@ struct GinSegmentWithRowIDRange
|
||||
UInt32 range_end;
|
||||
};
|
||||
|
||||
using GinSegmentWithRowIdRangeVector = std::vector<GinSegmentWithRowIdRange>;
|
||||
|
||||
/// 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_);
|
||||
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;
|
||||
/// Add term (located at 'data' with length 'len') 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);
|
||||
@ -45,47 +50,33 @@ public:
|
||||
/// Clear the content
|
||||
void clear();
|
||||
|
||||
/// Check if the filter(built from query string) contains any rows in given filter 'af' by using
|
||||
/// Check if the filter (built from query string) contains any rows in given filter 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; }
|
||||
bool contains(const GinFilter & filter, PostingsCacheForStore & cache_store) const;
|
||||
|
||||
/// Set the query string of the filter
|
||||
void setQueryString(const char* data, size_t len)
|
||||
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)
|
||||
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;}
|
||||
/// Getter
|
||||
const String & getQueryString() const { return query_string; }
|
||||
const std::vector<String> & getTerms() const { return terms; }
|
||||
const GinSegmentWithRowIdRangeVector & getFilter() const { return rowid_ranges; }
|
||||
GinSegmentWithRowIdRangeVector & getFilter() { return rowid_ranges; }
|
||||
|
||||
/// 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;
|
||||
const GinFilterParameters & params;
|
||||
|
||||
/// Query string of the filter
|
||||
String query_string;
|
||||
@ -94,15 +85,12 @@ private:
|
||||
std::vector<String> terms;
|
||||
|
||||
/// Row ID ranges which are (segmentID, RowIDStart, RowIDEnd)
|
||||
GinSegmentWithRowIDRanges rowid_ranges;
|
||||
GinSegmentWithRowIdRangeVector 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);
|
||||
/// Check if the given postings list cache has matched rows by using the filter
|
||||
bool match(const GinPostingsCache & postings_cache) const;
|
||||
};
|
||||
|
||||
using GinFilterPtr = std::shared_ptr<GinFilter>;
|
||||
using GinFilters = std::vector<GinFilter>;
|
||||
|
||||
}
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Core/Block.h>
|
||||
|
||||
#include <Common/MultiVersion.h>
|
||||
#include <Common/SharedMutex.h>
|
||||
|
||||
#include <mutex>
|
||||
|
||||
@ -135,7 +136,7 @@ private:
|
||||
TemporaryDataOnDiskPtr tmp_data;
|
||||
|
||||
Buckets buckets;
|
||||
mutable std::shared_mutex rehash_mutex;
|
||||
mutable SharedMutex rehash_mutex;
|
||||
|
||||
FileBucket * current_bucket = nullptr;
|
||||
mutable std::mutex current_bucket_mutex;
|
||||
|
@ -77,12 +77,15 @@ class ITokenExtractorHelper : public ITokenExtractor
|
||||
{
|
||||
size_t cur = 0;
|
||||
String token;
|
||||
|
||||
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;
|
||||
@ -94,6 +97,7 @@ class ITokenExtractorHelper : public ITokenExtractor
|
||||
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;
|
||||
@ -105,8 +109,10 @@ class ITokenExtractorHelper : public ITokenExtractor
|
||||
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());
|
||||
}
|
||||
|
@ -683,12 +683,13 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
|
||||
for (const auto & index : create.columns_list->indices->children)
|
||||
{
|
||||
IndexDescription index_desc = IndexDescription::getIndexFromAST(index->clone(), properties.columns, getContext());
|
||||
if (index_desc.type == GinFilter::FilterName && getContext()->getSettingsRef().allow_experimental_inverted_index == false)
|
||||
const auto & settings = getContext()->getSettingsRef();
|
||||
if (index_desc.type == INVERTED_INDEX_NAME && !settings.allow_experimental_inverted_index)
|
||||
{
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
|
||||
"Experimental Inverted Index feature is not enabled (the setting 'allow_experimental_inverted_index')");
|
||||
}
|
||||
if (index_desc.type == "annoy" && !getContext()->getSettingsRef().allow_experimental_annoy_index)
|
||||
if (index_desc.type == "annoy" && !settings.allow_experimental_annoy_index)
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index is disabled. Turn on allow_experimental_annoy_index");
|
||||
|
||||
properties.indices.push_back(index_desc);
|
||||
|
@ -287,6 +287,10 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(const String & table_name,
|
||||
table->drop();
|
||||
table->is_dropped = true;
|
||||
}
|
||||
else if (kind == ASTDropQuery::Kind::Detach)
|
||||
{
|
||||
table->is_detached = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -7,11 +7,11 @@
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/ActionBlocker.h>
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <base/types.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <map>
|
||||
#include <shared_mutex>
|
||||
#include <utility>
|
||||
|
||||
namespace zkutil
|
||||
@ -43,7 +43,7 @@ public:
|
||||
|
||||
/// You need to stop the data transfer if blocker is activated.
|
||||
ActionBlocker blocker;
|
||||
std::shared_mutex rwlock;
|
||||
SharedMutex rwlock;
|
||||
};
|
||||
|
||||
using InterserverIOEndpointPtr = std::shared_ptr<InterserverIOEndpoint>;
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <shared_mutex>
|
||||
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <Common/CacheBase.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/SortDescription.h>
|
||||
@ -72,7 +71,7 @@ private:
|
||||
|
||||
using Cache = CacheBase<size_t, Block, std::hash<size_t>, BlockByteWeight>;
|
||||
|
||||
mutable std::shared_mutex rwlock;
|
||||
mutable SharedMutex rwlock;
|
||||
std::shared_ptr<TableJoin> table_join;
|
||||
SizeLimits size_limits;
|
||||
SortDescription left_sort_description;
|
||||
|
@ -25,7 +25,6 @@
|
||||
#include <map>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <shared_mutex>
|
||||
#include <unordered_map>
|
||||
#include <vector>
|
||||
|
||||
|
@ -176,7 +176,7 @@ bool Set::insertFromBlock(const ColumnsWithTypeAndName & columns)
|
||||
|
||||
bool Set::insertFromBlock(const Columns & columns)
|
||||
{
|
||||
std::lock_guard<std::shared_mutex> lock(rwlock);
|
||||
std::lock_guard lock(rwlock);
|
||||
|
||||
if (data.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Method Set::setHeader must be called before Set::insertFromBlock");
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <shared_mutex>
|
||||
#include <Core/Block.h>
|
||||
#include <QueryPipeline/SizeLimits.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
@ -8,6 +7,7 @@
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Storages/MergeTree/BoolMask.h>
|
||||
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
|
||||
@ -131,7 +131,7 @@ private:
|
||||
/** Protects work with the set in the functions `insertFromBlock` and `execute`.
|
||||
* These functions can be called simultaneously from different threads only when using StorageSet,
|
||||
*/
|
||||
mutable std::shared_mutex rwlock;
|
||||
mutable SharedMutex rwlock;
|
||||
|
||||
template <typename Method>
|
||||
void insertFromBlockImpl(
|
||||
|
@ -208,73 +208,8 @@ struct CustomizeAggregateFunctionsMoveSuffixData
|
||||
}
|
||||
};
|
||||
|
||||
struct FuseSumCountAggregates
|
||||
{
|
||||
std::vector<ASTFunction *> sums {};
|
||||
std::vector<ASTFunction *> counts {};
|
||||
std::vector<ASTFunction *> avgs {};
|
||||
|
||||
void addFuncNode(ASTFunction * func)
|
||||
{
|
||||
if (func->name == "sum")
|
||||
sums.push_back(func);
|
||||
else if (func->name == "count")
|
||||
counts.push_back(func);
|
||||
else
|
||||
{
|
||||
assert(func->name == "avg");
|
||||
avgs.push_back(func);
|
||||
}
|
||||
}
|
||||
|
||||
bool canBeFused() const
|
||||
{
|
||||
// Need at least two different kinds of functions to fuse.
|
||||
if (sums.empty() && counts.empty())
|
||||
return false;
|
||||
if (sums.empty() && avgs.empty())
|
||||
return false;
|
||||
if (counts.empty() && avgs.empty())
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
struct FuseSumCountAggregatesVisitorData
|
||||
{
|
||||
using TypeToVisit = ASTFunction;
|
||||
|
||||
std::unordered_map<String, FuseSumCountAggregates> fuse_map;
|
||||
|
||||
void visit(ASTFunction & func, ASTPtr &)
|
||||
{
|
||||
if (func.name == "sum" || func.name == "avg" || func.name == "count")
|
||||
{
|
||||
if (func.arguments->children.empty())
|
||||
return;
|
||||
|
||||
// Probably we can extend it to match count() for non-nullable argument
|
||||
// to sum/avg with any other argument. Now we require strict match.
|
||||
const auto argument = func.arguments->children.at(0)->getColumnName();
|
||||
auto it = fuse_map.find(argument);
|
||||
if (it != fuse_map.end())
|
||||
{
|
||||
it->second.addFuncNode(&func);
|
||||
}
|
||||
else
|
||||
{
|
||||
FuseSumCountAggregates funcs{};
|
||||
funcs.addFuncNode(&func);
|
||||
fuse_map[argument] = funcs;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
using CustomizeAggregateFunctionsOrNullVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeAggregateFunctionsSuffixData>, true>;
|
||||
using CustomizeAggregateFunctionsMoveOrNullVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeAggregateFunctionsMoveSuffixData>, true>;
|
||||
using FuseSumCountAggregatesVisitor = InDepthNodeVisitor<OneTypeMatcher<FuseSumCountAggregatesVisitorData>, true>;
|
||||
|
||||
|
||||
struct ExistsExpressionData
|
||||
{
|
||||
@ -376,52 +311,6 @@ void translateQualifiedNames(ASTPtr & query, const ASTSelectQuery & select_query
|
||||
throw Exception(ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED, "Empty list of columns in SELECT query");
|
||||
}
|
||||
|
||||
// Replaces one avg/sum/count function with an appropriate expression with
|
||||
// sumCount().
|
||||
void replaceWithSumCount(String column_name, ASTFunction & func)
|
||||
{
|
||||
auto func_base = makeASTFunction("sumCount", std::make_shared<ASTIdentifier>(column_name));
|
||||
auto exp_list = std::make_shared<ASTExpressionList>();
|
||||
if (func.name == "sum" || func.name == "count")
|
||||
{
|
||||
/// Rewrite "sum" to sumCount().1, rewrite "count" to sumCount().2
|
||||
UInt8 idx = (func.name == "sum" ? 1 : 2);
|
||||
func.name = "tupleElement";
|
||||
exp_list->children.push_back(func_base);
|
||||
exp_list->children.push_back(std::make_shared<ASTLiteral>(idx));
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Rewrite "avg" to sumCount().1 / sumCount().2
|
||||
auto new_arg1 = makeASTFunction("tupleElement", func_base, std::make_shared<ASTLiteral>(UInt8(1)));
|
||||
auto new_arg2 = makeASTFunction("CAST",
|
||||
makeASTFunction("tupleElement", func_base, std::make_shared<ASTLiteral>(static_cast<UInt8>(2))),
|
||||
std::make_shared<ASTLiteral>("Float64"));
|
||||
|
||||
func.name = "divide";
|
||||
exp_list->children.push_back(new_arg1);
|
||||
exp_list->children.push_back(new_arg2);
|
||||
}
|
||||
func.arguments = exp_list;
|
||||
func.children.push_back(func.arguments);
|
||||
}
|
||||
|
||||
void fuseSumCountAggregates(std::unordered_map<String, FuseSumCountAggregates> & fuse_map)
|
||||
{
|
||||
for (auto & it : fuse_map)
|
||||
{
|
||||
if (it.second.canBeFused())
|
||||
{
|
||||
for (auto & func: it.second.sums)
|
||||
replaceWithSumCount(it.first, *func);
|
||||
for (auto & func: it.second.avgs)
|
||||
replaceWithSumCount(it.first, *func);
|
||||
for (auto & func: it.second.counts)
|
||||
replaceWithSumCount(it.first, *func);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
bool hasArrayJoin(const ASTPtr & ast)
|
||||
{
|
||||
if (const ASTFunction * function = ast->as<ASTFunction>())
|
||||
@ -1544,17 +1433,6 @@ void TreeRewriter::normalize(
|
||||
CustomizeGlobalNotInVisitor(data_global_not_null_in).visit(query);
|
||||
}
|
||||
|
||||
// Try to fuse sum/avg/count with identical arguments to one sumCount call,
|
||||
// if we have at least two different functions. E.g. we will replace sum(x)
|
||||
// and count(x) with sumCount(x).1 and sumCount(x).2, and sumCount() will
|
||||
// be calculated only once because of CSE.
|
||||
if (settings.optimize_fuse_sum_count_avg && settings.optimize_syntax_fuse_functions)
|
||||
{
|
||||
FuseSumCountAggregatesVisitor::Data data;
|
||||
FuseSumCountAggregatesVisitor(data).visit(query);
|
||||
fuseSumCountAggregates(data.fuse_map);
|
||||
}
|
||||
|
||||
/// Rewrite all aggregate functions to add -OrNull suffix to them
|
||||
if (settings.aggregate_functions_null_for_empty)
|
||||
{
|
||||
|
@ -915,13 +915,12 @@ bool AlterCommands::hasInvertedIndex(const StorageInMemoryMetadata & metadata, C
|
||||
{
|
||||
continue;
|
||||
}
|
||||
if (index.type == GinFilter::FilterName)
|
||||
{
|
||||
if (index.type == INVERTED_INDEX_NAME)
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
void AlterCommands::apply(StorageInMemoryMetadata & metadata, ContextPtr context) const
|
||||
{
|
||||
if (!prepared)
|
||||
|
@ -50,10 +50,10 @@ TableLockHolder IStorage::lockForShare(const String & query_id, const std::chron
|
||||
{
|
||||
TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout);
|
||||
|
||||
if (is_dropped)
|
||||
if (is_dropped || is_detached)
|
||||
{
|
||||
auto table_id = getStorageID();
|
||||
throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped", table_id.database_name, table_id.table_name);
|
||||
throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped or detached", table_id.database_name, table_id.table_name);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
@ -62,7 +62,7 @@ TableLockHolder IStorage::tryLockForShare(const String & query_id, const std::ch
|
||||
{
|
||||
TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout);
|
||||
|
||||
if (is_dropped)
|
||||
if (is_dropped || is_detached)
|
||||
{
|
||||
// Table was dropped while acquiring the lock
|
||||
result = nullptr;
|
||||
@ -81,7 +81,7 @@ IStorage::AlterLockHolder IStorage::lockForAlter(const std::chrono::milliseconds
|
||||
"Possible deadlock avoided. Client should retry.",
|
||||
getStorageID().getFullTableName(), acquire_timeout.count());
|
||||
|
||||
if (is_dropped)
|
||||
if (is_dropped || is_detached)
|
||||
throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} is dropped or detached", getStorageID());
|
||||
|
||||
return lock;
|
||||
@ -93,7 +93,7 @@ TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, cons
|
||||
TableExclusiveLockHolder result;
|
||||
result.drop_lock = tryLockTimed(drop_lock, RWLockImpl::Write, query_id, acquire_timeout);
|
||||
|
||||
if (is_dropped)
|
||||
if (is_dropped || is_detached)
|
||||
throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} is dropped or detached", getStorageID());
|
||||
|
||||
return result;
|
||||
|
@ -19,7 +19,6 @@
|
||||
#include <Common/TypePromotion.h>
|
||||
|
||||
#include <optional>
|
||||
#include <shared_mutex>
|
||||
#include <compare>
|
||||
|
||||
|
||||
@ -562,6 +561,7 @@ public:
|
||||
virtual void onActionLockRemove(StorageActionBlockType /* action_type */) {}
|
||||
|
||||
std::atomic<bool> is_dropped{false};
|
||||
std::atomic<bool> is_detached{false};
|
||||
|
||||
/// Does table support index for IN sections
|
||||
virtual bool supportsIndexForIn() const { return false; }
|
||||
|
@ -470,7 +470,6 @@ void StorageLiveView::drop()
|
||||
DatabaseCatalog::instance().removeViewDependency(select_table_id, table_id);
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
is_dropped = true;
|
||||
condition.notify_all();
|
||||
}
|
||||
|
||||
|
@ -735,10 +735,7 @@ std::unique_ptr<WriteBufferFromFileBase> DataPartStorageOnDisk::writeFile(
|
||||
size_t buf_size,
|
||||
const WriteSettings & settings)
|
||||
{
|
||||
if (transaction)
|
||||
return transaction->writeFile(fs::path(root_path) / part_dir / name, buf_size, WriteMode::Rewrite, settings, /* autocommit = */ false);
|
||||
|
||||
return volume->getDisk()->writeFile(fs::path(root_path) / part_dir / name, buf_size, WriteMode::Rewrite, settings);
|
||||
return writeFile(name, buf_size, WriteMode::Rewrite, settings);
|
||||
}
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DataPartStorageOnDisk::writeFile(
|
||||
@ -749,6 +746,7 @@ std::unique_ptr<WriteBufferFromFileBase> DataPartStorageOnDisk::writeFile(
|
||||
{
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -106,17 +106,17 @@ public:
|
||||
const String & name,
|
||||
size_t buf_size,
|
||||
const WriteSettings & settings) override;
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeFile(
|
||||
const String & name,
|
||||
size_t buf_size,
|
||||
DB::WriteMode mode,
|
||||
const WriteSettings & settings) override;
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeTransactionFile(WriteMode mode) const override;
|
||||
|
||||
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;
|
||||
|
@ -1,115 +1,84 @@
|
||||
#include <Storages/MergeTree/GinIndexStore.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Common/FST.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromVector.h>
|
||||
#include <IO/WriteHelpers.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)
|
||||
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;
|
||||
else
|
||||
{
|
||||
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;
|
||||
/// reset the postings list with MATCH ALWAYS;
|
||||
rowid_lst_length = 1; /// makes sure useRoaring() returns false;
|
||||
rowid_lst[0] = CONTAINS_ALL; /// 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)
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < rowid_lst_length; i++)
|
||||
rowid_bitmap.add(rowid_lst[i]);
|
||||
assert(rowid_lst_length < MIN_SIZE_FOR_ROARING_ENCODING);
|
||||
rowid_lst[rowid_lst_length] = row_id;
|
||||
rowid_lst_length++;
|
||||
|
||||
rowid_lst_length = UsesBitMap;
|
||||
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 = USES_BIT_MAP;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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 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
|
||||
if (useRoaring())
|
||||
{
|
||||
auto size = rowid_bitmap.getSizeInBytes();
|
||||
|
||||
@ -121,65 +90,85 @@ UInt64 GinIndexPostingsBuilder::serialize(WriteBuffer &buffer) const
|
||||
buffer.write(buf.get(), size);
|
||||
written_bytes += size;
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < rowid_lst_length; ++i)
|
||||
{
|
||||
writeVarUInt(rowid_lst[i], buffer);
|
||||
written_bytes += getLengthOfVarUInt(rowid_lst[i]);
|
||||
}
|
||||
}
|
||||
|
||||
return written_bytes;
|
||||
}
|
||||
|
||||
GinIndexPostingsListPtr GinIndexPostingsBuilder::deserialize(ReadBuffer &buffer)
|
||||
GinIndexPostingsListPtr GinIndexPostingsBuilder::deserialize(ReadBuffer & buffer)
|
||||
{
|
||||
UInt8 postings_list_size = 0;
|
||||
buffer.readStrict(reinterpret_cast<char&>(postings_list_size));
|
||||
buffer.readStrict(reinterpret_cast<char &>(postings_list_size));
|
||||
|
||||
if (postings_list_size != UsesBitMap)
|
||||
if (postings_list_size == USES_BIT_MAP)
|
||||
{
|
||||
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;
|
||||
}
|
||||
else
|
||||
{
|
||||
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;
|
||||
}
|
||||
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_)
|
||||
{
|
||||
}
|
||||
|
||||
bool GinIndexStore::exists() const
|
||||
{
|
||||
String id_file_name = getName() + GIN_SEGMENT_ID_FILE_TYPE;
|
||||
return storage->exists(id_file_name);
|
||||
String segment_id_file_name = getName() + GIN_SEGMENT_ID_FILE_TYPE;
|
||||
return storage->exists(segment_id_file_name);
|
||||
}
|
||||
|
||||
UInt32 GinIndexStore::getNextSegmentIDRange(const String& file_name, size_t n)
|
||||
UInt32 GinIndexStore::getNextSegmentIDRange(const String & file_name, size_t n)
|
||||
{
|
||||
std::lock_guard guard(gin_index_store_mutex);
|
||||
std::lock_guard guard(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.
|
||||
/// 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
|
||||
/// Create file
|
||||
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);
|
||||
|
||||
/// Write segment ID 1
|
||||
writeVarUInt(1, *ostr);
|
||||
ostr->sync();
|
||||
}
|
||||
|
||||
/// read id in file
|
||||
/// Read id in file
|
||||
UInt32 result = 0;
|
||||
{
|
||||
std::unique_ptr<DB::ReadBufferFromFileBase> istr = this->storage->readFile(file_name, {}, std::nullopt, std::nullopt);
|
||||
@ -189,7 +178,8 @@ UInt32 GinIndexStore::getNextSegmentIDRange(const String& file_name, size_t n)
|
||||
|
||||
readVarUInt(result, *istr);
|
||||
}
|
||||
//save result+n
|
||||
|
||||
/// Save result + n
|
||||
{
|
||||
std::unique_ptr<DB::WriteBufferFromFileBase> ostr = this->data_part_storage_builder->writeFile(file_name, DBMS_DEFAULT_BUFFER_SIZE, {});
|
||||
|
||||
@ -204,15 +194,15 @@ UInt32 GinIndexStore::getNextSegmentIDRange(const String& file_name, size_t n)
|
||||
|
||||
UInt32 GinIndexStore::getNextRowIDRange(size_t numIDs)
|
||||
{
|
||||
UInt32 result =current_segment.next_row_id;
|
||||
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);
|
||||
String segment_id_file_name = getName() + GIN_SEGMENT_ID_FILE_TYPE;
|
||||
return getNextSegmentIDRange(segment_id_file_name, 1);
|
||||
}
|
||||
|
||||
UInt32 GinIndexStore::getNumOfSegments()
|
||||
@ -220,18 +210,18 @@ 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))
|
||||
String segment_id_file_name = getName() + GIN_SEGMENT_ID_FILE_TYPE;
|
||||
if (!storage->exists(segment_id_file_name))
|
||||
return 0;
|
||||
|
||||
UInt32 result = 0;
|
||||
{
|
||||
std::unique_ptr<DB::ReadBufferFromFileBase> istr = this->storage->readFile(sid_file_name, {}, std::nullopt, std::nullopt);
|
||||
std::unique_ptr<DB::ReadBufferFromFileBase> istr = this->storage->readFile(segment_id_file_name, {}, std::nullopt, std::nullopt);
|
||||
|
||||
uint8_t version = 0;
|
||||
readBinary(version, *istr);
|
||||
|
||||
if (version > CURRENT_GIN_FILE_FORMAT_VERSION)
|
||||
if (version > static_cast<std::underlying_type_t<Format>>(CURRENT_GIN_FILE_FORMAT_VERSION))
|
||||
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported inverted index version {}", version);
|
||||
|
||||
readVarUInt(result, *istr);
|
||||
@ -250,88 +240,84 @@ bool GinIndexStore::needToWrite() const
|
||||
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 metadata_file_name = getName() + GIN_SEGMENT_METADATA_FILE_TYPE;
|
||||
String 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, {});
|
||||
metadata_file_stream = data_part_storage_builder->writeFile(metadata_file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {});
|
||||
dict_file_stream = data_part_storage_builder->writeFile(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)
|
||||
{
|
||||
if (metadata_file_stream == nullptr)
|
||||
initFileStreams();
|
||||
}
|
||||
|
||||
using TokenPostingsBuilderPair = std::pair<std::string_view, GinIndexPostingsBuilderPtr>;
|
||||
using TokenPostingsBuilderPairs = std::vector<TokenPostingsBuilderPair>;
|
||||
|
||||
/// Write segment
|
||||
segment_file_stream->write(reinterpret_cast<char*>(¤t_segment), sizeof(GinIndexSegment));
|
||||
metadata_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)
|
||||
{
|
||||
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)
|
||||
[](const TokenPostingsBuilderPair & x, const TokenPostingsBuilderPair & y)
|
||||
{
|
||||
return a.first < b.first;
|
||||
return x.first < y.first;
|
||||
});
|
||||
|
||||
///write postings
|
||||
/// 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)
|
||||
for (size_t i = 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++;
|
||||
posting_list_byte_sizes[i] = posting_list_byte_size;
|
||||
i++;
|
||||
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);
|
||||
FST::FstBuilder fst_builder(write_buf);
|
||||
|
||||
UInt64 offset = 0;
|
||||
for (size_t current_index = 0; const auto& [token, postings_list] : token_postings_list_pairs)
|
||||
for (size_t i = 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++;
|
||||
fst_builder.add(token, offset);
|
||||
offset += posting_list_byte_sizes[i];
|
||||
i++;
|
||||
}
|
||||
|
||||
builder.build();
|
||||
fst_builder.build();
|
||||
write_buf.finalize();
|
||||
|
||||
/// Write FST size
|
||||
writeVarUInt(buffer.size(), *term_dict_file_stream);
|
||||
current_segment.term_dict_start_offset += getLengthOfVarUInt(buffer.size());
|
||||
writeVarUInt(buffer.size(), *dict_file_stream);
|
||||
current_segment.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();
|
||||
/// Write FST blob
|
||||
dict_file_stream->write(reinterpret_cast<char *>(buffer.data()), buffer.size());
|
||||
current_segment.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();
|
||||
metadata_file_stream->sync();
|
||||
dict_file_stream->sync();
|
||||
postings_file_stream->sync();
|
||||
}
|
||||
|
||||
@ -343,82 +329,79 @@ GinIndexStoreDeserializer::GinIndexStoreDeserializer(const GinIndexStorePtr & st
|
||||
|
||||
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 metadata_file_name = store->getName() + GinIndexStore::GIN_SEGMENT_METADATA_FILE_TYPE;
|
||||
String 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);
|
||||
metadata_file_stream = store->storage->readFile(metadata_file_name, {}, std::nullopt, std::nullopt);
|
||||
dict_file_stream = store->storage->readFile(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();
|
||||
UInt32 num_segments = store->getNumOfSegments();
|
||||
if (num_segments == 0)
|
||||
return;
|
||||
|
||||
using GinIndexSegments = std::vector<GinIndexSegment>;
|
||||
GinIndexSegments segments (num_segments);
|
||||
|
||||
assert(segment_file_stream != nullptr);
|
||||
assert(metadata_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)
|
||||
metadata_file_stream->readStrict(reinterpret_cast<char *>(segments.data()), num_segments * sizeof(GinIndexSegment));
|
||||
for (UInt32 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;
|
||||
auto seg_dict = std::make_shared<GinSegmentDictionary>();
|
||||
seg_dict->postings_start_offset = segments[i].postings_start_offset;
|
||||
seg_dict->dict_start_offset = segments[i].dict_start_offset;
|
||||
store->segment_dictionaries[seg_id] = seg_dict;
|
||||
}
|
||||
}
|
||||
|
||||
void GinIndexStoreDeserializer::readSegmentTermDictionaries()
|
||||
void GinIndexStoreDeserializer::readSegmentDictionaries()
|
||||
{
|
||||
for (UInt32 seg_index = 0; seg_index < store->getNumOfSegments(); ++seg_index)
|
||||
{
|
||||
readSegmentTermDictionary(seg_index);
|
||||
}
|
||||
readSegmentDictionary(seg_index);
|
||||
}
|
||||
|
||||
void GinIndexStoreDeserializer::readSegmentTermDictionary(UInt32 segment_id)
|
||||
void GinIndexStoreDeserializer::readSegmentDictionary(UInt32 segment_id)
|
||||
{
|
||||
/// Check validity of segment_id
|
||||
auto it = store->term_dicts.find(segment_id);
|
||||
if (it == store->term_dicts.end())
|
||||
{
|
||||
auto it = store->segment_dictionaries.find(segment_id);
|
||||
if (it == store->segment_dictionaries.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid segment id {}", segment_id);
|
||||
}
|
||||
|
||||
assert(term_dict_file_stream != nullptr);
|
||||
assert(dict_file_stream != nullptr);
|
||||
|
||||
/// Set file pointer of term dictionary file
|
||||
term_dict_file_stream->seek(it->second->term_dict_start_offset, SEEK_SET);
|
||||
/// Set file pointer of dictionary file
|
||||
dict_file_stream->seek(it->second->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);
|
||||
size_t fst_size = 0;
|
||||
readVarUInt(fst_size, *dict_file_stream);
|
||||
|
||||
/// Read FST content
|
||||
/// Read FST blob
|
||||
it->second->offsets.getData().resize(fst_size);
|
||||
term_dict_file_stream->readStrict(reinterpret_cast<char*>(it->second->offsets.getData().data()), fst_size);
|
||||
dict_file_stream->readStrict(reinterpret_cast<char *>(it->second->offsets.getData().data()), fst_size);
|
||||
}
|
||||
|
||||
SegmentedPostingsListContainer GinIndexStoreDeserializer::readSegmentedPostingsLists(const String& term)
|
||||
GinSegmentedPostingsListContainer GinIndexStoreDeserializer::readSegmentedPostingsLists(const String & term)
|
||||
{
|
||||
assert(postings_file_stream != nullptr);
|
||||
|
||||
SegmentedPostingsListContainer container;
|
||||
for (auto const& seg_term_dict : store->term_dicts)
|
||||
GinSegmentedPostingsListContainer container;
|
||||
for (auto const & seg_dict : store->segment_dictionaries)
|
||||
{
|
||||
auto segment_id = seg_term_dict.first;
|
||||
auto segment_id = seg_dict.first;
|
||||
|
||||
auto [offset, found] = seg_term_dict.second->offsets.getOutput(term);
|
||||
auto [offset, found] = seg_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);
|
||||
postings_file_stream->seek(seg_dict.second->postings_start_offset + offset, SEEK_SET);
|
||||
|
||||
// Read posting list
|
||||
auto postings_list = GinIndexPostingsBuilder::deserialize(*postings_file_stream);
|
||||
@ -427,10 +410,10 @@ SegmentedPostingsListContainer GinIndexStoreDeserializer::readSegmentedPostingsL
|
||||
return container;
|
||||
}
|
||||
|
||||
PostingsCachePtr GinIndexStoreDeserializer::createPostingsCacheFromTerms(const std::vector<String>& terms)
|
||||
GinPostingsCachePtr GinIndexStoreDeserializer::createPostingsCacheFromTerms(const std::vector<String> & terms)
|
||||
{
|
||||
auto postings_cache = std::make_shared<PostingsCache>();
|
||||
for (const auto& term : terms)
|
||||
auto postings_cache = std::make_shared<GinPostingsCache>();
|
||||
for (const auto & term : terms)
|
||||
{
|
||||
// Make sure don't read for duplicated terms
|
||||
if (postings_cache->find(term) != postings_cache->end())
|
||||
@ -442,18 +425,26 @@ PostingsCachePtr GinIndexStoreDeserializer::createPostingsCacheFromTerms(const s
|
||||
return postings_cache;
|
||||
}
|
||||
|
||||
GinIndexStoreFactory& GinIndexStoreFactory::instance()
|
||||
GinPostingsCachePtr PostingsCacheForStore::getPostings(const String & query_string) const
|
||||
{
|
||||
auto it = cache.find(query_string);
|
||||
if (it == cache.end())
|
||||
return nullptr;
|
||||
return it->second;
|
||||
}
|
||||
|
||||
GinIndexStoreFactory & GinIndexStoreFactory::instance()
|
||||
{
|
||||
static GinIndexStoreFactory instance;
|
||||
return instance;
|
||||
}
|
||||
|
||||
GinIndexStorePtr GinIndexStoreFactory::get(const String& name, DataPartStoragePtr storage)
|
||||
GinIndexStorePtr GinIndexStoreFactory::get(const String & name, DataPartStoragePtr storage)
|
||||
{
|
||||
const String& part_path = storage->getRelativePath();
|
||||
const String & part_path = storage->getRelativePath();
|
||||
String key = name + ":" + part_path;
|
||||
|
||||
std::lock_guard lock(stores_mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
GinIndexStores::const_iterator it = stores.find(key);
|
||||
|
||||
if (it == stores.end())
|
||||
@ -464,7 +455,7 @@ GinIndexStorePtr GinIndexStoreFactory::get(const String& name, DataPartStoragePt
|
||||
|
||||
GinIndexStoreDeserializer deserializer(store);
|
||||
deserializer.readSegments();
|
||||
deserializer.readSegmentTermDictionaries();
|
||||
deserializer.readSegmentDictionaries();
|
||||
|
||||
stores[key] = store;
|
||||
|
||||
@ -473,9 +464,9 @@ GinIndexStorePtr GinIndexStoreFactory::get(const String& name, DataPartStoragePt
|
||||
return it->second;
|
||||
}
|
||||
|
||||
void GinIndexStoreFactory::remove(const String& part_path)
|
||||
void GinIndexStoreFactory::remove(const String & part_path)
|
||||
{
|
||||
std::lock_guard lock(stores_mutex);
|
||||
std::lock_guard lock(mutex);
|
||||
for (auto it = stores.begin(); it != stores.end();)
|
||||
{
|
||||
if (it->first.find(part_path) != String::npos)
|
||||
@ -484,4 +475,5 @@ void GinIndexStoreFactory::remove(const String& part_path)
|
||||
++it;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,18 +1,18 @@
|
||||
#pragma once
|
||||
|
||||
#include <array>
|
||||
#include <vector>
|
||||
#include <unordered_map>
|
||||
#include <mutex>
|
||||
#include <Common/FST.h>
|
||||
#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>
|
||||
#include <roaring.hh>
|
||||
#include <array>
|
||||
#include <mutex>
|
||||
#include <unordered_map>
|
||||
#include <vector>
|
||||
|
||||
/// GinIndexStore manages the inverted index for a data part, and it is made up of one or more immutable
|
||||
/// GinIndexStore manages the generalized inverted index ("gin") 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:
|
||||
@ -20,40 +20,30 @@
|
||||
/// 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.
|
||||
/// - dict_start_offset points to the file(.gin_dict) starting position for the segment's dictionaries.
|
||||
/// 3. Dictionary file(.gin_dict): it contains 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.
|
||||
/// the starting position of its 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.
|
||||
/// Build a postings list for a term
|
||||
class GinIndexPostingsBuilder
|
||||
{
|
||||
public:
|
||||
constexpr static int MIN_SIZE_FOR_ROARING_ENCODING = 16;
|
||||
|
||||
GinIndexPostingsBuilder(UInt64 limit);
|
||||
explicit GinIndexPostingsBuilder(UInt64 limit);
|
||||
|
||||
/// Check whether a row_id is already added
|
||||
bool contains(UInt32 row_id) const;
|
||||
@ -61,91 +51,88 @@ public:
|
||||
/// 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;
|
||||
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);
|
||||
static GinIndexPostingsListPtr deserialize(ReadBuffer & buffer);
|
||||
|
||||
private:
|
||||
constexpr static int MIN_SIZE_FOR_ROARING_ENCODING = 16;
|
||||
|
||||
/// When the list length is no greater than MIN_SIZE_FOR_ROARING_ENCODING, array 'rowid_lst' is used
|
||||
/// As a special case, rowid_lst[0] == CONTAINS_ALL encodes that all rowids are set.
|
||||
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
|
||||
/// 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};
|
||||
UInt8 rowid_lst_length = 0;
|
||||
|
||||
/// Indicates that all rowids are contained, see 'rowid_lst'
|
||||
static constexpr UInt32 CONTAINS_ALL = std::numeric_limits<UInt32>::max();
|
||||
|
||||
/// Indicates that roaring bitmap is used, see 'rowid_lst_length'.
|
||||
static constexpr UInt8 USES_BIT_MAP = 0xFF;
|
||||
|
||||
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;
|
||||
|
||||
/// Check whether the builder is using roaring bitmap
|
||||
bool useRoaring() const { return rowid_lst_length == USES_BIT_MAP; }
|
||||
|
||||
/// Check whether the postings list has been flagged to contain all row ids
|
||||
bool containsAllRows() const { return rowid_lst[0] == CONTAINS_ALL; }
|
||||
};
|
||||
|
||||
/// Container for postings lists for each segment
|
||||
using SegmentedPostingsListContainer = std::unordered_map<UInt32, GinIndexPostingsListPtr>;
|
||||
using GinIndexPostingsBuilderPtr = std::shared_ptr<GinIndexPostingsBuilder>;
|
||||
|
||||
/// 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:
|
||||
/// Gin index segment descriptor, 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
|
||||
/// Start 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;
|
||||
/// .gin_dict file offset of this segment's dictionaries
|
||||
UInt64 dict_start_offset = 0;
|
||||
};
|
||||
|
||||
using GinIndexSegments = std::vector<GinIndexSegment>;
|
||||
|
||||
struct SegmentTermDictionary
|
||||
struct GinSegmentDictionary
|
||||
{
|
||||
/// .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;
|
||||
/// .gin_dict file offset of this segment's dictionaries
|
||||
UInt64 dict_start_offset;
|
||||
|
||||
/// Finite State Transducer, which can be viewed as a map of <term, offset>, where offset is the
|
||||
/// (Minimized) 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>;
|
||||
using GinSegmentDictionaryPtr = std::shared_ptr<GinSegmentDictionary>;
|
||||
|
||||
/// 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
|
||||
/// Gin index store which has gin index meta data for the corresponding column 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_);
|
||||
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
|
||||
/// Get a range of next 'numIDs'-many available row IDs
|
||||
UInt32 getNextRowIDRange(size_t numIDs);
|
||||
|
||||
/// Get next available segment ID by updating file .gin_sid
|
||||
@ -155,25 +142,26 @@ public:
|
||||
UInt32 getNumOfSegments();
|
||||
|
||||
/// Get current postings list builder
|
||||
const GinIndexPostingsBuilderContainer& getPostings() const { return current_postings; }
|
||||
const GinIndexPostingsBuilderContainer & getPostingsListBuilder() 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;}
|
||||
UInt32 getCurrentSegmentID() const { return current_segment.segment_id; }
|
||||
|
||||
/// Do last segment writing
|
||||
void finalize();
|
||||
|
||||
/// method for writing segment data to Gin index files
|
||||
/// Method for writing segment data to Gin index files
|
||||
void writeSegment();
|
||||
|
||||
const String & getName() const {return name;}
|
||||
const String & getName() const { return name; }
|
||||
|
||||
private:
|
||||
friend class GinIndexStoreDeserializer;
|
||||
@ -182,7 +170,7 @@ private:
|
||||
void initFileStreams();
|
||||
|
||||
/// Get a range of next available segment IDs by updating file .gin_sid
|
||||
UInt32 getNextSegmentIDRange(const String &file_name, size_t n);
|
||||
UInt32 getNextSegmentIDRange(const String & file_name, size_t n);
|
||||
|
||||
String name;
|
||||
DataPartStoragePtr storage;
|
||||
@ -190,37 +178,89 @@ private:
|
||||
|
||||
UInt32 cached_segment_num = 0;
|
||||
|
||||
std::mutex gin_index_store_mutex;
|
||||
std::mutex mutex;
|
||||
|
||||
/// Terms dictionaries which are loaded from .gin_dict files
|
||||
SegmentTermDictionaries term_dicts;
|
||||
/// Dictionaries indexed by segment ID
|
||||
using GinSegmentDictionaries = std::unordered_map<UInt32, GinSegmentDictionaryPtr>;
|
||||
|
||||
/// container for building postings lists during index construction
|
||||
/// Term's dictionaries which are loaded from .gin_dict files
|
||||
GinSegmentDictionaries segment_dictionaries;
|
||||
|
||||
/// Container for building postings lists during index construction
|
||||
GinIndexPostingsBuilderContainer current_postings;
|
||||
|
||||
/// The following is for segmentation of Gin index
|
||||
GinIndexSegment current_segment{};
|
||||
/// For the segmentation of Gin indexes
|
||||
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;
|
||||
/// File streams for segment, dictionaries and postings lists
|
||||
std::unique_ptr<WriteBufferFromFileBase> metadata_file_stream;
|
||||
std::unique_ptr<WriteBufferFromFileBase> 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_SEGMENT_METADATA_FILE_TYPE = ".gin_seg";
|
||||
static constexpr auto GIN_DICTIONARY_FILE_TYPE = ".gin_dict";
|
||||
static constexpr auto GIN_POSTINGS_FILE_TYPE = ".gin_post";
|
||||
|
||||
enum class Format : uint8_t
|
||||
{
|
||||
v0 = 0,
|
||||
v1 = 1, /// Initial version
|
||||
};
|
||||
|
||||
static constexpr auto CURRENT_GIN_FILE_FORMAT_VERSION = Format::v0;
|
||||
};
|
||||
|
||||
using GinIndexStorePtr = std::shared_ptr<GinIndexStore>;
|
||||
|
||||
/// GinIndexStores indexed by part file path
|
||||
using GinIndexStores = std::unordered_map<std::string, GinIndexStorePtr>;
|
||||
/// Container for postings lists for each segment
|
||||
using GinSegmentedPostingsListContainer = std::unordered_map<UInt32, GinIndexPostingsListPtr>;
|
||||
|
||||
/// Postings lists and terms built from query string
|
||||
using GinPostingsCache = std::unordered_map<std::string, GinSegmentedPostingsListContainer>;
|
||||
using GinPostingsCachePtr = std::shared_ptr<GinPostingsCache>;
|
||||
|
||||
/// Gin index store reader which helps to read segments, dictionaries and postings list
|
||||
class GinIndexStoreDeserializer : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
explicit GinIndexStoreDeserializer(const GinIndexStorePtr & store_);
|
||||
|
||||
/// Read segment information from .gin_seg files
|
||||
void readSegments();
|
||||
|
||||
/// Read all dictionaries from .gin_dict files
|
||||
void readSegmentDictionaries();
|
||||
|
||||
/// Read dictionary for given segment id
|
||||
void readSegmentDictionary(UInt32 segment_id);
|
||||
|
||||
/// Read postings lists for the term
|
||||
GinSegmentedPostingsListContainer readSegmentedPostingsLists(const String & term);
|
||||
|
||||
/// Read postings lists for terms (which are created by tokenzing query string)
|
||||
GinPostingsCachePtr 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> metadata_file_stream;
|
||||
std::unique_ptr<ReadBufferFromFileBase> dict_file_stream;
|
||||
std::unique_ptr<ReadBufferFromFileBase> postings_file_stream;
|
||||
|
||||
/// Current segment, used in building index
|
||||
GinIndexSegment current_segment;
|
||||
};
|
||||
|
||||
/// 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
|
||||
/// GinPostingsCache 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
|
||||
@ -229,76 +269,31 @@ struct PostingsCacheForStore
|
||||
GinIndexStorePtr store;
|
||||
|
||||
/// map of <query, postings lists>
|
||||
std::unordered_map<String, PostingsCachePtr> cache;
|
||||
std::unordered_map<String, GinPostingsCachePtr> 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;
|
||||
}
|
||||
GinPostingsCachePtr getPostings(const String & query_string) const;
|
||||
};
|
||||
|
||||
/// GinIndexStore Factory, which is a singleton for storing GinIndexStores
|
||||
/// A singleton for storing GinIndexStores
|
||||
class GinIndexStoreFactory : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
/// Get singleton of GinIndexStoreFactory
|
||||
static GinIndexStoreFactory& instance();
|
||||
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);
|
||||
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);
|
||||
void remove(const String & part_path);
|
||||
|
||||
/// GinIndexStores indexed by part file path
|
||||
using GinIndexStores = std::unordered_map<std::string, GinIndexStorePtr>;
|
||||
|
||||
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;
|
||||
std::mutex mutex;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -216,7 +216,11 @@ 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;
|
||||
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
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include "IO/WriteSettings.h"
|
||||
#include <IO/WriteSettings.h>
|
||||
#include <Core/Block.h>
|
||||
#include <base/types.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
@ -22,8 +22,6 @@
|
||||
#include <DataTypes/Serializations/SerializationInfo.h>
|
||||
#include <Storages/MergeTree/IPartMetadataManager.h>
|
||||
|
||||
#include <shared_mutex>
|
||||
|
||||
|
||||
namespace zkutil
|
||||
{
|
||||
|
@ -2787,10 +2787,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
|
||||
}
|
||||
|
||||
if (commands.hasInvertedIndex(new_metadata, getContext()) && !settings.allow_experimental_inverted_index)
|
||||
{
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
|
||||
"Experimental Inverted Index feature is not enabled (the setting 'allow_experimental_inverted_index')");
|
||||
}
|
||||
"Experimental Inverted Index feature is not enabled (turn on setting 'allow_experimental_inverted_index')");
|
||||
|
||||
commands.apply(new_metadata, getContext());
|
||||
|
||||
/// Set of columns that shouldn't be altered.
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <base/defines.h>
|
||||
#include <Common/SimpleIncrement.h>
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <Common/MultiVersion.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
@ -1089,7 +1090,7 @@ protected:
|
||||
MultiVersion<MergeTreeSettings> storage_settings;
|
||||
|
||||
/// Used to determine which UUIDs to send to root query executor for deduplication.
|
||||
mutable std::shared_mutex pinned_part_uuids_mutex;
|
||||
mutable SharedMutex pinned_part_uuids_mutex;
|
||||
PinnedPartUUIDsPtr pinned_part_uuids;
|
||||
|
||||
/// True if at least one part was created/removed with transaction.
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexGin.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexInverted.h>
|
||||
#include <Common/MemoryTrackerBlockerInThread.h>
|
||||
#include <utility>
|
||||
#include "IO/WriteBufferFromFileDecorator.h"
|
||||
@ -215,7 +215,7 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices()
|
||||
settings.query_write_settings));
|
||||
|
||||
GinIndexStorePtr store = nullptr;
|
||||
if (dynamic_cast<const MergeTreeIndexGinFilter *>(&*index_helper) != nullptr)
|
||||
if (dynamic_cast<const MergeTreeIndexInverted *>(&*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;
|
||||
@ -276,15 +276,13 @@ 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)
|
||||
GinIndexStorePtr store;
|
||||
if (dynamic_cast<const MergeTreeIndexInverted *>(&*index_helper) != nullptr)
|
||||
{
|
||||
String stream_name = index_helper->getFileName();
|
||||
auto it = gin_index_stores.find(stream_name);
|
||||
if (it == gin_index_stores.cend())
|
||||
{
|
||||
if (it == gin_index_stores.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Index '{}' does not exist", stream_name);
|
||||
}
|
||||
store = it->second;
|
||||
}
|
||||
|
||||
@ -401,9 +399,7 @@ void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(bool sync)
|
||||
stream->sync();
|
||||
}
|
||||
for (auto & store: gin_index_stores)
|
||||
{
|
||||
store.second->finalize();
|
||||
}
|
||||
gin_index_stores.clear();
|
||||
skip_indices_streams.clear();
|
||||
skip_indices_aggregators.clear();
|
||||
|
@ -162,7 +162,7 @@ protected:
|
||||
/// Data is already written up to this mark.
|
||||
size_t current_mark = 0;
|
||||
|
||||
GinIndexStores gin_index_stores;
|
||||
GinIndexStoreFactory::GinIndexStores gin_index_stores;
|
||||
private:
|
||||
void initSkipIndices();
|
||||
void initPrimaryIndex();
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <Storages/MergeTree/KeyCondition.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
|
||||
#include <Storages/MergeTree/StorageFromMergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexGin.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexInverted.h>
|
||||
#include <Storages/ReadInOrderOptimizer.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
@ -1689,10 +1689,8 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
|
||||
|
||||
PostingsCacheForStore cache_in_store;
|
||||
|
||||
if (dynamic_cast<const MergeTreeIndexGinFilter *>(&*index_helper) != nullptr)
|
||||
{
|
||||
if (dynamic_cast<const MergeTreeIndexInverted *>(&*index_helper) != nullptr)
|
||||
cache_in_store.store = GinIndexStoreFactory::instance().get(index_helper->getFileName(), part->getDataPartStoragePtr());
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < ranges.size(); ++i)
|
||||
{
|
||||
@ -1707,7 +1705,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);
|
||||
const auto * gin_filter_condition = dynamic_cast<const MergeTreeConditionInverted *>(&*condition);
|
||||
// Cast to Ann condition
|
||||
auto ann_condition = std::dynamic_pointer_cast<ApproximateNearestNeighbour::IMergeTreeIndexConditionAnn>(condition);
|
||||
if (ann_condition != nullptr)
|
||||
@ -1734,7 +1732,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
|
||||
continue;
|
||||
}
|
||||
|
||||
bool result{false};
|
||||
bool result = false;
|
||||
if (!gin_filter_condition)
|
||||
result = condition->mayBeTrueOnGranule(granule);
|
||||
else
|
||||
|
@ -230,6 +230,7 @@ bool MergeTreeConditionFullText::alwaysUnknownOrTrue() const
|
||||
return rpn_stack[0];
|
||||
}
|
||||
|
||||
/// Keep in-sync with MergeTreeIndexConditionGin::mayBeTrueOnTranuleInPart
|
||||
bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const
|
||||
{
|
||||
std::shared_ptr<MergeTreeIndexGranuleFullText> granule
|
||||
|
@ -1,30 +1,28 @@
|
||||
|
||||
#include <algorithm>
|
||||
#include <Storages/MergeTree/MergeTreeIndexInverted.h>
|
||||
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Interpreters/GinFilter.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/GinFilter.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>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexUtils.h>
|
||||
#include <Storages/MergeTree/RPNBuilder.h>
|
||||
#include <algorithm>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -35,19 +33,18 @@ namespace ErrorCodes
|
||||
extern const int INCORRECT_QUERY;
|
||||
}
|
||||
|
||||
MergeTreeIndexGranuleGinFilter::MergeTreeIndexGranuleGinFilter(
|
||||
MergeTreeIndexGranuleInverted::MergeTreeIndexGranuleInverted(
|
||||
const String & index_name_,
|
||||
size_t columns_number,
|
||||
const GinFilterParameters & params_)
|
||||
: index_name(index_name_)
|
||||
, params(params_)
|
||||
, gin_filters(
|
||||
columns_number, GinFilter(params))
|
||||
, gin_filters(columns_number, GinFilter(params))
|
||||
, has_elems(false)
|
||||
{
|
||||
}
|
||||
|
||||
void MergeTreeIndexGranuleGinFilter::serializeBinary(WriteBuffer & ostr) const
|
||||
void MergeTreeIndexGranuleInverted::serializeBinary(WriteBuffer & ostr) const
|
||||
{
|
||||
if (empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty fulltext index {}.", backQuote(index_name));
|
||||
@ -59,11 +56,11 @@ void MergeTreeIndexGranuleGinFilter::serializeBinary(WriteBuffer & ostr) const
|
||||
{
|
||||
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));
|
||||
ostr.write(reinterpret_cast<const char *>(gin_filter.getFilter().data()), filter_size * sizeof(GinSegmentWithRowIdRangeVector::value_type));
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeIndexGranuleGinFilter::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version)
|
||||
void MergeTreeIndexGranuleInverted::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version)
|
||||
{
|
||||
if (version != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version);
|
||||
@ -81,13 +78,13 @@ void MergeTreeIndexGranuleGinFilter::deserializeBinary(ReadBuffer & istr, MergeT
|
||||
continue;
|
||||
|
||||
gin_filter.getFilter().assign(filter_size, {});
|
||||
istr.readStrict(reinterpret_cast<char*>(gin_filter.getFilter().data()), filter_size * sizeof(GinFilter::GinSegmentWithRowIDRanges::value_type));
|
||||
istr.readStrict(reinterpret_cast<char *>(gin_filter.getFilter().data()), filter_size * sizeof(GinSegmentWithRowIdRangeVector::value_type));
|
||||
}
|
||||
has_elems = true;
|
||||
}
|
||||
|
||||
|
||||
MergeTreeIndexAggregatorGinFilter::MergeTreeIndexAggregatorGinFilter(
|
||||
MergeTreeIndexAggregatorInverted::MergeTreeIndexAggregatorInverted(
|
||||
GinIndexStorePtr store_,
|
||||
const Names & index_columns_,
|
||||
const String & index_name_,
|
||||
@ -99,32 +96,30 @@ MergeTreeIndexAggregatorGinFilter::MergeTreeIndexAggregatorGinFilter(
|
||||
, params(params_)
|
||||
, token_extractor(token_extractor_)
|
||||
, granule(
|
||||
std::make_shared<MergeTreeIndexGranuleGinFilter>(
|
||||
std::make_shared<MergeTreeIndexGranuleInverted>(
|
||||
index_name, index_columns.size(), params))
|
||||
{
|
||||
}
|
||||
|
||||
MergeTreeIndexGranulePtr MergeTreeIndexAggregatorGinFilter::getGranuleAndReset()
|
||||
MergeTreeIndexGranulePtr MergeTreeIndexAggregatorInverted::getGranuleAndReset()
|
||||
{
|
||||
auto new_granule = std::make_shared<MergeTreeIndexGranuleGinFilter>(
|
||||
auto new_granule = std::make_shared<MergeTreeIndexGranuleInverted>(
|
||||
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)
|
||||
void MergeTreeIndexAggregatorInverted::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)
|
||||
void MergeTreeIndexAggregatorInverted::update(const Block & block, size_t * pos, size_t limit)
|
||||
{
|
||||
if (*pos >= block.rows())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. "
|
||||
@ -188,7 +183,7 @@ void MergeTreeIndexAggregatorGinFilter::update(const Block & block, size_t * pos
|
||||
*pos += rows_read;
|
||||
}
|
||||
|
||||
MergeTreeConditionGinFilter::MergeTreeConditionGinFilter(
|
||||
MergeTreeConditionInverted::MergeTreeConditionInverted(
|
||||
const SelectQueryInfo & query_info,
|
||||
ContextPtr context_,
|
||||
const Block & index_sample_block,
|
||||
@ -235,7 +230,7 @@ MergeTreeConditionGinFilter::MergeTreeConditionGinFilter(
|
||||
}
|
||||
|
||||
/// Keep in-sync with MergeTreeConditionFullText::alwaysUnknownOrTrue
|
||||
bool MergeTreeConditionGinFilter::alwaysUnknownOrTrue() const
|
||||
bool MergeTreeConditionInverted::alwaysUnknownOrTrue() const
|
||||
{
|
||||
/// Check like in KeyCondition.
|
||||
std::vector<bool> rpn_stack;
|
||||
@ -282,10 +277,10 @@ bool MergeTreeConditionGinFilter::alwaysUnknownOrTrue() const
|
||||
return rpn_stack[0];
|
||||
}
|
||||
|
||||
bool MergeTreeConditionGinFilter::mayBeTrueOnGranuleInPart(MergeTreeIndexGranulePtr idx_granule,[[maybe_unused]] PostingsCacheForStore &cache_store) const
|
||||
bool MergeTreeConditionInverted::mayBeTrueOnGranuleInPart(MergeTreeIndexGranulePtr idx_granule,[[maybe_unused]] PostingsCacheForStore & cache_store) const
|
||||
{
|
||||
std::shared_ptr<MergeTreeIndexGranuleGinFilter> granule
|
||||
= std::dynamic_pointer_cast<MergeTreeIndexGranuleGinFilter>(idx_granule);
|
||||
std::shared_ptr<MergeTreeIndexGranuleInverted> granule
|
||||
= std::dynamic_pointer_cast<MergeTreeIndexGranuleInverted>(idx_granule);
|
||||
if (!granule)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "GinFilter index condition got a granule with the wrong type.");
|
||||
|
||||
@ -373,7 +368,7 @@ bool MergeTreeConditionGinFilter::mayBeTrueOnGranuleInPart(MergeTreeIndexGranule
|
||||
return rpn_stack[0].can_be_true;
|
||||
}
|
||||
|
||||
bool MergeTreeConditionGinFilter::traverseAtomAST(const RPNBuilderTreeNode & node, RPNElement & out)
|
||||
bool MergeTreeConditionInverted::traverseAtomAST(const RPNBuilderTreeNode & node, RPNElement & out)
|
||||
{
|
||||
{
|
||||
Field const_value;
|
||||
@ -453,7 +448,7 @@ bool MergeTreeConditionGinFilter::traverseAtomAST(const RPNBuilderTreeNode & nod
|
||||
return false;
|
||||
}
|
||||
|
||||
bool MergeTreeConditionGinFilter::traverseASTEquals(
|
||||
bool MergeTreeConditionInverted::traverseASTEquals(
|
||||
const String & function_name,
|
||||
const RPNBuilderTreeNode & key_ast,
|
||||
const DataTypePtr & value_type,
|
||||
@ -606,7 +601,7 @@ bool MergeTreeConditionGinFilter::traverseASTEquals(
|
||||
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;
|
||||
std::vector<GinFilters> gin_filters;
|
||||
gin_filters.emplace_back();
|
||||
for (const auto & element : const_value.get<Array>())
|
||||
{
|
||||
@ -624,7 +619,7 @@ bool MergeTreeConditionGinFilter::traverseASTEquals(
|
||||
return false;
|
||||
}
|
||||
|
||||
bool MergeTreeConditionGinFilter::tryPrepareSetGinFilter(
|
||||
bool MergeTreeConditionInverted::tryPrepareSetGinFilter(
|
||||
const RPNBuilderTreeNode & lhs,
|
||||
const RPNBuilderTreeNode & rhs,
|
||||
RPNElement & out)
|
||||
@ -667,7 +662,7 @@ bool MergeTreeConditionGinFilter::tryPrepareSetGinFilter(
|
||||
if (data_type->getTypeId() != TypeIndex::String && data_type->getTypeId() != TypeIndex::FixedString)
|
||||
return false;
|
||||
|
||||
std::vector<std::vector<GinFilter>> gin_filters;
|
||||
std::vector<GinFilters> gin_filters;
|
||||
std::vector<size_t> key_position;
|
||||
|
||||
Columns columns = prepared_set->getSetElements();
|
||||
@ -693,55 +688,55 @@ bool MergeTreeConditionGinFilter::tryPrepareSetGinFilter(
|
||||
return true;
|
||||
}
|
||||
|
||||
MergeTreeIndexGranulePtr MergeTreeIndexGinFilter::createIndexGranule() const
|
||||
MergeTreeIndexGranulePtr MergeTreeIndexInverted::createIndexGranule() const
|
||||
{
|
||||
return std::make_shared<MergeTreeIndexGranuleGinFilter>(index.name, index.column_names.size(), params);
|
||||
return std::make_shared<MergeTreeIndexGranuleInverted>(index.name, index.column_names.size(), params);
|
||||
}
|
||||
|
||||
MergeTreeIndexAggregatorPtr MergeTreeIndexGinFilter::createIndexAggregator() const
|
||||
MergeTreeIndexAggregatorPtr MergeTreeIndexInverted::createIndexAggregator() const
|
||||
{
|
||||
/// should not be called: createIndexAggregatorForPart should be used
|
||||
assert(false);
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
MergeTreeIndexAggregatorPtr MergeTreeIndexGinFilter::createIndexAggregatorForPart(const GinIndexStorePtr &store) const
|
||||
MergeTreeIndexAggregatorPtr MergeTreeIndexInverted::createIndexAggregatorForPart(const GinIndexStorePtr & store) const
|
||||
{
|
||||
return std::make_shared<MergeTreeIndexAggregatorGinFilter>(store, index.column_names, index.name, params, token_extractor.get());
|
||||
return std::make_shared<MergeTreeIndexAggregatorInverted>(store, index.column_names, index.name, params, token_extractor.get());
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexGinFilter::createIndexCondition(
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexInverted::createIndexCondition(
|
||||
const SelectQueryInfo & query, ContextPtr context) const
|
||||
{
|
||||
return std::make_shared<MergeTreeConditionGinFilter>(query, context, index.sample_block, params, token_extractor.get());
|
||||
return std::make_shared<MergeTreeConditionInverted>(query, context, index.sample_block, params, token_extractor.get());
|
||||
};
|
||||
|
||||
bool MergeTreeIndexGinFilter::mayBenefitFromIndexForIn(const ASTPtr & node) const
|
||||
bool MergeTreeIndexInverted::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(
|
||||
MergeTreeIndexPtr invertedIndexCreator(
|
||||
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>();
|
||||
Float64 density = index.arguments.size() < 2 ? 1.0 : 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));
|
||||
return std::make_shared<MergeTreeIndexInverted>(index, params, std::move(tokenizer));
|
||||
}
|
||||
else
|
||||
{
|
||||
auto tokenizer = std::make_unique<SplitTokenExtractor>();
|
||||
return std::make_shared<MergeTreeIndexGinFilter>(index, params, std::move(tokenizer));
|
||||
return std::make_shared<MergeTreeIndexInverted>(index, params, std::move(tokenizer));
|
||||
}
|
||||
}
|
||||
|
||||
void ginIndexValidator(const IndexDescription & index, bool /*attach*/)
|
||||
void invertedIndexValidator(const IndexDescription & index, bool /*attach*/)
|
||||
{
|
||||
for (const auto & index_data_type : index.data_types)
|
||||
{
|
||||
@ -764,9 +759,6 @@ void ginIndexValidator(const IndexDescription & index, bool /*attach*/)
|
||||
"column or Array with `String` or `FixedString` values column.");
|
||||
}
|
||||
|
||||
if (index.type != GinFilter::getName())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index type: {}", backQuote(index.name));
|
||||
|
||||
if (index.arguments.size() > 2)
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY, "Inverted index must have less than two arguments.");
|
||||
|
||||
@ -776,10 +768,9 @@ void ginIndexValidator(const IndexDescription & index, bool /*attach*/)
|
||||
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(ErrorCodes::INCORRECT_QUERY, "The second Inverted index argument must be a float between 0 and 1.");
|
||||
|
||||
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
|
||||
size_t ngrams = index.arguments.empty() ? 0 : index.arguments[0].get<size_t>();
|
||||
Float64 density = index.arguments.size() < 2 ? 1.0 : index.arguments[1].get<Float64>();
|
||||
GinFilterParameters params(ngrams, density);
|
||||
}
|
||||
|
@ -1,24 +1,23 @@
|
||||
#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>
|
||||
#include <Interpreters/ITokenExtractor.h>
|
||||
#include <Storages/MergeTree/KeyCondition.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <base/types.h>
|
||||
#include <atomic>
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct MergeTreeIndexGranuleGinFilter final : public IMergeTreeIndexGranule
|
||||
struct MergeTreeIndexGranuleInverted final : public IMergeTreeIndexGranule
|
||||
{
|
||||
explicit MergeTreeIndexGranuleGinFilter(
|
||||
explicit MergeTreeIndexGranuleInverted(
|
||||
const String & index_name_,
|
||||
size_t columns_number,
|
||||
const GinFilterParameters & params_);
|
||||
|
||||
~MergeTreeIndexGranuleGinFilter() override = default;
|
||||
~MergeTreeIndexGranuleInverted() override = default;
|
||||
|
||||
void serializeBinary(WriteBuffer & ostr) const override;
|
||||
void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override;
|
||||
@ -27,30 +26,29 @@ struct MergeTreeIndexGranuleGinFilter final : public IMergeTreeIndexGranule
|
||||
|
||||
String index_name;
|
||||
GinFilterParameters params;
|
||||
|
||||
std::vector<GinFilter> gin_filters;
|
||||
GinFilters gin_filters;
|
||||
bool has_elems;
|
||||
};
|
||||
|
||||
using MergeTreeIndexGranuleGinFilterPtr = std::shared_ptr<MergeTreeIndexGranuleGinFilter>;
|
||||
using MergeTreeIndexGranuleInvertedPtr = std::shared_ptr<MergeTreeIndexGranuleInverted>;
|
||||
|
||||
struct MergeTreeIndexAggregatorGinFilter final : IMergeTreeIndexAggregator
|
||||
struct MergeTreeIndexAggregatorInverted final : IMergeTreeIndexAggregator
|
||||
{
|
||||
explicit MergeTreeIndexAggregatorGinFilter(
|
||||
explicit MergeTreeIndexAggregatorInverted(
|
||||
GinIndexStorePtr store_,
|
||||
const Names & index_columns_,
|
||||
const String & index_name_,
|
||||
const GinFilterParameters & params_,
|
||||
TokenExtractorPtr token_extractor_);
|
||||
|
||||
~MergeTreeIndexAggregatorGinFilter() override = default;
|
||||
~MergeTreeIndexAggregatorInverted() 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);
|
||||
void addToGinFilter(UInt32 rowID, const char * data, size_t length, GinFilter & gin_filter, UInt64 limit);
|
||||
|
||||
GinIndexStorePtr store;
|
||||
Names index_columns;
|
||||
@ -58,21 +56,21 @@ struct MergeTreeIndexAggregatorGinFilter final : IMergeTreeIndexAggregator
|
||||
const GinFilterParameters params;
|
||||
TokenExtractorPtr token_extractor;
|
||||
|
||||
MergeTreeIndexGranuleGinFilterPtr granule;
|
||||
MergeTreeIndexGranuleInvertedPtr granule;
|
||||
};
|
||||
|
||||
|
||||
class MergeTreeConditionGinFilter final : public IMergeTreeIndexCondition, WithContext
|
||||
class MergeTreeConditionInverted final : public IMergeTreeIndexCondition, WithContext
|
||||
{
|
||||
public:
|
||||
MergeTreeConditionGinFilter(
|
||||
MergeTreeConditionInverted(
|
||||
const SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
const Block & index_sample_block,
|
||||
const GinFilterParameters & params_,
|
||||
TokenExtractorPtr token_extactor_);
|
||||
|
||||
~MergeTreeConditionGinFilter() override = default;
|
||||
~MergeTreeConditionInverted() override = default;
|
||||
|
||||
bool alwaysUnknownOrTrue() const override;
|
||||
bool mayBeTrueOnGranule([[maybe_unused]]MergeTreeIndexGranulePtr idx_granule) const override
|
||||
@ -81,7 +79,8 @@ public:
|
||||
assert(false);
|
||||
return false;
|
||||
}
|
||||
bool mayBeTrueOnGranuleInPart(MergeTreeIndexGranulePtr idx_granule, [[maybe_unused]] PostingsCacheForStore& cache_store) const;
|
||||
bool mayBeTrueOnGranuleInPart(MergeTreeIndexGranulePtr idx_granule, [[maybe_unused]] PostingsCacheForStore & cache_store) const;
|
||||
|
||||
private:
|
||||
struct KeyTuplePositionMapping
|
||||
{
|
||||
@ -124,7 +123,7 @@ private:
|
||||
std::unique_ptr<GinFilter> gin_filter;
|
||||
|
||||
/// For FUNCTION_IN, FUNCTION_NOT_IN and FUNCTION_MULTI_SEARCH
|
||||
std::vector<std::vector<GinFilter>> set_gin_filters;
|
||||
std::vector<GinFilters> set_gin_filters;
|
||||
|
||||
/// For FUNCTION_IN and FUNCTION_NOT_IN
|
||||
std::vector<size_t> set_key_position;
|
||||
@ -154,10 +153,10 @@ private:
|
||||
PreparedSetsPtr prepared_sets;
|
||||
};
|
||||
|
||||
class MergeTreeIndexGinFilter final : public IMergeTreeIndex
|
||||
class MergeTreeIndexInverted final : public IMergeTreeIndex
|
||||
{
|
||||
public:
|
||||
MergeTreeIndexGinFilter(
|
||||
MergeTreeIndexInverted(
|
||||
const IndexDescription & index_,
|
||||
const GinFilterParameters & params_,
|
||||
std::unique_ptr<ITokenExtractor> && token_extractor_)
|
||||
@ -165,13 +164,12 @@ public:
|
||||
, params(params_)
|
||||
, token_extractor(std::move(token_extractor_)) {}
|
||||
|
||||
~MergeTreeIndexGinFilter() override = default;
|
||||
~MergeTreeIndexInverted() 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;
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregatorForPart(const GinIndexStorePtr & store) const override;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override;
|
||||
|
||||
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
|
||||
|
@ -105,8 +105,9 @@ MergeTreeIndexFactory::MergeTreeIndexFactory()
|
||||
registerCreator("annoy", annoyIndexCreator);
|
||||
registerValidator("annoy", annoyIndexValidator);
|
||||
#endif
|
||||
registerCreator("inverted", ginIndexCreator);
|
||||
registerValidator("inverted", ginIndexValidator);
|
||||
|
||||
registerCreator("inverted", invertedIndexCreator);
|
||||
registerValidator("inverted", invertedIndexValidator);
|
||||
|
||||
}
|
||||
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <mutex>
|
||||
#include <Core/Block.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <Storages/MergeTree/GinIndexStore.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/MergeTree/MarkRange.h>
|
||||
@ -16,7 +17,6 @@
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
|
||||
#include <Storages/MergeTree/GinIndexStore.h>
|
||||
|
||||
constexpr auto INDEX_FILE_PREFIX = "skp_idx_";
|
||||
|
||||
@ -237,7 +237,8 @@ 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);
|
||||
|
||||
MergeTreeIndexPtr invertedIndexCreator(const IndexDescription& index);
|
||||
void invertedIndexValidator(const IndexDescription& index, bool attach);
|
||||
|
||||
}
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/unit.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataFormatVersion.h>
|
||||
@ -25,7 +26,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) \
|
||||
M(UInt64, max_digestion_size_per_segment, 256_MiB, "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) \
|
||||
|
@ -33,7 +33,6 @@
|
||||
|
||||
#include <cstddef>
|
||||
#include <filesystem>
|
||||
#include <shared_mutex>
|
||||
#include <utility>
|
||||
|
||||
|
||||
@ -493,7 +492,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
|
||||
std::shared_ptr<rocksdb::Statistics> StorageEmbeddedRocksDB::getRocksDBStatistics() const
|
||||
{
|
||||
std::shared_lock<std::shared_mutex> lock(rocksdb_ptr_mx);
|
||||
std::shared_lock lock(rocksdb_ptr_mx);
|
||||
if (!rocksdb_ptr)
|
||||
return nullptr;
|
||||
return rocksdb_ptr->GetOptions().statistics;
|
||||
@ -501,7 +500,7 @@ std::shared_ptr<rocksdb::Statistics> StorageEmbeddedRocksDB::getRocksDBStatistic
|
||||
|
||||
std::vector<rocksdb::Status> StorageEmbeddedRocksDB::multiGet(const std::vector<rocksdb::Slice> & slices_keys, std::vector<String> & values) const
|
||||
{
|
||||
std::shared_lock<std::shared_mutex> lock(rocksdb_ptr_mx);
|
||||
std::shared_lock lock(rocksdb_ptr_mx);
|
||||
if (!rocksdb_ptr)
|
||||
return {};
|
||||
return rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values);
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
#include <shared_mutex>
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Interpreters/IKeyValueEntity.h>
|
||||
#include <rocksdb/status.h>
|
||||
@ -86,7 +86,7 @@ private:
|
||||
const String primary_key;
|
||||
using RocksDBPtr = std::unique_ptr<rocksdb::DB>;
|
||||
RocksDBPtr rocksdb_ptr;
|
||||
mutable std::shared_mutex rocksdb_ptr_mx;
|
||||
mutable SharedMutex rocksdb_ptr_mx;
|
||||
String rocksdb_dir;
|
||||
Int32 ttl;
|
||||
bool read_only;
|
||||
|
@ -49,6 +49,7 @@
|
||||
#include <unistd.h>
|
||||
#include <re2/re2.h>
|
||||
#include <filesystem>
|
||||
#include <shared_mutex>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -1442,11 +1442,11 @@ void StorageWindowView::writeIntoWindowView(
|
||||
});
|
||||
}
|
||||
|
||||
std::shared_lock<std::shared_mutex> fire_signal_lock;
|
||||
std::shared_lock<SharedMutex> fire_signal_lock;
|
||||
QueryPipelineBuilder builder;
|
||||
if (window_view.is_proctime)
|
||||
{
|
||||
fire_signal_lock = std::shared_lock<std::shared_mutex>(window_view.fire_signal_mutex);
|
||||
fire_signal_lock = std::shared_lock(window_view.fire_signal_mutex);
|
||||
|
||||
/// Fill ____timestamp column with current time in case of now() time column.
|
||||
if (window_view.is_time_column_func_now)
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
@ -213,7 +214,7 @@ private:
|
||||
|
||||
/// Mutex for the blocks and ready condition
|
||||
std::mutex mutex;
|
||||
std::shared_mutex fire_signal_mutex;
|
||||
SharedMutex fire_signal_mutex;
|
||||
mutable std::mutex sample_block_lock; /// Mutex to protect access to sample block
|
||||
|
||||
IntervalKind::Kind window_kind;
|
||||
|
@ -232,19 +232,52 @@ def need_retry(args, stdout, stderr, total_time):
|
||||
)
|
||||
|
||||
|
||||
def get_processlist(args):
|
||||
def get_processlist_with_stacktraces(args):
|
||||
try:
|
||||
if args.replicated_database:
|
||||
return clickhouse_execute_json(
|
||||
args,
|
||||
"""
|
||||
SELECT materialize((hostName(), tcpPort())) as host, *
|
||||
FROM clusterAllReplicas('test_cluster_database_replicated', system.processes)
|
||||
WHERE query NOT LIKE '%system.processes%'
|
||||
SELECT materialize(hostName() || '::' || tcpPort()::String) as host_port, *
|
||||
-- NOTE: view() here to do JOIN on shards, instead of initiator
|
||||
FROM clusterAllReplicas('test_cluster_database_replicated', view(
|
||||
SELECT
|
||||
groupArray((s.thread_id, arrayStringConcat(arrayMap(
|
||||
x -> concat(addressToLine(x), '::', demangle(addressToSymbol(x))),
|
||||
s.trace), '\n') AS stacktrace
|
||||
)) AS stacktraces,
|
||||
p.*
|
||||
FROM system.processes p
|
||||
JOIN system.stack_trace s USING (query_id)
|
||||
WHERE query NOT LIKE '%system.processes%'
|
||||
GROUP BY p.*
|
||||
))
|
||||
ORDER BY elapsed DESC
|
||||
""",
|
||||
settings={
|
||||
"allow_introspection_functions": 1,
|
||||
},
|
||||
)
|
||||
else:
|
||||
return clickhouse_execute_json(args, "SHOW PROCESSLIST")
|
||||
return clickhouse_execute_json(
|
||||
args,
|
||||
"""
|
||||
SELECT
|
||||
groupArray((s.thread_id, arrayStringConcat(arrayMap(
|
||||
x -> concat(addressToLine(x), '::', demangle(addressToSymbol(x))),
|
||||
s.trace), '\n') AS stacktrace
|
||||
)) AS stacktraces,
|
||||
p.*
|
||||
FROM system.processes p
|
||||
JOIN system.stack_trace s USING (query_id)
|
||||
WHERE query NOT LIKE '%system.processes%'
|
||||
GROUP BY p.*
|
||||
ORDER BY elapsed DESC
|
||||
""",
|
||||
settings={
|
||||
"allow_introspection_functions": 1,
|
||||
},
|
||||
)
|
||||
except Exception as e:
|
||||
return "Failed to get processlist: " + str(e)
|
||||
|
||||
@ -1223,7 +1256,7 @@ class TestSuite:
|
||||
line = line.strip()
|
||||
if line and not is_shebang(line):
|
||||
return line
|
||||
return ''
|
||||
return ""
|
||||
|
||||
def load_tags_from_file(filepath):
|
||||
comment_sign = get_comment_sign(filepath)
|
||||
@ -1750,7 +1783,7 @@ def removesuffix(text, *suffixes):
|
||||
return text
|
||||
|
||||
|
||||
def reportCoverageFor(args, what, query, permissive = False):
|
||||
def reportCoverageFor(args, what, query, permissive=False):
|
||||
value = clickhouse_execute(args, query).decode()
|
||||
|
||||
if value != "":
|
||||
@ -1763,10 +1796,11 @@ def reportCoverageFor(args, what, query, permissive = False):
|
||||
|
||||
|
||||
def reportCoverage(args):
|
||||
return reportCoverageFor(
|
||||
args,
|
||||
"functions",
|
||||
"""
|
||||
return (
|
||||
reportCoverageFor(
|
||||
args,
|
||||
"functions",
|
||||
"""
|
||||
SELECT name
|
||||
FROM system.functions
|
||||
WHERE NOT is_aggregate AND origin = 'System' AND alias_to = ''
|
||||
@ -1776,11 +1810,12 @@ def reportCoverage(args):
|
||||
)
|
||||
ORDER BY name
|
||||
""",
|
||||
True
|
||||
) and reportCoverageFor(
|
||||
args,
|
||||
"aggregate functions",
|
||||
"""
|
||||
True,
|
||||
)
|
||||
and reportCoverageFor(
|
||||
args,
|
||||
"aggregate functions",
|
||||
"""
|
||||
SELECT name
|
||||
FROM system.functions
|
||||
WHERE is_aggregate AND origin = 'System' AND alias_to = ''
|
||||
@ -1789,11 +1824,12 @@ def reportCoverage(args):
|
||||
SELECT arrayJoin(used_aggregate_functions) FROM system.query_log WHERE event_date >= yesterday()
|
||||
)
|
||||
ORDER BY name
|
||||
"""
|
||||
) and reportCoverageFor(
|
||||
args,
|
||||
"aggregate function combinators",
|
||||
"""
|
||||
""",
|
||||
)
|
||||
and reportCoverageFor(
|
||||
args,
|
||||
"aggregate function combinators",
|
||||
"""
|
||||
SELECT name
|
||||
FROM system.aggregate_function_combinators
|
||||
WHERE NOT is_internal
|
||||
@ -1802,11 +1838,12 @@ def reportCoverage(args):
|
||||
SELECT arrayJoin(used_aggregate_function_combinators) FROM system.query_log WHERE event_date >= yesterday()
|
||||
)
|
||||
ORDER BY name
|
||||
"""
|
||||
) and reportCoverageFor(
|
||||
args,
|
||||
"data type families",
|
||||
"""
|
||||
""",
|
||||
)
|
||||
and reportCoverageFor(
|
||||
args,
|
||||
"data type families",
|
||||
"""
|
||||
SELECT name
|
||||
FROM system.data_type_families
|
||||
WHERE alias_to = '' AND name NOT LIKE 'Interval%'
|
||||
@ -1815,7 +1852,8 @@ def reportCoverage(args):
|
||||
SELECT arrayJoin(used_data_type_families) FROM system.query_log WHERE event_date >= yesterday()
|
||||
)
|
||||
ORDER BY name
|
||||
"""
|
||||
""",
|
||||
)
|
||||
)
|
||||
|
||||
def reportLogStats(args):
|
||||
@ -1844,7 +1882,7 @@ def reportLogStats(args):
|
||||
LIMIT 100
|
||||
FORMAT TSVWithNamesAndTypes
|
||||
"""
|
||||
value = clickhouse_execute(args, query).decode()
|
||||
value = clickhouse_execute(args, query).decode(errors="replace")
|
||||
print("\nTop patterns of log messages:\n")
|
||||
print(value)
|
||||
print("\n")
|
||||
@ -1856,7 +1894,7 @@ def reportLogStats(args):
|
||||
count() AS count,
|
||||
substr(replaceRegexpAll(message, '[^A-Za-z]+', ''), 1, 32) AS pattern,
|
||||
substr(any(message), 1, 256) as runtime_message,
|
||||
any((extract(source_file, '\/[a-zA-Z0-9_]+\.[a-z]+'), source_line)) as line
|
||||
any((extract(source_file, '\/[a-zA-Z0-9_]+\.[a-z]+'), source_line)) as line
|
||||
FROM system.text_log
|
||||
WHERE (now() - toIntervalMinute(mins)) < event_time AND message_format_string = ''
|
||||
GROUP BY pattern
|
||||
@ -1864,7 +1902,7 @@ def reportLogStats(args):
|
||||
LIMIT 50
|
||||
FORMAT TSVWithNamesAndTypes
|
||||
"""
|
||||
value = clickhouse_execute(args, query).decode()
|
||||
value = clickhouse_execute(args, query).decode(errors="replace")
|
||||
print("\nTop messages without format string (fmt::runtime):\n")
|
||||
print(value)
|
||||
print("\n")
|
||||
@ -1891,7 +1929,9 @@ def main(args):
|
||||
args, "system", "processes", "is_all_data_sent"
|
||||
)
|
||||
|
||||
if args.s3_storage and (BuildFlags.THREAD in args.build_flags or BuildFlags.DEBUG in args.build_flags):
|
||||
if args.s3_storage and (
|
||||
BuildFlags.THREAD in args.build_flags or BuildFlags.DEBUG in args.build_flags
|
||||
):
|
||||
args.no_random_settings = True
|
||||
|
||||
if args.skip:
|
||||
@ -1963,10 +2003,9 @@ def main(args):
|
||||
exit_code.value = 1
|
||||
|
||||
if args.hung_check:
|
||||
|
||||
# Some queries may execute in background for some time after test was finished. This is normal.
|
||||
for _ in range(1, 60):
|
||||
processlist = get_processlist(args)
|
||||
processlist = get_processlist_with_stacktraces(args)
|
||||
if not processlist:
|
||||
break
|
||||
sleep(1)
|
||||
@ -1980,7 +2019,6 @@ def main(args):
|
||||
print(json.dumps(processlist, indent=4))
|
||||
print(get_transactions_list(args))
|
||||
|
||||
print_stacktraces()
|
||||
exit_code.value = 1
|
||||
else:
|
||||
print(colored("\nNo queries hung.", args, "green", attrs=["bold"]))
|
||||
|
@ -127,7 +127,7 @@ def test_backup_to_s3_multipart():
|
||||
backup_destination = f"S3('http://minio1:9001/root/data/backups/multipart/{backup_name}', 'minio', 'minio123')"
|
||||
check_backup_and_restore(storage_policy, backup_destination, size=1000000)
|
||||
assert node.contains_in_log(
|
||||
f"copyDataToS3: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}"
|
||||
f"copyDataToS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}"
|
||||
)
|
||||
|
||||
|
||||
@ -140,7 +140,7 @@ def test_backup_to_s3_native_copy():
|
||||
check_backup_and_restore(storage_policy, backup_destination)
|
||||
assert node.contains_in_log("using native copy")
|
||||
assert node.contains_in_log(
|
||||
f"copyFileS3ToS3: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}"
|
||||
f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}"
|
||||
)
|
||||
|
||||
|
||||
@ -153,7 +153,7 @@ def test_backup_to_s3_native_copy_other_bucket():
|
||||
check_backup_and_restore(storage_policy, backup_destination)
|
||||
assert node.contains_in_log("using native copy")
|
||||
assert node.contains_in_log(
|
||||
f"copyFileS3ToS3: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}"
|
||||
f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}"
|
||||
)
|
||||
|
||||
|
||||
@ -164,5 +164,5 @@ def test_backup_to_s3_native_copy_multipart():
|
||||
check_backup_and_restore(storage_policy, backup_destination, size=1000000)
|
||||
assert node.contains_in_log("using native copy")
|
||||
assert node.contains_in_log(
|
||||
f"copyFileS3ToS3: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}/"
|
||||
f"copyS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}/"
|
||||
)
|
||||
|
@ -131,24 +131,29 @@ def create_restore_file(node, revision=None, bucket=None, path=None, detached=No
|
||||
["bash", "-c", "touch /var/lib/clickhouse/disks/s3/restore"], user="root"
|
||||
)
|
||||
|
||||
add_restore_option = 'echo -en "{}={}\n" >> /var/lib/clickhouse/disks/s3/restore'
|
||||
if revision:
|
||||
num_restore_options = 0
|
||||
|
||||
def add_restore_option(key, value):
|
||||
nonlocal num_restore_options
|
||||
to = ">>" if num_restore_options else ">"
|
||||
node.exec_in_container(
|
||||
["bash", "-c", add_restore_option.format("revision", revision)], user="root"
|
||||
)
|
||||
if bucket:
|
||||
node.exec_in_container(
|
||||
["bash", "-c", add_restore_option.format("source_bucket", bucket)],
|
||||
[
|
||||
"bash",
|
||||
"-c",
|
||||
f'echo -en "{key}={value}\n" {to} /var/lib/clickhouse/disks/s3/restore',
|
||||
],
|
||||
user="root",
|
||||
)
|
||||
num_restore_options += 1
|
||||
|
||||
if revision:
|
||||
add_restore_option("revision", revision)
|
||||
if bucket:
|
||||
add_restore_option("source_bucket", bucket)
|
||||
if path:
|
||||
node.exec_in_container(
|
||||
["bash", "-c", add_restore_option.format("source_path", path)], user="root"
|
||||
)
|
||||
add_restore_option("source_path", path)
|
||||
if detached:
|
||||
node.exec_in_container(
|
||||
["bash", "-c", add_restore_option.format("detached", "true")], user="root"
|
||||
)
|
||||
add_restore_option("detached", "true")
|
||||
|
||||
|
||||
def get_revision_counter(node, backup_number):
|
||||
|
@ -1,12 +0,0 @@
|
||||
210 230 20
|
||||
SELECT
|
||||
sum(a),
|
||||
sumCount(b).1,
|
||||
sumCount(b).2
|
||||
FROM fuse_tbl
|
||||
---------NOT trigger fuse--------
|
||||
210 11.5
|
||||
SELECT
|
||||
sum(a),
|
||||
avg(b)
|
||||
FROM fuse_tbl
|
@ -1,14 +0,0 @@
|
||||
DROP TABLE IF EXISTS fuse_tbl;
|
||||
CREATE TABLE fuse_tbl(a Int8, b Int8) Engine = Log;
|
||||
INSERT INTO fuse_tbl SELECT number, number + 1 FROM numbers(1, 20);
|
||||
|
||||
SET optimize_syntax_fuse_functions = 1;
|
||||
SET optimize_fuse_sum_count_avg = 1;
|
||||
|
||||
SELECT sum(a), sum(b), count(b) from fuse_tbl;
|
||||
EXPLAIN SYNTAX SELECT sum(a), sum(b), count(b) from fuse_tbl;
|
||||
SELECT '---------NOT trigger fuse--------';
|
||||
SELECT sum(a), avg(b) from fuse_tbl;
|
||||
EXPLAIN SYNTAX SELECT sum(a), avg(b) from fuse_tbl;
|
||||
|
||||
DROP TABLE fuse_tbl;
|
@ -1,2 +0,0 @@
|
||||
0 0 nan
|
||||
0 0 nan
|
@ -1,5 +0,0 @@
|
||||
SELECT sum(x), count(x), avg(x) FROM (SELECT number :: Decimal32(0) AS x FROM numbers(0))
|
||||
SETTINGS optimize_syntax_fuse_functions = 0, optimize_fuse_sum_count_avg = 0;
|
||||
|
||||
SELECT sum(x), count(x), avg(x) FROM (SELECT number :: Decimal32(0) AS x FROM numbers(0))
|
||||
SETTINGS optimize_syntax_fuse_functions = 1, optimize_fuse_sum_count_avg = 1;
|
@ -1,5 +1,5 @@
|
||||
SET allow_experimental_analyzer = 1;
|
||||
SET optimize_syntax_fuse_functions = 1, optimize_fuse_sum_count_avg = 1;
|
||||
SET optimize_syntax_fuse_functions = 1;
|
||||
|
||||
DROP TABLE IF EXISTS fuse_tbl;
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
SET allow_experimental_analyzer = 1;
|
||||
SET optimize_syntax_fuse_functions = 1, optimize_fuse_sum_count_avg = 1;
|
||||
SET optimize_syntax_fuse_functions = 1;
|
||||
|
||||
DROP TABLE IF EXISTS fuse_tbl;
|
||||
|
||||
|
@ -1,10 +0,0 @@
|
||||
DROP TABLE IF EXISTS fuse_tbl__fuzz_35;
|
||||
|
||||
CREATE TABLE fuse_tbl__fuzz_35 (`a` UInt8, `b` Nullable(Int16)) ENGINE = Log;
|
||||
INSERT INTO fuse_tbl__fuzz_35 SELECT number, number + 1 FROM numbers(1000);
|
||||
|
||||
set allow_experimental_analyzer = 0, optimize_syntax_fuse_functions = 1, optimize_fuse_sum_count_avg = 1;
|
||||
|
||||
SELECT quantile(0.5)(b), quantile(0.9)(b) FROM (SELECT x + 2147483648 AS b FROM (SELECT quantile(0.5)(b) AS x FROM fuse_tbl__fuzz_35) GROUP BY x) FORMAT Null;
|
||||
|
||||
DROP TABLE IF EXISTS fuse_tbl__fuzz_35;
|
Loading…
Reference in New Issue
Block a user