Merge branch 'master' into parallel-compression

This commit is contained in:
Alexey Milovidov 2024-11-09 01:08:19 +01:00
commit 68c270abfc
49 changed files with 181 additions and 69 deletions

View File

@ -47,6 +47,7 @@ Upcoming meetups
* [Dubai Meetup](https://www.meetup.com/clickhouse-dubai-meetup-group/events/303096989/) - November 21
* [Paris Meetup](https://www.meetup.com/clickhouse-france-user-group/events/303096434) - November 26
* [Amsterdam Meetup](https://www.meetup.com/clickhouse-netherlands-user-group/events/303638814) - December 3
* [Stockholm Meetup](https://www.meetup.com/clickhouse-stockholm-user-group/events/304382411) - December 9
* [New York Meetup](https://www.meetup.com/clickhouse-new-york-user-group/events/304268174) - December 9
* [San Francisco Meetup](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/304286951/) - December 12

View File

@ -54,7 +54,7 @@ Parameters:
- `distance_function`: either `L2Distance` (the [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance) - the length of a
line between two points in Euclidean space), or `cosineDistance` (the [cosine
distance](https://en.wikipedia.org/wiki/Cosine_similarity#Cosine_distance)- the angle between two non-zero vectors).
- `quantization`: either `f64`, `f32`, `f16`, `bf16`, or `i8` for storing the vector with reduced precision (optional, default: `bf16`)
- `quantization`: either `f64`, `f32`, `f16`, `bf16`, or `i8` for storing vectors with reduced precision (optional, default: `bf16`)
- `hnsw_max_connections_per_layer`: the number of neighbors per HNSW graph node, also known as `M` in the [HNSW
paper](https://doi.org/10.1109/TPAMI.2018.2889473) (optional, default: 32)
- `hnsw_candidate_list_size_for_construction`: the size of the dynamic candidate list when constructing the HNSW graph, also known as
@ -92,8 +92,8 @@ Vector similarity indexes currently support two distance functions:
- `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors
([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)).
Vector similarity indexes allows storing the vectors in reduced precision formats. Supported scalar kinds are `f64`, `f32`, `f16` or `i8`.
If no scalar kind was specified during index creation, `f16` is used as default.
Vector similarity indexes allows storing the vectors in reduced precision formats. Supported scalar kinds are `f64`, `f32`, `f16`, `bf16`,
and `i8`. If no scalar kind was specified during index creation, `bf16` is used as default.
For normalized data, `L2Distance` is usually a better choice, otherwise `cosineDistance` is recommended to compensate for scale. If no
distance function was specified during index creation, `L2Distance` is used as default.

View File

@ -131,16 +131,6 @@ Type: UInt64
Default: 8
## background_pool_size
Sets the number of threads performing background merges and mutations for tables with MergeTree engines. You can only increase the number of threads at runtime. To lower the number of threads you have to restart the server. By adjusting this setting, you manage CPU and disk load. Smaller pool size utilizes less CPU and disk resources, but background processes advance slower which might eventually impact query performance.
Before changing it, please also take a look at related MergeTree settings, such as `number_of_free_entries_in_pool_to_lower_max_size_of_merge` and `number_of_free_entries_in_pool_to_execute_mutation`.
Type: UInt64
Default: 16
## background_schedule_pool_size
The maximum number of threads that will be used for constantly executing some lightweight periodic operations for replicated tables, Kafka streaming, and DNS cache updates.

View File

@ -59,7 +59,13 @@ void setUserAndGroup(std::string arg_uid, std::string arg_gid)
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot do 'getgrnam_r' to obtain gid from group name ({})", arg_gid);
if (!result)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Group {} is not found in the system", arg_gid);
{
if (0 != getgrgid_r(gid, &entry, buf.get(), buf_size, &result))
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot do 'getgrnam_r' to obtain gid from group name ({})", arg_gid);
if (!result)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Group {} is not found in the system", arg_gid);
}
gid = entry.gr_gid;
}
@ -84,7 +90,13 @@ void setUserAndGroup(std::string arg_uid, std::string arg_gid)
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot do 'getpwnam_r' to obtain uid from user name ({})", arg_uid);
if (!result)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "User {} is not found in the system", arg_uid);
{
if (0 != getpwuid_r(uid, &entry, buf.get(), buf_size, &result))
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot do 'getpwuid_r' to obtain uid from user name ({})", uid);
if (!result)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "User {} is not found in the system", arg_uid);
}
uid = entry.pw_uid;
}

View File

@ -330,7 +330,7 @@ TYPED_TEST(CoordinationTest, TestSummingRaft1)
this->setLogDirectory("./logs");
this->setStateFileDirectory(".");
SummingRaftServer s1(1, "localhost", 44444, this->keeper_context);
SummingRaftServer s1(1, "localhost", 0, this->keeper_context);
SCOPE_EXIT(if (std::filesystem::exists("./state")) std::filesystem::remove("./state"););
/// Single node is leader

View File

@ -2869,7 +2869,7 @@ Limit on size of multipart/form-data content. This setting cannot be parsed from
DECLARE(Bool, calculate_text_stack_trace, true, R"(
Calculate text stack trace in case of exceptions during query execution. This is the default. It requires symbol lookups that may slow down fuzzing tests when a huge amount of wrong queries are executed. In normal cases, you should not disable this option.
)", 0) \
DECLARE(Bool, enable_job_stack_trace, false, R"(
DECLARE(Bool, enable_job_stack_trace, true, R"(
Output stack trace of a job creator when job results in exception
)", 0) \
DECLARE(Bool, allow_ddl, true, R"(

View File

@ -64,6 +64,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
},
{"24.11",
{
{"enable_job_stack_trace", false, true, "Enable by default collecting stack traces from job's scheduling."},
{"allow_suspicious_types_in_group_by", true, false, "Don't allow Variant/Dynamic types in GROUP BY by default"},
{"allow_suspicious_types_in_order_by", true, false, "Don't allow Variant/Dynamic types in ORDER BY by default"},
{"distributed_cache_discard_connection_if_unread_data", true, true, "New setting"},

View File

@ -25,8 +25,10 @@ struct BitShiftLeftImpl
{
if constexpr (is_big_int_v<B>)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftLeft is not implemented for big integers as second argument");
else if (b < 0 || static_cast<UInt256>(b) > 8 * sizeof(A))
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift");
else if (b < 0)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value");
else if (static_cast<UInt256>(b) > 8 * sizeof(A))
return static_cast<Result>(0);
else if constexpr (is_big_int_v<A>)
return static_cast<Result>(a) << static_cast<UInt32>(b);
else
@ -43,9 +45,10 @@ struct BitShiftLeftImpl
const UInt8 word_size = 8 * sizeof(*pos);
size_t n = end - pos;
const UInt128 bit_limit = static_cast<UInt128>(word_size) * n;
if (b < 0 || static_cast<decltype(bit_limit)>(b) > bit_limit)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift");
if (b == bit_limit)
if (b < 0)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value");
if (b == bit_limit || static_cast<decltype(bit_limit)>(b) > bit_limit)
{
// insert default value
out_vec.push_back(0);
@ -111,9 +114,10 @@ struct BitShiftLeftImpl
const UInt8 word_size = 8;
size_t n = end - pos;
const UInt128 bit_limit = static_cast<UInt128>(word_size) * n;
if (b < 0 || static_cast<decltype(bit_limit)>(b) > bit_limit)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift");
if (b == bit_limit)
if (b < 0)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value");
if (b == bit_limit || static_cast<decltype(bit_limit)>(b) > bit_limit)
{
// insert default value
out_vec.resize_fill(out_vec.size() + n);

View File

@ -26,8 +26,10 @@ struct BitShiftRightImpl
{
if constexpr (is_big_int_v<B>)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftRight is not implemented for big integers as second argument");
else if (b < 0 || static_cast<UInt256>(b) > 8 * sizeof(A))
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift");
else if (b < 0)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value");
else if (static_cast<UInt256>(b) > 8 * sizeof(A))
return static_cast<Result>(0);
else if constexpr (is_big_int_v<A>)
return static_cast<Result>(a) >> static_cast<UInt32>(b);
else
@ -59,9 +61,10 @@ struct BitShiftRightImpl
const UInt8 word_size = 8;
size_t n = end - pos;
const UInt128 bit_limit = static_cast<UInt128>(word_size) * n;
if (b < 0 || static_cast<decltype(bit_limit)>(b) > bit_limit)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift");
if (b == bit_limit)
if (b < 0)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value");
if (b == bit_limit || static_cast<decltype(bit_limit)>(b) > bit_limit)
{
/// insert default value
out_vec.push_back(0);
@ -99,9 +102,10 @@ struct BitShiftRightImpl
const UInt8 word_size = 8;
size_t n = end - pos;
const UInt128 bit_limit = static_cast<UInt128>(word_size) * n;
if (b < 0 || static_cast<decltype(bit_limit)>(b) > bit_limit)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift");
if (b == bit_limit)
if (b < 0)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value");
if (b == bit_limit || static_cast<decltype(bit_limit)>(b) > bit_limit)
{
// insert default value
out_vec.resize_fill(out_vec.size() + n);

View File

@ -5,6 +5,8 @@
namespace DB
{
class Context;
/// Sink which is returned from Storage::write.
class SinkToStorage : public ExceptionKeepingTransform
{
@ -16,12 +18,14 @@ public:
const Block & getHeader() const { return inputs.front().getHeader(); }
void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); }
void addInterpreterContext(std::shared_ptr<const Context> context) { interpreter_context.emplace_back(std::move(context)); }
protected:
virtual void consume(Chunk & chunk) = 0;
private:
std::vector<TableLockHolder> table_locks;
std::vector<std::shared_ptr<const Context>> interpreter_context;
void onConsume(Chunk chunk) override;
GenerateResult onGenerate() override;

View File

@ -735,7 +735,9 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks
loadUUID();
loadColumns(require_columns_checksums);
loadChecksums(require_columns_checksums);
loadIndexGranularity();
index_granularity.shrinkToFitInMemory();
if (!(*storage.getSettings())[MergeTreeSetting::primary_key_lazy_load])
getIndex();

View File

@ -116,7 +116,7 @@ bool allow(
double sum_size,
double max_size,
double min_age,
double range_size,
size_t range_size,
double partition_size,
double min_size_to_lower_base_log,
double max_size_to_lower_base_log,
@ -125,6 +125,9 @@ bool allow(
if (settings.min_age_to_force_merge && min_age >= settings.min_age_to_force_merge)
return true;
if (settings.min_parts_to_merge_at_once && range_size < settings.min_parts_to_merge_at_once)
return false;
/// Map size to 0..1 using logarithmic scale
/// Use log(1 + x) instead of log1p(x) because our sum_size is always integer.
/// Also log1p seems to be slow and significantly affect performance of merges assignment.

View File

@ -90,6 +90,8 @@ public:
{
/// Zero means unlimited. Can be overridden by the same merge tree setting.
size_t max_parts_to_merge_at_once = 100;
/// Zero means no minimum. Can be overridden by the same merge tree setting.
size_t min_parts_to_merge_at_once = 0;
/// Some sort of a maximum number of parts in partition. Can be overridden by the same merge tree setting.
size_t parts_to_throw_insert = 3000;

View File

@ -82,6 +82,7 @@ namespace MergeTreeSetting
extern const MergeTreeSettingsMergeSelectorAlgorithm merge_selector_algorithm;
extern const MergeTreeSettingsBool merge_selector_enable_heuristic_to_remove_small_parts_at_right;
extern const MergeTreeSettingsFloat merge_selector_base;
extern const MergeTreeSettingsUInt64 min_parts_to_merge_at_once;
}
namespace ErrorCodes
@ -267,7 +268,8 @@ MergeTreeDataMergerMutator::PartitionIdsHint MergeTreeDataMergerMutator::getPart
if (status == SelectPartsDecision::SELECTED)
res.insert(all_partition_ids[i]);
else
LOG_TEST(log, "Nothing to merge in partition {}: {}", all_partition_ids[i], out_disable_reason.text);
LOG_TEST(log, "Nothing to merge in partition {} with max_total_size_to_merge = {} (looked up {} ranges): {}",
all_partition_ids[i], ReadableSize(max_total_size_to_merge), ranges_per_partition[i].size(), out_disable_reason.text);
}
String best_partition_id_to_optimize = getBestPartitionToOptimizeEntire(info.partitions_info);
@ -565,6 +567,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges(
simple_merge_settings.max_parts_to_merge_at_once = (*data_settings)[MergeTreeSetting::max_parts_to_merge_at_once];
simple_merge_settings.enable_heuristic_to_remove_small_parts_at_right = (*data_settings)[MergeTreeSetting::merge_selector_enable_heuristic_to_remove_small_parts_at_right];
simple_merge_settings.base = (*data_settings)[MergeTreeSetting::merge_selector_base];
simple_merge_settings.min_parts_to_merge_at_once = (*data_settings)[MergeTreeSetting::min_parts_to_merge_at_once];
if (!(*data_settings)[MergeTreeSetting::min_age_to_force_merge_on_partition_only])
simple_merge_settings.min_age_to_force_merge = (*data_settings)[MergeTreeSetting::min_age_to_force_merge_seconds];

View File

@ -122,4 +122,10 @@ std::string MergeTreeIndexGranularity::describe() const
{
return fmt::format("initialized: {}, marks_rows_partial_sums: [{}]", initialized, fmt::join(marks_rows_partial_sums, ", "));
}
void MergeTreeIndexGranularity::shrinkToFitInMemory()
{
marks_rows_partial_sums.shrink_to_fit();
}
}

View File

@ -100,6 +100,8 @@ public:
void resizeWithFixedGranularity(size_t size, size_t fixed_granularity);
std::string describe() const;
void shrinkToFitInMemory();
};
}

View File

@ -178,23 +178,20 @@ String USearchIndexWithSerialization::Statistics::toString() const
}
MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity(
const String & index_name_,
const Block & index_sample_block_,
unum::usearch::metric_kind_t metric_kind_,
unum::usearch::scalar_kind_t scalar_kind_,
UsearchHnswParams usearch_hnsw_params_)
: MergeTreeIndexGranuleVectorSimilarity(index_name_, index_sample_block_, metric_kind_, scalar_kind_, usearch_hnsw_params_, nullptr)
: MergeTreeIndexGranuleVectorSimilarity(index_name_, metric_kind_, scalar_kind_, usearch_hnsw_params_, nullptr)
{
}
MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity(
const String & index_name_,
const Block & index_sample_block_,
unum::usearch::metric_kind_t metric_kind_,
unum::usearch::scalar_kind_t scalar_kind_,
UsearchHnswParams usearch_hnsw_params_,
USearchIndexWithSerializationPtr index_)
: index_name(index_name_)
, index_sample_block(index_sample_block_)
, metric_kind(metric_kind_)
, scalar_kind(scalar_kind_)
, usearch_hnsw_params(usearch_hnsw_params_)
@ -261,7 +258,7 @@ MergeTreeIndexAggregatorVectorSimilarity::MergeTreeIndexAggregatorVectorSimilari
MergeTreeIndexGranulePtr MergeTreeIndexAggregatorVectorSimilarity::getGranuleAndReset()
{
auto granule = std::make_shared<MergeTreeIndexGranuleVectorSimilarity>(index_name, index_sample_block, metric_kind, scalar_kind, usearch_hnsw_params, index);
auto granule = std::make_shared<MergeTreeIndexGranuleVectorSimilarity>(index_name, metric_kind, scalar_kind, usearch_hnsw_params, index);
index = nullptr;
return granule;
}
@ -490,7 +487,7 @@ MergeTreeIndexVectorSimilarity::MergeTreeIndexVectorSimilarity(
MergeTreeIndexGranulePtr MergeTreeIndexVectorSimilarity::createIndexGranule() const
{
return std::make_shared<MergeTreeIndexGranuleVectorSimilarity>(index.name, index.sample_block, metric_kind, scalar_kind, usearch_hnsw_params);
return std::make_shared<MergeTreeIndexGranuleVectorSimilarity>(index.name, metric_kind, scalar_kind, usearch_hnsw_params);
}
MergeTreeIndexAggregatorPtr MergeTreeIndexVectorSimilarity::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const
@ -531,15 +528,17 @@ void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* atta
{
const bool has_two_args = (index.arguments.size() == 2);
const bool has_five_args = (index.arguments.size() == 5);
const bool has_six_args = (index.arguments.size() == 6); /// Legacy index creation syntax before #70616. Supported only to be able to load old tables, can be removed mid-2025.
/// The 6th argument (ef_search) is ignored.
/// Check number and type of arguments
if (!has_two_args && !has_five_args)
if (!has_two_args && !has_five_args && !has_six_args)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Vector similarity index must have two or five arguments");
if (index.arguments[0].getType() != Field::Types::String)
throw Exception(ErrorCodes::INCORRECT_QUERY, "First argument of vector similarity index (method) must be of type String");
if (index.arguments[1].getType() != Field::Types::String)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Second argument of vector similarity index (metric) must be of type String");
if (has_five_args)
if (has_five_args || has_six_args)
{
if (index.arguments[2].getType() != Field::Types::String)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Third argument of vector similarity index (quantization) must be of type String");

View File

@ -69,14 +69,12 @@ struct MergeTreeIndexGranuleVectorSimilarity final : public IMergeTreeIndexGranu
{
MergeTreeIndexGranuleVectorSimilarity(
const String & index_name_,
const Block & index_sample_block_,
unum::usearch::metric_kind_t metric_kind_,
unum::usearch::scalar_kind_t scalar_kind_,
UsearchHnswParams usearch_hnsw_params_);
MergeTreeIndexGranuleVectorSimilarity(
const String & index_name_,
const Block & index_sample_block_,
unum::usearch::metric_kind_t metric_kind_,
unum::usearch::scalar_kind_t scalar_kind_,
UsearchHnswParams usearch_hnsw_params_,
@ -90,7 +88,6 @@ struct MergeTreeIndexGranuleVectorSimilarity final : public IMergeTreeIndexGranu
bool empty() const override { return !index || index->size() == 0; }
const String index_name;
const Block index_sample_block;
const unum::usearch::metric_kind_t metric_kind;
const unum::usearch::scalar_kind_t scalar_kind;
const UsearchHnswParams usearch_hnsw_params;

View File

@ -102,6 +102,7 @@ namespace ErrorCodes
DECLARE(MergeSelectorAlgorithm, merge_selector_algorithm, MergeSelectorAlgorithm::SIMPLE, "The algorithm to select parts for merges assignment", EXPERIMENTAL) \
DECLARE(Bool, merge_selector_enable_heuristic_to_remove_small_parts_at_right, true, "Enable heuristic for selecting parts for merge which removes parts from right side of range, if their size is less than specified ratio (0.01) of sum_size. Works for Simple and StochasticSimple merge selectors", 0) \
DECLARE(Float, merge_selector_base, 5.0, "Affects write amplification of assigned merges (expert level setting, don't change if you don't understand what it is doing). Works for Simple and StochasticSimple merge selectors", 0) \
DECLARE(UInt64, min_parts_to_merge_at_once, 0, "Minimal amount of data parts which merge selector can pick to merge at once (expert level setting, don't change if you don't understand what it is doing). 0 - disabled. Works for Simple and StochasticSimple merge selectors.", 0) \
\
/** Inserts settings. */ \
DECLARE(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \

View File

@ -207,6 +207,8 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync(
new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk());
new_part->setBytesUncompressedOnDisk(checksums.getTotalSizeUncompressedOnDisk());
new_part->index_granularity = writer->getIndexGranularity();
/// Just in case
new_part->index_granularity.shrinkToFitInMemory();
new_part->calculateColumnsAndSecondaryIndicesSizesOnDisk();
/// In mutation, existing_rows_count is already calculated in PartMergerWriter

View File

@ -984,6 +984,8 @@ void finalizeMutatedPart(
new_data_part->rows_count = source_part->rows_count;
new_data_part->index_granularity = source_part->index_granularity;
/// Just in case
new_data_part->index_granularity.shrinkToFitInMemory();
new_data_part->setIndex(*source_part->getIndex());
new_data_part->minmax_idx = source_part->minmax_idx;
new_data_part->modification_time = time(nullptr);

View File

@ -392,6 +392,7 @@ void StorageMaterializedView::read(
}
query_plan.addStorageHolder(storage);
query_plan.addInterpreterContext(context);
query_plan.addTableLock(std::move(lock));
}
}
@ -415,6 +416,7 @@ SinkToStoragePtr StorageMaterializedView::write(const ASTPtr & query, const Stor
auto sink = storage->write(query, metadata_snapshot, context, async_insert);
sink->addInterpreterContext(context);
sink->addTableLock(lock);
return sink;
}

View File

@ -5,7 +5,7 @@ services:
environment:
MYSQL_ROOT_PASSWORD: clickhouse
MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST}
DATADIR: /mysql/
DATADIR: /var/log/mysql/
expose:
- ${MYSQL_PORT:-3306}
command: --server_id=100
@ -14,11 +14,11 @@ services:
--gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3
--log-error=/mysql/error.log
--log-error=/var/log/mysql/error.log
--general-log=ON
--general-log-file=/mysql/general.log
--general-log-file=/var/log/mysql/general.log
volumes:
- type: ${MYSQL_LOGS_FS:-tmpfs}
source: ${MYSQL_LOGS:-}
target: /mysql/
target: /var/log/mysql/
user: ${MYSQL_DOCKER_USER}

View File

@ -4,8 +4,8 @@ services:
restart: always
environment:
MYSQL_ROOT_PASSWORD: clickhouse
MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST}
DATADIR: /mysql/
MYSQL_ROOT_HOST: ${MYSQL8_ROOT_HOST}
DATADIR: /var/log/mysql/
expose:
- ${MYSQL8_PORT:-3306}
command: --server_id=100 --log-bin='mysql-bin-1.log'
@ -13,11 +13,11 @@ services:
--default-time-zone='+3:00' --gtid-mode="ON"
--enforce-gtid-consistency
--log-error-verbosity=3
--log-error=/mysql/error.log
--log-error=/var/log/mysql/error.log
--general-log=ON
--general-log-file=/mysql/general.log
--general-log-file=/var/log/mysql/general.log
volumes:
- type: ${MYSQL8_LOGS_FS:-tmpfs}
source: ${MYSQL8_LOGS:-}
target: /mysql/
target: /var/log/mysql/
user: ${MYSQL8_DOCKER_USER}

View File

@ -1403,8 +1403,8 @@ def test_shards_distributed(started_cluster, mode, processing_threads):
# A unique path is necessary for repeatable tests
keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}"
files_path = f"{table_name}_data"
files_to_generate = 300
row_num = 300
files_to_generate = 600
row_num = 1000
total_rows = row_num * files_to_generate
shards_num = 2

View File

@ -2,7 +2,7 @@
SET allow_experimental_vector_similarity_index = 1;
-- Issue #71381: Usage of vector similarity index and further skipping indexes on the same table
-- Usage of vector similarity index and further skipping indexes on the same table (issue #71381)
DROP TABLE IF EXISTS tab;

View File

@ -2,7 +2,7 @@
SET allow_experimental_vector_similarity_index = 1;
-- Issue #52258: Vector similarity indexes must reject empty Arrays or Arrays with default values
-- Vector similarity indexes must reject empty Arrays or Arrays with default values (issue #52258)
DROP TABLE IF EXISTS tab;

View File

@ -0,0 +1,13 @@
-- Tags: no-fasttest, no-ordinary-database
-- Tests the legacy syntax to create vector similarity indexes before #70616.
-- Support for this syntax can be removed after mid-2025.
SET allow_experimental_vector_similarity_index = 1;
DROP TABLE IF EXISTS tab;
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f32', 42, 99, 113)) ENGINE = MergeTree ORDER BY id; -- Note the 6th parameter: 133
DROP TABLE tab;

View File

@ -3,7 +3,7 @@
SET allow_experimental_vector_similarity_index = 1;
SET enable_analyzer = 0;
-- Issue #69085: Reference vector for vector search is computed by a subquery
-- Reference vector for vector search is computed by a subquery (issue #69085)
DROP TABLE IF EXISTS tab;

View File

@ -10,7 +10,7 @@ DROP TABLE IF EXISTS t1;
CREATE TABLE t0 (vkey UInt32, pkey UInt32, c0 UInt32) engine = TinyLog;
CREATE TABLE t1 (vkey UInt32) ENGINE = AggregatingMergeTree ORDER BY vkey;
INSERT INTO t0 VALUES (15, 25000, 58);
SELECT ref_5.pkey AS c_2_c2392_6 FROM t0 AS ref_5 WHERE 'J[' < multiIf(ref_5.pkey IN ( SELECT 1 ), bitShiftLeft(multiIf(ref_5.c0 > NULL, '1', ')'), 40), NULL); -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT ref_5.pkey AS c_2_c2392_6 FROM t0 AS ref_5 WHERE 'J[' < multiIf(ref_5.pkey IN ( SELECT 1 ), bitShiftLeft(multiIf(ref_5.c0 > NULL, '1', ')'), 40), NULL);
DROP TABLE t0;
DROP TABLE t1;

View File

@ -1,3 +1,9 @@
-- bitShiftRight
0
\0\0\0\0\0\0\0\0
-- bitShiftLeft
0
\0\0\0\0\0\0\0\0
OK

View File

@ -1,17 +1,17 @@
SELECT '-- bitShiftRight';
SELECT bitShiftRight(1, -1); -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT bitShiftRight(toUInt8(1), 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT bitShiftRight(toUInt8(1), 8 + 1);
SELECT bitShiftRight('hola', -1); -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT bitShiftRight('hola', 4 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT bitShiftRight('hola', 4 * 8 + 1);
SELECT bitShiftRight(toFixedString('hola', 8), -1); -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT bitShiftRight(toFixedString('hola', 8), 8 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT bitShiftRight(toFixedString('hola', 8), 8 * 8 + 1);
SELECT '-- bitShiftLeft';
SELECT bitShiftLeft(1, -1); -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT bitShiftLeft(toUInt8(1), 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT bitShiftLeft(toUInt8(1), 8 + 1);
SELECT bitShiftLeft('hola', -1); -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT bitShiftLeft('hola', 4 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT bitShiftLeft('hola', 4 * 8 + 1);
SELECT bitShiftLeft(toFixedString('hola', 8), -1); -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT bitShiftLeft(toFixedString('hola', 8), 8 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND }
SELECT bitShiftLeft(toFixedString('hola', 8), 8 * 8 + 1);
SELECT 'OK';

View File

@ -18,12 +18,12 @@ ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM ghdata WHERE NOT ignore(*)"
${CLICKHOUSE_CLIENT} -q \
"SELECT data.repo.name, count() AS stars FROM ghdata \
WHERE data.type = 'WatchEvent' GROUP BY data.repo.name ORDER BY stars DESC, data.repo.name LIMIT 5"
WHERE data.type = 'WatchEvent' GROUP BY data.repo.name ORDER BY stars DESC, data.repo.name LIMIT 5" --allow_suspicious_types_in_group_by=1 --allow_suspicious_types_in_order_by=1
${CLICKHOUSE_CLIENT} --enable_analyzer=1 -q \
"SELECT data.payload.commits[].author.name AS name, count() AS c FROM ghdata \
ARRAY JOIN data.payload.commits[].author.name \
GROUP BY name ORDER BY c DESC, name LIMIT 5"
GROUP BY name ORDER BY c DESC, name LIMIT 5" --allow_suspicious_types_in_group_by=1 --allow_suspicious_types_in_order_by=1
${CLICKHOUSE_CLIENT} -q "SELECT max(data.payload.pull_request.assignees[].size0) FROM ghdata"

View File

@ -0,0 +1,9 @@
DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.rview;
DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wview;
-- Read from view
CREATE MATERIALIZED VIEW rview ENGINE = File(CSV) POPULATE AS SELECT 1 AS c0;
SELECT 1 FROM rview;
-- Write through view populate
CREATE MATERIALIZED VIEW wview ENGINE = Join(ALL, INNER, c0) POPULATE AS SELECT 1 AS c0;

View File

@ -0,0 +1,4 @@
2
3
4
1

View File

@ -0,0 +1,42 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t;"
$CLICKHOUSE_CLIENT --query "CREATE TABLE t (key UInt64) ENGINE = MergeTree() ORDER BY tuple() SETTINGS min_parts_to_merge_at_once=5, merge_selector_base=1"
$CLICKHOUSE_CLIENT --query "INSERT INTO t VALUES (1)"
$CLICKHOUSE_CLIENT --query "INSERT INTO t VALUES (2);"
# doesn't make test flaky
sleep 1
$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.parts WHERE active and database = currentDatabase() and table = 't'"
$CLICKHOUSE_CLIENT --query "INSERT INTO t VALUES (3)"
$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.parts WHERE active and database = currentDatabase() and table = 't'"
$CLICKHOUSE_CLIENT --query "INSERT INTO t VALUES (4)"
$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.parts WHERE active and database = currentDatabase() and table = 't'"
$CLICKHOUSE_CLIENT --query "INSERT INTO t VALUES (5)"
counter=0 retries=60
while [[ $counter -lt $retries ]]; do
result=$($CLICKHOUSE_CLIENT --query "SELECT count() FROM system.parts WHERE active and database = currentDatabase() and table = 't'")
if [ "$result" -eq "1" ];then
break;
fi
sleep 0.5
counter=$((counter + 1))
done
$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.parts WHERE active and database = currentDatabase() and table = 't'"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t"