From 3e2f41f3010ca7b68762518738a35dee0f84f8e0 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Wed, 27 Mar 2024 18:35:44 +0300 Subject: [PATCH 01/75] support ATTACH PARTITION `ALL` FROM `TABLE` --- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- src/Storages/StorageMergeTree.cpp | 19 +- src/Storages/StorageReplicatedMergeTree.cpp | 451 +++++++++--------- ...626_replace_partition_from_table.reference | 11 +- .../00626_replace_partition_from_table.sql | 6 +- ...e_partition_from_table_zookeeper.reference | 1 + ..._replace_partition_from_table_zookeeper.sh | 9 + 7 files changed, 273 insertions(+), 228 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6cc8063d90a..7cdb18ce0b9 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4876,7 +4876,7 @@ void MergeTreeData::checkAlterPartitionIsPossible( const auto * partition_ast = command.partition->as(); if (partition_ast && partition_ast->all) { - if (command.type != PartitionCommand::DROP_PARTITION && command.type != PartitionCommand::ATTACH_PARTITION) + if (command.type != PartitionCommand::DROP_PARTITION && command.type != PartitionCommand::ATTACH_PARTITION && (command.type == PartitionCommand::REPLACE_PARTITION && command.replace)) throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH/ATTACH PARTITION ALL currently"); } else @@ -5625,7 +5625,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc const auto & partition_ast = ast->as(); if (partition_ast.all) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only Support DETACH PARTITION ALL currently"); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only Support DROP/DETACH/ATTACH PARTITION ALL currently"); if (!partition_ast.value) { diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 788f250e6a9..821b02b5b5f 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2078,9 +2078,21 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con ProfileEventsScope profile_events_scope; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, my_metadata_snapshot); - String partition_id = getPartitionIDFromQuery(partition, local_context); + DataPartsVector src_parts; + String partition_id; + bool is_all = partition->as()->all; + if (is_all) + { + if (replace) + throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH/ATTACH PARTITION ALL currently"); + + src_parts = src_data.getVisibleDataPartsVector(local_context); + } else + { + partition_id = getPartitionIDFromQuery(partition, local_context); + src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id); + } - DataPartsVector src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id); MutableDataPartsVector dst_parts; std::vector dst_parts_locks; @@ -2088,6 +2100,9 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con for (const DataPartPtr & src_part : src_parts) { + if (is_all) + partition_id = src_part->partition.getID(src_data); + if (!canReplacePartition(src_part)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot replace partition '{}' because part '{}' has inconsistent granularity with table", diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ce6735d9176..7d7ab712163 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7961,232 +7961,247 @@ void StorageReplicatedMergeTree::replacePartitionFrom( ProfileEventsScope profile_events_scope; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); - String partition_id = getPartitionIDFromQuery(partition, query_context); - /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. - DataPartsVector src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); - - LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); - - static const String TMP_PREFIX = "tmp_replace_from_"; + const String TMP_PREFIX = "tmp_replace_from_"; auto zookeeper = getZooKeeper(); - /// Retry if alter_partition_version changes - for (size_t retry = 0; retry < 1000; ++retry) + std::unordered_set partitions; + if (partition->as()->all) { - DataPartsVector src_parts; - MutableDataPartsVector dst_parts; - std::vector dst_parts_locks; - Strings block_id_paths; - Strings part_checksums; - std::vector ephemeral_locks; - String alter_partition_version_path = zookeeper_path + "/alter_partition_version"; - Coordination::Stat alter_partition_version_stat; - zookeeper->get(alter_partition_version_path, &alter_partition_version_stat); - - /// Firstly, generate last block number and compute drop_range - /// NOTE: Even if we make ATTACH PARTITION instead of REPLACE PARTITION drop_range will not be empty, it will contain a block. - /// So, such case has special meaning, if drop_range contains only one block it means that nothing to drop. - /// TODO why not to add normal DROP_RANGE entry to replication queue if `replace` is true? - MergeTreePartInfo drop_range; - std::optional delimiting_block_lock; - bool partition_was_empty = !getFakePartCoveringAllPartsInPartition(partition_id, drop_range, delimiting_block_lock, true); - if (replace && partition_was_empty) - { - /// Nothing to drop, will just attach new parts - LOG_INFO(log, "Partition {} was empty, REPLACE PARTITION will work as ATTACH PARTITION FROM", drop_range.partition_id); - replace = false; - } - - if (!replace) - { - /// It's ATTACH PARTITION FROM, not REPLACE PARTITION. We have to reset drop range - drop_range = makeDummyDropRangeForMovePartitionOrAttachPartitionFrom(partition_id); - } - - assert(replace == !LogEntry::ReplaceRangeEntry::isMovePartitionOrAttachFrom(drop_range)); - - String drop_range_fake_part_name = getPartNamePossiblyFake(format_version, drop_range); - - std::set replaced_parts; - for (const auto & src_part : src_all_parts) - { - /// We also make some kind of deduplication to avoid duplicated parts in case of ATTACH PARTITION - /// Assume that merges in the partition are quite rare - /// Save deduplication block ids with special prefix replace_partition - - if (!canReplacePartition(src_part)) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot replace partition '{}' because part '{}" - "' has inconsistent granularity with table", partition_id, src_part->name); - - String hash_hex = src_part->checksums.getTotalChecksumHex(); - const bool is_duplicated_part = replaced_parts.contains(hash_hex); - replaced_parts.insert(hash_hex); - - if (replace) - LOG_INFO(log, "Trying to replace {} with hash_hex {}", src_part->name, hash_hex); - else - LOG_INFO(log, "Trying to attach {} with hash_hex {}", src_part->name, hash_hex); - - String block_id_path = (replace || is_duplicated_part) ? "" : (fs::path(zookeeper_path) / "blocks" / (partition_id + "_replace_from_" + hash_hex)); - - auto lock = allocateBlockNumber(partition_id, zookeeper, block_id_path); - if (!lock) - { - LOG_INFO(log, "Part {} (hash {}) has been already attached", src_part->name, hash_hex); - continue; - } - - UInt64 index = lock->getNumber(); - MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - - bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication - || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; - - IDataPartStorage::ClonePartParams clone_params - { - .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), - .metadata_version_to_write = metadata_snapshot->getMetadataVersion() - }; - - auto [dst_part, part_lock] = cloneAndLoadDataPart( - src_part, - TMP_PREFIX, - dst_part_info, - metadata_snapshot, - clone_params, - query_context->getReadSettings(), - query_context->getWriteSettings()); - - dst_parts.emplace_back(std::move(dst_part)); - dst_parts_locks.emplace_back(std::move(part_lock)); - src_parts.emplace_back(src_part); - ephemeral_locks.emplace_back(std::move(*lock)); - block_id_paths.emplace_back(block_id_path); - part_checksums.emplace_back(hash_hex); - } - - ReplicatedMergeTreeLogEntryData entry; - { - auto src_table_id = src_data.getStorageID(); - entry.type = ReplicatedMergeTreeLogEntryData::REPLACE_RANGE; - entry.source_replica = replica_name; - entry.create_time = time(nullptr); - entry.replace_range_entry = std::make_shared(); - - auto & entry_replace = *entry.replace_range_entry; - entry_replace.drop_range_part_name = drop_range_fake_part_name; - entry_replace.from_database = src_table_id.database_name; - entry_replace.from_table = src_table_id.table_name; - for (const auto & part : src_parts) - entry_replace.src_part_names.emplace_back(part->name); - for (const auto & part : dst_parts) - entry_replace.new_part_names.emplace_back(part->name); - for (const String & checksum : part_checksums) - entry_replace.part_names_checksums.emplace_back(checksum); - entry_replace.columns_version = -1; - } - if (replace) - { - /// Cancel concurrent inserts in range - clearLockedBlockNumbersInPartition(*zookeeper, drop_range.partition_id, drop_range.min_block, drop_range.max_block); - /// Remove deduplication block_ids of replacing parts - clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.min_block, drop_range.max_block); - } + throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH/ATTACH PARTITION ALL currently"); - Coordination::Responses op_results; - DataPartsVector parts_holder; - - try - { - Coordination::Requests ops; - for (size_t i = 0; i < dst_parts.size(); ++i) - { - getCommitPartOps(ops, dst_parts[i], block_id_paths[i]); - ephemeral_locks[i].getUnlockOp(ops); - } - - if (auto txn = query_context->getZooKeeperMetadataTransaction()) - txn->moveOpsTo(ops); - - delimiting_block_lock->getUnlockOp(ops); - /// Check and update version to avoid race with DROP_RANGE - ops.emplace_back(zkutil::makeSetRequest(alter_partition_version_path, "", alter_partition_version_stat.version)); - /// Just update version, because merges assignment relies on it - ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); - ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); - - Transaction transaction(*this, NO_TRANSACTION_RAW); - { - auto data_parts_lock = lockParts(); - for (auto & part : dst_parts) - renameTempPartAndReplaceUnlocked(part, transaction, data_parts_lock); - } - - for (const auto & dst_part : dst_parts) - lockSharedData(*dst_part, false, /*hardlinked_files*/ {}); - - Coordination::Error code = zookeeper->tryMulti(ops, op_results); - if (code == Coordination::Error::ZOK) - delimiting_block_lock->assumeUnlocked(); - else if (code == Coordination::Error::ZBADVERSION) - { - /// Cannot retry automatically, because some zookeeper ops were lost on the first attempt. Will retry on DDLWorker-level. - if (query_context->getZooKeeperMetadataTransaction()) - throw Exception(ErrorCodes::CANNOT_ASSIGN_ALTER, - "Cannot execute alter, because alter partition version was suddenly changed due " - "to concurrent alter"); - continue; - } - else - zkutil::KeeperMultiException::check(code, ops, op_results); - - { - auto data_parts_lock = lockParts(); - transaction.commit(&data_parts_lock); - if (replace) - { - parts_holder = getDataPartsVectorInPartitionForInternalUsage(MergeTreeDataPartState::Active, drop_range.partition_id, &data_parts_lock); - /// We ignore the list of parts returned from the function below. We will remove them from zk when executing REPLACE_RANGE - removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper(NO_TRANSACTION_RAW, drop_range, data_parts_lock); - } - } - - PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed(), profile_events_scope.getSnapshot())); - } - catch (...) - { - PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed()), ExecutionStatus::fromCurrentException("", true)); - for (const auto & dst_part : dst_parts) - unlockSharedData(*dst_part); - - throw; - } - - String log_znode_path = dynamic_cast(*op_results.back()).path_created; - entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); - - for (auto & lock : ephemeral_locks) - lock.assumeUnlocked(); - - lock2.reset(); - lock1.reset(); - - /// We need to pull the DROP_RANGE before cleaning the replaced parts (otherwise CHeckThread may decide that parts are lost) - queue.pullLogsToQueue(getZooKeeperAndAssertNotReadonly(), {}, ReplicatedMergeTreeQueue::SYNC); - parts_holder.clear(); - cleanup_thread.wakeup(); - - - waitForLogEntryToBeProcessedIfNecessary(entry, query_context); - - return; + partitions = src_data.getAllPartitionIds(); + LOG_INFO(log, "Will try to attach {} partitions", partitions.size()); + } else + { + partitions = std::unordered_set(); + partitions.emplace(getPartitionIDFromQuery(partition, query_context)); } - throw Exception( - ErrorCodes::CANNOT_ASSIGN_ALTER, "Cannot assign ALTER PARTITION, because another ALTER PARTITION query was concurrently executed"); + for (const auto & partition_id : partitions) { + auto src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); + LOG_DEBUG(log, "Cloning {} parts from partition '{}'", src_all_parts.size(), partition_id); + + auto ok = false; + /// Retry if alter_partition_version changes + for (size_t retry = 0; retry < 1000; ++retry) + { + DataPartsVector src_parts; + MutableDataPartsVector dst_parts; + std::vector dst_parts_locks; + Strings block_id_paths; + Strings part_checksums; + std::vector ephemeral_locks; + String alter_partition_version_path = zookeeper_path + "/alter_partition_version"; + Coordination::Stat alter_partition_version_stat; + zookeeper->get(alter_partition_version_path, &alter_partition_version_stat); + + /// Firstly, generate last block number and compute drop_range + /// NOTE: Even if we make ATTACH PARTITION instead of REPLACE PARTITION drop_range will not be empty, it will contain a block. + /// So, such case has special meaning, if drop_range contains only one block it means that nothing to drop. + /// TODO why not to add normal DROP_RANGE entry to replication queue if `replace` is true? + MergeTreePartInfo drop_range; + std::optional delimiting_block_lock; + bool partition_was_empty = !getFakePartCoveringAllPartsInPartition(partition_id, drop_range, delimiting_block_lock, true); + if (replace && partition_was_empty) + { + /// Nothing to drop, will just attach new parts + LOG_INFO(log, "Partition {} was empty, REPLACE PARTITION will work as ATTACH PARTITION FROM", drop_range.partition_id); + replace = false; + } + + if (!replace) + { + /// It's ATTACH PARTITION FROM, not REPLACE PARTITION. We have to reset drop range + drop_range = makeDummyDropRangeForMovePartitionOrAttachPartitionFrom(partition_id); + } + + assert(replace == !LogEntry::ReplaceRangeEntry::isMovePartitionOrAttachFrom(drop_range)); + + String drop_range_fake_part_name = getPartNamePossiblyFake(format_version, drop_range); + + std::set replaced_parts; + for (const auto & src_part : src_all_parts) + { + /// We also make some kind of deduplication to avoid duplicated parts in case of ATTACH PARTITION + /// Assume that merges in the partition are quite rare + /// Save deduplication block ids with special prefix replace_partition + + if (!canReplacePartition(src_part)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot replace partition '{}' because part '{}" + "' has inconsistent granularity with table", partition_id, src_part->name); + + String hash_hex = src_part->checksums.getTotalChecksumHex(); + const bool is_duplicated_part = replaced_parts.contains(hash_hex); + replaced_parts.insert(hash_hex); + + if (replace) + LOG_INFO(log, "Trying to replace '{}' with hash_hex '{}'", src_part->name, hash_hex); + else + LOG_INFO(log, "Trying to attach '{}' with hash_hex '{}'", src_part->name, hash_hex); + + String block_id_path = (replace || is_duplicated_part) ? "" : (fs::path(zookeeper_path) / "blocks" / (partition_id + "_replace_from_" + hash_hex)); + + auto lock = allocateBlockNumber(partition_id, zookeeper, block_id_path); + if (!lock) + { + LOG_INFO(log, "Part '{}' (hash '{}') in partition '{}' has been already attached", src_part->name, hash_hex, partition_id); + continue; + } + + UInt64 index = lock->getNumber(); + MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); + + bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication + || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + + IDataPartStorage::ClonePartParams clone_params + { + .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), + .metadata_version_to_write = metadata_snapshot->getMetadataVersion() + }; + + auto [dst_part, part_lock] = cloneAndLoadDataPart( + src_part, + TMP_PREFIX, + dst_part_info, + metadata_snapshot, + clone_params, + query_context->getReadSettings(), + query_context->getWriteSettings()); + + dst_parts.emplace_back(std::move(dst_part)); + dst_parts_locks.emplace_back(std::move(part_lock)); + src_parts.emplace_back(src_part); + ephemeral_locks.emplace_back(std::move(*lock)); + block_id_paths.emplace_back(block_id_path); + part_checksums.emplace_back(hash_hex); + } + + ReplicatedMergeTreeLogEntryData entry; + { + auto src_table_id = src_data.getStorageID(); + entry.type = ReplicatedMergeTreeLogEntryData::REPLACE_RANGE; + entry.source_replica = replica_name; + entry.create_time = time(nullptr); + entry.replace_range_entry = std::make_shared(); + + auto & entry_replace = *entry.replace_range_entry; + entry_replace.drop_range_part_name = drop_range_fake_part_name; + entry_replace.from_database = src_table_id.database_name; + entry_replace.from_table = src_table_id.table_name; + for (const auto & part : src_parts) + entry_replace.src_part_names.emplace_back(part->name); + for (const auto & part : dst_parts) + entry_replace.new_part_names.emplace_back(part->name); + for (const String & checksum : part_checksums) + entry_replace.part_names_checksums.emplace_back(checksum); + entry_replace.columns_version = -1; + } + + if (replace) + { + /// Cancel concurrent inserts in range + clearLockedBlockNumbersInPartition(*zookeeper, drop_range.partition_id, drop_range.min_block, drop_range.max_block); + /// Remove deduplication block_ids of replacing parts + clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.min_block, drop_range.max_block); + } + + Coordination::Responses op_results; + DataPartsVector parts_holder; + + try + { + Coordination::Requests ops; + for (size_t i = 0; i < dst_parts.size(); ++i) + { + getCommitPartOps(ops, dst_parts[i], block_id_paths[i]); + ephemeral_locks[i].getUnlockOp(ops); + } + + if (auto txn = query_context->getZooKeeperMetadataTransaction()) + txn->moveOpsTo(ops); + + delimiting_block_lock->getUnlockOp(ops); + /// Check and update version to avoid race with DROP_RANGE + ops.emplace_back(zkutil::makeSetRequest(alter_partition_version_path, "", alter_partition_version_stat.version)); + /// Just update version, because merges assignment relies on it + ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Transaction transaction(*this, NO_TRANSACTION_RAW); + { + auto data_parts_lock = lockParts(); + for (auto & part : dst_parts) + renameTempPartAndReplaceUnlocked(part, transaction, data_parts_lock); + } + + for (const auto & dst_part : dst_parts) + lockSharedData(*dst_part, false, /*hardlinked_files*/ {}); + + Coordination::Error code = zookeeper->tryMulti(ops, op_results); + if (code == Coordination::Error::ZOK) + delimiting_block_lock->assumeUnlocked(); + else if (code == Coordination::Error::ZBADVERSION) + { + /// Cannot retry automatically, because some zookeeper ops were lost on the first attempt. Will retry on DDLWorker-level. + if (query_context->getZooKeeperMetadataTransaction()) + throw Exception(ErrorCodes::CANNOT_ASSIGN_ALTER, + "Cannot execute alter on partition '{}', because alter partition version was suddenly changed due " + "to concurrent alter", partition_id); + continue; + } + else + zkutil::KeeperMultiException::check(code, ops, op_results); + + { + auto data_parts_lock = lockParts(); + transaction.commit(&data_parts_lock); + if (replace) + { + parts_holder = getDataPartsVectorInPartitionForInternalUsage(MergeTreeDataPartState::Active, drop_range.partition_id, &data_parts_lock); + /// We ignore the list of parts returned from the function below. We will remove them from zk when executing REPLACE_RANGE + removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper(NO_TRANSACTION_RAW, drop_range, data_parts_lock); + } + } + + PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed(), profile_events_scope.getSnapshot())); + } + catch (...) + { + PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed()), ExecutionStatus::fromCurrentException("", true)); + for (const auto & dst_part : dst_parts) + unlockSharedData(*dst_part); + + throw; + } + + String log_znode_path = dynamic_cast(*op_results.back()).path_created; + entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + for (auto & lock : ephemeral_locks) + lock.assumeUnlocked(); + + /// We need to pull the DROP_RANGE before cleaning the replaced parts (otherwise CHeckThread may decide that parts are lost) + queue.pullLogsToQueue(getZooKeeperAndAssertNotReadonly(), {}, ReplicatedMergeTreeQueue::SYNC); + parts_holder.clear(); + cleanup_thread.wakeup(); + + waitForLogEntryToBeProcessedIfNecessary(entry, query_context); + + ok = true; + break; + } + + if (!ok) + throw Exception( + ErrorCodes::CANNOT_ASSIGN_ALTER, "Cannot assign ALTER PARTITION '{}', because another ALTER PARTITION query was concurrently executed", partition_id); + } + + lock2.reset(); + lock1.reset(); } void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table.reference b/tests/queries/0_stateless/00626_replace_partition_from_table.reference index 611f3a93ced..0f8ded245d0 100644 --- a/tests/queries/0_stateless/00626_replace_partition_from_table.reference +++ b/tests/queries/0_stateless/00626_replace_partition_from_table.reference @@ -10,11 +10,12 @@ REPLACE recursive 4 8 1 ATTACH FROM -5 8 +6 8 +10 12 OPTIMIZE -5 8 5 -5 8 3 +10 12 9 +10 12 5 After restart -5 8 +10 12 DETACH+ATTACH PARTITION -3 4 +7 7 diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table.sql b/tests/queries/0_stateless/00626_replace_partition_from_table.sql index 7224224334e..3f712f48c06 100644 --- a/tests/queries/0_stateless/00626_replace_partition_from_table.sql +++ b/tests/queries/0_stateless/00626_replace_partition_from_table.sql @@ -53,12 +53,16 @@ DROP TABLE src; CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k; INSERT INTO src VALUES (1, '0', 1); INSERT INTO src VALUES (1, '1', 1); +INSERT INTO src VALUES (2, '2', 1); +INSERT INTO src VALUES (3, '3', 1); SYSTEM STOP MERGES dst; -INSERT INTO dst VALUES (1, '1', 2); +INSERT INTO dst VALUES (1, '1', 2), (1, '2', 0); ALTER TABLE dst ATTACH PARTITION 1 FROM src; SELECT count(), sum(d) FROM dst; +ALTER TABLE dst ATTACH PARTITION ALL FROM src; +SELECT count(), sum(d) FROM dst; SELECT 'OPTIMIZE'; SELECT count(), sum(d), uniqExact(_part) FROM dst; diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.reference b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.reference index c6208941ac6..6a7c3478f86 100644 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.reference +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.reference @@ -16,6 +16,7 @@ REPLACE recursive ATTACH FROM 5 8 5 8 +7 12 REPLACE with fetch 4 6 4 6 diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index ffbf4df4ba7..49976df83b7 100755 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -82,6 +82,8 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE src;" $CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" $CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (3, '1', 2);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (4, '1', 2);" $CLICKHOUSE_CLIENT --query="INSERT INTO dst_r2 VALUES (1, '1', 2);" query_with_retry "ALTER TABLE dst_r2 ATTACH PARTITION 1 FROM src;" @@ -90,6 +92,13 @@ $CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r1;" $CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" $CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" +query_with_retry "ALTER TABLE dst_r2 ATTACH PARTITION ALL FROM src;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" +query_with_retry "ALTER TABLE dst_r2 DROP PARTITION 3;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +query_with_retry "ALTER TABLE dst_r2 DROP PARTITION 4;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'REPLACE with fetch';" $CLICKHOUSE_CLIENT --query="DROP TABLE src;" From 0fbf612e150d83d9d0dfb7b157ea4cef6127c86a Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Wed, 27 Mar 2024 18:41:24 +0300 Subject: [PATCH 02/75] support ATTACH PARTITION `ALL` FROM `TABLE` --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7d7ab712163..5434a8bd63e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7962,7 +7962,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); - const String TMP_PREFIX = "tmp_replace_from_"; + static const String TMP_PREFIX = "tmp_replace_from_"; auto zookeeper = getZooKeeper(); std::unordered_set partitions; From 9c5e094289df6f0c0063819b68ae96a23f032d5e Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Wed, 27 Mar 2024 19:57:41 +0300 Subject: [PATCH 03/75] fix cs --- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 5434a8bd63e..b4ffd5eff0a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7979,7 +7979,8 @@ void StorageReplicatedMergeTree::replacePartitionFrom( partitions.emplace(getPartitionIDFromQuery(partition, query_context)); } - for (const auto & partition_id : partitions) { + for (const auto & partition_id : partitions) + { auto src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); LOG_DEBUG(log, "Cloning {} parts from partition '{}'", src_all_parts.size(), partition_id); From 1d4aa10099fec19f2b60f9094a9f9c004ab06421 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Sun, 31 Mar 2024 00:09:40 +0300 Subject: [PATCH 04/75] fix cs --- src/Storages/StorageMergeTree.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 821b02b5b5f..04115cf0ede 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2087,7 +2087,8 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH/ATTACH PARTITION ALL currently"); src_parts = src_data.getVisibleDataPartsVector(local_context); - } else + } + else { partition_id = getPartitionIDFromQuery(partition, local_context); src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id); From 115d7cfa572556f321373e441e5becf3221426ea Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Fri, 26 Apr 2024 15:22:34 +0300 Subject: [PATCH 05/75] fix after merge --- src/Storages/StorageReplicatedMergeTree.cpp | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 84bb2e78eb6..b0d2faf1651 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7982,13 +7982,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom( ProfileEventsScope profile_events_scope; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); - String partition_id = getPartitionIDFromQuery(partition, query_context); - - /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. - DataPartsVector src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); - - LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); - static const String TMP_PREFIX = "tmp_replace_from_"; auto zookeeper = getZooKeeper(); @@ -7999,7 +7992,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH/ATTACH PARTITION ALL currently"); partitions = src_data.getAllPartitionIds(); - LOG_INFO(log, "Will try to attach {} partitions", partitions.size()); + LOG_INFO(log, "Will try to attach {} partitions without replace", partitions.size()); } else { partitions = std::unordered_set(); @@ -8008,6 +8001,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( for (const auto & partition_id : partitions) { + /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. auto src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); LOG_DEBUG(log, "Cloning {} parts from partition '{}'", src_all_parts.size(), partition_id); From 4712b79960d9da21741aa504cd508a5902e569fe Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 24 Jul 2024 16:41:37 +0800 Subject: [PATCH 06/75] Add min_max as statistics type --- src/Storages/Statistics/Statistics.cpp | 8 +- src/Storages/Statistics/StatisticsMinMax.cpp | 91 +++++++++++++++++++ src/Storages/Statistics/StatisticsMinMax.h | 31 +++++++ src/Storages/StatisticsDescription.cpp | 8 +- src/Storages/StatisticsDescription.h | 1 + ...2864_statistics_count_min_sketch.reference | 14 --- .../02864_statistics_estimation.reference | 25 +++++ ...ch.sql => 02864_statistics_estimation.sql} | 41 ++++++++- 8 files changed, 198 insertions(+), 21 deletions(-) create mode 100644 src/Storages/Statistics/StatisticsMinMax.cpp create mode 100644 src/Storages/Statistics/StatisticsMinMax.h delete mode 100644 tests/queries/0_stateless/02864_statistics_count_min_sketch.reference create mode 100644 tests/queries/0_stateless/02864_statistics_estimation.reference rename tests/queries/0_stateless/{02864_statistics_count_min_sketch.sql => 02864_statistics_estimation.sql} (57%) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index ade3326288a..5227f3d235e 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -103,6 +104,8 @@ Float64 ColumnStatistics::estimateLess(const Field & val) const { if (stats.contains(StatisticsType::TDigest)) return stats.at(StatisticsType::TDigest)->estimateLess(val); + if (stats.contains(StatisticsType::MinMax)) + return stats.at(StatisticsType::MinMax)->estimateLess(val); return rows * ConditionSelectivityEstimator::default_normal_cond_factor; } @@ -204,6 +207,9 @@ void MergeTreeStatisticsFactory::registerValidator(StatisticsType stats_type, Va MergeTreeStatisticsFactory::MergeTreeStatisticsFactory() { + registerValidator(StatisticsType::MinMax, minMaxValidator); + registerCreator(StatisticsType::MinMax, minMaxCreator); + registerValidator(StatisticsType::TDigest, tdigestValidator); registerCreator(StatisticsType::TDigest, tdigestCreator); @@ -240,7 +246,7 @@ ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnStatisticsDescri { auto it = creators.find(type); if (it == creators.end()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'tdigest' 'uniq' and 'count_min'", type); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'min_max', 'tdigest' 'uniq' and 'count_min'", type); auto stat_ptr = (it->second)(desc, stats.data_type); column_stat->stats[type] = stat_ptr; } diff --git a/src/Storages/Statistics/StatisticsMinMax.cpp b/src/Storages/Statistics/StatisticsMinMax.cpp new file mode 100644 index 00000000000..8138ea9fa87 --- /dev/null +++ b/src/Storages/Statistics/StatisticsMinMax.cpp @@ -0,0 +1,91 @@ +#include +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int ILLEGAL_STATISTICS; +} + +StatisticsMinMax::StatisticsMinMax(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type_) + : IStatistics(stat_) + , min(std::numeric_limits::max()) + , max(std::numeric_limits::min()) + , row_count(0) + , data_type(data_type_) +{ +} + +Float64 StatisticsMinMax::estimateLess(const Field & val) const +{ + Field val_converted = convertFieldToType(val, *data_type); + if (val_converted.isNull()) + return 0; + + auto val_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); + + if (val_float < min) + return 0; + + if (val_float > max) + return row_count; + + if (max == min) + return row_count; + + return ((val_float - min) / (max - min)) * row_count; +} + +void StatisticsMinMax::update(const ColumnPtr & column) +{ + for (size_t row = 0; row < column->size(); ++row) + { + if (column->isNullAt(row)) + continue; + + auto data = column->getFloat64(row); + min = std::min(data, min); + max = std::max(data, max); + } + row_count += column->size(); +} + +void StatisticsMinMax::serialize(WriteBuffer & buf) +{ + writeIntBinary(row_count, buf); + writeFloatBinary(min, buf); + writeFloatBinary(max, buf); +} + +void StatisticsMinMax::deserialize(ReadBuffer & buf) +{ + readIntBinary(row_count, buf); + readFloatBinary(min, buf); + readFloatBinary(max, buf); +} + + +void minMaxValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +{ + data_type = removeNullable(data_type); + data_type = removeLowCardinalityAndNullable(data_type); + if (!data_type->isValueRepresentedByNumber()) + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'min_max' do not support type {}", data_type->getName()); +} + +StatisticsPtr minMaxCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +{ + return std::make_shared(stat, data_type); +} + +} diff --git a/src/Storages/Statistics/StatisticsMinMax.h b/src/Storages/Statistics/StatisticsMinMax.h new file mode 100644 index 00000000000..913dcccb798 --- /dev/null +++ b/src/Storages/Statistics/StatisticsMinMax.h @@ -0,0 +1,31 @@ +#include + + +namespace DB +{ + +class StatisticsMinMax : public IStatistics +{ +public: + StatisticsMinMax(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type_); + ~StatisticsMinMax() override = default; + + Float64 estimateLess(const Field & val) const override; + + void update(const ColumnPtr & column) override; + + void serialize(WriteBuffer & buf) override; + void deserialize(ReadBuffer & buf) override; + +private: + Float64 min; + Float64 max; + Float64 row_count; + + DataTypePtr data_type; +}; + +void minMaxValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr minMaxCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); + +} diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 9c5fd3604b2..0d99c36259f 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -51,7 +51,9 @@ static StatisticsType stringToStatisticsType(String type) return StatisticsType::Uniq; if (type == "count_min") return StatisticsType::CountMinSketch; - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are 'tdigest', 'uniq' and 'count_min'.", type); + if (type == "min_max") + return StatisticsType::MinMax; + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are 'min_max', 'tdigest', 'uniq' and 'count_min'.", type); } String SingleStatisticsDescription::getTypeName() const @@ -64,8 +66,10 @@ String SingleStatisticsDescription::getTypeName() const return "Uniq"; case StatisticsType::CountMinSketch: return "count_min"; + case StatisticsType::MinMax: + return "min_max"; default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are 'tdigest', 'uniq' and 'count_min'.", type); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are 'min_max', 'tdigest', 'uniq' and 'count_min'.", type); } } diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index 03b8fb0d583..1aa51ed03dd 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -14,6 +14,7 @@ enum class StatisticsType : UInt8 TDigest = 0, Uniq = 1, CountMinSketch = 2, + MinMax = 3, Max = 63, }; diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference b/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference deleted file mode 100644 index 02c41656a36..00000000000 --- a/tests/queries/0_stateless/02864_statistics_count_min_sketch.reference +++ /dev/null @@ -1,14 +0,0 @@ -CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -Test statistics count_min: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) -Test statistics multi-types: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) -Test LowCardinality and Nullable data type: -tab2 diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference new file mode 100644 index 00000000000..a1db6b8dc96 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -0,0 +1,25 @@ +CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `d` DateTime64(3),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +Test statistics min_max: + Prewhere info + Prewhere filter + Prewhere filter column: and(greater(d, _CAST(9998_DateTime(\'UTC\'), \'DateTime(\'UTC\')\')), less(c, -1), greater(b, 0)) (removed) +Test statistics count_min: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) +Test statistics multi-types: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) +Test statistics min_max and tdigest: + Prewhere info + Prewhere filter + Prewhere filter column: and(less(b, 10), less(c, 0)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(less(b, 10), less(c, 0)) (removed) +Test LowCardinality and Nullable data type: +tab2 diff --git a/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql similarity index 57% rename from tests/queries/0_stateless/02864_statistics_count_min_sketch.sql rename to tests/queries/0_stateless/02864_statistics_estimation.sql index c730aa7b4a7..f3d085b837b 100644 --- a/tests/queries/0_stateless/02864_statistics_count_min_sketch.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -12,13 +12,28 @@ CREATE TABLE tab a String, b UInt64, c Int64, + d DateTime64, pk String, ) Engine = MergeTree() ORDER BY pk SETTINGS min_bytes_for_wide_part = 0; SHOW CREATE TABLE tab; -INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), generateUUIDv4() FROM system.numbers LIMIT 10000; +INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), toDateTime(number, 'UTC'), generateUUIDv4() FROM system.numbers LIMIT 10000; + +SELECT 'Test statistics min_max:'; + +ALTER TABLE tab ADD STATISTICS b TYPE min_max; +ALTER TABLE tab ADD STATISTICS c TYPE min_max; +ALTER TABLE tab ADD STATISTICS d TYPE min_max; +ALTER TABLE tab MATERIALIZE STATISTICS b, c, d; + +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*10000*/ and c < -1/*9990*/ and d > toDateTime(9998, 'UTC')/*1*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE tab DROP STATISTICS b, c, d; + SELECT 'Test statistics count_min:'; @@ -28,7 +43,7 @@ ALTER TABLE tab ADD STATISTICS c TYPE count_min; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) xx +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS a, b, c; @@ -39,7 +54,8 @@ SELECT 'Test statistics multi-types:'; ALTER TABLE tab ADD STATISTICS a TYPE count_min; ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; +ALTER TABLE tab ADD STATISTICS d TYPE count_min, uniq, tdigest; +ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/) @@ -49,7 +65,24 @@ SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -ALTER TABLE tab DROP STATISTICS a, b, c; +ALTER TABLE tab DROP STATISTICS a, b, c, d; + + +SELECT 'Test statistics min_max and tdigest:'; + +ALTER TABLE tab ADD STATISTICS b TYPE min_max; +ALTER TABLE tab MATERIALIZE STATISTICS b; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*5000*/ and b < 10/*100*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE tab ADD STATISTICS b TYPE tdigest; +ALTER TABLE tab MATERIALIZE STATISTICS b; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*5000*/ and b < 10/*100*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +ALTER TABLE tab DROP STATISTICS b; + DROP TABLE IF EXISTS tab SYNC; From 40fc0ca5743a6b1dcc464268a107ba206ca375ab Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 24 Jul 2024 17:43:27 +0800 Subject: [PATCH 07/75] Uniq statistics supports more datatypes and refactor logical when there is no statistics. --- .../ConditionSelectivityEstimator.cpp | 41 ++++++------------ .../ConditionSelectivityEstimator.h | 10 ++--- src/Storages/Statistics/Statistics.cpp | 26 ++++++------ src/Storages/Statistics/Statistics.h | 11 +++-- src/Storages/Statistics/StatisticsMinMax.cpp | 8 ++-- src/Storages/Statistics/StatisticsTDigest.cpp | 42 ++++++++++--------- src/Storages/Statistics/StatisticsTDigest.h | 9 ++-- src/Storages/Statistics/StatisticsUniq.cpp | 2 +- src/Storages/Statistics/StatisticsUniq.h | 4 +- .../02864_statistics_estimation.reference | 9 +++- .../02864_statistics_estimation.sql | 21 ++++++++-- 11 files changed, 94 insertions(+), 89 deletions(-) diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp index 57dff958b9a..86970a3ac46 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp @@ -19,7 +19,7 @@ void ConditionSelectivityEstimator::ColumnSelectivityEstimator::merge(String par Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateLess(const Field & val, Float64 rows) const { if (part_statistics.empty()) - return default_normal_cond_factor * rows; + return default_cond_range_factor * rows; Float64 result = 0; Float64 part_rows = 0; for (const auto & [key, estimator] : part_statistics) @@ -38,15 +38,8 @@ Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateGreat Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateEqual(const Field & val, Float64 rows) const { if (part_statistics.empty()) - { - auto float_val = StatisticsUtils::tryConvertToFloat64(val); - if (!float_val) - return default_unknown_cond_factor * rows; - else if (float_val.value() < - threshold || float_val.value() > threshold) - return default_normal_cond_factor * rows; - else - return default_good_cond_factor * rows; - } + return default_cond_equal_factor * rows; + Float64 result = 0; Float64 partial_cnt = 0; for (const auto & [key, estimator] : part_statistics) @@ -149,30 +142,22 @@ Float64 ConditionSelectivityEstimator::estimateRowCount(const RPNBuilderTreeNode auto [op, val] = extractBinaryOp(node, col); + if (dummy) + { + if (op == "equals") + return default_cond_equal_factor * total_rows; + else if (op == "less" || op == "lessOrEquals" || op == "greater" || op == "greaterOrEquals") + return default_cond_range_factor * total_rows; + else + return default_unknown_cond_factor * total_rows; + } + if (op == "equals") - { - if (dummy) - { - auto float_val = StatisticsUtils::tryConvertToFloat64(val); - if (!float_val || (float_val < - threshold || float_val > threshold)) - return default_normal_cond_factor * total_rows; - else - return default_good_cond_factor * total_rows; - } return estimator.estimateEqual(val, total_rows); - } else if (op == "less" || op == "lessOrEquals") - { - if (dummy) - return default_normal_cond_factor * total_rows; return estimator.estimateLess(val, total_rows); - } else if (op == "greater" || op == "greaterOrEquals") - { - if (dummy) - return default_normal_cond_factor * total_rows; return estimator.estimateGreater(val, total_rows); - } else return default_unknown_cond_factor * total_rows; } diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.h b/src/Storages/Statistics/ConditionSelectivityEstimator.h index ce7fdd12e92..269ee9ac6cb 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.h +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.h @@ -38,12 +38,10 @@ private: std::pair extractBinaryOp(const RPNBuilderTreeNode & node, const String & column_name) const; - static constexpr auto default_good_cond_factor = 0.1; - static constexpr auto default_normal_cond_factor = 0.5; - static constexpr auto default_unknown_cond_factor = 1.0; - /// Conditions like "x = N" are considered good if abs(N) > threshold. - /// This is used to assume that condition is likely to have good selectivity. - static constexpr auto threshold = 2; + /// Used to estimate the selectivity of a condition when there is no statistics. + static constexpr auto default_cond_range_factor = 0.5; + static constexpr auto default_cond_equal_factor = 0.01; + static constexpr auto default_unknown_cond_factor = 1; UInt64 total_rows = 0; std::map column_estimators; diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 5227f3d235e..3513638ea9e 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -51,13 +51,6 @@ std::optional StatisticsUtils::tryConvertToFloat64(const Field & field) } } -std::optional StatisticsUtils::tryConvertToString(const DB::Field & field) -{ - if (field.getType() == Field::Types::String) - return field.get(); - return {}; -} - IStatistics::IStatistics(const SingleStatisticsDescription & stat_) : stat(stat_) { @@ -106,7 +99,7 @@ Float64 ColumnStatistics::estimateLess(const Field & val) const return stats.at(StatisticsType::TDigest)->estimateLess(val); if (stats.contains(StatisticsType::MinMax)) return stats.at(StatisticsType::MinMax)->estimateLess(val); - return rows * ConditionSelectivityEstimator::default_normal_cond_factor; + return rows * ConditionSelectivityEstimator::default_cond_range_factor; } Float64 ColumnStatistics::estimateGreater(const Field & val) const @@ -116,8 +109,7 @@ Float64 ColumnStatistics::estimateGreater(const Field & val) const Float64 ColumnStatistics::estimateEqual(const Field & val) const { - auto float_val = StatisticsUtils::tryConvertToFloat64(val); - if (float_val.has_value() && stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest)) + if (stats_desc.data_type->isValueRepresentedByNumber() && stats.contains(StatisticsType::Uniq) && stats.contains(StatisticsType::TDigest)) { /// 2048 is the default number of buckets in TDigest. In this case, TDigest stores exactly one value (with many rows) for every bucket. if (stats.at(StatisticsType::Uniq)->estimateCardinality() < 2048) @@ -127,10 +119,16 @@ Float64 ColumnStatistics::estimateEqual(const Field & val) const if (stats.contains(StatisticsType::CountMinSketch)) return stats.at(StatisticsType::CountMinSketch)->estimateEqual(val); #endif - if (!float_val.has_value() && (float_val < - ConditionSelectivityEstimator::threshold || float_val > ConditionSelectivityEstimator::threshold)) - return rows * ConditionSelectivityEstimator::default_normal_cond_factor; - else - return rows * ConditionSelectivityEstimator::default_good_cond_factor; + if (stats.contains(StatisticsType::Uniq)) + { + auto cardinality = stats.at(StatisticsType::Uniq)->estimateCardinality(); + if (cardinality == 0) + return 0; + /// Assume that the value is uniformly distributed among the unique values. + return static_cast(1) / stats.at(StatisticsType::Uniq)->estimateCardinality(); + } + + return rows * ConditionSelectivityEstimator::default_cond_equal_factor; } /// ------------------------------------- diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 16f0c67eabd..f47a707663c 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -19,7 +19,6 @@ struct StatisticsUtils { /// Returns std::nullopt if input Field cannot be converted to a concrete value static std::optional tryConvertToFloat64(const Field & field); - static std::optional tryConvertToString(const Field & field); }; /// Statistics describe properties of the values in the column, @@ -32,11 +31,6 @@ public: explicit IStatistics(const SingleStatisticsDescription & stat_); virtual ~IStatistics() = default; - virtual void update(const ColumnPtr & column) = 0; - - virtual void serialize(WriteBuffer & buf) = 0; - virtual void deserialize(ReadBuffer & buf) = 0; - /// Estimate the cardinality of the column. /// Throws if the statistics object is not able to do a meaningful estimation. virtual UInt64 estimateCardinality() const; @@ -46,6 +40,11 @@ public: virtual Float64 estimateEqual(const Field & val) const; /// cardinality of val in the column virtual Float64 estimateLess(const Field & val) const; /// summarized cardinality of values < val in the column + virtual void update(const ColumnPtr & column) = 0; + + virtual void serialize(WriteBuffer & buf) = 0; + virtual void deserialize(ReadBuffer & buf) = 0; + protected: SingleStatisticsDescription stat; }; diff --git a/src/Storages/Statistics/StatisticsMinMax.cpp b/src/Storages/Statistics/StatisticsMinMax.cpp index 8138ea9fa87..083e8634841 100644 --- a/src/Storages/Statistics/StatisticsMinMax.cpp +++ b/src/Storages/Statistics/StatisticsMinMax.cpp @@ -32,18 +32,18 @@ Float64 StatisticsMinMax::estimateLess(const Field & val) const if (val_converted.isNull()) return 0; - auto val_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); + auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); - if (val_float < min) + if (val_as_float < min) return 0; - if (val_float > max) + if (val_as_float > max) return row_count; if (max == min) return row_count; - return ((val_float - min) / (max - min)) * row_count; + return ((val_as_float - min) / (max - min)) * row_count; } void StatisticsMinMax::update(const ColumnPtr & column) diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index 66150e00fdb..3544f5cdea3 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include namespace DB { @@ -10,24 +12,20 @@ extern const int ILLEGAL_STATISTICS; extern const int LOGICAL_ERROR; } -StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & stat_) - : IStatistics(stat_) +StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) + : IStatistics(stat_), data_type(data_type_) { } void StatisticsTDigest::update(const ColumnPtr & column) { - size_t rows = column->size(); - for (size_t row = 0; row < rows; ++row) + for (size_t row = 0; row < column->size(); ++row) { - Field field; - column->get(row, field); - - if (field.isNull()) + if (column->isNullAt(row)) continue; - if (auto field_as_float = StatisticsUtils::tryConvertToFloat64(field)) - t_digest.add(*field_as_float, 1); + auto data = column->getFloat64(row); + t_digest.add(data, 1); } } @@ -43,18 +41,22 @@ void StatisticsTDigest::deserialize(ReadBuffer & buf) Float64 StatisticsTDigest::estimateLess(const Field & val) const { - auto val_as_float = StatisticsUtils::tryConvertToFloat64(val); - if (val_as_float) - return t_digest.getCountLessThan(*val_as_float); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName()); + Field val_converted = convertFieldToType(val, *data_type); + if (val_converted.isNull()) + return 0; + + auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); + return t_digest.getCountLessThan(val_as_float); } Float64 StatisticsTDigest::estimateEqual(const Field & val) const { - auto val_as_float = StatisticsUtils::tryConvertToFloat64(val); - if (val_as_float) - return t_digest.getCountEqual(*val_as_float); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName()); + Field val_converted = convertFieldToType(val, *data_type); + if (val_converted.isNull()) + return 0; + + auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); + return t_digest.getCountEqual(val_as_float); } void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type) @@ -65,9 +67,9 @@ void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName()); } -StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr) +StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { - return std::make_shared(stat); + return std::make_shared(stat, data_type); } } diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index 614973e5d8b..f4de6cc41e0 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -9,18 +9,19 @@ namespace DB class StatisticsTDigest : public IStatistics { public: - explicit StatisticsTDigest(const SingleStatisticsDescription & stat_); + explicit StatisticsTDigest(const SingleStatisticsDescription & stat_, DataTypePtr data_type_); + + Float64 estimateLess(const Field & val) const override; + Float64 estimateEqual(const Field & val) const override; void update(const ColumnPtr & column) override; void serialize(WriteBuffer & buf) override; void deserialize(ReadBuffer & buf) override; - Float64 estimateLess(const Field & val) const override; - Float64 estimateEqual(const Field & val) const override; - private: QuantileTDigest t_digest; + DataTypePtr data_type; }; void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type); diff --git a/src/Storages/Statistics/StatisticsUniq.cpp b/src/Storages/Statistics/StatisticsUniq.cpp index 8f60ffcf0b5..aace45a14b1 100644 --- a/src/Storages/Statistics/StatisticsUniq.cpp +++ b/src/Storages/Statistics/StatisticsUniq.cpp @@ -56,7 +56,7 @@ void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); - if (!data_type->isValueRepresentedByNumber()) + if (!data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName()); } diff --git a/src/Storages/Statistics/StatisticsUniq.h b/src/Storages/Statistics/StatisticsUniq.h index faabde8d47c..81162b0a5b9 100644 --- a/src/Storages/Statistics/StatisticsUniq.h +++ b/src/Storages/Statistics/StatisticsUniq.h @@ -13,13 +13,13 @@ public: StatisticsUniq(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type); ~StatisticsUniq() override; + UInt64 estimateCardinality() const override; + void update(const ColumnPtr & column) override; void serialize(WriteBuffer & buf) override; void deserialize(ReadBuffer & buf) override; - UInt64 estimateCardinality() const override; - private: std::unique_ptr arena; AggregateFunctionPtr collector; diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index a1db6b8dc96..64f264017d8 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -14,12 +14,19 @@ Test statistics multi-types: Prewhere info Prewhere filter Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) -Test statistics min_max and tdigest: +Test estimating range condition: Prewhere info Prewhere filter Prewhere filter column: and(less(b, 10), less(c, 0)) (removed) Prewhere info Prewhere filter Prewhere filter column: and(less(b, 10), less(c, 0)) (removed) +Test estimating equals condition: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), equals(b, 10)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), equals(b, 10)) (removed) Test LowCardinality and Nullable data type: tab2 diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index f3d085b837b..0ea9eb38e60 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -68,7 +68,7 @@ WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS a, b, c, d; -SELECT 'Test statistics min_max and tdigest:'; +SELECT 'Test estimating range condition:'; ALTER TABLE tab ADD STATISTICS b TYPE min_max; ALTER TABLE tab MATERIALIZE STATISTICS b; @@ -84,6 +84,21 @@ WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS b; +SELECT 'Test estimating equals condition:'; + +ALTER TABLE tab ADD STATISTICS a TYPE uniq; +ALTER TABLE tab MATERIALIZE STATISTICS a; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b = 10/*100*/ and a = '0'/*1*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +ALTER TABLE tab ADD STATISTICS a TYPE count_min; +ALTER TABLE tab MATERIALIZE STATISTICS a; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b = 10/*100*/ and a = '0'/*1*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +ALTER TABLE tab DROP STATISTICS a; + DROP TABLE IF EXISTS tab SYNC; @@ -93,8 +108,8 @@ SET allow_suspicious_low_cardinality_types=1; CREATE TABLE tab2 ( a LowCardinality(Int64) STATISTICS(count_min), - b Nullable(Int64) STATISTICS(count_min), - c LowCardinality(Nullable(Int64)) STATISTICS(count_min), + b Nullable(Int64) STATISTICS(min_max, count_min), + c LowCardinality(Nullable(Int64)) STATISTICS(min_max, count_min), pk String, ) Engine = MergeTree() ORDER BY pk; From 5ae356e6df457e24220f0f63f4a4984334471202 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 24 Jul 2024 17:54:48 +0800 Subject: [PATCH 08/75] Add document for min_max statistics --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 7ffbd9a5bae..8e8d5e55772 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -991,6 +991,9 @@ They can be used for prewhere optimization only if we enable `set allow_statisti #### Available Types of Column Statistics {#available-types-of-column-statistics} +- `min_max` + + min_max statistics allows to estimate selectivity of range condition for numeric columns. - `TDigest` [TDigest](https://github.com/tdunning/t-digest) sketches which allow to compute approximate percentiles (e.g. the 90th percentile) for numeric columns. From 3f7cd09d168ea13e4ec281c8f3943c0aebe9f920 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 24 Jul 2024 18:02:26 +0800 Subject: [PATCH 09/75] Remove useless function tryConvertToFloat64 --- src/Storages/Statistics/Statistics.cpp | 22 ---------------------- src/Storages/Statistics/Statistics.h | 6 ------ 2 files changed, 28 deletions(-) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 3513638ea9e..0f0f1de4552 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -28,28 +28,6 @@ enum StatisticsFileVersion : UInt16 V0 = 0, }; -std::optional StatisticsUtils::tryConvertToFloat64(const Field & field) -{ - switch (field.getType()) - { - case Field::Types::Int64: - return field.get(); - case Field::Types::UInt64: - return field.get(); - case Field::Types::Float64: - return field.get(); - case Field::Types::Int128: - return field.get(); - case Field::Types::UInt128: - return field.get(); - case Field::Types::Int256: - return field.get(); - case Field::Types::UInt256: - return field.get(); - default: - return {}; - } -} IStatistics::IStatistics(const SingleStatisticsDescription & stat_) : stat(stat_) diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index f47a707663c..3eeb5cf92a8 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -15,12 +15,6 @@ constexpr auto STATS_FILE_PREFIX = "statistics_"; constexpr auto STATS_FILE_SUFFIX = ".stats"; -struct StatisticsUtils -{ - /// Returns std::nullopt if input Field cannot be converted to a concrete value - static std::optional tryConvertToFloat64(const Field & field); -}; - /// Statistics describe properties of the values in the column, /// e.g. how many unique values exist, /// what are the N most frequent values, From b9703ad9242f7492de6f6d10e13ac89a74231210 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 25 Jul 2024 15:14:12 +0800 Subject: [PATCH 10/75] Fix logical error when estimating --- src/Storages/Statistics/Statistics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 0f0f1de4552..17ff8deb720 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -103,7 +103,7 @@ Float64 ColumnStatistics::estimateEqual(const Field & val) const if (cardinality == 0) return 0; /// Assume that the value is uniformly distributed among the unique values. - return static_cast(1) / stats.at(StatisticsType::Uniq)->estimateCardinality(); + return static_cast(1) / stats.at(StatisticsType::Uniq)->estimateCardinality() * rows; } return rows * ConditionSelectivityEstimator::default_cond_equal_factor; From 3adbc4cd334a05560ca2397ac5e1a14d8c580167 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 31 Jul 2024 15:10:29 +0800 Subject: [PATCH 11/75] Fix code style --- src/Storages/Statistics/StatisticsMinMax.h | 2 ++ src/Storages/Statistics/StatisticsTDigest.cpp | 5 ++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/Statistics/StatisticsMinMax.h b/src/Storages/Statistics/StatisticsMinMax.h index 913dcccb798..3e08e6b5768 100644 --- a/src/Storages/Statistics/StatisticsMinMax.h +++ b/src/Storages/Statistics/StatisticsMinMax.h @@ -1,3 +1,5 @@ +#pragma once + #include diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index 3544f5cdea3..25cee4ac8e8 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -9,7 +9,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_STATISTICS; -extern const int LOGICAL_ERROR; } StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) @@ -44,7 +43,7 @@ Float64 StatisticsTDigest::estimateLess(const Field & val) const Field val_converted = convertFieldToType(val, *data_type); if (val_converted.isNull()) return 0; - + auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); return t_digest.getCountLessThan(val_as_float); } @@ -54,7 +53,7 @@ Float64 StatisticsTDigest::estimateEqual(const Field & val) const Field val_converted = convertFieldToType(val, *data_type); if (val_converted.isNull()) return 0; - + auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); return t_digest.getCountEqual(val_as_float); } From d1305d9fadf3ea2ad2393ee5f13a676110bf6ef7 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 5 Aug 2024 11:26:57 +0800 Subject: [PATCH 12/75] Some fixups and split tests --- .../mergetree-family/mergetree.md | 5 ++- src/Storages/Statistics/Statistics.cpp | 24 ++++++------ .../Statistics/StatisticsCountMinSketch.cpp | 4 +- .../Statistics/StatisticsCountMinSketch.h | 4 +- src/Storages/Statistics/StatisticsMinMax.cpp | 17 ++++----- src/Storages/Statistics/StatisticsMinMax.h | 11 +++--- src/Storages/Statistics/StatisticsTDigest.cpp | 4 +- src/Storages/Statistics/StatisticsTDigest.h | 4 +- src/Storages/Statistics/StatisticsUniq.cpp | 4 +- src/Storages/Statistics/StatisticsUniq.h | 4 +- src/Storages/StatisticsDescription.cpp | 8 ++-- ...atistics_create_materialize_drop.reference | 6 +++ ...864_statistics_create_materialize_drop.sql | 37 +++++++++++++++++++ .../02864_statistics_estimation.reference | 6 +-- .../02864_statistics_estimation.sql | 28 +++----------- 15 files changed, 94 insertions(+), 72 deletions(-) create mode 100644 tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference create mode 100644 tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 8e8d5e55772..42968097799 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -991,9 +991,10 @@ They can be used for prewhere optimization only if we enable `set allow_statisti #### Available Types of Column Statistics {#available-types-of-column-statistics} -- `min_max` +- `MinMax` + + MinMax statistics allows to estimate selectivity of range condition for numeric columns. - min_max statistics allows to estimate selectivity of range condition for numeric columns. - `TDigest` [TDigest](https://github.com/tdunning/t-digest) sketches which allow to compute approximate percentiles (e.g. the 90th percentile) for numeric columns. diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 17ff8deb720..42b742419f0 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -4,9 +4,9 @@ #include #include #include +#include #include #include -#include #include #include #include @@ -99,11 +99,11 @@ Float64 ColumnStatistics::estimateEqual(const Field & val) const #endif if (stats.contains(StatisticsType::Uniq)) { - auto cardinality = stats.at(StatisticsType::Uniq)->estimateCardinality(); + UInt64 cardinality = stats.at(StatisticsType::Uniq)->estimateCardinality(); if (cardinality == 0) return 0; /// Assume that the value is uniformly distributed among the unique values. - return static_cast(1) / stats.at(StatisticsType::Uniq)->estimateCardinality() * rows; + return 1.0 / cardinality * rows; } return rows * ConditionSelectivityEstimator::default_cond_equal_factor; @@ -183,18 +183,18 @@ void MergeTreeStatisticsFactory::registerValidator(StatisticsType stats_type, Va MergeTreeStatisticsFactory::MergeTreeStatisticsFactory() { - registerValidator(StatisticsType::MinMax, minMaxValidator); - registerCreator(StatisticsType::MinMax, minMaxCreator); + registerValidator(StatisticsType::MinMax, minMaxStatisticsValidator); + registerCreator(StatisticsType::MinMax, minMaxStatisticsCreator); - registerValidator(StatisticsType::TDigest, tdigestValidator); - registerCreator(StatisticsType::TDigest, tdigestCreator); + registerValidator(StatisticsType::TDigest, tdigestStatisticsValidator); + registerCreator(StatisticsType::TDigest, tdigestStatisticsCreator); - registerValidator(StatisticsType::Uniq, uniqValidator); - registerCreator(StatisticsType::Uniq, uniqCreator); + registerValidator(StatisticsType::Uniq, uniqStatisticsValidator); + registerCreator(StatisticsType::Uniq, uniqStatisticsCreator); #if USE_DATASKETCHES - registerValidator(StatisticsType::CountMinSketch, countMinSketchValidator); - registerCreator(StatisticsType::CountMinSketch, countMinSketchCreator); + registerValidator(StatisticsType::CountMinSketch, countMinSketchStatisticsValidator); + registerCreator(StatisticsType::CountMinSketch, countMinSketchStatisticsCreator); #endif } @@ -222,7 +222,7 @@ ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnStatisticsDescri { auto it = creators.find(type); if (it == creators.end()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'min_max', 'tdigest' 'uniq' and 'count_min'", type); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'count_min', 'minmax', 'tdigest' and 'uniq'", type); auto stat_ptr = (it->second)(desc, stats.data_type); column_stat->stats[type] = stat_ptr; } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index e69bbc1515b..50d3b6e515c 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -84,7 +84,7 @@ void StatisticsCountMinSketch::deserialize(ReadBuffer & buf) } -void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -92,7 +92,7 @@ void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr da throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName()); } -StatisticsPtr countMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { return std::make_shared(stat, data_type); } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.h b/src/Storages/Statistics/StatisticsCountMinSketch.h index 6c8b74f8c35..d10bc78a88e 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.h +++ b/src/Storages/Statistics/StatisticsCountMinSketch.h @@ -31,8 +31,8 @@ private: }; -void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr countMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr); } diff --git a/src/Storages/Statistics/StatisticsMinMax.cpp b/src/Storages/Statistics/StatisticsMinMax.cpp index 083e8634841..f689f23d75d 100644 --- a/src/Storages/Statistics/StatisticsMinMax.cpp +++ b/src/Storages/Statistics/StatisticsMinMax.cpp @@ -19,9 +19,6 @@ extern const int ILLEGAL_STATISTICS; StatisticsMinMax::StatisticsMinMax(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type_) : IStatistics(stat_) - , min(std::numeric_limits::max()) - , max(std::numeric_limits::min()) - , row_count(0) , data_type(data_type_) { } @@ -41,7 +38,7 @@ Float64 StatisticsMinMax::estimateLess(const Field & val) const return row_count; if (max == min) - return row_count; + return (val_as_float < max) ? 0 : row_count; return ((val_as_float - min) / (max - min)) * row_count; } @@ -53,9 +50,9 @@ void StatisticsMinMax::update(const ColumnPtr & column) if (column->isNullAt(row)) continue; - auto data = column->getFloat64(row); - min = std::min(data, min); - max = std::max(data, max); + auto value = column->getFloat64(row); + min = std::min(value, min); + max = std::max(value, max); } row_count += column->size(); } @@ -75,15 +72,15 @@ void StatisticsMinMax::deserialize(ReadBuffer & buf) } -void minMaxValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void minMaxStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); if (!data_type->isValueRepresentedByNumber()) - throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'min_max' do not support type {}", data_type->getName()); + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'minmax' do not support type {}", data_type->getName()); } -StatisticsPtr minMaxCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { return std::make_shared(stat, data_type); } diff --git a/src/Storages/Statistics/StatisticsMinMax.h b/src/Storages/Statistics/StatisticsMinMax.h index 3e08e6b5768..4a67504350b 100644 --- a/src/Storages/Statistics/StatisticsMinMax.h +++ b/src/Storages/Statistics/StatisticsMinMax.h @@ -10,7 +10,6 @@ class StatisticsMinMax : public IStatistics { public: StatisticsMinMax(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type_); - ~StatisticsMinMax() override = default; Float64 estimateLess(const Field & val) const override; @@ -20,14 +19,14 @@ public: void deserialize(ReadBuffer & buf) override; private: - Float64 min; - Float64 max; - Float64 row_count; + Float64 min = std::numeric_limits::max(); + Float64 max = std::numeric_limits::min(); + UInt64 row_count = 0; DataTypePtr data_type; }; -void minMaxValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr minMaxCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); +void minMaxStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); } diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index 25cee4ac8e8..d418a761e7c 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -58,7 +58,7 @@ Float64 StatisticsTDigest::estimateEqual(const Field & val) const return t_digest.getCountEqual(val_as_float); } -void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -66,7 +66,7 @@ void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName()); } -StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { return std::make_shared(stat, data_type); } diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index f4de6cc41e0..b83132c5e0d 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -24,7 +24,7 @@ private: DataTypePtr data_type; }; -void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr); } diff --git a/src/Storages/Statistics/StatisticsUniq.cpp b/src/Storages/Statistics/StatisticsUniq.cpp index aace45a14b1..628ba98aba3 100644 --- a/src/Storages/Statistics/StatisticsUniq.cpp +++ b/src/Storages/Statistics/StatisticsUniq.cpp @@ -52,7 +52,7 @@ UInt64 StatisticsUniq::estimateCardinality() const return column->getUInt(0); } -void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -60,7 +60,7 @@ void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName()); } -StatisticsPtr uniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) { return std::make_shared(stat, data_type); } diff --git a/src/Storages/Statistics/StatisticsUniq.h b/src/Storages/Statistics/StatisticsUniq.h index 81162b0a5b9..219e2ffbc55 100644 --- a/src/Storages/Statistics/StatisticsUniq.h +++ b/src/Storages/Statistics/StatisticsUniq.h @@ -27,7 +27,7 @@ private: }; -void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr uniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); +void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); } diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index c54752dd42f..f3ce9deb836 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -51,9 +51,9 @@ static StatisticsType stringToStatisticsType(String type) return StatisticsType::Uniq; if (type == "count_min") return StatisticsType::CountMinSketch; - if (type == "min_max") + if (type == "minmax") return StatisticsType::MinMax; - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are 'min_max', 'tdigest', 'uniq' and 'count_min'.", type); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are 'count_min', 'minmax', 'tdigest' and 'uniq'.", type); } String SingleStatisticsDescription::getTypeName() const @@ -67,9 +67,9 @@ String SingleStatisticsDescription::getTypeName() const case StatisticsType::CountMinSketch: return "count_min"; case StatisticsType::MinMax: - return "min_max"; + return "minmax"; default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are 'min_max', 'tdigest', 'uniq' and 'count_min'.", type); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are 'count_min', 'minmax', 'tdigest' and 'uniq'.", type); } } diff --git a/tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference new file mode 100644 index 00000000000..5aff1ac63d4 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference @@ -0,0 +1,6 @@ +Test create statistics: +CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64) STATISTICS(uniq, count_min, minmax),\n `b` Nullable(Int64),\n `c` LowCardinality(Nullable(Int64)) STATISTICS(count_min, minmax),\n `d` DateTime STATISTICS(tdigest, uniq, count_min, minmax),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 +tab +Test materialize and drop statistics: +CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64) STATISTICS(uniq, count_min, minmax),\n `b` Nullable(Int64) STATISTICS(tdigest, uniq, count_min, minmax),\n `c` LowCardinality(Nullable(Int64)) STATISTICS(count_min, minmax),\n `d` DateTime STATISTICS(tdigest, uniq, count_min, minmax),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64),\n `b` Nullable(Int64),\n `c` LowCardinality(Nullable(Int64)),\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql new file mode 100644 index 00000000000..442079ab947 --- /dev/null +++ b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql @@ -0,0 +1,37 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS tab SYNC; + +SET allow_experimental_statistics = 1; +SET allow_statistics_optimize = 1; +SET allow_suspicious_low_cardinality_types=1; +SET mutations_sync = 2; + + +SELECT 'Test create statistics:'; + +CREATE TABLE tab +( + a LowCardinality(Int64) STATISTICS(count_min, minmax, uniq), + b Nullable(Int64), + c LowCardinality(Nullable(Int64)) STATISTICS(minmax, count_min), + d DateTime STATISTICS(count_min, minmax, tdigest, uniq), + pk String, +) Engine = MergeTree() ORDER BY pk; + +SHOW CREATE TABLE tab; + +SELECT name FROM system.tables WHERE name = 'tab' AND database = currentDatabase(); +INSERT INTO tab select number, number, number, toDateTime(number, 'UTC'), generateUUIDv4() FROM system.numbers LIMIT 10000; + + +SELECT 'Test materialize and drop statistics:'; + +ALTER TABLE tab ADD STATISTICS b TYPE count_min, minmax, tdigest, uniq; +ALTER TABLE tab MATERIALIZE STATISTICS b; +SHOW CREATE TABLE tab; + +ALTER TABLE tab DROP STATISTICS a, b, c, d; +SHOW CREATE TABLE tab; + +DROP TABLE IF EXISTS tab SYNC; diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index 64f264017d8..a5225c6338f 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -1,5 +1,5 @@ -CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `d` DateTime64(3),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -Test statistics min_max: +CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +Test statistics minmax: Prewhere info Prewhere filter Prewhere filter column: and(greater(d, _CAST(9998_DateTime(\'UTC\'), \'DateTime(\'UTC\')\')), less(c, -1), greater(b, 0)) (removed) @@ -28,5 +28,3 @@ Test estimating equals condition: Prewhere info Prewhere filter Prewhere filter column: and(equals(a, \'0\'), equals(b, 10)) (removed) -Test LowCardinality and Nullable data type: -tab2 diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 0ea9eb38e60..117e24bd732 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -12,7 +12,7 @@ CREATE TABLE tab a String, b UInt64, c Int64, - d DateTime64, + d DateTime, pk String, ) Engine = MergeTree() ORDER BY pk SETTINGS min_bytes_for_wide_part = 0; @@ -21,11 +21,11 @@ SHOW CREATE TABLE tab; INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), toDateTime(number, 'UTC'), generateUUIDv4() FROM system.numbers LIMIT 10000; -SELECT 'Test statistics min_max:'; +SELECT 'Test statistics minmax:'; -ALTER TABLE tab ADD STATISTICS b TYPE min_max; -ALTER TABLE tab ADD STATISTICS c TYPE min_max; -ALTER TABLE tab ADD STATISTICS d TYPE min_max; +ALTER TABLE tab ADD STATISTICS b TYPE minmax; +ALTER TABLE tab ADD STATISTICS c TYPE minmax; +ALTER TABLE tab ADD STATISTICS d TYPE minmax; ALTER TABLE tab MATERIALIZE STATISTICS b, c, d; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') @@ -70,7 +70,7 @@ ALTER TABLE tab DROP STATISTICS a, b, c, d; SELECT 'Test estimating range condition:'; -ALTER TABLE tab ADD STATISTICS b TYPE min_max; +ALTER TABLE tab ADD STATISTICS b TYPE minmax; ALTER TABLE tab MATERIALIZE STATISTICS b; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*5000*/ and b < 10/*100*/) @@ -100,19 +100,3 @@ WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS a; DROP TABLE IF EXISTS tab SYNC; - - -SELECT 'Test LowCardinality and Nullable data type:'; -DROP TABLE IF EXISTS tab2 SYNC; -SET allow_suspicious_low_cardinality_types=1; -CREATE TABLE tab2 -( - a LowCardinality(Int64) STATISTICS(count_min), - b Nullable(Int64) STATISTICS(min_max, count_min), - c LowCardinality(Nullable(Int64)) STATISTICS(min_max, count_min), - pk String, -) Engine = MergeTree() ORDER BY pk; - -select name from system.tables where name = 'tab2' and database = currentDatabase(); - -DROP TABLE IF EXISTS tab2 SYNC; From 7765ff6d5227da58eecc426287c98c333c75402d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Aug 2024 06:46:04 +0000 Subject: [PATCH 13/75] Minor fixups --- .../mergetree-family/mergetree.md | 2 +- src/Storages/Statistics/Statistics.cpp | 22 ++++---- src/Storages/Statistics/Statistics.h | 12 ++-- .../Statistics/StatisticsCountMinSketch.cpp | 52 +++++++++--------- .../Statistics/StatisticsCountMinSketch.h | 11 ++-- src/Storages/Statistics/StatisticsMinMax.cpp | 55 +++++++++---------- src/Storages/Statistics/StatisticsMinMax.h | 11 ++-- src/Storages/Statistics/StatisticsTDigest.cpp | 11 ++-- src/Storages/Statistics/StatisticsTDigest.h | 13 +++-- src/Storages/Statistics/StatisticsUniq.cpp | 10 ++-- src/Storages/Statistics/StatisticsUniq.h | 12 ++-- 11 files changed, 107 insertions(+), 104 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 42968097799..c2438ac593e 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -993,7 +993,7 @@ They can be used for prewhere optimization only if we enable `set allow_statisti - `MinMax` - MinMax statistics allows to estimate selectivity of range condition for numeric columns. + The minimum and maximum column value which allows to estimate the selectivity of range filters on numeric columns. - `TDigest` diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 42b742419f0..1f159211a93 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -62,14 +62,17 @@ Float64 IStatistics::estimateLess(const Field & /*val*/) const } /// ------------------------------------- -/// Implementation of the estimation: -/// Note: Each statistics object supports certain types predicates natively, e.g. -/// - TDigest: '< X' (less-than predicates) -/// - Count-min sketches: '= X' (equal predicates) -/// - Uniq (HyperLogLog): 'count distinct(*)' (column cardinality) -/// If multiple statistics objects are available per column, it is sometimes also possible to combine them in a clever way. -/// For that reason, all estimation are performed in a central place (here), and we don't simply pass the predicate to the first statistics -/// object that supports it natively. +/// Notes: +/// - Statistics object usually only support estimation for certain types of predicates, e.g. +/// - TDigest: '< X' (less-than predicates) +/// - Count-min sketches: '= X' (equal predicates) +/// - Uniq (HyperLogLog): 'count distinct(*)' (column cardinality) +/// +/// If multiple statistics objects in a column support estimating a predicate, we want to try statistics in order of descending accuracy +/// (e.g. MinMax statistics are simpler than TDigest statistics and thus worse for estimating 'less' predicates). +/// +/// Sometimes, it is possible to combine multiple statistics in a clever way. For that reason, all estimation are performed in a central +/// place (here), and we don't simply pass the predicate to the first statistics object that supports it natively. Float64 ColumnStatistics::estimateLess(const Field & val) const { @@ -102,8 +105,7 @@ Float64 ColumnStatistics::estimateEqual(const Field & val) const UInt64 cardinality = stats.at(StatisticsType::Uniq)->estimateCardinality(); if (cardinality == 0) return 0; - /// Assume that the value is uniformly distributed among the unique values. - return 1.0 / cardinality * rows; + return 1.0 / cardinality * rows; /// assume uniform distribution } return rows * ConditionSelectivityEstimator::default_cond_equal_factor; diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 3eeb5cf92a8..1425be89d47 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -25,20 +25,20 @@ public: explicit IStatistics(const SingleStatisticsDescription & stat_); virtual ~IStatistics() = default; + virtual void update(const ColumnPtr & column) = 0; + + virtual void serialize(WriteBuffer & buf) = 0; + virtual void deserialize(ReadBuffer & buf) = 0; + /// Estimate the cardinality of the column. /// Throws if the statistics object is not able to do a meaningful estimation. virtual UInt64 estimateCardinality() const; /// Per-value estimations. - /// Throws if the statistics object is not able to do a meaningful estimation. + /// Throws a LOGICAL_ERROR if the statistics object is not able to do a meaningful estimation. virtual Float64 estimateEqual(const Field & val) const; /// cardinality of val in the column virtual Float64 estimateLess(const Field & val) const; /// summarized cardinality of values < val in the column - virtual void update(const ColumnPtr & column) = 0; - - virtual void serialize(WriteBuffer & buf) = 0; - virtual void deserialize(ReadBuffer & buf) = 0; - protected: SingleStatisticsDescription stat; }; diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.cpp b/src/Storages/Statistics/StatisticsCountMinSketch.cpp index 50d3b6e515c..c0ea7a84ded 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.cpp +++ b/src/Storages/Statistics/StatisticsCountMinSketch.cpp @@ -1,4 +1,3 @@ - #include #include #include @@ -25,34 +24,13 @@ extern const int ILLEGAL_STATISTICS; static constexpr auto num_hashes = 7uz; static constexpr auto num_buckets = 2718uz; -StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) - : IStatistics(stat_) +StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type_) + : IStatistics(statistics_description) , sketch(num_hashes, num_buckets) , data_type(data_type_) { } -Float64 StatisticsCountMinSketch::estimateEqual(const Field & val) const -{ - /// Try to convert field to data_type. Converting string to proper data types such as: number, date, datetime, IPv4, Decimal etc. - /// Return null if val larger than the range of data_type - /// - /// For example: if data_type is Int32: - /// 1. For 1.0, 1, '1', return Field(1) - /// 2. For 1.1, max_value_int64, return null - Field val_converted = convertFieldToType(val, *data_type); - if (val_converted.isNull()) - return 0; - - if (data_type->isValueRepresentedByNumber()) - return sketch.get_estimate(&val_converted, data_type->getSizeOfValueInMemory()); - - if (isStringOrFixedString(data_type)) - return sketch.get_estimate(val.get()); - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'count_min' does not support estimate data type of {}", data_type->getName()); -} - void StatisticsCountMinSketch::update(const ColumnPtr & column) { for (size_t row = 0; row < column->size(); ++row) @@ -83,8 +61,28 @@ void StatisticsCountMinSketch::deserialize(ReadBuffer & buf) sketch = Sketch::deserialize(bytes.data(), size); } +Float64 StatisticsCountMinSketch::estimateEqual(const Field & val) const +{ + /// Try to convert field to data_type. Converting string to proper data types such as: number, date, datetime, IPv4, Decimal etc. + /// Return null if val larger than the range of data_type + /// + /// For example: if data_type is Int32: + /// 1. For 1.0, 1, '1', return Field(1) + /// 2. For 1.1, max_value_int64, return null + Field val_converted = convertFieldToType(val, *data_type); + if (val_converted.isNull()) + return 0; -void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) + if (data_type->isValueRepresentedByNumber()) + return sketch.get_estimate(&val_converted, data_type->getSizeOfValueInMemory()); + + if (isStringOrFixedString(data_type)) + return sketch.get_estimate(val.get()); + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'count_min' does not support estimate data type of {}", data_type->getName()); +} + +void countMinSketchStatisticsValidator(const SingleStatisticsDescription & /*statistics_description*/, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -92,9 +90,9 @@ void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, Data throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName()); } -StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type) { - return std::make_shared(stat, data_type); + return std::make_shared(statistics_description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsCountMinSketch.h b/src/Storages/Statistics/StatisticsCountMinSketch.h index d10bc78a88e..452483c8773 100644 --- a/src/Storages/Statistics/StatisticsCountMinSketch.h +++ b/src/Storages/Statistics/StatisticsCountMinSketch.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include "config.h" @@ -14,15 +15,15 @@ namespace DB class StatisticsCountMinSketch : public IStatistics { public: - StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_); - - Float64 estimateEqual(const Field & val) const override; + StatisticsCountMinSketch(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type_); void update(const ColumnPtr & column) override; void serialize(WriteBuffer & buf) override; void deserialize(ReadBuffer & buf) override; + Float64 estimateEqual(const Field & val) const override; + private: using Sketch = datasketches::count_min_sketch; Sketch sketch; @@ -31,8 +32,8 @@ private: }; -void countMinSketchStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void countMinSketchStatisticsValidator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type); +StatisticsPtr countMinSketchStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr); } diff --git a/src/Storages/Statistics/StatisticsMinMax.cpp b/src/Storages/Statistics/StatisticsMinMax.cpp index f689f23d75d..11a0704dae7 100644 --- a/src/Storages/Statistics/StatisticsMinMax.cpp +++ b/src/Storages/Statistics/StatisticsMinMax.cpp @@ -1,10 +1,10 @@ #include -#include -#include -#include #include +#include +#include #include #include +#include #include @@ -17,32 +17,12 @@ namespace ErrorCodes extern const int ILLEGAL_STATISTICS; } -StatisticsMinMax::StatisticsMinMax(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type_) - : IStatistics(stat_) +StatisticsMinMax::StatisticsMinMax(const SingleStatisticsDescription & statistics_description, const DataTypePtr & data_type_) + : IStatistics(statistics_description) , data_type(data_type_) { } -Float64 StatisticsMinMax::estimateLess(const Field & val) const -{ - Field val_converted = convertFieldToType(val, *data_type); - if (val_converted.isNull()) - return 0; - - auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); - - if (val_as_float < min) - return 0; - - if (val_as_float > max) - return row_count; - - if (max == min) - return (val_as_float < max) ? 0 : row_count; - - return ((val_as_float - min) / (max - min)) * row_count; -} - void StatisticsMinMax::update(const ColumnPtr & column) { for (size_t row = 0; row < column->size(); ++row) @@ -71,8 +51,27 @@ void StatisticsMinMax::deserialize(ReadBuffer & buf) readFloatBinary(max, buf); } +Float64 StatisticsMinMax::estimateLess(const Field & val) const +{ + Field val_converted = convertFieldToType(val, *data_type); + if (val_converted.isNull()) + return 0; -void minMaxStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) + auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); + + if (val_as_float < min) + return 0; + + if (val_as_float > max) + return row_count; + + if (min == max) + return (val_as_float != max) ? 0 : row_count; + + return ((val_as_float - min) / (max - min)) * row_count; +} + +void minMaxStatisticsValidator(const SingleStatisticsDescription & /*statistics_description*/, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -80,9 +79,9 @@ void minMaxStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'minmax' do not support type {}", data_type->getName()); } -StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type) { - return std::make_shared(stat, data_type); + return std::make_shared(statistics_description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsMinMax.h b/src/Storages/Statistics/StatisticsMinMax.h index 4a67504350b..524494b520b 100644 --- a/src/Storages/Statistics/StatisticsMinMax.h +++ b/src/Storages/Statistics/StatisticsMinMax.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -9,15 +10,15 @@ namespace DB class StatisticsMinMax : public IStatistics { public: - StatisticsMinMax(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type_); - - Float64 estimateLess(const Field & val) const override; + StatisticsMinMax(const SingleStatisticsDescription & statistics_description, const DataTypePtr & data_type_); void update(const ColumnPtr & column) override; void serialize(WriteBuffer & buf) override; void deserialize(ReadBuffer & buf) override; + Float64 estimateLess(const Field & val) const override; + private: Float64 min = std::numeric_limits::max(); Float64 max = std::numeric_limits::min(); @@ -26,7 +27,7 @@ private: DataTypePtr data_type; }; -void minMaxStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); +void minMaxStatisticsValidator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type); +StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type); } diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index d418a761e7c..cfd626c8ea8 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -11,8 +11,9 @@ namespace ErrorCodes extern const int ILLEGAL_STATISTICS; } -StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & stat_, DataTypePtr data_type_) - : IStatistics(stat_), data_type(data_type_) +StatisticsTDigest::StatisticsTDigest(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type_) + : IStatistics(statistics_description) + , data_type(data_type_) { } @@ -58,7 +59,7 @@ Float64 StatisticsTDigest::estimateEqual(const Field & val) const return t_digest.getCountEqual(val_as_float); } -void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void tdigestStatisticsValidator(const SingleStatisticsDescription & /*statistics_description*/, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -66,9 +67,9 @@ void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName()); } -StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type) { - return std::make_shared(stat, data_type); + return std::make_shared(statistics_description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index b83132c5e0d..ad2a03836aa 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -9,22 +10,22 @@ namespace DB class StatisticsTDigest : public IStatistics { public: - explicit StatisticsTDigest(const SingleStatisticsDescription & stat_, DataTypePtr data_type_); - - Float64 estimateLess(const Field & val) const override; - Float64 estimateEqual(const Field & val) const override; + explicit StatisticsTDigest(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type_); void update(const ColumnPtr & column) override; void serialize(WriteBuffer & buf) override; void deserialize(ReadBuffer & buf) override; + Float64 estimateLess(const Field & val) const override; + Float64 estimateEqual(const Field & val) const override; + private: QuantileTDigest t_digest; DataTypePtr data_type; }; -void tdigestStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void tdigestStatisticsValidator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type); +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr); } diff --git a/src/Storages/Statistics/StatisticsUniq.cpp b/src/Storages/Statistics/StatisticsUniq.cpp index 628ba98aba3..e895157635f 100644 --- a/src/Storages/Statistics/StatisticsUniq.cpp +++ b/src/Storages/Statistics/StatisticsUniq.cpp @@ -11,8 +11,8 @@ namespace ErrorCodes extern const int ILLEGAL_STATISTICS; } -StatisticsUniq::StatisticsUniq(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type) - : IStatistics(stat_) +StatisticsUniq::StatisticsUniq(const SingleStatisticsDescription & statistics_description, const DataTypePtr & data_type) + : IStatistics(statistics_description) { arena = std::make_unique(); AggregateFunctionProperties properties; @@ -52,7 +52,7 @@ UInt64 StatisticsUniq::estimateCardinality() const return column->getUInt(0); } -void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void uniqStatisticsValidator(const SingleStatisticsDescription & /*statistics_description*/, DataTypePtr data_type) { data_type = removeNullable(data_type); data_type = removeLowCardinalityAndNullable(data_type); @@ -60,9 +60,9 @@ void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr da throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName()); } -StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type) { - return std::make_shared(stat, data_type); + return std::make_shared(statistics_description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsUniq.h b/src/Storages/Statistics/StatisticsUniq.h index 219e2ffbc55..b3000e4895a 100644 --- a/src/Storages/Statistics/StatisticsUniq.h +++ b/src/Storages/Statistics/StatisticsUniq.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -10,24 +11,23 @@ namespace DB class StatisticsUniq : public IStatistics { public: - StatisticsUniq(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type); + StatisticsUniq(const SingleStatisticsDescription & statistics_description, const DataTypePtr & data_type); ~StatisticsUniq() override; - UInt64 estimateCardinality() const override; - void update(const ColumnPtr & column) override; void serialize(WriteBuffer & buf) override; void deserialize(ReadBuffer & buf) override; + UInt64 estimateCardinality() const override; + private: std::unique_ptr arena; AggregateFunctionPtr collector; AggregateDataPtr data; - }; -void uniqStatisticsValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); +void uniqStatisticsValidator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type); +StatisticsPtr uniqStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type); } From c61225ede47e37bd773bd249804739f915feefd7 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 5 Aug 2024 18:37:36 +0800 Subject: [PATCH 14/75] Rewrite tests --- ...atistics_create_materialize_drop.reference | 7 +- ...864_statistics_create_materialize_drop.sql | 16 ++-- .../02864_statistics_estimation.reference | 34 +++---- .../02864_statistics_estimation.sql | 92 ++++++++----------- 4 files changed, 62 insertions(+), 87 deletions(-) diff --git a/tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference index 5aff1ac63d4..4ca2c5e5f9b 100644 --- a/tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference +++ b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.reference @@ -1,6 +1,5 @@ Test create statistics: -CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64) STATISTICS(uniq, count_min, minmax),\n `b` Nullable(Int64),\n `c` LowCardinality(Nullable(Int64)) STATISTICS(count_min, minmax),\n `d` DateTime STATISTICS(tdigest, uniq, count_min, minmax),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 -tab +CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64) STATISTICS(tdigest, uniq, count_min, minmax),\n `b` LowCardinality(Nullable(String)) STATISTICS(uniq, count_min),\n `c` LowCardinality(Nullable(Int64)) STATISTICS(tdigest, uniq, count_min, minmax),\n `d` DateTime STATISTICS(tdigest, uniq, count_min, minmax),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 Test materialize and drop statistics: -CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64) STATISTICS(uniq, count_min, minmax),\n `b` Nullable(Int64) STATISTICS(tdigest, uniq, count_min, minmax),\n `c` LowCardinality(Nullable(Int64)) STATISTICS(count_min, minmax),\n `d` DateTime STATISTICS(tdigest, uniq, count_min, minmax),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 -CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64),\n `b` Nullable(Int64),\n `c` LowCardinality(Nullable(Int64)),\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64),\n `b` LowCardinality(Nullable(String)) STATISTICS(uniq, count_min),\n `c` LowCardinality(Nullable(Int64)),\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `a` LowCardinality(Int64),\n `b` LowCardinality(Nullable(String)),\n `c` LowCardinality(Nullable(Int64)),\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql index 442079ab947..90a57c99624 100644 --- a/tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql +++ b/tests/queries/0_stateless/02864_statistics_create_materialize_drop.sql @@ -12,26 +12,24 @@ SELECT 'Test create statistics:'; CREATE TABLE tab ( - a LowCardinality(Int64) STATISTICS(count_min, minmax, uniq), - b Nullable(Int64), - c LowCardinality(Nullable(Int64)) STATISTICS(minmax, count_min), + a LowCardinality(Int64) STATISTICS(count_min, minmax, tdigest, uniq), + b LowCardinality(Nullable(String)) STATISTICS(count_min, uniq), + c LowCardinality(Nullable(Int64)) STATISTICS(count_min, minmax, tdigest, uniq), d DateTime STATISTICS(count_min, minmax, tdigest, uniq), pk String, ) Engine = MergeTree() ORDER BY pk; +INSERT INTO tab select number, number, number, toDateTime(number), generateUUIDv4() FROM system.numbers LIMIT 10000; SHOW CREATE TABLE tab; -SELECT name FROM system.tables WHERE name = 'tab' AND database = currentDatabase(); -INSERT INTO tab select number, number, number, toDateTime(number, 'UTC'), generateUUIDv4() FROM system.numbers LIMIT 10000; - SELECT 'Test materialize and drop statistics:'; - -ALTER TABLE tab ADD STATISTICS b TYPE count_min, minmax, tdigest, uniq; +ALTER TABLE tab DROP STATISTICS a, b, c, d; +ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq; ALTER TABLE tab MATERIALIZE STATISTICS b; SHOW CREATE TABLE tab; -ALTER TABLE tab DROP STATISTICS a, b, c, d; +ALTER TABLE tab DROP STATISTICS b; SHOW CREATE TABLE tab; DROP TABLE IF EXISTS tab SYNC; diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index a5225c6338f..f59f94013da 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -1,30 +1,24 @@ CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -Test statistics minmax: - Prewhere info - Prewhere filter - Prewhere filter column: and(greater(d, _CAST(9998_DateTime(\'UTC\'), \'DateTime(\'UTC\')\')), less(c, -1), greater(b, 0)) (removed) Test statistics count_min: Prewhere info Prewhere filter Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) +Test statistics minmax: + Prewhere info + Prewhere filter + Prewhere filter column: and(greater(d, _CAST(9998, \'DateTime\')), less(c, -1), greater(b, 0)) (removed) +Test statistics tdigest: + Prewhere info + Prewhere filter + Prewhere filter column: and(greater(d, _CAST(9998, \'DateTime\')), less(c, -1), greater(b, 0)) (removed) +Test statistics uniq: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'0\'), equals(d, _CAST(1, \'DateTime\')), equals(b, 0), equals(c, 0)) (removed) Test statistics multi-types: Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), less(c, -90), greater(b, 900)) (removed) + Prewhere filter column: and(equals(a, \'0\'), equals(d, _CAST(1, \'DateTime\')), less(c, -90), greater(b, 900)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0)) (removed) -Test estimating range condition: - Prewhere info - Prewhere filter - Prewhere filter column: and(less(b, 10), less(c, 0)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(less(b, 10), less(c, 0)) (removed) -Test estimating equals condition: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 10)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 10)) (removed) + Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, _CAST(1, \'DateTime\'))) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 117e24bd732..65b111c285b 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -19,84 +19,68 @@ SETTINGS min_bytes_for_wide_part = 0; SHOW CREATE TABLE tab; -INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), toDateTime(number, 'UTC'), generateUUIDv4() FROM system.numbers LIMIT 10000; +INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), cast(number, 'DateTime'), generateUUIDv4() FROM system.numbers LIMIT 10000; + + +SELECT 'Test statistics count_min:'; + +ALTER TABLE tab ADD STATISTICS a, b, c TYPE count_min; +ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +ALTER TABLE tab DROP STATISTICS a, b, c; + SELECT 'Test statistics minmax:'; -ALTER TABLE tab ADD STATISTICS b TYPE minmax; -ALTER TABLE tab ADD STATISTICS c TYPE minmax; -ALTER TABLE tab ADD STATISTICS d TYPE minmax; +ALTER TABLE tab ADD STATISTICS b, c, d TYPE minmax; ALTER TABLE tab MATERIALIZE STATISTICS b, c, d; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*10000*/ and c < -1/*9990*/ and d > toDateTime(9998, 'UTC')/*1*/) +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*10000*/ and c < -1/*9990*/ and d > cast(9998, 'DateTime')/*1*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS b, c, d; -SELECT 'Test statistics count_min:'; +SELECT 'Test statistics tdigest:'; -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab ADD STATISTICS b TYPE count_min; -ALTER TABLE tab ADD STATISTICS c TYPE count_min; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; - -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) +ALTER TABLE tab ADD STATISTICS b, c, d TYPE tdigest; +ALTER TABLE tab MATERIALIZE STATISTICS b, c, d; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*10000*/ and c < -1/*9990*/ and d > cast(9998, 'DateTime')/*1*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +ALTER TABLE tab DROP STATISTICS b, c, d; -ALTER TABLE tab DROP STATISTICS a, b, c; + +SELECT 'Test statistics uniq:'; + +ALTER TABLE tab ADD STATISTICS a, b, c, d TYPE uniq; +ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*100*/ and c = 0/*1000*/ and b = 0/*100*/ and a = '0'/*100*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +ALTER TABLE tab DROP STATISTICS a, b, c, d; SELECT 'Test statistics multi-types:'; -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab ADD STATISTICS b TYPE count_min, uniq, tdigest; -ALTER TABLE tab ADD STATISTICS c TYPE count_min, uniq, tdigest; -ALTER TABLE tab ADD STATISTICS d TYPE count_min, uniq, tdigest; +ALTER TABLE tab ADD STATISTICS a TYPE count_min, uniq; +ALTER TABLE tab ADD STATISTICS b TYPE count_min, minmax, uniq, tdigest; +ALTER TABLE tab ADD STATISTICS c TYPE count_min, minmax, uniq, tdigest; +ALTER TABLE tab ADD STATISTICS d TYPE count_min, minmax, uniq, tdigest; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/) +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*1*/ and c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d > cast(1, 'DateTime')/*9999*/ and c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS a, b, c, d; -SELECT 'Test estimating range condition:'; - -ALTER TABLE tab ADD STATISTICS b TYPE minmax; -ALTER TABLE tab MATERIALIZE STATISTICS b; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*5000*/ and b < 10/*100*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab ADD STATISTICS b TYPE tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS b; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c < 0/*5000*/ and b < 10/*100*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -ALTER TABLE tab DROP STATISTICS b; - - -SELECT 'Test estimating equals condition:'; - -ALTER TABLE tab ADD STATISTICS a TYPE uniq; -ALTER TABLE tab MATERIALIZE STATISTICS a; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b = 10/*100*/ and a = '0'/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; - -ALTER TABLE tab ADD STATISTICS a TYPE count_min; -ALTER TABLE tab MATERIALIZE STATISTICS a; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b = 10/*100*/ and a = '0'/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -ALTER TABLE tab DROP STATISTICS a; - DROP TABLE IF EXISTS tab SYNC; From a36424fc8c99fd5c20b52d80a10d232c8a61451c Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 5 Aug 2024 18:59:09 +0800 Subject: [PATCH 15/75] Add supported data types to documents --- .../table-engines/mergetree-family/mergetree.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index c2438ac593e..8fd60c36f94 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1007,6 +1007,19 @@ They can be used for prewhere optimization only if we enable `set allow_statisti [Count-min](https://en.wikipedia.org/wiki/Count%E2%80%93min_sketch) sketches which provide an approximate count of the frequency of each value in a column. +#### Supported Data Types {#supported-data-types} + +| Data Type / Statistics | count_min | minmax | tdigest | uniq | +|-------------------------|-----------|--------|---------|------| +| Integer types | ✔ | ✔ | ✔ | ✔ | +| Floating-point numbers | ✔ | ✔ | ✔ | ✔ | +| Boolean | ✔ | ✔ | ✔ | ✔ | +| Strings | ✔ | ✗ | ✗ | ✔ | +| Dates | ✔ | ✔ | ✔ | ✔ | + +You can also add `LowCardinality` and `Nullable` modifiers to the data types. + + ## Column-level Settings {#column-level-settings} Certain MergeTree settings can be override at column level: From 4fa30da1189a3ce8139dad6bce5da9264019d3b3 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 6 Aug 2024 09:56:38 +0800 Subject: [PATCH 16/75] Fix docs --- .../mergetree-family/mergetree.md | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 8fd60c36f94..80e5bf694d4 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -994,30 +994,36 @@ They can be used for prewhere optimization only if we enable `set allow_statisti - `MinMax` The minimum and maximum column value which allows to estimate the selectivity of range filters on numeric columns. + Supported data types: (U)Int*, Float*, Decimal(*), Boolean and Date*. - `TDigest` [TDigest](https://github.com/tdunning/t-digest) sketches which allow to compute approximate percentiles (e.g. the 90th percentile) for numeric columns. + Supported data types: (U)Int*, Float*, Decimal(*), Boolean and Date*. - `Uniq` [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) sketches which provide an estimation how many distinct values a column contains. + Supported data types: (U)Int*, Float*, Decimal(*), Boolean, Date* and (Fixed)String. - `count_min` [Count-min](https://en.wikipedia.org/wiki/Count%E2%80%93min_sketch) sketches which provide an approximate count of the frequency of each value in a column. + Supported data types: (U)Int*, Float*, Decimal(*), Boolean, Date* and (Fixed)String. -#### Supported Data Types {#supported-data-types} +Note that all statistics types support `LowCardinality` and `Nullable` modifiers to data types. -| Data Type / Statistics | count_min | minmax | tdigest | uniq | -|-------------------------|-----------|--------|---------|------| -| Integer types | ✔ | ✔ | ✔ | ✔ | -| Floating-point numbers | ✔ | ✔ | ✔ | ✔ | -| Boolean | ✔ | ✔ | ✔ | ✔ | -| Strings | ✔ | ✗ | ✗ | ✔ | -| Dates | ✔ | ✔ | ✔ | ✔ | +#### Supported operations of Column Statistics {#supported-operations-of-column-statistics} + +| | Equals | Range | +|-----------|---------|-------| +| count_min | ✔ | ✔ | +| MinMax | ✔ | ✔ | +| TDigest | ✔ | ✔ | +| Uniq | ✔ | ✗ | + +Please note that operation range represents >, >=, <, <=. -You can also add `LowCardinality` and `Nullable` modifiers to the data types. ## Column-level Settings {#column-level-settings} From 46da03030cd6ae115edb7d4b9054704a8312283c Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 6 Aug 2024 10:00:34 +0800 Subject: [PATCH 17/75] Add test for implicitly type conversion --- .../table-engines/mergetree-family/mergetree.md | 10 +++++++--- .../0_stateless/02864_statistics_estimation.sql | 6 ++++++ 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 80e5bf694d4..caf72e39467 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -994,22 +994,26 @@ They can be used for prewhere optimization only if we enable `set allow_statisti - `MinMax` The minimum and maximum column value which allows to estimate the selectivity of range filters on numeric columns. + Supported data types: (U)Int*, Float*, Decimal(*), Boolean and Date*. - `TDigest` [TDigest](https://github.com/tdunning/t-digest) sketches which allow to compute approximate percentiles (e.g. the 90th percentile) for numeric columns. + Supported data types: (U)Int*, Float*, Decimal(*), Boolean and Date*. - `Uniq` [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) sketches which provide an estimation how many distinct values a column contains. - Supported data types: (U)Int*, Float*, Decimal(*), Boolean, Date* and (Fixed)String. + + Supported data types: (U)Int*, Float*, Decimal*, Boolean, Date* and (Fixed)String. - `count_min` [Count-min](https://en.wikipedia.org/wiki/Count%E2%80%93min_sketch) sketches which provide an approximate count of the frequency of each value in a column. - Supported data types: (U)Int*, Float*, Decimal(*), Boolean, Date* and (Fixed)String. + + Supported data types: (U)Int*, Float*, Decimal*, Boolean, Date* and (Fixed)String. Note that all statistics types support `LowCardinality` and `Nullable` modifiers to data types. @@ -1022,7 +1026,7 @@ Note that all statistics types support `LowCardinality` and `Nullable` modifiers | TDigest | ✔ | ✔ | | Uniq | ✔ | ✗ | -Please note that operation range represents >, >=, <, <=. +Please note that operation `Range` represents >, >=, < or <=. diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 65b111c285b..4a221686069 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -80,6 +80,12 @@ SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTi FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d > cast(1, 'DateTime')/*9999*/ and c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + +SELECT 'Test statistics implicitly type conversion:'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and c = '0'/*100*/ and b > 0/*9990*/ and a = '10000'/*0*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; + ALTER TABLE tab DROP STATISTICS a, b, c, d; From 8259a9827e5f1764fffa508f8e721f643c102d15 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 6 Aug 2024 10:29:42 +0800 Subject: [PATCH 18/75] Update reference file for tests --- .../engines/table-engines/mergetree-family/mergetree.md | 9 ++++----- .../0_stateless/02864_statistics_estimation.reference | 4 ++++ 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index caf72e39467..751b07afe91 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1021,18 +1021,17 @@ Note that all statistics types support `LowCardinality` and `Nullable` modifiers | | Equals | Range | |-----------|---------|-------| -| count_min | ✔ | ✔ | -| MinMax | ✔ | ✔ | -| TDigest | ✔ | ✔ | +| count_min | ✔ | ✗ | +| MinMax | ✗ | ✔ | +| TDigest | ✗ | ✔ | | Uniq | ✔ | ✗ | Please note that operation `Range` represents >, >=, < or <=. - ## Column-level Settings {#column-level-settings} -Certain MergeTree settings can be override at column level: +Certain MergeTree settings can be overridden at column level: - `max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. - `min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index f59f94013da..cb274b2990e 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -22,3 +22,7 @@ Test statistics multi-types: Prewhere info Prewhere filter Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, _CAST(1, \'DateTime\'))) (removed) +Test statistics implicitly type conversion: + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(a, \'10000\'), equals(d, \'2024-08-06 09:58:09\'), equals(c, \'0\'), greater(b, 0)) (removed) From 3769f8a46537bdc2865d13ff948e48328efb3548 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 6 Aug 2024 20:44:42 +0800 Subject: [PATCH 19/75] Fix tests --- .../0_stateless/02864_statistics_estimation.reference | 6 +++--- tests/queries/0_stateless/02864_statistics_estimation.sql | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index cb274b2990e..fc370691830 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -14,15 +14,15 @@ Test statistics tdigest: Test statistics uniq: Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(d, _CAST(1, \'DateTime\')), equals(b, 0), equals(c, 0)) (removed) + Prewhere filter column: and(equals(d, _CAST(1, \'DateTime\')), equals(c, 0)) (removed) Test statistics multi-types: Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(d, _CAST(1, \'DateTime\')), less(c, -90), greater(b, 900)) (removed) + Prewhere filter column: and(equals(d, _CAST(1, \'DateTime\')), less(c, -90), greater(b, 900)) (removed) Prewhere info Prewhere filter Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, _CAST(1, \'DateTime\'))) (removed) Test statistics implicitly type conversion: Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'10000\'), equals(d, \'2024-08-06 09:58:09\'), equals(c, \'0\'), greater(b, 0)) (removed) + Prewhere filter column: and(equals(a, \'1\'), equals(d, \'2024-08-06 09:58:09\'), equals(c, \'0\'), greater(b, 0)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 4a221686069..6436cf68738 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -59,7 +59,7 @@ SELECT 'Test statistics uniq:'; ALTER TABLE tab ADD STATISTICS a, b, c, d TYPE uniq; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*100*/ and c = 0/*1000*/ and b = 0/*100*/ and a = '0'/*100*/) +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*100*/ and c = 0/*1000*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS a, b, c, d; @@ -73,7 +73,7 @@ ALTER TABLE tab ADD STATISTICS d TYPE count_min, minmax, uniq, tdigest; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*1*/ and c < -90/*900*/ and b > 900/*990*/ and a = '0'/*1*/) +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*1*/ and c < -90/*900*/ and b > 900/*990*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') @@ -83,7 +83,7 @@ WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; SELECT 'Test statistics implicitly type conversion:'; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and c = '0'/*100*/ and b > 0/*9990*/ and a = '10000'/*0*/) +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and c = '0'/*100*/ and b > 0/*9990*/ and a = '1'/*1*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS a, b, c, d; From aafe498b7f6045e75414c17abecad168aa0efb88 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 7 Aug 2024 10:43:49 +0800 Subject: [PATCH 20/75] Handle edge case: col_int32 > 10.6 --- src/Storages/Statistics/Statistics.cpp | 22 +++++++++++++++++++ src/Storages/Statistics/Statistics.h | 6 +++++ src/Storages/Statistics/StatisticsMinMax.cpp | 10 +++------ src/Storages/Statistics/StatisticsTDigest.cpp | 16 +++++--------- .../02864_statistics_estimation.sql | 10 +++++---- 5 files changed, 43 insertions(+), 21 deletions(-) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 1f159211a93..e94d8b67aff 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include #include #include #include @@ -10,6 +12,7 @@ #include #include #include +#include #include "config.h" /// USE_DATASKETCHES @@ -28,6 +31,25 @@ enum StatisticsFileVersion : UInt16 V0 = 0, }; +std::optional StatisticsUtils::tryConvertToFloat64(const Field & value, const DataTypePtr & value_data_type) +{ + if (value_data_type->isValueRepresentedByNumber()) + { + Field val_converted; + + /// For case val_int32 < 10.5 or val_int32 < '10.5' we should convert 10.5 to Float64. + if (isInteger(value_data_type) && (value.getType() == Field::Types::Float64 || value.getType() == Field::Types::String)) + val_converted = convertFieldToType(value, *DataTypeFactory::instance().get("Float64")); + + /// We should convert value to the real column data type and then translate it to Float64. + /// For example for expression col_date > '2024-08-07', if we directly convert '2024-08-07' to Float64, we will get null. + val_converted = convertFieldToType(value, *value_data_type); + if (val_converted.isNull()) + return {}; + return applyVisitor(FieldVisitorConvertToNumber(), val_converted); + } + return {}; +} IStatistics::IStatistics(const SingleStatisticsDescription & stat_) : stat(stat_) diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 1425be89d47..9f82f137f1c 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -15,6 +15,12 @@ constexpr auto STATS_FILE_PREFIX = "statistics_"; constexpr auto STATS_FILE_SUFFIX = ".stats"; +struct StatisticsUtils +{ + /// Returns std::nullopt if input Field cannot be converted to a concrete value + static std::optional tryConvertToFloat64(const Field & value, const DataTypePtr & value_data_type); +}; + /// Statistics describe properties of the values in the column, /// e.g. how many unique values exist, /// what are the N most frequent values, diff --git a/src/Storages/Statistics/StatisticsMinMax.cpp b/src/Storages/Statistics/StatisticsMinMax.cpp index 11a0704dae7..88522894427 100644 --- a/src/Storages/Statistics/StatisticsMinMax.cpp +++ b/src/Storages/Statistics/StatisticsMinMax.cpp @@ -1,10 +1,8 @@ #include -#include #include #include #include #include -#include #include @@ -53,12 +51,10 @@ void StatisticsMinMax::deserialize(ReadBuffer & buf) Float64 StatisticsMinMax::estimateLess(const Field & val) const { - Field val_converted = convertFieldToType(val, *data_type); - if (val_converted.isNull()) + auto val_as_float = StatisticsUtils::tryConvertToFloat64(val, data_type); + if (!val_as_float.has_value()) return 0; - auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); - if (val_as_float < min) return 0; @@ -68,7 +64,7 @@ Float64 StatisticsMinMax::estimateLess(const Field & val) const if (min == max) return (val_as_float != max) ? 0 : row_count; - return ((val_as_float - min) / (max - min)) * row_count; + return ((*val_as_float - min) / (max - min)) * row_count; } void minMaxStatisticsValidator(const SingleStatisticsDescription & /*statistics_description*/, DataTypePtr data_type) diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index cfd626c8ea8..a7ce43d2fe1 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -41,22 +41,18 @@ void StatisticsTDigest::deserialize(ReadBuffer & buf) Float64 StatisticsTDigest::estimateLess(const Field & val) const { - Field val_converted = convertFieldToType(val, *data_type); - if (val_converted.isNull()) + auto val_as_float = StatisticsUtils::tryConvertToFloat64(val, data_type); + if (!val_as_float.has_value()) return 0; - - auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); - return t_digest.getCountLessThan(val_as_float); + return t_digest.getCountLessThan(*val_as_float); } Float64 StatisticsTDigest::estimateEqual(const Field & val) const { - Field val_converted = convertFieldToType(val, *data_type); - if (val_converted.isNull()) + auto val_as_float = StatisticsUtils::tryConvertToFloat64(val, data_type); + if (!val_as_float.has_value()) return 0; - - auto val_as_float = applyVisitor(FieldVisitorConvertToNumber(), val_converted); - return t_digest.getCountEqual(val_as_float); + return t_digest.getCountEqual(*val_as_float); } void tdigestStatisticsValidator(const SingleStatisticsDescription & /*statistics_description*/, DataTypePtr data_type) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 6436cf68738..5eb51fe7111 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -67,9 +67,7 @@ ALTER TABLE tab DROP STATISTICS a, b, c, d; SELECT 'Test statistics multi-types:'; ALTER TABLE tab ADD STATISTICS a TYPE count_min, uniq; -ALTER TABLE tab ADD STATISTICS b TYPE count_min, minmax, uniq, tdigest; -ALTER TABLE tab ADD STATISTICS c TYPE count_min, minmax, uniq, tdigest; -ALTER TABLE tab ADD STATISTICS d TYPE count_min, minmax, uniq, tdigest; +ALTER TABLE tab ADD STATISTICS b, c, d TYPE count_min, minmax, uniq, tdigest; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') @@ -82,10 +80,14 @@ WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; SELECT 'Test statistics implicitly type conversion:'; + SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and c = '0'/*100*/ and b > 0/*9990*/ and a = '1'/*1*/) +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and c = '0'/*100*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and b > 50.1/*5000*/) +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; ALTER TABLE tab DROP STATISTICS a, b, c, d; From cfaa852af9e94bd25217bfbb690f2d6ad641f3d6 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 7 Aug 2024 11:40:01 +0800 Subject: [PATCH 21/75] Little fix --- src/Storages/Statistics/Statistics.cpp | 7 ++++--- .../0_stateless/02864_statistics_estimation.reference | 5 ++++- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index e94d8b67aff..691cf42aa42 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -40,10 +40,11 @@ std::optional StatisticsUtils::tryConvertToFloat64(const Field & value, /// For case val_int32 < 10.5 or val_int32 < '10.5' we should convert 10.5 to Float64. if (isInteger(value_data_type) && (value.getType() == Field::Types::Float64 || value.getType() == Field::Types::String)) val_converted = convertFieldToType(value, *DataTypeFactory::instance().get("Float64")); + else + /// We should convert value to the real column data type and then translate it to Float64. + /// For example for expression col_date > '2024-08-07', if we directly convert '2024-08-07' to Float64, we will get null. + val_converted = convertFieldToType(value, *value_data_type); - /// We should convert value to the real column data type and then translate it to Float64. - /// For example for expression col_date > '2024-08-07', if we directly convert '2024-08-07' to Float64, we will get null. - val_converted = convertFieldToType(value, *value_data_type); if (val_converted.isNull()) return {}; return applyVisitor(FieldVisitorConvertToNumber(), val_converted); diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index fc370691830..d475926df8b 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -25,4 +25,7 @@ Test statistics multi-types: Test statistics implicitly type conversion: Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'1\'), equals(d, \'2024-08-06 09:58:09\'), equals(c, \'0\'), greater(b, 0)) (removed) + Prewhere filter column: and(equals(d, \'2024-08-06 09:58:09\'), equals(c, \'0\')) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(equals(d, \'2024-08-06 09:58:09\'), greater(b, 50.1_Float64)) (removed) From 5f747bc41174cb03c06288393a026d2385d57e8b Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 8 Aug 2024 11:09:59 +0800 Subject: [PATCH 22/75] Add more test for implicitly type conversion --- .../02864_statistics_estimation.reference | 26 +++++++-- .../02864_statistics_estimation.sql | 55 ++++++++++++++++--- 2 files changed, 67 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index d475926df8b..077476d28c1 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -23,9 +23,23 @@ Test statistics multi-types: Prewhere filter Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, _CAST(1, \'DateTime\'))) (removed) Test statistics implicitly type conversion: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(d, \'2024-08-06 09:58:09\'), equals(c, \'0\')) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(d, \'2024-08-06 09:58:09\'), greater(b, 50.1_Float64)) (removed) +1 +0 +0 +0 +0 +50 +50 +0 +0 +50 +0 +0 +1 +1 +1 +0 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 5eb51fe7111..1189de46267 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -1,6 +1,7 @@ -- Tags: no-fasttest DROP TABLE IF EXISTS tab SYNC; +DROP TABLE IF EXISTS tab2 SYNC; SET allow_experimental_statistics = 1; SET allow_statistics_optimize = 1; @@ -78,17 +79,55 @@ SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTi FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d > cast(1, 'DateTime')/*9999*/ and c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +DROP TABLE IF EXISTS tab SYNC; SELECT 'Test statistics implicitly type conversion:'; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and c = '0'/*100*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +CREATE TABLE tab2 +( + a String, + b UInt64, + c UInt8, + d DateTime, + e Boolean, + f Float64, + g Decimal32(1), + pk String, +) Engine = MergeTree() ORDER BY pk; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = '2024-08-06 09:58:09'/*0*/ and b > 50.1/*5000*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; -ALTER TABLE tab DROP STATISTICS a, b, c, d; +ALTER TABLE tab2 ADD STATISTICS a TYPE count_min, uniq; +ALTER TABLE tab2 ADD STATISTICS b, c, d, e, f, g TYPE count_min, minmax, uniq, tdigest; +INSERT INTO tab2 select toString(number), number, number, cast(number, 'DateTime'), number % 2, number, toDecimal32(number, 1), toString(number) FROM system.numbers LIMIT 100; -DROP TABLE IF EXISTS tab SYNC; +SELECT count(*) FROM tab2 WHERE a = '0'; +SELECT count(*) FROM tab2 WHERE a = 0; -- { serverError NO_COMMON_TYPE } + +SELECT count(*) FROM tab2 WHERE b = 1.1; + +SELECT count(*) FROM tab2 WHERE c = 1.1; +SELECT count(*) FROM tab2 WHERE c = 1000; -- out of range of UInt16 + +SELECT count(*) FROM tab2 WHERE d = '2024-08-06 09:58:09'; +SELECT count(*) FROM tab2 WHERE d = '2024-08-06 09:58:0'; -- { serverError CANNOT_PARSE_DATETIME } + +SELECT count(*) FROM tab2 WHERE e = true; +SELECT count(*) FROM tab2 WHERE e = 'true'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab2 WHERE e = 1; +SELECT count(*) FROM tab2 WHERE e = 2; +SELECT count(*) FROM tab2 WHERE e = 1.1; +SELECT count(*) FROM tab2 WHERE e = '1'; + +SELECT count(*) FROM tab2 WHERE f = 1.1; +SELECT count(*) FROM tab2 WHERE f = '1.1'; +SELECT count(*) FROM tab2 WHERE f = 1; +SELECT count(*) FROM tab2 WHERE f = '1'; + +SELECT count(*) FROM tab2 WHERE g = toDecimal32(1.0, 1); +SELECT count(*) FROM tab2 WHERE g = toDecimal32(1.10, 1); +SELECT count(*) FROM tab2 WHERE g = toDecimal32(1.0, 2); +SELECT count(*) FROM tab2 WHERE g = 1.0; +SELECT count(*) FROM tab2 WHERE g = 1.0; +SELECT count(*) FROM tab2 WHERE g = '1.0'; + +DROP TABLE IF EXISTS tab2 SYNC; From 21cdbe03f94bd3bdaef8b02e61b09e5e79ab380c Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 8 Aug 2024 11:21:19 +0800 Subject: [PATCH 23/75] Add more test cases --- .../queries/0_stateless/02864_statistics_estimation.reference | 2 ++ tests/queries/0_stateless/02864_statistics_estimation.sql | 3 +++ 2 files changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index 077476d28c1..ad923d91dda 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -24,6 +24,8 @@ Test statistics multi-types: Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, _CAST(1, \'DateTime\'))) (removed) Test statistics implicitly type conversion: 1 +1 +1 0 0 0 diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 1189de46267..63cc2e47c97 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -103,7 +103,10 @@ INSERT INTO tab2 select toString(number), number, number, cast(number, 'DateTime SELECT count(*) FROM tab2 WHERE a = '0'; SELECT count(*) FROM tab2 WHERE a = 0; -- { serverError NO_COMMON_TYPE } +SELECT count(*) FROM tab2 WHERE b = 1; +SELECT count(*) FROM tab2 WHERE b = '1'; SELECT count(*) FROM tab2 WHERE b = 1.1; +SELECT count(*) FROM tab2 WHERE b = '1.1'; -- { serverError TYPE_MISMATCH } SELECT count(*) FROM tab2 WHERE c = 1.1; SELECT count(*) FROM tab2 WHERE c = 1000; -- out of range of UInt16 From 6e7587e9b3dbc791f6970fe23703c1d65aa4b7f0 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 8 Aug 2024 14:44:08 +0800 Subject: [PATCH 24/75] Fix test for old analyzer --- .../02864_statistics_estimation.reference | 10 ++++---- .../02864_statistics_estimation.sql | 25 ++++++++++--------- 2 files changed, 18 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference index ad923d91dda..0e6cad6528b 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ b/tests/queries/0_stateless/02864_statistics_estimation.reference @@ -6,22 +6,22 @@ Test statistics count_min: Test statistics minmax: Prewhere info Prewhere filter - Prewhere filter column: and(greater(d, _CAST(9998, \'DateTime\')), less(c, -1), greater(b, 0)) (removed) + Prewhere filter column: and(greater(d, CAST(9998, \'DateTime\')), less(c, -1), greater(b, 0)) (removed) Test statistics tdigest: Prewhere info Prewhere filter - Prewhere filter column: and(greater(d, _CAST(9998, \'DateTime\')), less(c, -1), greater(b, 0)) (removed) + Prewhere filter column: and(greater(d, CAST(9998, \'DateTime\')), less(c, -1), greater(b, 0)) (removed) Test statistics uniq: Prewhere info Prewhere filter - Prewhere filter column: and(equals(d, _CAST(1, \'DateTime\')), equals(c, 0)) (removed) + Prewhere filter column: and(equals(d, CAST(1, \'DateTime\')), equals(c, 0)) (removed) Test statistics multi-types: Prewhere info Prewhere filter - Prewhere filter column: and(equals(d, _CAST(1, \'DateTime\')), less(c, -90), greater(b, 900)) (removed) + Prewhere filter column: and(equals(d, CAST(1, \'DateTime\')), less(c, -90), greater(b, 900)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, _CAST(1, \'DateTime\'))) (removed) + Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, CAST(1, \'DateTime\'))) (removed) Test statistics implicitly type conversion: 1 1 diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql index 63cc2e47c97..94fd7d12e0a 100644 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ b/tests/queries/0_stateless/02864_statistics_estimation.sql @@ -6,6 +6,7 @@ DROP TABLE IF EXISTS tab2 SYNC; SET allow_experimental_statistics = 1; SET allow_statistics_optimize = 1; SET allow_suspicious_low_cardinality_types=1; +SET allow_experimental_analyzer=1; SET mutations_sync = 2; CREATE TABLE tab @@ -27,9 +28,9 @@ SELECT 'Test statistics count_min:'; ALTER TABLE tab ADD STATISTICS a, b, c TYPE count_min; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); ALTER TABLE tab DROP STATISTICS a, b, c; @@ -38,9 +39,9 @@ SELECT 'Test statistics minmax:'; ALTER TABLE tab ADD STATISTICS b, c, d TYPE minmax; ALTER TABLE tab MATERIALIZE STATISTICS b, c, d; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*10000*/ and c < -1/*9990*/ and d > cast(9998, 'DateTime')/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); ALTER TABLE tab DROP STATISTICS b, c, d; @@ -49,9 +50,9 @@ SELECT 'Test statistics tdigest:'; ALTER TABLE tab ADD STATISTICS b, c, d TYPE tdigest; ALTER TABLE tab MATERIALIZE STATISTICS b, c, d; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*10000*/ and c < -1/*9990*/ and d > cast(9998, 'DateTime')/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); ALTER TABLE tab DROP STATISTICS b, c, d; @@ -59,9 +60,9 @@ SELECT 'Test statistics uniq:'; ALTER TABLE tab ADD STATISTICS a, b, c, d TYPE uniq; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*100*/ and c = 0/*1000*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); ALTER TABLE tab DROP STATISTICS a, b, c, d; @@ -71,13 +72,13 @@ ALTER TABLE tab ADD STATISTICS a TYPE count_min, uniq; ALTER TABLE tab ADD STATISTICS b, c, d TYPE count_min, minmax, uniq, tdigest; ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*1*/ and c < -90/*900*/ and b > 900/*990*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); -SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') +SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d > cast(1, 'DateTime')/*9999*/ and c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); DROP TABLE IF EXISTS tab SYNC; From 624afd68c86bb4fc3eee76f3fb9779ffb2319c68 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Fri, 16 Aug 2024 01:24:52 +0400 Subject: [PATCH 25/75] parallel attach from multiple partitions --- src/Storages/StorageReplicatedMergeTree.cpp | 112 ++++++++++++++------ src/Storages/StorageReplicatedMergeTree.h | 13 +++ 2 files changed, 95 insertions(+), 30 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a3c1ab7cdff..67f51b186c5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8030,25 +8030,84 @@ void StorageReplicatedMergeTree::replacePartitionFrom( /// First argument is true, because we possibly will add new data to current table. auto lock1 = lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef().lock_acquire_timeout); auto lock2 = source_table->lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef().lock_acquire_timeout); - auto storage_settings_ptr = getSettings(); - auto source_metadata_snapshot = source_table->getInMemoryMetadataPtr(); - auto metadata_snapshot = getInMemoryMetadataPtr(); + const auto storage_settings_ptr = getSettings(); + const auto source_metadata_snapshot = source_table->getInMemoryMetadataPtr(); + const auto metadata_snapshot = getInMemoryMetadataPtr(); + const MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); - Stopwatch watch; + std::unordered_set partitions; + if (partition->as()->all) + { + if (replace) + throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH/ATTACH PARTITION ALL currently"); + + partitions = src_data.getAllPartitionIds(); + } + else + { + partitions = std::unordered_set(); + partitions.emplace(getPartitionIDFromQuery(partition, query_context)); + } + LOG_INFO(log, "Will try to attach {} partitions", partitions.size()); + + const Stopwatch watch; ProfileEventsScope profile_events_scope; + const auto zookeeper = getZooKeeper(); - MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); - String partition_id = getPartitionIDFromQuery(partition, query_context); + const bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication + || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + try + { + std::unique_ptr entries[partitions.size()]; + size_t idx = 0; + for (const auto & partition_id : partitions) + { + entries[idx] = replacePartitionFromImpl(watch, + profile_events_scope, + metadata_snapshot, + src_data, + partition_id, + zookeeper, + replace, + zero_copy_enabled, + storage_settings_ptr->always_use_copy_instead_of_hardlinks, + query_context); + ++idx; + } + + for (const auto & entry : entries) + waitForLogEntryToBeProcessedIfNecessary(*entry, query_context); + + lock2.reset(); + lock1.reset(); + } + catch(...) + { + lock2.reset(); + lock1.reset(); + + throw; + } +} + +std::unique_ptr StorageReplicatedMergeTree::replacePartitionFromImpl( + const Stopwatch & watch, + ProfileEventsScope & profile_events_scope, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreeData & src_data, + const String & partition_id, + const ZooKeeperPtr & zookeeper, + bool replace, + const bool & zero_copy_enabled, + const bool & always_use_copy_instead_of_hardlinks, + const ContextPtr & query_context) +{ /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. DataPartsVector src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); - LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); - static const String TMP_PREFIX = "tmp_replace_from_"; - auto zookeeper = getZooKeeper(); - /// Retry if alter_partition_version changes for (size_t retry = 0; retry < 1000; ++retry) { @@ -8133,11 +8192,9 @@ void StorageReplicatedMergeTree::replacePartitionFrom( UInt64 index = lock->getNumber(); MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication - || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; IDataPartStorage::ClonePartParams clone_params { - .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), + .copy_instead_of_hardlink = always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; if (replace) @@ -8145,7 +8202,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( /// Replace can only work on the same disk auto [dst_part, part_lock] = cloneAndLoadDataPart( src_part, - TMP_PREFIX, + TMP_PREFIX_REPLACE_PARTITION_FROM, dst_part_info, metadata_snapshot, clone_params, @@ -8160,7 +8217,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( /// Attach can work on another disk auto [dst_part, part_lock] = cloneAndLoadDataPart( src_part, - TMP_PREFIX, + TMP_PREFIX_REPLACE_PARTITION_FROM, dst_part_info, metadata_snapshot, clone_params, @@ -8176,15 +8233,16 @@ void StorageReplicatedMergeTree::replacePartitionFrom( part_checksums.emplace_back(hash_hex); } - ReplicatedMergeTreeLogEntryData entry; + //ReplicatedMergeTreeLogEntryData entry; + auto entry = std::make_unique(); { auto src_table_id = src_data.getStorageID(); - entry.type = ReplicatedMergeTreeLogEntryData::REPLACE_RANGE; - entry.source_replica = replica_name; - entry.create_time = time(nullptr); - entry.replace_range_entry = std::make_shared(); + entry->type = ReplicatedMergeTreeLogEntryData::REPLACE_RANGE; + entry->source_replica = replica_name; + entry->create_time = time(nullptr); + entry->replace_range_entry = std::make_shared(); - auto & entry_replace = *entry.replace_range_entry; + auto & entry_replace = *entry->replace_range_entry; entry_replace.drop_range_part_name = drop_range_fake_part_name; entry_replace.from_database = src_table_id.database_name; entry_replace.from_table = src_table_id.table_name; @@ -8225,7 +8283,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( ops.emplace_back(zkutil::makeSetRequest(alter_partition_version_path, "", alter_partition_version_stat.version)); /// Just update version, because merges assignment relies on it ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); - ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); + ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry->toString(), zkutil::CreateMode::PersistentSequential)); Transaction transaction(*this, NO_TRANSACTION_RAW); { @@ -8275,14 +8333,11 @@ void StorageReplicatedMergeTree::replacePartitionFrom( } String log_znode_path = dynamic_cast(*op_results.back()).path_created; - entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + entry->znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); for (auto & lock : ephemeral_locks) lock.assumeUnlocked(); - lock2.reset(); - lock1.reset(); - /// We need to pull the REPLACE_RANGE before cleaning the replaced parts (otherwise CHeckThread may decide that parts are lost) queue.pullLogsToQueue(getZooKeeperAndAssertNotReadonly(), {}, ReplicatedMergeTreeQueue::SYNC); // No need to block operations further, especially that in case we have to wait for mutation to finish, the intent would block @@ -8291,10 +8346,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( parts_holder.clear(); cleanup_thread.wakeup(); - - waitForLogEntryToBeProcessedIfNecessary(entry, query_context); - - return; + return entry; } throw Exception( diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 2e54f17d5d5..4291aebf3e8 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -37,6 +37,7 @@ #include #include #include +#include #include #include @@ -1013,6 +1014,18 @@ private: DataPartsVector::const_iterator it; }; + const String TMP_PREFIX_REPLACE_PARTITION_FROM = "tmp_replace_from_"; + std::unique_ptr replacePartitionFromImpl( + const Stopwatch & watch, + ProfileEventsScope & profile_events_scope, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreeData & src_data, + const String & partition_id, + const zkutil::ZooKeeperPtr & zookeeper, + bool replace, + const bool & zero_copy_enabled, + const bool & always_use_copy_instead_of_hardlinks, + const ContextPtr & query_context); }; String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info); From c31c4154ca47c6f1313c04974b7bb95bbcb34cff Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Fri, 16 Aug 2024 01:49:54 +0400 Subject: [PATCH 26/75] fix cs --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 67f51b186c5..38309282768 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8083,7 +8083,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( lock2.reset(); lock1.reset(); } - catch(...) + catch (...) { lock2.reset(); lock1.reset(); From 46f27b03f9dd2f122da6cfcb9905f5bb7ec944e7 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Fri, 16 Aug 2024 15:48:28 +0400 Subject: [PATCH 27/75] fix transaction --- src/Storages/StorageReplicatedMergeTree.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 38309282768..4915bf9f366 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8108,6 +8108,13 @@ std::unique_ptr StorageReplicatedMergeTree::rep DataPartsVector src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); + std::optional txn; + if (auto query_txn = query_context->getZooKeeperMetadataTransaction()) + txn.emplace(query_txn->getZooKeeper(), + query_txn->getDatabaseZooKeeperPath(), + query_txn->isInitialQuery(), + query_txn->getTaskZooKeeperPath()); + /// Retry if alter_partition_version changes for (size_t retry = 0; retry < 1000; ++retry) { @@ -8275,7 +8282,7 @@ std::unique_ptr StorageReplicatedMergeTree::rep ephemeral_locks[i].getUnlockOp(ops); } - if (auto txn = query_context->getZooKeeperMetadataTransaction()) + if (txn) txn->moveOpsTo(ops); delimiting_block_lock->getUnlockOp(ops); From 6fb5051237448addf82faf50677179c551958f1b Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 21 Aug 2024 12:47:12 +0800 Subject: [PATCH 28/75] Fix conflicts --- .../Statistics/ConditionSelectivityEstimator.cpp | 2 -- src/Storages/Statistics/StatisticsMinMax.cpp | 16 ++++++++-------- src/Storages/Statistics/StatisticsMinMax.h | 4 ++-- src/Storages/Statistics/StatisticsTDigest.cpp | 6 +++--- src/Storages/Statistics/StatisticsTDigest.h | 6 +++--- 5 files changed, 16 insertions(+), 18 deletions(-) diff --git a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp index eadd15a8f7f..432659f51f8 100644 --- a/src/Storages/Statistics/ConditionSelectivityEstimator.cpp +++ b/src/Storages/Statistics/ConditionSelectivityEstimator.cpp @@ -41,8 +41,6 @@ Float64 ConditionSelectivityEstimator::ColumnSelectivityEstimator::estimateEqual { return default_cond_equal_factor * rows; } - return default_cond_equal_factor * rows; - Float64 result = 0; Float64 partial_cnt = 0; for (const auto & [key, estimator] : part_statistics) diff --git a/src/Storages/Statistics/StatisticsMinMax.cpp b/src/Storages/Statistics/StatisticsMinMax.cpp index 88522894427..aafa34bc6d8 100644 --- a/src/Storages/Statistics/StatisticsMinMax.cpp +++ b/src/Storages/Statistics/StatisticsMinMax.cpp @@ -15,8 +15,8 @@ namespace ErrorCodes extern const int ILLEGAL_STATISTICS; } -StatisticsMinMax::StatisticsMinMax(const SingleStatisticsDescription & statistics_description, const DataTypePtr & data_type_) - : IStatistics(statistics_description) +StatisticsMinMax::StatisticsMinMax(const SingleStatisticsDescription & description, const DataTypePtr & data_type_) + : IStatistics(description) , data_type(data_type_) { } @@ -67,17 +67,17 @@ Float64 StatisticsMinMax::estimateLess(const Field & val) const return ((*val_as_float - min) / (max - min)) * row_count; } -void minMaxStatisticsValidator(const SingleStatisticsDescription & /*statistics_description*/, DataTypePtr data_type) +void minMaxStatisticsValidator(const SingleStatisticsDescription & /*description*/, const DataTypePtr & data_type) { - data_type = removeNullable(data_type); - data_type = removeLowCardinalityAndNullable(data_type); + auto inner_data_type = removeNullable(data_type); + inner_data_type = removeLowCardinalityAndNullable(data_type); if (!data_type->isValueRepresentedByNumber()) - throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'minmax' do not support type {}", data_type->getName()); + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'minmax' do not support type {}", inner_data_type->getName()); } -StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type) +StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type) { - return std::make_shared(statistics_description, data_type); + return std::make_shared(description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsMinMax.h b/src/Storages/Statistics/StatisticsMinMax.h index 524494b520b..c60fa810c47 100644 --- a/src/Storages/Statistics/StatisticsMinMax.h +++ b/src/Storages/Statistics/StatisticsMinMax.h @@ -27,7 +27,7 @@ private: DataTypePtr data_type; }; -void minMaxStatisticsValidator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type); -StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & statistics_description, DataTypePtr data_type); +void minMaxStatisticsValidator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); +StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); } diff --git a/src/Storages/Statistics/StatisticsTDigest.cpp b/src/Storages/Statistics/StatisticsTDigest.cpp index b90075b10be..285b779036f 100644 --- a/src/Storages/Statistics/StatisticsTDigest.cpp +++ b/src/Storages/Statistics/StatisticsTDigest.cpp @@ -53,7 +53,7 @@ Float64 StatisticsTDigest::estimateEqual(const Field & val) const return t_digest.getCountEqual(*val_as_float); } -void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type) +void tdigestStatisticsValidator(const SingleStatisticsDescription & /*description*/, const DataTypePtr & data_type) { DataTypePtr inner_data_type = removeNullable(data_type); inner_data_type = removeLowCardinalityAndNullable(inner_data_type); @@ -61,9 +61,9 @@ void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName()); } -StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr) +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type) { - return std::make_shared(stat); + return std::make_shared(description, data_type); } } diff --git a/src/Storages/Statistics/StatisticsTDigest.h b/src/Storages/Statistics/StatisticsTDigest.h index 91a477c2b94..5e744fee2ce 100644 --- a/src/Storages/Statistics/StatisticsTDigest.h +++ b/src/Storages/Statistics/StatisticsTDigest.h @@ -9,7 +9,7 @@ namespace DB class StatisticsTDigest : public IStatistics { public: - explicit StatisticsTDigest(const SingleStatisticsDescription & stat_); + explicit StatisticsTDigest(const SingleStatisticsDescription & description, const DataTypePtr & data_type_); void update(const ColumnPtr & column) override; @@ -24,7 +24,7 @@ private: DataTypePtr data_type; }; -void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type); -StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr); +void tdigestStatisticsValidator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); +StatisticsPtr tdigestStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type); } From fc704020e9d64e15b80f8dd3c6db8871925bca9b Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 21 Aug 2024 13:29:42 +0800 Subject: [PATCH 29/75] Update tests --- src/Storages/Statistics/StatisticsMinMax.cpp | 6 +- src/Storages/Statistics/StatisticsUniq.cpp | 2 +- .../0_stateless/02864_statistics_ddl.sql | 32 ++++ .../02864_statistics_estimation.reference | 47 ------ .../02864_statistics_estimation.sql | 137 ------------------ .../02864_statistics_predicates.reference | 23 +++ .../02864_statistics_predicates.sql | 46 +++++- 7 files changed, 100 insertions(+), 193 deletions(-) delete mode 100644 tests/queries/0_stateless/02864_statistics_estimation.reference delete mode 100644 tests/queries/0_stateless/02864_statistics_estimation.sql diff --git a/src/Storages/Statistics/StatisticsMinMax.cpp b/src/Storages/Statistics/StatisticsMinMax.cpp index aafa34bc6d8..6ae05cb0a6b 100644 --- a/src/Storages/Statistics/StatisticsMinMax.cpp +++ b/src/Storages/Statistics/StatisticsMinMax.cpp @@ -70,9 +70,9 @@ Float64 StatisticsMinMax::estimateLess(const Field & val) const void minMaxStatisticsValidator(const SingleStatisticsDescription & /*description*/, const DataTypePtr & data_type) { auto inner_data_type = removeNullable(data_type); - inner_data_type = removeLowCardinalityAndNullable(data_type); - if (!data_type->isValueRepresentedByNumber()) - throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'minmax' do not support type {}", inner_data_type->getName()); + inner_data_type = removeLowCardinalityAndNullable(inner_data_type); + if (!inner_data_type->isValueRepresentedByNumber()) + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'minmax' do not support type {}", data_type->getName()); } StatisticsPtr minMaxStatisticsCreator(const SingleStatisticsDescription & description, const DataTypePtr & data_type) diff --git a/src/Storages/Statistics/StatisticsUniq.cpp b/src/Storages/Statistics/StatisticsUniq.cpp index 07311b5b86d..21bc5d227ef 100644 --- a/src/Storages/Statistics/StatisticsUniq.cpp +++ b/src/Storages/Statistics/StatisticsUniq.cpp @@ -56,7 +56,7 @@ void uniqStatisticsValidator(const SingleStatisticsDescription & /*description*/ { DataTypePtr inner_data_type = removeNullable(data_type); inner_data_type = removeLowCardinalityAndNullable(inner_data_type); - if (!inner_data_type->isValueRepresentedByNumber()) + if (!inner_data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName()); } diff --git a/tests/queries/0_stateless/02864_statistics_ddl.sql b/tests/queries/0_stateless/02864_statistics_ddl.sql index 32b56a842b7..0fc29d0c362 100644 --- a/tests/queries/0_stateless/02864_statistics_ddl.sql +++ b/tests/queries/0_stateless/02864_statistics_ddl.sql @@ -7,6 +7,7 @@ SET mutations_sync = 1; DROP TABLE IF EXISTS tab; +SET allow_experimental_statistics = 0; -- Error case: Can't create statistics when allow_experimental_statistics = 0 CREATE TABLE tab (col Float64 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError INCORRECT_QUERY } @@ -94,6 +95,30 @@ CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(count_min)) Engine = MergeT CREATE TABLE tab (col UUID STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } CREATE TABLE tab (col IPv6 STATISTICS(count_min)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +-- minmax requires data_type.isValueRepresentedByInteger +-- These types work: +CREATE TABLE tab (col UInt8 STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col UInt256 STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Float32 STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Decimal32(3) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Date32 STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col DateTime64 STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Enum('hello', 'world') STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col IPv4 STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col Nullable(UInt8) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +-- These types don't work: +CREATE TABLE tab (col String STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col FixedString(1) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Array(Float64) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col UUID STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } +CREATE TABLE tab (col IPv6 STATISTICS(minmax)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } + -- CREATE TABLE was easy, ALTER is more fun CREATE TABLE tab @@ -173,6 +198,13 @@ ALTER TABLE tab MODIFY STATISTICS f64 TYPE count_min; ALTER TABLE tab DROP STATI -- Doesn't work: ALTER TABLE tab ADD STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS } ALTER TABLE tab MODIFY STATISTICS a TYPE count_min; -- { serverError ILLEGAL_STATISTICS } +-- minmax +-- Works: +ALTER TABLE tab ADD STATISTICS f64 TYPE minmax; ALTER TABLE tab DROP STATISTICS f64; +ALTER TABLE tab MODIFY STATISTICS f64 TYPE minmax; ALTER TABLE tab DROP STATISTICS f64; +-- Doesn't work: +ALTER TABLE tab ADD STATISTICS a TYPE minmax; -- { serverError ILLEGAL_STATISTICS } +ALTER TABLE tab MODIFY STATISTICS a TYPE minmax; -- { serverError ILLEGAL_STATISTICS } -- Any data type changes on columns with statistics are disallowed, for simplicity even if the new data type is compatible with all existing -- statistics objects (e.g. tdigest can be created on Float64 and UInt64) diff --git a/tests/queries/0_stateless/02864_statistics_estimation.reference b/tests/queries/0_stateless/02864_statistics_estimation.reference deleted file mode 100644 index 0e6cad6528b..00000000000 --- a/tests/queries/0_stateless/02864_statistics_estimation.reference +++ /dev/null @@ -1,47 +0,0 @@ -CREATE TABLE default.tab\n(\n `a` String,\n `b` UInt64,\n `c` Int64,\n `d` DateTime,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 -Test statistics count_min: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'0\'), equals(b, 0), equals(c, 0)) (removed) -Test statistics minmax: - Prewhere info - Prewhere filter - Prewhere filter column: and(greater(d, CAST(9998, \'DateTime\')), less(c, -1), greater(b, 0)) (removed) -Test statistics tdigest: - Prewhere info - Prewhere filter - Prewhere filter column: and(greater(d, CAST(9998, \'DateTime\')), less(c, -1), greater(b, 0)) (removed) -Test statistics uniq: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(d, CAST(1, \'DateTime\')), equals(c, 0)) (removed) -Test statistics multi-types: - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(d, CAST(1, \'DateTime\')), less(c, -90), greater(b, 900)) (removed) - Prewhere info - Prewhere filter - Prewhere filter column: and(equals(a, \'10000\'), equals(b, 0), less(c, 0), greater(d, CAST(1, \'DateTime\'))) (removed) -Test statistics implicitly type conversion: -1 -1 -1 -0 -0 -0 -0 -50 -50 -0 -0 -50 -0 -0 -1 -1 -1 -0 -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/02864_statistics_estimation.sql b/tests/queries/0_stateless/02864_statistics_estimation.sql deleted file mode 100644 index 94fd7d12e0a..00000000000 --- a/tests/queries/0_stateless/02864_statistics_estimation.sql +++ /dev/null @@ -1,137 +0,0 @@ --- Tags: no-fasttest - -DROP TABLE IF EXISTS tab SYNC; -DROP TABLE IF EXISTS tab2 SYNC; - -SET allow_experimental_statistics = 1; -SET allow_statistics_optimize = 1; -SET allow_suspicious_low_cardinality_types=1; -SET allow_experimental_analyzer=1; -SET mutations_sync = 2; - -CREATE TABLE tab -( - a String, - b UInt64, - c Int64, - d DateTime, - pk String, -) Engine = MergeTree() ORDER BY pk -SETTINGS min_bytes_for_wide_part = 0; - -SHOW CREATE TABLE tab; - -INSERT INTO tab select toString(number % 10000), number % 1000, -(number % 100), cast(number, 'DateTime'), generateUUIDv4() FROM system.numbers LIMIT 10000; - - -SELECT 'Test statistics count_min:'; - -ALTER TABLE tab ADD STATISTICS a, b, c TYPE count_min; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c; -SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE c = 0/*100*/ and b = 0/*10*/ and a = '0'/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); -ALTER TABLE tab DROP STATISTICS a, b, c; - - -SELECT 'Test statistics minmax:'; - -ALTER TABLE tab ADD STATISTICS b, c, d TYPE minmax; -ALTER TABLE tab MATERIALIZE STATISTICS b, c, d; - -SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*10000*/ and c < -1/*9990*/ and d > cast(9998, 'DateTime')/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); - -ALTER TABLE tab DROP STATISTICS b, c, d; - - -SELECT 'Test statistics tdigest:'; - -ALTER TABLE tab ADD STATISTICS b, c, d TYPE tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS b, c, d; -SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE b > 0/*10000*/ and c < -1/*9990*/ and d > cast(9998, 'DateTime')/*1*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); -ALTER TABLE tab DROP STATISTICS b, c, d; - - -SELECT 'Test statistics uniq:'; - -ALTER TABLE tab ADD STATISTICS a, b, c, d TYPE uniq; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; -SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*100*/ and c = 0/*1000*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); -ALTER TABLE tab DROP STATISTICS a, b, c, d; - - -SELECT 'Test statistics multi-types:'; - -ALTER TABLE tab ADD STATISTICS a TYPE count_min, uniq; -ALTER TABLE tab ADD STATISTICS b, c, d TYPE count_min, minmax, uniq, tdigest; -ALTER TABLE tab MATERIALIZE STATISTICS a, b, c, d; - -SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d = cast(1, 'DateTime')/*1*/ and c < -90/*900*/ and b > 900/*990*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); - -SELECT replaceRegexpAll(explain, '_CAST', 'CAST') FROM (SELECT replaceRegexpAll(explain, '__table1.|_UInt8|_Int8|_UInt16|_String|_DateTime', '') as explain -FROM (EXPLAIN actions=1 SELECT count(*) FROM tab WHERE d > cast(1, 'DateTime')/*9999*/ and c < 0/*9900*/ and b = 0/*10*/ and a = '10000'/*0*/) -WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'); - -DROP TABLE IF EXISTS tab SYNC; - -SELECT 'Test statistics implicitly type conversion:'; - -CREATE TABLE tab2 -( - a String, - b UInt64, - c UInt8, - d DateTime, - e Boolean, - f Float64, - g Decimal32(1), - pk String, -) Engine = MergeTree() ORDER BY pk; - -ALTER TABLE tab2 ADD STATISTICS a TYPE count_min, uniq; -ALTER TABLE tab2 ADD STATISTICS b, c, d, e, f, g TYPE count_min, minmax, uniq, tdigest; - -INSERT INTO tab2 select toString(number), number, number, cast(number, 'DateTime'), number % 2, number, toDecimal32(number, 1), toString(number) FROM system.numbers LIMIT 100; - -SELECT count(*) FROM tab2 WHERE a = '0'; -SELECT count(*) FROM tab2 WHERE a = 0; -- { serverError NO_COMMON_TYPE } - -SELECT count(*) FROM tab2 WHERE b = 1; -SELECT count(*) FROM tab2 WHERE b = '1'; -SELECT count(*) FROM tab2 WHERE b = 1.1; -SELECT count(*) FROM tab2 WHERE b = '1.1'; -- { serverError TYPE_MISMATCH } - -SELECT count(*) FROM tab2 WHERE c = 1.1; -SELECT count(*) FROM tab2 WHERE c = 1000; -- out of range of UInt16 - -SELECT count(*) FROM tab2 WHERE d = '2024-08-06 09:58:09'; -SELECT count(*) FROM tab2 WHERE d = '2024-08-06 09:58:0'; -- { serverError CANNOT_PARSE_DATETIME } - -SELECT count(*) FROM tab2 WHERE e = true; -SELECT count(*) FROM tab2 WHERE e = 'true'; -- { serverError TYPE_MISMATCH } -SELECT count(*) FROM tab2 WHERE e = 1; -SELECT count(*) FROM tab2 WHERE e = 2; -SELECT count(*) FROM tab2 WHERE e = 1.1; -SELECT count(*) FROM tab2 WHERE e = '1'; - -SELECT count(*) FROM tab2 WHERE f = 1.1; -SELECT count(*) FROM tab2 WHERE f = '1.1'; -SELECT count(*) FROM tab2 WHERE f = 1; -SELECT count(*) FROM tab2 WHERE f = '1'; - -SELECT count(*) FROM tab2 WHERE g = toDecimal32(1.0, 1); -SELECT count(*) FROM tab2 WHERE g = toDecimal32(1.10, 1); -SELECT count(*) FROM tab2 WHERE g = toDecimal32(1.0, 2); -SELECT count(*) FROM tab2 WHERE g = 1.0; -SELECT count(*) FROM tab2 WHERE g = 1.0; -SELECT count(*) FROM tab2 WHERE g = '1.0'; - -DROP TABLE IF EXISTS tab2 SYNC; diff --git a/tests/queries/0_stateless/02864_statistics_predicates.reference b/tests/queries/0_stateless/02864_statistics_predicates.reference index ffbd7269e05..c2d74300fa9 100644 --- a/tests/queries/0_stateless/02864_statistics_predicates.reference +++ b/tests/queries/0_stateless/02864_statistics_predicates.reference @@ -3,10 +3,13 @@ u64 and = 10 10 10 +10 0 0 0 0 +0 +10 10 10 10 @@ -16,10 +19,13 @@ u64 and < 70 70 70 +70 80 80 80 80 +80 +70 70 70 70 @@ -29,6 +35,8 @@ f64 and = 10 10 10 +10 +0 0 0 0 @@ -37,6 +45,8 @@ f64 and = 10 10 10 +10 +0 0 0 0 @@ -46,6 +56,8 @@ f64 and < 70 70 70 +70 +80 80 80 80 @@ -54,6 +66,8 @@ f64 and < 70 70 70 +70 +80 80 80 80 @@ -63,6 +77,8 @@ dt and = 0 0 0 +0 +10 10 10 10 @@ -72,6 +88,8 @@ dt and < 10000 10000 10000 +10000 +70 70 70 70 @@ -89,6 +107,10 @@ b and = 5000 5000 5000 +5000 +5000 +5000 +0 0 0 0 @@ -96,3 +118,4 @@ b and = s and = 10 10 +10 diff --git a/tests/queries/0_stateless/02864_statistics_predicates.sql b/tests/queries/0_stateless/02864_statistics_predicates.sql index 779116cf19a..473a7bc95ad 100644 --- a/tests/queries/0_stateless/02864_statistics_predicates.sql +++ b/tests/queries/0_stateless/02864_statistics_predicates.sql @@ -12,46 +12,56 @@ CREATE TABLE tab ( u64 UInt64, u64_tdigest UInt64 STATISTICS(tdigest), + u64_minmax UInt64 STATISTICS(minmax), u64_count_min UInt64 STATISTICS(count_min), u64_uniq UInt64 STATISTICS(uniq), f64 Float64, f64_tdigest Float64 STATISTICS(tdigest), + f64_minmax Float64 STATISTICS(minmax), f64_count_min Float64 STATISTICS(count_min), f64_uniq Float64 STATISTICS(uniq), dt DateTime, dt_tdigest DateTime STATISTICS(tdigest), + dt_minmax DateTime STATISTICS(minmax), dt_count_min DateTime STATISTICS(count_min), dt_uniq DateTime STATISTICS(uniq), b Bool, b_tdigest Bool STATISTICS(tdigest), + b_minmax Bool STATISTICS(minmax), b_count_min Bool STATISTICS(count_min), b_uniq Bool STATISTICS(uniq), s String, -- s_tdigest String STATISTICS(tdigest), -- not supported by tdigest - s_count_min String STATISTICS(count_min) - -- s_uniq String STATISTICS(uniq), -- not supported by uniq + -- s_minmax String STATISTICS(minmax), -- not supported by minmax + s_count_min String STATISTICS(count_min), + s_uniq String STATISTICS(uniq) ) Engine = MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO tab -- SELECT number % 10000, number % 1000, -(number % 100) FROM system.numbers LIMIT 10000; -SELECT number % 1000, +SELECT number % 1000, -- u64 number % 1000, number % 1000, number % 1000, number % 1000, + number % 1000, -- f64 number % 1000, number % 1000, number % 1000, number % 1000, + number % 1000, -- dt number % 1000, number % 1000, number % 1000, + number % 1000, + number % 2, -- b number % 2, number % 2, number % 2, number % 2, toString(number % 1000), + toString(number % 1000), toString(number % 1000) FROM system.numbers LIMIT 10000; @@ -61,21 +71,25 @@ SELECT 'u64 and ='; SELECT count(*) FROM tab WHERE u64 = 7; SELECT count(*) FROM tab WHERE u64_tdigest = 7; +SELECT count(*) FROM tab WHERE u64_minmax = 7; SELECT count(*) FROM tab WHERE u64_count_min = 7; SELECT count(*) FROM tab WHERE u64_uniq = 7; SELECT count(*) FROM tab WHERE u64 = 7.7; SELECT count(*) FROM tab WHERE u64_tdigest = 7.7; +SELECT count(*) FROM tab WHERE u64_minmax = 7.7; SELECT count(*) FROM tab WHERE u64_count_min = 7.7; SELECT count(*) FROM tab WHERE u64_uniq = 7.7; SELECT count(*) FROM tab WHERE u64 = '7'; SELECT count(*) FROM tab WHERE u64_tdigest = '7'; +SELECT count(*) FROM tab WHERE u64_minmax = '7'; SELECT count(*) FROM tab WHERE u64_count_min = '7'; SELECT count(*) FROM tab WHERE u64_uniq = '7'; SELECT count(*) FROM tab WHERE u64 = '7.7'; -- { serverError TYPE_MISMATCH } SELECT count(*) FROM tab WHERE u64_tdigest = '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_minmax = '7.7'; -- { serverError TYPE_MISMATCH } SELECT count(*) FROM tab WHERE u64_count_min = '7.7'; -- { serverError TYPE_MISMATCH } SELECT count(*) FROM tab WHERE u64_uniq = '7.7'; -- { serverError TYPE_MISMATCH } @@ -83,21 +97,25 @@ SELECT 'u64 and <'; SELECT count(*) FROM tab WHERE u64 < 7; SELECT count(*) FROM tab WHERE u64_tdigest < 7; +SELECT count(*) FROM tab WHERE u64_minmax < 7; SELECT count(*) FROM tab WHERE u64_count_min < 7; SELECT count(*) FROM tab WHERE u64_uniq < 7; SELECT count(*) FROM tab WHERE u64 < 7.7; SELECT count(*) FROM tab WHERE u64_tdigest < 7.7; +SELECT count(*) FROM tab WHERE u64_minmax < 7.7; SELECT count(*) FROM tab WHERE u64_count_min < 7.7; SELECT count(*) FROM tab WHERE u64_uniq < 7.7; SELECT count(*) FROM tab WHERE u64 < '7'; SELECT count(*) FROM tab WHERE u64_tdigest < '7'; +SELECT count(*) FROM tab WHERE u64_minmax < '7'; SELECT count(*) FROM tab WHERE u64_count_min < '7'; SELECT count(*) FROM tab WHERE u64_uniq < '7'; SELECT count(*) FROM tab WHERE u64 < '7.7'; -- { serverError TYPE_MISMATCH } SELECT count(*) FROM tab WHERE u64_tdigest < '7.7'; -- { serverError TYPE_MISMATCH } +SELECT count(*) FROM tab WHERE u64_minmax < '7.7'; -- { serverError TYPE_MISMATCH } SELECT count(*) FROM tab WHERE u64_count_min < '7.7'; -- { serverError TYPE_MISMATCH } SELECT count(*) FROM tab WHERE u64_uniq < '7.7'; -- { serverError TYPE_MISMATCH } @@ -107,21 +125,25 @@ SELECT 'f64 and ='; SELECT count(*) FROM tab WHERE f64 = 7; SELECT count(*) FROM tab WHERE f64_tdigest = 7; +SELECT count(*) FROM tab WHERE f64_minmax = 7; SELECT count(*) FROM tab WHERE f64_count_min = 7; SELECT count(*) FROM tab WHERE f64_uniq = 7; SELECT count(*) FROM tab WHERE f64 = 7.7; SELECT count(*) FROM tab WHERE f64_tdigest = 7.7; +SELECT count(*) FROM tab WHERE f64_minmax = 7.7; SELECT count(*) FROM tab WHERE f64_count_min = 7.7; SELECT count(*) FROM tab WHERE f64_uniq = 7.7; SELECT count(*) FROM tab WHERE f64 = '7'; SELECT count(*) FROM tab WHERE f64_tdigest = '7'; +SELECT count(*) FROM tab WHERE f64_minmax = '7'; SELECT count(*) FROM tab WHERE f64_count_min = '7'; SELECT count(*) FROM tab WHERE f64_uniq = '7'; SELECT count(*) FROM tab WHERE f64 = '7.7'; SELECT count(*) FROM tab WHERE f64_tdigest = '7.7'; +SELECT count(*) FROM tab WHERE f64_minmax = '7.7'; SELECT count(*) FROM tab WHERE f64_count_min = '7.7'; SELECT count(*) FROM tab WHERE f64_uniq = '7.7'; @@ -129,21 +151,25 @@ SELECT 'f64 and <'; SELECT count(*) FROM tab WHERE f64 < 7; SELECT count(*) FROM tab WHERE f64_tdigest < 7; +SELECT count(*) FROM tab WHERE f64_minmax < 7; SELECT count(*) FROM tab WHERE f64_count_min < 7; SELECT count(*) FROM tab WHERE f64_uniq < 7; SELECT count(*) FROM tab WHERE f64 < 7.7; SELECT count(*) FROM tab WHERE f64_tdigest < 7.7; +SELECT count(*) FROM tab WHERE f64_minmax < 7.7; SELECT count(*) FROM tab WHERE f64_count_min < 7.7; SELECT count(*) FROM tab WHERE f64_uniq < 7.7; SELECT count(*) FROM tab WHERE f64 < '7'; SELECT count(*) FROM tab WHERE f64_tdigest < '7'; +SELECT count(*) FROM tab WHERE f64_minmax < '7'; SELECT count(*) FROM tab WHERE f64_count_min < '7'; SELECT count(*) FROM tab WHERE f64_uniq < '7'; SELECT count(*) FROM tab WHERE f64 < '7.7'; SELECT count(*) FROM tab WHERE f64_tdigest < '7.7'; +SELECT count(*) FROM tab WHERE f64_minmax < '7.7'; SELECT count(*) FROM tab WHERE f64_count_min < '7.7'; SELECT count(*) FROM tab WHERE f64_uniq < '7.7'; @@ -153,11 +179,13 @@ SELECT 'dt and ='; SELECT count(*) FROM tab WHERE dt = '2024-08-08 11:12:13'; SELECT count(*) FROM tab WHERE dt_tdigest = '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_minmax = '2024-08-08 11:12:13'; SELECT count(*) FROM tab WHERE dt_count_min = '2024-08-08 11:12:13'; SELECT count(*) FROM tab WHERE dt_uniq = '2024-08-08 11:12:13'; SELECT count(*) FROM tab WHERE dt = 7; SELECT count(*) FROM tab WHERE dt_tdigest = 7; +SELECT count(*) FROM tab WHERE dt_minmax = 7; SELECT count(*) FROM tab WHERE dt_count_min = 7; SELECT count(*) FROM tab WHERE dt_uniq = 7; @@ -165,11 +193,13 @@ SELECT 'dt and <'; SELECT count(*) FROM tab WHERE dt < '2024-08-08 11:12:13'; SELECT count(*) FROM tab WHERE dt_tdigest < '2024-08-08 11:12:13'; +SELECT count(*) FROM tab WHERE dt_minmax < '2024-08-08 11:12:13'; SELECT count(*) FROM tab WHERE dt_count_min < '2024-08-08 11:12:13'; SELECT count(*) FROM tab WHERE dt_uniq < '2024-08-08 11:12:13'; SELECT count(*) FROM tab WHERE dt < 7; SELECT count(*) FROM tab WHERE dt_tdigest < 7; +SELECT count(*) FROM tab WHERE dt_minmax < 7; SELECT count(*) FROM tab WHERE dt_count_min < 7; SELECT count(*) FROM tab WHERE dt_uniq < 7; @@ -179,21 +209,25 @@ SELECT 'b and ='; SELECT count(*) FROM tab WHERE b = true; SELECT count(*) FROM tab WHERE b_tdigest = true; +SELECT count(*) FROM tab WHERE b_minmax = true; SELECT count(*) FROM tab WHERE b_count_min = true; SELECT count(*) FROM tab WHERE b_uniq = true; SELECT count(*) FROM tab WHERE b = 'true'; SELECT count(*) FROM tab WHERE b_tdigest = 'true'; +SELECT count(*) FROM tab WHERE b_minmax = 'true'; SELECT count(*) FROM tab WHERE b_count_min = 'true'; SELECT count(*) FROM tab WHERE b_uniq = 'true'; SELECT count(*) FROM tab WHERE b = 1; SELECT count(*) FROM tab WHERE b_tdigest = 1; +SELECT count(*) FROM tab WHERE b_minmax = 1; SELECT count(*) FROM tab WHERE b_count_min = 1; SELECT count(*) FROM tab WHERE b_uniq = 1; SELECT count(*) FROM tab WHERE b = 1.1; SELECT count(*) FROM tab WHERE b_tdigest = 1.1; +SELECT count(*) FROM tab WHERE b_minmax = 1.1; SELECT count(*) FROM tab WHERE b_count_min = 1.1; SELECT count(*) FROM tab WHERE b_uniq = 1.1; @@ -203,12 +237,14 @@ SELECT 's and ='; SELECT count(*) FROM tab WHERE s = 7; -- { serverError NO_COMMON_TYPE } -- SELECT count(*) FROM tab WHERE s_tdigest = 7; -- not supported +-- SELECT count(*) FROM tab WHERE s_minmax = 7; -- not supported SELECT count(*) FROM tab WHERE s_count_min = 7; -- { serverError NO_COMMON_TYPE } --- SELECT count(*) FROM tab WHERE s_uniq = 7; -- not supported +SELECT count(*) FROM tab WHERE s_uniq = 7; -- { serverError NO_COMMON_TYPE } SELECT count(*) FROM tab WHERE s = '7'; -- SELECT count(*) FROM tab WHERE s_tdigest = '7'; -- not supported +-- SELECT count(*) FROM tab WHERE s_minmax = '7'; -- not supported SELECT count(*) FROM tab WHERE s_count_min = '7'; --- SELECT count(*) FROM tab WHERE s_uniq = '7'; -- not supported +SELECT count(*) FROM tab WHERE s_uniq = '7'; DROP TABLE tab; From d71afba234f8260df19f66ada151e2f668c54aba Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 21 Aug 2024 18:58:34 +0800 Subject: [PATCH 30/75] Fix tests --- src/Storages/Statistics/StatisticsUniq.cpp | 2 +- tests/queries/0_stateless/02864_statistics_ddl.sql | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/Statistics/StatisticsUniq.cpp b/src/Storages/Statistics/StatisticsUniq.cpp index 21bc5d227ef..5e5b7a67b04 100644 --- a/src/Storages/Statistics/StatisticsUniq.cpp +++ b/src/Storages/Statistics/StatisticsUniq.cpp @@ -56,7 +56,7 @@ void uniqStatisticsValidator(const SingleStatisticsDescription & /*description*/ { DataTypePtr inner_data_type = removeNullable(data_type); inner_data_type = removeLowCardinalityAndNullable(inner_data_type); - if (!inner_data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type)) + if (!inner_data_type->isValueRepresentedByNumber() && !isStringOrFixedString(inner_data_type)) throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName()); } diff --git a/tests/queries/0_stateless/02864_statistics_ddl.sql b/tests/queries/0_stateless/02864_statistics_ddl.sql index 0fc29d0c362..bcaaa9e7b61 100644 --- a/tests/queries/0_stateless/02864_statistics_ddl.sql +++ b/tests/queries/0_stateless/02864_statistics_ddl.sql @@ -47,7 +47,7 @@ CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(tdigest)) Engine = MergeTre CREATE TABLE tab (col UUID STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } CREATE TABLE tab (col IPv6 STATISTICS(tdigest)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } --- uniq requires data_type.isValueRepresentedByInteger +-- uniq requires data_type.isValueRepresentedByInteger or (Fixed)String -- These types work: CREATE TABLE tab (col UInt8 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; CREATE TABLE tab (col UInt256 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; @@ -62,9 +62,9 @@ CREATE TABLE tab (col IPv4 STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple CREATE TABLE tab (col Nullable(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; CREATE TABLE tab (col LowCardinality(UInt8) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; CREATE TABLE tab (col LowCardinality(Nullable(UInt8)) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col String STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; +CREATE TABLE tab (col FixedString(1) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); DROP TABLE tab; -- These types don't work: -CREATE TABLE tab (col String STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } -CREATE TABLE tab (col FixedString(1) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } CREATE TABLE tab (col Array(Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } CREATE TABLE tab (col Tuple(Float64, Float64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } CREATE TABLE tab (col Map(UInt64, UInt64) STATISTICS(uniq)) Engine = MergeTree() ORDER BY tuple(); -- { serverError ILLEGAL_STATISTICS } From d218b04fba8422049ec2a6f73bd646c55cd50c88 Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Wed, 28 Aug 2024 11:52:51 +0000 Subject: [PATCH 31/75] Fix test_role & test_keeper_s3_snapshot --- tests/integration/test_keeper_s3_snapshot/test.py | 2 +- tests/integration/test_role/test.py | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_s3_snapshot/test.py b/tests/integration/test_keeper_s3_snapshot/test.py index 84ffc964621..1e766cb974b 100644 --- a/tests/integration/test_keeper_s3_snapshot/test.py +++ b/tests/integration/test_keeper_s3_snapshot/test.py @@ -92,7 +92,7 @@ def test_s3_upload(started_cluster): # Keeper sends snapshots asynchornously, hence we need to retry. @retry(AssertionError, tries=10, delay=2) def _check_snapshots(): - assert set(get_saved_snapshots()) == set( + assert set(get_saved_snapshots()) >= set( [ "snapshot_50.bin.zstd", "snapshot_100.bin.zstd", diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 225cab975ff..b746af56083 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -629,5 +629,6 @@ def test_roles_cache(): check() instance.query("DROP USER " + ", ".join(users)) - instance.query("DROP ROLE " + ", ".join(roles)) + if roles: + instance.query("DROP ROLE " + ", ".join(roles)) instance.query("DROP TABLE tbl") From d88aa3952d34eeecbca363a1a66df111c5b3e587 Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 29 Aug 2024 10:56:26 +0000 Subject: [PATCH 32/75] Disable alter table add vector similarity index if setting does not enabled --- src/Storages/AlterCommands.cpp | 10 ++++++++++ src/Storages/AlterCommands.h | 2 ++ src/Storages/MergeTree/MergeTreeData.cpp | 5 +++++ ...rbid_add_vector_similarity_index.reference | 0 ...231_forbid_add_vector_similarity_index.sql | 19 +++++++++++++++++++ 5 files changed, 36 insertions(+) create mode 100644 tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.reference create mode 100644 tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index d92d8b59f6e..e0563f9f1c6 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1142,6 +1142,16 @@ bool AlterCommands::hasFullTextIndex(const StorageInMemoryMetadata & metadata) return false; } +bool AlterCommands::hasVectorSimilarityIndex(const StorageInMemoryMetadata & metadata) +{ + for (const auto & index : metadata.secondary_indices) + { + if (index.type == "vector_similarity") + return true; + } + return false; +} + void AlterCommands::apply(StorageInMemoryMetadata & metadata, ContextPtr context) const { if (!prepared) diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index a91bac10214..b2f0f9a6abd 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -237,6 +237,8 @@ public: /// Check if commands have any full-text index static bool hasFullTextIndex(const StorageInMemoryMetadata & metadata); + + static bool hasVectorSimilarityIndex(const StorageInMemoryMetadata & metadata); }; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 94f6d196b99..8b12330c1a4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3230,6 +3230,11 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental full-text index feature is not enabled (turn on setting 'allow_experimental_full_text_index')"); + if (AlterCommands::hasVectorSimilarityIndex(new_metadata) && !settings.allow_experimental_vector_similarity_index) + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "Experimental vector_similarity index feature is not enabled (turn on setting 'allow_experimental_vector_similarity_index')"); + for (const auto & disk : getDisks()) if (!disk->supportsHardLinks() && !commands.isSettingsAlter() && !commands.isCommentAlter()) throw Exception( diff --git a/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.reference b/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql b/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql new file mode 100644 index 00000000000..a1e362e7bd1 --- /dev/null +++ b/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS test_embedding; + +CREATE TABLE test_embedding +( + id UInt32, + embedding Array(Float32), +) +ENGINE = MergeTree +ORDER BY tuple(); + +SET allow_experimental_vector_similarity_index = 0; + +alter table test_embedding add INDEX idx embedding TYPE vector_similarity('hnsw', 'cosineDistance'); -- { serverError SUPPORT_IS_DISABLED } + +SET allow_experimental_vector_similarity_index = 1; + +alter table test_embedding add INDEX idx embedding TYPE vector_similarity('hnsw', 'cosineDistance'); + +DROP TABLE test_embedding; From 0400dcb03eefdb6604d04f5ed3c70c179032f84d Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 29 Aug 2024 12:34:58 +0000 Subject: [PATCH 33/75] no-fastest --- .../0_stateless/03231_forbid_add_vector_similarity_index.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql b/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql index a1e362e7bd1..e91d7c71eac 100644 --- a/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql +++ b/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + DROP TABLE IF EXISTS test_embedding; CREATE TABLE test_embedding From 79c01e717f56132c1af6503b93fc7094dcd47637 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Thu, 29 Aug 2024 23:15:39 +0400 Subject: [PATCH 34/75] rm comment, add docs --- docs/en/sql-reference/statements/alter/partition.md | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 1bb7817364a..11926b2aa08 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -351,7 +351,7 @@ ALTER TABLE mt DELETE IN PARTITION ID '2' WHERE p = 2; You can specify the partition expression in `ALTER ... PARTITION` queries in different ways: - As a value from the `partition` column of the `system.parts` table. For example, `ALTER TABLE visits DETACH PARTITION 201901`. -- Using the keyword `ALL`. It can be used only with DROP/DETACH/ATTACH. For example, `ALTER TABLE visits ATTACH PARTITION ALL`. +- Using the keyword `ALL`. It can be used only with DROP/DETACH/ATTACH/ATTACH FROM. For example, `ALTER TABLE visits ATTACH PARTITION ALL`. - As a tuple of expressions or constants that matches (in types) the table partitioning keys tuple. In the case of a single element partitioning key, the expression should be wrapped in the `tuple (...)` function. For example, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. - Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`. - In the [ALTER ATTACH PART](#attach-partitionpart) and [DROP DETACHED PART](#drop-detached-partitionpart) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](/docs/en/operations/system-tables/detached_parts.md/#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 4915bf9f366..0688b05e2ad 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8240,7 +8240,6 @@ std::unique_ptr StorageReplicatedMergeTree::rep part_checksums.emplace_back(hash_hex); } - //ReplicatedMergeTreeLogEntryData entry; auto entry = std::make_unique(); { auto src_table_id = src_data.getStorageID(); From 6ad8e5558a99b43e3452c057346b9c44e8e27517 Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 30 Aug 2024 07:25:25 +0000 Subject: [PATCH 35/75] Fix typo --- src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index ae183d74782..58892d0dbf2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -195,7 +195,7 @@ void MergeTreeIndexGranuleVectorSimilarity::serializeBinary(WriteBuffer & ostr) LOG_TRACE(logger, "Start writing vector similarity index"); if (empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty minmax index {}", backQuote(index_name)); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty vector similarity index {}", backQuote(index_name)); writeIntBinary(FILE_FORMAT_VERSION, ostr); From c5b92413cac091c4e28acb5160b89fcabc37a853 Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 30 Aug 2024 08:28:31 +0000 Subject: [PATCH 36/75] Fix vector similarity index does not work for cosineDistance --- src/Storages/MergeTree/VectorSimilarityCondition.cpp | 2 ++ src/Storages/MergeTree/VectorSimilarityCondition.h | 3 ++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/VectorSimilarityCondition.cpp b/src/Storages/MergeTree/VectorSimilarityCondition.cpp index c8f33857640..251cdde65ab 100644 --- a/src/Storages/MergeTree/VectorSimilarityCondition.cpp +++ b/src/Storages/MergeTree/VectorSimilarityCondition.cpp @@ -44,6 +44,8 @@ VectorSimilarityCondition::Info::DistanceFunction stringToDistanceFunction(std:: { if (distance_function == "L2Distance") return VectorSimilarityCondition::Info::DistanceFunction::L2; + else if (distance_function == "cosineDistance") + return VectorSimilarityCondition::Info::DistanceFunction::Cosine; else return VectorSimilarityCondition::Info::DistanceFunction::Unknown; } diff --git a/src/Storages/MergeTree/VectorSimilarityCondition.h b/src/Storages/MergeTree/VectorSimilarityCondition.h index 2380f8f46b0..e2946222f49 100644 --- a/src/Storages/MergeTree/VectorSimilarityCondition.h +++ b/src/Storages/MergeTree/VectorSimilarityCondition.h @@ -57,7 +57,8 @@ public: enum class DistanceFunction : uint8_t { Unknown, - L2 + L2, + Cosine }; std::vector reference_vector; From af7adfe4b2f8e9b3b7354314a91d19a19b7955f9 Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 30 Aug 2024 09:03:23 +0000 Subject: [PATCH 37/75] Remove unused code --- src/Storages/MergeTree/VectorSimilarityCondition.h | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/VectorSimilarityCondition.h b/src/Storages/MergeTree/VectorSimilarityCondition.h index e2946222f49..83ae1e19bfb 100644 --- a/src/Storages/MergeTree/VectorSimilarityCondition.h +++ b/src/Storages/MergeTree/VectorSimilarityCondition.h @@ -143,6 +143,7 @@ private: void traverseOrderByAST(const ASTPtr & node, RPN & rpn); /// Returns true and stores ANNExpr if the query has valid WHERE section + /// TODO NOT implemented, WHERE does not supported. static bool matchRPNWhere(RPN & rpn, Info & info); /// Returns true and stores ANNExpr if the query has valid ORDERBY section @@ -151,9 +152,6 @@ private: /// Returns true and stores Length if we have valid LIMIT clause in query static bool matchRPNLimit(RPNElement & rpn, UInt64 & limit); - /// Matches dist function, reference vector, column name - static bool matchMainParts(RPN::iterator & iter, const RPN::iterator & end, Info & info); - /// Gets float or int from AST node static float getFloatOrIntLiteralOrPanic(const RPN::iterator& iter); From 81d0a04ecbcb9ad95a875c94467ab588bc5eeb7c Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Sun, 1 Sep 2024 13:53:56 +0000 Subject: [PATCH 38/75] Added restart of node1 for running multiple iterations of test --- tests/integration/test_keeper_s3_snapshot/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_keeper_s3_snapshot/test.py b/tests/integration/test_keeper_s3_snapshot/test.py index 1e766cb974b..5e2545ae0a7 100644 --- a/tests/integration/test_keeper_s3_snapshot/test.py +++ b/tests/integration/test_keeper_s3_snapshot/test.py @@ -2,6 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster from time import sleep from retry import retry +import helpers.keeper_utils as keeper_utils from kazoo.client import KazooClient @@ -125,3 +126,5 @@ def test_s3_upload(started_cluster): ) destroy_zk_client(node2_zk) + node1.start_clickhouse() # for next iteration + keeper_utils.wait_until_connected(cluster, node1) From 4f16797cd127a4dff7169908890b0010d82027ae Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Mon, 2 Sep 2024 00:35:59 +0000 Subject: [PATCH 39/75] Formatting --- tests/integration/test_keeper_s3_snapshot/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_s3_snapshot/test.py b/tests/integration/test_keeper_s3_snapshot/test.py index 5e2545ae0a7..f47a26a77ad 100644 --- a/tests/integration/test_keeper_s3_snapshot/test.py +++ b/tests/integration/test_keeper_s3_snapshot/test.py @@ -126,5 +126,5 @@ def test_s3_upload(started_cluster): ) destroy_zk_client(node2_zk) - node1.start_clickhouse() # for next iteration + node1.start_clickhouse() # for next iteration keeper_utils.wait_until_connected(cluster, node1) From d65b298268c35b0dd87ddc252e5cae88644c74e4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 2 Sep 2024 18:08:46 +0200 Subject: [PATCH 40/75] Update src/Storages/StorageReplicatedMergeTree.cpp --- src/Storages/StorageReplicatedMergeTree.cpp | 27 ++++++--------------- 1 file changed, 7 insertions(+), 20 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f884c25b071..def4d79b5c4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8058,13 +8058,11 @@ void StorageReplicatedMergeTree::replacePartitionFrom( const bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; - try + std::unique_ptr entries[partitions.size()]; + size_t idx = 0; + for (const auto & partition_id : partitions) { - std::unique_ptr entries[partitions.size()]; - size_t idx = 0; - for (const auto & partition_id : partitions) - { - entries[idx] = replacePartitionFromImpl(watch, + entries[idx] = replacePartitionFromImpl(watch, profile_events_scope, metadata_snapshot, src_data, @@ -8074,22 +8072,11 @@ void StorageReplicatedMergeTree::replacePartitionFrom( zero_copy_enabled, storage_settings_ptr->always_use_copy_instead_of_hardlinks, query_context); - ++idx; - } - - for (const auto & entry : entries) - waitForLogEntryToBeProcessedIfNecessary(*entry, query_context); - - lock2.reset(); - lock1.reset(); + ++idx; } - catch (...) - { - lock2.reset(); - lock1.reset(); - throw; - } + for (const auto & entry : entries) + waitForLogEntryToBeProcessedIfNecessary(*entry, query_context); } std::unique_ptr StorageReplicatedMergeTree::replacePartitionFromImpl( From 45556278c917a0053e902ae9ebcee60f98062fe5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 3 Sep 2024 11:06:28 +0000 Subject: [PATCH 41/75] Fix possible timeouts in `sh` tests with tsan, att. 2 --- src/Client/ClientBase.cpp | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index e34e263beb5..edcb98be90f 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1896,6 +1896,25 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin /// Temporarily apply query settings to context. std::optional old_settings; SCOPE_EXIT_SAFE({ + try + { + /// We need to park ParallelFormating threads, + /// because they can use settings from global context + /// and it can lead to data race with `setSettings` + if (output_format) + { + output_format->finalize(); + output_format.reset(); + } + } + catch (...) + { + if (!have_error) + { + client_exception = std::make_unique(getCurrentExceptionMessageAndPattern(print_stack_trace), getCurrentExceptionCode()); + have_error = true; + } + } if (old_settings) client_context->setSettings(*old_settings); }); From 239cdf908b0535efb58158fc1bc2c5aa8004e566 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 3 Sep 2024 23:35:11 +0200 Subject: [PATCH 42/75] Update src/Storages/MergeTree/MergeTreeData.cpp --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 54045c66af5..d56414edbeb 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5009,7 +5009,7 @@ void MergeTreeData::checkAlterPartitionIsPossible( const auto * partition_ast = command.partition->as(); if (partition_ast && partition_ast->all) { - if (command.type != PartitionCommand::DROP_PARTITION && command.type != PartitionCommand::ATTACH_PARTITION && (command.type == PartitionCommand::REPLACE_PARTITION && command.replace)) + if (command.type != PartitionCommand::DROP_PARTITION && command.type != PartitionCommand::ATTACH_PARTITION && !(command.type == PartitionCommand::REPLACE_PARTITION && !command.replace)) throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH/ATTACH PARTITION ALL currently"); } else From 0b1a0999e338325c2e5cd9ec25e9a5ecf3472d16 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 4 Sep 2024 08:29:55 +0000 Subject: [PATCH 43/75] Some fixups --- src/Interpreters/InterpreterCreateQuery.cpp | 4 +-- src/Storages/AlterCommands.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 5 ++- ...tor_search_experimental_setting.reference} | 0 ...354_vector_search_experimental_setting.sql | 32 +++++++++++++++++++ ...231_forbid_add_vector_similarity_index.sql | 21 ------------ 6 files changed, 37 insertions(+), 26 deletions(-) rename tests/queries/0_stateless/{03231_forbid_add_vector_similarity_index.reference => 02354_vector_search_experimental_setting.reference} (100%) create mode 100644 tests/queries/0_stateless/02354_vector_search_experimental_setting.sql delete mode 100644 tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index e9f40bdbaf5..c265154c9dd 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -781,14 +781,14 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti const auto & settings = getContext()->getSettingsRef(); if (index_desc.type == FULL_TEXT_INDEX_NAME && !settings.allow_experimental_full_text_index) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental full-text index feature is not enabled (the setting 'allow_experimental_full_text_index')"); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental full-text index feature is disabled. Turn on setting 'allow_experimental_full_text_index'"); /// ---- /// Temporary check during a transition period. Please remove at the end of 2024. if (index_desc.type == INVERTED_INDEX_NAME && !settings.allow_experimental_inverted_index) throw Exception(ErrorCodes::ILLEGAL_INDEX, "Please use index type 'full_text' instead of 'inverted'"); /// ---- if (index_desc.type == "vector_similarity" && !settings.allow_experimental_vector_similarity_index) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Vector similarity index is disabled. Turn on allow_experimental_vector_similarity_index"); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental vector similarity index is disabled. Turn on setting 'allow_experimental_vector_similarity_index'"); properties.indices.push_back(index_desc); } diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index b2f0f9a6abd..c4c792e7dec 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -238,6 +238,7 @@ public: /// Check if commands have any full-text index static bool hasFullTextIndex(const StorageInMemoryMetadata & metadata); + /// Check if commands have any vector similarity index static bool hasVectorSimilarityIndex(const StorageInMemoryMetadata & metadata); }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8b12330c1a4..ac39831b3df 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3231,9 +3231,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context "Experimental full-text index feature is not enabled (turn on setting 'allow_experimental_full_text_index')"); if (AlterCommands::hasVectorSimilarityIndex(new_metadata) && !settings.allow_experimental_vector_similarity_index) - throw Exception( - ErrorCodes::SUPPORT_IS_DISABLED, - "Experimental vector_similarity index feature is not enabled (turn on setting 'allow_experimental_vector_similarity_index')"); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Experimental vector similarity index is disabled (turn on setting 'allow_experimental_vector_similarity_index')"); for (const auto & disk : getDisks()) if (!disk->supportsHardLinks() && !commands.isSettingsAlter() && !commands.isCommentAlter()) diff --git a/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.reference b/tests/queries/0_stateless/02354_vector_search_experimental_setting.reference similarity index 100% rename from tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.reference rename to tests/queries/0_stateless/02354_vector_search_experimental_setting.reference diff --git a/tests/queries/0_stateless/02354_vector_search_experimental_setting.sql b/tests/queries/0_stateless/02354_vector_search_experimental_setting.sql new file mode 100644 index 00000000000..cce838f8e02 --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_experimental_setting.sql @@ -0,0 +1,32 @@ +-- Tags: no-fasttest, no-ordinary-database + +-- Tests that CREATE TABLE and ADD INDEX respect setting 'allow_experimental_vector_similarity_index'. + +DROP TABLE IF EXISTS tab; + +-- Test CREATE TABLE + +SET allow_experimental_vector_similarity_index = 0; +CREATE TABLE tab (id UInt32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY tuple(); -- { serverError SUPPORT_IS_DISABLED } + +SET allow_experimental_vector_similarity_index = 1; +CREATE TABLE tab (id UInt32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY tuple(); +DROP TABLE tab; + +-- Test ADD INDEX + +CREATE TABLE tab (id UInt32, vec Array(Float32)) ENGINE = MergeTree ORDER BY tuple(); + +SET allow_experimental_vector_similarity_index = 0; +ALTER TABLE tab ADD INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance'); -- { serverError SUPPORT_IS_DISABLED } + +SET allow_experimental_vector_similarity_index = 1; +ALTER TABLE tab ADD INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance'); + +-- Other index DDL must work regardless of the setting +SET allow_experimental_vector_similarity_index = 0; +ALTER TABLE tab MATERIALIZE INDEX idx; +-- ALTER TABLE tab CLEAR INDEX idx; -- <-- Should work but doesn't w/o enabled setting. Unexpected but not terrible. +ALTER TABLE tab DROP INDEX idx; + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql b/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql deleted file mode 100644 index e91d7c71eac..00000000000 --- a/tests/queries/0_stateless/03231_forbid_add_vector_similarity_index.sql +++ /dev/null @@ -1,21 +0,0 @@ --- Tags: no-fasttest - -DROP TABLE IF EXISTS test_embedding; - -CREATE TABLE test_embedding -( - id UInt32, - embedding Array(Float32), -) -ENGINE = MergeTree -ORDER BY tuple(); - -SET allow_experimental_vector_similarity_index = 0; - -alter table test_embedding add INDEX idx embedding TYPE vector_similarity('hnsw', 'cosineDistance'); -- { serverError SUPPORT_IS_DISABLED } - -SET allow_experimental_vector_similarity_index = 1; - -alter table test_embedding add INDEX idx embedding TYPE vector_similarity('hnsw', 'cosineDistance'); - -DROP TABLE test_embedding; From 2620325b303b9318fdd347372f66faa131ca9401 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 4 Sep 2024 08:59:52 +0000 Subject: [PATCH 44/75] Minor fixups + add a tests --- .../MergeTree/VectorSimilarityCondition.cpp | 4 ++-- .../MergeTree/VectorSimilarityCondition.h | 6 +----- .../02354_vector_search_queries.reference | 15 +++++++++++++++ .../0_stateless/02354_vector_search_queries.sql | 7 +++++++ 4 files changed, 25 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/VectorSimilarityCondition.cpp b/src/Storages/MergeTree/VectorSimilarityCondition.cpp index 251cdde65ab..641b0037e7b 100644 --- a/src/Storages/MergeTree/VectorSimilarityCondition.cpp +++ b/src/Storages/MergeTree/VectorSimilarityCondition.cpp @@ -40,7 +40,7 @@ void extractReferenceVectorFromLiteral(std::vector & reference_vector, } } -VectorSimilarityCondition::Info::DistanceFunction stringToDistanceFunction(std::string_view distance_function) +VectorSimilarityCondition::Info::DistanceFunction stringToDistanceFunction(const String & distance_function) { if (distance_function == "L2Distance") return VectorSimilarityCondition::Info::DistanceFunction::L2; @@ -59,7 +59,7 @@ VectorSimilarityCondition::VectorSimilarityCondition(const SelectQueryInfo & que , index_is_useful(checkQueryStructure(query_info)) {} -bool VectorSimilarityCondition::alwaysUnknownOrTrue(String distance_function) const +bool VectorSimilarityCondition::alwaysUnknownOrTrue(const String & distance_function) const { if (!index_is_useful) return true; /// query isn't supported diff --git a/src/Storages/MergeTree/VectorSimilarityCondition.h b/src/Storages/MergeTree/VectorSimilarityCondition.h index 83ae1e19bfb..2e9e06a31d0 100644 --- a/src/Storages/MergeTree/VectorSimilarityCondition.h +++ b/src/Storages/MergeTree/VectorSimilarityCondition.h @@ -69,7 +69,7 @@ public: }; /// Returns false if query can be speeded up by an ANN index, true otherwise. - bool alwaysUnknownOrTrue(String distance_function) const; + bool alwaysUnknownOrTrue(const String & distance_function) const; std::vector getReferenceVector() const; size_t getDimensions() const; @@ -142,10 +142,6 @@ private: /// Traverses the AST of ORDERBY section void traverseOrderByAST(const ASTPtr & node, RPN & rpn); - /// Returns true and stores ANNExpr if the query has valid WHERE section - /// TODO NOT implemented, WHERE does not supported. - static bool matchRPNWhere(RPN & rpn, Info & info); - /// Returns true and stores ANNExpr if the query has valid ORDERBY section static bool matchRPNOrderBy(RPN & rpn, Info & info); diff --git a/tests/queries/0_stateless/02354_vector_search_queries.reference b/tests/queries/0_stateless/02354_vector_search_queries.reference index faff306ef60..e42f91d05dc 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.reference +++ b/tests/queries/0_stateless/02354_vector_search_queries.reference @@ -41,6 +41,21 @@ Special cases 6 [1,9.3] 0.005731362878640178 1 [2,3.2] 0.15200169244542905 7 [5.5,4.7] 0.3503476876550442 +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: idx + Description: vector_similarity GRANULARITY 2 + Parts: 1/1 + Granules: 2/4 -- Setting "max_limit_for_ann_queries" Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) diff --git a/tests/queries/0_stateless/02354_vector_search_queries.sql b/tests/queries/0_stateless/02354_vector_search_queries.sql index 17939992165..8769e5c56bb 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.sql +++ b/tests/queries/0_stateless/02354_vector_search_queries.sql @@ -63,6 +63,13 @@ FROM tab ORDER BY cosineDistance(vec, reference_vec) LIMIT 3; +EXPLAIN indexes = 1 +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, cosineDistance(vec, reference_vec) +FROM tab +ORDER BY cosineDistance(vec, reference_vec) +LIMIT 3; + SELECT '-- Setting "max_limit_for_ann_queries"'; EXPLAIN indexes=1 WITH [0.0, 2.0] as reference_vec From 10d19b47b7e9f0f6d066e324b35e95d12bd8f2e9 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 4 Sep 2024 12:18:44 +0000 Subject: [PATCH 45/75] fix dropping of file cache in CHECK query in case of enabled transactions --- .../MergeTree/DataPartStorageOnDiskFull.cpp | 16 ++++++---------- .../MergeTree/DataPartStorageOnDiskFull.h | 2 +- src/Storages/MergeTree/IDataPartStorage.h | 2 +- src/Storages/MergeTree/checkDataPart.cpp | 14 +++----------- 4 files changed, 11 insertions(+), 23 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskFull.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskFull.cpp index bfe656111b0..bc2bb020836 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskFull.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskFull.cpp @@ -95,22 +95,18 @@ UInt32 DataPartStorageOnDiskFull::getRefCount(const String & file_name) const return volume->getDisk()->getRefCount(fs::path(root_path) / part_dir / file_name); } -std::string DataPartStorageOnDiskFull::getRemotePath(const std::string & file_name, bool if_exists) const +std::vector DataPartStorageOnDiskFull::getRemotePaths(const std::string & file_name) const { const std::string path = fs::path(root_path) / part_dir / file_name; auto objects = volume->getDisk()->getStorageObjects(path); - if (objects.empty() && if_exists) - return ""; + std::vector remote_paths; + remote_paths.reserve(objects.size()); - if (objects.size() != 1) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "One file must be mapped to one object on blob storage by path {} in MergeTree tables, have {}.", - path, objects.size()); - } + for (const auto & object : objects) + remote_paths.push_back(object.remote_path); - return objects[0].remote_path; + return remote_paths; } String DataPartStorageOnDiskFull::getUniqueId() const diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskFull.h b/src/Storages/MergeTree/DataPartStorageOnDiskFull.h index ba787809b63..c76ec41ccb0 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskFull.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskFull.h @@ -23,7 +23,7 @@ public: Poco::Timestamp getFileLastModified(const String & file_name) const override; size_t getFileSize(const std::string & file_name) const override; UInt32 getRefCount(const std::string & file_name) const override; - std::string getRemotePath(const std::string & file_name, bool if_exists) const override; + std::vector getRemotePaths(const std::string & file_name) const override; String getUniqueId() const override; std::unique_ptr readFile( diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index f6320a7e1e4..01670fc5c3e 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -126,7 +126,7 @@ public: virtual UInt32 getRefCount(const std::string & file_name) const = 0; /// Get path on remote filesystem from file name on local filesystem. - virtual std::string getRemotePath(const std::string & file_name, bool if_exists) const = 0; + virtual std::vector getRemotePaths(const std::string & file_name) const = 0; virtual UInt64 calculateTotalSizeOnDisk() const = 0; diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 3a22daa0011..0365f875409 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -391,17 +391,9 @@ IMergeTreeDataPart::Checksums checkDataPart( auto file_name = it->name(); if (!data_part_storage.isDirectory(file_name)) { - const bool is_projection_part = data_part->isProjectionPart(); - auto remote_path = data_part_storage.getRemotePath(file_name, /* if_exists */is_projection_part); - if (remote_path.empty()) - { - chassert(is_projection_part); - throw Exception( - ErrorCodes::BROKEN_PROJECTION, - "Remote path for {} does not exist for projection path. Projection {} is broken", - file_name, data_part->name); - } - cache.removePathIfExists(remote_path, FileCache::getCommonUser().user_id); + auto remote_paths = data_part_storage.getRemotePaths(file_name); + for (const auto & remote_path : remote_paths) + cache.removePathIfExists(remote_path, FileCache::getCommonUser().user_id); } } From 51464be3e1b53b839fd142ee6e7eed7de64f6c2e Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Wed, 4 Sep 2024 13:26:09 +0000 Subject: [PATCH 46/75] Set use_cluster=false --- .../test_keeper_s3_snapshot/configs/keeper_config1.xml | 1 + .../test_keeper_s3_snapshot/configs/keeper_config2.xml | 1 + .../test_keeper_s3_snapshot/configs/keeper_config3.xml | 1 + 3 files changed, 3 insertions(+) diff --git a/tests/integration/test_keeper_s3_snapshot/configs/keeper_config1.xml b/tests/integration/test_keeper_s3_snapshot/configs/keeper_config1.xml index 8459ea3e068..6af17946eec 100644 --- a/tests/integration/test_keeper_s3_snapshot/configs/keeper_config1.xml +++ b/tests/integration/test_keeper_s3_snapshot/configs/keeper_config1.xml @@ -5,6 +5,7 @@ minio minio123 + false 9181 1 /var/lib/clickhouse/coordination/log diff --git a/tests/integration/test_keeper_s3_snapshot/configs/keeper_config2.xml b/tests/integration/test_keeper_s3_snapshot/configs/keeper_config2.xml index dfe73628f66..25f2b0de812 100644 --- a/tests/integration/test_keeper_s3_snapshot/configs/keeper_config2.xml +++ b/tests/integration/test_keeper_s3_snapshot/configs/keeper_config2.xml @@ -5,6 +5,7 @@ minio minio123 + false 9181 2 /var/lib/clickhouse/coordination/log diff --git a/tests/integration/test_keeper_s3_snapshot/configs/keeper_config3.xml b/tests/integration/test_keeper_s3_snapshot/configs/keeper_config3.xml index 948d9527718..e274b5184f1 100644 --- a/tests/integration/test_keeper_s3_snapshot/configs/keeper_config3.xml +++ b/tests/integration/test_keeper_s3_snapshot/configs/keeper_config3.xml @@ -5,6 +5,7 @@ minio minio123 + false 9181 3 /var/lib/clickhouse/coordination/log From d38551a1788d2ebc1b7970cc912f894f87f3db1c Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 4 Sep 2024 13:29:45 +0000 Subject: [PATCH 47/75] resetOutput --- src/Client/ClientBase.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index edcb98be90f..445500a5f0f 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1901,11 +1901,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin /// We need to park ParallelFormating threads, /// because they can use settings from global context /// and it can lead to data race with `setSettings` - if (output_format) - { - output_format->finalize(); - output_format.reset(); - } + resetOutput(); } catch (...) { From d2ae77936887dcce2a5aaf88c06a6b6e7a558777 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 4 Sep 2024 17:18:16 +0200 Subject: [PATCH 48/75] Update 00626_replace_partition_from_table_zookeeper.sh --- .../0_stateless/00626_replace_partition_from_table_zookeeper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index 9a77f0c7a67..0cadfb6194c 100755 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-object-storage +# Tags: zookeeper, no-object-storage, long # Because REPLACE PARTITION does not forces immediate removal of replaced data parts from local filesystem # (it tries to do it as quick as possible, but it still performed in separate thread asynchronously) From ef177d2c9fff4e3594f0b11e3e72de3cb5d09055 Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Thu, 5 Sep 2024 11:37:57 +0000 Subject: [PATCH 49/75] Proper cleanup & restart for subsequent iterations --- .../test_keeper_s3_snapshot/test.py | 44 ++++++++++++++++++- 1 file changed, 42 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_s3_snapshot/test.py b/tests/integration/test_keeper_s3_snapshot/test.py index f47a26a77ad..bcb9c7cbd13 100644 --- a/tests/integration/test_keeper_s3_snapshot/test.py +++ b/tests/integration/test_keeper_s3_snapshot/test.py @@ -2,7 +2,9 @@ import pytest from helpers.cluster import ClickHouseCluster from time import sleep from retry import retry +from multiprocessing.dummy import Pool import helpers.keeper_utils as keeper_utils +from minio.deleteobjects import DeleteObject from kazoo.client import KazooClient @@ -76,7 +78,18 @@ def wait_node(node): raise Exception("Can't wait node", node.name, "to become ready") +def delete_keeper_snapshots_logs(nodex): + nodex.exec_in_container( + [ + "bash", + "-c", + "rm -rf /var/lib/clickhouse/coordination/log /var/lib/clickhouse/coordination/snapshots" + ] + ) + + def test_s3_upload(started_cluster): + node1_zk = get_fake_zk(node1.name) # we defined in configs snapshot_distance as 50 @@ -90,10 +103,17 @@ def test_s3_upload(started_cluster): for obj in list(cluster.minio_client.list_objects("snapshots")) ] + def delete_s3_snapshots(): + snapshots = cluster.minio_client.list_objects("snapshots") + for s in snapshots: + cluster.minio_client.remove_object( + "snapshots", + s.object_name) + # Keeper sends snapshots asynchornously, hence we need to retry. @retry(AssertionError, tries=10, delay=2) def _check_snapshots(): - assert set(get_saved_snapshots()) >= set( + assert set(get_saved_snapshots()) == set( [ "snapshot_50.bin.zstd", "snapshot_100.bin.zstd", @@ -126,5 +146,25 @@ def test_s3_upload(started_cluster): ) destroy_zk_client(node2_zk) - node1.start_clickhouse() # for next iteration + node2.stop_clickhouse() + delete_keeper_snapshots_logs(node2) + node3.stop_clickhouse() + delete_keeper_snapshots_logs(node3) + delete_keeper_snapshots_logs(node1) + p = Pool(3) + waiters = [] + def start_clickhouse(node): + node.start_clickhouse() + + waiters.append(p.apply_async(start_clickhouse, args=(node1,))) + waiters.append(p.apply_async(start_clickhouse, args=(node2,))) + waiters.append(p.apply_async(start_clickhouse, args=(node3,))) + + delete_s3_snapshots() # for next iteration + + for waiter in waiters: + waiter.wait() + keeper_utils.wait_until_connected(cluster, node1) + keeper_utils.wait_until_connected(cluster, node2) + keeper_utils.wait_until_connected(cluster, node3) From 57f80473cc74a826bd11f96f9741f762ce639f82 Mon Sep 17 00:00:00 2001 From: Christoph Wurm Date: Thu, 5 Sep 2024 13:42:09 +0100 Subject: [PATCH 50/75] Docs: Add info on lightweight_deletes_sync --- docs/en/sql-reference/statements/delete.md | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index 8ddb5840f2e..ded0d7dacac 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -24,9 +24,15 @@ DELETE FROM hits WHERE Title LIKE '%hello%'; ## Lightweight `DELETE` does not delete data immediately -Lightweight `DELETE` is implemented as a [mutation](/en/sql-reference/statements/alter#mutations), which is executed asynchronously in the background by default. The statement is going to return almost immediately, but the data can still be visible to queries until the mutation is finished. +Lightweight `DELETE` is implemented as a [mutation](/en/sql-reference/statements/alter#mutations) that marks rows as deleted but does not immediately physically delete them. -The mutation marks rows as deleted, and at that point, they will no longer show up in query results. It does not physically delete the data, this will happen during the next merge. As a result, it is possible that for an unspecified period, data is not actually deleted from storage and is only marked as deleted. +By default, `DELETE` statements wait until marking the rows as deleted is completed before returning. This can take a long time if the amount of data is large. Alternatively, you can run it asynchronously in the background using the setting [`lightweight_deletes_sync`](/en/operations/settings/settings#lightweight_deletes_sync). If disabled, the `DELETE` statement is going to return immediately, but the data can still be visible to queries until the background mutation is finished. + +:::note +Before version 24.4, lightweight deletes were asynchronous by default. +::: + +The mutation does not physically delete the rows that have been marked as deleted, this will only happen during the next merge. As a result, it is possible that for an unspecified period, data is not actually deleted from storage and is only marked as deleted. If you need to guarantee that your data is deleted from storage in a predictable time, consider using the table setting [`min_age_to_force_merge_seconds`](https://clickhouse.com/docs/en/operations/settings/merge-tree-settings#min_age_to_force_merge_seconds). Or you can use the [ALTER TABLE ... DELETE](/en/sql-reference/statements/alter/delete) command. Note that deleting data using `ALTER TABLE ... DELETE` may consume significant resources as it recreates all affected parts. From 4d87e349a04173cff7dd0814a6425a1865cf74a8 Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Thu, 5 Sep 2024 13:57:46 +0000 Subject: [PATCH 51/75] Python formatting --- tests/integration/test_keeper_s3_snapshot/test.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_keeper_s3_snapshot/test.py b/tests/integration/test_keeper_s3_snapshot/test.py index bcb9c7cbd13..b6c25305aef 100644 --- a/tests/integration/test_keeper_s3_snapshot/test.py +++ b/tests/integration/test_keeper_s3_snapshot/test.py @@ -83,7 +83,7 @@ def delete_keeper_snapshots_logs(nodex): [ "bash", "-c", - "rm -rf /var/lib/clickhouse/coordination/log /var/lib/clickhouse/coordination/snapshots" + "rm -rf /var/lib/clickhouse/coordination/log /var/lib/clickhouse/coordination/snapshots", ] ) @@ -106,9 +106,7 @@ def test_s3_upload(started_cluster): def delete_s3_snapshots(): snapshots = cluster.minio_client.list_objects("snapshots") for s in snapshots: - cluster.minio_client.remove_object( - "snapshots", - s.object_name) + cluster.minio_client.remove_object("snapshots", s.object_name) # Keeper sends snapshots asynchornously, hence we need to retry. @retry(AssertionError, tries=10, delay=2) @@ -153,6 +151,7 @@ def test_s3_upload(started_cluster): delete_keeper_snapshots_logs(node1) p = Pool(3) waiters = [] + def start_clickhouse(node): node.start_clickhouse() From 18a6b970ebd512568f69203c169c69928e89e15b Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 5 Sep 2024 13:58:21 +0000 Subject: [PATCH 52/75] Improve logical error trace This will provide meaningful information whenever the issue happens again. --- src/Client/ConnectionPoolWithFailover.h | 2 +- src/Common/PoolWithFailoverBase.h | 8 ++++++++ src/Storages/Distributed/DistributedAsyncInsertBatch.cpp | 9 ++------- .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 4 +--- src/Storages/Distributed/DistributedSink.cpp | 4 +--- 5 files changed, 13 insertions(+), 14 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index a2dc188eb7d..308644ebbdb 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -42,7 +42,7 @@ public: size_t max_error_cap = DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT); using Entry = IConnectionPool::Entry; - using PoolWithFailoverBase::isTryResultInvalid; + using PoolWithFailoverBase::checkTryResultIsValid; /** Allocates connection to work. */ Entry get(const ConnectionTimeouts & timeouts) override; diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index c44ab7df53a..53a746c316e 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -122,6 +122,14 @@ public: return result.entry.isNull() || !result.is_usable || (skip_read_only_replicas && result.is_readonly); } + void checkTryResultIsValid(const TryResult & result, bool skip_read_only_replicas) const + { + if (isTryResultInvalid(result, skip_read_only_replicas)) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, + "Got an invalid connection result: entry.isNull {}, is_usable {}, is_up_to_date {}, delay {}, is_readonly {}, skip_read_only_replicas {}", + result.entry.isNull(), result.is_usable, result.is_up_to_date, result.delay, result.is_readonly, skip_read_only_replicas); + } + size_t getPoolSize() const { return nested_pools.size(); } protected: diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index 2cf69b9f6b7..2db2bdf3981 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -28,7 +28,6 @@ namespace ErrorCodes extern const int TOO_MANY_PARTITIONS; extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES; extern const int ARGUMENT_OUT_OF_BOUND; - extern const int LOGICAL_ERROR; } /// Can the batch be split and send files from batch one-by-one instead? @@ -244,9 +243,7 @@ void DistributedAsyncInsertBatch::sendBatch(const SettingsChanges & settings_cha auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); auto results = parent.pool->getManyCheckedForInsert(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName()); auto result = results.front(); - if (parent.pool->isTryResultInvalid(result, insert_settings.distributed_insert_skip_read_only_replicas)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got an invalid connection result"); - + parent.pool->checkTryResultIsValid(result, insert_settings.distributed_insert_skip_read_only_replicas); connection = std::move(result.entry); compression_expected = connection->getCompression() == Protocol::Compression::Enable; @@ -306,9 +303,7 @@ void DistributedAsyncInsertBatch::sendSeparateFiles(const SettingsChanges & sett auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); auto results = parent.pool->getManyCheckedForInsert(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName()); auto result = results.front(); - if (parent.pool->isTryResultInvalid(result, insert_settings.distributed_insert_skip_read_only_replicas)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got an invalid connection result"); - + parent.pool->checkTryResultIsValid(result, insert_settings.distributed_insert_skip_read_only_replicas); auto connection = std::move(result.entry); bool compression_expected = connection->getCompression() == Protocol::Compression::Enable; diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 7616b384860..2400de4c07c 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -416,9 +416,7 @@ void DistributedAsyncInsertDirectoryQueue::processFile(std::string & file_path, auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); auto results = pool->getManyCheckedForInsert(timeouts, insert_settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); auto result = results.front(); - if (pool->isTryResultInvalid(result, insert_settings.distributed_insert_skip_read_only_replicas)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got an invalid connection result"); - + pool->checkTryResultIsValid(result, insert_settings.distributed_insert_skip_read_only_replicas); auto connection = std::move(result.entry); LOG_DEBUG(log, "Sending `{}` to {} ({} rows, {} bytes)", diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index e3e73e42096..daccbb78f2c 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -378,9 +378,7 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si /// (anyway fallback_to_stale_replicas_for_distributed_queries=true by default) auto results = shard_info.pool->getManyCheckedForInsert(timeouts, settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); auto result = results.front(); - if (shard_info.pool->isTryResultInvalid(result, settings.distributed_insert_skip_read_only_replicas)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got an invalid connection result"); - + shard_info.pool->checkTryResultIsValid(result, settings.distributed_insert_skip_read_only_replicas); job.connection_entry = std::move(result.entry); } else From 6621365526c4fc5072c254522a22088192f785b9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 5 Sep 2024 16:40:44 +0200 Subject: [PATCH 53/75] Update 01287_max_execution_speed.sql --- tests/queries/0_stateless/01287_max_execution_speed.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01287_max_execution_speed.sql b/tests/queries/0_stateless/01287_max_execution_speed.sql index 35bc4e02d38..0d132999481 100644 --- a/tests/queries/0_stateless/01287_max_execution_speed.sql +++ b/tests/queries/0_stateless/01287_max_execution_speed.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-debug, no-tsan, no-msan, no-asan SET min_execution_speed = 100000000000, timeout_before_checking_execution_speed = 0; SELECT count() FROM system.numbers; -- { serverError TOO_SLOW } From 0996ed5246b2b48aab375ccc870bc529f969cdec Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 5 Sep 2024 14:53:14 +0000 Subject: [PATCH 54/75] Create a deep copy of Settings to ensure they don't change --- src/Storages/Distributed/DistributedSink.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index daccbb78f2c..39f75fd7d56 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -347,7 +347,7 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si } const Block & shard_block = (num_shards > 1) ? job.current_shard_block : current_block; - const Settings & settings = context->getSettingsRef(); + const Settings settings = context->getSettingsCopy(); size_t rows = shard_block.rows(); From 5e0673d207b409b8a93d56975e1d05ddb378f6be Mon Sep 17 00:00:00 2001 From: Christoph Wurm Date: Thu, 5 Sep 2024 16:07:33 +0100 Subject: [PATCH 55/75] Remove version note --- docs/en/sql-reference/statements/delete.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index ded0d7dacac..5733efbf8f2 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -28,10 +28,6 @@ Lightweight `DELETE` is implemented as a [mutation](/en/sql-reference/statements By default, `DELETE` statements wait until marking the rows as deleted is completed before returning. This can take a long time if the amount of data is large. Alternatively, you can run it asynchronously in the background using the setting [`lightweight_deletes_sync`](/en/operations/settings/settings#lightweight_deletes_sync). If disabled, the `DELETE` statement is going to return immediately, but the data can still be visible to queries until the background mutation is finished. -:::note -Before version 24.4, lightweight deletes were asynchronous by default. -::: - The mutation does not physically delete the rows that have been marked as deleted, this will only happen during the next merge. As a result, it is possible that for an unspecified period, data is not actually deleted from storage and is only marked as deleted. If you need to guarantee that your data is deleted from storage in a predictable time, consider using the table setting [`min_age_to_force_merge_seconds`](https://clickhouse.com/docs/en/operations/settings/merge-tree-settings#min_age_to_force_merge_seconds). Or you can use the [ALTER TABLE ... DELETE](/en/sql-reference/statements/alter/delete) command. Note that deleting data using `ALTER TABLE ... DELETE` may consume significant resources as it recreates all affected parts. From cedddf6fa48c8f84088cb3f3c98ec89fe9d7a849 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 26 Aug 2024 19:10:49 +0200 Subject: [PATCH 56/75] PoC --- .../integration/helper_container/Dockerfile | 4 +- tests/integration/helpers/network.py | 135 ++++++++++++++++-- 2 files changed, 129 insertions(+), 10 deletions(-) diff --git a/docker/test/integration/helper_container/Dockerfile b/docker/test/integration/helper_container/Dockerfile index 49a3d3cd84b..1084d087e53 100644 --- a/docker/test/integration/helper_container/Dockerfile +++ b/docker/test/integration/helper_container/Dockerfile @@ -3,6 +3,8 @@ FROM alpine:3.18 RUN apk add --no-cache -U iproute2 \ - && for bin in iptables iptables-restore iptables-save; \ + && for bin in \ + iptables iptables-restore iptables-save \ + ip6tables ip6tables-restore ip6tables-save; \ do ln -sf xtables-nft-multi "/sbin/$bin"; \ done diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index e6e79dc7947..5219ac22f71 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -3,6 +3,7 @@ import subprocess import time import logging import docker +import ipaddress class PartitionManager: @@ -26,25 +27,76 @@ class PartitionManager: self._check_instance(instance) self._add_rule( - {"source": instance.ip_address, "destination_port": 2181, "action": action} + { + "source": instance.ipv4_address, + "destination_port": 2181, + "action": action, + } ) self._add_rule( - {"destination": instance.ip_address, "source_port": 2181, "action": action} + { + "destination": instance.ipv4_address, + "source_port": 2181, + "action": action, + } ) + if instance.ipv6_address: + self._add_rule( + { + "source": instance.ipv6_address, + "destination_port": 2181, + "action": action, + } + ) + self._add_rule( + { + "destination": instance.ipv6_address, + "source_port": 2181, + "action": action, + } + ) + def dump_rules(self): - return _NetworkManager.get().dump_rules() + v4 = _NetworkManager.get().dump_rules() + v6 = _NetworkManager.get().dump_v6_rules() + + return v4 + v6 def restore_instance_zk_connections(self, instance, action="DROP"): self._check_instance(instance) self._delete_rule( - {"source": instance.ip_address, "destination_port": 2181, "action": action} + { + "source": instance.ipv4_address, + "destination_port": 2181, + "action": action, + } ) self._delete_rule( - {"destination": instance.ip_address, "source_port": 2181, "action": action} + { + "destination": instance.ipv4_address, + "source_port": 2181, + "action": action, + } ) + if instance.ipv6_address: + self._delete_rule( + { + "source": instance.ipv6_address, + "destination_port": 2181, + "action": action, + } + ) + self._delete_rule( + { + "destination": instance.ipv6_address, + "source_port": 2181, + "action": action, + } + ) + def partition_instances(self, left, right, port=None, action="DROP"): self._check_instance(left) self._check_instance(right) @@ -59,16 +111,36 @@ class PartitionManager: rule["destination_port"] = port return rule + def create_rule_v6(src, dst): + rule = { + "source": src.ipv6_address, + "destination": dst.ipv6_address, + "action": action, + } + if port is not None: + rule["destination_port"] = port + return rule + self._add_rule(create_rule(left, right)) self._add_rule(create_rule(right, left)) + if left.ipv6_address and right.ipv6_address: + self._add_rule(create_rule_v6(left, right)) + self._add_rule(create_rule_v6(right, left)) + def add_network_delay(self, instance, delay_ms): self._add_tc_netem_delay(instance, delay_ms) def heal_all(self): while self._iptables_rules: rule = self._iptables_rules.pop() - _NetworkManager.get().delete_iptables_rule(**rule) + + if self._is_ipv6_rule(rule): + _NetworkManager.get().delete_ip6tables_rule(**rule) + else: + _NetworkManager.get().delete_iptables_rule(**rule) + # _NetworkManager.get().delete_iptables_rule(**rule) + # _NetworkManager.get().delete_ip6tables_rule(**rule) while self._netem_delayed_instances: instance = self._netem_delayed_instances.pop() @@ -90,12 +162,29 @@ class PartitionManager: if instance.ip_address is None: raise Exception("Instance + " + instance.name + " is not launched!") + @staticmethod + def _is_ipv6_rule(rule): + is_ipv6 = False + + if "source" in rule: + is_ipv6 = ipaddress.ip_address(rule["source"]).version == 6 + if "destination" in rule: + is_ipv6 = ipaddress.ip_address(rule["source"]).version == 6 + + return is_ipv6 + def _add_rule(self, rule): - _NetworkManager.get().add_iptables_rule(**rule) + if self._is_ipv6_rule(rule): + _NetworkManager.get().add_ip6tables_rule(**rule) + else: + _NetworkManager.get().add_iptables_rule(**rule) self._iptables_rules.append(rule) def _delete_rule(self, rule): - _NetworkManager.get().delete_iptables_rule(**rule) + if self._is_ipv6_rule(rule): + _NetworkManager.get().delete_ip6tables_rule(**rule) + else: + _NetworkManager.get().delete_iptables_rule(**rule) self._iptables_rules.remove(rule) def _add_tc_netem_delay(self, instance, delay_ms): @@ -155,15 +244,29 @@ class _NetworkManager: cmd.extend(self._iptables_cmd_suffix(**kwargs)) self._exec_run(cmd, privileged=True) + def add_ip6tables_rule(self, **kwargs): + cmd = ["ip6tables-legacy", "--wait", "-I", "DOCKER-USER", "1"] + cmd.extend(self._iptables_cmd_suffix(**kwargs)) + self._exec_run(cmd, privileged=True) + def delete_iptables_rule(self, **kwargs): cmd = ["iptables", "--wait", "-D", "DOCKER-USER"] cmd.extend(self._iptables_cmd_suffix(**kwargs)) self._exec_run(cmd, privileged=True) + def delete_ip6tables_rule(self, **kwargs): + cmd = ["ip6tables-legacy", "--wait", "-D", "DOCKER-USER"] + cmd.extend(self._iptables_cmd_suffix(**kwargs)) + self._exec_run(cmd, privileged=True) + def dump_rules(self): cmd = ["iptables", "-L", "DOCKER-USER"] return self._exec_run(cmd, privileged=True) + def dump_v6_rules(self): + cmd = ["ip6tables-legacy", "-L", "DOCKER-USER"] + return self._exec_run(cmd, privileged=True) + @staticmethod def clean_all_user_iptables_rules(): for i in range(1000): @@ -178,6 +281,20 @@ class _NetworkManager: + " iterations, last error: " + str(res.stderr) ) + break + + for i in range(1000): + iptables_iter = i + # when rules will be empty, it will return error + res = subprocess.run("ip6tables-legacy --wait -D DOCKER-USER 1", shell=True) + + if res.returncode != 0: + logging.info( + "All ip6tables rules cleared, " + + str(iptables_iter) + + " iterations, last error: " + + str(res.stderr) + ) return @staticmethod @@ -244,7 +361,7 @@ class _NetworkManager: def _ensure_container(self): if self._container is None or self._container_expire_time <= time.time(): image_name = "clickhouse/integration-helper:" + os.getenv( - "DOCKER_HELPER_TAG", "latest" + "DOCKER_HELPER_TAG", "" ) for i in range(5): if self._container is not None: From fe42299928dddd9d2e705305524566ded756f6ec Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 26 Aug 2024 19:11:12 +0200 Subject: [PATCH 57/75] Typo --- tests/integration/helpers/network.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 5219ac22f71..a92843a313b 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -361,7 +361,7 @@ class _NetworkManager: def _ensure_container(self): if self._container is None or self._container_expire_time <= time.time(): image_name = "clickhouse/integration-helper:" + os.getenv( - "DOCKER_HELPER_TAG", "" + "DOCKER_HELPER_TAG", "latest" ) for i in range(5): if self._container is not None: From d49d413c8d0ae7995bf4188ff8ce87ac85587c25 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 26 Aug 2024 19:11:48 +0200 Subject: [PATCH 58/75] No legacy --- tests/integration/helpers/network.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index a92843a313b..e62034a5104 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -245,7 +245,7 @@ class _NetworkManager: self._exec_run(cmd, privileged=True) def add_ip6tables_rule(self, **kwargs): - cmd = ["ip6tables-legacy", "--wait", "-I", "DOCKER-USER", "1"] + cmd = ["ip6tables", "--wait", "-I", "DOCKER-USER", "1"] cmd.extend(self._iptables_cmd_suffix(**kwargs)) self._exec_run(cmd, privileged=True) @@ -255,7 +255,7 @@ class _NetworkManager: self._exec_run(cmd, privileged=True) def delete_ip6tables_rule(self, **kwargs): - cmd = ["ip6tables-legacy", "--wait", "-D", "DOCKER-USER"] + cmd = ["ip6tables", "--wait", "-D", "DOCKER-USER"] cmd.extend(self._iptables_cmd_suffix(**kwargs)) self._exec_run(cmd, privileged=True) @@ -264,7 +264,7 @@ class _NetworkManager: return self._exec_run(cmd, privileged=True) def dump_v6_rules(self): - cmd = ["ip6tables-legacy", "-L", "DOCKER-USER"] + cmd = ["ip6tables", "-L", "DOCKER-USER"] return self._exec_run(cmd, privileged=True) @staticmethod @@ -286,7 +286,7 @@ class _NetworkManager: for i in range(1000): iptables_iter = i # when rules will be empty, it will return error - res = subprocess.run("ip6tables-legacy --wait -D DOCKER-USER 1", shell=True) + res = subprocess.run("ip6tables --wait -D DOCKER-USER 1", shell=True) if res.returncode != 0: logging.info( From 5ee5c8224ea715b9a78a9ee79f79419ae36db6f2 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 26 Aug 2024 22:16:05 +0200 Subject: [PATCH 59/75] Fix --- tests/integration/helpers/network.py | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index e62034a5104..c35ab65eef5 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -164,14 +164,10 @@ class PartitionManager: @staticmethod def _is_ipv6_rule(rule): - is_ipv6 = False - if "source" in rule: - is_ipv6 = ipaddress.ip_address(rule["source"]).version == 6 + return ipaddress.ip_address(rule["source"]).version == 6 if "destination" in rule: - is_ipv6 = ipaddress.ip_address(rule["source"]).version == 6 - - return is_ipv6 + return ipaddress.ip_address(rule["destination"]).version == 6 def _add_rule(self, rule): if self._is_ipv6_rule(rule): From 9be79614a3fd3d97ed1be76dddcd4182e75d3f8b Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 15:20:56 +0200 Subject: [PATCH 60/75] Fix --- tests/integration/helpers/network.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index c35ab65eef5..d3de4660acb 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -164,11 +164,13 @@ class PartitionManager: @staticmethod def _is_ipv6_rule(rule): - if "source" in rule: + if rule.get("source"): return ipaddress.ip_address(rule["source"]).version == 6 - if "destination" in rule: + if rule.get("destination"): return ipaddress.ip_address(rule["destination"]).version == 6 + return False + def _add_rule(self, rule): if self._is_ipv6_rule(rule): _NetworkManager.get().add_ip6tables_rule(**rule) From 335ca75174329ecabb9d8de59c1cfd4f799842ba Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 3 Sep 2024 00:16:40 +0200 Subject: [PATCH 61/75] Lint --- tests/integration/helpers/network.py | 41 +++++++++------------------- 1 file changed, 13 insertions(+), 28 deletions(-) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index d3de4660acb..4b02f99876f 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -139,8 +139,6 @@ class PartitionManager: _NetworkManager.get().delete_ip6tables_rule(**rule) else: _NetworkManager.get().delete_iptables_rule(**rule) - # _NetworkManager.get().delete_iptables_rule(**rule) - # _NetworkManager.get().delete_ip6tables_rule(**rule) while self._netem_delayed_instances: instance = self._netem_delayed_instances.pop() @@ -267,33 +265,20 @@ class _NetworkManager: @staticmethod def clean_all_user_iptables_rules(): - for i in range(1000): - iptables_iter = i - # when rules will be empty, it will return error - res = subprocess.run("iptables --wait -D DOCKER-USER 1", shell=True) + for iptables in ("iptables", "ip6tables"): + for i in range(1000): + iptables_iter = i + # when rules will be empty, it will return error + res = subprocess.run(f"{iptables}--wait -D DOCKER-USER 1", shell=True) - if res.returncode != 0: - logging.info( - "All iptables rules cleared, " - + str(iptables_iter) - + " iterations, last error: " - + str(res.stderr) - ) - break - - for i in range(1000): - iptables_iter = i - # when rules will be empty, it will return error - res = subprocess.run("ip6tables --wait -D DOCKER-USER 1", shell=True) - - if res.returncode != 0: - logging.info( - "All ip6tables rules cleared, " - + str(iptables_iter) - + " iterations, last error: " - + str(res.stderr) - ) - return + if res.returncode != 0: + logging.info( + f"All {iptables} rules cleared, " + + str(iptables_iter) + + " iterations, last error: " + + str(res.stderr) + ) + break @staticmethod def _iptables_cmd_suffix( From d46065360b13c9dfbab11f642d54702610cd550d Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 3 Sep 2024 05:24:03 +0200 Subject: [PATCH 62/75] Fix --- tests/integration/helpers/network.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 4b02f99876f..8d96add9501 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -28,14 +28,14 @@ class PartitionManager: self._add_rule( { - "source": instance.ipv4_address, + "source": instance.ip_address, "destination_port": 2181, "action": action, } ) self._add_rule( { - "destination": instance.ipv4_address, + "destination": instance.ip_address, "source_port": 2181, "action": action, } @@ -68,14 +68,14 @@ class PartitionManager: self._delete_rule( { - "source": instance.ipv4_address, + "source": instance.ip_address, "destination_port": 2181, "action": action, } ) self._delete_rule( { - "destination": instance.ipv4_address, + "destination": instance.ip_address, "source_port": 2181, "action": action, } From 58b84e9e3d75846428b6839aa03207a4ce8f5cbd Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 3 Sep 2024 17:29:44 +0200 Subject: [PATCH 63/75] Update network.py --- tests/integration/helpers/network.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 8d96add9501..c9e2df0b2a0 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -269,7 +269,7 @@ class _NetworkManager: for i in range(1000): iptables_iter = i # when rules will be empty, it will return error - res = subprocess.run(f"{iptables}--wait -D DOCKER-USER 1", shell=True) + res = subprocess.run(f"{iptables} --wait -D DOCKER-USER 1", shell=True) if res.returncode != 0: logging.info( From 1f5eb2cd2c1de8639d017f769d04975d21cc590a Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 3 Sep 2024 18:36:32 +0200 Subject: [PATCH 64/75] Set IPv6 instance to the node object --- tests/integration/helpers/cluster.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 53f4f1e1f26..821bb887435 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2112,6 +2112,7 @@ class ClickHouseCluster: self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name] ) node.ip_address = self.get_instance_ip(node.name) + node.ipv6_address = self.get_instance_global_ipv6(node.name) node.client = Client(node.ip_address, command=self.client_bin_path) logging.info("Restart node with ip change") @@ -3182,6 +3183,7 @@ class ClickHouseCluster: for instance in self.instances.values(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) + instance.ipv6_address = self.get_instance_global_ipv6(instance.name) logging.debug( f"Waiting for ClickHouse start in {instance.name}, ip: {instance.ip_address}..." From 26add45c70cef4dd2c0fc18891c3db9bf90bb06e Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 3 Sep 2024 18:49:04 +0200 Subject: [PATCH 65/75] Poke CI From da3a7d069173b7508584a63190a5e24a9452e5bd Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 5 Sep 2024 06:23:20 +0200 Subject: [PATCH 66/75] Setup missing ip6tables chain --- tests/integration/helpers/network.py | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index c9e2df0b2a0..39f413cdf3b 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -235,12 +235,29 @@ class _NetworkManager: cls._instance = cls(**kwargs) return cls._instance + def setup_ip6tables_docker_user_chain(self): + _rules = subprocess.check_output( + f"ip6tables-save", shell=True + ) + if "DOCKER-USER" in _rules.decode("utf-8"): + return + + setup_cmds = [ + ["ip6tables", "--wait", "-N", "DOCKER-USER"], + ["ip6tables", "--wait", "-I", "FORWARD", "-j", "DOCKER-USER"], + ["ip6tables", "--wait", "-A", "DOCKER-USER", "-j", "RETURN"], + ] + for cmd in setup_cmds: + self._exec_run(cmd, privileged=True) + def add_iptables_rule(self, **kwargs): cmd = ["iptables", "--wait", "-I", "DOCKER-USER", "1"] cmd.extend(self._iptables_cmd_suffix(**kwargs)) self._exec_run(cmd, privileged=True) def add_ip6tables_rule(self, **kwargs): + self.setup_ip6tables_docker_user_chain() + cmd = ["ip6tables", "--wait", "-I", "DOCKER-USER", "1"] cmd.extend(self._iptables_cmd_suffix(**kwargs)) self._exec_run(cmd, privileged=True) From 798f4b4c3b4667b6cd09760449758b6b7e503e56 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 5 Sep 2024 04:35:04 +0000 Subject: [PATCH 67/75] Automatic style fix --- tests/integration/helpers/network.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 39f413cdf3b..065836396f3 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -236,9 +236,7 @@ class _NetworkManager: return cls._instance def setup_ip6tables_docker_user_chain(self): - _rules = subprocess.check_output( - f"ip6tables-save", shell=True - ) + _rules = subprocess.check_output(f"ip6tables-save", shell=True) if "DOCKER-USER" in _rules.decode("utf-8"): return From a463b2b44c9162a9ee7cb3d830a3ad6a3b5a7ff6 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 5 Sep 2024 15:59:21 +0200 Subject: [PATCH 68/75] Poke CI From 3974e9060a4a8da4bd1f9e4c1f96f2a25bc36518 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 6 Sep 2024 18:50:02 +0800 Subject: [PATCH 69/75] Fix docs and some fixups --- .../mergetree-family/mergetree.md | 33 +++++++++---------- src/Storages/Statistics/Statistics.cpp | 2 +- src/Storages/Statistics/StatisticsMinMax.cpp | 3 ++ 3 files changed, 20 insertions(+), 18 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index d63ab54ed1f..74f1295ef96 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -989,44 +989,43 @@ ALTER TABLE tab DROP STATISTICS a; These lightweight statistics aggregate information about distribution of values in columns. Statistics are stored in every part and updated when every insert comes. They can be used for prewhere optimization only if we enable `set allow_statistics_optimize = 1`. -#### Available Types of Column Statistics {#available-types-of-column-statistics} +### Available Types of Column Statistics {#available-types-of-column-statistics} - `MinMax` The minimum and maximum column value which allows to estimate the selectivity of range filters on numeric columns. - Supported data types: (U)Int*, Float*, Decimal(*), Boolean and Date*. - - `TDigest` [TDigest](https://github.com/tdunning/t-digest) sketches which allow to compute approximate percentiles (e.g. the 90th percentile) for numeric columns. - Supported data types: (U)Int*, Float*, Decimal(*), Boolean and Date*. - - `Uniq` [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) sketches which provide an estimation how many distinct values a column contains. - Supported data types: (U)Int*, Float*, Decimal*, Boolean, Date* and (Fixed)String. - - `count_min` [Count-min](https://en.wikipedia.org/wiki/Count%E2%80%93min_sketch) sketches which provide an approximate count of the frequency of each value in a column. - Supported data types: (U)Int*, Float*, Decimal*, Boolean, Date* and (Fixed)String. -Note that all statistics types support `LowCardinality` and `Nullable` modifiers to data types. +### Supported Data Types {#supported-data-types} -#### Supported operations of Column Statistics {#supported-operations-of-column-statistics} +| | (U)Int* | Float* | Decimal(*) | Date* | Boolean | Enum* | (Fixed)String | +|-----------|---------|--------|------------|-------|---------|-------|------------------| +| count_min | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | +| MinMax | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | ✗ | +| TDigest | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | ✗ | +| Uniq | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | ✔ | -| | Equals | Range | -|-----------|---------|-------| -| count_min | ✔ | ✗ | -| MinMax | ✗ | ✔ | -| TDigest | ✗ | ✔ | -| Uniq | ✔ | ✗ | -Please note that operation `Range` represents >, >=, < or <=. +### Supported Operations {#supported-operations} + +| | Equality filters (==) | Range filters (>, >=, <, <=) | +|-----------|-----------------------|------------------------------| +| count_min | ✔ | ✗ | +| MinMax | ✗ | ✔ | +| TDigest | ✗ | ✔ | +| Uniq | ✔ | ✗ | ## Column-level Settings {#column-level-settings} diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 5e22c5d121a..795963bd55d 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -127,7 +127,7 @@ Float64 ColumnStatistics::estimateEqual(const Field & val) const if (stats.contains(StatisticsType::Uniq)) { UInt64 cardinality = stats.at(StatisticsType::Uniq)->estimateCardinality(); - if (cardinality == 0) + if (cardinality == 0 || rows == 0) return 0; return 1.0 / cardinality * rows; /// assume uniform distribution } diff --git a/src/Storages/Statistics/StatisticsMinMax.cpp b/src/Storages/Statistics/StatisticsMinMax.cpp index 6ae05cb0a6b..27072d225de 100644 --- a/src/Storages/Statistics/StatisticsMinMax.cpp +++ b/src/Storages/Statistics/StatisticsMinMax.cpp @@ -51,6 +51,9 @@ void StatisticsMinMax::deserialize(ReadBuffer & buf) Float64 StatisticsMinMax::estimateLess(const Field & val) const { + if (row_count == 0) + return 0; + auto val_as_float = StatisticsUtils::tryConvertToFloat64(val, data_type); if (!val_as_float.has_value()) return 0; From fde9f23829bfd6d0c69af1fa90938b0c69be9afb Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 6 Sep 2024 13:48:20 +0200 Subject: [PATCH 70/75] Disable long tests with sanitizers --- tests/integration/test_backup_restore_new/test.py | 7 +++++++ .../test_backward_compatibility/test_functions.py | 7 +++++++ 2 files changed, 14 insertions(+) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 4806625f3f0..56e007dcf5d 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -343,6 +343,13 @@ def test_increment_backup_without_changes(): def test_incremental_backup_overflow(): + if ( + instance.is_built_with_thread_sanitizer() + or instance.is_built_with_memory_sanitizer() + or instance.is_built_with_address_sanitizer() + ): + pytest.skip("The test is slow in builds with sanitizer") + backup_name = new_backup_name() incremental_backup_name = new_backup_name() diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index 202a741bfb5..acf4bd28c9b 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -154,6 +154,13 @@ def test_aggregate_states(start_cluster): def test_string_functions(start_cluster): + if ( + upstream.is_built_with_thread_sanitizer() + or upstream.is_built_with_memory_sanitizer() + or upstream.is_built_with_address_sanitizer() + ): + pytest.skip("The test is slow in builds with sanitizer") + functions = backward.query( """ SELECT if(NOT empty(alias_to), alias_to, name) From dea714d1696e0ff0ceb277b37e7f36f5e3af03cc Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 6 Sep 2024 14:25:02 +0000 Subject: [PATCH 71/75] Update version_date.tsv and changelogs after v24.3.11.7-lts --- docs/changelogs/v24.3.11.7-lts.md | 17 +++++++++++++++++ utils/list-versions/version_date.tsv | 3 +++ 2 files changed, 20 insertions(+) create mode 100644 docs/changelogs/v24.3.11.7-lts.md diff --git a/docs/changelogs/v24.3.11.7-lts.md b/docs/changelogs/v24.3.11.7-lts.md new file mode 100644 index 00000000000..2d97dfbaad2 --- /dev/null +++ b/docs/changelogs/v24.3.11.7-lts.md @@ -0,0 +1,17 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.3.11.7-lts (28795d0a47e) FIXME as compared to v24.3.10.33-lts (37b6502ebf0) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#67479](https://github.com/ClickHouse/ClickHouse/issues/67479): In rare cases ClickHouse could consider parts as broken because of some unexpected projections on disk. Now it's fixed. [#66898](https://github.com/ClickHouse/ClickHouse/pull/66898) ([alesapin](https://github.com/alesapin)). +* Backported in [#69243](https://github.com/ClickHouse/ClickHouse/issues/69243): `UNION` clause in subqueries wasn't handled correctly in queries with parallel replicas and lead to LOGICAL_ERROR `Duplicate announcement received for replica`. [#69146](https://github.com/ClickHouse/ClickHouse/pull/69146) ([Igor Nikonov](https://github.com/devcrafter)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#69221](https://github.com/ClickHouse/ClickHouse/issues/69221): Disable memory test with sanitizer. [#69193](https://github.com/ClickHouse/ClickHouse/pull/69193) ([alesapin](https://github.com/alesapin)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 4fa387731ce..d979577677d 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -3,12 +3,14 @@ v24.8.2.3-lts 2024-08-22 v24.8.1.2684-lts 2024-08-21 v24.7.5.37-stable 2024-09-03 v24.7.4.51-stable 2024-08-23 +v24.7.3.47-stable 2024-09-04 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.5.30-stable 2024-09-03 v24.6.4.42-stable 2024-08-23 v24.6.3.95-stable 2024-08-06 +v24.6.3.38-stable 2024-09-04 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.7.31-stable 2024-09-03 @@ -22,6 +24,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.11.7-lts 2024-09-06 v24.3.10.33-lts 2024-09-03 v24.3.9.5-lts 2024-08-22 v24.3.8.13-lts 2024-08-20 From 9b87f2e6d40135c8f6fc9eb3dcd352e85a10ab21 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 6 Sep 2024 15:10:13 +0000 Subject: [PATCH 72/75] Update version_date.tsv and changelogs after v24.5.8.10-stable --- docs/changelogs/v24.5.8.10-stable.md | 18 ++++++++++++++++++ utils/list-versions/version_date.tsv | 4 ++++ 2 files changed, 22 insertions(+) create mode 100644 docs/changelogs/v24.5.8.10-stable.md diff --git a/docs/changelogs/v24.5.8.10-stable.md b/docs/changelogs/v24.5.8.10-stable.md new file mode 100644 index 00000000000..c7a2baa4e4c --- /dev/null +++ b/docs/changelogs/v24.5.8.10-stable.md @@ -0,0 +1,18 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.5.8.10-stable (f11729638ea) FIXME as compared to v24.5.7.31-stable (6c185e9aec1) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#69295](https://github.com/ClickHouse/ClickHouse/issues/69295): TODO. [#68744](https://github.com/ClickHouse/ClickHouse/pull/68744) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#69245](https://github.com/ClickHouse/ClickHouse/issues/69245): `UNION` clause in subqueries wasn't handled correctly in queries with parallel replicas and lead to LOGICAL_ERROR `Duplicate announcement received for replica`. [#69146](https://github.com/ClickHouse/ClickHouse/pull/69146) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix crash when using `s3` table function with GLOB paths and filters. [#69176](https://github.com/ClickHouse/ClickHouse/pull/69176) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#69223](https://github.com/ClickHouse/ClickHouse/issues/69223): Disable memory test with sanitizer. [#69193](https://github.com/ClickHouse/ClickHouse/pull/69193) ([alesapin](https://github.com/alesapin)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 4fa387731ce..09edea9af21 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -3,14 +3,17 @@ v24.8.2.3-lts 2024-08-22 v24.8.1.2684-lts 2024-08-21 v24.7.5.37-stable 2024-09-03 v24.7.4.51-stable 2024-08-23 +v24.7.3.47-stable 2024-09-04 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 v24.6.5.30-stable 2024-09-03 v24.6.4.42-stable 2024-08-23 v24.6.3.95-stable 2024-08-06 +v24.6.3.38-stable 2024-09-04 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 +v24.5.8.10-stable 2024-09-06 v24.5.7.31-stable 2024-09-03 v24.5.6.45-stable 2024-08-23 v24.5.5.78-stable 2024-08-05 @@ -22,6 +25,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.11.7-lts 2024-09-06 v24.3.10.33-lts 2024-09-03 v24.3.9.5-lts 2024-08-22 v24.3.8.13-lts 2024-08-20 From 4665867400e70fbf002759cf9114bbff826fbe56 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 6 Sep 2024 15:39:40 +0000 Subject: [PATCH 73/75] Update version_date.tsv and changelogs after v24.6.6.6-stable --- docs/changelogs/v24.6.6.6-stable.md | 16 ++++++++++++++++ utils/list-versions/version_date.tsv | 5 +++++ 2 files changed, 21 insertions(+) create mode 100644 docs/changelogs/v24.6.6.6-stable.md diff --git a/docs/changelogs/v24.6.6.6-stable.md b/docs/changelogs/v24.6.6.6-stable.md new file mode 100644 index 00000000000..8cf83e8fc18 --- /dev/null +++ b/docs/changelogs/v24.6.6.6-stable.md @@ -0,0 +1,16 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.6.6.6-stable (a4c4580e639) FIXME as compared to v24.6.5.30-stable (e6e196c92d6) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#69197](https://github.com/ClickHouse/ClickHouse/issues/69197): TODO. [#68744](https://github.com/ClickHouse/ClickHouse/pull/68744) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#69225](https://github.com/ClickHouse/ClickHouse/issues/69225): Disable memory test with sanitizer. [#69193](https://github.com/ClickHouse/ClickHouse/pull/69193) ([alesapin](https://github.com/alesapin)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 4fa387731ce..f0ae5f641f0 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -3,14 +3,18 @@ v24.8.2.3-lts 2024-08-22 v24.8.1.2684-lts 2024-08-21 v24.7.5.37-stable 2024-09-03 v24.7.4.51-stable 2024-08-23 +v24.7.3.47-stable 2024-09-04 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 +v24.6.6.6-stable 2024-09-06 v24.6.5.30-stable 2024-09-03 v24.6.4.42-stable 2024-08-23 v24.6.3.95-stable 2024-08-06 +v24.6.3.38-stable 2024-09-04 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 +v24.5.8.10-stable 2024-09-06 v24.5.7.31-stable 2024-09-03 v24.5.6.45-stable 2024-08-23 v24.5.5.78-stable 2024-08-05 @@ -22,6 +26,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.11.7-lts 2024-09-06 v24.3.10.33-lts 2024-09-03 v24.3.9.5-lts 2024-08-22 v24.3.8.13-lts 2024-08-20 From 4c9713717ff83f4cf1676398a432209d783a8d62 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 6 Sep 2024 16:10:11 +0000 Subject: [PATCH 74/75] Update version_date.tsv and changelogs after v24.7.6.8-stable --- docs/changelogs/v24.7.6.8-stable.md | 17 +++++++++++++++++ utils/list-versions/version_date.tsv | 6 ++++++ 2 files changed, 23 insertions(+) create mode 100644 docs/changelogs/v24.7.6.8-stable.md diff --git a/docs/changelogs/v24.7.6.8-stable.md b/docs/changelogs/v24.7.6.8-stable.md new file mode 100644 index 00000000000..13102a97b40 --- /dev/null +++ b/docs/changelogs/v24.7.6.8-stable.md @@ -0,0 +1,17 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.7.6.8-stable (7779883593a) FIXME as compared to v24.7.5.37-stable (f2533ca97be) + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#69198](https://github.com/ClickHouse/ClickHouse/issues/69198): TODO. [#68744](https://github.com/ClickHouse/ClickHouse/pull/68744) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#69249](https://github.com/ClickHouse/ClickHouse/issues/69249): `UNION` clause in subqueries wasn't handled correctly in queries with parallel replicas and lead to LOGICAL_ERROR `Duplicate announcement received for replica`. [#69146](https://github.com/ClickHouse/ClickHouse/pull/69146) ([Igor Nikonov](https://github.com/devcrafter)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#69227](https://github.com/ClickHouse/ClickHouse/issues/69227): Disable memory test with sanitizer. [#69193](https://github.com/ClickHouse/ClickHouse/pull/69193) ([alesapin](https://github.com/alesapin)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 4fa387731ce..f3876e1c51d 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,16 +1,21 @@ v24.8.3.59-lts 2024-09-03 v24.8.2.3-lts 2024-08-22 v24.8.1.2684-lts 2024-08-21 +v24.7.6.8-stable 2024-09-06 v24.7.5.37-stable 2024-09-03 v24.7.4.51-stable 2024-08-23 +v24.7.3.47-stable 2024-09-04 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 +v24.6.6.6-stable 2024-09-06 v24.6.5.30-stable 2024-09-03 v24.6.4.42-stable 2024-08-23 v24.6.3.95-stable 2024-08-06 +v24.6.3.38-stable 2024-09-04 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 +v24.5.8.10-stable 2024-09-06 v24.5.7.31-stable 2024-09-03 v24.5.6.45-stable 2024-08-23 v24.5.5.78-stable 2024-08-05 @@ -22,6 +27,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.11.7-lts 2024-09-06 v24.3.10.33-lts 2024-09-03 v24.3.9.5-lts 2024-08-22 v24.3.8.13-lts 2024-08-20 From 8be745e8c223de9150552cc2a561bfb325fd0163 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 6 Sep 2024 16:43:55 +0000 Subject: [PATCH 75/75] Update version_date.tsv and changelogs after v24.8.4.13-lts --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.8.4.13-lts.md | 22 ++++++++++++++++++++++ utils/list-versions/version_date.tsv | 7 +++++++ 5 files changed, 32 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v24.8.4.13-lts.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 39b9fa7c395..19c5903461c 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.8.3.59" +ARG VERSION="24.8.4.13" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 60b038d0da9..e74dcf6d73d 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.8.3.59" +ARG VERSION="24.8.4.13" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 77c403bb046..456bed14a7e 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.8.3.59" +ARG VERSION="24.8.4.13" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.8.4.13-lts.md b/docs/changelogs/v24.8.4.13-lts.md new file mode 100644 index 00000000000..66385172cf8 --- /dev/null +++ b/docs/changelogs/v24.8.4.13-lts.md @@ -0,0 +1,22 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.8.4.13-lts (53195bc189b) FIXME as compared to v24.8.3.59-lts (e729b9fa40e) + +#### Improvement +* Backported in [#68699](https://github.com/ClickHouse/ClickHouse/issues/68699): Delete old code of named collections from dictionaries and substitute it to the new, which allows to use DDL created named collections in dictionaries. Closes [#60936](https://github.com/ClickHouse/ClickHouse/issues/60936), closes [#36890](https://github.com/ClickHouse/ClickHouse/issues/36890). [#68412](https://github.com/ClickHouse/ClickHouse/pull/68412) ([Kseniia Sumarokova](https://github.com/kssenii)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#69231](https://github.com/ClickHouse/ClickHouse/issues/69231): Fix parsing error when null should be inserted as default in some cases during JSON type parsing. [#68955](https://github.com/ClickHouse/ClickHouse/pull/68955) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#69251](https://github.com/ClickHouse/ClickHouse/issues/69251): `UNION` clause in subqueries wasn't handled correctly in queries with parallel replicas and lead to LOGICAL_ERROR `Duplicate announcement received for replica`. [#69146](https://github.com/ClickHouse/ClickHouse/pull/69146) ([Igor Nikonov](https://github.com/devcrafter)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#69189](https://github.com/ClickHouse/ClickHouse/issues/69189): Don't create Object type if use_json_alias_for_old_object_type=1 but allow_experimental_object_type=0. [#69150](https://github.com/ClickHouse/ClickHouse/pull/69150) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#69229](https://github.com/ClickHouse/ClickHouse/issues/69229): Disable memory test with sanitizer. [#69193](https://github.com/ClickHouse/ClickHouse/pull/69193) ([alesapin](https://github.com/alesapin)). +* Backported in [#69219](https://github.com/ClickHouse/ClickHouse/issues/69219): Disable perf-like test with sanitizers. [#69194](https://github.com/ClickHouse/ClickHouse/pull/69194) ([alesapin](https://github.com/alesapin)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 4fa387731ce..a621f5a7ddf 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,16 +1,22 @@ +v24.8.4.13-lts 2024-09-06 v24.8.3.59-lts 2024-09-03 v24.8.2.3-lts 2024-08-22 v24.8.1.2684-lts 2024-08-21 +v24.7.6.8-stable 2024-09-06 v24.7.5.37-stable 2024-09-03 v24.7.4.51-stable 2024-08-23 +v24.7.3.47-stable 2024-09-04 v24.7.3.42-stable 2024-08-08 v24.7.2.13-stable 2024-08-01 v24.7.1.2915-stable 2024-07-30 +v24.6.6.6-stable 2024-09-06 v24.6.5.30-stable 2024-09-03 v24.6.4.42-stable 2024-08-23 v24.6.3.95-stable 2024-08-06 +v24.6.3.38-stable 2024-09-04 v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 +v24.5.8.10-stable 2024-09-06 v24.5.7.31-stable 2024-09-03 v24.5.6.45-stable 2024-08-23 v24.5.5.78-stable 2024-08-05 @@ -22,6 +28,7 @@ v24.4.4.113-stable 2024-08-02 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.11.7-lts 2024-09-06 v24.3.10.33-lts 2024-09-03 v24.3.9.5-lts 2024-08-22 v24.3.8.13-lts 2024-08-20