mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge remote-tracking branch 'origin/master' into add-tests-24.5-release
This commit is contained in:
commit
ebab4f3af3
20
.github/ISSUE_TEMPLATE/10_question.md
vendored
20
.github/ISSUE_TEMPLATE/10_question.md
vendored
@ -1,20 +0,0 @@
|
|||||||
---
|
|
||||||
name: Question
|
|
||||||
about: Ask a question about ClickHouse
|
|
||||||
title: ''
|
|
||||||
labels: question
|
|
||||||
assignees: ''
|
|
||||||
|
|
||||||
---
|
|
||||||
|
|
||||||
> Make sure to check documentation https://clickhouse.com/docs/en/ first. If the question is concise and probably has a short answer, asking it in [community Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-1gh9ds7f4-PgDhJAaF8ad5RbWBAAjzFg) is probably the fastest way to find the answer. For more complicated questions, consider asking them on StackOverflow with "clickhouse" tag https://stackoverflow.com/questions/tagged/clickhouse
|
|
||||||
|
|
||||||
> If you still prefer GitHub issues, remove all this text and ask your question here.
|
|
||||||
|
|
||||||
**Company or project name**
|
|
||||||
|
|
||||||
Put your company name or project description here
|
|
||||||
|
|
||||||
**Question**
|
|
||||||
|
|
||||||
Your question
|
|
20
.github/ISSUE_TEMPLATE/10_question.yaml
vendored
Normal file
20
.github/ISSUE_TEMPLATE/10_question.yaml
vendored
Normal file
@ -0,0 +1,20 @@
|
|||||||
|
name: Question
|
||||||
|
description: Ask a question about ClickHouse
|
||||||
|
labels: ["question"]
|
||||||
|
body:
|
||||||
|
- type: markdown
|
||||||
|
attributes:
|
||||||
|
value: |
|
||||||
|
> Make sure to check documentation https://clickhouse.com/docs/en/ first. If the question is concise and probably has a short answer, asking it in [community Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-1gh9ds7f4-PgDhJAaF8ad5RbWBAAjzFg) is probably the fastest way to find the answer. For more complicated questions, consider asking them on StackOverflow with "clickhouse" tag https://stackoverflow.com/questions/tagged/clickhouse
|
||||||
|
- type: textarea
|
||||||
|
attributes:
|
||||||
|
label: Company or project name
|
||||||
|
description: Put your company name or project description here.
|
||||||
|
validations:
|
||||||
|
required: false
|
||||||
|
- type: textarea
|
||||||
|
attributes:
|
||||||
|
label: Question
|
||||||
|
description: Please put your question here.
|
||||||
|
validations:
|
||||||
|
required: true
|
@ -25,7 +25,8 @@ azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log &
|
|||||||
./setup_minio.sh stateless # to have a proper environment
|
./setup_minio.sh stateless # to have a proper environment
|
||||||
|
|
||||||
echo "Get previous release tag"
|
echo "Get previous release tag"
|
||||||
previous_release_tag=$(dpkg --info package_folder/clickhouse-client*.deb | grep "Version: " | awk '{print $2}' | cut -f1 -d'+' | get_previous_release_tag)
|
# shellcheck disable=SC2016
|
||||||
|
previous_release_tag=$(dpkg-deb --showformat='${Version}' --show package_folder/clickhouse-client*.deb | get_previous_release_tag)
|
||||||
echo $previous_release_tag
|
echo $previous_release_tag
|
||||||
|
|
||||||
echo "Clone previous release repository"
|
echo "Clone previous release repository"
|
||||||
|
@ -1206,6 +1206,16 @@ Expired time for HSTS in seconds. The default value is 0 means clickhouse disabl
|
|||||||
<hsts_max_age>600000</hsts_max_age>
|
<hsts_max_age>600000</hsts_max_age>
|
||||||
```
|
```
|
||||||
|
|
||||||
|
## mlock_executable {#mlock_executable}
|
||||||
|
|
||||||
|
Perform mlockall after startup to lower first queries latency and to prevent clickhouse executable from being paged out under high IO load. Enabling this option is recommended but will lead to increased startup time for up to a few seconds.
|
||||||
|
Keep in mind that this parameter would not work without "CAP_IPC_LOCK" capability.
|
||||||
|
**Example**
|
||||||
|
|
||||||
|
``` xml
|
||||||
|
<mlock_executable>false</mlock_executable>
|
||||||
|
```
|
||||||
|
|
||||||
## include_from {#include_from}
|
## include_from {#include_from}
|
||||||
|
|
||||||
The path to the file with substitutions. Both XML and YAML formats are supported.
|
The path to the file with substitutions. Both XML and YAML formats are supported.
|
||||||
@ -1353,6 +1363,26 @@ Examples:
|
|||||||
<listen_host>127.0.0.1</listen_host>
|
<listen_host>127.0.0.1</listen_host>
|
||||||
```
|
```
|
||||||
|
|
||||||
|
## listen_try {#listen_try}
|
||||||
|
|
||||||
|
The server will not exit if IPv6 or IPv4 networks are unavailable while trying to listen.
|
||||||
|
|
||||||
|
Examples:
|
||||||
|
|
||||||
|
``` xml
|
||||||
|
<listen_try>0</listen_try>
|
||||||
|
```
|
||||||
|
|
||||||
|
## listen_reuse_port {#listen_reuse_port}
|
||||||
|
|
||||||
|
Allow multiple servers to listen on the same address:port. Requests will be routed to a random server by the operating system. Enabling this setting is not recommended.
|
||||||
|
|
||||||
|
Examples:
|
||||||
|
|
||||||
|
``` xml
|
||||||
|
<listen_reuse_port>0</listen_reuse_port>
|
||||||
|
```
|
||||||
|
|
||||||
## listen_backlog {#listen_backlog}
|
## listen_backlog {#listen_backlog}
|
||||||
|
|
||||||
Backlog (queue size of pending connections) of the listen socket.
|
Backlog (queue size of pending connections) of the listen socket.
|
||||||
|
@ -3170,6 +3170,18 @@ Possible values:
|
|||||||
|
|
||||||
Default value: `0`.
|
Default value: `0`.
|
||||||
|
|
||||||
|
## lightweight_deletes_sync {#lightweight_deletes_sync}
|
||||||
|
|
||||||
|
The same as 'mutation_sync', but controls only execution of lightweight deletes.
|
||||||
|
|
||||||
|
Possible values:
|
||||||
|
|
||||||
|
- 0 - Mutations execute asynchronously.
|
||||||
|
- 1 - The query waits for the lightweight deletes to complete on the current server.
|
||||||
|
- 2 - The query waits for the lightweight deletes to complete on all replicas (if they exist).
|
||||||
|
|
||||||
|
Default value: `2`.
|
||||||
|
|
||||||
**See Also**
|
**See Also**
|
||||||
|
|
||||||
- [Synchronicity of ALTER Queries](../../sql-reference/statements/alter/index.md#synchronicity-of-alter-queries)
|
- [Synchronicity of ALTER Queries](../../sql-reference/statements/alter/index.md#synchronicity-of-alter-queries)
|
||||||
|
@ -139,7 +139,7 @@ For the query to run successfully, the following conditions must be met:
|
|||||||
ALTER TABLE table2 [ON CLUSTER cluster] REPLACE PARTITION partition_expr FROM table1
|
ALTER TABLE table2 [ON CLUSTER cluster] REPLACE PARTITION partition_expr FROM table1
|
||||||
```
|
```
|
||||||
|
|
||||||
This query copies the data partition from the `table1` to `table2` and replaces existing partition in the `table2`.
|
This query copies the data partition from `table1` to `table2` and replaces the existing partition in `table2`. The operation is atomic.
|
||||||
|
|
||||||
Note that:
|
Note that:
|
||||||
|
|
||||||
|
@ -15,6 +15,7 @@ struct MemoryTrackerSwitcher
|
|||||||
return;
|
return;
|
||||||
|
|
||||||
auto * thread_tracker = CurrentThread::getMemoryTracker();
|
auto * thread_tracker = CurrentThread::getMemoryTracker();
|
||||||
|
|
||||||
prev_untracked_memory = current_thread->untracked_memory;
|
prev_untracked_memory = current_thread->untracked_memory;
|
||||||
prev_memory_tracker_parent = thread_tracker->getParent();
|
prev_memory_tracker_parent = thread_tracker->getParent();
|
||||||
|
|
||||||
@ -31,8 +32,10 @@ struct MemoryTrackerSwitcher
|
|||||||
CurrentThread::flushUntrackedMemory();
|
CurrentThread::flushUntrackedMemory();
|
||||||
auto * thread_tracker = CurrentThread::getMemoryTracker();
|
auto * thread_tracker = CurrentThread::getMemoryTracker();
|
||||||
|
|
||||||
current_thread->untracked_memory = prev_untracked_memory;
|
/// It is important to set untracked memory after the call of
|
||||||
|
/// 'setParent' because it may flush untracked memory to the wrong parent.
|
||||||
thread_tracker->setParent(prev_memory_tracker_parent);
|
thread_tracker->setParent(prev_memory_tracker_parent);
|
||||||
|
current_thread->untracked_memory = prev_untracked_memory;
|
||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
@ -160,8 +160,8 @@ class IColumn;
|
|||||||
M(Bool, enable_multiple_prewhere_read_steps, true, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \
|
M(Bool, enable_multiple_prewhere_read_steps, true, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \
|
||||||
M(Bool, move_primary_key_columns_to_end_of_prewhere, true, "Move PREWHERE conditions containing primary key columns to the end of AND chain. It is likely that these conditions are taken into account during primary key analysis and thus will not contribute a lot to PREWHERE filtering.", 0) \
|
M(Bool, move_primary_key_columns_to_end_of_prewhere, true, "Move PREWHERE conditions containing primary key columns to the end of AND chain. It is likely that these conditions are taken into account during primary key analysis and thus will not contribute a lot to PREWHERE filtering.", 0) \
|
||||||
\
|
\
|
||||||
M(Bool, allow_statistics_optimize, false, "Allows using statistics to optimize queries", 0) \
|
M(Bool, allow_statistics_optimize, false, "Allows using statistics to optimize queries", 0) ALIAS(allow_statistic_optimize) \
|
||||||
M(Bool, allow_experimental_statistics, false, "Allows using statistics", 0) \
|
M(Bool, allow_experimental_statistics, false, "Allows using statistics", 0) ALIAS(allow_experimental_statistic) \
|
||||||
\
|
\
|
||||||
M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \
|
M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \
|
||||||
M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \
|
M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \
|
||||||
|
@ -83,7 +83,7 @@ namespace SettingsChangesHistory
|
|||||||
/// For newly added setting choose the most appropriate previous_value (for example, if new setting
|
/// For newly added setting choose the most appropriate previous_value (for example, if new setting
|
||||||
/// controls new feature and it's 'true' by default, use 'false' as previous_value).
|
/// controls new feature and it's 'true' by default, use 'false' as previous_value).
|
||||||
/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972)
|
/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972)
|
||||||
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
|
static const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
|
||||||
{
|
{
|
||||||
{"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"},
|
{"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"},
|
||||||
{"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"},
|
{"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"},
|
||||||
@ -97,6 +97,10 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
|
|||||||
{"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"},
|
{"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"},
|
||||||
{"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"},
|
{"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"},
|
||||||
{"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"},
|
{"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"},
|
||||||
|
{"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."},
|
||||||
|
{"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."},
|
||||||
|
{"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."},
|
||||||
|
{"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}
|
||||||
}},
|
}},
|
||||||
{"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"},
|
{"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"},
|
||||||
{"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."},
|
{"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."},
|
||||||
|
@ -543,6 +543,7 @@ template <typename DataType> constexpr bool IsDataTypeNumber = false;
|
|||||||
template <typename DataType> constexpr bool IsDataTypeDateOrDateTime = false;
|
template <typename DataType> constexpr bool IsDataTypeDateOrDateTime = false;
|
||||||
template <typename DataType> constexpr bool IsDataTypeDate = false;
|
template <typename DataType> constexpr bool IsDataTypeDate = false;
|
||||||
template <typename DataType> constexpr bool IsDataTypeEnum = false;
|
template <typename DataType> constexpr bool IsDataTypeEnum = false;
|
||||||
|
template <typename DataType> constexpr bool IsDataTypeStringOrFixedString = false;
|
||||||
|
|
||||||
template <typename DataType> constexpr bool IsDataTypeDecimalOrNumber = IsDataTypeDecimal<DataType> || IsDataTypeNumber<DataType>;
|
template <typename DataType> constexpr bool IsDataTypeDecimalOrNumber = IsDataTypeDecimal<DataType> || IsDataTypeNumber<DataType>;
|
||||||
|
|
||||||
@ -556,6 +557,8 @@ class DataTypeDate;
|
|||||||
class DataTypeDate32;
|
class DataTypeDate32;
|
||||||
class DataTypeDateTime;
|
class DataTypeDateTime;
|
||||||
class DataTypeDateTime64;
|
class DataTypeDateTime64;
|
||||||
|
class DataTypeString;
|
||||||
|
class DataTypeFixedString;
|
||||||
|
|
||||||
template <is_decimal T> constexpr bool IsDataTypeDecimal<DataTypeDecimal<T>> = true;
|
template <is_decimal T> constexpr bool IsDataTypeDecimal<DataTypeDecimal<T>> = true;
|
||||||
|
|
||||||
@ -572,6 +575,9 @@ template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDate32> = tru
|
|||||||
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDateTime> = true;
|
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDateTime> = true;
|
||||||
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDateTime64> = true;
|
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDateTime64> = true;
|
||||||
|
|
||||||
|
template <> inline constexpr bool IsDataTypeStringOrFixedString<DataTypeString> = true;
|
||||||
|
template <> inline constexpr bool IsDataTypeStringOrFixedString<DataTypeFixedString> = true;
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
class DataTypeEnum;
|
class DataTypeEnum;
|
||||||
|
|
||||||
|
@ -670,7 +670,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat
|
|||||||
for (auto it = metadata_files.begin(); it < metadata_files.end(); std::advance(it, batch_size))
|
for (auto it = metadata_files.begin(); it < metadata_files.end(); std::advance(it, batch_size))
|
||||||
{
|
{
|
||||||
std::span batch{it, std::min(std::next(it, batch_size), metadata_files.end())};
|
std::span batch{it, std::min(std::next(it, batch_size), metadata_files.end())};
|
||||||
pool.scheduleOrThrowOnError(
|
pool.scheduleOrThrow(
|
||||||
[batch, &process_metadata_file, &process_tmp_drop_metadata_file]() mutable
|
[batch, &process_metadata_file, &process_tmp_drop_metadata_file]() mutable
|
||||||
{
|
{
|
||||||
setThreadName("DatabaseOnDisk");
|
setThreadName("DatabaseOnDisk");
|
||||||
@ -679,7 +679,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat
|
|||||||
process_metadata_file(file.first);
|
process_metadata_file(file.first);
|
||||||
else
|
else
|
||||||
process_tmp_drop_metadata_file(file.first);
|
process_tmp_drop_metadata_file(file.first);
|
||||||
});
|
}, Priority{}, getContext()->getSettingsRef().lock_acquire_timeout.totalMicroseconds());
|
||||||
}
|
}
|
||||||
pool.wait();
|
pool.wait();
|
||||||
}
|
}
|
||||||
|
@ -36,30 +36,24 @@ void IObjectStorageIteratorAsync::deactivate()
|
|||||||
void IObjectStorageIteratorAsync::nextBatch()
|
void IObjectStorageIteratorAsync::nextBatch()
|
||||||
{
|
{
|
||||||
std::lock_guard lock(mutex);
|
std::lock_guard lock(mutex);
|
||||||
|
|
||||||
if (is_finished)
|
if (is_finished)
|
||||||
{
|
{
|
||||||
current_batch.clear();
|
current_batch.clear();
|
||||||
current_batch_iterator = current_batch.begin();
|
current_batch_iterator = current_batch.begin();
|
||||||
|
return;
|
||||||
}
|
}
|
||||||
else
|
|
||||||
{
|
|
||||||
if (!is_initialized)
|
if (!is_initialized)
|
||||||
{
|
{
|
||||||
outcome_future = scheduleBatch();
|
outcome_future = scheduleBatch();
|
||||||
is_initialized = true;
|
is_initialized = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
chassert(outcome_future.valid());
|
|
||||||
BatchAndHasNext result;
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
result = outcome_future.get();
|
chassert(outcome_future.valid());
|
||||||
}
|
BatchAndHasNext result = outcome_future.get();
|
||||||
catch (...)
|
|
||||||
{
|
|
||||||
is_finished = true;
|
|
||||||
throw;
|
|
||||||
}
|
|
||||||
|
|
||||||
current_batch = std::move(result.batch);
|
current_batch = std::move(result.batch);
|
||||||
current_batch_iterator = current_batch.begin();
|
current_batch_iterator = current_batch.begin();
|
||||||
@ -71,6 +65,11 @@ void IObjectStorageIteratorAsync::nextBatch()
|
|||||||
else
|
else
|
||||||
is_finished = true;
|
is_finished = true;
|
||||||
}
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
is_finished = true;
|
||||||
|
throw;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void IObjectStorageIteratorAsync::next()
|
void IObjectStorageIteratorAsync::next()
|
||||||
@ -95,35 +94,39 @@ std::future<IObjectStorageIteratorAsync::BatchAndHasNext> IObjectStorageIterator
|
|||||||
|
|
||||||
bool IObjectStorageIteratorAsync::isValid()
|
bool IObjectStorageIteratorAsync::isValid()
|
||||||
{
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
|
||||||
if (!is_initialized)
|
if (!is_initialized)
|
||||||
nextBatch();
|
nextBatch();
|
||||||
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
return current_batch_iterator != current_batch.end();
|
return current_batch_iterator != current_batch.end();
|
||||||
}
|
}
|
||||||
|
|
||||||
RelativePathWithMetadataPtr IObjectStorageIteratorAsync::current()
|
RelativePathWithMetadataPtr IObjectStorageIteratorAsync::current()
|
||||||
{
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
|
||||||
if (!isValid())
|
if (!isValid())
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator");
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator");
|
||||||
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
return *current_batch_iterator;
|
return *current_batch_iterator;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
RelativePathsWithMetadata IObjectStorageIteratorAsync::currentBatch()
|
RelativePathsWithMetadata IObjectStorageIteratorAsync::currentBatch()
|
||||||
{
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
|
||||||
if (!isValid())
|
if (!isValid())
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator");
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator");
|
||||||
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
return current_batch;
|
return current_batch;
|
||||||
}
|
}
|
||||||
|
|
||||||
std::optional<RelativePathsWithMetadata> IObjectStorageIteratorAsync::getCurrentBatchAndScheduleNext()
|
std::optional<RelativePathsWithMetadata> IObjectStorageIteratorAsync::getCurrentBatchAndScheduleNext()
|
||||||
{
|
{
|
||||||
std::lock_guard lock(mutex);
|
std::lock_guard lock(mutex);
|
||||||
|
|
||||||
if (!is_initialized)
|
if (!is_initialized)
|
||||||
nextBatch();
|
nextBatch();
|
||||||
|
|
||||||
|
@ -709,7 +709,7 @@ bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateL
|
|||||||
else
|
else
|
||||||
return tryReadFloatTextFast(x, rb);
|
return tryReadFloatTextFast(x, rb);
|
||||||
}
|
}
|
||||||
else /*if constexpr (is_integer_v<typename DataType::FieldType>)*/
|
else /*if constexpr (is_integral_v<typename DataType::FieldType>)*/
|
||||||
return tryReadIntText(x, rb);
|
return tryReadIntText(x, rb);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -814,6 +814,16 @@ enum class ConvertFromStringParsingMode : uint8_t
|
|||||||
BestEffortUS
|
BestEffortUS
|
||||||
};
|
};
|
||||||
|
|
||||||
|
struct AccurateConvertStrategyAdditions
|
||||||
|
{
|
||||||
|
UInt32 scale { 0 };
|
||||||
|
};
|
||||||
|
|
||||||
|
struct AccurateOrNullConvertStrategyAdditions
|
||||||
|
{
|
||||||
|
UInt32 scale { 0 };
|
||||||
|
};
|
||||||
|
|
||||||
template <typename FromDataType, typename ToDataType, typename Name,
|
template <typename FromDataType, typename ToDataType, typename Name,
|
||||||
ConvertFromStringExceptionMode exception_mode, ConvertFromStringParsingMode parsing_mode>
|
ConvertFromStringExceptionMode exception_mode, ConvertFromStringParsingMode parsing_mode>
|
||||||
struct ConvertThroughParsing
|
struct ConvertThroughParsing
|
||||||
@ -1020,6 +1030,12 @@ struct ConvertThroughParsing
|
|||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
if constexpr (std::is_same_v<Additions, AccurateConvertStrategyAdditions>)
|
||||||
|
{
|
||||||
|
if (!tryParseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone, precise_float_parsing))
|
||||||
|
throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse string to type {}", TypeName<typename ToDataType::FieldType>);
|
||||||
|
}
|
||||||
|
else
|
||||||
parseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone, precise_float_parsing);
|
parseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone, precise_float_parsing);
|
||||||
} while (false);
|
} while (false);
|
||||||
}
|
}
|
||||||
@ -1120,16 +1136,6 @@ struct ConvertThroughParsing
|
|||||||
/// Function toUnixTimestamp has exactly the same implementation as toDateTime of String type.
|
/// Function toUnixTimestamp has exactly the same implementation as toDateTime of String type.
|
||||||
struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; };
|
struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; };
|
||||||
|
|
||||||
struct AccurateConvertStrategyAdditions
|
|
||||||
{
|
|
||||||
UInt32 scale { 0 };
|
|
||||||
};
|
|
||||||
|
|
||||||
struct AccurateOrNullConvertStrategyAdditions
|
|
||||||
{
|
|
||||||
UInt32 scale { 0 };
|
|
||||||
};
|
|
||||||
|
|
||||||
enum class BehaviourOnErrorFromString : uint8_t
|
enum class BehaviourOnErrorFromString : uint8_t
|
||||||
{
|
{
|
||||||
ConvertDefaultBehaviorTag,
|
ConvertDefaultBehaviorTag,
|
||||||
@ -3174,8 +3180,11 @@ private:
|
|||||||
{
|
{
|
||||||
TypeIndex from_type_index = from_type->getTypeId();
|
TypeIndex from_type_index = from_type->getTypeId();
|
||||||
WhichDataType which(from_type_index);
|
WhichDataType which(from_type_index);
|
||||||
|
TypeIndex to_type_index = to_type->getTypeId();
|
||||||
|
WhichDataType to(to_type_index);
|
||||||
bool can_apply_accurate_cast = (cast_type == CastType::accurate || cast_type == CastType::accurateOrNull)
|
bool can_apply_accurate_cast = (cast_type == CastType::accurate || cast_type == CastType::accurateOrNull)
|
||||||
&& (which.isInt() || which.isUInt() || which.isFloat());
|
&& (which.isInt() || which.isUInt() || which.isFloat());
|
||||||
|
can_apply_accurate_cast |= cast_type == CastType::accurate && which.isStringOrFixedString() && to.isNativeInteger();
|
||||||
|
|
||||||
FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior;
|
FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior;
|
||||||
if (context)
|
if (context)
|
||||||
@ -3260,6 +3269,20 @@ private:
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
else if constexpr (IsDataTypeStringOrFixedString<LeftDataType>)
|
||||||
|
{
|
||||||
|
if constexpr (IsDataTypeNumber<RightDataType>)
|
||||||
|
{
|
||||||
|
chassert(wrapper_cast_type == CastType::accurate);
|
||||||
|
result_column = ConvertImpl<LeftDataType, RightDataType, FunctionCastName>::execute(
|
||||||
|
arguments,
|
||||||
|
result_type,
|
||||||
|
input_rows_count,
|
||||||
|
BehaviourOnErrorFromString::ConvertDefaultBehaviorTag,
|
||||||
|
AccurateConvertStrategyAdditions());
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
return false;
|
return false;
|
||||||
});
|
});
|
||||||
|
@ -438,6 +438,12 @@ BlockIO InterpreterGrantQuery::execute()
|
|||||||
RolesOrUsersSet roles_to_revoke;
|
RolesOrUsersSet roles_to_revoke;
|
||||||
collectRolesToGrantOrRevoke(access_control, query, roles_to_grant, roles_to_revoke);
|
collectRolesToGrantOrRevoke(access_control, query, roles_to_grant, roles_to_revoke);
|
||||||
|
|
||||||
|
/// Replacing empty database with the default. This step must be done before replication to avoid privilege escalation.
|
||||||
|
String current_database = getContext()->getCurrentDatabase();
|
||||||
|
elements_to_grant.replaceEmptyDatabase(current_database);
|
||||||
|
elements_to_revoke.replaceEmptyDatabase(current_database);
|
||||||
|
query.access_rights_elements.replaceEmptyDatabase(current_database);
|
||||||
|
|
||||||
/// Executing on cluster.
|
/// Executing on cluster.
|
||||||
if (!query.cluster.empty())
|
if (!query.cluster.empty())
|
||||||
{
|
{
|
||||||
@ -453,9 +459,6 @@ BlockIO InterpreterGrantQuery::execute()
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Check if the current user has corresponding access rights granted with grant option.
|
/// Check if the current user has corresponding access rights granted with grant option.
|
||||||
String current_database = getContext()->getCurrentDatabase();
|
|
||||||
elements_to_grant.replaceEmptyDatabase(current_database);
|
|
||||||
elements_to_revoke.replaceEmptyDatabase(current_database);
|
|
||||||
bool need_check_grantees_are_allowed = true;
|
bool need_check_grantees_are_allowed = true;
|
||||||
if (!query.current_grants)
|
if (!query.current_grants)
|
||||||
checkGrantOption(access_control, *current_user_access, grantees, need_check_grantees_are_allowed, elements_to_grant, elements_to_revoke);
|
checkGrantOption(access_control, *current_user_access, grantees, need_check_grantees_are_allowed, elements_to_grant, elements_to_revoke);
|
||||||
|
@ -240,4 +240,34 @@ bool SplitTokenExtractor::nextInStringLike(const char * data, size_t length, siz
|
|||||||
return !bad_token && !token.empty();
|
return !bad_token && !token.empty();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void SplitTokenExtractor::substringToBloomFilter(const char * data, size_t length, BloomFilter & bloom_filter, bool is_prefix, bool is_suffix) const
|
||||||
|
{
|
||||||
|
size_t cur = 0;
|
||||||
|
size_t token_start = 0;
|
||||||
|
size_t token_len = 0;
|
||||||
|
|
||||||
|
while (cur < length && nextInString(data, length, &cur, &token_start, &token_len))
|
||||||
|
// In order to avoid filter updates with incomplete tokens,
|
||||||
|
// first token is ignored, unless substring is prefix and
|
||||||
|
// last token is ignored, unless substring is suffix
|
||||||
|
if ((token_start > 0 || is_prefix) && (token_start + token_len < length || is_suffix))
|
||||||
|
bloom_filter.add(data + token_start, token_len);
|
||||||
|
}
|
||||||
|
|
||||||
|
void SplitTokenExtractor::substringToGinFilter(const char * data, size_t length, GinFilter & gin_filter, bool is_prefix, bool is_suffix) const
|
||||||
|
{
|
||||||
|
gin_filter.setQueryString(data, length);
|
||||||
|
|
||||||
|
size_t cur = 0;
|
||||||
|
size_t token_start = 0;
|
||||||
|
size_t token_len = 0;
|
||||||
|
|
||||||
|
while (cur < length && nextInString(data, length, &cur, &token_start, &token_len))
|
||||||
|
// In order to avoid filter updates with incomplete tokens,
|
||||||
|
// first token is ignored, unless substring is prefix and
|
||||||
|
// last token is ignored, unless substring is suffix
|
||||||
|
if ((token_start > 0 || is_prefix) && (token_start + token_len < length || is_suffix))
|
||||||
|
gin_filter.addTerm(data + token_start, token_len);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -28,8 +28,22 @@ struct ITokenExtractor
|
|||||||
/// It skips unescaped `%` and `_` and supports escaping symbols, but it is less lightweight.
|
/// It skips unescaped `%` and `_` and supports escaping symbols, but it is less lightweight.
|
||||||
virtual bool nextInStringLike(const char * data, size_t length, size_t * pos, String & out) const = 0;
|
virtual bool nextInStringLike(const char * data, size_t length, size_t * pos, String & out) const = 0;
|
||||||
|
|
||||||
|
/// Updates Bloom filter from exact-match string filter value
|
||||||
virtual void stringToBloomFilter(const char * data, size_t length, BloomFilter & bloom_filter) const = 0;
|
virtual void stringToBloomFilter(const char * data, size_t length, BloomFilter & bloom_filter) const = 0;
|
||||||
|
|
||||||
|
/// Updates Bloom filter from substring-match string filter value.
|
||||||
|
/// An `ITokenExtractor` implementation may decide to skip certain
|
||||||
|
/// tokens depending on whether the substring is a prefix or a suffix.
|
||||||
|
virtual void substringToBloomFilter(
|
||||||
|
const char * data,
|
||||||
|
size_t length,
|
||||||
|
BloomFilter & bloom_filter,
|
||||||
|
bool is_prefix [[maybe_unused]],
|
||||||
|
bool is_suffix [[maybe_unused]]) const
|
||||||
|
{
|
||||||
|
stringToBloomFilter(data, length, bloom_filter);
|
||||||
|
}
|
||||||
|
|
||||||
virtual void stringPaddedToBloomFilter(const char * data, size_t length, BloomFilter & bloom_filter) const
|
virtual void stringPaddedToBloomFilter(const char * data, size_t length, BloomFilter & bloom_filter) const
|
||||||
{
|
{
|
||||||
stringToBloomFilter(data, length, bloom_filter);
|
stringToBloomFilter(data, length, bloom_filter);
|
||||||
@ -37,8 +51,22 @@ struct ITokenExtractor
|
|||||||
|
|
||||||
virtual void stringLikeToBloomFilter(const char * data, size_t length, BloomFilter & bloom_filter) const = 0;
|
virtual void stringLikeToBloomFilter(const char * data, size_t length, BloomFilter & bloom_filter) const = 0;
|
||||||
|
|
||||||
|
/// Updates GIN filter from exact-match string filter value
|
||||||
virtual void stringToGinFilter(const char * data, size_t length, GinFilter & gin_filter) const = 0;
|
virtual void stringToGinFilter(const char * data, size_t length, GinFilter & gin_filter) const = 0;
|
||||||
|
|
||||||
|
/// Updates GIN filter from substring-match string filter value.
|
||||||
|
/// An `ITokenExtractor` implementation may decide to skip certain
|
||||||
|
/// tokens depending on whether the substring is a prefix or a suffix.
|
||||||
|
virtual void substringToGinFilter(
|
||||||
|
const char * data,
|
||||||
|
size_t length,
|
||||||
|
GinFilter & gin_filter,
|
||||||
|
bool is_prefix [[maybe_unused]],
|
||||||
|
bool is_suffix [[maybe_unused]]) const
|
||||||
|
{
|
||||||
|
stringToGinFilter(data, length, gin_filter);
|
||||||
|
}
|
||||||
|
|
||||||
virtual void stringPaddedToGinFilter(const char * data, size_t length, GinFilter & gin_filter) const
|
virtual void stringPaddedToGinFilter(const char * data, size_t length, GinFilter & gin_filter) const
|
||||||
{
|
{
|
||||||
stringToGinFilter(data, length, gin_filter);
|
stringToGinFilter(data, length, gin_filter);
|
||||||
@ -148,6 +176,11 @@ struct SplitTokenExtractor final : public ITokenExtractorHelper<SplitTokenExtrac
|
|||||||
|
|
||||||
bool nextInStringLike(const char * data, size_t length, size_t * __restrict pos, String & token) const override;
|
bool nextInStringLike(const char * data, size_t length, size_t * __restrict pos, String & token) const override;
|
||||||
|
|
||||||
|
void substringToBloomFilter(const char * data, size_t length, BloomFilter & bloom_filter, bool is_prefix, bool is_suffix) const override;
|
||||||
|
|
||||||
|
void substringToGinFilter(const char * data, size_t length, GinFilter & gin_filter, bool is_prefix, bool is_suffix) const override;
|
||||||
|
|
||||||
|
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -67,6 +67,8 @@ namespace ErrorCodes
|
|||||||
{
|
{
|
||||||
extern const int BAD_ARGUMENTS;
|
extern const int BAD_ARGUMENTS;
|
||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
|
extern const int CANNOT_COMPILE_REGEXP;
|
||||||
|
extern const int CANNOT_OPEN_FILE;
|
||||||
extern const int CANNOT_PARSE_TEXT;
|
extern const int CANNOT_PARSE_TEXT;
|
||||||
extern const int CANNOT_PARSE_ESCAPE_SEQUENCE;
|
extern const int CANNOT_PARSE_ESCAPE_SEQUENCE;
|
||||||
extern const int CANNOT_PARSE_QUOTED_STRING;
|
extern const int CANNOT_PARSE_QUOTED_STRING;
|
||||||
@ -78,8 +80,7 @@ namespace ErrorCodes
|
|||||||
extern const int CANNOT_PARSE_IPV6;
|
extern const int CANNOT_PARSE_IPV6;
|
||||||
extern const int CANNOT_PARSE_UUID;
|
extern const int CANNOT_PARSE_UUID;
|
||||||
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
|
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
|
||||||
extern const int CANNOT_OPEN_FILE;
|
extern const int CANNOT_SCHEDULE_TASK;
|
||||||
extern const int CANNOT_COMPILE_REGEXP;
|
|
||||||
extern const int DUPLICATE_COLUMN;
|
extern const int DUPLICATE_COLUMN;
|
||||||
extern const int ILLEGAL_COLUMN;
|
extern const int ILLEGAL_COLUMN;
|
||||||
extern const int THERE_IS_NO_COLUMN;
|
extern const int THERE_IS_NO_COLUMN;
|
||||||
@ -267,6 +268,10 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti
|
|||||||
{
|
{
|
||||||
return HTTPResponse::HTTP_REQUEST_TIMEOUT;
|
return HTTPResponse::HTTP_REQUEST_TIMEOUT;
|
||||||
}
|
}
|
||||||
|
else if (exception_code == ErrorCodes::CANNOT_SCHEDULE_TASK)
|
||||||
|
{
|
||||||
|
return HTTPResponse::HTTP_SERVICE_UNAVAILABLE;
|
||||||
|
}
|
||||||
|
|
||||||
return HTTPResponse::HTTP_INTERNAL_SERVER_ERROR;
|
return HTTPResponse::HTTP_INTERNAL_SERVER_ERROR;
|
||||||
}
|
}
|
||||||
|
@ -760,9 +760,16 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
|||||||
CurrentMetrics::MergeTreeDataSelectExecutorThreadsScheduled,
|
CurrentMetrics::MergeTreeDataSelectExecutorThreadsScheduled,
|
||||||
num_threads);
|
num_threads);
|
||||||
|
|
||||||
|
|
||||||
|
/// Instances of ThreadPool "borrow" threads from the global thread pool.
|
||||||
|
/// We intentionally use scheduleOrThrow here to avoid a deadlock.
|
||||||
|
/// For example, queries can already be running with threads from the
|
||||||
|
/// global pool, and if we saturate max_thread_pool_size whilst requesting
|
||||||
|
/// more in this loop, queries will block infinitely.
|
||||||
|
/// So we wait until lock_acquire_timeout, and then raise an exception.
|
||||||
for (size_t part_index = 0; part_index < parts.size(); ++part_index)
|
for (size_t part_index = 0; part_index < parts.size(); ++part_index)
|
||||||
{
|
{
|
||||||
pool.scheduleOrThrowOnError([&, part_index, thread_group = CurrentThread::getGroup()]
|
pool.scheduleOrThrow([&, part_index, thread_group = CurrentThread::getGroup()]
|
||||||
{
|
{
|
||||||
setThreadName("MergeTreeIndex");
|
setThreadName("MergeTreeIndex");
|
||||||
|
|
||||||
@ -774,7 +781,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
|||||||
CurrentThread::attachToGroupIfDetached(thread_group);
|
CurrentThread::attachToGroupIfDetached(thread_group);
|
||||||
|
|
||||||
process_part(part_index);
|
process_part(part_index);
|
||||||
});
|
}, Priority{}, context->getSettingsRef().lock_acquire_timeout.totalMicroseconds());
|
||||||
}
|
}
|
||||||
|
|
||||||
pool.wait();
|
pool.wait();
|
||||||
|
@ -566,7 +566,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals(
|
|||||||
out.function = RPNElement::FUNCTION_EQUALS;
|
out.function = RPNElement::FUNCTION_EQUALS;
|
||||||
out.bloom_filter = std::make_unique<BloomFilter>(params);
|
out.bloom_filter = std::make_unique<BloomFilter>(params);
|
||||||
const auto & value = const_value.get<String>();
|
const auto & value = const_value.get<String>();
|
||||||
token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter);
|
token_extractor->substringToBloomFilter(value.data(), value.size(), *out.bloom_filter, true, false);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
else if (function_name == "endsWith")
|
else if (function_name == "endsWith")
|
||||||
@ -575,7 +575,7 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals(
|
|||||||
out.function = RPNElement::FUNCTION_EQUALS;
|
out.function = RPNElement::FUNCTION_EQUALS;
|
||||||
out.bloom_filter = std::make_unique<BloomFilter>(params);
|
out.bloom_filter = std::make_unique<BloomFilter>(params);
|
||||||
const auto & value = const_value.get<String>();
|
const auto & value = const_value.get<String>();
|
||||||
token_extractor->stringToBloomFilter(value.data(), value.size(), *out.bloom_filter);
|
token_extractor->substringToBloomFilter(value.data(), value.size(), *out.bloom_filter, false, true);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
else if (function_name == "multiSearchAny"
|
else if (function_name == "multiSearchAny"
|
||||||
@ -596,8 +596,16 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals(
|
|||||||
|
|
||||||
bloom_filters.back().emplace_back(params);
|
bloom_filters.back().emplace_back(params);
|
||||||
const auto & value = element.get<String>();
|
const auto & value = element.get<String>();
|
||||||
|
|
||||||
|
if (function_name == "multiSearchAny")
|
||||||
|
{
|
||||||
|
token_extractor->substringToBloomFilter(value.data(), value.size(), bloom_filters.back().back(), false, false);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
token_extractor->stringToBloomFilter(value.data(), value.size(), bloom_filters.back().back());
|
token_extractor->stringToBloomFilter(value.data(), value.size(), bloom_filters.back().back());
|
||||||
}
|
}
|
||||||
|
}
|
||||||
out.set_bloom_filters = std::move(bloom_filters);
|
out.set_bloom_filters = std::move(bloom_filters);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
@ -625,12 +633,12 @@ bool MergeTreeConditionBloomFilterText::traverseTreeEquals(
|
|||||||
for (const auto & alternative : alternatives)
|
for (const auto & alternative : alternatives)
|
||||||
{
|
{
|
||||||
bloom_filters.back().emplace_back(params);
|
bloom_filters.back().emplace_back(params);
|
||||||
token_extractor->stringToBloomFilter(alternative.data(), alternative.size(), bloom_filters.back().back());
|
token_extractor->substringToBloomFilter(alternative.data(), alternative.size(), bloom_filters.back().back(), false, false);
|
||||||
}
|
}
|
||||||
out.set_bloom_filters = std::move(bloom_filters);
|
out.set_bloom_filters = std::move(bloom_filters);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
token_extractor->stringToBloomFilter(required_substring.data(), required_substring.size(), *out.bloom_filter);
|
token_extractor->substringToBloomFilter(required_substring.data(), required_substring.size(), *out.bloom_filter, false, false);
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -595,7 +595,7 @@ bool MergeTreeConditionFullText::traverseASTEquals(
|
|||||||
out.function = RPNElement::FUNCTION_EQUALS;
|
out.function = RPNElement::FUNCTION_EQUALS;
|
||||||
out.gin_filter = std::make_unique<GinFilter>(params);
|
out.gin_filter = std::make_unique<GinFilter>(params);
|
||||||
const auto & value = const_value.get<String>();
|
const auto & value = const_value.get<String>();
|
||||||
token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter);
|
token_extractor->substringToGinFilter(value.data(), value.size(), *out.gin_filter, true, false);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
else if (function_name == "endsWith")
|
else if (function_name == "endsWith")
|
||||||
@ -604,7 +604,7 @@ bool MergeTreeConditionFullText::traverseASTEquals(
|
|||||||
out.function = RPNElement::FUNCTION_EQUALS;
|
out.function = RPNElement::FUNCTION_EQUALS;
|
||||||
out.gin_filter = std::make_unique<GinFilter>(params);
|
out.gin_filter = std::make_unique<GinFilter>(params);
|
||||||
const auto & value = const_value.get<String>();
|
const auto & value = const_value.get<String>();
|
||||||
token_extractor->stringToGinFilter(value.data(), value.size(), *out.gin_filter);
|
token_extractor->substringToGinFilter(value.data(), value.size(), *out.gin_filter, false, true);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
else if (function_name == "multiSearchAny")
|
else if (function_name == "multiSearchAny")
|
||||||
@ -622,7 +622,7 @@ bool MergeTreeConditionFullText::traverseASTEquals(
|
|||||||
|
|
||||||
gin_filters.back().emplace_back(params);
|
gin_filters.back().emplace_back(params);
|
||||||
const auto & value = element.get<String>();
|
const auto & value = element.get<String>();
|
||||||
token_extractor->stringToGinFilter(value.data(), value.size(), gin_filters.back().back());
|
token_extractor->substringToGinFilter(value.data(), value.size(), gin_filters.back().back(), false, false);
|
||||||
}
|
}
|
||||||
out.set_gin_filters = std::move(gin_filters);
|
out.set_gin_filters = std::move(gin_filters);
|
||||||
return true;
|
return true;
|
||||||
@ -650,14 +650,14 @@ bool MergeTreeConditionFullText::traverseASTEquals(
|
|||||||
for (const auto & alternative : alternatives)
|
for (const auto & alternative : alternatives)
|
||||||
{
|
{
|
||||||
gin_filters.back().emplace_back(params);
|
gin_filters.back().emplace_back(params);
|
||||||
token_extractor->stringToGinFilter(alternative.data(), alternative.size(), gin_filters.back().back());
|
token_extractor->substringToGinFilter(alternative.data(), alternative.size(), gin_filters.back().back(), false, false);
|
||||||
}
|
}
|
||||||
out.set_gin_filters = std::move(gin_filters);
|
out.set_gin_filters = std::move(gin_filters);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
out.gin_filter = std::make_unique<GinFilter>(params);
|
out.gin_filter = std::make_unique<GinFilter>(params);
|
||||||
token_extractor->stringToGinFilter(required_substring.data(), required_substring.size(), *out.gin_filter);
|
token_extractor->substringToGinFilter(required_substring.data(), required_substring.size(), *out.gin_filter, false, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
|
@ -82,7 +82,6 @@ private:
|
|||||||
const fs::path zookeeper_path;
|
const fs::path zookeeper_path;
|
||||||
const size_t buckets_num;
|
const size_t buckets_num;
|
||||||
|
|
||||||
bool initialized = false;
|
|
||||||
LoggerPtr log;
|
LoggerPtr log;
|
||||||
|
|
||||||
std::atomic_bool shutdown_called = false;
|
std::atomic_bool shutdown_called = false;
|
||||||
|
@ -13,7 +13,7 @@ class ASTStorage;
|
|||||||
#define S3QUEUE_RELATED_SETTINGS(M, ALIAS) \
|
#define S3QUEUE_RELATED_SETTINGS(M, ALIAS) \
|
||||||
M(S3QueueMode, \
|
M(S3QueueMode, \
|
||||||
mode, \
|
mode, \
|
||||||
S3QueueMode::UNORDERED, \
|
S3QueueMode::ORDERED, \
|
||||||
"With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer." \
|
"With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer." \
|
||||||
"With ordered mode, only the max name of the successfully consumed file stored.", \
|
"With ordered mode, only the max name of the successfully consumed file stored.", \
|
||||||
0) \
|
0) \
|
||||||
|
@ -71,8 +71,14 @@ namespace
|
|||||||
return zkutil::extractZooKeeperPath(result_zk_path, true);
|
return zkutil::extractZooKeeperPath(result_zk_path, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
void checkAndAdjustSettings(S3QueueSettings & s3queue_settings, const Settings & settings)
|
void checkAndAdjustSettings(S3QueueSettings & s3queue_settings, const Settings & settings, bool is_attach)
|
||||||
{
|
{
|
||||||
|
if (!is_attach && !s3queue_settings.mode.changed)
|
||||||
|
{
|
||||||
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `mode` (Unordered/Ordered) is not specified, but is required.");
|
||||||
|
}
|
||||||
|
/// In case !is_attach, we leave Ordered mode as default for compatibility.
|
||||||
|
|
||||||
if (!s3queue_settings.s3queue_processing_threads_num)
|
if (!s3queue_settings.s3queue_processing_threads_num)
|
||||||
{
|
{
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `s3queue_processing_threads_num` cannot be set to zero");
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `s3queue_processing_threads_num` cannot be set to zero");
|
||||||
@ -125,15 +131,7 @@ StorageS3Queue::StorageS3Queue(
|
|||||||
throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue url must either end with '/' or contain globs");
|
throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue url must either end with '/' or contain globs");
|
||||||
}
|
}
|
||||||
|
|
||||||
if (mode == LoadingStrictnessLevel::CREATE
|
checkAndAdjustSettings(*s3queue_settings, context_->getSettingsRef(), mode > LoadingStrictnessLevel::CREATE);
|
||||||
&& !context_->getSettingsRef().s3queue_allow_experimental_sharded_mode
|
|
||||||
&& s3queue_settings->mode == S3QueueMode::ORDERED
|
|
||||||
&& (s3queue_settings->s3queue_buckets > 1 || s3queue_settings->s3queue_processing_threads_num > 1))
|
|
||||||
{
|
|
||||||
throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue sharded mode is not allowed. To enable use `s3queue_allow_experimental_sharded_mode`");
|
|
||||||
}
|
|
||||||
|
|
||||||
checkAndAdjustSettings(*s3queue_settings, context_->getSettingsRef());
|
|
||||||
|
|
||||||
object_storage = configuration->createObjectStorage(context_, /* is_readonly */true);
|
object_storage = configuration->createObjectStorage(context_, /* is_readonly */true);
|
||||||
FormatFactory::instance().checkFormatName(configuration->format);
|
FormatFactory::instance().checkFormatName(configuration->format);
|
||||||
|
@ -212,6 +212,20 @@ FROM merge('system', '^asynchronous_metric_log')
|
|||||||
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'MaxPartCountForPartition'
|
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'MaxPartCountForPartition'
|
||||||
GROUP BY t
|
GROUP BY t
|
||||||
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||||
|
)EOQ") }
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{ "dashboard", "Overview" },
|
||||||
|
{ "title", "Concurrent network connections" },
|
||||||
|
{ "query", trim(R"EOQ(
|
||||||
|
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,
|
||||||
|
sum(CurrentMetric_TCPConnection) AS TCP_Connections,
|
||||||
|
sum(CurrentMetric_MySQLConnection) AS MySQL_Connections,
|
||||||
|
sum(CurrentMetric_HTTPConnection) AS HTTP_Connections
|
||||||
|
FROM merge('system', '^metric_log')
|
||||||
|
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
|
||||||
|
GROUP BY t
|
||||||
|
ORDER BY t WITH FILL STEP {rounding:UInt32}
|
||||||
)EOQ") }
|
)EOQ") }
|
||||||
},
|
},
|
||||||
/// Default dashboard for ClickHouse Cloud
|
/// Default dashboard for ClickHouse Cloud
|
||||||
@ -349,6 +363,11 @@ ORDER BY t WITH FILL STEP {rounding:UInt32}
|
|||||||
{ "dashboard", "Cloud overview" },
|
{ "dashboard", "Cloud overview" },
|
||||||
{ "title", "Network send bytes/sec" },
|
{ "title", "Network send bytes/sec" },
|
||||||
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value)\nFROM (\n SELECT event_time, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric LIKE 'NetworkSendBytes%'\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value)\nFROM (\n SELECT event_time, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric LIKE 'NetworkSendBytes%'\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{ "dashboard", "Cloud overview" },
|
||||||
|
{ "title", "Concurrent network connections" },
|
||||||
|
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, max(TCP_Connections), max(MySQL_Connections), max(HTTP_Connections) FROM (SELECT event_time, sum(CurrentMetric_TCPConnection) AS TCP_Connections, sum(CurrentMetric_MySQLConnection) AS MySQL_Connections, sum(CurrentMetric_HTTPConnection) AS HTTP_Connections FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -10,9 +10,21 @@ from typing import Any, Callable, List, Optional, Union
|
|||||||
|
|
||||||
import requests
|
import requests
|
||||||
|
|
||||||
import get_robot_token as grt # we need an updated ROBOT_TOKEN
|
|
||||||
from ci_config import CI_CONFIG
|
from ci_config import CI_CONFIG
|
||||||
|
|
||||||
|
try:
|
||||||
|
# A work around for scripts using this downloading module without required deps
|
||||||
|
import get_robot_token as grt # we need an updated ROBOT_TOKEN
|
||||||
|
except ImportError:
|
||||||
|
|
||||||
|
class grt: # type: ignore
|
||||||
|
ROBOT_TOKEN = None
|
||||||
|
|
||||||
|
@staticmethod
|
||||||
|
def get_best_robot_token() -> str:
|
||||||
|
return ""
|
||||||
|
|
||||||
|
|
||||||
DOWNLOAD_RETRIES_COUNT = 5
|
DOWNLOAD_RETRIES_COUNT = 5
|
||||||
|
|
||||||
|
|
||||||
@ -63,15 +75,10 @@ def get_gh_api(
|
|||||||
"""
|
"""
|
||||||
|
|
||||||
def set_auth_header():
|
def set_auth_header():
|
||||||
if "headers" in kwargs:
|
headers = kwargs.get("headers", {})
|
||||||
if "Authorization" not in kwargs["headers"]:
|
if "Authorization" not in headers:
|
||||||
kwargs["headers"][
|
headers["Authorization"] = f"Bearer {grt.get_best_robot_token()}"
|
||||||
"Authorization"
|
kwargs["headers"] = headers
|
||||||
] = f"Bearer {grt.get_best_robot_token()}"
|
|
||||||
else:
|
|
||||||
kwargs["headers"] = {
|
|
||||||
"Authorization": f"Bearer {grt.get_best_robot_token()}"
|
|
||||||
}
|
|
||||||
|
|
||||||
if grt.ROBOT_TOKEN is not None:
|
if grt.ROBOT_TOKEN is not None:
|
||||||
set_auth_header()
|
set_auth_header()
|
||||||
|
@ -1,79 +1,38 @@
|
|||||||
#!/usr/bin/env python3
|
#!/usr/bin/env python3
|
||||||
|
|
||||||
import logging
|
import logging
|
||||||
import os
|
from pathlib import Path
|
||||||
|
|
||||||
import requests
|
from build_download_helper import DownloadException, download_build_with_progress
|
||||||
from requests.adapters import HTTPAdapter # type: ignore
|
from get_previous_release_tag import (
|
||||||
from urllib3.util.retry import Retry # type: ignore
|
ReleaseInfo,
|
||||||
|
get_previous_release,
|
||||||
from get_previous_release_tag import ReleaseInfo, get_previous_release
|
get_release_by_tag,
|
||||||
|
|
||||||
CLICKHOUSE_TAGS_URL = "https://api.github.com/repos/ClickHouse/ClickHouse/tags"
|
|
||||||
|
|
||||||
DOWNLOAD_PREFIX = (
|
|
||||||
"https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/"
|
|
||||||
)
|
)
|
||||||
CLICKHOUSE_COMMON_STATIC_PACKAGE_NAME = "clickhouse-common-static_{version}_amd64.deb"
|
|
||||||
CLICKHOUSE_COMMON_STATIC_DBG_PACKAGE_NAME = (
|
|
||||||
"clickhouse-common-static-dbg_{version}_amd64.deb"
|
|
||||||
)
|
|
||||||
CLICKHOUSE_CLIENT_PACKAGE_NAME = "clickhouse-client_{version}_amd64.deb"
|
|
||||||
CLICKHOUSE_LIBRARY_BRIDGE_PACKAGE_NAME = "clickhouse-library-bridge_{version}_amd64.deb"
|
|
||||||
CLICKHOUSE_ODBC_BRIDGE_PACKAGE_NAME = "clickhouse-odbc-bridge_{version}_amd64.deb"
|
|
||||||
CLICKHOUSE_SERVER_PACKAGE_NAME = "clickhouse-server_{version}_amd64.deb"
|
|
||||||
|
|
||||||
PACKAGES_DIR = "previous_release_package_folder/"
|
PACKAGES_DIR = Path("previous_release_package_folder")
|
||||||
VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-[a-zA-Z]*)"
|
|
||||||
|
|
||||||
|
|
||||||
def download_package(url, out_path, retries=10, backoff_factor=0.3):
|
def download_packages(release: ReleaseInfo, dest_path: Path = PACKAGES_DIR) -> None:
|
||||||
session = requests.Session()
|
dest_path.mkdir(parents=True, exist_ok=True)
|
||||||
retry = Retry(
|
|
||||||
total=retries,
|
|
||||||
read=retries,
|
|
||||||
connect=retries,
|
|
||||||
backoff_factor=backoff_factor,
|
|
||||||
status_forcelist=[500, 502, 503, 504],
|
|
||||||
)
|
|
||||||
adapter = HTTPAdapter(max_retries=retry)
|
|
||||||
session.mount("http://", adapter)
|
|
||||||
session.mount("https://", adapter)
|
|
||||||
response = session.get(url)
|
|
||||||
response.raise_for_status()
|
|
||||||
print(f"Download {url} to {out_path}")
|
|
||||||
with open(out_path, "wb") as fd:
|
|
||||||
fd.write(response.content)
|
|
||||||
|
|
||||||
|
|
||||||
def download_packages(release, dest_path=PACKAGES_DIR):
|
|
||||||
if not os.path.exists(dest_path):
|
|
||||||
os.makedirs(dest_path)
|
|
||||||
|
|
||||||
logging.info("Will download %s", release)
|
logging.info("Will download %s", release)
|
||||||
|
|
||||||
def get_dest_path(pkg_name):
|
for pkg, url in release.assets.items():
|
||||||
return os.path.join(dest_path, pkg_name)
|
if not pkg.endswith("_amd64.deb") or "-dbg_" in pkg:
|
||||||
|
continue
|
||||||
for pkg in (
|
pkg_name = dest_path / pkg
|
||||||
CLICKHOUSE_COMMON_STATIC_PACKAGE_NAME,
|
download_build_with_progress(url, pkg_name)
|
||||||
CLICKHOUSE_COMMON_STATIC_DBG_PACKAGE_NAME,
|
|
||||||
CLICKHOUSE_CLIENT_PACKAGE_NAME,
|
|
||||||
CLICKHOUSE_LIBRARY_BRIDGE_PACKAGE_NAME,
|
|
||||||
CLICKHOUSE_ODBC_BRIDGE_PACKAGE_NAME,
|
|
||||||
CLICKHOUSE_SERVER_PACKAGE_NAME,
|
|
||||||
):
|
|
||||||
url = (DOWNLOAD_PREFIX + pkg).format(version=release.version, type=release.type)
|
|
||||||
pkg_name = get_dest_path(pkg.format(version=release.version))
|
|
||||||
download_package(url, pkg_name)
|
|
||||||
|
|
||||||
|
|
||||||
def download_last_release(dest_path):
|
def download_last_release(dest_path: Path) -> None:
|
||||||
current_release = get_previous_release(None)
|
current_release = get_previous_release(None)
|
||||||
|
if current_release is None:
|
||||||
|
raise DownloadException("The current release is not found")
|
||||||
download_packages(current_release, dest_path=dest_path)
|
download_packages(current_release, dest_path=dest_path)
|
||||||
|
|
||||||
|
|
||||||
if __name__ == "__main__":
|
if __name__ == "__main__":
|
||||||
logging.basicConfig(level=logging.INFO)
|
logging.basicConfig(level=logging.INFO)
|
||||||
release = ReleaseInfo(input())
|
release = get_release_by_tag(input())
|
||||||
download_packages(release)
|
download_packages(release)
|
||||||
|
@ -2,47 +2,37 @@
|
|||||||
|
|
||||||
import logging
|
import logging
|
||||||
import re
|
import re
|
||||||
from typing import List, Optional, Tuple
|
from typing import Dict, List, Optional, Tuple
|
||||||
|
|
||||||
import requests
|
from build_download_helper import get_gh_api
|
||||||
|
from git_helper import TAG_REGEXP
|
||||||
CLICKHOUSE_TAGS_URL = "https://api.github.com/repos/ClickHouse/ClickHouse/tags"
|
from version_helper import (
|
||||||
CLICKHOUSE_PACKAGE_URL = (
|
ClickHouseVersion,
|
||||||
"https://github.com/ClickHouse/ClickHouse/releases/download/"
|
get_version_from_string,
|
||||||
"v{version}-{type}/clickhouse-common-static_{version}_amd64.deb"
|
get_version_from_tag,
|
||||||
)
|
)
|
||||||
VERSION_PATTERN = r"(v(?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-[a-zA-Z]*)"
|
|
||||||
|
CLICKHOUSE_TAGS_URL = "https://api.github.com/repos/ClickHouse/ClickHouse/releases"
|
||||||
|
PACKAGE_REGEXP = r"\Aclickhouse-common-static_.+[.]deb"
|
||||||
|
|
||||||
logger = logging.getLogger(__name__)
|
logger = logging.getLogger(__name__)
|
||||||
|
|
||||||
|
|
||||||
class Version:
|
|
||||||
def __init__(self, version: str):
|
|
||||||
self.version = version
|
|
||||||
|
|
||||||
def __lt__(self, other: "Version") -> bool:
|
|
||||||
return list(map(int, self.version.split("."))) < list(
|
|
||||||
map(int, other.version.split("."))
|
|
||||||
)
|
|
||||||
|
|
||||||
def __str__(self):
|
|
||||||
return self.version
|
|
||||||
|
|
||||||
|
|
||||||
class ReleaseInfo:
|
class ReleaseInfo:
|
||||||
def __init__(self, release_tag: str):
|
def __init__(self, release_tag: str, assets: Dict[str, str]):
|
||||||
self.version = Version(release_tag[1:].split("-")[0])
|
self.version = get_version_from_tag(release_tag)
|
||||||
self.type = release_tag[1:].split("-")[1]
|
self.type = self.version.description
|
||||||
|
self.assets = assets
|
||||||
|
|
||||||
def __str__(self):
|
def __str__(self):
|
||||||
return f"v{self.version}-{self.type}"
|
return self.version.describe
|
||||||
|
|
||||||
def __repr__(self):
|
def __repr__(self):
|
||||||
return f"ReleaseInfo: {self.version}-{self.type}"
|
return f"ReleaseInfo: {self.version.describe}"
|
||||||
|
|
||||||
|
|
||||||
def find_previous_release(
|
def find_previous_release(
|
||||||
server_version: Optional[Version], releases: List[ReleaseInfo]
|
server_version: Optional[ClickHouseVersion], releases: List[ReleaseInfo]
|
||||||
) -> Tuple[bool, Optional[ReleaseInfo]]:
|
) -> Tuple[bool, Optional[ReleaseInfo]]:
|
||||||
releases.sort(key=lambda x: x.version, reverse=True)
|
releases.sort(key=lambda x: x.version, reverse=True)
|
||||||
|
|
||||||
@ -54,15 +44,7 @@ def find_previous_release(
|
|||||||
# Check if the artifact exists on GitHub.
|
# Check if the artifact exists on GitHub.
|
||||||
# It can be not true for a short period of time
|
# It can be not true for a short period of time
|
||||||
# after creating a tag for a new release before uploading the packages.
|
# after creating a tag for a new release before uploading the packages.
|
||||||
if (
|
if any(re.match(PACKAGE_REGEXP, name) for name in release.assets.keys()):
|
||||||
requests.head(
|
|
||||||
CLICKHOUSE_PACKAGE_URL.format(
|
|
||||||
version=release.version, type=release.type
|
|
||||||
),
|
|
||||||
timeout=10,
|
|
||||||
).status_code
|
|
||||||
!= 404
|
|
||||||
):
|
|
||||||
return True, release
|
return True, release
|
||||||
|
|
||||||
logger.debug(
|
logger.debug(
|
||||||
@ -74,12 +56,14 @@ def find_previous_release(
|
|||||||
return False, None
|
return False, None
|
||||||
|
|
||||||
|
|
||||||
def get_previous_release(server_version: Optional[Version]) -> Optional[ReleaseInfo]:
|
def get_previous_release(
|
||||||
|
server_version: Optional[ClickHouseVersion],
|
||||||
|
) -> Optional[ReleaseInfo]:
|
||||||
page = 1
|
page = 1
|
||||||
found = False
|
found = False
|
||||||
while not found:
|
while not found:
|
||||||
response = requests.get(
|
response = get_gh_api(
|
||||||
CLICKHOUSE_TAGS_URL, {"page": page, "per_page": 100}, timeout=10
|
CLICKHOUSE_TAGS_URL, params={"page": page, "per_page": 100}, timeout=10
|
||||||
)
|
)
|
||||||
if not response.ok:
|
if not response.ok:
|
||||||
logger.error(
|
logger.error(
|
||||||
@ -87,24 +71,42 @@ def get_previous_release(server_version: Optional[Version]) -> Optional[ReleaseI
|
|||||||
)
|
)
|
||||||
response.raise_for_status()
|
response.raise_for_status()
|
||||||
|
|
||||||
releases_str = set(re.findall(VERSION_PATTERN, response.text))
|
releases = response.json()
|
||||||
if len(releases_str) == 0:
|
|
||||||
raise ValueError(
|
|
||||||
"Cannot find previous release for "
|
|
||||||
+ str(server_version)
|
|
||||||
+ " server version"
|
|
||||||
)
|
|
||||||
|
|
||||||
releases = [ReleaseInfo(release) for release in releases_str]
|
release_infos = [] # type: List[ReleaseInfo]
|
||||||
found, previous_release = find_previous_release(server_version, releases)
|
for r in releases:
|
||||||
|
if re.match(TAG_REGEXP, r["tag_name"]):
|
||||||
|
assets = {
|
||||||
|
a["name"]: a["browser_download_url"]
|
||||||
|
for a in r["assets"]
|
||||||
|
if a["state"] == "uploaded"
|
||||||
|
}
|
||||||
|
release_infos.append(ReleaseInfo(r["tag_name"], assets))
|
||||||
|
found, previous_release = find_previous_release(server_version, release_infos)
|
||||||
page += 1
|
page += 1
|
||||||
|
|
||||||
return previous_release
|
return previous_release
|
||||||
|
|
||||||
|
|
||||||
|
def get_release_by_tag(tag: str) -> ReleaseInfo:
|
||||||
|
response = get_gh_api(f"{CLICKHOUSE_TAGS_URL}/tags/{tag}", timeout=10)
|
||||||
|
release = response.json()
|
||||||
|
assets = {
|
||||||
|
a["name"]: a["browser_download_url"]
|
||||||
|
for a in release["assets"]
|
||||||
|
if a["state"] == "uploaded"
|
||||||
|
}
|
||||||
|
return ReleaseInfo(release["tag_name"], assets)
|
||||||
|
|
||||||
|
|
||||||
def main():
|
def main():
|
||||||
logging.basicConfig(level=logging.INFO)
|
logging.basicConfig(level=logging.INFO)
|
||||||
server_version = Version(input())
|
version_string = input()
|
||||||
|
version_string = version_string.split("+", maxsplit=1)[0]
|
||||||
|
try:
|
||||||
|
server_version = get_version_from_string(version_string)
|
||||||
|
except ValueError:
|
||||||
|
server_version = get_version_from_tag(version_string)
|
||||||
print(get_previous_release(server_version))
|
print(get_previous_release(server_version))
|
||||||
|
|
||||||
|
|
||||||
|
@ -23,7 +23,7 @@ from lambda_shared_package.lambda_shared.pr import (
|
|||||||
check_pr_description,
|
check_pr_description,
|
||||||
)
|
)
|
||||||
from pr_info import PRInfo
|
from pr_info import PRInfo
|
||||||
from report import FAILURE, PENDING, SUCCESS
|
from report import FAILURE, PENDING, SUCCESS, StatusType
|
||||||
|
|
||||||
TRUSTED_ORG_IDS = {
|
TRUSTED_ORG_IDS = {
|
||||||
54801242, # clickhouse
|
54801242, # clickhouse
|
||||||
@ -58,7 +58,7 @@ def pr_is_by_trusted_user(pr_user_login, pr_user_orgs):
|
|||||||
# Returns can_run, description
|
# Returns can_run, description
|
||||||
def should_run_ci_for_pr(pr_info: PRInfo) -> Tuple[bool, str]:
|
def should_run_ci_for_pr(pr_info: PRInfo) -> Tuple[bool, str]:
|
||||||
# Consider the labels and whether the user is trusted.
|
# Consider the labels and whether the user is trusted.
|
||||||
print("Got labels", pr_info.labels)
|
logging.info("Got labels: %s", pr_info.labels)
|
||||||
|
|
||||||
if OK_SKIP_LABELS.intersection(pr_info.labels):
|
if OK_SKIP_LABELS.intersection(pr_info.labels):
|
||||||
return True, "Don't try new checks for release/backports/cherry-picks"
|
return True, "Don't try new checks for release/backports/cherry-picks"
|
||||||
@ -66,9 +66,10 @@ def should_run_ci_for_pr(pr_info: PRInfo) -> Tuple[bool, str]:
|
|||||||
if Labels.CAN_BE_TESTED not in pr_info.labels and not pr_is_by_trusted_user(
|
if Labels.CAN_BE_TESTED not in pr_info.labels and not pr_is_by_trusted_user(
|
||||||
pr_info.user_login, pr_info.user_orgs
|
pr_info.user_login, pr_info.user_orgs
|
||||||
):
|
):
|
||||||
print(
|
logging.info(
|
||||||
f"PRs by untrusted users need the '{Labels.CAN_BE_TESTED}' label - "
|
"PRs by untrusted users need the '%s' label - "
|
||||||
"please contact a member of the core team"
|
"please contact a member of the core team",
|
||||||
|
Labels.CAN_BE_TESTED,
|
||||||
)
|
)
|
||||||
return False, "Needs 'can be tested' label"
|
return False, "Needs 'can be tested' label"
|
||||||
|
|
||||||
@ -93,6 +94,7 @@ def main():
|
|||||||
description = format_description(description)
|
description = format_description(description)
|
||||||
gh = Github(get_best_robot_token(), per_page=100)
|
gh = Github(get_best_robot_token(), per_page=100)
|
||||||
commit = get_commit(gh, pr_info.sha)
|
commit = get_commit(gh, pr_info.sha)
|
||||||
|
status = SUCCESS # type: StatusType
|
||||||
|
|
||||||
description_error, category = check_pr_description(pr_info.body, GITHUB_REPOSITORY)
|
description_error, category = check_pr_description(pr_info.body, GITHUB_REPOSITORY)
|
||||||
pr_labels_to_add = []
|
pr_labels_to_add = []
|
||||||
@ -125,13 +127,16 @@ def main():
|
|||||||
f"::notice :: Add backport labels [{backport_labels}] for a given PR category"
|
f"::notice :: Add backport labels [{backport_labels}] for a given PR category"
|
||||||
)
|
)
|
||||||
|
|
||||||
print(f"Change labels: add {pr_labels_to_add}, remove {pr_labels_to_remove}")
|
logging.info(
|
||||||
|
"Change labels: add %s, remove %s", pr_labels_to_add, pr_labels_to_remove
|
||||||
|
)
|
||||||
if pr_labels_to_add:
|
if pr_labels_to_add:
|
||||||
post_labels(gh, pr_info, pr_labels_to_add)
|
post_labels(gh, pr_info, pr_labels_to_add)
|
||||||
|
|
||||||
if pr_labels_to_remove:
|
if pr_labels_to_remove:
|
||||||
remove_labels(gh, pr_info, pr_labels_to_remove)
|
remove_labels(gh, pr_info, pr_labels_to_remove)
|
||||||
|
|
||||||
|
# 1. Next three IFs are in a correct order. First - fatal error
|
||||||
if description_error:
|
if description_error:
|
||||||
print(
|
print(
|
||||||
"::error ::Cannot run, PR description does not match the template: "
|
"::error ::Cannot run, PR description does not match the template: "
|
||||||
@ -146,9 +151,10 @@ def main():
|
|||||||
f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/"
|
f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/"
|
||||||
"blob/master/.github/PULL_REQUEST_TEMPLATE.md?plain=1"
|
"blob/master/.github/PULL_REQUEST_TEMPLATE.md?plain=1"
|
||||||
)
|
)
|
||||||
|
status = FAILURE
|
||||||
post_commit_status(
|
post_commit_status(
|
||||||
commit,
|
commit,
|
||||||
FAILURE,
|
status,
|
||||||
url,
|
url,
|
||||||
format_description(description_error),
|
format_description(description_error),
|
||||||
PR_CHECK,
|
PR_CHECK,
|
||||||
@ -156,41 +162,38 @@ def main():
|
|||||||
)
|
)
|
||||||
sys.exit(1)
|
sys.exit(1)
|
||||||
|
|
||||||
|
# 2. Then we check if the documentation is not created to fail the Mergeable check
|
||||||
if (
|
if (
|
||||||
Labels.PR_FEATURE in pr_info.labels
|
Labels.PR_FEATURE in pr_info.labels
|
||||||
and not pr_info.has_changes_in_documentation()
|
and not pr_info.has_changes_in_documentation()
|
||||||
):
|
):
|
||||||
print(
|
print(
|
||||||
f"The '{Labels.PR_FEATURE}' in the labels, "
|
f"::error ::The '{Labels.PR_FEATURE}' in the labels, "
|
||||||
"but there's no changed documentation"
|
"but there's no changed documentation"
|
||||||
)
|
)
|
||||||
post_commit_status(
|
status = FAILURE
|
||||||
commit,
|
description = f"expect adding docs for {Labels.PR_FEATURE}"
|
||||||
FAILURE,
|
# 3. But we allow the workflow to continue
|
||||||
"",
|
|
||||||
f"expect adding docs for {Labels.PR_FEATURE}",
|
|
||||||
PR_CHECK,
|
|
||||||
pr_info,
|
|
||||||
)
|
|
||||||
# allow the workflow to continue
|
|
||||||
|
|
||||||
|
# 4. And post only a single commit status on a failure
|
||||||
if not can_run:
|
if not can_run:
|
||||||
post_commit_status(
|
post_commit_status(
|
||||||
commit,
|
commit,
|
||||||
FAILURE,
|
status,
|
||||||
"",
|
"",
|
||||||
description,
|
description,
|
||||||
PR_CHECK,
|
PR_CHECK,
|
||||||
pr_info,
|
pr_info,
|
||||||
)
|
)
|
||||||
print("::notice ::Cannot run")
|
print("::error ::Cannot run")
|
||||||
sys.exit(1)
|
sys.exit(1)
|
||||||
|
|
||||||
|
# The status for continue can be posted only one time, not more.
|
||||||
post_commit_status(
|
post_commit_status(
|
||||||
commit,
|
commit,
|
||||||
SUCCESS,
|
status,
|
||||||
"",
|
"",
|
||||||
"ok",
|
description,
|
||||||
PR_CHECK,
|
PR_CHECK,
|
||||||
pr_info,
|
pr_info,
|
||||||
)
|
)
|
||||||
|
@ -74,3 +74,18 @@ def test_grant_all_on_cluster():
|
|||||||
assert ch2.query("SHOW GRANTS FOR Alex") == "GRANT ALL ON *.* TO Alex\n"
|
assert ch2.query("SHOW GRANTS FOR Alex") == "GRANT ALL ON *.* TO Alex\n"
|
||||||
|
|
||||||
ch1.query("DROP USER Alex ON CLUSTER 'cluster'")
|
ch1.query("DROP USER Alex ON CLUSTER 'cluster'")
|
||||||
|
|
||||||
|
|
||||||
|
def test_grant_current_database_on_cluster():
|
||||||
|
ch1.query("CREATE DATABASE user_db ON CLUSTER 'cluster'")
|
||||||
|
ch1.query(
|
||||||
|
"CREATE USER IF NOT EXISTS test_user ON CLUSTER 'cluster' DEFAULT DATABASE user_db"
|
||||||
|
)
|
||||||
|
ch1.query(
|
||||||
|
"GRANT SELECT ON user_db.* TO test_user ON CLUSTER 'cluster' WITH GRANT OPTION"
|
||||||
|
)
|
||||||
|
ch1.query("GRANT CLUSTER ON *.* TO test_user ON CLUSTER 'cluster'")
|
||||||
|
|
||||||
|
assert ch1.query("SHOW DATABASES", user="test_user") == "user_db\n"
|
||||||
|
ch1.query("GRANT SELECT ON * TO test_user ON CLUSTER 'cluster'", user="test_user")
|
||||||
|
assert ch1.query("SHOW DATABASES", user="test_user") == "user_db\n"
|
||||||
|
@ -195,10 +195,10 @@ def test_create_table():
|
|||||||
f"DeltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')",
|
f"DeltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')",
|
||||||
"DNS_ERROR",
|
"DNS_ERROR",
|
||||||
),
|
),
|
||||||
f"S3Queue('http://minio1:9001/root/data/', 'CSV')",
|
f"S3Queue('http://minio1:9001/root/data/', 'CSV') settings mode = 'ordered'",
|
||||||
f"S3Queue('http://minio1:9001/root/data/', 'CSV', 'gzip')",
|
f"S3Queue('http://minio1:9001/root/data/', 'CSV', 'gzip') settings mode = 'ordered'",
|
||||||
f"S3Queue('http://minio1:9001/root/data/', 'minio', '{password}', 'CSV')",
|
f"S3Queue('http://minio1:9001/root/data/', 'minio', '{password}', 'CSV') settings mode = 'ordered'",
|
||||||
f"S3Queue('http://minio1:9001/root/data/', 'minio', '{password}', 'CSV', 'gzip')",
|
f"S3Queue('http://minio1:9001/root/data/', 'minio', '{password}', 'CSV', 'gzip') settings mode = 'ordered'",
|
||||||
]
|
]
|
||||||
|
|
||||||
def make_test_case(i):
|
def make_test_case(i):
|
||||||
@ -258,10 +258,11 @@ def test_create_table():
|
|||||||
"CREATE TABLE table14 (x int) ENGINE = S3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV', 'gzip')",
|
"CREATE TABLE table14 (x int) ENGINE = S3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV', 'gzip')",
|
||||||
"CREATE TABLE table15 (`x` int) ENGINE = S3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '[HIDDEN]')",
|
"CREATE TABLE table15 (`x` int) ENGINE = S3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '[HIDDEN]')",
|
||||||
"CREATE TABLE table16 (`x` int) ENGINE = DeltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')",
|
"CREATE TABLE table16 (`x` int) ENGINE = DeltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')",
|
||||||
"CREATE TABLE table17 (x int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'CSV')",
|
"CREATE TABLE table17 (x int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'CSV') settings mode = 'ordered'",
|
||||||
"CREATE TABLE table18 (x int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'CSV', 'gzip')",
|
"CREATE TABLE table18 (x int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'CSV', 'gzip') settings mode = 'ordered'",
|
||||||
"CREATE TABLE table19 (`x` int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'minio', '[HIDDEN]', 'CSV')",
|
# due to sensitive data substituion the query will be normalized, so not "settings" but "SETTINGS"
|
||||||
"CREATE TABLE table20 (`x` int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'minio', '[HIDDEN]', 'CSV', 'gzip')",
|
"CREATE TABLE table19 (`x` int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'minio', '[HIDDEN]', 'CSV') SETTINGS mode = 'ordered'",
|
||||||
|
"CREATE TABLE table20 (`x` int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'minio', '[HIDDEN]', 'CSV', 'gzip') SETTINGS mode = 'ordered'",
|
||||||
],
|
],
|
||||||
must_not_contain=[password],
|
must_not_contain=[password],
|
||||||
)
|
)
|
||||||
|
@ -0,0 +1,6 @@
|
|||||||
|
<clickhouse>
|
||||||
|
<!-- Run with a small thread pool size so it's easier to saturate -->
|
||||||
|
<max_thread_pool_size>300</max_thread_pool_size>
|
||||||
|
<thread_pool_queue_size>1</thread_pool_queue_size>
|
||||||
|
<background_schedule_pool_size>128</background_schedule_pool_size>
|
||||||
|
</clickhouse>
|
@ -0,0 +1,68 @@
|
|||||||
|
import concurrent.futures
|
||||||
|
|
||||||
|
import pytest
|
||||||
|
from helpers.cluster import ClickHouseCluster
|
||||||
|
|
||||||
|
|
||||||
|
MAX_THREADS = 60
|
||||||
|
|
||||||
|
cluster = ClickHouseCluster(__file__)
|
||||||
|
|
||||||
|
node1 = cluster.add_instance(
|
||||||
|
"node1",
|
||||||
|
macros={"cluster": "test-cluster", "replica": "node1"},
|
||||||
|
main_configs=["configs/settings.xml"],
|
||||||
|
with_zookeeper=True,
|
||||||
|
)
|
||||||
|
|
||||||
|
|
||||||
|
def prepare_cluster():
|
||||||
|
node1.query("DROP TABLE IF EXISTS test_threads_busy SYNC")
|
||||||
|
node1.query(
|
||||||
|
"""
|
||||||
|
CREATE TABLE test_threads_busy(d Date, i Int64, s String) ENGINE=MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY d
|
||||||
|
"""
|
||||||
|
)
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.fixture(scope="module")
|
||||||
|
def started_cluster():
|
||||||
|
try:
|
||||||
|
cluster.start()
|
||||||
|
yield cluster
|
||||||
|
|
||||||
|
finally:
|
||||||
|
cluster.shutdown()
|
||||||
|
|
||||||
|
|
||||||
|
def do_slow_select():
|
||||||
|
# Do a bunch of slow queries that use a large number of threads to saturate max_thread_pool_size
|
||||||
|
# explicitly set max_threads as otherwise it's relative to the number of CPU cores
|
||||||
|
query = (
|
||||||
|
"SELECT d, i, s, sleepEachRow(3) from test_threads_busy SETTINGS max_threads=40"
|
||||||
|
)
|
||||||
|
node1.query(query)
|
||||||
|
|
||||||
|
|
||||||
|
def test_query_exception_on_thread_pool_full(started_cluster):
|
||||||
|
prepare_cluster()
|
||||||
|
# Generate some sample data so sleepEachRow in do_slow_select works
|
||||||
|
node1.query(
|
||||||
|
f"INSERT INTO test_threads_busy VALUES ('2024-01-01', 1, 'thread-test')"
|
||||||
|
)
|
||||||
|
|
||||||
|
futures = []
|
||||||
|
errors = []
|
||||||
|
with concurrent.futures.ThreadPoolExecutor(max_workers=MAX_THREADS) as executor:
|
||||||
|
for _ in range(MAX_THREADS):
|
||||||
|
futures.append(executor.submit(do_slow_select))
|
||||||
|
|
||||||
|
for f in futures:
|
||||||
|
try:
|
||||||
|
f.result()
|
||||||
|
except Exception as err:
|
||||||
|
errors.append(str(err))
|
||||||
|
assert len(errors) > 0, "Should be 'Cannot schedule a task' exceptions"
|
||||||
|
assert all(
|
||||||
|
"Cannot schedule a task" in err for err in errors
|
||||||
|
), "Query threads are stuck, or returned an unexpected error"
|
@ -4,6 +4,11 @@
|
|||||||
5
|
5
|
||||||
5
|
5
|
||||||
5
|
5
|
||||||
|
5
|
||||||
|
5
|
||||||
|
5
|
||||||
|
5
|
||||||
|
5
|
||||||
1
|
1
|
||||||
12
|
12
|
||||||
2023-05-30 14:38:20
|
2023-05-30 14:38:20
|
||||||
|
@ -16,6 +16,21 @@ SELECT accurateCast(-129, 'Int8'); -- { serverError CANNOT_CONVERT_TYPE }
|
|||||||
SELECT accurateCast(5, 'Int8');
|
SELECT accurateCast(5, 'Int8');
|
||||||
SELECT accurateCast(128, 'Int8'); -- { serverError CANNOT_CONVERT_TYPE }
|
SELECT accurateCast(128, 'Int8'); -- { serverError CANNOT_CONVERT_TYPE }
|
||||||
|
|
||||||
|
SELECT accurateCast('-1', 'UInt8'); -- { serverError CANNOT_PARSE_TEXT }
|
||||||
|
SELECT accurateCast('5', 'UInt8');
|
||||||
|
SELECT accurateCast('257', 'UInt8'); -- { serverError CANNOT_PARSE_TEXT }
|
||||||
|
SELECT accurateCast('-1', 'UInt16'); -- { serverError CANNOT_PARSE_TEXT }
|
||||||
|
SELECT accurateCast('5', 'UInt16');
|
||||||
|
SELECT accurateCast('65536', 'UInt16'); -- { serverError CANNOT_PARSE_TEXT }
|
||||||
|
SELECT accurateCast('-1', 'UInt32'); -- { serverError CANNOT_PARSE_TEXT }
|
||||||
|
SELECT accurateCast('5', 'UInt32');
|
||||||
|
SELECT accurateCast('4294967296', 'UInt32'); -- { serverError CANNOT_PARSE_TEXT }
|
||||||
|
SELECT accurateCast('-1', 'UInt64'); -- { serverError CANNOT_PARSE_TEXT }
|
||||||
|
SELECT accurateCast('5', 'UInt64');
|
||||||
|
SELECT accurateCast('-129', 'Int8'); -- { serverError CANNOT_PARSE_TEXT }
|
||||||
|
SELECT accurateCast('5', 'Int8');
|
||||||
|
SELECT accurateCast('128', 'Int8'); -- { serverError CANNOT_PARSE_TEXT }
|
||||||
|
|
||||||
SELECT accurateCast(10, 'Decimal32(9)'); -- { serverError DECIMAL_OVERFLOW }
|
SELECT accurateCast(10, 'Decimal32(9)'); -- { serverError DECIMAL_OVERFLOW }
|
||||||
SELECT accurateCast(1, 'Decimal32(9)');
|
SELECT accurateCast(1, 'Decimal32(9)');
|
||||||
SELECT accurateCast(-10, 'Decimal32(9)'); -- { serverError DECIMAL_OVERFLOW }
|
SELECT accurateCast(-10, 'Decimal32(9)'); -- { serverError DECIMAL_OVERFLOW }
|
||||||
|
@ -68,4 +68,10 @@ with
|
|||||||
last_month as ( select repo_name, count() as count_last_month, rowNumberInAllBlocks() + 1 as position_last_month from github_events where repo_name in (select repo_name from top_repos) and toStartOfMonth(created_at) = toStartOfMonth(today()) - interval 1 month group by repo_name order by count_last_month desc )
|
last_month as ( select repo_name, count() as count_last_month, rowNumberInAllBlocks() + 1 as position_last_month from github_events where repo_name in (select repo_name from top_repos) and toStartOfMonth(created_at) = toStartOfMonth(today()) - interval 1 month group by repo_name order by count_last_month desc )
|
||||||
select d.repo_name, columns('count') from last_day d join last_week w on d.repo_name = w.repo_name join last_month m on d.repo_name = m.repo_name;
|
select d.repo_name, columns('count') from last_day d join last_week w on d.repo_name = w.repo_name join last_month m on d.repo_name = m.repo_name;
|
||||||
|
|
||||||
|
set allow_suspicious_low_cardinality_types=1;
|
||||||
|
|
||||||
|
CREATE TABLE github_events__fuzz_0 (`file_time` Int64, `event_type` Enum8('CommitCommentEvent' = 1, 'CreateEvent' = 2, 'DeleteEvent' = 3, 'ForkEvent' = 4, 'GollumEvent' = 5, 'IssueCommentEvent' = 6, 'IssuesEvent' = 7, 'MemberEvent' = 8, 'PublicEvent' = 9, 'PullRequestEvent' = 10, 'PullRequestReviewCommentEvent' = 11, 'PushEvent' = 12, 'ReleaseEvent' = 13, 'SponsorshipEvent' = 14, 'WatchEvent' = 15, 'GistEvent' = 16, 'FollowEvent' = 17, 'DownloadEvent' = 18, 'PullRequestReviewEvent' = 19, 'ForkApplyEvent' = 20, 'Event' = 21, 'TeamAddEvent' = 22), `actor_login` LowCardinality(String), `repo_name` LowCardinality(Nullable(String)), `created_at` DateTime, `updated_at` DateTime, `action` Array(Enum8('none' = 0, 'created' = 1, 'added' = 2, 'edited' = 3, 'deleted' = 4, 'opened' = 5, 'closed' = 6, 'reopened' = 7, 'assigned' = 8, 'unassigned' = 9, 'labeled' = 10, 'unlabeled' = 11, 'review_requested' = 12, 'review_request_removed' = 13, 'synchronize' = 14, 'started' = 15, 'published' = 16, 'update' = 17, 'create' = 18, 'fork' = 19, 'merged' = 20)), `comment_id` UInt64, `body` String, `path` LowCardinality(String), `position` Int32, `line` Int32, `ref` String, `ref_type` Enum8('none' = 0, 'branch' = 1, 'tag' = 2, 'repository' = 3, 'unknown' = 4), `creator_user_login` Int16, `number` UInt32, `title` String, `labels` Array(Array(LowCardinality(String))), `state` Enum8('none' = 0, 'open' = 1, 'closed' = 2), `locked` UInt8, `assignee` Array(LowCardinality(String)), `assignees` Array(LowCardinality(String)), `comments` UInt32, `author_association` Array(Enum8('NONE' = 0, 'CONTRIBUTOR' = 1, 'OWNER' = 2, 'COLLABORATOR' = 3, 'MEMBER' = 4, 'MANNEQUIN' = 5)), `closed_at` UUID, `merged_at` DateTime, `merge_commit_sha` Nullable(String), `requested_reviewers` Array(LowCardinality(Int64)), `requested_teams` Array(String), `head_ref` String, `head_sha` String, `base_ref` String, `base_sha` String, `merged` Nullable(UInt8), `mergeable` Nullable(UInt8), `rebaseable` LowCardinality(UInt8), `mergeable_state` Array(Enum8('unknown' = 0, 'dirty' = 1, 'clean' = 2, 'unstable' = 3, 'draft' = 4)), `merged_by` LowCardinality(String), `review_comments` UInt32, `maintainer_can_modify` Nullable(UInt8), `commits` UInt32, `additions` Nullable(UInt32), `deletions` UInt32, `changed_files` UInt32, `diff_hunk` Nullable(String), `original_position` UInt32, `commit_id` String, `original_commit_id` String, `push_size` UInt32, `push_distinct_size` UInt32, `member_login` LowCardinality(String), `release_tag_name` LowCardinality(String), `release_name` String, `review_state` Int16) ENGINE = MergeTree ORDER BY (event_type, repo_name, created_at) settings allow_nullable_key=1;
|
||||||
|
|
||||||
|
EXPLAIN PIPELINE header = true, compact = true WITH top_repos AS (SELECT repo_name FROM github_events__fuzz_0 WHERE (event_type = 'WatchEvent') AND (toDate(created_at) = (today() - 1)) GROUP BY repo_name ORDER BY count() DESC LIMIT 100 UNION DISTINCT SELECT repo_name FROM github_events__fuzz_0 WHERE (event_type = 'WatchEvent') AND (toMonday(created_at) = toMonday(today() - toIntervalWeek(1))) GROUP BY repo_name ORDER BY count() DESC LIMIT 100 UNION DISTINCT SELECT repo_name FROM github_events__fuzz_0 PREWHERE (event_type = 'WatchEvent') AND (toStartOfMonth(created_at) = (toStartOfMonth(today()) - toIntervalMonth(1))) GROUP BY repo_name ORDER BY count() DESC LIMIT 100 UNION DISTINCT SELECT repo_name FROM github_events WHERE (event_type = 'WatchEvent') AND (toYear(created_at) = (toYear(today()) - 1)) GROUP BY repo_name ORDER BY count() DESC LIMIT 100), last_day AS (SELECT repo_name, count() AS count_last_day, rowNumberInAllBlocks() + 1 AS position_last_day FROM github_events WHERE (repo_name IN (SELECT repo_name FROM top_repos)) AND (toDate(created_at) = (today() - 1)) GROUP BY repo_name ORDER BY count_last_day DESC), last_week AS (SELECT repo_name, count() AS count_last_week, rowNumberInAllBlocks() + 1 AS position_last_week FROM github_events WHERE (repo_name IN (SELECT repo_name FROM top_repos)) AND (toMonday(created_at) = (toMonday(today()) - toIntervalWeek(2))) GROUP BY repo_name ORDER BY count_last_week DESC), last_month AS (SELECT repo_name, count() AS count_last_month, rowNumberInAllBlocks() + 1 AS position_last_month FROM github_events__fuzz_0 WHERE ('deleted' = 4) AND in(repo_name) AND (toStartOfMonth(created_at) = (toStartOfMonth(today()) - toIntervalMonth(1))) GROUP BY repo_name ORDER BY count_last_month DESC) SELECT d.repo_name, COLUMNS(count) FROM last_day AS d INNER JOIN last_week AS w ON d.repo_name = w.repo_name INNER JOIN last_month AS m ON d.repo_name = m.repo_name format Null; -- { serverError INVALID_SETTING_VALUE }
|
||||||
|
|
||||||
DROP TABLE github_events;
|
DROP TABLE github_events;
|
||||||
|
@ -1,19 +1,19 @@
|
|||||||
1 Hello ClickHouse
|
1 Well, Hello ClickHouse !
|
||||||
2 Hello World
|
2 Well, Hello World !
|
||||||
Granules: 6/6
|
Granules: 6/6
|
||||||
Granules: 2/6
|
Granules: 2/6
|
||||||
Granules: 6/6
|
Granules: 6/6
|
||||||
Granules: 2/6
|
Granules: 2/6
|
||||||
---
|
---
|
||||||
1 Hello ClickHouse
|
1 Well, Hello ClickHouse !
|
||||||
2 Hello World
|
2 Well, Hello World !
|
||||||
6 World Champion
|
6 True World Champion
|
||||||
Granules: 6/6
|
Granules: 6/6
|
||||||
Granules: 3/6
|
Granules: 3/6
|
||||||
Granules: 6/6
|
Granules: 6/6
|
||||||
Granules: 3/6
|
Granules: 3/6
|
||||||
---
|
---
|
||||||
5 OLAP Database
|
5 Its An OLAP Database
|
||||||
Granules: 6/6
|
Granules: 6/6
|
||||||
Granules: 1/6
|
Granules: 1/6
|
||||||
Granules: 6/6
|
Granules: 6/6
|
||||||
|
@ -14,19 +14,19 @@ ENGINE = MergeTree
|
|||||||
ORDER BY id
|
ORDER BY id
|
||||||
SETTINGS index_granularity = 1;
|
SETTINGS index_granularity = 1;
|
||||||
|
|
||||||
INSERT INTO tab VALUES (1, 'Hello ClickHouse'), (2, 'Hello World'), (3, 'Good Weather'), (4, 'Say Hello'), (5, 'OLAP Database'), (6, 'World Champion');
|
INSERT INTO tab VALUES (1, 'Well, Hello ClickHouse !'), (2, 'Well, Hello World !'), (3, 'Good Weather !'), (4, 'Say Hello !'), (5, 'Its An OLAP Database'), (6, 'True World Champion');
|
||||||
|
|
||||||
SELECT * FROM tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id;
|
SELECT * FROM tab WHERE match(str, ' Hello (ClickHouse|World) ') ORDER BY id;
|
||||||
|
|
||||||
-- Read 2/6 granules
|
-- Read 2/6 granules
|
||||||
-- Required string: 'Hello '
|
-- Required string: ' Hello '
|
||||||
-- Alternatives: 'Hello ClickHouse', 'Hello World'
|
-- Alternatives: ' Hello ClickHouse ', ' Hello World '
|
||||||
|
|
||||||
SELECT *
|
SELECT *
|
||||||
FROM
|
FROM
|
||||||
(
|
(
|
||||||
EXPLAIN PLAN indexes=1
|
EXPLAIN PLAN indexes=1
|
||||||
SELECT * FROM tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id
|
SELECT * FROM tab WHERE match(str, ' Hello (ClickHouse|World) ') ORDER BY id
|
||||||
)
|
)
|
||||||
WHERE
|
WHERE
|
||||||
explain LIKE '%Granules: %'
|
explain LIKE '%Granules: %'
|
||||||
@ -37,7 +37,7 @@ SELECT *
|
|||||||
FROM
|
FROM
|
||||||
(
|
(
|
||||||
EXPLAIN PLAN indexes=1
|
EXPLAIN PLAN indexes=1
|
||||||
SELECT * FROM tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id
|
SELECT * FROM tab WHERE match(str, ' Hello (ClickHouse|World) ') ORDER BY id
|
||||||
)
|
)
|
||||||
WHERE
|
WHERE
|
||||||
explain LIKE '%Granules: %'
|
explain LIKE '%Granules: %'
|
||||||
@ -46,17 +46,17 @@ SETTINGS
|
|||||||
|
|
||||||
SELECT '---';
|
SELECT '---';
|
||||||
|
|
||||||
SELECT * FROM tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id;
|
SELECT * FROM tab WHERE match(str, '.* (ClickHouse|World) ') ORDER BY id;
|
||||||
|
|
||||||
-- Read 3/6 granules
|
-- Read 3/6 granules
|
||||||
-- Required string: -
|
-- Required string: -
|
||||||
-- Alternatives: 'ClickHouse', 'World'
|
-- Alternatives: ' ClickHouse ', ' World '
|
||||||
|
|
||||||
SELECT *
|
SELECT *
|
||||||
FROM
|
FROM
|
||||||
(
|
(
|
||||||
EXPLAIN PLAN indexes = 1
|
EXPLAIN PLAN indexes = 1
|
||||||
SELECT * FROM tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id
|
SELECT * FROM tab WHERE match(str, '.* (ClickHouse|World) ') ORDER BY id
|
||||||
)
|
)
|
||||||
WHERE
|
WHERE
|
||||||
explain LIKE '%Granules: %'
|
explain LIKE '%Granules: %'
|
||||||
@ -67,7 +67,7 @@ SELECT *
|
|||||||
FROM
|
FROM
|
||||||
(
|
(
|
||||||
EXPLAIN PLAN indexes = 1
|
EXPLAIN PLAN indexes = 1
|
||||||
SELECT * FROM tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id
|
SELECT * FROM tab WHERE match(str, '.* (ClickHouse|World) ') ORDER BY id
|
||||||
)
|
)
|
||||||
WHERE
|
WHERE
|
||||||
explain LIKE '%Granules: %'
|
explain LIKE '%Granules: %'
|
||||||
@ -76,17 +76,17 @@ SETTINGS
|
|||||||
|
|
||||||
SELECT '---';
|
SELECT '---';
|
||||||
|
|
||||||
SELECT * FROM tab WHERE match(str, 'OLAP.*') ORDER BY id;
|
SELECT * FROM tab WHERE match(str, ' OLAP .*') ORDER BY id;
|
||||||
|
|
||||||
-- Read 1/6 granules
|
-- Read 1/6 granules
|
||||||
-- Required string: 'OLAP'
|
-- Required string: ' OLAP '
|
||||||
-- Alternatives: -
|
-- Alternatives: -
|
||||||
|
|
||||||
SELECT *
|
SELECT *
|
||||||
FROM
|
FROM
|
||||||
(
|
(
|
||||||
EXPLAIN PLAN indexes = 1
|
EXPLAIN PLAN indexes = 1
|
||||||
SELECT * FROM tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id
|
SELECT * FROM tab WHERE match(str, ' OLAP (.*?)*') ORDER BY id
|
||||||
)
|
)
|
||||||
WHERE
|
WHERE
|
||||||
explain LIKE '%Granules: %'
|
explain LIKE '%Granules: %'
|
||||||
@ -97,7 +97,7 @@ SELECT *
|
|||||||
FROM
|
FROM
|
||||||
(
|
(
|
||||||
EXPLAIN PLAN indexes = 1
|
EXPLAIN PLAN indexes = 1
|
||||||
SELECT * FROM tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id
|
SELECT * FROM tab WHERE match(str, ' OLAP (.*?)*') ORDER BY id
|
||||||
)
|
)
|
||||||
WHERE
|
WHERE
|
||||||
explain LIKE '%Granules: %'
|
explain LIKE '%Granules: %'
|
||||||
|
@ -13,19 +13,19 @@ af full_text
|
|||||||
1
|
1
|
||||||
Test full_text()
|
Test full_text()
|
||||||
af full_text
|
af full_text
|
||||||
101 Alick a01
|
101 x Alick a01 y
|
||||||
106 Alick a06
|
106 x Alick a06 y
|
||||||
111 Alick b01
|
111 x Alick b01 y
|
||||||
116 Alick b06
|
116 x Alick b06 y
|
||||||
101 Alick a01
|
101 x Alick a01 y
|
||||||
106 Alick a06
|
106 x Alick a06 y
|
||||||
1
|
1
|
||||||
101 Alick a01
|
101 x Alick a01 y
|
||||||
111 Alick b01
|
111 x Alick b01 y
|
||||||
1
|
1
|
||||||
Test on array columns
|
Test on array columns
|
||||||
af full_text
|
af full_text
|
||||||
3 ['Click a03','Click b03']
|
3 ['x Click a03 y','x Click b03 y']
|
||||||
1
|
1
|
||||||
Test on map columns
|
Test on map columns
|
||||||
af full_text
|
af full_text
|
||||||
|
@ -67,7 +67,7 @@ CREATE TABLE tab_x(k UInt64, s String, INDEX af(s) TYPE full_text())
|
|||||||
ENGINE = MergeTree() ORDER BY k
|
ENGINE = MergeTree() ORDER BY k
|
||||||
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';
|
||||||
|
|
||||||
INSERT INTO tab_x VALUES (101, 'Alick a01'), (102, 'Blick a02'), (103, 'Click a03'), (104, 'Dlick a04'), (105, 'Elick a05'), (106, 'Alick a06'), (107, 'Blick a07'), (108, 'Click a08'), (109, 'Dlick a09'), (110, 'Elick a10'), (111, 'Alick b01'), (112, 'Blick b02'), (113, 'Click b03'), (114, 'Dlick b04'), (115, 'Elick b05'), (116, 'Alick b06'), (117, 'Blick b07'), (118, 'Click b08'), (119, 'Dlick b09'), (120, 'Elick b10');
|
INSERT INTO tab_x VALUES (101, 'x Alick a01 y'), (102, 'x Blick a02 y'), (103, 'x Click a03 y'), (104, 'x Dlick a04 y'), (105, 'x Elick a05 y'), (106, 'x Alick a06 y'), (107, 'x Blick a07 y'), (108, 'x Click a08 y'), (109, 'x Dlick a09 y'), (110, 'x Elick a10 y'), (111, 'x Alick b01 y'), (112, 'x Blick b02 y'), (113, 'x Click b03 y'), (114, 'x Dlick b04 y'), (115, 'x Elick b05 y'), (116, 'x Alick b06 y'), (117, 'x Blick b07 y'), (118, 'x Click b08 y'), (119, 'x Dlick b09 y'), (120, 'x Elick b10 y');
|
||||||
|
|
||||||
-- check full_text index was created
|
-- check full_text index was created
|
||||||
SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab_x' AND database = currentDatabase() LIMIT 1;
|
SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab_x' AND database = currentDatabase() LIMIT 1;
|
||||||
@ -86,27 +86,27 @@ SELECT read_rows==8 from system.query_log
|
|||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
|
|
||||||
-- search full_text index with IN operator
|
-- search full_text index with IN operator
|
||||||
SELECT * FROM tab_x WHERE s IN ('Alick a01', 'Alick a06') ORDER BY k;
|
SELECT * FROM tab_x WHERE s IN ('x Alick a01 y', 'x Alick a06 y') ORDER BY k;
|
||||||
|
|
||||||
-- check the query only read 2 granules (4 rows total; each granule has 2 rows)
|
-- check the query only read 2 granules (4 rows total; each granule has 2 rows)
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT read_rows==4 from system.query_log
|
SELECT read_rows==4 from system.query_log
|
||||||
WHERE query_kind ='Select'
|
WHERE query_kind ='Select'
|
||||||
AND current_database = currentDatabase()
|
AND current_database = currentDatabase()
|
||||||
AND endsWith(trimRight(query), 'SELECT * FROM tab_x WHERE s IN (\'Alick a01\', \'Alick a06\') ORDER BY k;')
|
AND endsWith(trimRight(query), 'SELECT * FROM tab_x WHERE s IN (\'x Alick a01 y\', \'x Alick a06 y\') ORDER BY k;')
|
||||||
AND type='QueryFinish'
|
AND type='QueryFinish'
|
||||||
AND result_rows==2
|
AND result_rows==2
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
|
|
||||||
-- search full_text index with multiSearch
|
-- search full_text index with multiSearch
|
||||||
SELECT * FROM tab_x WHERE multiSearchAny(s, ['a01', 'b01']) ORDER BY k;
|
SELECT * FROM tab_x WHERE multiSearchAny(s, [' a01 ', ' b01 ']) ORDER BY k;
|
||||||
|
|
||||||
-- check the query only read 2 granules (4 rows total; each granule has 2 rows)
|
-- check the query only read 2 granules (4 rows total; each granule has 2 rows)
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT read_rows==4 from system.query_log
|
SELECT read_rows==4 from system.query_log
|
||||||
WHERE query_kind ='Select'
|
WHERE query_kind ='Select'
|
||||||
AND current_database = currentDatabase()
|
AND current_database = currentDatabase()
|
||||||
AND endsWith(trimRight(query), 'SELECT * FROM tab_x WHERE multiSearchAny(s, [\'a01\', \'b01\']) ORDER BY k;')
|
AND endsWith(trimRight(query), 'SELECT * FROM tab_x WHERE multiSearchAny(s, [\' a01 \', \' b01 \']) ORDER BY k;')
|
||||||
AND type='QueryFinish'
|
AND type='QueryFinish'
|
||||||
AND result_rows==2
|
AND result_rows==2
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
@ -126,14 +126,14 @@ INSERT INTO tab SELECT rowNumberInBlock(), groupArray(s) FROM tab_x GROUP BY k%1
|
|||||||
SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab' AND database = currentDatabase() LIMIT 1;
|
SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab' AND database = currentDatabase() LIMIT 1;
|
||||||
|
|
||||||
-- search full_text index with has
|
-- search full_text index with has
|
||||||
SELECT * FROM tab WHERE has(s, 'Click a03') ORDER BY k;
|
SELECT * FROM tab WHERE has(s, 'x Click a03 y') ORDER BY k;
|
||||||
|
|
||||||
-- check the query must read all 10 granules (20 rows total; each granule has 2 rows)
|
-- check the query must read all 10 granules (20 rows total; each granule has 2 rows)
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT read_rows==2 from system.query_log
|
SELECT read_rows==2 from system.query_log
|
||||||
WHERE query_kind ='Select'
|
WHERE query_kind ='Select'
|
||||||
AND current_database = currentDatabase()
|
AND current_database = currentDatabase()
|
||||||
AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE has(s, \'Click a03\') ORDER BY k;')
|
AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE has(s, \'x Click a03 y\') ORDER BY k;')
|
||||||
AND type='QueryFinish'
|
AND type='QueryFinish'
|
||||||
AND result_rows==1
|
AND result_rows==1
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
|
@ -1,5 +1,5 @@
|
|||||||
1 Hello ClickHouse
|
1 Well, Hello ClickHouse !
|
||||||
2 Hello World
|
2 Well, Hello World !
|
||||||
1 Hello ClickHouse
|
1 Hello ClickHouse
|
||||||
2 Hello World
|
2 Hello World
|
||||||
Granules: 6/6
|
Granules: 6/6
|
||||||
@ -11,9 +11,9 @@
|
|||||||
Granules: 6/6
|
Granules: 6/6
|
||||||
Granules: 2/6
|
Granules: 2/6
|
||||||
---
|
---
|
||||||
1 Hello ClickHouse
|
1 Well, Hello ClickHouse !
|
||||||
2 Hello World
|
2 Well, Hello World !
|
||||||
6 World Champion
|
6 True World Champion
|
||||||
1 Hello ClickHouse
|
1 Hello ClickHouse
|
||||||
2 Hello World
|
2 Hello World
|
||||||
6 World Champion
|
6 World Champion
|
||||||
@ -26,7 +26,7 @@
|
|||||||
Granules: 6/6
|
Granules: 6/6
|
||||||
Granules: 3/6
|
Granules: 3/6
|
||||||
---
|
---
|
||||||
5 OLAP Database
|
5 Its An OLAP Database
|
||||||
5 OLAP Database
|
5 OLAP Database
|
||||||
Granules: 6/6
|
Granules: 6/6
|
||||||
Granules: 1/6
|
Granules: 1/6
|
||||||
|
@ -21,21 +21,22 @@ ENGINE = MergeTree
|
|||||||
ORDER BY id
|
ORDER BY id
|
||||||
SETTINGS index_granularity = 1;
|
SETTINGS index_granularity = 1;
|
||||||
|
|
||||||
INSERT INTO tokenbf_tab VALUES (1, 'Hello ClickHouse'), (2, 'Hello World'), (3, 'Good Weather'), (4, 'Say Hello'), (5, 'OLAP Database'), (6, 'World Champion');
|
INSERT INTO tokenbf_tab VALUES (1, 'Well, Hello ClickHouse !'), (2, 'Well, Hello World !'), (3, 'Good Weather !'), (4, 'Say Hello !'), (5, 'Its An OLAP Database'), (6, 'True World Champion');
|
||||||
INSERT INTO ngrambf_tab VALUES (1, 'Hello ClickHouse'), (2, 'Hello World'), (3, 'Good Weather'), (4, 'Say Hello'), (5, 'OLAP Database'), (6, 'World Champion');
|
INSERT INTO ngrambf_tab VALUES (1, 'Hello ClickHouse'), (2, 'Hello World'), (3, 'Good Weather'), (4, 'Say Hello'), (5, 'OLAP Database'), (6, 'World Champion');
|
||||||
|
|
||||||
SELECT * FROM tokenbf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id;
|
SELECT * FROM tokenbf_tab WHERE match(str, ' Hello (ClickHouse|World) ') ORDER BY id;
|
||||||
SELECT * FROM ngrambf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id;
|
SELECT * FROM ngrambf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id;
|
||||||
|
|
||||||
-- Read 2/6 granules
|
-- Read 2/6 granules
|
||||||
-- Required string: 'Hello '
|
-- Required string: 'Hello '
|
||||||
-- Alternatives: 'Hello ClickHouse', 'Hello World'
|
-- Alternatives: 'Hello ClickHouse', 'Hello World'
|
||||||
|
-- Surrounded by spaces for tokenbf
|
||||||
|
|
||||||
SELECT *
|
SELECT *
|
||||||
FROM
|
FROM
|
||||||
(
|
(
|
||||||
EXPLAIN PLAN indexes=1
|
EXPLAIN PLAN indexes=1
|
||||||
SELECT * FROM tokenbf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id
|
SELECT * FROM tokenbf_tab WHERE match(str, ' Hello (ClickHouse|World) ') ORDER BY id
|
||||||
)
|
)
|
||||||
WHERE
|
WHERE
|
||||||
explain LIKE '%Granules: %'
|
explain LIKE '%Granules: %'
|
||||||
@ -46,7 +47,7 @@ SELECT *
|
|||||||
FROM
|
FROM
|
||||||
(
|
(
|
||||||
EXPLAIN PLAN indexes=1
|
EXPLAIN PLAN indexes=1
|
||||||
SELECT * FROM tokenbf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id
|
SELECT * FROM tokenbf_tab WHERE match(str, ' Hello (ClickHouse|World) ') ORDER BY id
|
||||||
)
|
)
|
||||||
WHERE
|
WHERE
|
||||||
explain LIKE '%Granules: %'
|
explain LIKE '%Granules: %'
|
||||||
@ -78,18 +79,19 @@ SETTINGS
|
|||||||
|
|
||||||
SELECT '---';
|
SELECT '---';
|
||||||
|
|
||||||
SELECT * FROM tokenbf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id;
|
SELECT * FROM tokenbf_tab WHERE match(str, '.* (ClickHouse|World) ') ORDER BY id;
|
||||||
SELECT * FROM ngrambf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id;
|
SELECT * FROM ngrambf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id;
|
||||||
|
|
||||||
-- Read 3/6 granules
|
-- Read 3/6 granules
|
||||||
-- Required string: -
|
-- Required string: -
|
||||||
-- Alternatives: 'ClickHouse', 'World'
|
-- Alternatives: 'ClickHouse', 'World'
|
||||||
|
-- Surrounded by spaces for tokenbf
|
||||||
|
|
||||||
SELECT *
|
SELECT *
|
||||||
FROM
|
FROM
|
||||||
(
|
(
|
||||||
EXPLAIN PLAN indexes = 1
|
EXPLAIN PLAN indexes = 1
|
||||||
SELECT * FROM tokenbf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id
|
SELECT * FROM tokenbf_tab WHERE match(str, '.* (ClickHouse|World) ') ORDER BY id
|
||||||
)
|
)
|
||||||
WHERE
|
WHERE
|
||||||
explain LIKE '%Granules: %'
|
explain LIKE '%Granules: %'
|
||||||
@ -100,7 +102,7 @@ SELECT *
|
|||||||
FROM
|
FROM
|
||||||
(
|
(
|
||||||
EXPLAIN PLAN indexes = 1
|
EXPLAIN PLAN indexes = 1
|
||||||
SELECT * FROM tokenbf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id
|
SELECT * FROM tokenbf_tab WHERE match(str, '.* (ClickHouse|World) ') ORDER BY id
|
||||||
)
|
)
|
||||||
WHERE
|
WHERE
|
||||||
explain LIKE '%Granules: %'
|
explain LIKE '%Granules: %'
|
||||||
@ -131,18 +133,19 @@ SETTINGS
|
|||||||
|
|
||||||
SELECT '---';
|
SELECT '---';
|
||||||
|
|
||||||
SELECT * FROM tokenbf_tab WHERE match(str, 'OLAP.*') ORDER BY id;
|
SELECT * FROM tokenbf_tab WHERE match(str, ' OLAP .*') ORDER BY id;
|
||||||
SELECT * FROM ngrambf_tab WHERE match(str, 'OLAP.*') ORDER BY id;
|
SELECT * FROM ngrambf_tab WHERE match(str, 'OLAP.*') ORDER BY id;
|
||||||
|
|
||||||
-- Read 1/6 granules
|
-- Read 1/6 granules
|
||||||
-- Required string: 'OLAP'
|
-- Required string: 'OLAP'
|
||||||
-- Alternatives: -
|
-- Alternatives: -
|
||||||
|
-- Surrounded by spaces for tokenbf
|
||||||
|
|
||||||
SELECT *
|
SELECT *
|
||||||
FROM
|
FROM
|
||||||
(
|
(
|
||||||
EXPLAIN PLAN indexes = 1
|
EXPLAIN PLAN indexes = 1
|
||||||
SELECT * FROM tokenbf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id
|
SELECT * FROM tokenbf_tab WHERE match(str, ' OLAP (.*?)*') ORDER BY id
|
||||||
)
|
)
|
||||||
WHERE
|
WHERE
|
||||||
explain LIKE '%Granules: %'
|
explain LIKE '%Granules: %'
|
||||||
@ -152,7 +155,7 @@ SELECT *
|
|||||||
FROM
|
FROM
|
||||||
(
|
(
|
||||||
EXPLAIN PLAN indexes = 1
|
EXPLAIN PLAN indexes = 1
|
||||||
SELECT * FROM tokenbf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id
|
SELECT * FROM tokenbf_tab WHERE match(str, ' OLAP (.*?)*') ORDER BY id
|
||||||
)
|
)
|
||||||
WHERE
|
WHERE
|
||||||
explain LIKE '%Granules: %'
|
explain LIKE '%Granules: %'
|
||||||
|
@ -0,0 +1,83 @@
|
|||||||
|
-------- Bloom filter --------
|
||||||
|
|
||||||
|
-- No skip for prefix
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 1/1
|
||||||
|
1 Service is not ready
|
||||||
|
|
||||||
|
-- Skip for prefix with complete token
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 0/1
|
||||||
|
|
||||||
|
-- No skip for suffix
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 1/1
|
||||||
|
1 Service is not ready
|
||||||
|
|
||||||
|
-- Skip for suffix with complete token
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 0/1
|
||||||
|
|
||||||
|
-- No skip for substring
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 1/1
|
||||||
|
1 Service is not ready
|
||||||
|
|
||||||
|
-- Skip for substring with complete token
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 0/1
|
||||||
|
|
||||||
|
-- No skip for multiple substrings
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 1/1
|
||||||
|
1 Service is not ready
|
||||||
|
|
||||||
|
-- Skip for multiple substrings with complete tokens
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 0/1
|
||||||
|
|
||||||
|
-- No skip for multiple non-existsing substrings, only one with complete token
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 1/1
|
||||||
|
|
||||||
|
-------- GIN filter --------
|
||||||
|
|
||||||
|
-- No skip for prefix
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 1/1
|
||||||
|
1 Service is not ready
|
||||||
|
|
||||||
|
-- Skip for prefix with complete token
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 0/1
|
||||||
|
|
||||||
|
-- No skip for suffix
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 1/1
|
||||||
|
1 Service is not ready
|
||||||
|
|
||||||
|
-- Skip for suffix with complete token
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 0/1
|
||||||
|
|
||||||
|
-- No skip for substring
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 1/1
|
||||||
|
1 Service is not ready
|
||||||
|
|
||||||
|
-- Skip for substring with complete token
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 0/1
|
||||||
|
|
||||||
|
-- No skip for multiple substrings
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 1/1
|
||||||
|
1 Service is not ready
|
||||||
|
|
||||||
|
-- Skip for multiple substrings with complete tokens
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 0/1
|
||||||
|
|
||||||
|
-- No skip for multiple non-existsing substrings, only one with complete token
|
||||||
|
Parts: 1/1
|
||||||
|
Parts: 1/1
|
@ -0,0 +1,227 @@
|
|||||||
|
SELECT '-------- Bloom filter --------';
|
||||||
|
SELECT '';
|
||||||
|
DROP TABLE IF EXISTS 03165_token_bf;
|
||||||
|
|
||||||
|
CREATE TABLE 03165_token_bf
|
||||||
|
(
|
||||||
|
id Int64,
|
||||||
|
message String,
|
||||||
|
INDEX idx_message message TYPE tokenbf_v1(32768, 3, 2) GRANULARITY 1
|
||||||
|
)
|
||||||
|
ENGINE = MergeTree
|
||||||
|
ORDER BY id;
|
||||||
|
|
||||||
|
INSERT INTO 03165_token_bf VALUES(1, 'Service is not ready');
|
||||||
|
|
||||||
|
SELECT '-- No skip for prefix';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE startsWith(message, 'Serv')
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_bf WHERE startsWith(message, 'Serv');
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- Skip for prefix with complete token';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE startsWith(message, 'Serv i')
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_bf WHERE startsWith(message, 'Serv i');
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- No skip for suffix';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE endsWith(message, 'eady')
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_bf WHERE endsWith(message, 'eady');
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- Skip for suffix with complete token';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE endsWith(message, ' eady')
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_bf WHERE endsWith(message, ' eady');
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- No skip for substring';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE match(message, 'no')
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_bf WHERE match(message, 'no');
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- Skip for substring with complete token';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE match(message, ' xyz ')
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_bf WHERE match(message, ' xyz ');
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- No skip for multiple substrings';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE multiSearchAny(message, ['ce', 'no'])
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_bf WHERE multiSearchAny(message, ['ce', 'no']);
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- Skip for multiple substrings with complete tokens';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE multiSearchAny(message, [' wx ', ' yz '])
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_bf WHERE multiSearchAny(message, [' wx ', ' yz ']);
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- No skip for multiple non-existsing substrings, only one with complete token';
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_bf WHERE multiSearchAny(message, [' wx ', 'yz'])
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_bf WHERE multiSearchAny(message, [' wx ', 'yz']);
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS 03165_token_bf;
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-------- GIN filter --------';
|
||||||
|
SELECT '';
|
||||||
|
|
||||||
|
SET allow_experimental_inverted_index=1;
|
||||||
|
DROP TABLE IF EXISTS 03165_token_ft;
|
||||||
|
CREATE TABLE 03165_token_ft
|
||||||
|
(
|
||||||
|
id Int64,
|
||||||
|
message String,
|
||||||
|
INDEX idx_message message TYPE full_text() GRANULARITY 1
|
||||||
|
)
|
||||||
|
ENGINE = MergeTree
|
||||||
|
ORDER BY id;
|
||||||
|
|
||||||
|
INSERT INTO 03165_token_ft VALUES(1, 'Service is not ready');
|
||||||
|
|
||||||
|
SELECT '-- No skip for prefix';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE startsWith(message, 'Serv')
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_ft WHERE startsWith(message, 'Serv');
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- Skip for prefix with complete token';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE startsWith(message, 'Serv i')
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_ft WHERE startsWith(message, 'Serv i');
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- No skip for suffix';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE endsWith(message, 'eady')
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_ft WHERE endsWith(message, 'eady');
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- Skip for suffix with complete token';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE endsWith(message, ' eady')
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_ft WHERE endsWith(message, ' eady');
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- No skip for substring';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE match(message, 'no')
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_ft WHERE match(message, 'no');
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- Skip for substring with complete token';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE match(message, ' xyz ')
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_ft WHERE match(message, ' xyz ');
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- No skip for multiple substrings';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE multiSearchAny(message, ['ce', 'no'])
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_ft WHERE multiSearchAny(message, ['ce', 'no']);
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- Skip for multiple substrings with complete tokens';
|
||||||
|
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE multiSearchAny(message, [' wx ', ' yz '])
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_ft WHERE multiSearchAny(message, [' wx ', ' yz ']);
|
||||||
|
|
||||||
|
SELECT '';
|
||||||
|
SELECT '-- No skip for multiple non-existsing substrings, only one with complete token';
|
||||||
|
SELECT trim(explain)
|
||||||
|
FROM (
|
||||||
|
EXPLAIN indexes = 1 SELECT * FROM 03165_token_ft WHERE multiSearchAny(message, [' wx ', 'yz'])
|
||||||
|
)
|
||||||
|
WHERE explain LIKE '%Parts:%';
|
||||||
|
|
||||||
|
SELECT * FROM 03165_token_ft WHERE multiSearchAny(message, [' wx ', 'yz']);
|
@ -2000,6 +2000,8 @@ minmax
|
|||||||
mins
|
mins
|
||||||
misconfiguration
|
misconfiguration
|
||||||
mispredictions
|
mispredictions
|
||||||
|
mlock
|
||||||
|
mlockall
|
||||||
mmap
|
mmap
|
||||||
mmapped
|
mmapped
|
||||||
modularization
|
modularization
|
||||||
|
Loading…
Reference in New Issue
Block a user