From 3e2f41f3010ca7b68762518738a35dee0f84f8e0 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Wed, 27 Mar 2024 18:35:44 +0300 Subject: [PATCH 001/259] 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 002/259] 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 003/259] 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 004/259] 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 005/259] 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 096616bd1f5f105a760243dfaec5f4493bccabeb Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 28 Jun 2024 02:28:02 +0000 Subject: [PATCH 006/259] introduce mutations snapshot --- src/Interpreters/MutationsInterpreter.cpp | 17 ++- src/Interpreters/MutationsInterpreter.h | 6 +- .../optimizeUseAggregateProjection.cpp | 2 +- .../optimizeUseNormalProjection.cpp | 2 +- .../Optimizations/projectionsCommon.cpp | 9 +- src/Processors/QueryPlan/PartsSplitter.cpp | 1 - .../QueryPlan/ReadFromMergeTree.cpp | 24 ++-- src/Processors/QueryPlan/ReadFromMergeTree.h | 10 +- src/Storages/MergeTree/AlterConversions.cpp | 9 +- src/Storages/MergeTree/AlterConversions.h | 3 +- src/Storages/MergeTree/MergeTask.cpp | 14 ++- src/Storages/MergeTree/MergeTask.h | 2 + src/Storages/MergeTree/MergeTreeData.cpp | 77 ++++++++---- src/Storages/MergeTree/MergeTreeData.h | 46 +++++-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 51 ++------ .../MergeTree/MergeTreeDataSelectExecutor.h | 8 +- .../MergeTree/MergeTreeMutationEntry.cpp | 12 +- .../MergeTree/MergeTreeMutationEntry.h | 2 +- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 2 + .../MergeTree/MergeTreePrefetchedReadPool.h | 1 + src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 + src/Storages/MergeTree/MergeTreeReadPool.h | 1 + .../MergeTree/MergeTreeReadPoolBase.cpp | 6 +- .../MergeTree/MergeTreeReadPoolBase.h | 4 + .../MergeTree/MergeTreeReadPoolInOrder.cpp | 2 + .../MergeTree/MergeTreeReadPoolInOrder.h | 1 + .../MergeTreeReadPoolParallelReplicas.cpp | 2 + .../MergeTreeReadPoolParallelReplicas.h | 1 + ...rgeTreeReadPoolParallelReplicasInOrder.cpp | 2 + ...MergeTreeReadPoolParallelReplicasInOrder.h | 1 + .../MergeTree/MergeTreeSequentialSource.cpp | 21 +++- .../MergeTree/MergeTreeSequentialSource.h | 2 + src/Storages/MergeTree/MutateTask.cpp | 27 ++-- src/Storages/MergeTree/RangesInDataPart.h | 6 +- .../MergeTree/ReplicatedMergeTreeLogEntry.h | 1 - .../MergeTree/ReplicatedMergeTreeQueue.cpp | 82 ++++++++---- .../MergeTree/ReplicatedMergeTreeQueue.h | 24 +++- .../MergeTree/StorageFromMergeTreeDataPart.h | 11 +- src/Storages/StorageMergeTree.cpp | 117 ++++++++++-------- src/Storages/StorageMergeTree.h | 24 +++- src/Storages/StorageReplicatedMergeTree.cpp | 6 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- 42 files changed, 413 insertions(+), 230 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 6d3a4f30b34..fc15a20f992 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -145,6 +145,7 @@ ColumnDependencies getAllColumnDependencies( bool isStorageTouchedByMutations( MergeTreeData & storage, MergeTreeData::DataPartPtr source_part, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, ContextPtr context) @@ -181,7 +182,7 @@ bool isStorageTouchedByMutations( if (all_commands_can_be_skipped) return false; - auto storage_from_part = std::make_shared(source_part); + auto storage_from_part = std::make_shared(source_part, mutations_snapshot); std::optional interpreter_select_query; BlockIO io; @@ -283,8 +284,13 @@ MutationsInterpreter::Source::Source(StoragePtr storage_) : storage(std::move(st { } -MutationsInterpreter::Source::Source(MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_) - : data(&storage_), part(std::move(source_part_)) +MutationsInterpreter::Source::Source( + MergeTreeData & storage_, + MergeTreeData::DataPartPtr source_part_, + AlterConversionsPtr alter_conversions_) + : data(&storage_) + , part(std::move(source_part_)) + , alter_conversions(std::move(alter_conversions_)) { } @@ -384,13 +390,14 @@ MutationsInterpreter::MutationsInterpreter( MutationsInterpreter::MutationsInterpreter( MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_, + AlterConversionsPtr alter_conversions_, StorageMetadataPtr metadata_snapshot_, MutationCommands commands_, Names available_columns_, ContextPtr context_, Settings settings_) : MutationsInterpreter( - Source(storage_, std::move(source_part_)), + Source(storage_, std::move(source_part_), std::move(alter_conversions_)), std::move(metadata_snapshot_), std::move(commands_), std::move(available_columns_), std::move(context_), std::move(settings_)) { @@ -1210,7 +1217,7 @@ void MutationsInterpreter::Source::read( createReadFromPartStep( MergeTreeSequentialSourceType::Mutation, plan, *data, storage_snapshot, - part, required_columns, + part, alter_conversions, required_columns, apply_deleted_mask_, filter, context_, getLogger("MutationsInterpreter")); } diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 6aaa233cda3..8ae438efc19 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -6,6 +6,7 @@ #include #include #include +#include "Storages/MergeTree/AlterConversions.h" namespace DB @@ -21,6 +22,7 @@ using QueryPipelineBuilderPtr = std::unique_ptr; bool isStorageTouchedByMutations( MergeTreeData & storage, MergeTreeData::DataPartPtr source_part, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, ContextPtr context @@ -71,6 +73,7 @@ public: MutationsInterpreter( MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_, + AlterConversionsPtr alter_conversions_, StorageMetadataPtr metadata_snapshot_, MutationCommands commands_, Names available_columns_, @@ -138,7 +141,7 @@ public: bool can_execute_) const; explicit Source(StoragePtr storage_); - Source(MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_); + Source(MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_, AlterConversionsPtr alter_conversions_); private: StoragePtr storage; @@ -146,6 +149,7 @@ public: /// Special case for *MergeTree. MergeTreeData * data = nullptr; MergeTreeData::DataPartPtr part; + AlterConversionsPtr alter_conversions; }; private: diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 70327bc95b4..7e69734a7e5 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -764,7 +764,7 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu projection_reading = reader.readFromParts( /* parts = */ {}, - /* alter_conversions = */ {}, + reading->getMutationsSnapshot()->cloneEmpty(), best_candidate->dag->getRequiredColumnsNames(), proj_snapshot, projection_query_info, diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 0af3869ccf1..43e60318004 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -199,7 +199,7 @@ std::optional optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod auto projection_reading = reader.readFromParts( /*parts=*/ {}, - /*alter_conversions=*/ {}, + reading->getMutationsSnapshot()->cloneEmpty(), required_columns, proj_snapshot, query_info_copy, diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index af1578d6af8..e6939581b9e 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -217,20 +217,15 @@ bool analyzeProjectionCandidate( { MergeTreeData::DataPartsVector projection_parts; MergeTreeData::DataPartsVector normal_parts; - std::vector alter_conversions; + for (const auto & part_with_ranges : parts_with_ranges) { const auto & created_projections = part_with_ranges.data_part->getProjectionParts(); auto it = created_projections.find(candidate.projection->name); if (it != created_projections.end() && !it->second->is_broken) - { projection_parts.push_back(it->second); - } else - { normal_parts.push_back(part_with_ranges.data_part); - alter_conversions.push_back(part_with_ranges.alter_conversions); - } } if (projection_parts.empty()) @@ -255,7 +250,7 @@ bool analyzeProjectionCandidate( if (!normal_parts.empty()) { /// TODO: We can reuse existing analysis_result by filtering out projection parts - auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts), std::move(alter_conversions)); + auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts)); if (normal_result_ptr->selected_marks != 0) { diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index ed4b1906635..65f1c89f990 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -229,7 +229,6 @@ public: { ranges_in_data_parts.emplace_back( initial_ranges_in_data_parts[part_index].data_part, - initial_ranges_in_data_parts[part_index].alter_conversions, initial_ranges_in_data_parts[part_index].part_index_in_query, MarkRanges{mark_range}); part_index_to_initial_ranges_in_data_parts_index[it->second] = part_index; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index aba3f6ff2da..e958430ff16 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -262,7 +262,7 @@ void ReadFromMergeTree::AnalysisResult::checkLimits(const Settings & settings, c ReadFromMergeTree::ReadFromMergeTree( MergeTreeData::DataPartsVector parts_, - std::vector alter_conversions_, + MergeTreeData::MutationsSnapshotPtr mutations_, Names all_column_names_, const MergeTreeData & data_, const SelectQueryInfo & query_info_, @@ -279,7 +279,7 @@ ReadFromMergeTree::ReadFromMergeTree( query_info_.prewhere_info)}, all_column_names_, query_info_, storage_snapshot_, context_) , reader_settings(getMergeTreeReaderSettings(context_, query_info_)) , prepared_parts(std::move(parts_)) - , alter_conversions_for_parts(std::move(alter_conversions_)) + , mutations_snapshot(std::move(mutations_)) , all_column_names(std::move(all_column_names_)) , data(data_) , actions_settings(ExpressionActionsSettings::fromContext(context_)) @@ -361,6 +361,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( auto pool = std::make_shared( std::move(extension), std::move(parts_with_range), + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -442,6 +443,7 @@ Pipe ReadFromMergeTree::readFromPool( { pool = std::make_shared( std::move(parts_with_range), + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -455,6 +457,7 @@ Pipe ReadFromMergeTree::readFromPool( { pool = std::make_shared( std::move(parts_with_range), + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -535,6 +538,7 @@ Pipe ReadFromMergeTree::readInOrder( std::move(extension), mode, parts_with_ranges, + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -550,6 +554,7 @@ Pipe ReadFromMergeTree::readInOrder( has_limit_below_one_block, read_type, parts_with_ranges, + mutations_snapshot, shared_virtual_fields, storage_snapshot, prewhere_info, @@ -1016,7 +1021,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( } ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction); - new_parts.emplace_back(part.data_part, part.alter_conversions, part.part_index_in_query, std::move(ranges_to_get_from_part)); + new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part)); } splitted_parts_and_ranges.emplace_back(std::move(new_parts)); @@ -1243,7 +1248,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( RangesInDataParts new_parts; for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) - new_parts.emplace_back(part_it->data_part, part_it->alter_conversions, part_it->part_index_in_query, part_it->ranges); + new_parts.emplace_back(part_it->data_part, part_it->part_index_in_query, part_it->ranges); if (new_parts.empty()) continue; @@ -1356,15 +1361,13 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(bool find_exact_ranges) const { - return selectRangesToRead(prepared_parts, alter_conversions_for_parts, find_exact_ranges); + return selectRangesToRead(prepared_parts, find_exact_ranges); } -ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( - MergeTreeData::DataPartsVector parts, std::vector alter_conversions, bool find_exact_ranges) const +ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(MergeTreeData::DataPartsVector parts, bool find_exact_ranges) const { return selectRangesToRead( std::move(parts), - std::move(alter_conversions), metadata_for_reading, query_info, context, @@ -1534,7 +1537,6 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( MergeTreeData::DataPartsVector parts, - std::vector alter_conversions, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info_, ContextPtr context_, @@ -1596,10 +1598,9 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( { MergeTreeDataSelectExecutor::filterPartsByPartition( + parts, indexes->partition_pruner, indexes->minmax_idx_condition, - parts, - alter_conversions, indexes->part_values, metadata_snapshot, data, @@ -1628,7 +1629,6 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( auto reader_settings = getMergeTreeReaderSettings(context_, query_info_); result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( std::move(parts), - std::move(alter_conversions), metadata_snapshot, context_, indexes->key_condition, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index caa8aa2e1bd..57e19441b82 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -110,7 +110,7 @@ public: ReadFromMergeTree( MergeTreeData::DataPartsVector parts_, - std::vector alter_conversions_, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot_, Names all_column_names_, const MergeTreeData & data_, const SelectQueryInfo & query_info_, @@ -154,7 +154,6 @@ public: static AnalysisResultPtr selectRangesToRead( MergeTreeData::DataPartsVector parts, - std::vector alter_conversions, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, ContextPtr context, @@ -166,8 +165,7 @@ public: std::optional & indexes, bool find_exact_ranges); - AnalysisResultPtr selectRangesToRead( - MergeTreeData::DataPartsVector parts, std::vector alter_conversions, bool find_exact_ranges = false) const; + AnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts, bool find_exact_ranges = false) const; AnalysisResultPtr selectRangesToRead(bool find_exact_ranges = false) const; @@ -188,7 +186,7 @@ public: void setAnalyzedResult(AnalysisResultPtr analyzed_result_ptr_) { analyzed_result_ptr = std::move(analyzed_result_ptr_); } const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; } - const std::vector & getAlterConvertionsForParts() const { return alter_conversions_for_parts; } + MergeTreeData::MutationsSnapshotPtr getMutationsSnapshot() const { return mutations_snapshot; } const MergeTreeData & getMergeTreeData() const { return data; } size_t getMaxBlockSize() const { return block_size.max_block_size_rows; } @@ -209,7 +207,7 @@ private: MergeTreeReaderSettings reader_settings; MergeTreeData::DataPartsVector prepared_parts; - std::vector alter_conversions_for_parts; + MergeTreeData::MutationsSnapshotPtr mutations_snapshot; Names all_column_names; diff --git a/src/Storages/MergeTree/AlterConversions.cpp b/src/Storages/MergeTree/AlterConversions.cpp index 31f8f17e2c1..82bef500b34 100644 --- a/src/Storages/MergeTree/AlterConversions.cpp +++ b/src/Storages/MergeTree/AlterConversions.cpp @@ -9,9 +9,14 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -bool AlterConversions::supportsMutationCommandType(MutationCommand::Type t) +bool AlterConversions::isSupportedDataMutation(MutationCommand::Type) { - return t == MutationCommand::Type::RENAME_COLUMN; + return false; +} + +bool AlterConversions::isSupportedMetadataMutation(MutationCommand::Type type) +{ + return type == MutationCommand::Type::RENAME_COLUMN; } void AlterConversions::addMutationCommand(const MutationCommand & command) diff --git a/src/Storages/MergeTree/AlterConversions.h b/src/Storages/MergeTree/AlterConversions.h index 0f857d351dd..68966f88f84 100644 --- a/src/Storages/MergeTree/AlterConversions.h +++ b/src/Storages/MergeTree/AlterConversions.h @@ -35,7 +35,8 @@ public: /// Get column old name before rename (lookup by key in rename_map) std::string getColumnOldName(const std::string & new_name) const; - static bool supportsMutationCommandType(MutationCommand::Type); + static bool isSupportedDataMutation(MutationCommand::Type type); + static bool isSupportedMetadataMutation(MutationCommand::Type type); private: /// Rename map new_name -> old_name. diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 7ab8fa2430a..5dab0cd0c08 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -257,6 +257,10 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() if (enabledBlockOffsetColumn(global_ctx)) addGatheringColumn(global_ctx, BlockOffsetColumn::name, BlockOffsetColumn::type); + auto mutations_snapshot = global_ctx->data->getMutationsSnapshot( + global_ctx->metadata_snapshot->getMetadataVersion(), + /*need_data_mutations=*/ false); + SerializationInfo::Settings info_settings = { .ratio_of_defaults_for_sparse = global_ctx->data->getSettings()->ratio_of_defaults_for_sparse_serialization, @@ -264,10 +268,12 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() }; SerializationInfoByName infos(global_ctx->storage_columns, info_settings); + global_ctx->alter_conversions.reserve(global_ctx->future_part->parts.size()); for (const auto & part : global_ctx->future_part->parts) { global_ctx->new_data_part->ttl_infos.update(part->ttl_infos); + if (global_ctx->metadata_snapshot->hasAnyTTL() && !part->checkAllTTLCalculated(global_ctx->metadata_snapshot)) { LOG_INFO(ctx->log, "Some TTL values were not calculated for part {}. Will calculate them forcefully during merge.", part->name); @@ -288,6 +294,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() infos.add(part_infos); } + + global_ctx->alter_conversions.push_back(MergeTreeData::getAlterConversionsForPart(part, mutations_snapshot)); } const auto & local_part_min_ttl = global_ctx->new_data_part->ttl_infos.part_min_ttl; @@ -604,6 +612,7 @@ Pipe MergeTask::VerticalMergeStage::createPipeForReadingOneColumn(const String & *global_ctx->data, global_ctx->storage_snapshot, global_ctx->future_part->parts[part_num], + global_ctx->alter_conversions[part_num], Names{column_name}, /*mark_ranges=*/ {}, global_ctx->input_rows_filtered, @@ -996,13 +1005,14 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() global_ctx->horizontal_stage_progress = std::make_unique( ctx->column_sizes ? ctx->column_sizes->keyColumnsWeight() : 1.0); - for (const auto & part : global_ctx->future_part->parts) + for (size_t i = 0; i < global_ctx->future_part->parts.size(); ++i) { Pipe pipe = createMergeTreeSequentialSource( MergeTreeSequentialSourceType::Merge, *global_ctx->data, global_ctx->storage_snapshot, - part, + global_ctx->future_part->parts[i], + global_ctx->alter_conversions[i], global_ctx->merging_columns.getNames(), /*mark_ranges=*/ {}, global_ctx->input_rows_filtered, diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 56909d1b7a0..c394a47aff0 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -4,6 +4,7 @@ #include #include +#include "Storages/MergeTree/AlterConversions.h" #include #include @@ -154,6 +155,7 @@ private: StorageSnapshotPtr storage_snapshot{nullptr}; StorageMetadataPtr metadata_snapshot{nullptr}; FutureMergedMutatedPartPtr future_part{nullptr}; + std::vector alter_conversions; /// This will be either nullptr or new_data_part, so raw pointer is ok. IMergeTreeDataPart * parent_part{nullptr}; ContextPtr context{nullptr}; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f9cc65871fe..98c308f5fd1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8115,11 +8115,13 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S return true; } -AlterConversionsPtr MergeTreeData::getAlterConversionsForPart(MergeTreeDataPartPtr part) const +AlterConversionsPtr MergeTreeData::getAlterConversionsForPart( + const MergeTreeDataPartPtr & part, + const MutationsSnapshotPtr & snapshot) { - auto commands = getAlterMutationCommandsForPart(part); - + auto commands = snapshot->getAlterMutationCommandsForPart(part); auto result = std::make_shared(); + for (const auto & command : commands | std::views::reverse) result->addMutationCommand(command); @@ -8427,9 +8429,9 @@ StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & object_columns_copy = object_columns; } - snapshot_data->alter_conversions.reserve(snapshot_data->parts.size()); - for (const auto & part : snapshot_data->parts) - snapshot_data->alter_conversions.push_back(getAlterConversionsForPart(part)); + snapshot_data->mutations_snapshot = getMutationsSnapshot( + metadata_snapshot->getMetadataVersion(), + query_context->getSettingsRef().apply_mutations_on_fly); return std::make_shared(*this, metadata_snapshot, std::move(object_columns_copy), std::move(snapshot_data)); } @@ -8616,28 +8618,59 @@ void MergeTreeData::verifySortingKey(const KeyDescription & sorting_key) } } -bool updateAlterConversionsMutations(const MutationCommands & commands, std::atomic & alter_conversions_mutations, bool remove) +static void updateMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + Int64 increment) { + if (data_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); + + if (metadata_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); + + bool has_data_mutation = false; + bool has_metadata_mutation = false; + for (const auto & command : commands) { - if (AlterConversions::supportsMutationCommandType(command.type)) + if (!has_data_mutation && AlterConversions::isSupportedDataMutation(command.type)) { - if (remove) - { - --alter_conversions_mutations; - if (alter_conversions_mutations < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly mutations counter is negative ({})", alter_conversions_mutations); - } - else - { - if (alter_conversions_mutations < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly mutations counter is negative ({})", alter_conversions_mutations); - ++alter_conversions_mutations; - } - return true; + data_mutations_to_apply += increment; + has_data_mutation = true; + + if (data_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); + } + + if (!has_metadata_mutation && AlterConversions::isSupportedMetadataMutation(command.type)) + { + metadata_mutations_to_apply += increment; + has_metadata_mutation = true; + + if (metadata_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); } } - return false; +} + +void incrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & /*lock*/) +{ + return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, 1); +} + +void decrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & /*lock*/) +{ + return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, -1); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index c6f736a4afd..765b68b7559 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -35,6 +35,7 @@ #include #include #include +#include "Storages/ProjectionsDescription.h" #include #include @@ -445,12 +446,27 @@ public: bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override; + struct IMutationsSnapshot + { + /// Return pending mutations that weren't applied to `part` yet and should be applied on the fly + /// (i.e. when reading from the part). Mutations not supported by AlterConversions + /// (supportsMutationCommandType()) can be omitted. + /// + /// @return list of mutations, in *reverse* order (newest to oldest) + virtual MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; + virtual std::shared_ptr cloneEmpty() const = 0; + + virtual ~IMutationsSnapshot() = default; + }; + + using MutationsSnapshotPtr = std::shared_ptr; + /// Snapshot for MergeTree contains the current set of data parts - /// at the moment of the start of query. + /// and mutations required to be applied at the moment of the start of query. struct SnapshotData : public StorageSnapshot::Data { DataPartsVector parts; - std::vector alter_conversions; + MutationsSnapshotPtr mutations_snapshot; }; StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const override; @@ -934,8 +950,13 @@ public: Disks getDisks() const { return getStoragePolicy()->getDisks(); } + /// TODO: comment + virtual MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const = 0; + /// Return alter conversions for part which must be applied on fly. - AlterConversionsPtr getAlterConversionsForPart(MergeTreeDataPartPtr part) const; + static AlterConversionsPtr getAlterConversionsForPart( + const MergeTreeDataPartPtr & part, + const MutationsSnapshotPtr & snapshot); /// Returns destination disk or volume for the TTL rule according to current storage policy. SpacePtr getDestinationForMoveTTL(const TTLDescription & move_ttl) const; @@ -1448,13 +1469,6 @@ protected: /// mechanisms for parts locking virtual bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const = 0; - /// Return pending mutations that weren't applied to `part` yet and should be applied on the fly - /// (i.e. when reading from the part). Mutations not supported by AlterConversions - /// (supportsMutationCommandType()) can be omitted. - /// - /// @return list of mutations, in *reverse* order (newest to oldest) - virtual MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; - struct PartBackupEntries { String part_name; @@ -1738,6 +1752,16 @@ struct CurrentlySubmergingEmergingTagger /// Look at MutationCommands if it contains mutations for AlterConversions, update the counter. /// Return true if the counter had been updated -bool updateAlterConversionsMutations(const MutationCommands & commands, std::atomic & alter_conversions_mutations, bool remove); +void incrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & lock); + +void decrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & lock); } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 2e287ff3042..0ad7bd47648 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -130,12 +130,10 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( bool enable_parallel_reading) const { const auto & snapshot_data = assert_cast(*storage_snapshot->data); - const auto & parts = snapshot_data.parts; - const auto & alter_conversions = snapshot_data.alter_conversions; auto step = readFromParts( - parts, - alter_conversions, + snapshot_data.parts, + snapshot_data.mutations_snapshot, column_names_to_return, storage_snapshot, query_info, @@ -491,10 +489,9 @@ std::optional> MergeTreeDataSelectExecutor::filterPar } void MergeTreeDataSelectExecutor::filterPartsByPartition( + MergeTreeData::DataPartsVector & parts, const std::optional & partition_pruner, const std::optional & minmax_idx_condition, - MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, @@ -503,8 +500,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( LoggerPtr log, ReadFromMergeTree::IndexStats & index_stats) { - chassert(alter_conversions.empty() || parts.size() == alter_conversions.size()); - const Settings & settings = context->getSettingsRef(); DataTypes minmax_columns_types; @@ -528,7 +523,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( if (query_context->getSettingsRef().allow_experimental_query_deduplication) selectPartsToReadWithUUIDFilter( parts, - alter_conversions, part_values, data.getPinnedPartUUIDs(), minmax_idx_condition, @@ -541,7 +535,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( else selectPartsToRead( parts, - alter_conversions, part_values, minmax_idx_condition, minmax_columns_types, @@ -580,7 +573,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, - std::vector && alter_conversions, StorageMetadataPtr metadata_snapshot, const ContextPtr & context, const KeyCondition & key_condition, @@ -593,8 +585,6 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd bool use_skip_indexes, bool find_exact_ranges) { - chassert(alter_conversions.empty() || parts.size() == alter_conversions.size()); - RangesInDataParts parts_with_ranges; parts_with_ranges.resize(parts.size()); const Settings & settings = context->getSettingsRef(); @@ -653,11 +643,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd auto process_part = [&](size_t part_index) { auto & part = parts[part_index]; - auto alter_conversions_for_part = !alter_conversions.empty() - ? alter_conversions[part_index] - : std::make_shared(); - RangesInDataPart ranges(part, alter_conversions_for_part, part_index); + RangesInDataPart ranges(part, part_index); size_t total_marks_count = part->index_granularity.getMarksCountWithoutFinal(); if (metadata_snapshot->hasPrimaryKey() || part_offset_condition) @@ -907,11 +894,11 @@ ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar return std::make_shared(); std::optional indexes; - /// NOTE: We don't need alter_conversions because the returned analysis_result is only used for: - /// 1. estimate the number of rows to read; 2. projection reading, which doesn't have alter_conversions. + /// NOTE: We don't need mutations snapshot because the returned analysis_result is only used for: + /// 1. estimate the number of rows to read; + /// 2. projection reading, which doesn't have alter conversions. return ReadFromMergeTree::selectRangesToRead( std::move(parts), - /*alter_conversions=*/{}, metadata_snapshot, query_info, context, @@ -926,7 +913,7 @@ ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( MergeTreeData::DataPartsVector parts, - std::vector alter_conversions, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const Names & column_names_to_return, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info, @@ -948,7 +935,7 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( return std::make_unique( std::move(parts), - std::move(alter_conversions), + std::move(mutations_snapshot), column_names_to_return, data, query_info, @@ -1546,7 +1533,6 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingMergedIndex( void MergeTreeDataSelectExecutor::selectPartsToRead( MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, @@ -1555,10 +1541,7 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( PartFilterCounters & counters) { MergeTreeData::DataPartsVector prev_parts; - std::vector prev_conversions; - std::swap(prev_parts, parts); - std::swap(prev_conversions, alter_conversions); for (size_t i = 0; i < prev_parts.size(); ++i) { @@ -1600,14 +1583,11 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( counters.num_granules_after_partition_pruner += num_granules; parts.push_back(prev_parts[i]); - if (!prev_conversions.empty()) - alter_conversions.push_back(prev_conversions[i]); } } void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids, const std::optional & minmax_idx_condition, @@ -1620,18 +1600,13 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( { /// process_parts prepare parts that have to be read for the query, /// returns false if duplicated parts' UUID have been met - auto select_parts = [&] ( - MergeTreeData::DataPartsVector & selected_parts, - std::vector & selected_conversions) -> bool + auto select_parts = [&](MergeTreeData::DataPartsVector & selected_parts) -> bool { auto ignored_part_uuids = query_context->getIgnoredPartUUIDs(); std::unordered_set temp_part_uuids; MergeTreeData::DataPartsVector prev_parts; - std::vector prev_conversions; - std::swap(prev_parts, selected_parts); - std::swap(prev_conversions, selected_conversions); for (size_t i = 0; i < prev_parts.size(); ++i) { @@ -1686,8 +1661,6 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( } selected_parts.push_back(prev_parts[i]); - if (!prev_conversions.empty()) - selected_conversions.push_back(prev_conversions[i]); } if (!temp_part_uuids.empty()) @@ -1706,7 +1679,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( }; /// Process parts that have to be read for a query. - auto needs_retry = !select_parts(parts, alter_conversions); + auto needs_retry = !select_parts(parts); /// If any duplicated part UUIDs met during the first step, try to ignore them in second pass. /// This may happen when `prefer_localhost_replica` is set and "distributed" stage runs in the same process with "remote" stage. @@ -1717,7 +1690,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( counters = PartFilterCounters(); /// Second attempt didn't help, throw an exception - if (!select_parts(parts, alter_conversions)) + if (!select_parts(parts)) throw Exception(ErrorCodes::DUPLICATED_PART_UUIDS, "Found duplicate UUIDs while processing query."); } } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 788355c1e59..0d02456e480 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -40,7 +40,7 @@ public: /// The same as read, but with specified set of parts. QueryPlanStepPtr readFromParts( MergeTreeData::DataPartsVector parts, - std::vector alter_conversions, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info, @@ -120,7 +120,6 @@ private: /// as well as `max_block_number_to_read`. static void selectPartsToRead( MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, @@ -131,7 +130,6 @@ private: /// Same as previous but also skip parts uuids if any to the query context, or skip parts which uuids marked as excluded. static void selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids, const std::optional & minmax_idx_condition, @@ -175,10 +173,9 @@ public: /// Filter parts using minmax index and partition key. static void filterPartsByPartition( + MergeTreeData::DataPartsVector & parts, const std::optional & partition_pruner, const std::optional & minmax_idx_condition, - MergeTreeData::DataPartsVector & parts, - std::vector & alter_conversions, const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, @@ -192,7 +189,6 @@ public: /// If 'check_limits = true' it will throw exception if the amount of data exceed the limits from settings. static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, - std::vector && alter_conversions, StorageMetadataPtr metadata_snapshot, const ContextPtr & context, const KeyCondition & key_condition, diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 4dbccb91620..6f06b921031 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -6,6 +6,7 @@ #include #include #include +#include "Storages/MutationCommands.h" #include @@ -50,7 +51,7 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number, const TransactionID & tid_, const WriteSettings & settings) : create_time(time(nullptr)) - , commands(std::move(commands_)) + , commands(std::make_shared(std::move(commands_))) , disk(std::move(disk_)) , path_prefix(path_prefix_) , file_name("tmp_mutation_" + toString(tmp_number) + ".txt") @@ -63,7 +64,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP *out << "format version: 1\n" << "create time: " << LocalDateTime(create_time, DateLUT::serverTimezoneInstance()) << "\n"; *out << "commands: "; - commands.writeText(*out, /* with_pure_metadata_commands = */ false); + commands->writeText(*out, /* with_pure_metadata_commands = */ false); *out << "\n"; if (tid.isPrehistoric()) { @@ -116,7 +117,8 @@ void MergeTreeMutationEntry::writeCSN(CSN csn_) } MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & path_prefix_, const String & file_name_) - : disk(std::move(disk_)) + : commands(std::make_shared()) + , disk(std::move(disk_)) , path_prefix(path_prefix_) , file_name(file_name_) , is_temp(false) @@ -133,7 +135,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second()); *buf >> "commands: "; - commands.readText(*buf); + commands->readText(*buf); *buf >> "\n"; if (buf->eof()) @@ -177,7 +179,7 @@ std::shared_ptr MergeTreeMutationEntry::backup() const out << "block number: " << block_number << "\n"; out << "commands: "; - commands.writeText(out, /* with_pure_metadata_commands = */ false); + commands->writeText(out, /* with_pure_metadata_commands = */ false); out << "\n"; return std::make_shared(out.str()); diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index 04297f2852a..f41ad2a17f8 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -16,7 +16,7 @@ class IBackupEntry; struct MergeTreeMutationEntry { time_t create_time = 0; - MutationCommands commands; + std::shared_ptr commands; DiskPtr disk; String path_prefix; diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 2c249f7b63b..2aaf06fde7f 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -84,6 +84,7 @@ MergeTreeReadTask::Readers MergeTreePrefetchedReadPool::PrefetchedReaders::get() MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -94,6 +95,7 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index a3a57227630..65a7d62ad2d 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -19,6 +19,7 @@ class MergeTreePrefetchedReadPool : public MergeTreeReadPoolBase, private WithCo public: MergeTreePrefetchedReadPool( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index dc1ba030f45..d45e2e9c578 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -35,6 +35,7 @@ size_t getApproxSizeOfPart(const IMergeTreeDataPart & part, const Names & column MergeTreeReadPool::MergeTreeReadPool( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -45,6 +46,7 @@ MergeTreeReadPool::MergeTreeReadPool( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index cb0e8a9657f..d7b354a2799 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -26,6 +26,7 @@ public: MergeTreeReadPool( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 0ea19370d45..2935890cba5 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -13,6 +13,7 @@ namespace ErrorCodes MergeTreeReadPoolBase::MergeTreeReadPoolBase( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -22,6 +23,7 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( const PoolSettings & pool_settings_, const ContextPtr & context_) : parts_ranges(std::move(parts_)) + , mutations_snapshot(std::move(mutations_snapshot_)) , shared_virtual_fields(std::move(shared_virtual_fields_)) , storage_snapshot(storage_snapshot_) , prewhere_info(prewhere_info_) @@ -67,9 +69,9 @@ void MergeTreeReadPoolBase::fillPerPartInfos() } read_task_info.part_index_in_query = part_with_ranges.part_index_in_query; - read_task_info.alter_conversions = part_with_ranges.alter_conversions; + read_task_info.alter_conversions = MergeTreeData::getAlterConversionsForPart(part_with_ranges.data_part, mutations_snapshot); - LoadedMergeTreeDataPartInfoForReader part_info(part_with_ranges.data_part, part_with_ranges.alter_conversions); + LoadedMergeTreeDataPartInfoForReader part_info(part_with_ranges.data_part, read_task_info.alter_conversions); read_task_info.task_columns = getReadTaskColumns( part_info, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.h b/src/Storages/MergeTree/MergeTreeReadPoolBase.h index 1b5bfec5898..8286ff52a5c 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.h @@ -9,6 +9,8 @@ namespace DB class MergeTreeReadPoolBase : public IMergeTreeReadPool { public: + using MutationsSnapshotPtr = MergeTreeData::MutationsSnapshotPtr; + struct PoolSettings { size_t threads = 0; @@ -23,6 +25,7 @@ public: MergeTreeReadPoolBase( RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -37,6 +40,7 @@ public: protected: /// Initialized in constructor const RangesInDataParts parts_ranges; + const MutationsSnapshotPtr mutations_snapshot; const VirtualFields shared_virtual_fields; const StorageSnapshotPtr storage_snapshot; const PrewhereInfoPtr prewhere_info; diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp index 4c0391ffa57..60f127acdae 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp @@ -12,6 +12,7 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( bool has_limit_below_one_block_, MergeTreeReadType read_type_, RangesInDataParts parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -22,6 +23,7 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h index 9fedf396a6b..a3668acb170 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h @@ -11,6 +11,7 @@ public: bool has_limit_below_one_block_, MergeTreeReadType read_type_, RangesInDataParts parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index 38035d97f56..0d615fae443 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -13,6 +13,7 @@ namespace ErrorCodes MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( ParallelReadingExtension extension_, RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -23,6 +24,7 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h index ca159edb91c..d9d628b8be2 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h @@ -11,6 +11,7 @@ public: MergeTreeReadPoolParallelReplicas( ParallelReadingExtension extension_, RangesInDataParts && parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp index 01c0a9f91be..b1f9ffc8ea4 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp @@ -12,6 +12,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd ParallelReadingExtension extension_, CoordinationMode mode_, RangesInDataParts parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, @@ -22,6 +23,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), + std::move(mutations_snapshot_), std::move(shared_virtual_fields_), storage_snapshot_, prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h index 4fe3f7a699c..7c549ed3c4a 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h @@ -12,6 +12,7 @@ public: ParallelReadingExtension extension_, CoordinationMode mode_, RangesInDataParts parts_, + MutationsSnapshotPtr mutations_snapshot_, VirtualFields shared_virtual_fields_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 02f8d6f4f6a..bd4aa066dfd 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -13,6 +13,7 @@ #include #include #include +#include "Storages/MergeTree/AlterConversions.h" #include namespace DB @@ -38,6 +39,7 @@ public: const MergeTreeData & storage_, const StorageSnapshotPtr & storage_snapshot_, MergeTreeData::DataPartPtr data_part_, + AlterConversionsPtr alter_conversions_, Names columns_to_read_, std::optional mark_ranges_, bool apply_deleted_mask, @@ -62,6 +64,9 @@ private: /// Data part will not be removed if the pointer owns it MergeTreeData::DataPartPtr data_part; + /// TODO: comment. + AlterConversionsPtr alter_conversions; + /// Columns we have to read (each Block from read will contain them) Names columns_to_read; @@ -91,6 +96,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( const MergeTreeData & storage_, const StorageSnapshotPtr & storage_snapshot_, MergeTreeData::DataPartPtr data_part_, + AlterConversionsPtr alter_conversions_, Names columns_to_read_, std::optional mark_ranges_, bool apply_deleted_mask, @@ -100,6 +106,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( , storage(storage_) , storage_snapshot(storage_snapshot_) , data_part(std::move(data_part_)) + , alter_conversions(std::move(alter_conversions_)) , columns_to_read(std::move(columns_to_read_)) , read_with_direct_io(read_with_direct_io_) , mark_ranges(std::move(mark_ranges_)) @@ -113,8 +120,6 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( LOG_DEBUG(log, "Reading {} marks from part {}, total {} rows starting from the beginning of the part", data_part->getMarksCount(), data_part->name, data_part->rows_count); - auto alter_conversions = storage.getAlterConversionsForPart(data_part); - /// Note, that we don't check setting collaborate_with_coordinator presence, because this source /// is only used in background merges. addTotalRowsApprox(data_part->rows_count); @@ -300,6 +305,7 @@ Pipe createMergeTreeSequentialSource( const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, + AlterConversionsPtr alter_conversions, Names columns_to_read, std::optional mark_ranges, std::shared_ptr> filtered_rows_count, @@ -316,7 +322,8 @@ Pipe createMergeTreeSequentialSource( columns_to_read.emplace_back(RowExistsColumn::name); auto column_part_source = std::make_shared(type, - storage, storage_snapshot, data_part, columns_to_read, std::move(mark_ranges), + storage, storage_snapshot, data_part, alter_conversions, + columns_to_read, std::move(mark_ranges), /*apply_deleted_mask=*/ false, read_with_direct_io, prefetch); Pipe pipe(std::move(column_part_source)); @@ -347,6 +354,7 @@ public: const MergeTreeData & storage_, const StorageSnapshotPtr & storage_snapshot_, MergeTreeData::DataPartPtr data_part_, + AlterConversionsPtr alter_conversions_, Names columns_to_read_, bool apply_deleted_mask_, ActionsDAGPtr filter_, @@ -357,6 +365,7 @@ public: , storage(storage_) , storage_snapshot(storage_snapshot_) , data_part(std::move(data_part_)) + , alter_conversions(std::move(alter_conversions_)) , columns_to_read(std::move(columns_to_read_)) , apply_deleted_mask(apply_deleted_mask_) , filter(std::move(filter_)) @@ -400,6 +409,7 @@ public: storage, storage_snapshot, data_part, + alter_conversions, columns_to_read, std::move(mark_ranges), /*filtered_rows_count=*/ nullptr, @@ -415,6 +425,7 @@ private: const MergeTreeData & storage; StorageSnapshotPtr storage_snapshot; MergeTreeData::DataPartPtr data_part; + AlterConversionsPtr alter_conversions; Names columns_to_read; bool apply_deleted_mask; ActionsDAGPtr filter; @@ -428,6 +439,7 @@ void createReadFromPartStep( const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, + AlterConversionsPtr alter_conversions, Names columns_to_read, bool apply_deleted_mask, ActionsDAGPtr filter, @@ -435,7 +447,8 @@ void createReadFromPartStep( LoggerPtr log) { auto reading = std::make_unique(type, - storage, storage_snapshot, std::move(data_part), + storage, storage_snapshot, + std::move(data_part), std::move(alter_conversions), std::move(columns_to_read), apply_deleted_mask, filter, std::move(context), log); diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index e6f055f776c..1ecc721d4a8 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -21,6 +21,7 @@ Pipe createMergeTreeSequentialSource( const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, + AlterConversionsPtr alter_conversions, Names columns_to_read, std::optional mark_ranges, std::shared_ptr> filtered_rows_count, @@ -36,6 +37,7 @@ void createReadFromPartStep( const MergeTreeData & storage, const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, + AlterConversionsPtr alter_conversions, Names columns_to_read, bool apply_deleted_mask, ActionsDAGPtr filter, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a552ee89aee..3c4ef44dbd8 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -29,6 +29,7 @@ #include #include #include +#include "Storages/MergeTree/AlterConversions.h" #include @@ -104,6 +105,7 @@ static UInt64 getExistingRowsCount(const Block & block) static void splitAndModifyMutationCommands( MergeTreeData::DataPartPtr part, StorageMetadataPtr metadata_snapshot, + AlterConversionsPtr alter_conversions, const MutationCommands & commands, MutationCommands & for_interpreter, MutationCommands & for_file_renames, @@ -169,8 +171,6 @@ static void splitAndModifyMutationCommands( } - auto alter_conversions = part->storage.getAlterConversionsForPart(part); - /// We don't add renames from commands, instead we take them from rename_map. /// It's important because required renames depend not only on part's data version (i.e. mutation version) /// but also on part's metadata version. Why we have such logic only for renames? Because all other types of alter @@ -286,7 +286,6 @@ static void splitAndModifyMutationCommands( } } - auto alter_conversions = part->storage.getAlterConversionsForPart(part); /// We don't add renames from commands, instead we take them from rename_map. /// It's important because required renames depend not only on part's data version (i.e. mutation version) /// but also on part's metadata version. Why we have such logic only for renames? Because all other types of alter @@ -2119,6 +2118,14 @@ bool MutateTask::prepare() ctx->num_mutations = std::make_unique(CurrentMetrics::PartMutation); + auto mutations_snapshot = ctx->data->getMutationsSnapshot( + ctx->metadata_snapshot->getMetadataVersion(), + /*need_data_mutations=*/ false); + + auto alter_conversions = MergeTreeData::getAlterConversionsForPart( + ctx->source_part, + mutations_snapshot); + auto context_for_reading = Context::createCopy(ctx->context); /// Allow mutations to work when force_index_by_date or force_primary_key is on. @@ -2133,7 +2140,7 @@ bool MutateTask::prepare() ctx->commands_for_part.emplace_back(command); if (ctx->source_part->isStoredOnDisk() && !isStorageTouchedByMutations( - *ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->commands_for_part, context_for_reading)) + *ctx->data, ctx->source_part, mutations_snapshot, ctx->metadata_snapshot, ctx->commands_for_part, context_for_reading)) { NameSet files_to_copy_instead_of_hardlinks; auto settings_ptr = ctx->data->getSettings(); @@ -2192,8 +2199,13 @@ bool MutateTask::prepare() context_for_reading->setSetting("read_from_filesystem_cache_if_exists_otherwise_bypass_cache", 1); MutationHelpers::splitAndModifyMutationCommands( - ctx->source_part, ctx->metadata_snapshot, - ctx->commands_for_part, ctx->for_interpreter, ctx->for_file_renames, ctx->log); + ctx->source_part, + ctx->metadata_snapshot, + alter_conversions, + ctx->commands_for_part, + ctx->for_interpreter, + ctx->for_file_renames, + ctx->log); ctx->stage_progress = std::make_unique(1.0); @@ -2205,7 +2217,8 @@ bool MutateTask::prepare() settings.apply_deleted_mask = false; ctx->interpreter = std::make_unique( - *ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->for_interpreter, + *ctx->data, ctx->source_part, alter_conversions, + ctx->metadata_snapshot, ctx->for_interpreter, ctx->metadata_snapshot->getColumns().getNamesOfPhysical(), context_for_reading, settings); ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices(); diff --git a/src/Storages/MergeTree/RangesInDataPart.h b/src/Storages/MergeTree/RangesInDataPart.h index bf9e4c7dfb2..966637d0812 100644 --- a/src/Storages/MergeTree/RangesInDataPart.h +++ b/src/Storages/MergeTree/RangesInDataPart.h @@ -42,7 +42,6 @@ struct RangesInDataPartsDescription: public std::deque #include #include +#include +#include #include #include @@ -949,7 +951,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper { const auto commands = entry.commands; it = mutations_by_znode.erase(it); - updateAlterConversionsMutations(commands, alter_conversions_mutations, /* remove= */ true); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, lock); } else it = mutations_by_znode.erase(it); @@ -1001,7 +1003,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version)) .first->second; - updateAlterConversionsMutations(entry->commands, alter_conversions_mutations, /* remove= */ false); + incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); NOEXCEPT_SCOPE({ for (const auto & pair : entry->block_numbers) { @@ -1075,7 +1077,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation( } mutations_by_znode.erase(it); - /// updateAlterConversionsMutations() will be called in updateMutations() + /// decrementMutationsCounters() will be called in updateMutations() LOG_DEBUG(log, "Removed mutation {} from local state.", entry->znode_name); } @@ -1899,25 +1901,15 @@ ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue::getMergePredicate(zk return ReplicatedMergeTreeMergePredicate(*this, zookeeper, std::move(partition_ids_hint)); } - -MutationCommands ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const +MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const { - int32_t part_metadata_version = part->getMetadataVersion(); - int32_t metadata_version = storage.getInMemoryMetadataPtr()->getMetadataVersion(); - - chassert(alter_conversions_mutations >= 0); - /// NOTE: that just checking part_metadata_version is not enough, since we - /// need to check for non-metadata mutations as well. - if (alter_conversions_mutations == 0 && metadata_version == part_metadata_version) - return {}; - - std::unique_lock lock(state_mutex); - auto in_partition = mutations_by_partition.find(part->info.partition_id); if (in_partition == mutations_by_partition.end()) return {}; Int64 part_data_version = part->info.getDataVersion(); + int32_t part_metadata_version = part->getMetadataVersion(); + MutationCommands result; bool seen_all_data_mutations = false; @@ -1926,20 +1918,22 @@ MutationCommands ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const auto add_to_result = [&](const ReplicatedMergeTreeMutationEntryPtr & entry) { for (const auto & command : entry->commands | std::views::reverse) - if (AlterConversions::supportsMutationCommandType(command.type)) - result.emplace_back(command); + { + if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + result.push_back(command); + else if (AlterConversions::isSupportedMetadataMutation(command.type)) + result.push_back(command); + } }; /// Here we return mutation commands for part which has bigger alter version than part metadata version. /// Please note, we don't use getDataVersion(). It's because these alter commands are used for in-fly conversions /// of part's metadata. - for (const auto & [mutation_version, mutation_status] : in_partition->second | std::views::reverse) + for (const auto & [mutation_version, entry] : in_partition->second | std::views::reverse) { if (seen_all_data_mutations && seen_all_metadata_mutations) break; - auto & entry = mutation_status->entry; - auto alter_version = entry->alter_version; if (alter_version != -1) { @@ -1964,6 +1958,48 @@ MutationCommands ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const return result; } +MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const +{ + auto res = std::make_shared(); + res->metadata_version = metadata_version; + res->need_data_mutations = need_data_mutations; + + std::lock_guard lock(state_mutex); + + bool have_data_mutations = res->need_data_mutations && data_mutations_to_apply > 0; + bool have_metadata_mutations = metadata_mutations_to_apply > 0; + + if (!have_data_mutations && !have_metadata_mutations) + return res; + + for (const auto & [partition_id, mutations] : mutations_by_partition) + { + auto & in_partition = res->mutations_by_partition[partition_id]; + + for (const auto & [version, status] : mutations | std::views::reverse) + { + if (status->is_done) + break; + + bool has_required_command = std::ranges::any_of(status->entry->commands, [&](const auto & command) + { + if (have_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + return true; + + if (have_metadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) + return true; + + return false; + }); + + if (has_required_command) + in_partition.emplace(version, status->entry); + } + } + + return res; +} + MutationCommands ReplicatedMergeTreeQueue::getMutationCommands( const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version, Strings & mutation_ids) const { @@ -2044,7 +2080,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep mutation.parts_to_do.clear(); } - updateAlterConversionsMutations(mutation.entry->commands, alter_conversions_mutations, /* remove= */ true); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, mutation.entry->commands, lock); } else if (mutation.parts_to_do.size() == 0) { @@ -2101,7 +2137,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep LOG_TRACE(log, "Finishing data alter with version {} for entry {}", entry->alter_version, entry->znode_name); alter_sequence.finishDataAlter(entry->alter_version, lock); } - updateAlterConversionsMutations(entry->commands, alter_conversions_mutations, /* remove= */ true); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); } } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 89ef6240558..f9d5487ee3f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -151,8 +152,11 @@ private: /// Mapping from znode path to Mutations Status std::map mutations_by_znode; - /// Unfinished mutations that is required AlterConversions (see getAlterMutationCommandsForPart()) - std::atomic alter_conversions_mutations = 0; + + /// Unfinished mutations that are required for AlterConversions. + Int64 data_mutations_to_apply = 0; + Int64 metadata_mutations_to_apply = 0; + /// Partition -> (block_number -> MutationStatus) std::unordered_map> mutations_by_partition; /// Znode ID of the latest mutation that is done. @@ -409,10 +413,24 @@ public: MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version, Strings & mutation_ids) const; + struct MutationsSnapshot : public MergeTreeData::IMutationsSnapshot + { + MutationsSnapshot() = default; + + Int64 metadata_version = -1; + bool need_data_mutations = false; + + using MutationsByPartititon = std::unordered_map>; + MutationsByPartititon mutations_by_partition; + + MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const override; + std::shared_ptr cloneEmpty() const override { return std::make_shared(); } + }; + /// Return mutation commands for part which could be not applied to /// it according to part mutation version. Used when we apply alter commands on fly, /// without actual data modification on disk. - MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const; + MergeTreeData::MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const; /// Mark finished mutations as done. If the function needs to be called again at some later time /// (because some mutations are probably done but we are not sure yet), returns true. diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index a94508ad41f..f871157c2c9 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -26,10 +26,12 @@ class StorageFromMergeTreeDataPart final : public IStorage { public: /// Used in part mutation. - explicit StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) + explicit StorageFromMergeTreeDataPart( + const MergeTreeData::DataPartPtr & part_, + const MergeTreeData::MutationsSnapshotPtr & mutations_snapshot_) : IStorage(getIDFromPart(part_)) , parts({part_}) - , alter_conversions({part_->storage.getAlterConversionsForPart(part_)}) + , mutations_snapshot(mutations_snapshot_) , storage(part_->storage) , partition_id(part_->info.partition_id) { @@ -71,10 +73,11 @@ public: size_t max_block_size, size_t num_streams) override { + /// TODO: fix query_plan.addStep(MergeTreeDataSelectExecutor(storage) .readFromParts( parts, - alter_conversions, + mutations_snapshot, column_names, storage_snapshot, query_info, @@ -121,7 +124,7 @@ public: private: const MergeTreeData::DataPartsVector parts; - const std::vector alter_conversions; + const MergeTreeData::MutationsSnapshotPtr mutations_snapshot; const MergeTreeData & storage; const String partition_id; const ReadFromMergeTree::AnalysisResultPtr analysis_result_ptr; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9255ee00340..636d2ba5d53 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -498,18 +498,11 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context if (txn) txn->addMutation(shared_from_this(), mutation_id); - bool alter_conversions_mutations_updated = updateAlterConversionsMutations(entry.commands, alter_conversions_mutations, /* remove= */ false); - bool inserted = current_mutations_by_version.try_emplace(version, std::move(entry)).second; + auto [it, inserted] = current_mutations_by_version.try_emplace(version, std::move(entry)); if (!inserted) - { - if (alter_conversions_mutations_updated) - { - --alter_conversions_mutations; - chassert(alter_conversions_mutations >= 0); - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); - } + incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info); } background_operations_assignee.trigger(); @@ -545,7 +538,7 @@ void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr re if (static_cast(result_part->part_info.mutation) == it->first) mutation_backoff_policy.removePartFromFailed(failed_part->name); - updateAlterConversionsMutations(it->second.commands, alter_conversions_mutations, /* remove= */ true); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry.commands, lock); } } else @@ -744,17 +737,15 @@ std::map StorageMergeTree::getUnfinishedMutationC std::map result; - for (const auto & kv : current_mutations_by_version) + for (const auto & [mutation_version, entry] : current_mutations_by_version) { - Int64 mutation_version = kv.first; - const MergeTreeMutationEntry & entry = kv.second; - const PartVersionWithName needle{mutation_version, ""}; + const PartVersionWithName needle{static_cast(mutation_version), ""}; auto versions_it = std::lower_bound( part_versions_with_names.begin(), part_versions_with_names.end(), needle, comparator); size_t parts_to_do = versions_it - part_versions_with_names.begin(); if (parts_to_do > 0) - result.emplace(entry.file_name, entry.commands); + result.emplace(entry.file_name, *entry.commands); } return result; } @@ -787,7 +778,7 @@ std::vector StorageMergeTree::getMutationsStatus() cons std::map block_numbers_map({{"", entry.block_number}}); - for (const MutationCommand & command : entry.commands) + for (const MutationCommand & command : *entry.commands) { WriteBufferFromOwnString buf; formatAST(*command.ast, buf, false, true); @@ -824,20 +815,15 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) auto it = current_mutations_by_version.find(mutation_version); if (it != current_mutations_by_version.end()) { - bool mutation_finished = true; if (std::optional min_version = getMinPartDataVersion()) - mutation_finished = *min_version > static_cast(mutation_version); + { + bool mutation_finished = *min_version > static_cast(mutation_version); + if (!mutation_finished) + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); + } to_kill.emplace(std::move(it->second)); - - if (!mutation_finished) - { - const auto commands = it->second.commands; - current_mutations_by_version.erase(it); - updateAlterConversionsMutations(commands, alter_conversions_mutations, /* remove= */ true); - } - else - current_mutations_by_version.erase(it); + current_mutations_by_version.erase(it); } } @@ -885,6 +871,8 @@ void StorageMergeTree::loadDeduplicationLog() void StorageMergeTree::loadMutations() { + std::lock_guard lock(currently_processing_in_background_mutex); + for (const auto & disk : getDisks()) { for (auto it = disk->iterateDirectory(relative_data_path); it->isValid(); it->next()) @@ -893,7 +881,7 @@ void StorageMergeTree::loadMutations() { MergeTreeMutationEntry entry(disk, relative_data_path, it->name()); UInt64 block_number = entry.block_number; - LOG_DEBUG(log, "Loading mutation: {} entry, commands size: {}", it->name(), entry.commands.size()); + LOG_DEBUG(log, "Loading mutation: {} entry, commands size: {}", it->name(), entry.commands->size()); if (!entry.tid.isPrehistoric() && !entry.csn) { @@ -912,10 +900,11 @@ void StorageMergeTree::loadMutations() } } - auto inserted = current_mutations_by_version.try_emplace(block_number, std::move(entry)).second; + auto [entry_it, inserted] = current_mutations_by_version.try_emplace(block_number, std::move(entry)); if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number); - updateAlterConversionsMutations(entry.commands, alter_conversions_mutations, /* remove= */ false); + + incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry_it->second.commands, lock); } else if (startsWith(it->name(), "tmp_mutation_")) { @@ -1264,7 +1253,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( size_t commands_size = 0; MutationCommands commands_for_size_validation; - for (const auto & command : it->second.commands) + for (const auto & command : *it->second.commands) { if (command.type != MutationCommand::Type::DROP_COLUMN && command.type != MutationCommand::Type::DROP_INDEX @@ -1308,11 +1297,11 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( const auto & single_mutation_commands = it->second.commands; - if (single_mutation_commands.containBarrierCommand()) + if (single_mutation_commands->containBarrierCommand()) { if (commands->empty()) { - commands->insert(commands->end(), single_mutation_commands.begin(), single_mutation_commands.end()); + commands->insert(commands->end(), single_mutation_commands->begin(), single_mutation_commands->end()); last_mutation_to_apply = it; } break; @@ -1320,7 +1309,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( else { current_ast_elements += commands_size; - commands->insert(commands->end(), single_mutation_commands.begin(), single_mutation_commands.end()); + commands->insert(commands->end(), single_mutation_commands->begin(), single_mutation_commands->end()); last_mutation_to_apply = it; } @@ -2431,34 +2420,62 @@ void StorageMergeTree::attachRestoredParts(MutableDataPartsVector && parts) } } - -MutationCommands StorageMergeTree::getAlterMutationCommandsForPart(const DataPartPtr & part) const +MutationCommands StorageMergeTree::MutationsSnapshot::getAlterMutationCommandsForPart(const DataPartPtr & part) const { - /// NOTE: there is no need to check part metadata_version, since - /// ALTER_METADATA cannot be done asynchronously, like in - /// ReplicatedMergeTree. - chassert(alter_conversions_mutations >= 0); - if (alter_conversions_mutations == 0) - return {}; - - std::lock_guard lock(currently_processing_in_background_mutex); - - UInt64 part_data_version = part->info.getDataVersion(); MutationCommands result; + UInt64 part_data_version = part->info.getDataVersion(); - for (const auto & [mutation_version, entry] : current_mutations_by_version | std::views::reverse) + for (const auto & [mutation_version, commands] : mutations_by_version | std::views::reverse) { if (mutation_version <= part_data_version) break; - for (const auto & command : entry.commands | std::views::reverse) - if (AlterConversions::supportsMutationCommandType(command.type)) - result.emplace_back(command); + for (const auto & command : *commands | std::views::reverse) + { + if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + result.push_back(command); + else if (AlterConversions::isSupportedMetadataMutation(command.type)) + result.push_back(command); + } } return result; } +MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const +{ + auto res = std::make_shared(); + res->metadata_version = metadata_version; + res->need_data_mutations = need_data_mutations; + + std::lock_guard lock(currently_processing_in_background_mutex); + + bool have_data_mutations = res->need_data_mutations && data_mutations_to_apply > 0; + bool have_metadata_mutations = metadata_mutations_to_apply > 0; + + if (!have_data_mutations && !have_metadata_mutations) + return res; + + for (const auto & [version, entry] : current_mutations_by_version) + { + bool has_required_command = std::ranges::any_of(*entry.commands, [&](const auto & command) + { + if (have_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + return true; + + if (have_metadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) + return true; + + return false; + }); + + if (has_required_command) + res->mutations_by_version.emplace(version, entry.commands); + } + + return res; +} + void StorageMergeTree::startBackgroundMovesIfNeeded() { if (areBackgroundMovesNeeded()) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 4d819508934..1f2af8b9571 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -17,6 +17,7 @@ #include #include +#include "Storages/MutationCommands.h" namespace DB @@ -147,8 +148,10 @@ private: DataParts currently_merging_mutating_parts; std::map current_mutations_by_version; - /// Unfinished mutations that is required AlterConversions (see getAlterMutationCommandsForPart()) - std::atomic alter_conversions_mutations = 0; + + /// Unfinished mutations that are required for AlterConversions. + Int64 data_mutations_to_apply = 0; + Int64 metadata_mutations_to_apply = 0; std::atomic shutdown_called {false}; std::atomic flush_called {false}; @@ -309,8 +312,21 @@ private: ContextPtr context; }; -protected: - MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const override; + struct MutationsSnapshot : public IMutationsSnapshot + { + MutationsSnapshot() = default; + + Int64 metadata_version = -1; + bool need_data_mutations = false; + + using MutationsByVersion = std::map>; + MutationsByVersion mutations_by_version; + + MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const override; + std::shared_ptr cloneEmpty() const override { return std::make_shared(); } + }; + + MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const override; }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a127384c03c..e2bba1e8068 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -9149,13 +9149,11 @@ bool StorageReplicatedMergeTree::canUseAdaptiveGranularity() const (!has_non_adaptive_index_granularity_parts && !other_replicas_fixed_granularity)); } - -MutationCommands StorageReplicatedMergeTree::getAlterMutationCommandsForPart(const DataPartPtr & part) const +MergeTreeData::MutationsSnapshotPtr StorageReplicatedMergeTree::getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const { - return queue.getAlterMutationCommandsForPart(part); + return queue.getMutationsSnapshot(metadata_version, need_data_mutations); } - void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() { if (areBackgroundMovesNeeded()) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index f96206ce657..3ef367d09ce 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -932,7 +932,7 @@ private: void waitMutationToFinishOnReplicas( const Strings & replicas, const String & mutation_id) const; - MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const override; + MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const override; void startBackgroundMovesIfNeeded() override; From cb884d0ac7461499badc169380b9136941258693 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 3 Jul 2024 15:35:52 +0000 Subject: [PATCH 007/259] fix applying of metadata mutations --- src/Storages/MergeTree/AlterConversions.cpp | 1 + src/Storages/MergeTree/MergeTask.cpp | 10 +- src/Storages/MergeTree/MergeTreeData.cpp | 72 ++++++------ src/Storages/MergeTree/MergeTreeData.h | 34 +++--- src/Storages/MergeTree/MutateTask.cpp | 13 ++- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 103 ++++++++++++------ .../MergeTree/ReplicatedMergeTreeQueue.h | 11 +- src/Storages/StorageMergeTree.cpp | 26 ++--- src/Storages/StorageMergeTree.h | 9 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- 11 files changed, 158 insertions(+), 127 deletions(-) diff --git a/src/Storages/MergeTree/AlterConversions.cpp b/src/Storages/MergeTree/AlterConversions.cpp index 82bef500b34..a36611e3d87 100644 --- a/src/Storages/MergeTree/AlterConversions.cpp +++ b/src/Storages/MergeTree/AlterConversions.cpp @@ -11,6 +11,7 @@ namespace ErrorCodes bool AlterConversions::isSupportedDataMutation(MutationCommand::Type) { + /// Currently there is no such mutations. See setting 'apply_mutations_on_fly'. return false; } diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 5dab0cd0c08..08e6f654f15 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -257,9 +257,13 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() if (enabledBlockOffsetColumn(global_ctx)) addGatheringColumn(global_ctx, BlockOffsetColumn::name, BlockOffsetColumn::type); - auto mutations_snapshot = global_ctx->data->getMutationsSnapshot( - global_ctx->metadata_snapshot->getMetadataVersion(), - /*need_data_mutations=*/ false); + MergeTreeData::IMutationsSnapshot::Params params + { + .metadata_version = global_ctx->metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = MergeTreeData::getMinMetadataVersion(global_ctx->future_part->parts), + }; + + auto mutations_snapshot = global_ctx->data->getMutationsSnapshot(params); SerializationInfo::Settings info_settings = { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 98c308f5fd1..d06570c3ed8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8418,6 +8418,18 @@ bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr &, return !hasLightweightDeletedMask(); } +Int64 MergeTreeData::getMinMetadataVersion(const DataPartsVector & parts) +{ + Int64 version = -1; + for (const auto & part : parts) + { + Int64 part_version = part->getMetadataVersion(); + if (version == -1 || part_version < version) + version = part_version; + } + return version; +} + StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const { auto snapshot_data = std::make_unique(); @@ -8429,10 +8441,14 @@ StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & object_columns_copy = object_columns; } - snapshot_data->mutations_snapshot = getMutationsSnapshot( - metadata_snapshot->getMetadataVersion(), - query_context->getSettingsRef().apply_mutations_on_fly); + IMutationsSnapshot::Params params + { + .metadata_version = metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = getMinMetadataVersion(snapshot_data->parts), + .need_data_mutations = query_context->getSettingsRef().apply_mutations_on_fly, + }; + snapshot_data->mutations_snapshot = getMutationsSnapshot(params); return std::make_shared(*this, metadata_snapshot, std::move(object_columns_copy), std::move(snapshot_data)); } @@ -8618,59 +8634,33 @@ void MergeTreeData::verifySortingKey(const KeyDescription & sorting_key) } } -static void updateMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, - const MutationCommands & commands, - Int64 increment) +static void updateAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, Int64 increment) { - if (data_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); - - if (metadata_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); - - bool has_data_mutation = false; - bool has_metadata_mutation = false; + if (num_alter_conversions < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data alter conversions counter is negative ({})", num_alter_conversions); for (const auto & command : commands) { - if (!has_data_mutation && AlterConversions::isSupportedDataMutation(command.type)) + if (AlterConversions::isSupportedDataMutation(command.type) || AlterConversions::isSupportedMetadataMutation(command.type)) { - data_mutations_to_apply += increment; - has_data_mutation = true; + num_alter_conversions += increment; - if (data_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); - } + if (num_alter_conversions < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", num_alter_conversions); - if (!has_metadata_mutation && AlterConversions::isSupportedMetadataMutation(command.type)) - { - metadata_mutations_to_apply += increment; - has_metadata_mutation = true; - - if (metadata_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); + return; } } } -void incrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & /*lock*/) +void incrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & /*lock*/) { - return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, 1); + updateAlterConversionsCounter(num_alter_conversions, commands, 1); } -void decrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & /*lock*/) +void decrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & /*lock*/) { - return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, -1); + updateAlterConversionsCounter(num_alter_conversions, commands, -1); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 765b68b7559..faf55292257 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -448,6 +448,21 @@ public: struct IMutationsSnapshot { + struct Params + { + Int64 metadata_version = -1; + Int64 min_part_metadata_version = -1; + bool need_data_mutations = false; + + bool needAnyMutations() const { return need_data_mutations || needMetadataMutations(); } + bool needMetadataMutations() const { return min_part_metadata_version < metadata_version; } + }; + + Params params; + + IMutationsSnapshot() = default; + explicit IMutationsSnapshot(Params params_) : params(std::move(params_)) {} + /// Return pending mutations that weren't applied to `part` yet and should be applied on the fly /// (i.e. when reading from the part). Mutations not supported by AlterConversions /// (supportsMutationCommandType()) can be omitted. @@ -455,7 +470,6 @@ public: /// @return list of mutations, in *reverse* order (newest to oldest) virtual MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; virtual std::shared_ptr cloneEmpty() const = 0; - virtual ~IMutationsSnapshot() = default; }; @@ -951,7 +965,10 @@ public: Disks getDisks() const { return getStoragePolicy()->getDisks(); } /// TODO: comment - virtual MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const = 0; + virtual MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const = 0; + + /// TODO: comment + static Int64 getMinMetadataVersion(const DataPartsVector & parts); /// Return alter conversions for part which must be applied on fly. static AlterConversionsPtr getAlterConversionsForPart( @@ -1752,16 +1769,7 @@ struct CurrentlySubmergingEmergingTagger /// Look at MutationCommands if it contains mutations for AlterConversions, update the counter. /// Return true if the counter had been updated -void incrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & lock); - -void decrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & lock); +void incrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & lock); +void decrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & lock); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 3c4ef44dbd8..4a9138c10da 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2118,13 +2118,14 @@ bool MutateTask::prepare() ctx->num_mutations = std::make_unique(CurrentMetrics::PartMutation); - auto mutations_snapshot = ctx->data->getMutationsSnapshot( - ctx->metadata_snapshot->getMetadataVersion(), - /*need_data_mutations=*/ false); + MergeTreeData::IMutationsSnapshot::Params params + { + .metadata_version = ctx->metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = ctx->source_part->getMetadataVersion(), + }; - auto alter_conversions = MergeTreeData::getAlterConversionsForPart( - ctx->source_part, - mutations_snapshot); + auto mutations_snapshot = ctx->data->getMutationsSnapshot(params); + auto alter_conversions = MergeTreeData::getAlterConversionsForPart(ctx->source_part, mutations_snapshot); auto context_for_reading = Context::createCopy(ctx->context); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index fbf949b47f5..807fbeebfc4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -951,7 +951,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper { const auto commands = entry.commands; it = mutations_by_znode.erase(it); - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, lock); + decrementAlterConversionsCounter(num_alter_conversions, commands, state_lock); } else it = mutations_by_znode.erase(it); @@ -1000,10 +1000,9 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper for (const ReplicatedMergeTreeMutationEntryPtr & entry : new_mutations) { - auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version)) - .first->second; + auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version)).first->second; + incrementAlterConversionsCounter(num_alter_conversions, entry->commands, lock); - incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); NOEXCEPT_SCOPE({ for (const auto & pair : entry->block_numbers) { @@ -1077,7 +1076,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation( } mutations_by_znode.erase(it); - /// decrementMutationsCounters() will be called in updateMutations() + /// decrementAlterConversionsCounter() will be called in updateMutations() LOG_DEBUG(log, "Removed mutation {} from local state.", entry->znode_name); } @@ -1901,6 +1900,7 @@ ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue::getMergePredicate(zk return ReplicatedMergeTreeMergePredicate(*this, zookeeper, std::move(partition_ids_hint)); } + MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const { auto in_partition = mutations_by_partition.find(part->info.partition_id); @@ -1908,20 +1908,23 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo return {}; Int64 part_data_version = part->info.getDataVersion(); - int32_t part_metadata_version = part->getMetadataVersion(); + Int64 part_metadata_version = part->getMetadataVersion(); MutationCommands result; - bool seen_all_data_mutations = false; - bool seen_all_metadata_mutations = false; + bool seen_all_data_mutations = !params.need_data_mutations; + bool seen_all_metadata_mutations = !params.needMetadataMutations(); + + if (seen_all_data_mutations && seen_all_metadata_mutations) + return {}; auto add_to_result = [&](const ReplicatedMergeTreeMutationEntryPtr & entry) { for (const auto & command : entry->commands | std::views::reverse) { - if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + if (AlterConversions::isSupportedMetadataMutation(command.type)) result.push_back(command); - else if (AlterConversions::isSupportedMetadataMutation(command.type)) + else if (params.need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) result.push_back(command); } }; @@ -1935,9 +1938,10 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo break; auto alter_version = entry->alter_version; - if (alter_version != -1) + + if (!seen_all_metadata_mutations && alter_version != -1) { - if (alter_version > metadata_version) + if (alter_version > params.metadata_version) continue; /// We take commands with bigger metadata version @@ -1946,7 +1950,7 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo else seen_all_metadata_mutations = true; } - else + else if (!seen_all_data_mutations) { if (mutation_version > part_data_version) add_to_result(entry); @@ -1958,42 +1962,71 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo return result; } -MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const +MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapshot(const MutationsSnapshot::Params & params) const { - auto res = std::make_shared(); - res->metadata_version = metadata_version; - res->need_data_mutations = need_data_mutations; + auto res = std::make_shared(params); std::lock_guard lock(state_mutex); - bool have_data_mutations = res->need_data_mutations && data_mutations_to_apply > 0; - bool have_metadata_mutations = metadata_mutations_to_apply > 0; + bool need_data_mutations = res->params.need_data_mutations && num_alter_conversions > 0; + bool need_metatadata_mutations = res->params.needMetadataMutations(); - if (!have_data_mutations && !have_metadata_mutations) + if (!need_data_mutations && !need_metatadata_mutations) return res; + auto is_supported_command = [&](const auto & command) + { + if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + return true; + + if (need_metatadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) + return true; + + return false; + }; + for (const auto & [partition_id, mutations] : mutations_by_partition) { auto & in_partition = res->mutations_by_partition[partition_id]; - for (const auto & [version, status] : mutations | std::views::reverse) + bool seen_all_data_mutations = !need_data_mutations; + bool seen_all_metadata_mutations = !need_metatadata_mutations; + + for (const auto & [mutation_version, status] : mutations | std::views::reverse) { - if (status->is_done) + if (seen_all_data_mutations && seen_all_metadata_mutations) break; - bool has_required_command = std::ranges::any_of(status->entry->commands, [&](const auto & command) + auto alter_version = status->entry->alter_version; + + if (!seen_all_metadata_mutations && alter_version != -1) { - if (have_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) - return true; + if (alter_version > params.metadata_version) + continue; - if (have_metadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) - return true; - - return false; - }); - - if (has_required_command) - in_partition.emplace(version, status->entry); + /// We take commands with bigger metadata version + if (alter_version > params.min_part_metadata_version) + { + if (std::ranges::any_of(status->entry->commands, is_supported_command)) + in_partition.emplace(mutation_version, status->entry); + } + else + { + seen_all_metadata_mutations = true; + } + } + else if (!seen_all_data_mutations) + { + if (!status->is_done) + { + if (std::ranges::any_of(status->entry->commands, is_supported_command)) + in_partition.emplace(mutation_version, status->entry); + } + else + { + seen_all_data_mutations = true; + } + } } } @@ -2080,7 +2113,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep mutation.parts_to_do.clear(); } - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, mutation.entry->commands, lock); + decrementAlterConversionsCounter(num_alter_conversions, mutation.entry->commands, lock); } else if (mutation.parts_to_do.size() == 0) { @@ -2137,7 +2170,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep LOG_TRACE(log, "Finishing data alter with version {} for entry {}", entry->alter_version, entry->znode_name); alter_sequence.finishDataAlter(entry->alter_version, lock); } - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); + decrementAlterConversionsCounter(num_alter_conversions, entry->commands, lock); } } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index f9d5487ee3f..954e2fd951e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -154,8 +154,7 @@ private: std::map mutations_by_znode; /// Unfinished mutations that are required for AlterConversions. - Int64 data_mutations_to_apply = 0; - Int64 metadata_mutations_to_apply = 0; + Int64 num_alter_conversions; /// Partition -> (block_number -> MutationStatus) std::unordered_map> mutations_by_partition; @@ -416,11 +415,11 @@ public: struct MutationsSnapshot : public MergeTreeData::IMutationsSnapshot { MutationsSnapshot() = default; + explicit MutationsSnapshot(Params params_) : IMutationsSnapshot(std::move(params_)) {} - Int64 metadata_version = -1; - bool need_data_mutations = false; - + using Params = MergeTreeData::IMutationsSnapshot::Params; using MutationsByPartititon = std::unordered_map>; + MutationsByPartititon mutations_by_partition; MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const override; @@ -430,7 +429,7 @@ public: /// Return mutation commands for part which could be not applied to /// it according to part mutation version. Used when we apply alter commands on fly, /// without actual data modification on disk. - MergeTreeData::MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const; + MergeTreeData::MutationsSnapshotPtr getMutationsSnapshot(const MutationsSnapshot::Params & params) const; /// Mark finished mutations as done. If the function needs to be called again at some later time /// (because some mutations are probably done but we are not sure yet), returns true. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 636d2ba5d53..da90ffddc8d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -502,7 +502,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); - incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); + incrementAlterConversionsCounter(num_alter_conversions, *it->second.commands, lock); LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info); } background_operations_assignee.trigger(); @@ -538,7 +538,7 @@ void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr re if (static_cast(result_part->part_info.mutation) == it->first) mutation_backoff_policy.removePartFromFailed(failed_part->name); - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry.commands, lock); + decrementAlterConversionsCounter(num_alter_conversions, *entry.commands, lock); } } else @@ -819,7 +819,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) { bool mutation_finished = *min_version > static_cast(mutation_version); if (!mutation_finished) - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); + decrementAlterConversionsCounter(num_alter_conversions, *it->second.commands, lock); } to_kill.emplace(std::move(it->second)); @@ -904,7 +904,7 @@ void StorageMergeTree::loadMutations() if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number); - incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry_it->second.commands, lock); + incrementAlterConversionsCounter(num_alter_conversions, *entry_it->second.commands, lock); } else if (startsWith(it->name(), "tmp_mutation_")) { @@ -2432,7 +2432,7 @@ MutationCommands StorageMergeTree::MutationsSnapshot::getAlterMutationCommandsFo for (const auto & command : *commands | std::views::reverse) { - if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + if (params.need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) result.push_back(command); else if (AlterConversions::isSupportedMetadataMutation(command.type)) result.push_back(command); @@ -2442,28 +2442,26 @@ MutationCommands StorageMergeTree::MutationsSnapshot::getAlterMutationCommandsFo return result; } -MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const +MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(const IMutationsSnapshot::Params & params) const { - auto res = std::make_shared(); - res->metadata_version = metadata_version; - res->need_data_mutations = need_data_mutations; + auto res = std::make_shared(params); std::lock_guard lock(currently_processing_in_background_mutex); - bool have_data_mutations = res->need_data_mutations && data_mutations_to_apply > 0; - bool have_metadata_mutations = metadata_mutations_to_apply > 0; + bool need_data_mutations = res->params.need_data_mutations && num_alter_conversions > 0; + bool need_metatadata_mutations = res->params.needMetadataMutations(); - if (!have_data_mutations && !have_metadata_mutations) + if (!need_data_mutations && !need_metatadata_mutations) return res; for (const auto & [version, entry] : current_mutations_by_version) { bool has_required_command = std::ranges::any_of(*entry.commands, [&](const auto & command) { - if (have_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) + if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) return true; - if (have_metadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) + if (need_metatadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) return true; return false; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 1f2af8b9571..d05c6739738 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -150,8 +150,7 @@ private: std::map current_mutations_by_version; /// Unfinished mutations that are required for AlterConversions. - Int64 data_mutations_to_apply = 0; - Int64 metadata_mutations_to_apply = 0; + Int64 num_alter_conversions; std::atomic shutdown_called {false}; std::atomic flush_called {false}; @@ -315,9 +314,7 @@ private: struct MutationsSnapshot : public IMutationsSnapshot { MutationsSnapshot() = default; - - Int64 metadata_version = -1; - bool need_data_mutations = false; + explicit MutationsSnapshot(Params params_) : IMutationsSnapshot(std::move(params_)) {} using MutationsByVersion = std::map>; MutationsByVersion mutations_by_version; @@ -326,7 +323,7 @@ private: std::shared_ptr cloneEmpty() const override { return std::make_shared(); } }; - MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const override; + MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const override; }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e2bba1e8068..baca8cb2695 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -9149,9 +9149,9 @@ bool StorageReplicatedMergeTree::canUseAdaptiveGranularity() const (!has_non_adaptive_index_granularity_parts && !other_replicas_fixed_granularity)); } -MergeTreeData::MutationsSnapshotPtr StorageReplicatedMergeTree::getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const +MergeTreeData::MutationsSnapshotPtr StorageReplicatedMergeTree::getMutationsSnapshot(const IMutationsSnapshot::Params & params) const { - return queue.getMutationsSnapshot(metadata_version, need_data_mutations); + return queue.getMutationsSnapshot(params); } void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 3ef367d09ce..cd12cfd3c02 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -932,7 +932,7 @@ private: void waitMutationToFinishOnReplicas( const Strings & replicas, const String & mutation_id) const; - MutationsSnapshotPtr getMutationsSnapshot(Int64 metadata_version, bool need_data_mutations) const override; + MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const override; void startBackgroundMovesIfNeeded() override; From 826f6c1ce59893e0e1787a207a182c2eb012c7ae Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 4 Jul 2024 17:56:08 +0000 Subject: [PATCH 008/259] fix initialization --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.h | 2 +- src/Storages/StorageMergeTree.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 954e2fd951e..223ca989feb 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -154,7 +154,7 @@ private: std::map mutations_by_znode; /// Unfinished mutations that are required for AlterConversions. - Int64 num_alter_conversions; + Int64 num_alter_conversions = 0; /// Partition -> (block_number -> MutationStatus) std::unordered_map> mutations_by_partition; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index d05c6739738..40e5a8e5ea4 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -150,7 +150,7 @@ private: std::map current_mutations_by_version; /// Unfinished mutations that are required for AlterConversions. - Int64 num_alter_conversions; + Int64 num_alter_conversions = 0; std::atomic shutdown_called {false}; std::atomic flush_called {false}; From 7e790d80845fcb3ae955a773c22ab7a96a595d4b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 8 Jul 2024 13:25:51 +0000 Subject: [PATCH 009/259] fix rename --- src/Storages/MergeTree/MergeTreeData.cpp | 50 ++++++++++++++----- src/Storages/MergeTree/MergeTreeData.h | 16 ++++-- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 16 +++--- .../MergeTree/ReplicatedMergeTreeQueue.h | 3 +- src/Storages/StorageMergeTree.cpp | 16 +++--- src/Storages/StorageMergeTree.h | 3 +- 6 files changed, 69 insertions(+), 35 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6083a441a35..d9529b6870c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8668,33 +8668,59 @@ void MergeTreeData::verifySortingKey(const KeyDescription & sorting_key) } } -static void updateAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, Int64 increment) +static void updateMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + Int64 increment) { - if (num_alter_conversions < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data alter conversions counter is negative ({})", num_alter_conversions); + if (data_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); + + if (metadata_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); + + bool has_data_mutation = false; + bool has_metadata_mutation = false; for (const auto & command : commands) { - if (AlterConversions::isSupportedDataMutation(command.type) || AlterConversions::isSupportedMetadataMutation(command.type)) + if (!has_data_mutation && AlterConversions::isSupportedDataMutation(command.type)) { - num_alter_conversions += increment; + data_mutations_to_apply += increment; + has_data_mutation = true; - if (num_alter_conversions < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", num_alter_conversions); + if (data_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); + } - return; + if (!has_metadata_mutation && AlterConversions::isSupportedMetadataMutation(command.type)) + { + metadata_mutations_to_apply += increment; + has_metadata_mutation = true; + + if (metadata_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); } } } -void incrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & /*lock*/) +void incrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & /*lock*/) { - updateAlterConversionsCounter(num_alter_conversions, commands, 1); + return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, 1); } -void decrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & /*lock*/) +void decrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & /*lock*/) { - updateAlterConversionsCounter(num_alter_conversions, commands, -1); + return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, -1); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index b0773cb4a20..75a7553d157 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -453,9 +453,6 @@ public: Int64 metadata_version = -1; Int64 min_part_metadata_version = -1; bool need_data_mutations = false; - - bool needAnyMutations() const { return need_data_mutations || needMetadataMutations(); } - bool needMetadataMutations() const { return min_part_metadata_version < metadata_version; } }; Params params; @@ -1776,7 +1773,16 @@ struct CurrentlySubmergingEmergingTagger /// Look at MutationCommands if it contains mutations for AlterConversions, update the counter. /// Return true if the counter had been updated -void incrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & lock); -void decrementAlterConversionsCounter(Int64 & num_alter_conversions, const MutationCommands & commands, std::lock_guard & lock); +void incrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & lock); + +void decrementMutationsCounters( + Int64 & data_mutations_to_apply, + Int64 & metadata_mutations_to_apply, + const MutationCommands & commands, + std::lock_guard & lock); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 807fbeebfc4..0175e427079 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -951,7 +951,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper { const auto commands = entry.commands; it = mutations_by_znode.erase(it); - decrementAlterConversionsCounter(num_alter_conversions, commands, state_lock); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, state_lock); } else it = mutations_by_znode.erase(it); @@ -1001,7 +1001,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper for (const ReplicatedMergeTreeMutationEntryPtr & entry : new_mutations) { auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version)).first->second; - incrementAlterConversionsCounter(num_alter_conversions, entry->commands, lock); + incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); NOEXCEPT_SCOPE({ for (const auto & pair : entry->block_numbers) @@ -1076,7 +1076,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation( } mutations_by_znode.erase(it); - /// decrementAlterConversionsCounter() will be called in updateMutations() + /// decrementMutationsCounters() will be called in updateMutations() LOG_DEBUG(log, "Removed mutation {} from local state.", entry->znode_name); } @@ -1913,7 +1913,7 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo MutationCommands result; bool seen_all_data_mutations = !params.need_data_mutations; - bool seen_all_metadata_mutations = !params.needMetadataMutations(); + bool seen_all_metadata_mutations = part_metadata_version >= params.metadata_version; if (seen_all_data_mutations && seen_all_metadata_mutations) return {}; @@ -1968,8 +1968,8 @@ MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapsh std::lock_guard lock(state_mutex); - bool need_data_mutations = res->params.need_data_mutations && num_alter_conversions > 0; - bool need_metatadata_mutations = res->params.needMetadataMutations(); + bool need_data_mutations = params.need_data_mutations && data_mutations_to_apply > 0; + bool need_metatadata_mutations = params.min_part_metadata_version < params.metadata_version; if (!need_data_mutations && !need_metatadata_mutations) return res; @@ -2113,7 +2113,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep mutation.parts_to_do.clear(); } - decrementAlterConversionsCounter(num_alter_conversions, mutation.entry->commands, lock); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, mutation.entry->commands, lock); } else if (mutation.parts_to_do.size() == 0) { @@ -2170,7 +2170,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep LOG_TRACE(log, "Finishing data alter with version {} for entry {}", entry->alter_version, entry->znode_name); alter_sequence.finishDataAlter(entry->alter_version, lock); } - decrementAlterConversionsCounter(num_alter_conversions, entry->commands, lock); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); } } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 223ca989feb..a46fdaf3ac4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -154,7 +154,8 @@ private: std::map mutations_by_znode; /// Unfinished mutations that are required for AlterConversions. - Int64 num_alter_conversions = 0; + Int64 data_mutations_to_apply = 0; + Int64 metadata_mutations_to_apply = 0; /// Partition -> (block_number -> MutationStatus) std::unordered_map> mutations_by_partition; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ef2a7bda118..5c15818632c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -502,7 +502,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); - incrementAlterConversionsCounter(num_alter_conversions, *it->second.commands, lock); + incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info); } background_operations_assignee.trigger(); @@ -538,7 +538,7 @@ void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr re if (static_cast(result_part->part_info.mutation) == it->first) mutation_backoff_policy.removePartFromFailed(failed_part->name); - decrementAlterConversionsCounter(num_alter_conversions, *entry.commands, lock); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry.commands, lock); } } else @@ -819,7 +819,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) { bool mutation_finished = *min_version > static_cast(mutation_version); if (!mutation_finished) - decrementAlterConversionsCounter(num_alter_conversions, *it->second.commands, lock); + decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); } to_kill.emplace(std::move(it->second)); @@ -904,7 +904,7 @@ void StorageMergeTree::loadMutations() if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number); - incrementAlterConversionsCounter(num_alter_conversions, *entry_it->second.commands, lock); + incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry_it->second.commands, lock); } else if (startsWith(it->name(), "tmp_mutation_")) { @@ -2449,10 +2449,10 @@ MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(const std::lock_guard lock(currently_processing_in_background_mutex); - bool need_data_mutations = res->params.need_data_mutations && num_alter_conversions > 0; - bool need_metatadata_mutations = res->params.needMetadataMutations(); + bool need_data_mutations = res->params.need_data_mutations && data_mutations_to_apply > 0; + bool need_metadata_mutations = metadata_mutations_to_apply > 0; - if (!need_data_mutations && !need_metatadata_mutations) + if (!need_data_mutations && !need_metadata_mutations) return res; for (const auto & [version, entry] : current_mutations_by_version) @@ -2462,7 +2462,7 @@ MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(const if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type)) return true; - if (need_metatadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) + if (need_metadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type)) return true; return false; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 40e5a8e5ea4..d2ade17e309 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -150,7 +150,8 @@ private: std::map current_mutations_by_version; /// Unfinished mutations that are required for AlterConversions. - Int64 num_alter_conversions = 0; + Int64 data_mutations_to_apply = 0; + Int64 metadata_mutations_to_apply = 0; std::atomic shutdown_called {false}; std::atomic flush_called {false}; From 2df3e99666e22e8838bc500eda37393772e1a4f1 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 19 Jul 2024 17:04:16 +0000 Subject: [PATCH 010/259] better mutations snapshot --- src/Interpreters/MutationsInterpreter.h | 1 - src/Storages/MergeTree/MergeTask.h | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 41 ++++++++++--------- src/Storages/MergeTree/MergeTreeData.h | 38 ++++++++++------- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 3 -- .../MergeTree/MergeTreeMutationEntry.cpp | 1 - .../MergeTree/MergeTreeSequentialSource.cpp | 3 +- src/Storages/MergeTree/MutateTask.cpp | 1 - .../MergeTree/ReplicatedMergeTreeQueue.cpp | 36 +++++++++------- .../MergeTree/ReplicatedMergeTreeQueue.h | 7 ++-- src/Storages/StorageMergeTree.cpp | 24 +++++++---- src/Storages/StorageMergeTree.h | 7 ++-- 12 files changed, 91 insertions(+), 72 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 8ae438efc19..b3a8dd41736 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -6,7 +6,6 @@ #include #include #include -#include "Storages/MergeTree/AlterConversions.h" namespace DB diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 619e2fc7b2b..180538480fa 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -4,7 +4,6 @@ #include #include -#include "Storages/MergeTree/AlterConversions.h" #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d631a8833d0..cdfa41eef78 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8444,9 +8444,10 @@ void MergeTreeData::updateObjectColumns(const DataPartPtr & part, const DataPart DB::updateObjectColumns(object_columns, columns, part->getColumns()); } -bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const +bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr) const { - return !hasLightweightDeletedMask(); + const auto & snapshot_data = assert_cast(*storage_snapshot->data); + return !hasLightweightDeletedMask() && !snapshot_data.mutations_snapshot->hasDataMutations(); } Int64 MergeTreeData::getMinMetadataVersion(const DataPartsVector & parts) @@ -8698,16 +8699,16 @@ void MergeTreeData::verifySortingKey(const KeyDescription & sorting_key) } static void updateMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, const MutationCommands & commands, Int64 increment) { - if (data_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); + if (num_data_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", num_data_mutations_to_apply); - if (metadata_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); + if (num_metadata_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", num_metadata_mutations_to_apply); bool has_data_mutation = false; bool has_metadata_mutation = false; @@ -8716,40 +8717,40 @@ static void updateMutationsCounters( { if (!has_data_mutation && AlterConversions::isSupportedDataMutation(command.type)) { - data_mutations_to_apply += increment; + num_data_mutations_to_apply += increment; has_data_mutation = true; - if (data_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", data_mutations_to_apply); + if (num_data_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", num_data_mutations_to_apply); } if (!has_metadata_mutation && AlterConversions::isSupportedMetadataMutation(command.type)) { - metadata_mutations_to_apply += increment; + num_metadata_mutations_to_apply += increment; has_metadata_mutation = true; - if (metadata_mutations_to_apply < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", metadata_mutations_to_apply); + if (num_metadata_mutations_to_apply < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", num_metadata_mutations_to_apply); } } } void incrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, const MutationCommands & commands, std::lock_guard & /*lock*/) { - return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, 1); + return updateMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, 1); } void decrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, const MutationCommands & commands, std::lock_guard & /*lock*/) { - return updateMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, -1); + return updateMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, -1); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 984662daff0..d5cfddb70af 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -34,7 +34,6 @@ #include #include #include -#include "Storages/ProjectionsDescription.h" #include #include @@ -444,10 +443,14 @@ public: bool areAsynchronousInsertsEnabled() const override; - bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override; + bool supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr) const override; + /// A snapshot of pending mutations that weren't applied to some of the parts yet + /// and should be applied on the fly (i.e. when reading from the part). + /// Mutations not supported by AlterConversions (supportsMutationCommandType()) can be omitted. struct IMutationsSnapshot { + /// Contains info that doesn't depend on state of mutations. struct Params { Int64 metadata_version = -1; @@ -455,18 +458,25 @@ public: bool need_data_mutations = false; }; + /// Contains info that depends on state of mutations. + struct Info + { + Int64 num_data_mutations = 0; + Int64 num_metadata_mutations = 0; + }; + Params params; + Info info; IMutationsSnapshot() = default; - explicit IMutationsSnapshot(Params params_) : params(std::move(params_)) {} + IMutationsSnapshot(Params params_, Info info_): params(std::move(params_)), info(std::move(info_)) {} - /// Return pending mutations that weren't applied to `part` yet and should be applied on the fly - /// (i.e. when reading from the part). Mutations not supported by AlterConversions - /// (supportsMutationCommandType()) can be omitted. - /// - /// @return list of mutations, in *reverse* order (newest to oldest) + /// Returns mutation commands that are required to be applied to the `part`. + /// @return list of mutation commands, in *reverse* order (newest to oldest) virtual MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; virtual std::shared_ptr cloneEmpty() const = 0; + bool hasDataMutations() const { return params.need_data_mutations && info.num_data_mutations > 0; } + virtual ~IMutationsSnapshot() = default; }; @@ -956,10 +966,10 @@ public: Disks getDisks() const { return getStoragePolicy()->getDisks(); } - /// TODO: comment + /// Returns a snapshot of mutations that probably will be applied on the fly to parts during reading. virtual MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const = 0; - /// TODO: comment + /// Returns the minimum version of metadata among parts. static Int64 getMinMetadataVersion(const DataPartsVector & parts); /// Return alter conversions for part which must be applied on fly. @@ -1761,14 +1771,14 @@ struct CurrentlySubmergingEmergingTagger /// Look at MutationCommands if it contains mutations for AlterConversions, update the counter. /// Return true if the counter had been updated void incrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, const MutationCommands & commands, std::lock_guard & lock); void decrementMutationsCounters( - Int64 & data_mutations_to_apply, - Int64 & metadata_mutations_to_apply, + Int64 & num_data_mutations_to_apply, + Int64 & num_metadata_mutations_to_apply, const MutationCommands & commands, std::lock_guard & lock); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 94e00c531ba..1e10e4adc9d 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -896,9 +896,6 @@ ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar return std::make_shared(); std::optional indexes; - /// NOTE: We don't need mutations snapshot because the returned analysis_result is only used for: - /// 1. estimate the number of rows to read; - /// 2. projection reading, which doesn't have alter conversions. return ReadFromMergeTree::selectRangesToRead( std::move(parts), metadata_snapshot, diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 6f06b921031..5970aed497e 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -6,7 +6,6 @@ #include #include #include -#include "Storages/MutationCommands.h" #include diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 8368f4e80f7..31e1c0f235f 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -14,7 +14,6 @@ #include #include #include -#include "Storages/MergeTree/AlterConversions.h" #include namespace DB @@ -65,7 +64,7 @@ private: /// Data part will not be removed if the pointer owns it MergeTreeData::DataPartPtr data_part; - /// TODO: comment. + /// Alter and mutation commands that are required to be applied to the part on-fly. AlterConversionsPtr alter_conversions; /// Columns we have to read (each Block from read will contain them) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 74069d8b06e..a95cccdc0d2 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -33,7 +33,6 @@ #include #include #include -#include "Storages/MergeTree/AlterConversions.h" #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index c2fcfbd1810..c3d91ca0705 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -14,8 +14,6 @@ #include #include #include -#include -#include #include #include @@ -952,7 +950,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper { const auto commands = entry.commands; it = mutations_by_znode.erase(it); - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, commands, state_lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, state_lock); } else it = mutations_by_znode.erase(it); @@ -1002,7 +1000,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper for (const ReplicatedMergeTreeMutationEntryPtr & entry : new_mutations) { auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version)).first->second; - incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands, lock); NOEXCEPT_SCOPE({ for (const auto & pair : entry->block_numbers) @@ -1913,7 +1911,7 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo MutationCommands result; - bool seen_all_data_mutations = !params.need_data_mutations; + bool seen_all_data_mutations = !hasDataMutations(); bool seen_all_metadata_mutations = part_metadata_version >= params.metadata_version; if (seen_all_data_mutations && seen_all_metadata_mutations) @@ -1940,9 +1938,9 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo auto alter_version = entry->alter_version; - if (!seen_all_metadata_mutations && alter_version != -1) + if (alter_version != -1) { - if (alter_version > params.metadata_version) + if (seen_all_metadata_mutations || alter_version > params.metadata_version) continue; /// We take commands with bigger metadata version @@ -1965,11 +1963,17 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapshot(const MutationsSnapshot::Params & params) const { - auto res = std::make_shared(params); - std::lock_guard lock(state_mutex); - bool need_data_mutations = params.need_data_mutations && data_mutations_to_apply > 0; + MutationsSnapshot::Info info + { + .num_data_mutations = num_data_mutations_to_apply, + .num_metadata_mutations = num_metadata_mutations_to_apply, + }; + + auto res = std::make_shared(params, std::move(info)); + + bool need_data_mutations = res->hasDataMutations(); bool need_metatadata_mutations = params.min_part_metadata_version < params.metadata_version; if (!need_data_mutations && !need_metatadata_mutations) @@ -2000,14 +2004,16 @@ MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapsh auto alter_version = status->entry->alter_version; - if (!seen_all_metadata_mutations && alter_version != -1) + if (alter_version != -1) { - if (alter_version > params.metadata_version) + if (seen_all_metadata_mutations || alter_version > params.metadata_version) continue; /// We take commands with bigger metadata version if (alter_version > params.min_part_metadata_version) { + /// Copy a pointer to the whole entry to avoid extracting and copying commands. + /// Required commands will be copied later only for specific parts. if (std::ranges::any_of(status->entry->commands, is_supported_command)) in_partition.emplace(mutation_version, status->entry); } @@ -2020,6 +2026,8 @@ MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapsh { if (!status->is_done) { + /// Copy a pointer to the whole entry to avoid extracting and copying commands. + /// Required commands will be copied later only for specific parts. if (std::ranges::any_of(status->entry->commands, is_supported_command)) in_partition.emplace(mutation_version, status->entry); } @@ -2114,7 +2122,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep mutation.parts_to_do.clear(); } - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, mutation.entry->commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, mutation.entry->commands, lock); } else if (mutation.parts_to_do.size() == 0) { @@ -2171,7 +2179,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep LOG_TRACE(log, "Finishing data alter with version {} for entry {}", entry->alter_version, entry->znode_name); alter_sequence.finishDataAlter(entry->alter_version, lock); } - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, entry->commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands, lock); } } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index a46fdaf3ac4..af8e2521f81 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -154,8 +154,8 @@ private: std::map mutations_by_znode; /// Unfinished mutations that are required for AlterConversions. - Int64 data_mutations_to_apply = 0; - Int64 metadata_mutations_to_apply = 0; + Int64 num_data_mutations_to_apply = 0; + Int64 num_metadata_mutations_to_apply = 0; /// Partition -> (block_number -> MutationStatus) std::unordered_map> mutations_by_partition; @@ -415,8 +415,9 @@ public: struct MutationsSnapshot : public MergeTreeData::IMutationsSnapshot { + public: MutationsSnapshot() = default; - explicit MutationsSnapshot(Params params_) : IMutationsSnapshot(std::move(params_)) {} + MutationsSnapshot(Params params_, Info info_) : IMutationsSnapshot(std::move(params_), std::move(info_)) {} using Params = MergeTreeData::IMutationsSnapshot::Params; using MutationsByPartititon = std::unordered_map>; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c7189925812..900b592d32d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -520,7 +520,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); - incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands, lock); LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info); } background_operations_assignee.trigger(); @@ -556,7 +556,7 @@ void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr re if (static_cast(result_part->part_info.mutation) == it->first) mutation_backoff_policy.removePartFromFailed(failed_part->name); - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry.commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry.commands, lock); } } else @@ -838,7 +838,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) { bool mutation_finished = *min_version > static_cast(mutation_version); if (!mutation_finished) - decrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *it->second.commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands, lock); } to_kill.emplace(std::move(it->second)); @@ -923,7 +923,7 @@ void StorageMergeTree::loadMutations() if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number); - incrementMutationsCounters(data_mutations_to_apply, metadata_mutations_to_apply, *entry_it->second.commands, lock); + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry_it->second.commands, lock); } else if (startsWith(it->name(), "tmp_mutation_")) { @@ -2466,12 +2466,18 @@ MutationCommands StorageMergeTree::MutationsSnapshot::getAlterMutationCommandsFo MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(const IMutationsSnapshot::Params & params) const { - auto res = std::make_shared(params); - std::lock_guard lock(currently_processing_in_background_mutex); - bool need_data_mutations = res->params.need_data_mutations && data_mutations_to_apply > 0; - bool need_metadata_mutations = metadata_mutations_to_apply > 0; + MutationsSnapshot::Info info + { + .num_data_mutations = num_data_mutations_to_apply, + .num_metadata_mutations = num_metadata_mutations_to_apply, + }; + + auto res = std::make_shared(params, std::move(info)); + + bool need_data_mutations = res->params.need_data_mutations && num_data_mutations_to_apply > 0; + bool need_metadata_mutations = num_metadata_mutations_to_apply > 0; if (!need_data_mutations && !need_metadata_mutations) return res; @@ -2489,6 +2495,8 @@ MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(const return false; }); + /// Copy a pointer to all commands to avoid extracting and copying them. + /// Required commands will be copied later only for specific parts. if (has_required_command) res->mutations_by_version.emplace(version, entry.commands); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index a3633284ac6..16ad79d7586 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -17,7 +17,6 @@ #include #include -#include "Storages/MutationCommands.h" namespace DB @@ -150,8 +149,8 @@ private: std::map current_mutations_by_version; /// Unfinished mutations that are required for AlterConversions. - Int64 data_mutations_to_apply = 0; - Int64 metadata_mutations_to_apply = 0; + Int64 num_data_mutations_to_apply = 0; + Int64 num_metadata_mutations_to_apply = 0; std::atomic shutdown_called {false}; std::atomic flush_called {false}; @@ -314,7 +313,7 @@ private: struct MutationsSnapshot : public IMutationsSnapshot { MutationsSnapshot() = default; - explicit MutationsSnapshot(Params params_) : IMutationsSnapshot(std::move(params_)) {} + MutationsSnapshot(Params params_, Info info_) : IMutationsSnapshot(std::move(params_), std::move(info_)) {} using MutationsByVersion = std::map>; MutationsByVersion mutations_by_version; From 383221b04d57948364514106b2f48b03140bfbd0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 23 Jul 2024 10:36:08 +0000 Subject: [PATCH 011/259] Analyzer: Storage Join support joining with nullable columns --- src/Planner/PlannerJoins.cpp | 8 +- ...multiple_joins_with_storage_join.reference | 23 +++++ ...03208_multiple_joins_with_storage_join.sql | 84 +++++++++++++++++++ 3 files changed, 114 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03208_multiple_joins_with_storage_join.reference create mode 100644 tests/queries/0_stateless/03208_multiple_joins_with_storage_join.sql diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index d9360a58240..04011233ca1 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -494,6 +494,12 @@ JoinClausesAndActions buildJoinClausesAndActions( necessary_names.push_back(name); }; + bool is_join_with_special_storage = false; + if (const auto * right_table_node = join_node.getRightTableExpression()->as()) + { + is_join_with_special_storage = dynamic_cast(right_table_node->getStorage().get()); + } + for (auto & join_clause : result.join_clauses) { const auto & left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); @@ -561,7 +567,7 @@ JoinClausesAndActions buildJoinClausesAndActions( if (!left_key_node->result_type->equals(*common_type)) left_key_node = &left_join_actions->addCast(*left_key_node, common_type, {}); - if (!right_key_node->result_type->equals(*common_type)) + if (!is_join_with_special_storage && !right_key_node->result_type->equals(*common_type)) right_key_node = &right_join_actions->addCast(*right_key_node, common_type, {}); } diff --git a/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.reference b/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.reference new file mode 100644 index 00000000000..5ebdaa1c81f --- /dev/null +++ b/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.reference @@ -0,0 +1,23 @@ +----- +1 1 1 a 1 A 1 A +2 2 2 b 2 B 2 B +----- +\N \N \N 0 3 B +1 1 1 a 1 A 1 A +2 2 2 b 2 B 2 B +----- +1 1 1 a 1 A 1 A +2 2 2 b 2 B 2 B +\N \N \N \N 3 B \N \N +\N \N \N \N \N \N 3 B +----- +\N \N \N 3 3 B 0 0 +\N \N \N 0 0 3 3 B +1 1 1 a 1 1 A 1 1 A +2 2 2 b 2 2 B 2 2 B +----- +3 3 \N B B +1 1 1 a A A +2 2 2 b B B +----- +7 diff --git a/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.sql b/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.sql new file mode 100644 index 00000000000..83be4c3f1d2 --- /dev/null +++ b/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.sql @@ -0,0 +1,84 @@ +#!/usr/bin/env -S ${HOME}/clickhouse-client --queries-file + +DROP TABLE IF EXISTS tab; +CREATE TABLE tab ( `k` Nullable(UInt32), `k1` Nullable(UInt32), `k2` Nullable(UInt32), `v` String ) ENGINE = Memory; +INSERT INTO tab VALUES (1, 1, 1, 'a'), (2, 2, 2, 'b'); + +DROP TABLE IF EXISTS mem; +CREATE TABLE mem ( `k` UInt64, `v` String ) ENGINE = Join(ANY, LEFT, k); +INSERT INTO mem VALUES (1, 'A'), (2, 'B'), (3, 'B'); + +DROP TABLE IF EXISTS mem2; +CREATE TABLE mem2 ( `k` UInt64, `v` String ) ENGINE = Join(ANY, RIGHT, k); +INSERT INTO mem2 VALUES (1, 'A'), (2, 'B'), (3, 'B'); + +DROP TABLE IF EXISTS mem3; +CREATE TABLE mem3 ( `k` UInt64, `v` String ) ENGINE = Join(ALL, FULL, k) SETTINGS join_use_nulls = 1; +INSERT INTO mem3 VALUES (1, 'A'), (2, 'B'), (3, 'B'); + +DROP TABLE IF EXISTS mem4; +CREATE TABLE mem4 ( `k1` UInt64, `k2` UInt64, `v` String ) ENGINE = Join(ALL, FULL, k1, k2); +INSERT INTO mem4 VALUES (1, 1, 'A'), (2, 2, 'B'), (3, 3, 'B'); + +SET allow_experimental_analyzer = 1; + +SELECT '-----'; + +SELECT * +FROM tab +ANY LEFT JOIN mem ON k1 = mem.k +ANY LEFT JOIN mem AS t ON k2 = t.k +ORDER BY tab.v +; + +SELECT '-----'; + +SELECT * +FROM tab +ANY LEFT JOIN mem ON k1 = mem.k +ANY RIGHT JOIN mem2 ON k2 = mem2.k +ORDER BY tab.v +; + +SELECT '-----'; + +SELECT * +FROM tab +FULL JOIN mem3 AS t1 ON k1 = t1.k +FULL JOIN mem3 AS t2 ON k2 = t2.k +ORDER BY tab.v +SETTINGS join_use_nulls = 1 +; +SELECT '-----'; + +SELECT * +FROM tab +FULL JOIN mem4 AS t1 ON tab.k1 = t1.k1 AND tab.k2 = t1.k2 +FULL JOIN mem4 AS t2 ON tab.k1 = t2.k1 AND tab.k2 = t2.k2 +ORDER BY tab.v +; + +SELECT '-----'; + +SELECT * +FROM tab +FULL JOIN mem4 AS t1 USING (k1, k2) +FULL JOIN mem4 AS t2 USING (k1, k2) +ORDER BY tab.v +; + +SELECT '-----'; + +SELECT count() FROM ( + EXPLAIN PLAN + SELECT * FROM tab + ANY LEFT JOIN mem AS t1 ON tab.k = t1.k + ANY LEFT JOIN mem AS t2 ON tab.k = t2.k + ANY LEFT JOIN mem AS t3 ON tab.k = t3.k + ANY LEFT JOIN mem AS t4 ON tab.k = t4.k + ANY RIGHT JOIN mem2 AS t5 ON tab.k = t5.k + ANY LEFT JOIN mem AS t6 ON tab.k = t6.k + ANY LEFT JOIN mem AS t7 ON tab.k = t7.k +) +WHERE explain like '%FilledJoin%' +; From 4712b79960d9da21741aa504cd508a5902e569fe Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 24 Jul 2024 16:41:37 +0800 Subject: [PATCH 012/259] 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 013/259] 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 014/259] 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 015/259] 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 016/259] 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 97ceca4b92289352eb71b6cfd0f5280162334609 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 25 Jul 2024 12:20:39 +0000 Subject: [PATCH 017/259] fix reading from Merge tables --- .../QueryPlan/Optimizations/projectionsCommon.cpp | 11 +++++++++-- src/Storages/IStorage.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 10 ++++++++-- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageMerge.cpp | 6 ++++-- 5 files changed, 23 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 487367b8c30..e7df58e1a86 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -41,12 +41,19 @@ bool canUseProjectionForReadingStep(ReadFromMergeTree * reading) if (reading->readsInOrder()) return false; + const auto & query_settings = reading->getContext()->getSettingsRef(); + // Currently projection don't support deduplication when moving parts between shards. - if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication) + if (query_settings.allow_experimental_query_deduplication) return false; // Currently projection don't support settings which implicitly modify aggregate functions. - if (reading->getContext()->getSettingsRef().aggregate_functions_null_for_empty) + if (query_settings.aggregate_functions_null_for_empty) + return false; + + /// Don't use projections if have mutations to apply + /// because we need to apply them on original data. + if (query_settings.apply_mutations_on_fly && reading->getMutationsSnapshot()->hasDataMutations()) return false; return true; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6217470780d..92de82934e3 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -271,6 +271,7 @@ public: /// Return true if the trivial count query could be optimized without reading the data at all /// in totalRows() or totalRowsByPartitionPredicate() methods or with optimized reading in read() method. + /// 'storage_snapshot' may be nullptr. virtual bool supportsTrivialCountOptimization(const StorageSnapshotPtr & /*storage_snapshot*/, ContextPtr /*query_context*/) const { return false; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b32a3f5eda3..f1054355b07 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8444,10 +8444,16 @@ void MergeTreeData::updateObjectColumns(const DataPartPtr & part, const DataPart DB::updateObjectColumns(object_columns, columns, part->getColumns()); } -bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr) const +bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr query_context) const { + if (hasLightweightDeletedMask()) + return false; + + if (!storage_snapshot) + return !query_context->getSettingsRef().apply_mutations_on_fly; + const auto & snapshot_data = assert_cast(*storage_snapshot->data); - return !hasLightweightDeletedMask() && !snapshot_data.mutations_snapshot->hasDataMutations(); + return !snapshot_data.mutations_snapshot->hasDataMutations(); } Int64 MergeTreeData::getMinMetadataVersion(const DataPartsVector & parts) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index d5cfddb70af..588b31db7b9 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -443,7 +443,7 @@ public: bool areAsynchronousInsertsEnabled() const override; - bool supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr) const override; + bool supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr query_context) const override; /// A snapshot of pending mutations that weren't applied to some of the parts yet /// and should be applied on the fly (i.e. when reading from the part). diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index f5bc183931f..0cc369c87fa 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1617,9 +1617,11 @@ std::tuple StorageMerge::evaluateDatabaseName(cons return {false, ast}; } -bool StorageMerge::supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr ctx) const +bool StorageMerge::supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr ctx) const { - return getFirstTable([&](const auto & table) { return !table->supportsTrivialCountOptimization(storage_snapshot, ctx); }) == nullptr; + /// Here we actually need storage snapshot of all nested tables. + /// But to avoid complexity pass nullptr to make more lightweight check in MergeTreeData. + return getFirstTable([&](const auto & table) { return !table->supportsTrivialCountOptimization(nullptr, ctx); }) == nullptr; } std::optional StorageMerge::totalRows(const Settings & settings) const From f00928afa14864cd8efbc186f63538c8cd99f643 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 Jul 2024 18:41:43 +0000 Subject: [PATCH 018/259] Improve castOrDefault from Variant/Dynamic columns --- src/Functions/FunctionsConversion.cpp | 52 +++++++++++++++++-- ..._variant_dynamic_cast_or_default.reference | 32 ++++++++++++ .../03212_variant_dynamic_cast_or_default.sql | 9 ++++ 3 files changed, 89 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference create mode 100644 tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 675283d011e..e516d1dbe54 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4086,9 +4086,30 @@ private: /// Create conversion wrapper for each variant. for (const auto & variant_type : variant_types) - variant_wrappers.push_back(prepareUnpackDictionaries(variant_type, to_type)); + { + WrapperType wrapper; + if (cast_type == CastType::accurateOrNull) + { + /// With accurateOrNull cast type we should insert default values on variants that cannot be casted. + /// We can avoid try/catch here if we will implement check that 2 types can be casted, but it + /// requires quite a lot of work. By now let's simply use try/catch. + try + { + wrapper = prepareUnpackDictionaries(variant_type, to_type); + } + catch (...) + { + /// Leave wrapper empty and check it later. + } + } + else + { + wrapper = prepareUnpackDictionaries(variant_type, to_type); + } + variant_wrappers.push_back(wrapper); + } - return [variant_wrappers, variant_types, to_type] + return [variant_wrappers, variant_types, to_type, cast_type_ = this->cast_type] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr { const auto & column_variant = assert_cast(*arguments.front().column.get()); @@ -4101,7 +4122,30 @@ private: auto variant_col = column_variant.getVariantPtrByGlobalDiscriminator(i); ColumnsWithTypeAndName variant = {{variant_col, variant_types[i], "" }}; const auto & variant_wrapper = variant_wrappers[i]; - casted_variant_columns.push_back(variant_wrapper(variant, result_type, nullptr, variant_col->size())); + ColumnPtr casted_variant; + /// Check if we have wrapper for this variant. + if (variant_wrapper) + { + if (cast_type_ == CastType::accurateOrNull) + { + /// With accurateOrNull cast type wrapper should throw an exception + /// only when the cast between types is not supported. + /// In this case we will insert default values on rows with this variant. + try + { + casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); + } + catch (...) + { + /// Do nothing. + } + } + else + { + casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); + } + } + casted_variant_columns.push_back(std::move(casted_variant)); } /// Second, construct resulting column from casted variant columns according to discriminators. @@ -4111,7 +4155,7 @@ private: for (size_t i = 0; i != input_rows_count; ++i) { auto global_discr = column_variant.globalDiscriminatorByLocal(local_discriminators[i]); - if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR || !casted_variant_columns[global_discr]) res->insertDefault(); else res->insertFrom(*casted_variant_columns[global_discr], column_variant.offsetAt(i)); diff --git a/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference new file mode 100644 index 00000000000..8b1a342181c --- /dev/null +++ b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference @@ -0,0 +1,32 @@ +0 \N +1 1 +0 str_2 +0 [0,1,2] +0 \N +5 5 +0 str_6 +0 [0,1,2,3,4,5,6] +\N \N +1 1 +\N str_2 +\N [0,1,2] +\N \N +5 5 +\N str_6 +\N [0,1,2,3,4,5,6] +0 \N +1 1 +0 str_2 +0 [0,1,2] +0 \N +5 5 +0 str_6 +0 [0,1,2,3,4,5,6] +\N \N +1 1 +\N str_2 +\N [0,1,2] +\N \N +5 5 +\N str_6 +\N [0,1,2,3,4,5,6] diff --git a/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql new file mode 100644 index 00000000000..1e71e36780c --- /dev/null +++ b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql @@ -0,0 +1,9 @@ +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; +set allow_experimental_dynamic_type = 1; + +select accurateCastOrDefault(variant, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number)) as variant from numbers(8); +select accurateCastOrNull(variant, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number)) as variant from numbers(8); + +select accurateCastOrDefault(dynamic, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number))::Dynamic as dynamic from numbers(8); +select accurateCastOrNull(dynamic, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number))::Dynamic as dynamic from numbers(8); From eb300f4f782bf9a6b216624bddd6e6deffd55d0f Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 25 Jul 2024 18:59:06 +0000 Subject: [PATCH 019/259] Better implementation --- src/Functions/FunctionsConversion.cpp | 56 ++++++++++++--------------- 1 file changed, 24 insertions(+), 32 deletions(-) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index e516d1dbe54..c14fa3187d8 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4078,6 +4078,26 @@ private: }; } + /// Create wrapper only if we support this conversion. + WrapperType createWrapperIfCanConvert(const DataTypePtr & from, const DataTypePtr & to) const + { + try + { + /// We can avoid try/catch here if we will implement check that 2 types can be casted, but it + /// requires quite a lot of work. By now let's simply use try/catch. + /// First, check that we can create a wrapper. + WrapperType wrapper = prepareUnpackDictionaries(from, to); + /// Second, check if we can perform a conversion on empty columns. + ColumnsWithTypeAndName column_from = {{from->createColumn(), from, "" }}; + wrapper(column_from, to, nullptr, 0); + return wrapper; + } + catch (...) + { + return {}; + } + } + WrapperType createVariantToColumnWrapper(const DataTypeVariant & from_variant, const DataTypePtr & to_type) const { const auto & variant_types = from_variant.getVariants(); @@ -4090,17 +4110,8 @@ private: WrapperType wrapper; if (cast_type == CastType::accurateOrNull) { - /// With accurateOrNull cast type we should insert default values on variants that cannot be casted. - /// We can avoid try/catch here if we will implement check that 2 types can be casted, but it - /// requires quite a lot of work. By now let's simply use try/catch. - try - { - wrapper = prepareUnpackDictionaries(variant_type, to_type); - } - catch (...) - { - /// Leave wrapper empty and check it later. - } + /// Create wrapper only if we support conversion from variant to the resulting type. + wrapper = createWrapperIfCanConvert(variant_type, to_type); } else { @@ -4109,7 +4120,7 @@ private: variant_wrappers.push_back(wrapper); } - return [variant_wrappers, variant_types, to_type, cast_type_ = this->cast_type] + return [variant_wrappers, variant_types, to_type] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr { const auto & column_variant = assert_cast(*arguments.front().column.get()); @@ -4125,26 +4136,7 @@ private: ColumnPtr casted_variant; /// Check if we have wrapper for this variant. if (variant_wrapper) - { - if (cast_type_ == CastType::accurateOrNull) - { - /// With accurateOrNull cast type wrapper should throw an exception - /// only when the cast between types is not supported. - /// In this case we will insert default values on rows with this variant. - try - { - casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); - } - catch (...) - { - /// Do nothing. - } - } - else - { - casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); - } - } + casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); casted_variant_columns.push_back(std::move(casted_variant)); } From 3adbc4cd334a05560ca2397ac5e1a14d8c580167 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 31 Jul 2024 15:10:29 +0800 Subject: [PATCH 020/259] 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 021/259] 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 022/259] 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 023/259] 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 024/259] 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 025/259] 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 026/259] 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 027/259] 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 028/259] 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 0a54ba65756a5789c6e64e78141fe7ad962f2356 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 6 Aug 2024 15:28:45 +0000 Subject: [PATCH 029/259] better on-fly mutations --- .../Optimizations/projectionsCommon.cpp | 1 + .../QueryPlan/ReadFromMergeTree.cpp | 107 +++++++++++------- src/Processors/QueryPlan/ReadFromMergeTree.h | 1 + src/Storages/MergeTree/AlterConversions.h | 13 ++- src/Storages/MergeTree/MergeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 18 +-- src/Storages/MergeTree/MergeTreeData.h | 13 ++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 + .../MergeTree/MergeTreeDataSelectExecutor.h | 1 + .../MergeTree/MergeTreePrefetchedReadPool.cpp | 1 - .../MergeTree/MergeTreePrefetchedReadPool.h | 2 +- .../MergeTree/MergeTreeReadPoolBase.cpp | 8 +- .../MergeTree/MergeTreeReadPoolBase.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 25 +++- .../MergeTree/ReplicatedMergeTreeQueue.h | 1 + src/Storages/MutationCommands.cpp | 9 ++ src/Storages/MutationCommands.h | 1 + src/Storages/StorageMergeTree.cpp | 22 +++- src/Storages/StorageMergeTree.h | 1 + 20 files changed, 156 insertions(+), 76 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index e5f9c3b23ea..998b606ec57 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -243,6 +243,7 @@ bool analyzeProjectionCandidate( auto projection_result_ptr = reader.estimateNumMarksToRead( std::move(projection_parts), + reading.getMutationsSnapshot()->cloneEmpty(), required_column_names, candidate.projection->metadata, projection_query_info, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 7d6e650cba0..de29e7a9d5a 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1373,6 +1373,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge { return selectRangesToRead( std::move(parts), + mutations_snapshot, metadata_for_reading, query_info, context, @@ -1390,9 +1391,11 @@ static void buildIndexes( const ActionsDAG * filter_actions_dag, const MergeTreeData & data, const MergeTreeData::DataPartsVector & parts, + const MergeTreeData::MutationsSnapshotPtr & mutations_snapshot, const ContextPtr & context, const SelectQueryInfo & query_info, - const StorageMetadataPtr & metadata_snapshot) + const StorageMetadataPtr & metadata_snapshot, + const LoggerPtr & log) { indexes.reset(); @@ -1418,19 +1421,21 @@ static void buildIndexes( indexes->partition_pruner.emplace(metadata_snapshot, filter_actions_dag, context, false /* strict */); } - indexes->part_values - = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(metadata_snapshot, data, parts, filter_actions_dag, context); + indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(metadata_snapshot, data, parts, filter_actions_dag, context); MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset(indexes->part_offset_condition, filter_actions_dag, context); indexes->use_skip_indexes = settings.use_skip_indexes; - bool final = query_info.isFinal(); - - if (final && !settings.use_skip_indexes_if_final) + if (query_info.isFinal() && !settings.use_skip_indexes_if_final) indexes->use_skip_indexes = false; if (!indexes->use_skip_indexes) return; + const auto & all_indexes = metadata_snapshot->getSecondaryIndices(); + + if (all_indexes.empty()) + return; + std::unordered_set ignored_index_names; if (settings.ignore_data_skipping_indices.changed) @@ -1455,49 +1460,68 @@ static void buildIndexes( throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse ignore_data_skipping_indices ('{}')", indices); } + auto all_updated_columns = mutations_snapshot->getAllUpdatedColumns(); + UsefulSkipIndexes skip_indexes; using Key = std::pair; std::map merged; - for (const auto & index : metadata_snapshot->getSecondaryIndices()) + for (const auto & index : all_indexes) { - if (!ignored_index_names.contains(index.name)) + if (ignored_index_names.contains(index.name)) + continue; + + auto index_helper = MergeTreeIndexFactory::instance().get(index); + + if (!all_updated_columns.empty()) { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - if (index_helper->isMergeable()) + auto required_columns = index_helper->getColumnsRequiredForIndexCalc(); + auto it = std::ranges::find_if(required_columns, [&](const auto & column_name) { - auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); - if (inserted) - { - skip_indexes.merged_indices.emplace_back(); - skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); - } + return all_updated_columns.contains(column_name); + }); - skip_indexes.merged_indices[it->second].addIndex(index_helper); - } - else + if (it != required_columns.end()) { - MergeTreeIndexConditionPtr condition; - if (index_helper->isVectorSearch()) - { -#ifdef ENABLE_ANNOY - if (const auto * annoy = typeid_cast(index_helper.get())) - condition = annoy->createIndexCondition(query_info, context); -#endif -#ifdef ENABLE_USEARCH - if (const auto * usearch = typeid_cast(index_helper.get())) - condition = usearch->createIndexCondition(query_info, context); -#endif - if (!condition) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown vector search index {}", index_helper->index.name); - } - else - condition = index_helper->createIndexCondition(filter_actions_dag, context); - - if (!condition->alwaysUnknownOrTrue()) - skip_indexes.useful_indices.emplace_back(index_helper, condition); + LOG_TRACE(log, "Index {} is not used because it depends on column {} which will be updated on fly", index.name, *it); + continue; } } + + if (index_helper->isMergeable()) + { + auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); + if (inserted) + { + skip_indexes.merged_indices.emplace_back(); + skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); + } + + skip_indexes.merged_indices[it->second].addIndex(index_helper); + continue; + } + + MergeTreeIndexConditionPtr condition; + if (index_helper->isVectorSearch()) + { +#ifdef ENABLE_ANNOY + if (const auto * annoy = typeid_cast(index_helper.get())) + condition = annoy->createIndexCondition(query_info, context); +#endif +#ifdef ENABLE_USEARCH + if (const auto * usearch = typeid_cast(index_helper.get())) + condition = usearch->createIndexCondition(query_info, context); +#endif + if (!condition) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown vector search index {}", index_helper->index.name); + } + else + { + condition = index_helper->createIndexCondition(filter_actions_dag, context); + } + + if (!condition->alwaysUnknownOrTrue()) + skip_indexes.useful_indices.emplace_back(index_helper, condition); } // move minmax indices to first positions, so they will be applied first as cheapest ones @@ -1535,14 +1559,17 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) query_info.filter_actions_dag.get(), data, prepared_parts, + mutations_snapshot, context, query_info, - metadata_for_reading); + metadata_for_reading, + log); } } ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( MergeTreeData::DataPartsVector parts, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info_, ContextPtr context_, @@ -1573,7 +1600,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( const Names & primary_key_column_names = primary_key.column_names; if (!indexes) - buildIndexes(indexes, query_info_.filter_actions_dag.get(), data, parts, context_, query_info_, metadata_snapshot); + buildIndexes(indexes, query_info_.filter_actions_dag.get(), data, parts, mutations_snapshot, context_, query_info_, metadata_snapshot, log); if (indexes->part_values && indexes->part_values->empty()) return std::make_shared(std::move(result)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 10b9e92d99b..e441eda7505 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -154,6 +154,7 @@ public: static AnalysisResultPtr selectRangesToRead( MergeTreeData::DataPartsVector parts, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, ContextPtr context, diff --git a/src/Storages/MergeTree/AlterConversions.h b/src/Storages/MergeTree/AlterConversions.h index cee23bc4efc..046cc1d2491 100644 --- a/src/Storages/MergeTree/AlterConversions.h +++ b/src/Storages/MergeTree/AlterConversions.h @@ -1,8 +1,8 @@ #pragma once #include -#include -#include +#include +#include namespace DB @@ -11,11 +11,17 @@ namespace DB /// Alter conversions which should be applied on-fly for part. /// Built from of the most recent mutation commands for part. /// Now only ALTER RENAME COLUMN is applied. -class AlterConversions : private boost::noncopyable +class AlterConversions : private WithContext, boost::noncopyable { public: AlterConversions() = default; + AlterConversions(StorageMetadataPtr metadata_snapshot_, ContextPtr context_) + : WithContext(context_) + , metadata_snapshot(std::move(metadata_snapshot_)) + { + } + struct RenamePair { std::string rename_to; @@ -40,6 +46,7 @@ public: private: /// Rename map new_name -> old_name. std::vector rename_map; + StorageMetadataPtr metadata_snapshot; }; using AlterConversionsPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index e9a48c655e8..ea4b9261af8 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -301,7 +301,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() infos.add(part_infos); } - global_ctx->alter_conversions.push_back(MergeTreeData::getAlterConversionsForPart(part, mutations_snapshot)); + global_ctx->alter_conversions.push_back(MergeTreeData::getAlterConversionsForPart(part, mutations_snapshot, global_ctx->metadata_snapshot, global_ctx->context)); } const auto & local_part_min_ttl = global_ctx->new_data_part->ttl_infos.part_min_ttl; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3852f282f65..32c0251ef53 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7137,11 +7137,11 @@ UInt64 MergeTreeData::estimateNumberOfRowsToRead( ContextPtr query_context, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info) const { const auto & snapshot_data = assert_cast(*storage_snapshot->data); - const auto & parts = snapshot_data.parts; MergeTreeDataSelectExecutor reader(*this); auto result_ptr = reader.estimateNumMarksToRead( - parts, + snapshot_data.parts, + snapshot_data.mutations_snapshot, storage_snapshot->getMetadataForQuery()->getColumns().getAll().getNames(), storage_snapshot->metadata, query_info, @@ -8162,10 +8162,12 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S AlterConversionsPtr MergeTreeData::getAlterConversionsForPart( const MergeTreeDataPartPtr & part, - const MutationsSnapshotPtr & snapshot) + const MutationsSnapshotPtr & mutations, + const StorageMetadataPtr & metadata, + const ContextPtr & query_context) { - auto commands = snapshot->getAlterMutationCommandsForPart(part); - auto result = std::make_shared(); + auto commands = mutations->getAlterMutationCommandsForPart(part); + auto result = std::make_shared(metadata, query_context); for (const auto & command : commands | std::views::reverse) result->addMutationCommand(command); @@ -8758,8 +8760,7 @@ static void updateMutationsCounters( void incrementMutationsCounters( Int64 & num_data_mutations_to_apply, Int64 & num_metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & /*lock*/) + const MutationCommands & commands) { return updateMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, 1); } @@ -8767,8 +8768,7 @@ void incrementMutationsCounters( void decrementMutationsCounters( Int64 & num_data_mutations_to_apply, Int64 & num_metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & /*lock*/) + const MutationCommands & commands) { return updateMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, -1); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index b14121e0c78..dc37d5e7dad 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -475,6 +475,8 @@ public: /// @return list of mutation commands, in *reverse* order (newest to oldest) virtual MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; virtual std::shared_ptr cloneEmpty() const = 0; + virtual NameSet getAllUpdatedColumns() const = 0; + bool hasDataMutations() const { return params.need_data_mutations && info.num_data_mutations > 0; } virtual ~IMutationsSnapshot() = default; @@ -975,7 +977,9 @@ public: /// Return alter conversions for part which must be applied on fly. static AlterConversionsPtr getAlterConversionsForPart( const MergeTreeDataPartPtr & part, - const MutationsSnapshotPtr & snapshot); + const MutationsSnapshotPtr & mutations, + const StorageMetadataPtr & metadata, + const ContextPtr & query_context); /// Returns destination disk or volume for the TTL rule according to current storage policy. SpacePtr getDestinationForMoveTTL(const TTLDescription & move_ttl) const; @@ -1769,17 +1773,14 @@ struct CurrentlySubmergingEmergingTagger }; /// Look at MutationCommands if it contains mutations for AlterConversions, update the counter. -/// Return true if the counter had been updated void incrementMutationsCounters( Int64 & num_data_mutations_to_apply, Int64 & num_metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & lock); + const MutationCommands & commands); void decrementMutationsCounters( Int64 & num_data_mutations_to_apply, Int64 & num_metadata_mutations_to_apply, - const MutationCommands & commands, - std::lock_guard & lock); + const MutationCommands & commands); } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index e294def040b..2d9a5c6084c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -884,6 +884,7 @@ std::shared_ptr MergeTreeDataSelectExecutor::checkLimits( ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMarksToRead( MergeTreeData::DataPartsVector parts, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const Names & column_names_to_return, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, @@ -898,6 +899,7 @@ ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar std::optional indexes; return ReadFromMergeTree::selectRangesToRead( std::move(parts), + mutations_snapshot, metadata_snapshot, query_info, context, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 8e4c55f2c1d..3668eb0ad90 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -56,6 +56,7 @@ public: /// This method is used to select best projection for table. ReadFromMergeTree::AnalysisResultPtr estimateNumMarksToRead( MergeTreeData::DataPartsVector parts, + MergeTreeData::MutationsSnapshotPtr mutations_snapshot, const Names & column_names, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index faed543420c..09bbf33ba9b 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -105,7 +105,6 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( column_names_, settings_, context_) - , WithContext(context_) , prefetch_threadpool(getContext()->getPrefetchThreadpool()) , log(getLogger("MergeTreePrefetchedReadPool(" + (parts_ranges.empty() ? "" : parts_ranges.front().data_part->storage.getStorageID().getNameForLogs()) + ")")) { diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index 65a7d62ad2d..1a709250937 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -14,7 +14,7 @@ using MergeTreeReaderPtr = std::unique_ptr; /// A class which is responsible for creating read tasks /// which are later taken by readers via getTask method. /// Does prefetching for the read tasks it creates. -class MergeTreePrefetchedReadPool : public MergeTreeReadPoolBase, private WithContext +class MergeTreePrefetchedReadPool : public MergeTreeReadPoolBase { public: MergeTreePrefetchedReadPool( diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index fce8d649617..021b340d746 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -4,9 +4,6 @@ #include #include -#include - - namespace DB { @@ -26,7 +23,8 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( const Names & column_names_, const PoolSettings & pool_settings_, const ContextPtr & context_) - : parts_ranges(std::move(parts_)) + : WithContext(context_) + , parts_ranges(std::move(parts_)) , mutations_snapshot(std::move(mutations_snapshot_)) , shared_virtual_fields(std::move(shared_virtual_fields_)) , storage_snapshot(storage_snapshot_) @@ -121,7 +119,7 @@ void MergeTreeReadPoolBase::fillPerPartInfos(const Settings & settings) } read_task_info.part_index_in_query = part_with_ranges.part_index_in_query; - read_task_info.alter_conversions = MergeTreeData::getAlterConversionsForPart(part_with_ranges.data_part, mutations_snapshot); + read_task_info.alter_conversions = MergeTreeData::getAlterConversionsForPart(part_with_ranges.data_part, mutations_snapshot, storage_snapshot->metadata, getContext()); LoadedMergeTreeDataPartInfoForReader part_info(part_with_ranges.data_part, read_task_info.alter_conversions); diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.h b/src/Storages/MergeTree/MergeTreeReadPoolBase.h index 7b4e034d892..7f9106d476e 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.h @@ -6,7 +6,7 @@ namespace DB { -class MergeTreeReadPoolBase : public IMergeTreeReadPool +class MergeTreeReadPoolBase : public IMergeTreeReadPool, protected WithContext { public: using MutationsSnapshotPtr = MergeTreeData::MutationsSnapshotPtr; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a95cccdc0d2..86e63782c18 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2125,7 +2125,7 @@ bool MutateTask::prepare() }; auto mutations_snapshot = ctx->data->getMutationsSnapshot(params); - auto alter_conversions = MergeTreeData::getAlterConversionsForPart(ctx->source_part, mutations_snapshot); + auto alter_conversions = MergeTreeData::getAlterConversionsForPart(ctx->source_part, mutations_snapshot, ctx->metadata_snapshot, ctx->context); auto context_for_reading = Context::createCopy(ctx->context); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index c3d91ca0705..8661b2aa784 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -950,7 +950,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper { const auto commands = entry.commands; it = mutations_by_znode.erase(it); - decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, state_lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands); } else it = mutations_by_znode.erase(it); @@ -1000,7 +1000,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper for (const ReplicatedMergeTreeMutationEntryPtr & entry : new_mutations) { auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version)).first->second; - incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands, lock); + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands); NOEXCEPT_SCOPE({ for (const auto & pair : entry->block_numbers) @@ -1961,6 +1961,23 @@ MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCo return result; } +NameSet ReplicatedMergeTreeQueue::MutationsSnapshot::getAllUpdatedColumns() const +{ + if (!params.need_data_mutations) + return {}; + + NameSet res; + for (const auto & [partition_id, mutations] : mutations_by_partition) + { + for (const auto & [version, entry] : mutations) + { + auto names = entry->commands.getAllUpdatedColumns(); + std::move(names.begin(), names.end(), std::inserter(res, res.end())); + } + } + return res; +} + MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapshot(const MutationsSnapshot::Params & params) const { std::lock_guard lock(state_mutex); @@ -2122,7 +2139,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep mutation.parts_to_do.clear(); } - decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, mutation.entry->commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, mutation.entry->commands); } else if (mutation.parts_to_do.size() == 0) { @@ -2179,7 +2196,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep LOG_TRACE(log, "Finishing data alter with version {} for entry {}", entry->alter_version, entry->znode_name); alter_sequence.finishDataAlter(entry->alter_version, lock); } - decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands); } } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index af8e2521f81..91a23b6a3b6 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -426,6 +426,7 @@ public: MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const override; std::shared_ptr cloneEmpty() const override { return std::make_shared(); } + NameSet getAllUpdatedColumns() const override; }; /// Return mutation commands for part which could be not applied to diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index f736c863eee..1aa9f5e23f8 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -268,4 +268,13 @@ bool MutationCommands::containBarrierCommand() const return false; } +NameSet MutationCommands::getAllUpdatedColumns() const +{ + NameSet res; + for (const auto & command : *this) + for (const auto & [column_name, _] : command.column_to_update_expression) + res.insert(column_name); + return res; +} + } diff --git a/src/Storages/MutationCommands.h b/src/Storages/MutationCommands.h index f999aab1f4d..5ae537bb657 100644 --- a/src/Storages/MutationCommands.h +++ b/src/Storages/MutationCommands.h @@ -92,6 +92,7 @@ public: /// stick with other commands. Commands from one set have already been validated /// to be executed without issues on the creation state. bool containBarrierCommand() const; + NameSet getAllUpdatedColumns() const; }; using MutationCommandsConstPtr = std::shared_ptr; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index da782b3634a..19cb2bec2e7 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -519,7 +519,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); - incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands, lock); + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands); } LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info); @@ -556,7 +556,7 @@ void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr re if (static_cast(result_part->part_info.mutation) == it->first) mutation_backoff_policy.removePartFromFailed(failed_part->name); - decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry.commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry.commands); } } else @@ -838,7 +838,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) { bool mutation_finished = *min_version > static_cast(mutation_version); if (!mutation_finished) - decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands, lock); + decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands); } to_kill.emplace(std::move(it->second)); @@ -923,7 +923,7 @@ void StorageMergeTree::loadMutations() if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number); - incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry_it->second.commands, lock); + incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry_it->second.commands); } else if (startsWith(it->name(), "tmp_mutation_")) { @@ -2464,6 +2464,20 @@ MutationCommands StorageMergeTree::MutationsSnapshot::getAlterMutationCommandsFo return result; } +NameSet StorageMergeTree::MutationsSnapshot::getAllUpdatedColumns() const +{ + if (!params.need_data_mutations) + return {}; + + NameSet res; + for (const auto & [version, commands] : mutations_by_version) + { + auto names = commands->getAllUpdatedColumns(); + std::move(names.begin(), names.end(), std::inserter(res, res.end())); + } + return res; +} + MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(const IMutationsSnapshot::Params & params) const { std::lock_guard lock(currently_processing_in_background_mutex); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 8ea0db37528..ef333fe3f18 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -320,6 +320,7 @@ private: MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const override; std::shared_ptr cloneEmpty() const override { return std::make_shared(); } + NameSet getAllUpdatedColumns() const override; }; MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const override; From aafe498b7f6045e75414c17abecad168aa0efb88 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 7 Aug 2024 10:43:49 +0800 Subject: [PATCH 030/259] 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 031/259] 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 032/259] 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 033/259] 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 034/259] 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 66534e379573875821b9c68c08642465a695c177 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Aug 2024 16:32:40 +0000 Subject: [PATCH 035/259] fix build --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 18cd6f2c1dd..37a59576edd 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1561,12 +1561,8 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) mutations_snapshot, context, query_info, -<<<<<<< HEAD - metadata_for_reading, + storage_snapshot->metadata, log); -======= - storage_snapshot->metadata); ->>>>>>> upstream/master } } From 0dc7cd7eb40bd3fb80e2c05871ce193a720b296c Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 14 Aug 2024 01:12:11 +0000 Subject: [PATCH 036/259] Update musl to have unwind info --- contrib/llvm-project-cmake/CMakeLists.txt | 6 ++++++ contrib/sysroot | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/contrib/llvm-project-cmake/CMakeLists.txt b/contrib/llvm-project-cmake/CMakeLists.txt index 76e620314a2..f5dce1c4178 100644 --- a/contrib/llvm-project-cmake/CMakeLists.txt +++ b/contrib/llvm-project-cmake/CMakeLists.txt @@ -140,6 +140,12 @@ if (CMAKE_CROSSCOMPILING) message (STATUS "CROSS COMPILING SET LLVM HOST TRIPLE ${LLVM_HOST_TRIPLE}") endif() +# llvm-project/llvm/cmake/config-ix.cmake does a weird thing: it defines _LARGEFILE64_SOURCE, +# then checks if lseek64() function exists, then undefines _LARGEFILE64_SOURCE. +# Then the actual code that uses this function *doesn't* define _LARGEFILE64_SOURCE, so lseek64() +# may not exist and compilation fails. This happens with musl. +add_compile_definitions("_LARGEFILE64_SOURCE") + add_subdirectory ("${LLVM_SOURCE_DIR}" "${LLVM_BINARY_DIR}") set_directory_properties (PROPERTIES diff --git a/contrib/sysroot b/contrib/sysroot index cc385041b22..866364fea62 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit cc385041b226d1fc28ead14dbab5d40a5f821dd8 +Subproject commit 866364fea629aa3e519ec967836561a6b3b21885 From 0716b460db35524f7cfa82501b5d1d2812904688 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 14 Aug 2024 03:00:51 +0000 Subject: [PATCH 037/259] Fix duplicate symbol errors --- base/harmful/harmful.c | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/base/harmful/harmful.c b/base/harmful/harmful.c index 54b552a84ea..19bb962999f 100644 --- a/base/harmful/harmful.c +++ b/base/harmful/harmful.c @@ -66,13 +66,11 @@ TRAP(gethostbyname) TRAP(gethostbyname2) TRAP(gethostent) TRAP(getlogin) -TRAP(getmntent) TRAP(getnetbyaddr) TRAP(getnetbyname) TRAP(getnetent) TRAP(getnetgrent) TRAP(getnetgrent_r) -TRAP(getopt) TRAP(getopt_long) TRAP(getopt_long_only) TRAP(getpass) @@ -133,7 +131,6 @@ TRAP(nrand48) TRAP(__ppc_get_timebase_freq) TRAP(ptsname) TRAP(putchar_unlocked) -TRAP(putenv) TRAP(pututline) TRAP(pututxline) TRAP(putwchar_unlocked) @@ -148,7 +145,6 @@ TRAP(sethostent) TRAP(sethostid) TRAP(setkey) //TRAP(setlocale) // Used by replxx at startup -TRAP(setlogmask) TRAP(setnetent) TRAP(setnetgrent) TRAP(setprotoent) @@ -203,7 +199,6 @@ TRAP(lgammal) TRAP(nftw) TRAP(nl_langinfo) TRAP(putc_unlocked) -TRAP(rand) /** In the current POSIX.1 specification (POSIX.1-2008), readdir() is not required to be thread-safe. However, in modern * implementations (including the glibc implementation), concurrent calls to readdir() that specify different directory streams * are thread-safe. In cases where multiple threads must read from the same directory stream, using readdir() with external @@ -288,4 +283,14 @@ TRAP(tss_get) TRAP(tss_set) TRAP(tss_delete) +#ifndef USE_MUSL +/// These produce duplicate symbol errors when statically linking with musl. +/// Maybe we can remove them from the musl fork. +TRAP(getopt) +TRAP(putenv) +TRAP(setlogmask) +TRAP(rand) +TRAP(getmntent) +#endif + #endif From c869b0651932bc61d5040395a3a2d0689485a5a1 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 14 Aug 2024 03:13:57 +0000 Subject: [PATCH 038/259] Remove getpwuid() calls in Poco::PathImpl --- base/poco/Foundation/src/Path_UNIX.cpp | 18 +++++------------- contrib/sysroot | 2 +- 2 files changed, 6 insertions(+), 14 deletions(-) diff --git a/base/poco/Foundation/src/Path_UNIX.cpp b/base/poco/Foundation/src/Path_UNIX.cpp index 957a62db180..fb2ed71622f 100644 --- a/base/poco/Foundation/src/Path_UNIX.cpp +++ b/base/poco/Foundation/src/Path_UNIX.cpp @@ -48,25 +48,17 @@ std::string PathImpl::currentImpl() std::string PathImpl::homeImpl() { std::string path; -#if defined(_POSIX_C_SOURCE) || defined(_BSD_SOURCE) || defined(_POSIX_C_SOURCE) size_t buf_size = 1024; // Same as glibc use for getpwuid std::vector buf(buf_size); struct passwd res; struct passwd* pwd = nullptr; getpwuid_r(getuid(), &res, buf.data(), buf_size, &pwd); -#else - struct passwd* pwd = getpwuid(getuid()); -#endif if (pwd) path = pwd->pw_dir; else { -#if defined(_POSIX_C_SOURCE) || defined(_BSD_SOURCE) || defined(_POSIX_C_SOURCE) getpwuid_r(getuid(), &res, buf.data(), buf_size, &pwd); -#else - pwd = getpwuid(geteuid()); -#endif if (pwd) path = pwd->pw_dir; else @@ -82,7 +74,7 @@ std::string PathImpl::configHomeImpl() { std::string path = PathImpl::homeImpl(); std::string::size_type n = path.size(); - if (n > 0 && path[n - 1] == '/') + if (n > 0 && path[n - 1] == '/') #if POCO_OS == POCO_OS_MAC_OS_X path.append("Library/Preferences/"); #else @@ -97,7 +89,7 @@ std::string PathImpl::dataHomeImpl() { std::string path = PathImpl::homeImpl(); std::string::size_type n = path.size(); - if (n > 0 && path[n - 1] == '/') + if (n > 0 && path[n - 1] == '/') #if POCO_OS == POCO_OS_MAC_OS_X path.append("Library/Application Support/"); #else @@ -112,7 +104,7 @@ std::string PathImpl::cacheHomeImpl() { std::string path = PathImpl::homeImpl(); std::string::size_type n = path.size(); - if (n > 0 && path[n - 1] == '/') + if (n > 0 && path[n - 1] == '/') #if POCO_OS == POCO_OS_MAC_OS_X path.append("Library/Caches/"); #else @@ -127,7 +119,7 @@ std::string PathImpl::tempHomeImpl() { std::string path = PathImpl::homeImpl(); std::string::size_type n = path.size(); - if (n > 0 && path[n - 1] == '/') + if (n > 0 && path[n - 1] == '/') #if POCO_OS == POCO_OS_MAC_OS_X path.append("Library/Caches/"); #else @@ -159,7 +151,7 @@ std::string PathImpl::tempImpl() std::string PathImpl::configImpl() { std::string path; - + #if POCO_OS == POCO_OS_MAC_OS_X path = "/Library/Preferences/"; #else diff --git a/contrib/sysroot b/contrib/sysroot index 866364fea62..c2d74e21ba1 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit 866364fea629aa3e519ec967836561a6b3b21885 +Subproject commit c2d74e21ba1b8a27966e344693e176f927e4eb50 From 7a4bd49c4285ef3489d90aa163fb6b1858a43562 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 14 Aug 2024 17:16:36 +0000 Subject: [PATCH 039/259] More improvements in integration test --- .../test_alter_settings_on_cluster/test.py | 5 +++ .../test_always_fetch_merged/test.py | 3 ++ .../test.py | 6 +-- .../test_async_load_databases/test.py | 14 +++++-- .../test.py | 42 +++++++++---------- 5 files changed, 42 insertions(+), 28 deletions(-) diff --git a/tests/integration/test_alter_settings_on_cluster/test.py b/tests/integration/test_alter_settings_on_cluster/test.py index 32f7f2efa30..7a552c383c9 100644 --- a/tests/integration/test_alter_settings_on_cluster/test.py +++ b/tests/integration/test_alter_settings_on_cluster/test.py @@ -73,3 +73,8 @@ def test_default_database_on_cluster(started_cluster): database="test_default_database", sql="SHOW CREATE test_local_table FORMAT TSV", ).endswith("old_parts_lifetime = 100\n") + + ch1.query( + database="test_default_database", + sql="DROP TABLE test_local_table ON CLUSTER 'cluster' SYNC", + ) diff --git a/tests/integration/test_always_fetch_merged/test.py b/tests/integration/test_always_fetch_merged/test.py index ca8e775fb97..3bbfc8867f8 100644 --- a/tests/integration/test_always_fetch_merged/test.py +++ b/tests/integration/test_always_fetch_merged/test.py @@ -80,3 +80,6 @@ def test_replica_always_download(started_cluster): assert int(node1_parts) < 10 assert int(node2_parts) < 10 + + node1.query_with_retry("DROP TABLE test_table SYNC") + node2.query_with_retry("DROP TABLE test_table SYNC") diff --git a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py index 93319a56d0f..0ea076b1468 100644 --- a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py +++ b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py @@ -157,7 +157,7 @@ def test_with_replicated_merge_tree(): array_size_range=[10, 50], ) - node.query("DROP TABLE IF EXISTS {}".format(table_name)) + node.query("DROP TABLE {} SYNC".format(table_name)) def test_with_replicated_merge_tree_multithread(): @@ -185,7 +185,7 @@ def test_with_replicated_merge_tree_multithread(): array_size_range=[10, 15], ) - node.query("DROP TABLE IF EXISTS {}".format(table_name)) + node.query("DROP TABLE {} SYNC".format(table_name)) # Ensure that the combined duration of inserts with adaptive timeouts is less than @@ -369,4 +369,4 @@ def test_change_queries_frequency(): for line in res.splitlines(): assert int(line) == min_ms - node.query("DROP TABLE IF EXISTS {}".format(table_name)) + node.query("DROP TABLE IF EXISTS {} SYNC".format(table_name)) diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py index d06897b1045..f36cff76ea2 100644 --- a/tests/integration/test_async_load_databases/test.py +++ b/tests/integration/test_async_load_databases/test.py @@ -28,9 +28,6 @@ def started_cluster(): """ CREATE DATABASE IF NOT EXISTS dict ENGINE=Dictionary; CREATE DATABASE IF NOT EXISTS test; - DROP TABLE IF EXISTS test.elements; - CREATE TABLE test.elements (id UInt64, a String, b Int32, c Float64) ENGINE=Log; - INSERT INTO test.elements VALUES (0, 'water', 10, 1), (1, 'air', 40, 0.01), (2, 'earth', 100, 1.7); """ ) @@ -49,6 +46,13 @@ def get_status(dictionary_name): def test_dict_get_data(started_cluster): query = instance.query + query( + "CREATE TABLE test.elements (id UInt64, a String, b Int32, c Float64) ENGINE=Log;" + ) + query( + "INSERT INTO test.elements VALUES (0, 'water', 10, 1), (1, 'air', 40, 0.01), (2, 'earth', 100, 1.7);" + ) + # dictionaries_lazy_load == false, so these dictionary are not loaded. assert get_status("dep_x") == "NOT_LOADED" assert get_status("dep_y") == "NOT_LOADED" @@ -97,6 +101,8 @@ def test_dict_get_data(started_cluster): assert query("SELECT dictGetString('dep_x', 'a', toUInt64(4))") == "XX\n" assert query("SELECT dictGetString('dep_y', 'a', toUInt64(4))") == "ether\n" assert query("SELECT dictGetString('dep_z', 'a', toUInt64(4))") == "ZZ\n" + query("DROP TABLE IF EXISTS test.elements;") + instance.restart_clickhouse() def dependent_tables_assert(): @@ -175,3 +181,5 @@ def test_multiple_tables(started_cluster): random.shuffle(order) for i in order: assert query(f"select count() from test.table_{i}") == "100\n" + for i in range(tables_count): + query(f"drop table test.table_{i}") diff --git a/tests/integration/test_asynchronous_metric_log_table/test.py b/tests/integration/test_asynchronous_metric_log_table/test.py index 622620e232a..efdd96004d2 100644 --- a/tests/integration/test_asynchronous_metric_log_table/test.py +++ b/tests/integration/test_asynchronous_metric_log_table/test.py @@ -26,26 +26,24 @@ def started_cluster(): # asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and # are available for querying during the test. def test_event_time_microseconds_field(started_cluster): - try: - cluster.start() - node1.query("SET log_queries = 1;") - node1.query("CREATE DATABASE replica;") - query_create = """CREATE TABLE replica.test - ( - id Int64, - event_time DateTime - ) - Engine=MergeTree() - PARTITION BY toYYYYMMDD(event_time) - ORDER BY id;""" - time.sleep(2) - node1.query(query_create) - node1.query("""INSERT INTO replica.test VALUES (1, now())""") - node1.query("SYSTEM FLUSH LOGS;") + node1.query("SET log_queries = 1;") + node1.query("CREATE DATABASE replica;") + query_create = """CREATE TABLE replica.test + ( + id Int64, + event_time DateTime + ) + Engine=MergeTree() + PARTITION BY toYYYYMMDD(event_time) + ORDER BY id;""" + time.sleep(2) + node1.query(query_create) + node1.query("""INSERT INTO replica.test VALUES (1, now())""") + node1.query("SYSTEM FLUSH LOGS;") - test_query = ( - "SELECT count() > 0 ? 'ok' : 'fail' FROM system.asynchronous_metric_log" - ) - assert "ok\n" in node1.query(test_query) - finally: - cluster.shutdown() + test_query = ( + "SELECT count() > 0 ? 'ok' : 'fail' FROM system.asynchronous_metric_log" + ) + assert "ok\n" in node1.query(test_query) + node1.query("DROP TABLE replica.test") + node1.query("DROP DATABASE replica") From d31b36ca50d8905b8eaac945c197c425d5ddb208 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 15 Aug 2024 07:16:01 +0000 Subject: [PATCH 040/259] Rework integration flaky check as it was timeouting too fast as run as a single group --- tests/ci/integration_tests_runner.py | 98 +++++++++++++++++----------- 1 file changed, 60 insertions(+), 38 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 2b348be8b51..1513a8bee4f 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -784,40 +784,65 @@ class ClickhouseIntegrationTestsRunner: logging.info("Starting check with retries") final_retry = 0 logs = [] - tires_num = 1 if should_fail else FLAKY_TRIES_COUNT - for i in range(tires_num): - final_retry += 1 - logging.info("Running tests for the %s time", i) - counters, tests_times, log_paths = self.try_run_test_group( - repo_path, - "bugfix" if should_fail else "flaky", - tests_to_run, - 1, - 1, - FLAKY_REPEAT_COUNT, - ) - logs += log_paths - if counters["FAILED"]: - logging.info("Found failed tests: %s", " ".join(counters["FAILED"])) - description_prefix = "Failed tests found: " - result_state = "failure" - if not should_fail: + counters = { + "ERROR": [], + "PASSED": [], + "FAILED": [], + "SKIPPED": [], + "BROKEN": [], + "NOT_FAILED": [], + } # type: Dict + tests_times = defaultdict(float) # type: Dict + tests_log_paths = defaultdict(list) + + for test_to_run in tests_to_run: + tries_num = 1 if should_fail else FLAKY_TRIES_COUNT + for i in range(tries_num): + final_retry += 1 + logging.info("Running tests for the %s time", i) + group_counters, group_test_times, log_paths = self.try_run_test_group( + repo_path, + "bugfix" if should_fail else "flaky", + [test_to_run], + 1, + 1, + FLAKY_REPEAT_COUNT, + ) + for counter, value in group_counters.items(): + logging.info( + "Tests from group %s stats, %s count %s", + test_to_run, + counter, + len(value), + ) + counters[counter] += value + + for test_name, test_time in group_test_times.items(): + tests_times[test_name] = test_time + tests_log_paths[test_name] = log_paths + if not should_fail and ( + group_counters["FAILED"] or group_counters["ERROR"] + ): + logging.info( + "Unexpected failure in group %s. Fail fast for current group", + test_to_run, + ) break - if counters["ERROR"]: - description_prefix = "Failed tests found: " - logging.info("Found error tests: %s", " ".join(counters["ERROR"])) - # NOTE "error" result state will restart the whole test task, - # so we use "failure" here - result_state = "failure" - if not should_fail: - break - logging.info("Try is OK, all tests passed, going to clear env") - clear_ip_tables_and_restart_daemons() - logging.info("And going to sleep for some time") - if time.time() - start > MAX_TIME_SECONDS: - logging.info("Timeout reached, going to finish flaky check") - break - time.sleep(5) + + if group_counters["FAILED"]: + logging.info("Found failed tests: %s", " ".join(counters["FAILED"])) + description_prefix = "Failed tests found: " + result_state = "failure" + if group_counters["ERROR"]: + description_prefix = "Failed tests found: " + logging.info("Found error tests: %s", " ".join(counters["ERROR"])) + # NOTE "error" result state will restart the whole test task, + # so we use "failure" here + result_state = "failure" + logging.info("Try is OK, all tests passed, going to clear env") + clear_ip_tables_and_restart_daemons() + logging.info("And going to sleep for some time") + time.sleep(5) test_result = [] for state in ("ERROR", "FAILED", "PASSED", "SKIPPED"): @@ -828,13 +853,10 @@ class ClickhouseIntegrationTestsRunner: else: text_state = state test_result += [ - ( - c + " (✕" + str(final_retry) + ")", - text_state, - f"{tests_times[c]:.2f}", - ) + (c, text_state, f"{tests_times[c]:.2f}", tests_log_paths[c]) for c in counters[state] ] + status_text = description_prefix + ", ".join( [ str(n).lower().replace("failed", "fail") + ": " + str(len(c)) From c48b6d25f763d9c38b4e7ed35dec9a98d913e5c7 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 15 Aug 2024 08:58:36 +0000 Subject: [PATCH 041/259] more tests --- .../test_filesystem_layout/test.py | 4 ++++ .../integration/test_grant_and_revoke/test.py | 2 ++ .../test_parallel_replicas_failover/test.py | 19 ++++++++++--------- .../test.py | 4 ++-- 4 files changed, 18 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_filesystem_layout/test.py b/tests/integration/test_filesystem_layout/test.py index 4e719aa0fe9..31d6c830a2f 100644 --- a/tests/integration/test_filesystem_layout/test.py +++ b/tests/integration/test_filesystem_layout/test.py @@ -79,3 +79,7 @@ def test_file_path_escaping(started_cluster): "test -f /var/lib/clickhouse/shadow/2/store/123/12345678-1000-4000-8000-000000000001/1_1_1_0/%7EId.bin", ] ) + node.query("DROP TABLE test.`T.a_b,l-e!` SYNC") + node.query("DROP TABLE `test 2`.`T.a_b,l-e!` SYNC") + node.query("DROP DATABASE test") + node.query("DROP DATABASE `test 2`") diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index e533cced1e4..81cba966cae 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -359,6 +359,8 @@ def test_implicit_create_view_grant(): instance.query("GRANT CREATE VIEW ON test.* TO B", user="A") instance.query("CREATE VIEW test.view_2 AS SELECT 1", user="B") assert instance.query("SELECT * FROM test.view_2") == "1\n" + instance.query("DROP USER A") + instance.query("DROP VIEW test.view_2") def test_implicit_create_temporary_table_grant(): diff --git a/tests/integration/test_parallel_replicas_failover/test.py b/tests/integration/test_parallel_replicas_failover/test.py index bf25136bff7..2da26ee03c9 100644 --- a/tests/integration/test_parallel_replicas_failover/test.py +++ b/tests/integration/test_parallel_replicas_failover/test.py @@ -1,5 +1,5 @@ import pytest - +import uuid from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -25,19 +25,15 @@ def start_cluster(): def create_tables(cluster, table_name, skip_last_replica): - node1.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node2.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node3.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node1.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" ) node2.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') ORDER BY (key)" ) if not skip_last_replica: node3.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" ) # populate data @@ -67,7 +63,7 @@ def test_skip_replicas_without_table(start_cluster): for i in range(4): expected_result += f"{i}\t1000\n" - log_comment = "5230b069-9574-407d-9b80-891b5a175f41" + log_comment = uuid.uuid4() assert ( node1.query( f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", @@ -88,6 +84,8 @@ def test_skip_replicas_without_table(start_cluster): ) == "1\t1\n" ) + node1.query(f"DROP TABLE {table_name} SYNC") + node2.query(f"DROP TABLE {table_name} SYNC") def test_skip_unresponsive_replicas(start_cluster): @@ -112,3 +110,6 @@ def test_skip_unresponsive_replicas(start_cluster): ) == expected_result ) + node1.query(f"DROP TABLE {table_name} SYNC") + node2.query(f"DROP TABLE {table_name} SYNC") + node3.query(f"DROP TABLE {table_name} SYNC") diff --git a/tests/integration/test_parallel_replicas_invisible_parts/test.py b/tests/integration/test_parallel_replicas_invisible_parts/test.py index cab3fb46fe9..7093e3b3292 100644 --- a/tests/integration/test_parallel_replicas_invisible_parts/test.py +++ b/tests/integration/test_parallel_replicas_invisible_parts/test.py @@ -35,11 +35,10 @@ def start_cluster(): def _create_tables(table_name, table_size, index_granularity): - nodes[0].query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER {cluster_name}") nodes[0].query( f""" - CREATE TABLE IF NOT EXISTS {table_name} ON CLUSTER '{cluster_name}' (key Int64, value String) + CREATE TABLE {table_name} ON CLUSTER '{cluster_name}' (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard/{table_name}/', '{{replica}}') ORDER BY (key) SETTINGS index_granularity = {index_granularity}, max_bytes_to_merge_at_max_space_in_pool = 0, max_bytes_to_merge_at_max_space_in_pool = 1 @@ -128,3 +127,4 @@ def test_reading_with_invisible_parts( ) == f"{expected}\n" ) + nodes[0].query(f"DROP TABLE {table_name} ON CLUSTER {cluster_name} SYNC") From 624afd68c86bb4fc3eee76f3fb9779ffb2319c68 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Fri, 16 Aug 2024 01:24:52 +0400 Subject: [PATCH 042/259] 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 b1963738bdf8e02e78a0be1b2cce1b992a6d533c Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 15 Aug 2024 21:41:40 +0000 Subject: [PATCH 043/259] Rebase --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index c2d74e21ba1..eb35c10ac57 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit c2d74e21ba1b8a27966e344693e176f927e4eb50 +Subproject commit eb35c10ac5725da7ef4be88b303895e1b5d153be From c31c4154ca47c6f1313c04974b7bb95bbcb34cff Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Fri, 16 Aug 2024 01:49:54 +0400 Subject: [PATCH 044/259] 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 b3a3d1e7202837e2da45b6775b4ea8c389c881a7 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 16 Aug 2024 05:53:17 +0000 Subject: [PATCH 045/259] another rebase --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index eb35c10ac57..73752937366 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit eb35c10ac5725da7ef4be88b303895e1b5d153be +Subproject commit 737529373665bc067971ba098a12d6928580a0ae From 7c691cbb797594d0c6649b36be13ff99c3c4664b Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Fri, 16 Aug 2024 10:44:11 +0200 Subject: [PATCH 046/259] style --- tests/ci/integration_tests_runner.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index f63afc297e6..006958abe6d 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -774,10 +774,8 @@ class ClickhouseIntegrationTestsRunner: logging.info("Found '%s' tests to run", " ".join(tests_to_run)) result_state = "success" description_prefix = "No flaky tests: " - start = time.time() logging.info("Starting check with retries") final_retry = 0 - logs = [] counters = { "ERROR": [], "PASSED": [], @@ -858,7 +856,7 @@ class ClickhouseIntegrationTestsRunner: ] ) - return result_state, status_text, test_result, logs + return result_state, status_text, test_result, tests_log_paths def run_impl(self, repo_path, build_path): if self.flaky_check or self.bugfix_validate_check: From 46f27b03f9dd2f122da6cfcb9905f5bb7ec944e7 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Fri, 16 Aug 2024 15:48:28 +0400 Subject: [PATCH 047/259] 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 32f4b1f89143a293047beb83169d83c3e20c5480 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 17:49:38 +0000 Subject: [PATCH 048/259] Add aggregate functions distinctDynamicTypes/distinctJSONPaths/distinctJSONPathsAndTypes --- .../reference/distinctdynamictypes.md | 44 +++ .../reference/distinctjsonpaths.md | 84 +++++ docs/en/sql-reference/data-types/newjson.md | 125 ++++++- .../AggregateFunctionDistinctDynamicTypes.cpp | 155 ++++++++ .../AggregateFunctionDistinctJSONPaths.cpp | 331 ++++++++++++++++++ .../registerAggregateFunctions.cpp | 4 + src/Columns/ColumnDynamic.cpp | 35 ++ src/Columns/ColumnDynamic.h | 3 + ...istinct_dynamic_types_json_paths.reference | 121 +++++++ ...3227_distinct_dynamic_types_json_paths.sql | 50 +++ 10 files changed, 951 insertions(+), 1 deletion(-) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/distinctdynamictypes.md create mode 100644 docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md create mode 100644 src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp create mode 100644 tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference create mode 100644 tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql diff --git a/docs/en/sql-reference/aggregate-functions/reference/distinctdynamictypes.md b/docs/en/sql-reference/aggregate-functions/reference/distinctdynamictypes.md new file mode 100644 index 00000000000..970209252fa --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/distinctdynamictypes.md @@ -0,0 +1,44 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/distinctdynamictypes +sidebar_position: 215 +--- + +# distinctDynamicTypes + +Calculates the list of distinct data types stored in [Dynamic](../../data-types/dynamic.md) column. + +**Syntax** + +```sql +distinctDynamicTypes(dynamic) +``` + +**Arguments** + +- `dynamic` — [Dynamic](../../data-types/dynamic.md) column. + +**Returned Value** + +- The sorted list of data type names [Array(String)](../../data-types/array.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test_dynamic; +CREATE TABLE test_dynamic(d Dynamic) ENGINE = Memory; +INSERT INTO test_dynamic VALUES (42), (NULL), ('Hello'), ([1, 2, 3]), ('2020-01-01'), (map(1, 2)), (43), ([4, 5]), (NULL), ('World'), (map(3, 4)) +``` + +```sql +SELECT distinctDynamicTypes(d) FROM test_dynamic; +``` + +Result: + +```reference +┌─distinctDynamicTypes(d)──────────────────────────────────────┐ +│ ['Array(Int64)','Date','Int64','Map(UInt8, UInt8)','String'] │ +└──────────────────────────────────────────────────────────────┘ +``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md b/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md new file mode 100644 index 00000000000..f916734ca44 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md @@ -0,0 +1,84 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/distinctjsonpaths +sidebar_position: 216 +--- + +# distinctJSONPaths + +Calculates the list of distinct paths stored in [JSON](../../data-types/newjson.md) column. + +**Syntax** + +```sql +distinctJSONPaths(json) +``` + +**Arguments** + +- `json` — [JSON](../../data-types/newjson.md) column. + +**Returned Value** + +- The sorted list of paths [Array(String)](../../data-types/array.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test_json; +CREATE TABLE test_json(json JSON) ENGINE = Memory; +INSERT INTO test_json VALUES ('{"a" : 42, "b" : "Hello"}'), ('{"b" : [1, 2, 3], "c" : {"d" : {"e" : "2020-01-01"}}}'), ('{"a" : 43, "c" : {"d" : {"f" : [{"g" : 42}]}}}') +``` + +```sql +SELECT distinctJSONPaths(json) FROM test_json; +``` + +Result: + +```reference +┌─distinctJSONPaths(json)───┐ +│ ['a','b','c.d.e','c.d.f'] │ +└───────────────────────────┘ +``` + +# distinctJSONPathsAndTypes + +Calculates the list of distinct paths and their types stored in [JSON](../../data-types/newjson.md) column. + +**Syntax** + +```sql +distinctJSONPathsAndTypes(json) +``` + +**Arguments** + +- `json` — [JSON](../../data-types/newjson.md) column. + +**Returned Value** + +- The sorted map of paths and types [Map(String, Array(String))](../../data-types/map.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test_json; +CREATE TABLE test_json(json JSON) ENGINE = Memory; +INSERT INTO test_json VALUES ('{"a" : 42, "b" : "Hello"}'), ('{"b" : [1, 2, 3], "c" : {"d" : {"e" : "2020-01-01"}}}'), ('{"a" : 43, "c" : {"d" : {"f" : [{"g" : 42}]}}}') +``` + +```sql +SELECT distinctJSONPathsAndTypes(json) FROM test_json; +``` + +Result: + +```reference +┌─distinctJSONPathsAndTypes(json)───────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ {'a':['Int64'],'b':['Array(Nullable(Int64))','String'],'c.d.e':['Date'],'c.d.f':['Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))']} │ +└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index 9e43216df6c..16cc324a168 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -505,7 +505,130 @@ As we can see, ClickHouse kept the most frequent paths `a`, `b` and `c` and move ## Introspection functions -There are several functions that can help to inspect the content of the JSON column: [JSONAllPaths](../functions/json-functions.md#jsonallpaths), [JSONAllPathsWithTypes](../functions/json-functions.md#jsonallpathswithtypes), [JSONDynamicPaths](../functions/json-functions.md#jsondynamicpaths), [JSONDynamicPathsWithTypes](../functions/json-functions.md#jsondynamicpathswithtypes), [JSONSharedDataPaths](../functions/json-functions.md#jsonshareddatapaths), [JSONSharedDataPathsWithTypes](../functions/json-functions.md#jsonshareddatapathswithtypes). +There are several functions that can help to inspect the content of the JSON column: [JSONAllPaths](../functions/json-functions.md#jsonallpaths), [JSONAllPathsWithTypes](../functions/json-functions.md#jsonallpathswithtypes), [JSONDynamicPaths](../functions/json-functions.md#jsondynamicpaths), [JSONDynamicPathsWithTypes](../functions/json-functions.md#jsondynamicpathswithtypes), [JSONSharedDataPaths](../functions/json-functions.md#jsonshareddatapaths), [JSONSharedDataPathsWithTypes](../functions/json-functions.md#jsonshareddatapathswithtypes), [distinctDynamicTypes](../aggregate-functions/reference/distinctdynamictypes.md), [distinctJSONPaths and distinctJSONPathsAndTypes](../aggregate-functions/reference/distinctjsonpaths.md) + +**Examples** + +Let's investigate the content of [GH Archive](https://www.gharchive.org/) dataset for `2020-01-01` date: + +```sql +SELECT arrayJoin(distinctJSONPaths(json)) FROM s3('s3://clickhouse-public-datasets/gharchive/original/2020-01-01-*.json.gz', JSONAsObject) +``` + +```text +┌─arrayJoin(distinctJSONPaths(json))─────────────────────────┐ +│ actor.avatar_url │ +│ actor.display_login │ +│ actor.gravatar_id │ +│ actor.id │ +│ actor.login │ +│ actor.url │ +│ created_at │ +│ id │ +│ org.avatar_url │ +│ org.gravatar_id │ +│ org.id │ +│ org.login │ +│ org.url │ +│ payload.action │ +│ payload.before │ +│ payload.comment._links.html.href │ +│ payload.comment._links.pull_request.href │ +│ payload.comment._links.self.href │ +│ payload.comment.author_association │ +│ payload.comment.body │ +│ payload.comment.commit_id │ +│ payload.comment.created_at │ +│ payload.comment.diff_hunk │ +│ payload.comment.html_url │ +│ payload.comment.id │ +│ payload.comment.in_reply_to_id │ +│ payload.comment.issue_url │ +│ payload.comment.line │ +│ payload.comment.node_id │ +│ payload.comment.original_commit_id │ +│ payload.comment.original_position │ +│ payload.comment.path │ +│ payload.comment.position │ +│ payload.comment.pull_request_review_id │ +... +│ payload.release.node_id │ +│ payload.release.prerelease │ +│ payload.release.published_at │ +│ payload.release.tag_name │ +│ payload.release.tarball_url │ +│ payload.release.target_commitish │ +│ payload.release.upload_url │ +│ payload.release.url │ +│ payload.release.zipball_url │ +│ payload.size │ +│ public │ +│ repo.id │ +│ repo.name │ +│ repo.url │ +│ type │ +└─arrayJoin(distinctJSONPaths(json))─────────────────────────┘ +``` + +```sql +SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-public-datasets/gharchive/original/2020-01-01-*.json.gz', JSONAsObject) +``` + + +```text +┌─arrayJoin(distinctJSONPathsAndTypes(json))──────────────────┐ +│ ('actor.avatar_url',['String']) │ +│ ('actor.display_login',['String']) │ +│ ('actor.gravatar_id',['String']) │ +│ ('actor.id',['Int64']) │ +│ ('actor.login',['String']) │ +│ ('actor.url',['String']) │ +│ ('created_at',['String']) │ +│ ('id',['String']) │ +│ ('org.avatar_url',['String']) │ +│ ('org.gravatar_id',['String']) │ +│ ('org.id',['Int64']) │ +│ ('org.login',['String']) │ +│ ('org.url',['String']) │ +│ ('payload.action',['String']) │ +│ ('payload.before',['String']) │ +│ ('payload.comment._links.html.href',['String']) │ +│ ('payload.comment._links.pull_request.href',['String']) │ +│ ('payload.comment._links.self.href',['String']) │ +│ ('payload.comment.author_association',['String']) │ +│ ('payload.comment.body',['String']) │ +│ ('payload.comment.commit_id',['String']) │ +│ ('payload.comment.created_at',['String']) │ +│ ('payload.comment.diff_hunk',['String']) │ +│ ('payload.comment.html_url',['String']) │ +│ ('payload.comment.id',['Int64']) │ +│ ('payload.comment.in_reply_to_id',['Int64']) │ +│ ('payload.comment.issue_url',['String']) │ +│ ('payload.comment.line',['Int64']) │ +│ ('payload.comment.node_id',['String']) │ +│ ('payload.comment.original_commit_id',['String']) │ +│ ('payload.comment.original_position',['Int64']) │ +│ ('payload.comment.path',['String']) │ +│ ('payload.comment.position',['Int64']) │ +│ ('payload.comment.pull_request_review_id',['Int64']) │ +... +│ ('payload.release.node_id',['String']) │ +│ ('payload.release.prerelease',['Bool']) │ +│ ('payload.release.published_at',['String']) │ +│ ('payload.release.tag_name',['String']) │ +│ ('payload.release.tarball_url',['String']) │ +│ ('payload.release.target_commitish',['String']) │ +│ ('payload.release.upload_url',['String']) │ +│ ('payload.release.url',['String']) │ +│ ('payload.release.zipball_url',['String']) │ +│ ('payload.size',['Int64']) │ +│ ('public',['Bool']) │ +│ ('repo.id',['Int64']) │ +│ ('repo.name',['String']) │ +│ ('repo.url',['String']) │ +│ ('type',['String']) │ +└─arrayJoin(distinctJSONPathsAndTypes(json))──────────────────┘ +``` ## Tips for better usage of the JSON type diff --git a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp new file mode 100644 index 00000000000..6c899d6cda8 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp @@ -0,0 +1,155 @@ +#include +#include +#include + +#include +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +struct AggregateFunctionDistinctDynamicTypesData +{ + std::unordered_set data; + + void add(const String & type) + { + data.insert(type); + } + + void merge(const AggregateFunctionDistinctDynamicTypesData & other) + { + data.insert(other.data.begin(), other.data.end()); + } + + void serialize(WriteBuffer & buf) const + { + writeVarUInt(data.size(), buf); + for (const auto & type : data) + writeStringBinary(type, buf); + } + + void deserialize(ReadBuffer & buf) + { + size_t size; + readVarUInt(size, buf); + data.reserve(size); + String type; + for (size_t i = 0; i != size; ++i) + { + readStringBinary(type, buf); + data.insert(type); + } + } + + void insertResultInto(IColumn & column) + { + /// Insert types in sorted order for better output. + auto & array_column = assert_cast(column); + auto & string_column = assert_cast(array_column.getData()); + std::vector sorted_data(data.begin(), data.end()); + std::sort(sorted_data.begin(), sorted_data.end()); + for (const auto & type : sorted_data) + string_column.insertData(type.data(), type.size()); + array_column.getOffsets().push_back(string_column.size()); + } +}; + +/// Calculates the list of distinct data types in Dynamic column. +class AggregateFunctionDistinctDynamicTypes final : public IAggregateFunctionDataHelper +{ +public: + explicit AggregateFunctionDistinctDynamicTypes(const DataTypes & argument_types_) + : IAggregateFunctionDataHelper(argument_types_, {}, std::make_shared(std::make_shared())) + { + } + + String getName() const override { return "distinctDynamicTypes"; } + + bool allocatesMemoryInArena() const override { return false; } + + void ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override + { + const auto & dynamic_column = assert_cast(*columns[0]); + if (dynamic_column.isNullAt(row_num)) + return; + + this->data(place).add(dynamic_column.getTypeNameAt(row_num)); + } + + void ALWAYS_INLINE addBatchSinglePlace( + size_t row_begin, size_t row_end, AggregateDataPtr __restrict place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos) + const override + { + if (if_argument_pos >= 0 || row_begin != 0 || row_end != columns[0]->size()) + IAggregateFunctionDataHelper::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos); + /// Optimization for case when we add all rows from the column into single place. + /// In this case we can avoid iterating over all rows because we can get all types + /// in Dynamic column in a more efficient way. + else + assert_cast(*columns[0]).getAllTypeNames(this->data(place).data); + } + + void addManyDefaults( + AggregateDataPtr __restrict /*place*/, + const IColumn ** /*columns*/, + size_t /*length*/, + Arena * /*arena*/) const override + { + /// Default value for Dynamic is NULL, so nothing to add. + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override + { + this->data(place).merge(this->data(rhs)); + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override + { + this->data(place).serialize(buf); + } + + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + { + this->data(place).deserialize(buf); + } + + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override + { + this->data(place).insertResultInto(to); + } +}; + +AggregateFunctionPtr createAggregateFunctionDistinctDynamicTypes( + const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) +{ + assertNoParameters(name, parameters); + if (argument_types.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Incorrect number of arguments for aggregate function {}. Expected single argument with type Dynamic, got {} arguments", name, argument_types.size()); + + if (!isDynamic(argument_types[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}. Expected type Dynamic", argument_types[0]->getName(), name); + + return std::make_shared(argument_types); +} + +void registerAggregateFunctionDistinctDynamicTypes(AggregateFunctionFactory & factory) +{ + factory.registerFunction("distinctDynamicTypes", createAggregateFunctionDistinctDynamicTypes); +} + +} diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp new file mode 100644 index 00000000000..a3ede0750f2 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -0,0 +1,331 @@ +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ + +struct AggregateFunctionDistinctJSONPathsData +{ + static constexpr auto name = "distinctJSONPaths"; + + std::unordered_set data; + + void add(const ColumnObject & column, size_t row_num, const std::unordered_map &) + { + for (const auto & [path, _] : column.getTypedPaths()) + data.insert(path); + for (const auto & [path, dynamic_column] : column.getDynamicPathsPtrs()) + { + /// Add path from dynamic paths only if it's not NULL in this row. + if (!dynamic_column->isNullAt(row_num)) + data.insert(path); + } + + /// Iterate over paths in shared data in this row. + const auto [shared_data_paths, _] = column.getSharedDataPathsAndValues(); + const auto & shared_data_offsets = column.getSharedDataOffsets(); + size_t start = shared_data_offsets[static_cast(row_num) - 1]; + size_t end = shared_data_offsets[static_cast(row_num)]; + for (size_t i = start; i != end; ++i) + data.insert(shared_data_paths->getDataAt(i).toString()); + } + + void addWholeColumn(const ColumnObject & column, const std::unordered_map &) + { + for (const auto & [path, _] : column.getTypedPaths()) + data.insert(path); + for (const auto & [path, dynamic_column] : column.getDynamicPathsPtrs()) + { + /// Add dynamic path only if it has at least one non-null value. + /// getNumberOfDefaultRows for Dynamic column is O(1). + if (dynamic_column->getNumberOfDefaultRows() != dynamic_column->size()) + data.insert(path); + } + + /// Iterate over all paths in shared data. + const auto [shared_data_paths, _] = column.getSharedDataPathsAndValues(); + for (size_t i = 0; i != shared_data_paths->size(); ++i) + data.insert(shared_data_paths->getDataAt(i).toString()); + } + + void merge(const AggregateFunctionDistinctJSONPathsData & other) + { + data.insert(other.data.begin(), other.data.end()); + } + + void serialize(WriteBuffer & buf) const + { + writeVarUInt(data.size(), buf); + for (const auto & path : data) + writeStringBinary(path, buf); + } + + void deserialize(ReadBuffer & buf) + { + size_t size; + readVarUInt(size, buf); + String path; + for (size_t i = 0; i != size; ++i) + { + readStringBinary(path, buf); + data.insert(path); + } + } + + void insertResultInto(IColumn & column) + { + /// Insert paths in sorted order for better output. + auto & array_column = assert_cast(column); + auto & string_column = assert_cast(array_column.getData()); + std::vector sorted_data(data.begin(), data.end()); + std::sort(sorted_data.begin(), sorted_data.end()); + for (const auto & path : sorted_data) + string_column.insertData(path.data(), path.size()); + array_column.getOffsets().push_back(string_column.size()); + } + + static DataTypePtr getResultType() + { + return std::make_shared(std::make_shared()); + } +}; + +struct AggregateFunctionDistinctJSONPathsAndTypesData +{ + static constexpr auto name = "distinctJSONPathsAndTypes"; + + std::unordered_map> data; + + void add(const ColumnObject & column, size_t row_num, const std::unordered_map & typed_paths_type_names) + { + for (const auto & [path, _] : column.getTypedPaths()) + data[path].insert(typed_paths_type_names.at(path)); + for (const auto & [path, dynamic_column] : column.getDynamicPathsPtrs()) + { + if (!dynamic_column->isNullAt(row_num)) + data[path].insert(dynamic_column->getTypeNameAt(row_num)); + } + + /// Iterate over paths om shared data in this row and decode the data types. + const auto [shared_data_paths, shared_data_values] = column.getSharedDataPathsAndValues(); + const auto & shared_data_offsets = column.getSharedDataOffsets(); + size_t start = shared_data_offsets[static_cast(row_num) - 1]; + size_t end = shared_data_offsets[static_cast(row_num)]; + for (size_t i = start; i != end; ++i) + { + auto path = shared_data_paths->getDataAt(i).toString(); + auto value = shared_data_values->getDataAt(i); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + /// We should not have Nulls here but let's check just in case. + if (!isNothing(type)) + data[path].insert(type->getName()); + } + } + + void addWholeColumn(const ColumnObject & column, const std::unordered_map & typed_paths_type_names) + { + for (const auto & [path, _] : column.getTypedPaths()) + data[path].insert(typed_paths_type_names.at(path)); + for (const auto & [path, dynamic_column] : column.getDynamicPathsPtrs()) + { + /// Add dynamic path only if it has at least one non-null value. + /// getNumberOfDefaultRows for Dynamic column is O(1). + if (dynamic_column->getNumberOfDefaultRows() != dynamic_column->size()) + dynamic_column->getAllTypeNames(data[path]); + } + + /// Iterate over all paths in shared data and decode the data types. + const auto [shared_data_paths, shared_data_values] = column.getSharedDataPathsAndValues(); + for (size_t i = 0; i != shared_data_paths->size(); ++i) + { + auto path = shared_data_paths->getDataAt(i).toString(); + auto value = shared_data_values->getDataAt(i); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + /// We should not have Nulls here but let's check just in case. + if (!isNothing(type)) + data[path].insert(type->getName()); + } + } + + void merge(const AggregateFunctionDistinctJSONPathsAndTypesData & other) + { + for (const auto & [path, types] : other.data) + data[path].insert(types.begin(), types.end()); + } + + void serialize(WriteBuffer & buf) const + { + writeVarUInt(data.size(), buf); + for (const auto & [path, types] : data) + { + writeStringBinary(path, buf); + writeVarUInt(types.size(), buf); + for (const auto & type : types) + writeStringBinary(type, buf); + } + } + + void deserialize(ReadBuffer & buf) + { + size_t paths_size, types_size; + readVarUInt(paths_size, buf); + data.reserve(paths_size); + String path, type; + for (size_t i = 0; i != paths_size; ++i) + { + readStringBinary(path, buf); + readVarUInt(types_size, buf); + data[path].reserve(types_size); + for (size_t j = 0; j != types_size; ++j) + { + readStringBinary(type, buf); + data[path].insert(type); + } + } + } + + void insertResultInto(IColumn & column) + { + /// Insert sorted paths and types for better output. + auto & array_column = assert_cast(column).getNestedColumn(); + auto & tuple_column = assert_cast(array_column.getData()); + auto & key_column = assert_cast(tuple_column.getColumn(0)); + auto & value_column = assert_cast(tuple_column.getColumn(1)); + auto & value_column_data = assert_cast(value_column.getData()); + std::vector>> sorted_data; + sorted_data.reserve(data.size()); + for (const auto & [path, types] : data) + { + std::vector sorted_types(types.begin(), types.end()); + std::sort(sorted_types.begin(), sorted_types.end()); + sorted_data.emplace_back(path, std::move(sorted_types)); + } + std::sort(sorted_data.begin(), sorted_data.end()); + + for (const auto & [path, types] : sorted_data) + { + key_column.insertData(path.data(), path.size()); + for (const auto & type : types) + value_column_data.insertData(type.data(), type.size()); + value_column.getOffsets().push_back(value_column_data.size()); + } + + array_column.getOffsets().push_back(key_column.size()); + } + + static DataTypePtr getResultType() + { + return std::make_shared(std::make_shared(), std::make_shared(std::make_shared())); + } +}; + +/// Calculates the list of distinct data types in Dynamic column. +template +class AggregateFunctionDistinctJSONPathsAndTypes final : public IAggregateFunctionDataHelper> +{ +public: + explicit AggregateFunctionDistinctJSONPathsAndTypes(const DataTypes & argument_types_) + : IAggregateFunctionDataHelper>( + argument_types_, {}, Data::getResultType()) + { + const auto & typed_paths_types = assert_cast(*argument_types_[0]).getTypedPaths(); + typed_paths_type_names.reserve(typed_paths_types.size()); + for (const auto & [path, type] : typed_paths_types) + typed_paths_type_names[path] = type->getName(); + } + + String getName() const override { return Data::name; } + + bool allocatesMemoryInArena() const override { return false; } + + void ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override + { + const auto & object_column = assert_cast(*columns[0]); + this->data(place).add(object_column, row_num, typed_paths_type_names); + } + + void ALWAYS_INLINE addBatchSinglePlace( + size_t row_begin, size_t row_end, AggregateDataPtr __restrict place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos) + const override + { + if (if_argument_pos >= 0 || row_begin != 0 || row_end != columns[0]->size()) + IAggregateFunctionDataHelper>::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos); + /// Optimization for case when we add all rows from the column into single place. + /// In this case we can avoid iterating over all rows because we can get all paths + /// and types in JSON column in a more efficient way. + else + this->data(place).addWholeColumn(assert_cast(*columns[0]), typed_paths_type_names); + } + + void addManyDefaults( + AggregateDataPtr __restrict /*place*/, + const IColumn ** /*columns*/, + size_t /*length*/, + Arena * /*arena*/) const override + { + /// Default value for JSON is empty object, so nothing to add. + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override + { + this->data(place).merge(this->data(rhs)); + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override + { + this->data(place).serialize(buf); + } + + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + { + this->data(place).deserialize(buf); + } + + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override + { + this->data(place).insertResultInto(to); + } + +private: + std::unordered_map typed_paths_type_names; +}; + +template +AggregateFunctionPtr createAggregateFunctionDistinctJSONPathsAndTypes( + const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) +{ + assertNoParameters(name, parameters); + if (argument_types.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Incorrect number of arguments for aggregate function {}. Expected single argument with type JSON, got {} arguments", name, argument_types.size()); + + if (!isObject(argument_types[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}. Expected type JSON", argument_types[0]->getName(), name); + + return std::make_shared>(argument_types); +} + +void registerAggregateFunctionDistinctJSONPathsAndTypes(AggregateFunctionFactory & factory) +{ + factory.registerFunction("distinctJSONPaths", createAggregateFunctionDistinctJSONPathsAndTypes); + factory.registerFunction("distinctJSONPathsAndTypes", createAggregateFunctionDistinctJSONPathsAndTypes); +} + +} diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 4ac25e14ee6..068f1aebd87 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -89,6 +89,8 @@ void registerAggregateFunctionAnalysisOfVariance(AggregateFunctionFactory &); void registerAggregateFunctionFlameGraph(AggregateFunctionFactory &); void registerAggregateFunctionKolmogorovSmirnovTest(AggregateFunctionFactory & factory); void registerAggregateFunctionLargestTriangleThreeBuckets(AggregateFunctionFactory & factory); +void registerAggregateFunctionDistinctDynamicTypes(AggregateFunctionFactory & factory); +void registerAggregateFunctionDistinctJSONPathsAndTypes(AggregateFunctionFactory & factory); class AggregateFunctionCombinatorFactory; void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); @@ -191,6 +193,8 @@ void registerAggregateFunctions() registerAggregateFunctionFlameGraph(factory); registerAggregateFunctionKolmogorovSmirnovTest(factory); registerAggregateFunctionLargestTriangleThreeBuckets(factory); + registerAggregateFunctionDistinctDynamicTypes(factory); + registerAggregateFunctionDistinctJSONPathsAndTypes(factory); registerWindowFunctions(factory); } diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 1f37add9d2d..b5378e983c6 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -980,6 +980,41 @@ ColumnPtr ColumnDynamic::compress() const }); } +String ColumnDynamic::getTypeNameAt(size_t row_num) const +{ + const auto & variant_col = getVariantColumn(); + size_t discr = variant_col.globalDiscriminatorAt(row_num); + if (discr == ColumnVariant::NULL_DISCRIMINATOR) + return ""; + + if (discr == getSharedVariantDiscriminator()) + { + const auto value = getSharedVariant().getDataAt(variant_col.offsetAt(row_num)); + ReadBufferFromMemory buf(value.data, value.size); + return decodeDataType(buf)->getName(); + } + + return variant_info.variant_names[discr]; +} + +void ColumnDynamic::getAllTypeNames(std::unordered_set & names) const +{ + auto shared_variant_discr = getSharedVariantDiscriminator(); + for (size_t i = 0; i != variant_info.variant_names.size(); ++i) + { + if (i != shared_variant_discr && !variant_column_ptr->getVariantByGlobalDiscriminator(i).empty()) + names.insert(variant_info.variant_names[i]); + } + + const auto & shared_variant = getSharedVariant(); + for (size_t i = 0; i != shared_variant.size(); ++i) + { + const auto value = shared_variant.getDataAt(i); + ReadBufferFromMemory buf(value.data, value.size); + names.insert(decodeDataType(buf)->getName()); + } +} + void ColumnDynamic::prepareForSquashing(const Columns & source_columns) { if (source_columns.empty()) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 2ae862de3af..c06c31bb8c9 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -430,6 +430,9 @@ public: const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type) const { return getVariantSerialization(variant_type, variant_type->getName()); } + String getTypeNameAt(size_t row_num) const; + void getAllTypeNames(std::unordered_set & names) const; + private: void createVariantInfo(const DataTypePtr & variant_type); diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference new file mode 100644 index 00000000000..d6538a1c4eb --- /dev/null +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference @@ -0,0 +1,121 @@ +a0 +a1 +a10 +a11 +a12 +a2 +a3 +a4 +a5 +a6 +a7 +a8 +a9 +('a0',['Array(Nullable(Int64))','Date','Int64','String']) +('a1',['String']) +('a10',['Array(Nullable(Int64))','Date','Int64','String']) +('a11',['Array(Nullable(Int64))','Date','Int64','String']) +('a12',['Array(Nullable(Int64))','Date','Int64','String']) +('a2',['Array(Nullable(Int64))','Date','Int64','String']) +('a3',['Array(Nullable(Int64))','Date','Int64','String']) +('a4',['Array(Nullable(Int64))','Date','Int64','String']) +('a5',['Array(Nullable(Int64))','Date','Int64','String']) +('a6',['Array(Nullable(Int64))','Date','Int64','String']) +('a7',['Array(Nullable(Int64))','Date','Int64','String']) +('a8',['Array(Nullable(Int64))','Date','Int64','String']) +('a9',['Array(Nullable(Int64))','Date','Int64','String']) +Array(Nullable(Int64)) +Date +Int64 +String +Array(Nullable(Int64)) +Date +Int64 +String +Filter +a1 +a2 +('a1',['String']) +('a2',['String']) +String +If +a1 +a2 +('a1',['String']) +('a2',['String']) +String +Group by +Array(Nullable(Int64)) ['a1','a2'] +Date ['a1','a2'] +Int64 ['a1','a2'] +None ['a0','a1','a10','a11','a12','a3','a4','a5','a6','a7','a8','a9'] +String ['a1','a2'] +Array(Nullable(Int64)) {'a1':['String'],'a2':['Array(Nullable(Int64))']} +Date {'a1':['String'],'a2':['Date']} +Int64 {'a1':['String'],'a2':['Int64']} +None {'a0':['Array(Nullable(Int64))','Date','Int64','String'],'a1':['String'],'a10':['Array(Nullable(Int64))','Date','Int64','String'],'a11':['Array(Nullable(Int64))','Date','Int64','String'],'a12':['Array(Nullable(Int64))','Date','Int64','String'],'a3':['Array(Nullable(Int64))','Date','Int64','String'],'a4':['Array(Nullable(Int64))','Date','Int64','String'],'a5':['Array(Nullable(Int64))','Date','Int64','String'],'a6':['Array(Nullable(Int64))','Date','Int64','String'],'a7':['Array(Nullable(Int64))','Date','Int64','String'],'a8':['Array(Nullable(Int64))','Date','Int64','String'],'a9':['Array(Nullable(Int64))','Date','Int64','String']} +String {'a1':['String'],'a2':['String']} +Array(Nullable(Int64)) ['Array(Nullable(Int64))'] +Date ['Date'] +Int64 ['Int64'] +None [] +String ['String'] +Remote +a0 +a1 +a10 +a11 +a12 +a2 +a3 +a4 +a5 +a6 +a7 +a8 +a9 +('a0',['Array(Nullable(Int64))','Date','Int64','String']) +('a1',['String']) +('a10',['Array(Nullable(Int64))','Date','Int64','String']) +('a11',['Array(Nullable(Int64))','Date','Int64','String']) +('a12',['Array(Nullable(Int64))','Date','Int64','String']) +('a2',['Array(Nullable(Int64))','Date','Int64','String']) +('a3',['Array(Nullable(Int64))','Date','Int64','String']) +('a4',['Array(Nullable(Int64))','Date','Int64','String']) +('a5',['Array(Nullable(Int64))','Date','Int64','String']) +('a6',['Array(Nullable(Int64))','Date','Int64','String']) +('a7',['Array(Nullable(Int64))','Date','Int64','String']) +('a8',['Array(Nullable(Int64))','Date','Int64','String']) +('a9',['Array(Nullable(Int64))','Date','Int64','String']) +Array(Nullable(Int64)) +Date +Int64 +String +Remote filter +a1 +a2 +('a1',['String']) +('a2',['String']) +String +Remote if +a1 +a2 +('a1',['String']) +('a2',['String']) +String +Remote group by +Array(Nullable(Int64)) ['a1','a2'] +Date ['a1','a2'] +Int64 ['a1','a2'] +None ['a0','a1','a10','a11','a12','a3','a4','a5','a6','a7','a8','a9'] +String ['a1','a2'] +Array(Nullable(Int64)) {'a1':['String'],'a2':['Array(Nullable(Int64))']} +Date {'a1':['String'],'a2':['Date']} +Int64 {'a1':['String'],'a2':['Int64']} +None {'a0':['Array(Nullable(Int64))','Date','Int64','String'],'a1':['String'],'a10':['Array(Nullable(Int64))','Date','Int64','String'],'a11':['Array(Nullable(Int64))','Date','Int64','String'],'a12':['Array(Nullable(Int64))','Date','Int64','String'],'a3':['Array(Nullable(Int64))','Date','Int64','String'],'a4':['Array(Nullable(Int64))','Date','Int64','String'],'a5':['Array(Nullable(Int64))','Date','Int64','String'],'a6':['Array(Nullable(Int64))','Date','Int64','String'],'a7':['Array(Nullable(Int64))','Date','Int64','String'],'a8':['Array(Nullable(Int64))','Date','Int64','String'],'a9':['Array(Nullable(Int64))','Date','Int64','String']} +String {'a1':['String'],'a2':['String']} +Array(Nullable(Int64)) ['Array(Nullable(Int64))'] +Date ['Date'] +Int64 ['Int64'] +None [] +String ['String'] diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql new file mode 100644 index 00000000000..4f33cc10d46 --- /dev/null +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql @@ -0,0 +1,50 @@ +set allow_experimental_dynamic_type = 1; +set allow_experimental_json_type = 1; +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; + +drop table if exists test_json_dynamic_aggregate_functions; +create table test_json_dynamic_aggregate_functions (json JSON(a1 String, max_dynamic_paths=2, max_dynamic_types=2)) engine=Memory; +insert into test_json_dynamic_aggregate_functions select toJSONString(map('a' || number % 13, multiIf(number % 5 == 0, NULL, number % 5 == 1, number::UInt32, number % 5 == 2, 'str_' || number, number % 5 == 3, range(number % 5), toDate(number)))) from numbers(200000); +select arrayJoin(distinctJSONPaths(json)) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctJSONPathsAndTypes(json)) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctDynamicTypes(json.a2)) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctDynamicTypes(json.a3)) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctDynamicTypes(json.a42)) from test_json_dynamic_aggregate_functions; + +select 'Filter'; +select arrayJoin(distinctJSONPaths(json)) from test_json_dynamic_aggregate_functions where dynamicType(json.a2) == 'String'; +select arrayJoin(distinctJSONPathsAndTypes(json)) from test_json_dynamic_aggregate_functions where dynamicType(json.a2) == 'String'; +select arrayJoin(distinctDynamicTypes(json.a2)) from test_json_dynamic_aggregate_functions where dynamicType(json.a2) == 'String'; + +select 'If'; +select arrayJoin(distinctJSONPathsIf(json, dynamicType(json.a2) == 'String')) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctJSONPathsAndTypesIf(json, dynamicType(json.a2) == 'String')) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctDynamicTypesIf(json.a2, dynamicType(json.a2) == 'String')) from test_json_dynamic_aggregate_functions; + +select 'Group by'; +select dynamicType(json.a2), distinctJSONPaths(json) from test_json_dynamic_aggregate_functions group by dynamicType(json.a2) order by dynamicType(json.a2); +select dynamicType(json.a2), distinctJSONPathsAndTypes(json) from test_json_dynamic_aggregate_functions group by dynamicType(json.a2) order by dynamicType(json.a2); +select dynamicType(json.a2), distinctDynamicTypes(json.a2) from test_json_dynamic_aggregate_functions group by dynamicType(json.a2) order by dynamicType(json.a2); + +select 'Remote'; +select arrayJoin(distinctJSONPaths(json)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); +select arrayJoin(distinctJSONPathsAndTypes(json)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); +select arrayJoin(distinctDynamicTypes(json.a2)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); + +select 'Remote filter'; +select arrayJoin(distinctJSONPaths(json)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) where dynamicType(json.a2) == 'String'; +select arrayJoin(distinctJSONPathsAndTypes(json)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) where dynamicType(json.a2) == 'String'; +select arrayJoin(distinctDynamicTypes(json.a2)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) where dynamicType(json.a2) == 'String'; + +select 'Remote if'; +select arrayJoin(distinctJSONPathsIf(json, dynamicType(json.a2) == 'String')) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); +select arrayJoin(distinctJSONPathsAndTypesIf(json, dynamicType(json.a2) == 'String')) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); +select arrayJoin(distinctDynamicTypesIf(json.a2, dynamicType(json.a2) == 'String')) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); + +select 'Remote group by'; +select dynamicType(json.a2), distinctJSONPaths(json) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); +select dynamicType(json.a2), distinctJSONPathsAndTypes(json) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); +select dynamicType(json.a2), distinctDynamicTypes(json.a2) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); + +drop table test_json_dynamic_aggregate_functions; From 85a813bbedeec62fdf4704041a62092bbf6a9789 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 17:55:57 +0000 Subject: [PATCH 049/259] Better example --- docs/en/sql-reference/data-types/newjson.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index 16cc324a168..99156b11a0b 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -571,7 +571,7 @@ SELECT arrayJoin(distinctJSONPaths(json)) FROM s3('s3://clickhouse-public-datase ``` ```sql -SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-public-datasets/gharchive/original/2020-01-01-*.json.gz', JSONAsObject) +SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-public-datasets/gharchive/original/2020-01-01-*.json.gz', JSONAsObject) SETTINGS date_time_input_format='best_effort' ``` @@ -583,7 +583,7 @@ SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-publi │ ('actor.id',['Int64']) │ │ ('actor.login',['String']) │ │ ('actor.url',['String']) │ -│ ('created_at',['String']) │ +│ ('created_at',['DateTime']) │ │ ('id',['String']) │ │ ('org.avatar_url',['String']) │ │ ('org.gravatar_id',['String']) │ @@ -598,7 +598,7 @@ SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-publi │ ('payload.comment.author_association',['String']) │ │ ('payload.comment.body',['String']) │ │ ('payload.comment.commit_id',['String']) │ -│ ('payload.comment.created_at',['String']) │ +│ ('payload.comment.created_at',['DateTime']) │ │ ('payload.comment.diff_hunk',['String']) │ │ ('payload.comment.html_url',['String']) │ │ ('payload.comment.id',['Int64']) │ @@ -614,7 +614,7 @@ SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-publi ... │ ('payload.release.node_id',['String']) │ │ ('payload.release.prerelease',['Bool']) │ -│ ('payload.release.published_at',['String']) │ +│ ('payload.release.published_at',['DateTime']) │ │ ('payload.release.tag_name',['String']) │ │ ('payload.release.tarball_url',['String']) │ │ ('payload.release.target_commitish',['String']) │ From f21e982f03488634cd09ec7ab570fec07d78a0e1 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 18:01:35 +0000 Subject: [PATCH 050/259] Fix style, update tests --- .../AggregateFunctionDistinctJSONPaths.cpp | 18 ++++++++++++------ ...03227_distinct_dynamic_types_json_paths.sql | 10 ++++++++++ .../aspell-ignore/en/aspell-dict.txt | 5 +++++ 3 files changed, 27 insertions(+), 6 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp index a3ede0750f2..58ce7e27ab0 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -19,6 +19,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + struct AggregateFunctionDistinctJSONPathsData { static constexpr auto name = "distinctJSONPaths"; @@ -44,7 +50,7 @@ struct AggregateFunctionDistinctJSONPathsData for (size_t i = start; i != end; ++i) data.insert(shared_data_paths->getDataAt(i).toString()); } - + void addWholeColumn(const ColumnObject & column, const std::unordered_map &) { for (const auto & [path, _] : column.getTypedPaths()) @@ -98,7 +104,7 @@ struct AggregateFunctionDistinctJSONPathsData string_column.insertData(path.data(), path.size()); array_column.getOffsets().push_back(string_column.size()); } - + static DataTypePtr getResultType() { return std::make_shared(std::make_shared()); @@ -110,7 +116,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData static constexpr auto name = "distinctJSONPathsAndTypes"; std::unordered_map> data; - + void add(const ColumnObject & column, size_t row_num, const std::unordered_map & typed_paths_type_names) { for (const auto & [path, _] : column.getTypedPaths()) @@ -137,7 +143,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData data[path].insert(type->getName()); } } - + void addWholeColumn(const ColumnObject & column, const std::unordered_map & typed_paths_type_names) { for (const auto & [path, _] : column.getTypedPaths()) @@ -229,7 +235,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData array_column.getOffsets().push_back(key_column.size()); } - + static DataTypePtr getResultType() { return std::make_shared(std::make_shared(), std::make_shared(std::make_shared())); @@ -302,7 +308,7 @@ public: { this->data(place).insertResultInto(to); } - + private: std::unordered_map typed_paths_type_names; }; diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql index 4f33cc10d46..f7707b536e6 100644 --- a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql @@ -47,4 +47,14 @@ select dynamicType(json.a2), distinctJSONPaths(json) from remote('127.0.0.{1,2,3 select dynamicType(json.a2), distinctJSONPathsAndTypes(json) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); select dynamicType(json.a2), distinctDynamicTypes(json.a2) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); +select distinctJSONPaths() from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctJSONPaths(json, 42) from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctJSONPaths(42) from test_json_dynamic_aggregate_functions; -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select distinctJSONPathsAndTypes() from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctJSONPathsAndTypes(json, 42) from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctJSONPathsAndTypes(42) from test_json_dynamic_aggregate_functions; -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select distinctDynamicTypes() from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctDynamicTypes(json.a2, 42) from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctDynamicTypes(42) from test_json_dynamic_aggregate_functions; -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} + drop table test_json_dynamic_aggregate_functions; diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index fd836d93143..a5dd1ac5f9d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1551,6 +1551,11 @@ disjunctions displayName displaySecretsInShowAndSelect distro +distinctdynamictypes +distinctDynamicTypes +distinctjsonpaths +distinctJSONPaths +distinctJSONPathsAndTypes divideDecimal dmesg doesnt From 76960eff8005818db7f744115798888e72e4b2e5 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 16 Aug 2024 18:38:03 +0000 Subject: [PATCH 051/259] This should work --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index 73752937366..b0fce6066fc 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit 737529373665bc067971ba098a12d6928580a0ae +Subproject commit b0fce6066fc2678fa17ee7a98f794da9da8492ff From b29d5242be83294af8429d30fd560a83de3c08b4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 22:10:11 +0000 Subject: [PATCH 052/259] Fix tidy build --- .../AggregateFunctionDistinctDynamicTypes.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp index 6c899d6cda8..649d64a3904 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp @@ -87,7 +87,7 @@ public: if (dynamic_column.isNullAt(row_num)) return; - this->data(place).add(dynamic_column.getTypeNameAt(row_num)); + data(place).add(dynamic_column.getTypeNameAt(row_num)); } void ALWAYS_INLINE addBatchSinglePlace( @@ -100,7 +100,7 @@ public: /// In this case we can avoid iterating over all rows because we can get all types /// in Dynamic column in a more efficient way. else - assert_cast(*columns[0]).getAllTypeNames(this->data(place).data); + assert_cast(*columns[0]).getAllTypeNames(data(place).data); } void addManyDefaults( @@ -114,22 +114,22 @@ public: void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override { - this->data(place).merge(this->data(rhs)); + data(place).merge(data(rhs)); } void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { - this->data(place).serialize(buf); + data(place).serialize(buf); } void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { - this->data(place).deserialize(buf); + data(place).deserialize(buf); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { - this->data(place).insertResultInto(to); + data(place).insertResultInto(to); } }; From 41f6e06cbe07f6bd1ce3bdb48154f1f2745b3cc2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sat, 17 Aug 2024 13:08:30 +0200 Subject: [PATCH 053/259] Update 03227_distinct_dynamic_types_json_paths.sql --- .../0_stateless/03227_distinct_dynamic_types_json_paths.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql index f7707b536e6..c9c86277bce 100644 --- a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql @@ -1,3 +1,5 @@ +-- Tags: long + set allow_experimental_dynamic_type = 1; set allow_experimental_json_type = 1; set allow_experimental_variant_type = 1; From 9d183b890529d686a673d5f6ecd2d057cf534478 Mon Sep 17 00:00:00 2001 From: avogar Date: Sun, 18 Aug 2024 20:35:30 +0000 Subject: [PATCH 054/259] Add check for loo large array size --- .../AggregateFunctionDistinctDynamicTypes.cpp | 6 ++++++ .../AggregateFunctionDistinctJSONPaths.cpp | 13 +++++++++++++ src/Core/Settings.h | 1 - 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp index 649d64a3904..17e32b20a99 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp @@ -19,10 +19,13 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int TOO_LARGE_ARRAY_SIZE; } struct AggregateFunctionDistinctDynamicTypesData { + constexpr static size_t MAX_ARRAY_SIZE = 0xFFFFFF; + std::unordered_set data; void add(const String & type) @@ -46,6 +49,9 @@ struct AggregateFunctionDistinctDynamicTypesData { size_t size; readVarUInt(size, buf); + if (size > MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", MAX_ARRAY_SIZE, size); + data.reserve(size); String type; for (size_t i = 0; i != size; ++i) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp index 58ce7e27ab0..6100bd57515 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -23,8 +23,12 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int TOO_LARGE_ARRAY_SIZE; } +constexpr static size_t DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE = 0xFFFFFF; + + struct AggregateFunctionDistinctJSONPathsData { static constexpr auto name = "distinctJSONPaths"; @@ -85,6 +89,9 @@ struct AggregateFunctionDistinctJSONPathsData { size_t size; readVarUInt(size, buf); + if (size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, size); + String path; for (size_t i = 0; i != size; ++i) { @@ -192,12 +199,18 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData { size_t paths_size, types_size; readVarUInt(paths_size, buf); + if (paths_size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, paths_size); + data.reserve(paths_size); String path, type; for (size_t i = 0; i != paths_size; ++i) { readStringBinary(path, buf); readVarUInt(types_size, buf); + if (types_size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, types_size); + data[path].reserve(types_size); for (size_t j = 0; j != types_size; ++j) { diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0d84ad9022a..dfcff052740 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -616,7 +616,6 @@ class IColumn; M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ - M(Bool, allow_materialized_view_with_bad_select, true, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ From 228cdfde63100267ce1b8c80ddfeb689979487ba Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 15 Aug 2024 08:58:36 +0000 Subject: [PATCH 055/259] more tests --- .../test_filesystem_layout/test.py | 4 ++++ .../test_with_table_engine_grant.py | 2 ++ .../test_parallel_replicas_failover/test.py | 19 ++++++++++--------- .../test.py | 4 ++-- 4 files changed, 18 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_filesystem_layout/test.py b/tests/integration/test_filesystem_layout/test.py index 4e719aa0fe9..31d6c830a2f 100644 --- a/tests/integration/test_filesystem_layout/test.py +++ b/tests/integration/test_filesystem_layout/test.py @@ -79,3 +79,7 @@ def test_file_path_escaping(started_cluster): "test -f /var/lib/clickhouse/shadow/2/store/123/12345678-1000-4000-8000-000000000001/1_1_1_0/%7EId.bin", ] ) + node.query("DROP TABLE test.`T.a_b,l-e!` SYNC") + node.query("DROP TABLE `test 2`.`T.a_b,l-e!` SYNC") + node.query("DROP DATABASE test") + node.query("DROP DATABASE `test 2`") diff --git a/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py index 25ca7913e4e..5fc8f67b75b 100644 --- a/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py +++ b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py @@ -359,6 +359,8 @@ def test_implicit_create_view_grant(): instance.query("GRANT CREATE VIEW ON test.* TO B", user="A") instance.query("CREATE VIEW test.view_2 AS SELECT 1", user="B") assert instance.query("SELECT * FROM test.view_2") == "1\n" + instance.query("DROP USER A") + instance.query("DROP VIEW test.view_2") def test_implicit_create_temporary_table_grant(): diff --git a/tests/integration/test_parallel_replicas_failover/test.py b/tests/integration/test_parallel_replicas_failover/test.py index bf25136bff7..2da26ee03c9 100644 --- a/tests/integration/test_parallel_replicas_failover/test.py +++ b/tests/integration/test_parallel_replicas_failover/test.py @@ -1,5 +1,5 @@ import pytest - +import uuid from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -25,19 +25,15 @@ def start_cluster(): def create_tables(cluster, table_name, skip_last_replica): - node1.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node2.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node3.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node1.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" ) node2.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') ORDER BY (key)" ) if not skip_last_replica: node3.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" ) # populate data @@ -67,7 +63,7 @@ def test_skip_replicas_without_table(start_cluster): for i in range(4): expected_result += f"{i}\t1000\n" - log_comment = "5230b069-9574-407d-9b80-891b5a175f41" + log_comment = uuid.uuid4() assert ( node1.query( f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", @@ -88,6 +84,8 @@ def test_skip_replicas_without_table(start_cluster): ) == "1\t1\n" ) + node1.query(f"DROP TABLE {table_name} SYNC") + node2.query(f"DROP TABLE {table_name} SYNC") def test_skip_unresponsive_replicas(start_cluster): @@ -112,3 +110,6 @@ def test_skip_unresponsive_replicas(start_cluster): ) == expected_result ) + node1.query(f"DROP TABLE {table_name} SYNC") + node2.query(f"DROP TABLE {table_name} SYNC") + node3.query(f"DROP TABLE {table_name} SYNC") diff --git a/tests/integration/test_parallel_replicas_invisible_parts/test.py b/tests/integration/test_parallel_replicas_invisible_parts/test.py index cab3fb46fe9..7093e3b3292 100644 --- a/tests/integration/test_parallel_replicas_invisible_parts/test.py +++ b/tests/integration/test_parallel_replicas_invisible_parts/test.py @@ -35,11 +35,10 @@ def start_cluster(): def _create_tables(table_name, table_size, index_granularity): - nodes[0].query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER {cluster_name}") nodes[0].query( f""" - CREATE TABLE IF NOT EXISTS {table_name} ON CLUSTER '{cluster_name}' (key Int64, value String) + CREATE TABLE {table_name} ON CLUSTER '{cluster_name}' (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard/{table_name}/', '{{replica}}') ORDER BY (key) SETTINGS index_granularity = {index_granularity}, max_bytes_to_merge_at_max_space_in_pool = 0, max_bytes_to_merge_at_max_space_in_pool = 1 @@ -128,3 +127,4 @@ def test_reading_with_invisible_parts( ) == f"{expected}\n" ) + nodes[0].query(f"DROP TABLE {table_name} ON CLUSTER {cluster_name} SYNC") From 90ad110a7dddbd1c9430b34b0016dc22eaa3d646 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 19 Aug 2024 13:58:44 +0000 Subject: [PATCH 056/259] more --- .../test.py | 28 +++++++++++-------- 1 file changed, 17 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 5e04c9e4d12..fb7aab366be 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -1,5 +1,6 @@ import pytest +import uuid import time import psycopg2 import os.path as p @@ -59,8 +60,6 @@ instance2 = cluster.add_instance( pg_manager = PostgresManager() pg_manager2 = PostgresManager() pg_manager_instance2 = PostgresManager() -pg_manager3 = PostgresManager() - @pytest.fixture(scope="module") def started_cluster(): @@ -82,12 +81,6 @@ def started_cluster(): pg_manager2.init( instance2, cluster.postgres_ip, cluster.postgres_port, "postgres_database2" ) - pg_manager3.init( - instance, - cluster.postgres_ip, - cluster.postgres_port, - default_database="postgres-postgres", - ) yield cluster @@ -924,16 +917,27 @@ def test_failed_load_from_snapshot(started_cluster): def test_symbols_in_publication_name(started_cluster): - table = "test_symbols_in_publication_name" + id = uuid.uuid4() + db = f'test_{id}' + table = f"test_symbols_in_publication_name" + + pg_manager3 = PostgresManager() + pg_manager3.init( + instance, + cluster.postgres_ip, + cluster.postgres_port, + default_database=db, + ) pg_manager3.create_postgres_table(table) instance.query( - f"INSERT INTO `{pg_manager3.get_default_database()}`.`{table}` SELECT number, number from numbers(0, 50)" + f"INSERT INTO `{db}`.`{table}` SELECT number, number from numbers(0, 50)" ) pg_manager3.create_materialized_db( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, + materialized_database=db, settings=[ f"materialized_postgresql_tables_list = '{table}'", "materialized_postgresql_backoff_min_ms = 100", @@ -941,8 +945,10 @@ def test_symbols_in_publication_name(started_cluster): ], ) check_tables_are_synchronized( - instance, table, postgres_database=pg_manager3.get_default_database() + instance, table, materialized_database=db, postgres_database=db ) + pg_manager3.drop_materialized_db(db) + pg_manager3.execute(f'drop table "{table}"') def test_generated_columns(started_cluster): From 0793585acde8f469bf534f0639850a91efc254f1 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 19 Aug 2024 16:59:30 +0200 Subject: [PATCH 057/259] Fix bad conflict resolution --- src/Core/Settings.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index dfcff052740..215d333f6c4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -616,6 +616,7 @@ class IColumn; M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ + M(Bool, allow_materialized_view_with_bad_select, true, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ From 64bce7afa115f13ff72e2b4f0f6e1b4eacb881e7 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 19 Aug 2024 17:22:17 +0200 Subject: [PATCH 058/259] Fix spaces --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 215d333f6c4..0d84ad9022a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -616,7 +616,7 @@ class IColumn; M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ - M(Bool, allow_materialized_view_with_bad_select, true, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ + M(Bool, allow_materialized_view_with_bad_select, true, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ From a2b08e93e43c357f8adbdd604a54ccb75056e4ec Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 19 Aug 2024 20:16:50 +0000 Subject: [PATCH 059/259] Change test --- ...istinct_dynamic_types_json_paths.reference | 70 +++++++++---------- ...3227_distinct_dynamic_types_json_paths.sql | 2 +- 2 files changed, 36 insertions(+), 36 deletions(-) diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference index d6538a1c4eb..891982fdc81 100644 --- a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference @@ -11,25 +11,25 @@ a6 a7 a8 a9 -('a0',['Array(Nullable(Int64))','Date','Int64','String']) +('a0',['Array(Nullable(Int64))','Bool','Int64','String']) ('a1',['String']) -('a10',['Array(Nullable(Int64))','Date','Int64','String']) -('a11',['Array(Nullable(Int64))','Date','Int64','String']) -('a12',['Array(Nullable(Int64))','Date','Int64','String']) -('a2',['Array(Nullable(Int64))','Date','Int64','String']) -('a3',['Array(Nullable(Int64))','Date','Int64','String']) -('a4',['Array(Nullable(Int64))','Date','Int64','String']) -('a5',['Array(Nullable(Int64))','Date','Int64','String']) -('a6',['Array(Nullable(Int64))','Date','Int64','String']) -('a7',['Array(Nullable(Int64))','Date','Int64','String']) -('a8',['Array(Nullable(Int64))','Date','Int64','String']) -('a9',['Array(Nullable(Int64))','Date','Int64','String']) +('a10',['Array(Nullable(Int64))','Bool','Int64','String']) +('a11',['Array(Nullable(Int64))','Bool','Int64','String']) +('a12',['Array(Nullable(Int64))','Bool','Int64','String']) +('a2',['Array(Nullable(Int64))','Bool','Int64','String']) +('a3',['Array(Nullable(Int64))','Bool','Int64','String']) +('a4',['Array(Nullable(Int64))','Bool','Int64','String']) +('a5',['Array(Nullable(Int64))','Bool','Int64','String']) +('a6',['Array(Nullable(Int64))','Bool','Int64','String']) +('a7',['Array(Nullable(Int64))','Bool','Int64','String']) +('a8',['Array(Nullable(Int64))','Bool','Int64','String']) +('a9',['Array(Nullable(Int64))','Bool','Int64','String']) Array(Nullable(Int64)) -Date +Bool Int64 String Array(Nullable(Int64)) -Date +Bool Int64 String Filter @@ -46,17 +46,17 @@ a2 String Group by Array(Nullable(Int64)) ['a1','a2'] -Date ['a1','a2'] +Bool ['a1','a2'] Int64 ['a1','a2'] None ['a0','a1','a10','a11','a12','a3','a4','a5','a6','a7','a8','a9'] String ['a1','a2'] Array(Nullable(Int64)) {'a1':['String'],'a2':['Array(Nullable(Int64))']} -Date {'a1':['String'],'a2':['Date']} +Bool {'a1':['String'],'a2':['Bool']} Int64 {'a1':['String'],'a2':['Int64']} -None {'a0':['Array(Nullable(Int64))','Date','Int64','String'],'a1':['String'],'a10':['Array(Nullable(Int64))','Date','Int64','String'],'a11':['Array(Nullable(Int64))','Date','Int64','String'],'a12':['Array(Nullable(Int64))','Date','Int64','String'],'a3':['Array(Nullable(Int64))','Date','Int64','String'],'a4':['Array(Nullable(Int64))','Date','Int64','String'],'a5':['Array(Nullable(Int64))','Date','Int64','String'],'a6':['Array(Nullable(Int64))','Date','Int64','String'],'a7':['Array(Nullable(Int64))','Date','Int64','String'],'a8':['Array(Nullable(Int64))','Date','Int64','String'],'a9':['Array(Nullable(Int64))','Date','Int64','String']} +None {'a0':['Array(Nullable(Int64))','Bool','Int64','String'],'a1':['String'],'a10':['Array(Nullable(Int64))','Bool','Int64','String'],'a11':['Array(Nullable(Int64))','Bool','Int64','String'],'a12':['Array(Nullable(Int64))','Bool','Int64','String'],'a3':['Array(Nullable(Int64))','Bool','Int64','String'],'a4':['Array(Nullable(Int64))','Bool','Int64','String'],'a5':['Array(Nullable(Int64))','Bool','Int64','String'],'a6':['Array(Nullable(Int64))','Bool','Int64','String'],'a7':['Array(Nullable(Int64))','Bool','Int64','String'],'a8':['Array(Nullable(Int64))','Bool','Int64','String'],'a9':['Array(Nullable(Int64))','Bool','Int64','String']} String {'a1':['String'],'a2':['String']} Array(Nullable(Int64)) ['Array(Nullable(Int64))'] -Date ['Date'] +Bool ['Bool'] Int64 ['Int64'] None [] String ['String'] @@ -74,21 +74,21 @@ a6 a7 a8 a9 -('a0',['Array(Nullable(Int64))','Date','Int64','String']) +('a0',['Array(Nullable(Int64))','Bool','Int64','String']) ('a1',['String']) -('a10',['Array(Nullable(Int64))','Date','Int64','String']) -('a11',['Array(Nullable(Int64))','Date','Int64','String']) -('a12',['Array(Nullable(Int64))','Date','Int64','String']) -('a2',['Array(Nullable(Int64))','Date','Int64','String']) -('a3',['Array(Nullable(Int64))','Date','Int64','String']) -('a4',['Array(Nullable(Int64))','Date','Int64','String']) -('a5',['Array(Nullable(Int64))','Date','Int64','String']) -('a6',['Array(Nullable(Int64))','Date','Int64','String']) -('a7',['Array(Nullable(Int64))','Date','Int64','String']) -('a8',['Array(Nullable(Int64))','Date','Int64','String']) -('a9',['Array(Nullable(Int64))','Date','Int64','String']) +('a10',['Array(Nullable(Int64))','Bool','Int64','String']) +('a11',['Array(Nullable(Int64))','Bool','Int64','String']) +('a12',['Array(Nullable(Int64))','Bool','Int64','String']) +('a2',['Array(Nullable(Int64))','Bool','Int64','String']) +('a3',['Array(Nullable(Int64))','Bool','Int64','String']) +('a4',['Array(Nullable(Int64))','Bool','Int64','String']) +('a5',['Array(Nullable(Int64))','Bool','Int64','String']) +('a6',['Array(Nullable(Int64))','Bool','Int64','String']) +('a7',['Array(Nullable(Int64))','Bool','Int64','String']) +('a8',['Array(Nullable(Int64))','Bool','Int64','String']) +('a9',['Array(Nullable(Int64))','Bool','Int64','String']) Array(Nullable(Int64)) -Date +Bool Int64 String Remote filter @@ -105,17 +105,17 @@ a2 String Remote group by Array(Nullable(Int64)) ['a1','a2'] -Date ['a1','a2'] +Bool ['a1','a2'] Int64 ['a1','a2'] None ['a0','a1','a10','a11','a12','a3','a4','a5','a6','a7','a8','a9'] String ['a1','a2'] Array(Nullable(Int64)) {'a1':['String'],'a2':['Array(Nullable(Int64))']} -Date {'a1':['String'],'a2':['Date']} +Bool {'a1':['String'],'a2':['Bool']} Int64 {'a1':['String'],'a2':['Int64']} -None {'a0':['Array(Nullable(Int64))','Date','Int64','String'],'a1':['String'],'a10':['Array(Nullable(Int64))','Date','Int64','String'],'a11':['Array(Nullable(Int64))','Date','Int64','String'],'a12':['Array(Nullable(Int64))','Date','Int64','String'],'a3':['Array(Nullable(Int64))','Date','Int64','String'],'a4':['Array(Nullable(Int64))','Date','Int64','String'],'a5':['Array(Nullable(Int64))','Date','Int64','String'],'a6':['Array(Nullable(Int64))','Date','Int64','String'],'a7':['Array(Nullable(Int64))','Date','Int64','String'],'a8':['Array(Nullable(Int64))','Date','Int64','String'],'a9':['Array(Nullable(Int64))','Date','Int64','String']} +None {'a0':['Array(Nullable(Int64))','Bool','Int64','String'],'a1':['String'],'a10':['Array(Nullable(Int64))','Bool','Int64','String'],'a11':['Array(Nullable(Int64))','Bool','Int64','String'],'a12':['Array(Nullable(Int64))','Bool','Int64','String'],'a3':['Array(Nullable(Int64))','Bool','Int64','String'],'a4':['Array(Nullable(Int64))','Bool','Int64','String'],'a5':['Array(Nullable(Int64))','Bool','Int64','String'],'a6':['Array(Nullable(Int64))','Bool','Int64','String'],'a7':['Array(Nullable(Int64))','Bool','Int64','String'],'a8':['Array(Nullable(Int64))','Bool','Int64','String'],'a9':['Array(Nullable(Int64))','Bool','Int64','String']} String {'a1':['String'],'a2':['String']} Array(Nullable(Int64)) ['Array(Nullable(Int64))'] -Date ['Date'] +Bool ['Bool'] Int64 ['Int64'] None [] String ['String'] diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql index c9c86277bce..6930f5a3d44 100644 --- a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql @@ -7,7 +7,7 @@ set use_variant_as_common_type = 1; drop table if exists test_json_dynamic_aggregate_functions; create table test_json_dynamic_aggregate_functions (json JSON(a1 String, max_dynamic_paths=2, max_dynamic_types=2)) engine=Memory; -insert into test_json_dynamic_aggregate_functions select toJSONString(map('a' || number % 13, multiIf(number % 5 == 0, NULL, number % 5 == 1, number::UInt32, number % 5 == 2, 'str_' || number, number % 5 == 3, range(number % 5), toDate(number)))) from numbers(200000); +insert into test_json_dynamic_aggregate_functions select toJSONString(map('a' || number % 13, multiIf(number % 5 == 0, NULL, number % 5 == 1, number::UInt32, number % 5 == 2, 'str_' || number, number % 5 == 3, range(number % 5), toBool(number % 2)))) from numbers(200000); select arrayJoin(distinctJSONPaths(json)) from test_json_dynamic_aggregate_functions; select arrayJoin(distinctJSONPathsAndTypes(json)) from test_json_dynamic_aggregate_functions; select arrayJoin(distinctDynamicTypes(json.a2)) from test_json_dynamic_aggregate_functions; From 6fb5051237448addf82faf50677179c551958f1b Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 21 Aug 2024 12:47:12 +0800 Subject: [PATCH 060/259] 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 061/259] 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 062/259] 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 b4a0a9b00cafc86399c62c244b8ada6e0f8d2db4 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 22 Aug 2024 16:58:19 +0000 Subject: [PATCH 063/259] rework test_async_insert_adaptive_busy_timeout --- .../test.py | 55 ++++++++++--------- .../test_with_table_engine_grant.py | 1 + 2 files changed, 31 insertions(+), 25 deletions(-) diff --git a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py index 0ea076b1468..5599786026f 100644 --- a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py +++ b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py @@ -104,7 +104,7 @@ def test_with_merge_tree(): _insert_queries_sequentially( table_name, _query_settings, - iterations=100, + iterations=10, max_values_size=1000, array_size_range=[10, 50], ) @@ -125,7 +125,7 @@ def test_with_merge_tree_multithread(): table_name, _query_settings, thread_num=15, - tasks=1000, + tasks=100, max_values_size=1000, array_size_range=[10, 15], ) @@ -152,7 +152,7 @@ def test_with_replicated_merge_tree(): _insert_queries_sequentially( table_name, settings, - iterations=100, + iterations=10, max_values_size=1000, array_size_range=[10, 50], ) @@ -180,7 +180,7 @@ def test_with_replicated_merge_tree_multithread(): table_name, _query_settings, thread_num=15, - tasks=1000, + tasks=100, max_values_size=1000, array_size_range=[10, 15], ) @@ -200,13 +200,13 @@ def test_compare_sequential_inserts_durations_for_adaptive_and_fixed_async_timeo fixed_tm_settings = copy.copy(_query_settings) fixed_tm_settings["async_insert_use_adaptive_busy_timeout"] = 0 - fixed_tm_settings["async_insert_busy_timeout_ms"] = 200 + fixed_tm_settings["async_insert_busy_timeout_ms"] = 100 fixed_tm_run_duration = timeit.timeit( lambda: _insert_queries_sequentially( fixed_tm_table_name, fixed_tm_settings, - iterations=100, + iterations=50, max_values_size=1000, array_size_range=[10, 50], ), @@ -231,13 +231,13 @@ def test_compare_sequential_inserts_durations_for_adaptive_and_fixed_async_timeo adaptive_tm_settings = copy.copy(_query_settings) adaptive_tm_settings["async_insert_busy_timeout_min_ms"] = 10 - adaptive_tm_settings["async_insert_busy_timeout_max_ms"] = 1000 + adaptive_tm_settings["async_insert_busy_timeout_max_ms"] = 500 adaptive_tm_run_duration = timeit.timeit( lambda: _insert_queries_sequentially( adaptive_tm_table_name, adaptive_tm_settings, - iterations=100, + iterations=50, max_values_size=1000, array_size_range=[10, 50], ), @@ -268,14 +268,14 @@ def test_compare_parallel_inserts_durations_for_adaptive_and_fixed_async_timeout fixed_tm_settings = copy.copy(_query_settings) fixed_tm_settings["async_insert_use_adaptive_busy_timeout"] = 0 - fixed_tm_settings["async_insert_busy_timeout_ms"] = 200 + fixed_tm_settings["async_insert_busy_timeout_ms"] = 500 fixed_tm_run_duration = timeit.timeit( lambda: _insert_queries_in_parallel( fixed_tm_table_name, fixed_tm_settings, thread_num=15, - tasks=1000, + tasks=150, max_values_size=1000, array_size_range=[10, 50], ), @@ -300,14 +300,14 @@ def test_compare_parallel_inserts_durations_for_adaptive_and_fixed_async_timeout adaptive_tm_settings = copy.copy(_query_settings) adaptive_tm_settings["async_insert_busy_timeout_min_ms"] = 10 - adaptive_tm_settings["async_insert_busy_timeout_max_ms"] = 200 + adaptive_tm_settings["async_insert_busy_timeout_max_ms"] = 500 adaptive_tm_run_duration = timeit.timeit( lambda: _insert_queries_in_parallel( adaptive_tm_table_name, adaptive_tm_settings, thread_num=15, - tasks=100, + tasks=150, max_values_size=1000, array_size_range=[10, 50], ), @@ -344,29 +344,34 @@ def test_change_queries_frequency(): settings = copy.copy(_query_settings) min_ms = 50 + max_ms = 200 + settings["async_insert_busy_timeout_min_ms"] = min_ms - settings["async_insert_busy_timeout_max_ms"] = 2000 + settings["async_insert_busy_timeout_max_ms"] = max_ms + + _insert_queries_sequentially( + table_name, + settings, + iterations=50, + max_values_size=1000, + array_size_range=[10, 50], + ) + + select_log_query = f"SELECT countIf(timeout_milliseconds - {min_ms} < 25) FROM (SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 10)" + res = node.query(select_log_query) + assert int(res) >= 5 _insert_queries_in_parallel( table_name, settings, - thread_num=15, + thread_num=20, tasks=2000, max_values_size=1000, array_size_range=[10, 15], ) - _insert_queries_sequentially( - table_name, - settings, - iterations=200, - max_values_size=1000, - array_size_range=[10, 50], - ) - - select_log_query = "SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 50" + select_log_query = f"SELECT countIf({max_ms} - timeout_milliseconds < 100) FROM (SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 10)" res = node.query(select_log_query) - for line in res.splitlines(): - assert int(line) == min_ms + assert int(res) >= 5 node.query("DROP TABLE IF EXISTS {} SYNC".format(table_name)) diff --git a/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py index 5fc8f67b75b..ad3d35d8bbd 100644 --- a/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py +++ b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py @@ -532,6 +532,7 @@ def test_current_database(): assert "Not enough privileges" in instance.query_and_get_error( "SELECT * FROM table", user="A" ) + instance.query("DROP TABLE default.table SYNC") def test_grant_with_replace_option(): From e6b18f2b300c7aec883d60619f432f7e4b4cca3a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 22 Aug 2024 19:05:07 +0000 Subject: [PATCH 064/259] black --- .../test_postgresql_replica_database_engine_2/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 428e9ecdb96..0a364d7802b 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -61,6 +61,7 @@ pg_manager = PostgresManager() pg_manager2 = PostgresManager() pg_manager_instance2 = PostgresManager() + @pytest.fixture(scope="module") def started_cluster(): try: @@ -918,7 +919,7 @@ def test_failed_load_from_snapshot(started_cluster): def test_symbols_in_publication_name(started_cluster): id = uuid.uuid4() - db = f'test_{id}' + db = f"test_{id}" table = f"test_symbols_in_publication_name" pg_manager3 = PostgresManager() From 1aaf9a08c4e0433dcf2afc97770b5a595bd108ee Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 22 Aug 2024 23:25:38 +0000 Subject: [PATCH 065/259] fix async inserts with alter --- src/Interpreters/AsynchronousInsertQueue.cpp | 62 +++++++++++--------- 1 file changed, 34 insertions(+), 28 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 62777524c2a..52f8a8dd42a 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -33,6 +33,8 @@ #include #include #include +#include +#include namespace CurrentMetrics { @@ -308,6 +310,7 @@ void AsynchronousInsertQueue::preprocessInsertQuery(const ASTPtr & query, const /* no_squash */ false, /* no_destination */ false, /* async_insert */ false); + auto table = interpreter.getTable(insert_query); auto sample_block = InterpreterInsertQuery::getSampleBlock(insert_query, table, table->getInMemoryMetadataPtr(), query_context); @@ -318,6 +321,10 @@ void AsynchronousInsertQueue::preprocessInsertQuery(const ASTPtr & query, const /// InterpreterInsertQuery::getTable() -> ITableFunction::execute(). if (insert_query.table_id) query_context->checkAccess(AccessType::INSERT, insert_query.table_id, sample_block.getNames()); + + insert_query.columns = std::make_shared(); + for (const auto & column : sample_block) + insert_query.columns->children.push_back(std::make_shared(column.name)); } AsynchronousInsertQueue::PushResult @@ -872,36 +879,35 @@ try } }; - Chunk chunk; - auto header = pipeline.getHeader(); - - if (key.data_kind == DataKind::Parsed) - chunk = processEntriesWithParsing(key, data, header, insert_context, log, add_entry_to_asynchronous_insert_log); - else - chunk = processPreprocessedEntries(key, data, header, insert_context, add_entry_to_asynchronous_insert_log); - - ProfileEvents::increment(ProfileEvents::AsyncInsertRows, chunk.getNumRows()); - - auto log_and_add_finish_to_query_log = [&](size_t num_rows, size_t num_bytes) - { - LOG_DEBUG(log, "Flushed {} rows, {} bytes for query '{}'", num_rows, num_bytes, key.query_str); - queue_shard_flush_time_history.updateWithCurrentTime(); - - bool pulling_pipeline = false; - logQueryFinish( - query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); - }; - - - if (chunk.getNumRows() == 0) - { - finish_entries(); - log_and_add_finish_to_query_log(0, 0); - return; - } - try { + Chunk chunk; + auto header = pipeline.getHeader(); + + if (key.data_kind == DataKind::Parsed) + chunk = processEntriesWithParsing(key, data, header, insert_context, log, add_entry_to_asynchronous_insert_log); + else + chunk = processPreprocessedEntries(key, data, header, insert_context, add_entry_to_asynchronous_insert_log); + + ProfileEvents::increment(ProfileEvents::AsyncInsertRows, chunk.getNumRows()); + + auto log_and_add_finish_to_query_log = [&](size_t num_rows, size_t num_bytes) + { + LOG_DEBUG(log, "Flushed {} rows, {} bytes for query '{}'", num_rows, num_bytes, key.query_str); + queue_shard_flush_time_history.updateWithCurrentTime(); + + bool pulling_pipeline = false; + logQueryFinish( + query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); + }; + + if (chunk.getNumRows() == 0) + { + finish_entries(); + log_and_add_finish_to_query_log(0, 0); + return; + } + size_t num_rows = chunk.getNumRows(); size_t num_bytes = chunk.bytes(); From 7517ef4cc9f990e1ac949366a1ec8688aba0420f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 23 Aug 2024 16:08:04 +0000 Subject: [PATCH 066/259] fix async insert with alter modify column --- src/Interpreters/AsynchronousInsertQueue.cpp | 178 ++++++++++-------- src/Interpreters/AsynchronousInsertQueue.h | 5 + .../03229_async_insert_alter.reference | 8 + .../0_stateless/03229_async_insert_alter.sql | 46 +++++ .../03229_async_insert_alter_http.reference | 8 + .../03229_async_insert_alter_http.sh | 55 ++++++ 6 files changed, 226 insertions(+), 74 deletions(-) create mode 100644 tests/queries/0_stateless/03229_async_insert_alter.reference create mode 100644 tests/queries/0_stateless/03229_async_insert_alter.sql create mode 100644 tests/queries/0_stateless/03229_async_insert_alter_http.reference create mode 100755 tests/queries/0_stateless/03229_async_insert_alter_http.sh diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 52f8a8dd42a..5ed0d6d6257 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -699,6 +699,17 @@ catch (...) tryLogCurrentException("AsynchronousInsertQueue", "Failed to add elements to AsynchronousInsertLog"); } +void convertBlockToHeader(Block & block, const Block & header) +{ + auto converting_dag = ActionsDAG::makeConvertingActions( + block.getColumnsWithTypeAndName(), + header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting_actions = std::make_shared(std::move(converting_dag)); + converting_actions->execute(block); +} + String serializeQuery(const IAST & query, size_t max_length) { return query.hasSecretParts() @@ -794,6 +805,48 @@ try if (async_insert_log) log_elements.reserve(data->entries.size()); + auto add_entry_to_asynchronous_insert_log = [&]( + const InsertData::EntryPtr & entry, + const NameToNameMap & query_by_format, + const String & parsing_exception, + size_t num_rows, + size_t num_bytes) + { + if (!async_insert_log) + return; + + AsynchronousInsertLogElement elem; + elem.event_time = timeInSeconds(entry->create_time); + elem.event_time_microseconds = timeInMicroseconds(entry->create_time); + elem.database = query_database; + elem.table = query_table; + elem.format = entry->format; + elem.query_id = entry->query_id; + elem.bytes = num_bytes; + elem.rows = num_rows; + elem.exception = parsing_exception; + elem.data_kind = entry->chunk.getDataKind(); + elem.timeout_milliseconds = data->timeout_ms.count(); + elem.flush_query_id = insert_query_id; + + auto it = query_by_format.find(entry->format); + elem.query_for_logging = it != query_by_format.end() ? it->second : key.query_str; + + /// If there was a parsing error, + /// the entry won't be flushed anyway, + /// so add the log element immediately. + if (!elem.exception.empty()) + { + elem.status = AsynchronousInsertLogElement::ParsingError; + async_insert_log->add(std::move(elem)); + } + else + { + elem.status = AsynchronousInsertLogElement::Ok; + log_elements.push_back(elem); + } + }; + try { interpreter = std::make_unique( @@ -822,49 +875,21 @@ try catch (...) { logExceptionBeforeStart(query_for_logging, insert_context, key.query, query_span, start_watch.elapsedMilliseconds()); + + if (async_insert_log) + { + auto query_by_format = getQueriesByFormat(key.query, data->entries, insert_context); + for (const auto & entry : data->entries) + add_entry_to_asynchronous_insert_log(entry, query_by_format, "", 0, entry->chunk.byteSize()); + + auto exception = getCurrentExceptionMessage(false); + auto flush_time = std::chrono::system_clock::now(); + appendElementsToLogSafe(*async_insert_log, std::move(log_elements), flush_time, exception); + } throw; } - auto add_entry_to_asynchronous_insert_log = [&](const auto & entry, - const auto & entry_query_for_logging, - const auto & exception, - size_t num_rows, - size_t num_bytes, - Milliseconds timeout_ms) - { - if (!async_insert_log) - return; - - AsynchronousInsertLogElement elem; - elem.event_time = timeInSeconds(entry->create_time); - elem.event_time_microseconds = timeInMicroseconds(entry->create_time); - elem.query_for_logging = entry_query_for_logging; - elem.database = query_database; - elem.table = query_table; - elem.format = entry->format; - elem.query_id = entry->query_id; - elem.bytes = num_bytes; - elem.rows = num_rows; - elem.exception = exception; - elem.data_kind = entry->chunk.getDataKind(); - elem.timeout_milliseconds = timeout_ms.count(); - elem.flush_query_id = insert_query_id; - - /// If there was a parsing error, - /// the entry won't be flushed anyway, - /// so add the log element immediately. - if (!elem.exception.empty()) - { - elem.status = AsynchronousInsertLogElement::ParsingError; - async_insert_log->add(std::move(elem)); - } - else - { - log_elements.push_back(elem); - } - }; - - auto finish_entries = [&] + auto finish_entries = [&](size_t num_rows, size_t num_bytes) { for (const auto & entry : data->entries) { @@ -877,6 +902,13 @@ try auto flush_time = std::chrono::system_clock::now(); appendElementsToLogSafe(*async_insert_log, std::move(log_elements), flush_time, ""); } + + LOG_DEBUG(log, "Flushed {} rows, {} bytes for query '{}'", num_rows, num_bytes, key.query_str); + queue_shard_flush_time_history.updateWithCurrentTime(); + + bool pulling_pipeline = false; + logQueryFinish( + query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); }; try @@ -891,20 +923,9 @@ try ProfileEvents::increment(ProfileEvents::AsyncInsertRows, chunk.getNumRows()); - auto log_and_add_finish_to_query_log = [&](size_t num_rows, size_t num_bytes) - { - LOG_DEBUG(log, "Flushed {} rows, {} bytes for query '{}'", num_rows, num_bytes, key.query_str); - queue_shard_flush_time_history.updateWithCurrentTime(); - - bool pulling_pipeline = false; - logQueryFinish( - query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); - }; - if (chunk.getNumRows() == 0) { - finish_entries(); - log_and_add_finish_to_query_log(0, 0); + finish_entries(0, 0); return; } @@ -917,7 +938,7 @@ try CompletedPipelineExecutor completed_executor(pipeline); completed_executor.execute(); - log_and_add_finish_to_query_log(num_rows, num_bytes); + finish_entries(num_rows, num_bytes); } catch (...) { @@ -931,8 +952,6 @@ try } throw; } - - finish_entries(); } catch (const Exception & e) { @@ -1013,7 +1032,7 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, query_for_logging, current_exception, num_rows, num_bytes, data->timeout_ms); + add_to_async_insert_log(entry, {}, current_exception, num_rows, num_bytes); current_exception.clear(); entry->resetChunk(); @@ -1036,19 +1055,7 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( auto chunk_info = std::make_shared(); auto result_columns = header.cloneEmptyColumns(); - std::unordered_map format_to_query; - - auto get_query_by_format = [&](const String & format) -> const String & - { - auto [it, inserted] = format_to_query.try_emplace(format); - if (!inserted) - return it->second; - - auto query = key.query->clone(); - assert_cast(*query).format = format; - it->second = serializeQuery(*query, insert_context->getSettingsRef().log_queries_cut_to_length); - return it->second; - }; + auto query_by_format = getQueriesByFormat(key.query, data->entries, insert_context); for (const auto & entry : data->entries) { @@ -1057,17 +1064,19 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected entry with data kind Preprocessed. Got: {}", entry->chunk.getDataKind()); - auto columns = block->getColumns(); + Block block_to_insert = *block; + if (!isCompatibleHeader(block_to_insert, header)) + convertBlockToHeader(block_to_insert, header); + + auto columns = block_to_insert.getColumns(); for (size_t i = 0, s = columns.size(); i < s; ++i) result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); - total_rows += block->rows(); + total_rows += block_to_insert.rows(); chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - const auto & query_for_logging = get_query_by_format(entry->format); - add_to_async_insert_log(entry, query_for_logging, "", block->rows(), block->bytes(), data->timeout_ms); - + add_to_async_insert_log(entry, query_by_format, "", block_to_insert.rows(), block_to_insert.bytes()); entry->resetChunk(); } @@ -1076,6 +1085,27 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( return chunk; } +NameToNameMap AsynchronousInsertQueue::getQueriesByFormat( + const ASTPtr & query, + const std::list & entries, + const ContextPtr & insert_context) +{ + std::unordered_map format_to_query; + auto query_copy = query->clone(); + + for (const auto & entry : entries) + { + auto [it, inserted] = format_to_query.try_emplace(entry->format); + if (!inserted) + continue; + + assert_cast(*query_copy).format = entry->format; + it->second = serializeQuery(*query_copy, insert_context->getSettingsRef().log_queries_cut_to_length); + } + + return format_to_query; +} + template void AsynchronousInsertQueue::finishWithException( const ASTPtr & query, const std::list & entries, const E & exception) diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index da14b43d276..9a84fe8bb12 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -293,6 +293,11 @@ private: const ContextPtr & insert_context, LogFunc && add_to_async_insert_log); + static NameToNameMap getQueriesByFormat( + const ASTPtr & query, + const std::list & entries, + const ContextPtr & insert_context); + template static void finishWithException(const ASTPtr & query, const std::list & entries, const E & exception); diff --git a/tests/queries/0_stateless/03229_async_insert_alter.reference b/tests/queries/0_stateless/03229_async_insert_alter.reference new file mode 100644 index 00000000000..f66021d0bfe --- /dev/null +++ b/tests/queries/0_stateless/03229_async_insert_alter.reference @@ -0,0 +1,8 @@ +42 24 0 +42 24 0 +43 34 55 +42 24 +43 34 +INSERT INTO default.t_async_insert_alter (id, v1) FORMAT Values Preprocessed Ok +INSERT INTO default.t_async_insert_alter (id, v1, value2) FORMAT Values Preprocessed Ok +INSERT INTO default.t_async_insert_alter (id, v1, value2) FORMAT Values Preprocessed FlushError diff --git a/tests/queries/0_stateless/03229_async_insert_alter.sql b/tests/queries/0_stateless/03229_async_insert_alter.sql new file mode 100644 index 00000000000..a95bbc6e55f --- /dev/null +++ b/tests/queries/0_stateless/03229_async_insert_alter.sql @@ -0,0 +1,46 @@ +-- Tags: no-parallel + +SET wait_for_async_insert = 0; +SET async_insert_busy_timeout_max_ms = 300000; +SET async_insert_busy_timeout_min_ms = 300000; +SET async_insert_use_adaptive_busy_timeout = 0; + +DROP TABLE IF EXISTS t_async_insert_alter; + +CREATE TABLE t_async_insert_alter (id Int64, v1 Int64) ENGINE = MergeTree ORDER BY id SETTINGS async_insert = 1; + +-- ADD COLUMN + +INSERT INTO t_async_insert_alter VALUES (42, 24); + +ALTER TABLE t_async_insert_alter ADD COLUMN value2 Int64; + +SYSTEM FLUSH ASYNC INSERT QUEUE; +SYSTEM FLUSH LOGS; + +SELECT * FROM t_async_insert_alter ORDER BY id; + +-- MODIFY COLUMN + +INSERT INTO t_async_insert_alter VALUES (43, 34, 55); + +ALTER TABLE t_async_insert_alter MODIFY COLUMN value2 String; + +SYSTEM FLUSH ASYNC INSERT QUEUE; +SYSTEM FLUSH LOGS; + +SELECT * FROM t_async_insert_alter ORDER BY id; + +-- DROP COLUMN + +INSERT INTO t_async_insert_alter VALUES ('100', '200', '300'); + +ALTER TABLE t_async_insert_alter DROP COLUMN value2; + +SYSTEM FLUSH ASYNC INSERT QUEUE; +SYSTEM FLUSH LOGS; + +SELECT * FROM t_async_insert_alter ORDER BY id; +SELECT query, data_kind, status FROM system.asynchronous_insert_log WHERE database = currentDatabase() AND table = 't_async_insert_alter' ORDER BY event_time_microseconds; + +DROP TABLE t_async_insert_alter; diff --git a/tests/queries/0_stateless/03229_async_insert_alter_http.reference b/tests/queries/0_stateless/03229_async_insert_alter_http.reference new file mode 100644 index 00000000000..195701d2b82 --- /dev/null +++ b/tests/queries/0_stateless/03229_async_insert_alter_http.reference @@ -0,0 +1,8 @@ +42 24 0 +42 24 0 +43 34 55 +42 24 +43 34 +INSERT INTO default.t_async_insert_alter (id, v1) FORMAT Values Parsed Ok +INSERT INTO default.t_async_insert_alter (id, v1, value2) FORMAT Values Parsed Ok +INSERT INTO default.t_async_insert_alter (id, v1, value2) FORMAT Values Parsed FlushError diff --git a/tests/queries/0_stateless/03229_async_insert_alter_http.sh b/tests/queries/0_stateless/03229_async_insert_alter_http.sh new file mode 100755 index 00000000000..18e68f51285 --- /dev/null +++ b/tests/queries/0_stateless/03229_async_insert_alter_http.sh @@ -0,0 +1,55 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q " + DROP TABLE IF EXISTS t_async_insert_alter; + CREATE TABLE t_async_insert_alter (id Int64, v1 Int64) ENGINE = MergeTree ORDER BY id SETTINGS async_insert = 1; +" + +url="${CLICKHOUSE_URL}&async_insert=1&async_insert_busy_timeout_max_ms=300000&async_insert_busy_timeout_min_ms=300000&wait_for_async_insert=0&async_insert_use_adaptive_busy_timeout=0" + +# ADD COLUMN + +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_async_insert_alter VALUES (42, 24)" + +$CLICKHOUSE_CLIENT -q " + ALTER TABLE t_async_insert_alter ADD COLUMN value2 Int64; + + SYSTEM FLUSH ASYNC INSERT QUEUE; + SYSTEM FLUSH LOGS; + + SELECT * FROM t_async_insert_alter ORDER BY id; +" + +# MODIFY COLUMN + +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_async_insert_alter VALUES (43, 34, 55)" + +$CLICKHOUSE_CLIENT -q " + ALTER TABLE t_async_insert_alter MODIFY COLUMN value2 String; + + SYSTEM FLUSH ASYNC INSERT QUEUE; + SYSTEM FLUSH LOGS; + + SELECT * FROM t_async_insert_alter ORDER BY id; +" + +## DROP COLUMN + +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_async_insert_alter VALUES ('100', '200', '300')" + +$CLICKHOUSE_CLIENT -q " + ALTER TABLE t_async_insert_alter DROP COLUMN value2; + + SYSTEM FLUSH ASYNC INSERT QUEUE; + SYSTEM FLUSH LOGS; + + SELECT * FROM t_async_insert_alter ORDER BY id; + SELECT query, data_kind, status FROM system.asynchronous_insert_log WHERE database = currentDatabase() AND table = 't_async_insert_alter' ORDER BY event_time_microseconds; + + DROP TABLE t_async_insert_alter; +" From b39c5cdb4021fd2b52fde4d320e421abcd868c77 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 23 Aug 2024 16:17:14 +0000 Subject: [PATCH 067/259] fix tests --- .../02790_async_queries_in_query_log.reference | 12 ++++++------ ...03148_async_queries_in_query_log_errors.reference | 6 +++--- .../queries/0_stateless/03229_async_insert_alter.sql | 1 + .../0_stateless/03229_async_insert_alter_http.sh | 1 + 4 files changed, 11 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02790_async_queries_in_query_log.reference b/tests/queries/0_stateless/02790_async_queries_in_query_log.reference index 567e8d4f4b5..84254cc8735 100644 --- a/tests/queries/0_stateless/02790_async_queries_in_query_log.reference +++ b/tests/queries/0_stateless/02790_async_queries_in_query_log.reference @@ -9,7 +9,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing'] @@ -26,7 +26,7 @@ written_rows: 4 written_bytes: 16 result_rows: 4 result_bytes: 16 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing'] @@ -54,7 +54,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing','default.async_insert_target'] @@ -71,7 +71,7 @@ written_rows: 6 written_bytes: 24 result_rows: 6 result_bytes: 24 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing','default.async_insert_target'] @@ -118,7 +118,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing','default.async_insert_target'] @@ -135,7 +135,7 @@ written_rows: 3 written_bytes: 12 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing','default.async_insert_target'] diff --git a/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.reference b/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.reference index 685d28268f6..7cc2efd50ec 100644 --- a/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.reference +++ b/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.reference @@ -4,7 +4,7 @@ Row 1: ────── database: default table: async_insert_landing -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values format: Values error: DB::Exc*****on: Cannot parse string 'Invalid' as UInt32: populated_flush_query_id: 1 @@ -18,7 +18,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing'] @@ -35,7 +35,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing'] diff --git a/tests/queries/0_stateless/03229_async_insert_alter.sql b/tests/queries/0_stateless/03229_async_insert_alter.sql index a95bbc6e55f..022e386bef0 100644 --- a/tests/queries/0_stateless/03229_async_insert_alter.sql +++ b/tests/queries/0_stateless/03229_async_insert_alter.sql @@ -1,4 +1,5 @@ -- Tags: no-parallel +-- no-parallel because the test uses FLUSH ASYNC INSERT QUEUE SET wait_for_async_insert = 0; SET async_insert_busy_timeout_max_ms = 300000; diff --git a/tests/queries/0_stateless/03229_async_insert_alter_http.sh b/tests/queries/0_stateless/03229_async_insert_alter_http.sh index 18e68f51285..fe72ed3299a 100755 --- a/tests/queries/0_stateless/03229_async_insert_alter_http.sh +++ b/tests/queries/0_stateless/03229_async_insert_alter_http.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash # Tags: no-parallel +# no-parallel because the test uses FLUSH ASYNC INSERT QUEUE CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 66be44e7aa719681b7f13e31a91f2c105e705a4f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 23 Aug 2024 19:09:09 +0000 Subject: [PATCH 068/259] better --- tests/ci/integration_tests_runner.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 074c947fe02..deddcb95cf4 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -33,7 +33,7 @@ CLICKHOUSE_BINARY_PATH = "usr/bin/clickhouse" CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-odbc-bridge" CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-library-bridge" -FLAKY_TRIES_COUNT = 10 # run whole pytest several times +FLAKY_TRIES_COUNT = 5 # run whole pytest several times FLAKY_REPEAT_COUNT = 5 # runs test case in single module several times MAX_TIME_SECONDS = 3600 @@ -794,7 +794,7 @@ class ClickhouseIntegrationTestsRunner: } # type: Dict tests_times = defaultdict(float) # type: Dict tests_log_paths = defaultdict(list) - + id_counter = 0 for test_to_run in tests_to_run: tries_num = 1 if should_fail else FLAKY_TRIES_COUNT for i in range(tries_num): @@ -805,12 +805,13 @@ class ClickhouseIntegrationTestsRunner: logging.info("Running tests for the %s time", i) group_counters, group_test_times, log_paths = self.try_run_test_group( repo_path, - "bugfix" if should_fail else "flaky", + f"bugfix_{id_counter}" if should_fail else f"flaky{id_counter}", [test_to_run], 1, 1, FLAKY_REPEAT_COUNT, ) + id_counter = id_counter + 1 for counter, value in group_counters.items(): logging.info( "Tests from group %s stats, %s count %s", From 9339c0593b354dbb54b418425c6cfe1ac37e556c Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 26 Aug 2024 13:37:37 +0200 Subject: [PATCH 069/259] handle timeout in flaky check --- tests/ci/integration_tests_runner.py | 67 ++++++++++++++-------------- 1 file changed, 33 insertions(+), 34 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index deddcb95cf4..a7d5a8c4cf6 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -873,21 +873,45 @@ class ClickhouseIntegrationTestsRunner: def run_impl(self, repo_path, build_path): stopwatch = Stopwatch() if self.flaky_check or self.bugfix_validate_check: - return self.run_flaky_check( - repo_path, build_path, should_fail=self.bugfix_validate_check + result_state, status_text, test_result, tests_log_paths = ( + self.run_flaky_check( + repo_path, build_path, should_fail=self.bugfix_validate_check + ) + ) + else: + result_state, status_text, test_result, tests_log_paths = ( + self.run_normal_check(build_path, repo_path) ) - self._install_clickhouse(build_path) + if self.soft_deadline_time < time.time(): + status_text = "Timeout, " + status_text + result_state = "failure" + if timeout_expired: + logging.error( + "Job killed by external timeout signal - setting status to failure!" + ) + status_text = "Job timeout expired, " + status_text + result_state = "failure" + # add mock test case to make timeout visible in job report and in ci db + test_result.insert( + 0, (JOB_TIMEOUT_TEST_NAME, "FAIL", f"{stopwatch.duration_seconds}", "") + ) + + if "(memory)" in self.params["context_name"]: + result_state = "success" + + return result_state, status_text, test_result, tests_log_paths + + def run_normal_check(self, build_path, repo_path): + self._install_clickhouse(build_path) logging.info("Pulling images") self._pre_pull_images(repo_path) - logging.info( "Dump iptables before run %s", subprocess.check_output("sudo iptables -nvL", shell=True), ) all_tests = self._get_all_tests(repo_path) - if self.run_by_hash_total != 0: grouped_tests = self.group_test_by_file(all_tests) all_filtered_by_hash_tests = [] @@ -895,7 +919,6 @@ class ClickhouseIntegrationTestsRunner: if stringhash(group) % self.run_by_hash_total == self.run_by_hash_num: all_filtered_by_hash_tests += tests_in_group all_tests = all_filtered_by_hash_tests - parallel_skip_tests = self._get_parallel_tests_skip_list(repo_path) logging.info( "Found %s tests first 3 %s", len(all_tests), " ".join(all_tests[:3]) @@ -927,14 +950,12 @@ class ClickhouseIntegrationTestsRunner: len(not_found_tests), " ".join(not_found_tests[:3]), ) - grouped_tests = self.group_test_by_file(filtered_sequential_tests) i = 0 for par_group in chunks(filtered_parallel_tests, PARALLEL_GROUP_SIZE): grouped_tests[f"parallel{i}"] = par_group i += 1 logging.info("Found %s tests groups", len(grouped_tests)) - counters = { "ERROR": [], "PASSED": [], @@ -945,14 +966,11 @@ class ClickhouseIntegrationTestsRunner: } # type: Dict tests_times = defaultdict(float) tests_log_paths = defaultdict(list) - items_to_run = list(grouped_tests.items()) - logging.info("Total test groups %s", len(items_to_run)) if self.shuffle_test_groups(): logging.info("Shuffling test groups") random.shuffle(items_to_run) - for group, tests in items_to_run: if timeout_expired: print("Timeout expired - break tests execution") @@ -980,7 +998,6 @@ class ClickhouseIntegrationTestsRunner: if len(counters["FAILED"]) + len(counters["ERROR"]) >= 20: logging.info("Collected more than 20 failed/error tests, stopping") break - if counters["FAILED"] or counters["ERROR"]: logging.info( "Overall status failure, because we have tests in FAILED or ERROR state" @@ -989,7 +1006,6 @@ class ClickhouseIntegrationTestsRunner: else: logging.info("Overall success!") result_state = "success" - test_result = [] for state in ( "ERROR", @@ -1009,33 +1025,14 @@ class ClickhouseIntegrationTestsRunner: (c, text_state, f"{tests_times[c]:.2f}", tests_log_paths[c]) for c in counters[state] ] - failed_sum = len(counters["FAILED"]) + len(counters["ERROR"]) status_text = f"fail: {failed_sum}, passed: {len(counters['PASSED'])}" - if self.soft_deadline_time < time.time(): - status_text = "Timeout, " + status_text - result_state = "failure" - - if timeout_expired: - logging.error( - "Job killed by external timeout signal - setting status to failure!" - ) - status_text = "Job timeout expired, " + status_text - result_state = "failure" - # add mock test case to make timeout visible in job report and in ci db - test_result.insert( - 0, (JOB_TIMEOUT_TEST_NAME, "FAIL", f"{stopwatch.duration_seconds}", "") - ) - if not counters or sum(len(counter) for counter in counters.values()) == 0: status_text = "No tests found for some reason! It's a bug" result_state = "failure" - if "(memory)" in self.params["context_name"]: - result_state = "success" - - return result_state, status_text, test_result, [] + return result_state, status_text, test_result, tests_log_paths def write_results(results_file, status_file, results, status): @@ -1068,7 +1065,9 @@ def run(): logging.info("Clearing dmesg before run") subprocess.check_call("sudo -E dmesg --clear", shell=True) - state, description, test_results, _ = runner.run_impl(repo_path, build_path) + state, description, test_results, _test_log_paths = runner.run_impl( + repo_path, build_path + ) logging.info("Tests finished") if IS_CI: From 5bad631d468a8cddbd1af40dc43c2d5db2931b46 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 070/259] Add config for outout_format_identifers_quoting_style --- src/Core/Settings.h | 2 ++ src/Core/SettingsChangesHistory.cpp | 5 +++++ src/Core/SettingsEnums.cpp | 6 ++++++ src/Core/SettingsEnums.h | 3 +++ src/Interpreters/formatWithPossiblyHidingSecrets.h | 2 +- src/Parsers/IAST.cpp | 5 ++++- src/Parsers/IAST.h | 2 +- 7 files changed, 22 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 23dc2a8fdc5..c33562e25ed 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1296,6 +1296,8 @@ class IColumn; M(Bool, precise_float_parsing, false, "Prefer more precise (but slower) float parsing algorithm", 0) \ M(DateTimeOverflowBehavior, date_time_overflow_behavior, "ignore", "Overflow mode for Date, Date32, DateTime, DateTime64 types. Possible values: 'ignore', 'throw', 'saturate'.", 0) \ M(Bool, validate_experimental_and_suspicious_types_inside_nested_types, true, "Validate usage of experimental and suspicious types inside nested types like Array/Map/Tuple", 0) \ + \ + M(IdentifierQuotingStyle, output_format_identifier_quoting_style, IdentifierQuotingStyle::Backticks, "Set the quoting style for identifiers", 0) \ // End of FORMAT_FACTORY_SETTINGS diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 5e831c6301c..76ec440aaf2 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,6 +57,11 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { + {"24.12", + { + {"output_format_identifier_quoting_style", 1, 1, "New setting."} + } + }, {"24.12", { } diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 30e60f246f4..0c43b5256d6 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -244,4 +244,10 @@ IMPLEMENT_SETTING_ENUM( GroupArrayActionWhenLimitReached, ErrorCodes::BAD_ARGUMENTS, {{"throw", GroupArrayActionWhenLimitReached::THROW}, {"discard", GroupArrayActionWhenLimitReached::DISCARD}}) + +IMPLEMENT_SETTING_ENUM(IdentifierQuotingStyle, ErrorCodes::BAD_ARGUMENTS, + {{"None", IdentifierQuotingStyle::None}, + {"Backticks", IdentifierQuotingStyle::Backticks}, + {"DoubleQuotes", IdentifierQuotingStyle::DoubleQuotes}, + {"BackticksMySQL", IdentifierQuotingStyle::BackticksMySQL}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 5c2d8825e76..d358a2d44a0 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -351,6 +352,8 @@ DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOverflowBehavior, FormatSettings::DateT DECLARE_SETTING_ENUM(SQLSecurityType) +DECLARE_SETTING_ENUM(IdentifierQuotingStyle) + enum class GroupArrayActionWhenLimitReached : uint8_t { THROW, diff --git a/src/Interpreters/formatWithPossiblyHidingSecrets.h b/src/Interpreters/formatWithPossiblyHidingSecrets.h index 14e84f1d1a4..8ec40d3dcc3 100644 --- a/src/Interpreters/formatWithPossiblyHidingSecrets.h +++ b/src/Interpreters/formatWithPossiblyHidingSecrets.h @@ -26,7 +26,7 @@ inline String format(const SecretHidingFormatSettings & settings) && settings.ctx->getAccess()->isGranted(AccessType::displaySecretsInShowAndSelect); return settings.query.formatWithPossiblyHidingSensitiveData( - settings.max_length, settings.one_line, show_secrets, settings.ctx->getSettingsRef().print_pretty_type_names); + settings.max_length, settings.one_line, show_secrets, settings.ctx->getSettingsRef().print_pretty_type_names, settings.ctx->getSettingsRef().output_format_identifier_quoting_style); } } diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 5bd2c92c60a..90cdd330ee8 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -165,12 +165,15 @@ size_t IAST::checkDepthImpl(size_t max_depth) const return res; } -String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names) const +String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names, IdentifierQuotingStyle identifier_quoting_style) const { + WriteBufferFromOwnString buf; FormatSettings settings(buf, one_line); settings.show_secrets = show_secrets; settings.print_pretty_type_names = print_pretty_type_names; + settings.always_quote_identifiers = identifier_quoting_style != IdentifierQuotingStyle::None; + settings.identifier_quoting_style = identifier_quoting_style; format(settings); return wipeSensitiveDataAndCutToLength(buf.str(), max_length); } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 2293d50b0ec..777265aaa7c 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -278,7 +278,7 @@ public: /// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied. /// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience. - String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names) const; + String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names, IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks) const; /** formatForLogging and formatForErrorMessage always hide secrets. This inconsistent * behaviour is due to the fact such functions are called from Client which knows nothing about From d6ca589ebed7a16f498fbbe4365d9ac07686af08 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 071/259] Treat PROJECTION, DICTIONARY, COLUMN, INDEX names as identifiers when formating --- src/Core/SettingsChangesHistory.cpp | 2 +- src/Parsers/ASTColumnDeclaration.cpp | 3 +-- src/Parsers/ASTDictionaryAttributeDeclaration.cpp | 2 +- src/Parsers/ASTIndexDeclaration.cpp | 2 +- src/Parsers/ASTProjectionDeclaration.cpp | 2 +- 5 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 76ec440aaf2..46f6590b724 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,7 +59,7 @@ static std::initializer_listformatImpl(s, state, frame); } diff --git a/src/Parsers/ASTProjectionDeclaration.cpp b/src/Parsers/ASTProjectionDeclaration.cpp index 60050986161..af79745a88e 100644 --- a/src/Parsers/ASTProjectionDeclaration.cpp +++ b/src/Parsers/ASTProjectionDeclaration.cpp @@ -17,7 +17,7 @@ ASTPtr ASTProjectionDeclaration::clone() const void ASTProjectionDeclaration::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << backQuoteIfNeed(name); + settings.writeIdentifier(name); std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' '); std::string nl_or_nothing = settings.one_line ? "" : "\n"; settings.ostr << settings.nl_or_ws << indent_str << "(" << nl_or_nothing; From e1d66b9e262466695bff2a30cccec047a953e269 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 072/259] Add output_format_always_quote_identifiers into the settings --- src/Core/ExternalTable.cpp | 12 ++++++++--- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/Interpreters/executeQuery.cpp | 4 ++-- .../formatWithPossiblyHidingSecrets.h | 7 ++++++- src/Parsers/ASTColumnDeclaration.cpp | 3 ++- src/Parsers/IAST.cpp | 20 ++++++++++++------- src/Parsers/IAST.h | 15 ++++++++++---- 8 files changed, 45 insertions(+), 18 deletions(-) diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 4ff0d7092d8..1f8e63db72e 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -17,11 +17,12 @@ #include #include -#include #include #include #include #include +#include +#include "Parsers/IdentifierQuotingStyle.h" namespace DB @@ -85,7 +86,9 @@ void BaseExternalTable::parseStructureFromStructureField(const std::string & arg /// We use `formatWithPossiblyHidingSensitiveData` instead of `getColumnNameWithoutAlias` because `column->type` is an ASTFunction. /// `getColumnNameWithoutAlias` will return name of the function with `(arguments)` even if arguments is empty. if (column) - structure.emplace_back(column->name, column->type->formatWithPossiblyHidingSensitiveData(0, true, true, false)); + structure.emplace_back( + column->name, + column->type->formatWithPossiblyHidingSensitiveData(0, true, true, false, false, IdentifierQuotingStyle::Backticks)); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: expected column definition, got {}", child->formatForErrorMessage()); } @@ -102,7 +105,10 @@ void BaseExternalTable::parseStructureFromTypesField(const std::string & argumen throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: {}", error); for (size_t i = 0; i < type_list_raw->children.size(); ++i) - structure.emplace_back("_" + toString(i + 1), type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData(0, true, true, false)); + structure.emplace_back( + "_" + toString(i + 1), + type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData( + 0, true, true, false, false, IdentifierQuotingStyle::Backticks)); } void BaseExternalTable::initSampleBlock() diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c33562e25ed..0e58562c745 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1297,6 +1297,7 @@ class IColumn; M(DateTimeOverflowBehavior, date_time_overflow_behavior, "ignore", "Overflow mode for Date, Date32, DateTime, DateTime64 types. Possible values: 'ignore', 'throw', 'saturate'.", 0) \ M(Bool, validate_experimental_and_suspicious_types_inside_nested_types, true, "Validate usage of experimental and suspicious types inside nested types like Array/Map/Tuple", 0) \ \ + M(Bool, output_format_always_quote_identifiers, false, "Always quote identifiers", 0) \ M(IdentifierQuotingStyle, output_format_identifier_quoting_style, IdentifierQuotingStyle::Backticks, "Set the quoting style for identifiers", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 46f6590b724..ac7100d3706 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,6 +59,7 @@ static std::initializer_list executeQueryImpl( /// Verify that AST formatting is consistent: /// If you format AST, parse it back, and format it again, you get the same string. - String formatted1 = ast->formatWithPossiblyHidingSensitiveData(0, true, true, false); + String formatted1 = ast->formatWithPossiblyHidingSensitiveData(0, true, true, false, false, IdentifierQuotingStyle::Backticks); /// The query can become more verbose after formatting, so: size_t new_max_query_size = max_query_size > 0 ? (1000 + 2 * max_query_size) : 0; @@ -811,7 +811,7 @@ static std::tuple executeQueryImpl( chassert(ast2); - String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(0, true, true, false); + String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(0, true, true, false, false, IdentifierQuotingStyle::Backticks); if (formatted1 != formatted2) throw Exception(ErrorCodes::LOGICAL_ERROR, diff --git a/src/Interpreters/formatWithPossiblyHidingSecrets.h b/src/Interpreters/formatWithPossiblyHidingSecrets.h index 8ec40d3dcc3..61f689fb821 100644 --- a/src/Interpreters/formatWithPossiblyHidingSecrets.h +++ b/src/Interpreters/formatWithPossiblyHidingSecrets.h @@ -26,7 +26,12 @@ inline String format(const SecretHidingFormatSettings & settings) && settings.ctx->getAccess()->isGranted(AccessType::displaySecretsInShowAndSelect); return settings.query.formatWithPossiblyHidingSensitiveData( - settings.max_length, settings.one_line, show_secrets, settings.ctx->getSettingsRef().print_pretty_type_names, settings.ctx->getSettingsRef().output_format_identifier_quoting_style); + settings.max_length, + settings.one_line, + show_secrets, + settings.ctx->getSettingsRef().print_pretty_type_names, + settings.ctx->getSettingsRef().output_format_always_quote_identifiers, + settings.ctx->getSettingsRef().output_format_identifier_quoting_style); } } diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 39b1da68a86..7b5ea49b63d 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -66,7 +66,8 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo { frame.need_parens = false; - format_settings.writeIdentifier(name); + /// We have to always quote column names to avoid ambiguity with INDEX and other declarations in CREATE query. + format_settings.writeIdentifier(name, true); if (type) { diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 90cdd330ee8..858630cef2e 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -165,14 +165,20 @@ size_t IAST::checkDepthImpl(size_t max_depth) const return res; } -String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names, IdentifierQuotingStyle identifier_quoting_style) const +String IAST::formatWithPossiblyHidingSensitiveData( + size_t max_length, + bool one_line, + bool show_secrets, + bool print_pretty_type_names, + bool always_quote_identifiers, + IdentifierQuotingStyle identifier_quoting_style) const { WriteBufferFromOwnString buf; FormatSettings settings(buf, one_line); settings.show_secrets = show_secrets; settings.print_pretty_type_names = print_pretty_type_names; - settings.always_quote_identifiers = identifier_quoting_style != IdentifierQuotingStyle::None; + settings.always_quote_identifiers = always_quote_identifiers; settings.identifier_quoting_style = identifier_quoting_style; format(settings); return wipeSensitiveDataAndCutToLength(buf.str(), max_length); @@ -211,13 +217,13 @@ String IAST::getColumnNameWithoutAlias() const } -void IAST::FormatSettings::writeIdentifier(const String & name) const +void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoting) const { switch (identifier_quoting_style) { case IdentifierQuotingStyle::None: { - if (always_quote_identifiers) + if (force_quoting || always_quote_identifiers) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incompatible arguments: always_quote_identifiers = true && " "identifier_quoting_style == IdentifierQuotingStyle::None"); @@ -226,7 +232,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const } case IdentifierQuotingStyle::Backticks: { - if (always_quote_identifiers) + if (force_quoting || always_quote_identifiers) writeBackQuotedString(name, ostr); else writeProbablyBackQuotedString(name, ostr); @@ -234,7 +240,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const } case IdentifierQuotingStyle::DoubleQuotes: { - if (always_quote_identifiers) + if (force_quoting || always_quote_identifiers) writeDoubleQuotedString(name, ostr); else writeProbablyDoubleQuotedString(name, ostr); @@ -242,7 +248,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const } case IdentifierQuotingStyle::BackticksMySQL: { - if (always_quote_identifiers) + if (force_quoting || always_quote_identifiers) writeBackQuotedStringMySQL(name, ostr); else writeProbablyBackQuotedStringMySQL(name, ostr); diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 777265aaa7c..c8fb93d3bf6 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -237,7 +237,8 @@ public: { } - void writeIdentifier(const String & name) const; + // If `force_quoting` is true, `name` is always quoted regardless of `always_quote_identifiers` + void writeIdentifier(const String & name, bool force_quoting = false) const; }; /// State. For example, a set of nodes can be remembered, which we already walk through. @@ -278,7 +279,13 @@ public: /// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied. /// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience. - String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names, IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks) const; + String formatWithPossiblyHidingSensitiveData( + size_t max_length, + bool one_line, + bool show_secrets, + bool print_pretty_type_names, + bool always_quote_identifiers, + IdentifierQuotingStyle identifier_quoting_style) const; /** formatForLogging and formatForErrorMessage always hide secrets. This inconsistent * behaviour is due to the fact such functions are called from Client which knows nothing about @@ -287,12 +294,12 @@ public: */ String formatForLogging(size_t max_length = 0) const { - return formatWithPossiblyHidingSensitiveData(max_length, true, false, false); + return formatWithPossiblyHidingSensitiveData(max_length, true, false, false, false, IdentifierQuotingStyle::Backticks); } String formatForErrorMessage() const { - return formatWithPossiblyHidingSensitiveData(0, true, false, false); + return formatWithPossiblyHidingSensitiveData(0, true, false, false, false, IdentifierQuotingStyle::Backticks); } virtual bool hasSecretParts() const { return childrenHaveSecretParts(); } From 32e995acbb1ac613217190561fbdda976c4386b5 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 073/259] Update settings change history version --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index ac7100d3706..982cd0389a9 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,7 +57,7 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.12", + {"24.13", { {"output_format_always_quote_identifiers", false, false, "New setting."}, {"output_format_identifier_quoting_style", "Backticks", "Backticks", "New setting."} From 8e6de7e767e3db62efab622a7cf05e81da5d7866 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 074/259] 1) Fix: move new setting to 24.9 key. 2) Fix: fource quoting dictionary attribute names --- src/Core/SettingsChangesHistory.cpp | 8 ++------ src/Parsers/ASTDictionaryAttributeDeclaration.cpp | 2 +- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 982cd0389a9..258065dcfd4 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,12 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.13", - { - {"output_format_always_quote_identifiers", false, false, "New setting."}, - {"output_format_identifier_quoting_style", "Backticks", "Backticks", "New setting."} - } - }, {"24.12", { } @@ -81,6 +75,8 @@ static std::initializer_list Date: Mon, 26 Aug 2024 18:43:21 +0200 Subject: [PATCH 075/259] increase timeout for flaky check --- tests/ci/ci_config.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 5453bffd9c6..5de572bc0cb 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -424,6 +424,7 @@ class CI: pr_only=True, # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf # reference_job_name=JobNames.INTEGRATION_TEST_TSAN, + timeout=4*3600 # to be able to process many updated tests ), JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], From 34d13bd7467610daec7004640e1ab07da8a03f81 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 26 Aug 2024 19:33:04 +0200 Subject: [PATCH 076/259] Update ci_config.py --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 5de572bc0cb..0d5f29d705f 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -424,7 +424,7 @@ class CI: pr_only=True, # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf # reference_job_name=JobNames.INTEGRATION_TEST_TSAN, - timeout=4*3600 # to be able to process many updated tests + timeout=4 * 3600 # to be able to process many updated tests ), JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], From f9f960a5ed65b479257fff70ce2ef7234c61406d Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 26 Aug 2024 22:04:54 +0200 Subject: [PATCH 077/259] Update ci_config.py --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 0d5f29d705f..e594168ece3 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -424,7 +424,7 @@ class CI: pr_only=True, # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf # reference_job_name=JobNames.INTEGRATION_TEST_TSAN, - timeout=4 * 3600 # to be able to process many updated tests + timeout=4 * 3600, # to be able to process many updated tests ), JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], From 69cf54192c3e4a74db645f48afad90f52dd2da1e Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Tue, 27 Aug 2024 09:33:45 +0000 Subject: [PATCH 078/259] 1) Add tests for the identifier quoting style setting. 2) Update uni-test for table overrides --- src/Parsers/ASTColumnDeclaration.cpp | 2 +- .../ASTDictionaryAttributeDeclaration.cpp | 2 +- src/Parsers/IAST.cpp | 38 +- src/Parsers/IAST.h | 6 +- src/Parsers/tests/gtest_Parser.cpp | 2 +- ..._format_identifier_quoting_style.reference | 40 +++ ...output_format_identifier_quoting_style.sql | 328 ++++++++++++++++++ 7 files changed, 408 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/03230_output_format_identifier_quoting_style.reference create mode 100644 tests/queries/0_stateless/03230_output_format_identifier_quoting_style.sql diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 7b5ea49b63d..d7728462df3 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -67,7 +67,7 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo frame.need_parens = false; /// We have to always quote column names to avoid ambiguity with INDEX and other declarations in CREATE query. - format_settings.writeIdentifier(name, true); + format_settings.quoteIdentifier(name); if (type) { diff --git a/src/Parsers/ASTDictionaryAttributeDeclaration.cpp b/src/Parsers/ASTDictionaryAttributeDeclaration.cpp index 070cc6f222a..a600987dc45 100644 --- a/src/Parsers/ASTDictionaryAttributeDeclaration.cpp +++ b/src/Parsers/ASTDictionaryAttributeDeclaration.cpp @@ -35,7 +35,7 @@ void ASTDictionaryAttributeDeclaration::formatImpl(const FormatSettings & settin { frame.need_parens = false; - settings.writeIdentifier(name, true); + settings.quoteIdentifier(name); if (type) { diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 858630cef2e..54b11f2888e 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -217,13 +217,13 @@ String IAST::getColumnNameWithoutAlias() const } -void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoting) const +void IAST::FormatSettings::writeIdentifier(const String & name) const { switch (identifier_quoting_style) { case IdentifierQuotingStyle::None: { - if (force_quoting || always_quote_identifiers) + if (always_quote_identifiers) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incompatible arguments: always_quote_identifiers = true && " "identifier_quoting_style == IdentifierQuotingStyle::None"); @@ -232,7 +232,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoti } case IdentifierQuotingStyle::Backticks: { - if (force_quoting || always_quote_identifiers) + if (always_quote_identifiers) writeBackQuotedString(name, ostr); else writeProbablyBackQuotedString(name, ostr); @@ -240,7 +240,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoti } case IdentifierQuotingStyle::DoubleQuotes: { - if (force_quoting || always_quote_identifiers) + if (always_quote_identifiers) writeDoubleQuotedString(name, ostr); else writeProbablyDoubleQuotedString(name, ostr); @@ -248,7 +248,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoti } case IdentifierQuotingStyle::BackticksMySQL: { - if (force_quoting || always_quote_identifiers) + if (always_quote_identifiers) writeBackQuotedStringMySQL(name, ostr); else writeProbablyBackQuotedStringMySQL(name, ostr); @@ -257,6 +257,34 @@ void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoti } } + +void IAST::FormatSettings::quoteIdentifier(const String & name) const +{ + switch (identifier_quoting_style) + { + case IdentifierQuotingStyle::None: + { + writeBackQuotedString(name, ostr); + break; + } + case IdentifierQuotingStyle::Backticks: + { + writeBackQuotedString(name, ostr); + break; + } + case IdentifierQuotingStyle::DoubleQuotes: + { + writeDoubleQuotedString(name, ostr); + break; + } + case IdentifierQuotingStyle::BackticksMySQL: + { + writeBackQuotedStringMySQL(name, ostr); + break; + } + } +} + void IAST::dumpTree(WriteBuffer & ostr, size_t indent) const { String indent_str(indent, '-'); diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index c8fb93d3bf6..dfb6a6cbeba 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -237,8 +237,10 @@ public: { } - // If `force_quoting` is true, `name` is always quoted regardless of `always_quote_identifiers` - void writeIdentifier(const String & name, bool force_quoting = false) const; + void writeIdentifier(const String & name) const; + // Quote identifier `name` even when `always_quote_identifiers` is false. + // If `identifier_quoting_style` is `IdentifierQuotingStyle::None`, quote it with `IdentifierQuotingStyle::Backticks` + void quoteIdentifier(const String & name) const; }; /// State. For example, a set of nodes can be remembered, which we already walk through. diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index f0abc68f966..98cd9682c9c 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -243,7 +243,7 @@ INSTANTIATE_TEST_SUITE_P(ParserCreateDatabaseQuery, ParserTest, }, { "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", - "CREATE DATABASE db\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n INDEX foo `foo` TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(`_staged` = 1, 'staging', toYYYYMM(`created`))\n)" + "CREATE DATABASE db\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n INDEX `foo` `foo` TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(`_staged` = 1, 'staging', toYYYYMM(`created`))\n)" }, { "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", diff --git a/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.reference b/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.reference new file mode 100644 index 00000000000..c563617a01c --- /dev/null +++ b/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.reference @@ -0,0 +1,40 @@ +CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE `default`.`uk_price_paid`\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX `county_index` `county` TYPE set(10) GRANULARITY 1,\n PROJECTION `town_date_projection`\n (\n SELECT \n `town`,\n `date`,\n `price`\n ORDER BY \n `town`,\n `date`\n ),\n PROJECTION `handy_aggs_projection`\n (\n SELECT \n avg(`price`),\n max(`price`),\n sum(`price`)\n GROUP BY `town`\n )\n)\nENGINE = MergeTree\nORDER BY (`postcode1`, `postcode2`, `date`)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW `default`.`prices_by_year_view` TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n `price`,\n `date`,\n `addr1`,\n `addr2`,\n `street`,\n `town`,\n `district`,\n `county`\nFROM `default`.`uk_price_paid` +CREATE TABLE `default`.`uk_prices_aggs_dest`\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `month`\nORDER BY `month`\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW `default`.`uk_prices_aggs_view` TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nAS WITH toStartOfMonth(`date`) AS `month`\nSELECT\n `month`,\n minSimpleState(`price`) AS `min_price`,\n maxSimpleState(`price`) AS `max_price`,\n countState(`price`) AS `volume`,\n avgState(`price`) AS `avg_price`\nFROM `default`.`uk_price_paid`\nGROUP BY `month` +CREATE DICTIONARY `default`.`uk_mortgage_rates_dict`\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY `date`\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE "default"."uk_price_paid"\n(\n "price" UInt32,\n "date" Date,\n "postcode1" LowCardinality(String),\n "postcode2" LowCardinality(String),\n "type" Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n "is_new" UInt8,\n "duration" Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "locality" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String),\n INDEX "county_index" "county" TYPE set(10) GRANULARITY 1,\n PROJECTION "town_date_projection"\n (\n SELECT \n "town",\n "date",\n "price"\n ORDER BY \n "town",\n "date"\n ),\n PROJECTION "handy_aggs_projection"\n (\n SELECT \n avg("price"),\n max("price"),\n sum("price")\n GROUP BY "town"\n )\n)\nENGINE = MergeTree\nORDER BY ("postcode1", "postcode2", "date")\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW "default"."prices_by_year_view" TO default.prices_by_year_dest\n(\n "price" UInt32,\n "date" Date,\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String)\n)\nAS SELECT\n "price",\n "date",\n "addr1",\n "addr2",\n "street",\n "town",\n "district",\n "county"\nFROM "default"."uk_price_paid" +CREATE TABLE "default"."uk_prices_aggs_dest"\n(\n "month" Date,\n "min_price" SimpleAggregateFunction("min", UInt32),\n "max_price" SimpleAggregateFunction("max", UInt32),\n "volume" AggregateFunction("count", UInt32),\n "avg_price" AggregateFunction("avg", UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY "month"\nORDER BY "month"\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW "default"."uk_prices_aggs_view" TO default.uk_prices_aggs_dest\n(\n "month" Date,\n "min_price" SimpleAggregateFunction("min", UInt32),\n "max_price" SimpleAggregateFunction("max", UInt32),\n "volume" AggregateFunction("count", UInt32),\n "avg_price" AggregateFunction("avg", UInt32)\n)\nAS WITH toStartOfMonth("date") AS "month"\nSELECT\n "month",\n minSimpleState("price") AS "min_price",\n maxSimpleState("price") AS "max_price",\n countState("price") AS "volume",\n avgState("price") AS "avg_price"\nFROM "default"."uk_price_paid"\nGROUP BY "month" +CREATE DICTIONARY "default"."uk_mortgage_rates_dict"\n(\n "date" DateTime64,\n "variable" Decimal32(2),\n "fixed" Decimal32(2),\n "bank" Decimal32(2)\n)\nPRIMARY KEY "date"\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE default.uk_price_paid\n(\n "price" UInt32,\n "date" Date,\n "postcode1" LowCardinality(String),\n "postcode2" LowCardinality(String),\n "type" Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n "is_new" UInt8,\n "duration" Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "locality" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n "price" UInt32,\n "date" Date,\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n "month" Date,\n "min_price" SimpleAggregateFunction(min, UInt32),\n "max_price" SimpleAggregateFunction(max, UInt32),\n "volume" AggregateFunction(count, UInt32),\n "avg_price" AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n "month" Date,\n "min_price" SimpleAggregateFunction(min, UInt32),\n "max_price" SimpleAggregateFunction(max, UInt32),\n "volume" AggregateFunction(count, UInt32),\n "avg_price" AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n "date" DateTime64,\n "variable" Decimal32(2),\n "fixed" Decimal32(2),\n "bank" Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE `default`.`uk_price_paid`\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX `county_index` `county` TYPE set(10) GRANULARITY 1,\n PROJECTION `town_date_projection`\n (\n SELECT \n `town`,\n `date`,\n `price`\n ORDER BY \n `town`,\n `date`\n ),\n PROJECTION `handy_aggs_projection`\n (\n SELECT \n avg(`price`),\n max(`price`),\n sum(`price`)\n GROUP BY `town`\n )\n)\nENGINE = MergeTree\nORDER BY (`postcode1`, `postcode2`, `date`)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW `default`.`prices_by_year_view` TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n `price`,\n `date`,\n `addr1`,\n `addr2`,\n `street`,\n `town`,\n `district`,\n `county`\nFROM `default`.`uk_price_paid` +CREATE TABLE `default`.`uk_prices_aggs_dest`\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `month`\nORDER BY `month`\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW `default`.`uk_prices_aggs_view` TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nAS WITH toStartOfMonth(`date`) AS `month`\nSELECT\n `month`,\n minSimpleState(`price`) AS `min_price`,\n maxSimpleState(`price`) AS `max_price`,\n countState(`price`) AS `volume`,\n avgState(`price`) AS `avg_price`\nFROM `default`.`uk_price_paid`\nGROUP BY `month` +CREATE DICTIONARY `default`.`uk_mortgage_rates_dict`\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY `date`\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) diff --git a/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.sql b/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.sql new file mode 100644 index 00000000000..c500dd4e4c6 --- /dev/null +++ b/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.sql @@ -0,0 +1,328 @@ +DROP DICTIONARY IF EXISTS uk_mortgage_rates_dict; +DROP TABLE IF EXISTS uk_mortgage_rates; +DROP VIEW IF EXISTS uk_prices_aggs_view; +DROP TABLE IF EXISTS uk_prices_aggs_dest; +DROP VIEW IF EXISTS prices_by_year_view; +DROP TABLE IF EXISTS prices_by_year_dest; +DROP TABLE IF EXISTS uk_price_paid; + +-- Create tables, views, dictionaries + +CREATE TABLE uk_price_paid +( + price UInt32, + date Date, + postcode1 LowCardinality(String), + postcode2 LowCardinality(String), + type Enum('terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4, 'other' = 0), + is_new UInt8, + duration Enum('freehold' = 1, 'leasehold' = 2, 'unknown' = 0), + addr1 String, + addr2 String, + street LowCardinality(String), + locality LowCardinality(String), + town LowCardinality(String), + district LowCardinality(String), + county LowCardinality(String), + INDEX county_index county TYPE set(10) GRANULARITY 1, + PROJECTION town_date_projection + ( + SELECT + town, + date, + price + ORDER BY + town, + date + ), + PROJECTION handy_aggs_projection + ( + SELECT + avg(price), + max(price), + sum(price) + GROUP BY town + ) +) +ENGINE = MergeTree +ORDER BY (postcode1, postcode2, date); + +CREATE TABLE prices_by_year_dest ( + price UInt32, + date Date, + addr1 String, + addr2 String, + street LowCardinality(String), + town LowCardinality(String), + district LowCardinality(String), + county LowCardinality(String) +) +ENGINE = MergeTree +PRIMARY KEY (town, date) +PARTITION BY toYear(date); + +CREATE MATERIALIZED VIEW prices_by_year_view +TO prices_by_year_dest +AS + SELECT + price, + date, + addr1, + addr2, + street, + town, + district, + county + FROM uk_price_paid; + +CREATE TABLE uk_prices_aggs_dest ( + month Date, + min_price SimpleAggregateFunction(min, UInt32), + max_price SimpleAggregateFunction(max, UInt32), + volume AggregateFunction(count, UInt32), + avg_price AggregateFunction(avg, UInt32) +) +ENGINE = AggregatingMergeTree +PRIMARY KEY month; + +CREATE MATERIALIZED VIEW uk_prices_aggs_view +TO uk_prices_aggs_dest +AS + WITH + toStartOfMonth(date) AS month + SELECT + month, + minSimpleState(price) AS min_price, + maxSimpleState(price) AS max_price, + countState(price) AS volume, + avgState(price) AS avg_price + FROM uk_price_paid + GROUP BY month; + +CREATE TABLE uk_mortgage_rates ( + date DateTime64, + variable Decimal32(2), + fixed Decimal32(2), + bank Decimal32(2) +) +ENGINE Memory(); + +INSERT INTO uk_mortgage_rates VALUES ('2004-02-29', 5.02, 4.9, 4); +INSERT INTO uk_mortgage_rates VALUES ('2004-03-31', 5.11, 4.91, 4); + +CREATE DICTIONARY uk_mortgage_rates_dict ( + date DateTime64, + variable Decimal32(2), + fixed Decimal32(2), + bank Decimal32(2) +) +PRIMARY KEY date +SOURCE( + CLICKHOUSE(TABLE 'uk_mortgage_rates') +) +LAYOUT(COMPLEX_KEY_HASHED()) +LIFETIME(2628000000); + + +-- Show tables, views, dictionaries with default settings +SHOW CREATE TABLE uk_price_paid; + +SHOW CREATE VIEW prices_by_year_view; + +SHOW CREATE uk_prices_aggs_dest; + +SHOW CREATE VIEW uk_prices_aggs_view; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict; + + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='None' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=true, output_format_identifier_quoting_style='Backticks' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='Backticks' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=true, output_format_identifier_quoting_style='DoubleQuotes' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='DoubleQuotes' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=true, output_format_identifier_quoting_style='BackticksMySQL' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='BackticksMySQL' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +DROP DICTIONARY uk_mortgage_rates_dict; +DROP TABLE uk_mortgage_rates; +DROP VIEW uk_prices_aggs_view; +DROP TABLE uk_prices_aggs_dest; +DROP VIEW prices_by_year_view; +DROP TABLE prices_by_year_dest; +DROP TABLE uk_price_paid; From 1523df6ec310ae7a431b32ef084c50e4377d11ec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 27 Aug 2024 10:31:11 +0000 Subject: [PATCH 079/259] Ignore `MODIFY_COLUMN` commands without column type when parsing mutation commands --- src/Storages/MutationCommands.cpp | 5 ++++- ...xed_mutations_and_remove_materialized.reference | 8 ++++++++ ...ith_mixed_mutations_and_remove_materialized.sql | 14 ++++++++++++++ 3 files changed, 26 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference create mode 100644 tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index f736c863eee..f5ccc80f1d8 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -115,7 +115,10 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.column_name = getIdentifierName(command->column); return res; } - else if (parse_alter_commands && command->type == ASTAlterCommand::MODIFY_COLUMN) + /// MODIFY COLUMN x REMOVE MATERIALIZED is a valid alter command, but doesn't have any specified column type, thus no mutation is needed + else if ( + parse_alter_commands && command->type == ASTAlterCommand::MODIFY_COLUMN && command->col_decl + && command->col_decl->as().type) { MutationCommand res; res.ast = command->ptr(); diff --git a/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference new file mode 100644 index 00000000000..9166a82f472 --- /dev/null +++ b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference @@ -0,0 +1,8 @@ +BEFORE a_r1 x String +BEFORE a_r1 y String MATERIALIZED \'str\' +BEFORE a_r2 x String +BEFORE a_r2 y String MATERIALIZED \'str\' +AFTER a_r1 x String +AFTER a_r1 y String +AFTER a_r2 x String +AFTER a_r2 y String diff --git a/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql new file mode 100644 index 00000000000..3c43b9a8521 --- /dev/null +++ b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS a_r1 SYNC; +DROP TABLE IF EXISTS a_r2 SYNC; +CREATE TABLE a_r1 (x String, y String MATERIALIZED 'str') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/a', 'r1') ORDER BY x; +CREATE TABLE a_r2 (x String, y String MATERIALIZED 'str') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/a', 'r2') ORDER BY x; + +INSERT INTO a_r1 SELECT toString(number) FROM numbers(100); +SELECT 'BEFORE', table, name, type, default_kind, default_expression FROM system.columns WHERE database = currentDatabase() AND table LIKE 'a\_r%' ORDER BY table, name; + +ALTER TABLE a_r1 + ADD INDEX IF NOT EXISTS some_index x TYPE set(16) GRANULARITY 1, + MODIFY COLUMN y REMOVE MATERIALIZED +SETTINGS alter_sync = 2, mutations_sync = 2; + +SELECT 'AFTER', table, name, type, default_kind, default_expression FROM system.columns WHERE database = currentDatabase() AND table LIKE 'a\_r%' ORDER BY table, name; From 793b549291db336be2596c2fe7382d7b7943b547 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 27 Aug 2024 10:31:22 +0000 Subject: [PATCH 080/259] Fix typo in docs --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7bd36ccd00f..816d1caa632 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3226,7 +3226,7 @@ Default value: `0`. ## lightweight_deletes_sync {#lightweight_deletes_sync} -The same as 'mutation_sync', but controls only execution of lightweight deletes. +The same as [`mutations_sync`](#mutations_sync), but controls only execution of lightweight deletes. Possible values: From c44cc71310f982bd9d4101ee02001f5590971f17 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 27 Aug 2024 11:32:05 +0000 Subject: [PATCH 081/259] forgot several files --- tests/integration/helpers/postgres_utility.py | 14 +++++++------- .../test.py | 8 ++++---- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 468c3b3bb63..4bb68284314 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -245,9 +245,9 @@ class PostgresManager: ): postgres_database = self.database_or_default(postgres_database) self.created_materialized_postgres_db_list.add(materialized_database) - self.instance.query(f"DROP DATABASE IF EXISTS {materialized_database}") + self.instance.query(f"DROP DATABASE IF EXISTS `{materialized_database}`") - create_query = f"CREATE DATABASE {materialized_database} ENGINE = MaterializedPostgreSQL('{ip}:{port}', '{postgres_database}', '{user}', '{password}')" + create_query = f"CREATE DATABASE `{materialized_database}` ENGINE = MaterializedPostgreSQL('{ip}:{port}', '{postgres_database}', '{user}', '{password}')" if len(settings) > 0: create_query += " SETTINGS " for i in range(len(settings)): @@ -259,7 +259,7 @@ class PostgresManager: assert materialized_database in self.instance.query("SHOW DATABASES") def drop_materialized_db(self, materialized_database="test_database"): - self.instance.query(f"DROP DATABASE IF EXISTS {materialized_database} SYNC") + self.instance.query(f"DROP DATABASE IF EXISTS `{materialized_database}` SYNC") if materialized_database in self.created_materialized_postgres_db_list: self.created_materialized_postgres_db_list.remove(materialized_database) @@ -329,11 +329,11 @@ def assert_nested_table_is_created( table = schema_name + "." + table_name print(f"Checking table {table} exists in {materialized_database}") - database_tables = instance.query(f"SHOW TABLES FROM {materialized_database}") + database_tables = instance.query(f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'") while table not in database_tables: time.sleep(0.2) - database_tables = instance.query(f"SHOW TABLES FROM {materialized_database}") + database_tables = instance.query(f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'") assert table in database_tables @@ -366,9 +366,9 @@ def check_tables_are_synchronized( table_path = "" if len(schema_name) == 0: - table_path = f"{materialized_database}.{table_name}" + table_path = f"`{materialized_database}`.`{table_name}`" else: - table_path = f"{materialized_database}.`{schema_name}.{table_name}`" + table_path = f"`{materialized_database}`.`{schema_name}.{table_name}`" print(f"Checking table is synchronized: {table_path}") result_query = f"select * from {table_path} order by {order_by};" diff --git a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py index 5599786026f..280985b55b3 100644 --- a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py +++ b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py @@ -356,7 +356,7 @@ def test_change_queries_frequency(): max_values_size=1000, array_size_range=[10, 50], ) - + node.query("SYSTEM FLUSH LOGS") select_log_query = f"SELECT countIf(timeout_milliseconds - {min_ms} < 25) FROM (SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 10)" res = node.query(select_log_query) assert int(res) >= 5 @@ -364,12 +364,12 @@ def test_change_queries_frequency(): _insert_queries_in_parallel( table_name, settings, - thread_num=20, - tasks=2000, + thread_num=10, + tasks=1000, max_values_size=1000, array_size_range=[10, 15], ) - + node.query("SYSTEM FLUSH LOGS") select_log_query = f"SELECT countIf({max_ms} - timeout_milliseconds < 100) FROM (SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 10)" res = node.query(select_log_query) assert int(res) >= 5 From cc701c92b4d870a4a2028cccbc46c2cfda8c405d Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 27 Aug 2024 15:05:05 +0200 Subject: [PATCH 082/259] Add plan_step_name and plan_step_description columns to system.processors_profile_log --- src/Interpreters/ProcessorsProfileLog.cpp | 4 ++++ src/Interpreters/ProcessorsProfileLog.h | 2 ++ src/Processors/IProcessor.cpp | 12 ++++++++++++ src/Processors/IProcessor.h | 10 +++++----- 4 files changed, 23 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ProcessorsProfileLog.cpp b/src/Interpreters/ProcessorsProfileLog.cpp index 7dec2a3163a..8a646b5d0e7 100644 --- a/src/Interpreters/ProcessorsProfileLog.cpp +++ b/src/Interpreters/ProcessorsProfileLog.cpp @@ -30,6 +30,8 @@ ColumnsDescription ProcessorProfileLogElement::getColumnsDescription() {"id", std::make_shared(), "ID of processor."}, {"parent_ids", std::make_shared(std::make_shared()), "Parent processors IDs."}, {"plan_step", std::make_shared(), "ID of the query plan step which created this processor. The value is zero if the processor was not added from any step."}, + {"plan_step_name", std::make_shared(), "Name of the query plan step which created this processor. The value is empty if the processor was not added from any step."}, + {"plan_step_description", std::make_shared(), "Description of the query plan step which created this processor. The value is empty if the processor was not added from any step."}, {"plan_group", std::make_shared(), "Group of the processor if it was created by query plan step. A group is a logical partitioning of processors added from the same query plan step. Group is used only for beautifying the result of EXPLAIN PIPELINE result."}, {"initial_query_id", std::make_shared(), "ID of the initial query (for distributed query execution)."}, @@ -64,6 +66,8 @@ void ProcessorProfileLogElement::appendToBlock(MutableColumns & columns) const } columns[i++]->insert(plan_step); + columns[i++]->insert(plan_step_name); + columns[i++]->insert(plan_step_description); columns[i++]->insert(plan_group); columns[i++]->insertData(initial_query_id.data(), initial_query_id.size()); columns[i++]->insertData(query_id.data(), query_id.size()); diff --git a/src/Interpreters/ProcessorsProfileLog.h b/src/Interpreters/ProcessorsProfileLog.h index abece2604f2..fbf52f45f56 100644 --- a/src/Interpreters/ProcessorsProfileLog.h +++ b/src/Interpreters/ProcessorsProfileLog.h @@ -19,6 +19,8 @@ struct ProcessorProfileLogElement UInt64 plan_step{}; UInt64 plan_group{}; + String plan_step_name; + String plan_step_description; String initial_query_id; String query_id; diff --git a/src/Processors/IProcessor.cpp b/src/Processors/IProcessor.cpp index fc595a7b565..d9bd5074c09 100644 --- a/src/Processors/IProcessor.cpp +++ b/src/Processors/IProcessor.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -9,6 +10,17 @@ namespace DB { +void IProcessor::setQueryPlanStep(IQueryPlanStep * step, size_t group) +{ + query_plan_step = step; + query_plan_step_group = group; + if (step) + { + plan_step_name = step->getName(); + plan_step_description = step->getStepDescription(); + } +} + void IProcessor::cancel() noexcept { diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 02b8a3daa28..fd75eb530aa 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -311,14 +311,12 @@ public: constexpr static size_t NO_STREAM = std::numeric_limits::max(); /// Step of QueryPlan from which processor was created. - void setQueryPlanStep(IQueryPlanStep * step, size_t group = 0) - { - query_plan_step = step; - query_plan_step_group = group; - } + void setQueryPlanStep(IQueryPlanStep * step, size_t group = 0); IQueryPlanStep * getQueryPlanStep() const { return query_plan_step; } size_t getQueryPlanStepGroup() const { return query_plan_step_group; } + const String & getPlanStepName() const { return plan_step_name; } + const String & getPlanStepDescription() const { return plan_step_description; } uint64_t getElapsedNs() const { return elapsed_ns; } uint64_t getInputWaitElapsedNs() const { return input_wait_elapsed_ns; } @@ -410,6 +408,8 @@ private: IQueryPlanStep * query_plan_step = nullptr; size_t query_plan_step_group = 0; + String plan_step_name; + String plan_step_description; }; From 2a68ad8704d62b179a123a8fe3355d58450f2b99 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 27 Aug 2024 15:12:01 +0200 Subject: [PATCH 083/259] Fill plan_step_name and plan_step_description --- src/Interpreters/executeQuery.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index decc16a3704..d8ceae77d13 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -471,6 +471,8 @@ void logQueryFinish( processor_elem.parent_ids = std::move(parents); processor_elem.plan_step = reinterpret_cast(processor->getQueryPlanStep()); + processor_elem.plan_step_name = processor->getPlanStepName(); + processor_elem.plan_step_description = processor->getPlanStepDescription(); processor_elem.plan_group = processor->getQueryPlanStepGroup(); processor_elem.processor_name = processor->getName(); From b8d5a82f975e4da9be8dff890b2d0caf3832a1b3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 27 Aug 2024 14:31:37 +0000 Subject: [PATCH 084/259] Fix test --- ..._mutations_and_remove_materialized.reference | 12 ++++-------- ..._mixed_mutations_and_remove_materialized.sql | 17 ++++++++--------- 2 files changed, 12 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference index 9166a82f472..66cf66367c1 100644 --- a/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference +++ b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.reference @@ -1,8 +1,4 @@ -BEFORE a_r1 x String -BEFORE a_r1 y String MATERIALIZED \'str\' -BEFORE a_r2 x String -BEFORE a_r2 y String MATERIALIZED \'str\' -AFTER a_r1 x String -AFTER a_r1 y String -AFTER a_r2 x String -AFTER a_r2 y String +BEFORE a x String +BEFORE a y String MATERIALIZED \'str\' +AFTER a x String +AFTER a y String diff --git a/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql index 3c43b9a8521..d8ac3280792 100644 --- a/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql +++ b/tests/queries/0_stateless/03230_alter_with_mixed_mutations_and_remove_materialized.sql @@ -1,14 +1,13 @@ -DROP TABLE IF EXISTS a_r1 SYNC; -DROP TABLE IF EXISTS a_r2 SYNC; -CREATE TABLE a_r1 (x String, y String MATERIALIZED 'str') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/a', 'r1') ORDER BY x; -CREATE TABLE a_r2 (x String, y String MATERIALIZED 'str') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/a', 'r2') ORDER BY x; +DROP TABLE IF EXISTS a SYNC; +CREATE TABLE a (x String, y String MATERIALIZED 'str') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/a', 'r1') ORDER BY x; -INSERT INTO a_r1 SELECT toString(number) FROM numbers(100); -SELECT 'BEFORE', table, name, type, default_kind, default_expression FROM system.columns WHERE database = currentDatabase() AND table LIKE 'a\_r%' ORDER BY table, name; +INSERT INTO a SELECT toString(number) FROM numbers(100); +SELECT 'BEFORE', table, name, type, default_kind, default_expression FROM system.columns WHERE database = currentDatabase() AND table = 'a' ORDER BY table, name; -ALTER TABLE a_r1 - ADD INDEX IF NOT EXISTS some_index x TYPE set(16) GRANULARITY 1, +-- DROP INDEX is important to make the mutation not a pure metadata mutation +ALTER TABLE a + DROP INDEX IF EXISTS some_index, MODIFY COLUMN y REMOVE MATERIALIZED SETTINGS alter_sync = 2, mutations_sync = 2; -SELECT 'AFTER', table, name, type, default_kind, default_expression FROM system.columns WHERE database = currentDatabase() AND table LIKE 'a\_r%' ORDER BY table, name; +SELECT 'AFTER', table, name, type, default_kind, default_expression FROM system.columns WHERE database = currentDatabase() AND table = 'a' ORDER BY table, name; From 9ae0d5161349bd89a9a185a8f60aaab0e9357f34 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Tue, 27 Aug 2024 15:02:02 +0000 Subject: [PATCH 085/259] Reformat the code, pass variable names in comments when calling --- src/Core/ExternalTable.cpp | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 1f8e63db72e..7fe23f844b6 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -18,11 +18,11 @@ #include #include #include +#include #include #include #include #include -#include "Parsers/IdentifierQuotingStyle.h" namespace DB @@ -88,7 +88,13 @@ void BaseExternalTable::parseStructureFromStructureField(const std::string & arg if (column) structure.emplace_back( column->name, - column->type->formatWithPossiblyHidingSensitiveData(0, true, true, false, false, IdentifierQuotingStyle::Backticks)); + column->type->formatWithPossiblyHidingSensitiveData( + /*max_length=*/0, + /*one_line=*/true, + /*show_secrets=*/true, + /*print_pretty_type_names=*/false, + /*always_quote_identifiers=*/false, + /*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks)); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: expected column definition, got {}", child->formatForErrorMessage()); } @@ -108,7 +114,12 @@ void BaseExternalTable::parseStructureFromTypesField(const std::string & argumen structure.emplace_back( "_" + toString(i + 1), type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData( - 0, true, true, false, false, IdentifierQuotingStyle::Backticks)); + /*max_length=*/0, + /*one_line=*/true, + /*show_secrets=*/true, + /*print_pretty_type_names=*/false, + /*always_quote_identifiers=*/false, + /*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks)); } void BaseExternalTable::initSampleBlock() From aed953bf943eb57feef7ae5a0f2ebf9336543f88 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 27 Aug 2024 16:00:33 +0000 Subject: [PATCH 086/259] black --- tests/integration/helpers/postgres_utility.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 4bb68284314..c61c535bd62 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -329,11 +329,15 @@ def assert_nested_table_is_created( table = schema_name + "." + table_name print(f"Checking table {table} exists in {materialized_database}") - database_tables = instance.query(f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'") + database_tables = instance.query( + f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'" + ) while table not in database_tables: time.sleep(0.2) - database_tables = instance.query(f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'") + database_tables = instance.query( + f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'" + ) assert table in database_tables From e0b9c0f14f6a7e817fddc6eedc5f6bd69b6d9f83 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Wed, 28 Aug 2024 03:47:15 +0000 Subject: [PATCH 087/259] Remove settings update in ASTTableOverride::formatImpl --- src/Parsers/ASTTableOverrides.cpp | 4 +- src/Parsers/tests/gtest_Parser.cpp | 98 +++++++++++++++--------------- 2 files changed, 50 insertions(+), 52 deletions(-) diff --git a/src/Parsers/ASTTableOverrides.cpp b/src/Parsers/ASTTableOverrides.cpp index ccb485f6c69..8352e68b156 100644 --- a/src/Parsers/ASTTableOverrides.cpp +++ b/src/Parsers/ASTTableOverrides.cpp @@ -22,10 +22,8 @@ ASTPtr ASTTableOverride::clone() const return res; } -void ASTTableOverride::formatImpl(const FormatSettings & settings_, FormatState & state, FormatStateStacked frame) const +void ASTTableOverride::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - FormatSettings settings = settings_; - settings.always_quote_identifiers = true; String nl_or_nothing = settings.one_line ? "" : "\n"; String nl_or_ws = settings.one_line ? " " : "\n"; String hl_keyword = settings.hilite ? hilite_keyword : ""; diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 98cd9682c9c..1b50d4480d7 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -225,55 +225,55 @@ INSTANTIATE_TEST_SUITE_P(ParserCreateDatabaseQuery, ParserTest, ::testing::Combine( ::testing::Values(std::make_shared()), ::testing::ValuesIn(std::initializer_list{ - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw')", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')" - }, - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl`\n(PARTITION BY toYYYYMM(created))", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE `tbl`\n(\n PARTITION BY toYYYYMM(`created`)\n)" - }, - { - "CREATE DATABASE db ENGINE=Foo TABLE OVERRIDE `tbl` (), TABLE OVERRIDE a (COLUMNS (_created DateTime MATERIALIZED now())), TABLE OVERRIDE b (PARTITION BY rand())", - "CREATE DATABASE db\nENGINE = Foo\nTABLE OVERRIDE `tbl`\n(\n\n),\nTABLE OVERRIDE `a`\n(\n COLUMNS\n (\n `_created` DateTime MATERIALIZED now()\n )\n),\nTABLE OVERRIDE `b`\n(\n PARTITION BY rand()\n)" - }, - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE tbl (COLUMNS (id UUID) PARTITION BY toYYYYMM(created))", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n `id` UUID\n )\n PARTITION BY toYYYYMM(`created`)\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", - "CREATE DATABASE db\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n INDEX `foo` `foo` TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(`_staged` = 1, 'staging', toYYYYMM(`created`))\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", - "CREATE DATABASE db\nTABLE OVERRIDE `t1`\n(\n TTL `inserted` + toIntervalMonth(1)\n),\nTABLE OVERRIDE `t2`\n(\n TTL `inserted` + toIntervalMonth(2)\n)" - }, - { - "CREATE DATABASE db ENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw') SETTINGS allows_query_when_mysql_lost = 1 TABLE OVERRIDE tab3 (COLUMNS (_staged UInt8 MATERIALIZED 1) PARTITION BY (c3) TTL c3 + INTERVAL 10 minute), TABLE OVERRIDE tab5 (PARTITION BY (c3) TTL c3 + INTERVAL 10 minute)", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw')\nSETTINGS allows_query_when_mysql_lost = 1\nTABLE OVERRIDE `tab3`\n(\n COLUMNS\n (\n `_staged` UInt8 MATERIALIZED 1\n )\n PARTITION BY `c3`\n TTL `c3` + toIntervalMinute(10)\n),\nTABLE OVERRIDE `tab5`\n(\n PARTITION BY `c3`\n TTL `c3` + toIntervalMinute(10)\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE tbl (PARTITION BY toYYYYMM(created) COLUMNS (created DateTime CODEC(Delta)))", - "CREATE DATABASE db\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n `created` DateTime CODEC(Delta)\n )\n PARTITION BY toYYYYMM(`created`)\n)" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 TABLE OVERRIDE a (ORDER BY (id, version))", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE `a`\n(\n ORDER BY (`id`, `version`)\n)" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 COMMENT 'db comment' TABLE OVERRIDE a (ORDER BY (id, version))", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE `a`\n(\n ORDER BY (`id`, `version`)\n)\nCOMMENT 'db comment'" - } -}))); + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw')", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')" + }, + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl`\n(PARTITION BY toYYYYMM(created))", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db ENGINE=Foo TABLE OVERRIDE `tbl` (), TABLE OVERRIDE a (COLUMNS (_created DateTime MATERIALIZED now())), TABLE OVERRIDE b (PARTITION BY rand())", + "CREATE DATABASE db\nENGINE = Foo\nTABLE OVERRIDE tbl\n(\n\n),\nTABLE OVERRIDE a\n(\n COLUMNS\n (\n `_created` DateTime MATERIALIZED now()\n )\n),\nTABLE OVERRIDE b\n(\n PARTITION BY rand()\n)" + }, + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE tbl (COLUMNS (id UUID) PARTITION BY toYYYYMM(created))", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `id` UUID\n )\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", + "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n INDEX foo foo TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created))\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", + "CREATE DATABASE db\nTABLE OVERRIDE t1\n(\n TTL inserted + toIntervalMonth(1)\n),\nTABLE OVERRIDE t2\n(\n TTL inserted + toIntervalMonth(2)\n)" + }, + { + "CREATE DATABASE db ENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw') SETTINGS allows_query_when_mysql_lost = 1 TABLE OVERRIDE tab3 (COLUMNS (_staged UInt8 MATERIALIZED 1) PARTITION BY (c3) TTL c3 + INTERVAL 10 minute), TABLE OVERRIDE tab5 (PARTITION BY (c3) TTL c3 + INTERVAL 10 minute)", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw')\nSETTINGS allows_query_when_mysql_lost = 1\nTABLE OVERRIDE tab3\n(\n COLUMNS\n (\n `_staged` UInt8 MATERIALIZED 1\n )\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n),\nTABLE OVERRIDE tab5\n(\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE tbl (PARTITION BY toYYYYMM(created) COLUMNS (created DateTime CODEC(Delta)))", + "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `created` DateTime CODEC(Delta)\n )\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 TABLE OVERRIDE a (ORDER BY (id, version))", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 COMMENT 'db comment' TABLE OVERRIDE a (ORDER BY (id, version))", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)\nCOMMENT 'db comment'" + } + }))); INSTANTIATE_TEST_SUITE_P(ParserCreateUserQuery, ParserTest, ::testing::Combine( From d218b04fba8422049ec2a6f73bd646c55cd50c88 Mon Sep 17 00:00:00 2001 From: shiyer7474 Date: Wed, 28 Aug 2024 11:52:51 +0000 Subject: [PATCH 088/259] 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 2c6563ee691a7c1a0cee61af815ba037393957d2 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Wed, 28 Aug 2024 12:37:28 +0000 Subject: [PATCH 089/259] Fix test in tests/integration/test_materialized_mysql_database/materialized_with_ddl.py --- .../test_materialized_mysql_database/materialized_with_ddl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 86000799ae4..9a99f0c9aa8 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -2353,7 +2353,7 @@ def table_overrides(clickhouse_node, mysql_node, service_name): ) check_query(clickhouse_node, "SELECT count() FROM table_overrides.t1", "1001\n") show_db = clickhouse_node.query("SHOW CREATE DATABASE table_overrides") - assert "TABLE OVERRIDE `t1`\\n(\\n\\n)" in show_db, show_db + assert "TABLE OVERRIDE t1\\n(\\n\\n)" in show_db, show_db clickhouse_node.query("DROP DATABASE IF EXISTS table_overrides") mysql_node.query("DROP DATABASE IF EXISTS table_overrides") From 3c6b2a48c379a9aa6f2bb2a6d99c235b27f2c4c9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Aug 2024 13:05:43 +0000 Subject: [PATCH 090/259] Prevent specifying properties in `MODIFY COLUMN` queries when using `REMOVE`/`RESET SETTING`/`MODIFY SETTING` --- src/Parsers/ParserAlterQuery.cpp | 34 +++- ...rties_before_remove_modify_reset.reference | 13 ++ ..._properties_before_remove_modify_reset.sql | 169 ++++++++++++++++++ 3 files changed, 212 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.reference create mode 100644 tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 73fd563faf6..3920f09918a 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -1,6 +1,8 @@ -#include -#include #include + +#include +#include +#include #include #include #include @@ -9,14 +11,19 @@ #include #include #include -#include -#include +#include #include +#include namespace DB { +namespace ErrorCodes +{ +extern const int SYNTAX_ERROR; +} + bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { auto command = std::make_shared(); @@ -725,8 +732,23 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_modify_col_decl.parse(pos, command_col_decl, expected)) return false; + auto check_no_type = [&](const std::string_view keyword) + { + if (!command_col_decl) + return; + const auto & column_decl = command_col_decl->as(); + + if (!column_decl.children.empty() || column_decl.null_modifier.has_value() || !column_decl.default_specifier.empty() + || column_decl.ephemeral_default || column_decl.primary_key_specifier) + { + throw Exception(ErrorCodes::SYNTAX_ERROR, "Cannot specify column properties before '{}'", keyword); + } + }; + if (s_remove.ignore(pos, expected)) { + check_no_type(s_remove.getName()); + if (s_default.ignore(pos, expected)) command->remove_property = toStringView(Keyword::DEFAULT); else if (s_materialized.ignore(pos, expected)) @@ -746,11 +768,15 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (s_modify_setting.ignore(pos, expected)) { + check_no_type(s_modify_setting.getName()); + if (!parser_settings.parse(pos, command_settings_changes, expected)) return false; } else if (s_reset_setting.ignore(pos, expected)) { + check_no_type(s_reset_setting.getName()); + if (!parser_reset_setting.parse(pos, command_settings_resets, expected)) return false; } diff --git a/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.reference b/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.reference new file mode 100644 index 00000000000..60c67ceac92 --- /dev/null +++ b/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.reference @@ -0,0 +1,13 @@ +REMOVE +The same, but with type +MODIFY SETTING +The same, but with type +RESET SETTING +The same, but with type +All the above, but on server side +REMOVE +The same, but with type +MODIFY SETTING +The same, but with type +RESET SETTING +The same, but with type diff --git a/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql b/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql new file mode 100644 index 00000000000..13ad11bb139 --- /dev/null +++ b/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql @@ -0,0 +1,169 @@ +DROP TABLE IF EXISTS a SYNC; +CREATE TABLE a (x Int64, y Int64 MATERIALIZED 1 SETTINGS (max_compress_block_size = 30000)) ENGINE = MergeTree ORDER BY x; + + +SELECT 'REMOVE'; +ALTER TABLE a MODIFY COLUMN y Int64 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y NOT NULL REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y DEFAULT 2 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y COMMENT 5 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y TTL toDate('2025-01-01') + toIntervalDay(x) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y COLLATE binary REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y SETTINGS (max_compress_block_size = 20000) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y PRIMARY KEY REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } + +SELECT 'The same, but with type'; +ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 COMMENT 5 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate('2025-01-01') + toIntervalDay(x) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (max_compress_block_size = 20000) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } + +SELECT 'MODIFY SETTING'; +ALTER TABLE a MODIFY COLUMN y Int64 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y NOT NULL MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y COMMENT 5 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y TTL toDate('2025-01-01') + toIntervalDay(x) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y COLLATE binary MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } + +SELECT 'The same, but with type'; +ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 COMMENT 5 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate('2025-01-01') + toIntervalDay(x) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (some_setting = 2) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } + +SELECT 'RESET SETTING'; +ALTER TABLE a MODIFY COLUMN y Int64 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y NOT NULL RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y DEFAULT 2 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y COMMENT 5 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y TTL toDate('2025-01-01') + toIntervalDay(x) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y COLLATE binary RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y PRIMARY KEY RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } + +SELECT 'The same, but with type'; +ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 COMMENT 5 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate('2025-01-01') + toIntervalDay(x) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (some_setting = 2) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } +ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } + + + +SELECT 'All the above, but on server side'; + +SELECT 'REMOVE'; +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y NOT NULL REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COMMENT 5 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y TTL toDate(\'2025-01-01\') + toIntervalDay(x) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COLLATE binary REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (max_compress_block_size = 20000) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } + +SELECT 'The same, but with type'; +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COMMENT 5 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate(\'2025-01-01\') + toIntervalDay(x) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (max_compress_block_size = 20000) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } + +SELECT 'MODIFY SETTING'; +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y NOT NULL MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COMMENT 5 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y TTL toDate(\'2025-01-01\') + toIntervalDay(x) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COLLATE binary MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } + +SELECT 'The same, but with type'; +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COMMENT 5 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate(\'2025-01-01\') + toIntervalDay(x) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (some_setting = 2) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } + +SELECT 'RESET SETTING'; +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y NOT NULL RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COMMENT 5 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y TTL toDate(\'2025-01-01\') + toIntervalDay(x) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COLLATE binary RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } + +SELECT 'The same, but with type'; +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COMMENT 5 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate(\'2025-01-01\') + toIntervalDay(x) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (some_setting = 2) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } +SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } From a0fa693f0b56390a8be4b18aef03a9808fcd63db Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Aug 2024 13:20:58 +0000 Subject: [PATCH 091/259] Add safety assertion --- src/Parsers/ParserAlterQuery.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 3920f09918a..90e0d0cade0 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -736,6 +736,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected { if (!command_col_decl) return; + const auto & column_decl = command_col_decl->as(); if (!column_decl.children.empty() || column_decl.null_modifier.has_value() || !column_decl.default_specifier.empty() @@ -791,6 +792,11 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } } command->type = ASTAlterCommand::MODIFY_COLUMN; + + /// Make sure that type is not populated when REMOVE/MODIFY SETTING/RESET SETTING is used, because we wouldn't modify the type, which can be confusing + chassert( + nullptr == command_col_decl->as().type + || (command->remove_property.empty() && nullptr == command_settings_changes && nullptr == command_settings_resets)); } else if (s_modify_order_by.ignore(pos, expected)) { From 0d463b839e2e0c89e61754fc53aa904ae6728e81 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Aug 2024 13:29:41 +0000 Subject: [PATCH 092/259] Remove unused parser --- src/Parsers/ParserAlterQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 90e0d0cade0..54caf574e03 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -129,7 +129,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserCompoundIdentifier parser_name; ParserStringLiteral parser_string_literal; ParserStringAndSubstitution parser_string_and_substituion; - ParserIdentifier parser_remove_property; ParserCompoundColumnDeclaration parser_col_decl; ParserIndexDeclaration parser_idx_decl; ParserStatisticsDeclaration parser_stat_decl; From f76e6ecdaf8c2bb005e3b2712b64db4670120c34 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Aug 2024 13:30:25 +0000 Subject: [PATCH 093/259] Make check more specific --- src/Storages/MutationCommands.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index f5ccc80f1d8..8276e9de232 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes { extern const int UNKNOWN_MUTATION_COMMAND; extern const int MULTIPLE_ASSIGNMENTS_TO_COLUMN; + extern const int LOGICAL_ERROR; } @@ -115,10 +116,10 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.column_name = getIdentifierName(command->column); return res; } - /// MODIFY COLUMN x REMOVE MATERIALIZED is a valid alter command, but doesn't have any specified column type, thus no mutation is needed + /// MODIFY COLUMN x REMOVE MATERIALIZED/RESET SETTING/MODIFY SETTING is a valid alter command, but doesn't have any specified column type, + /// thus no mutation is needed else if ( - parse_alter_commands && command->type == ASTAlterCommand::MODIFY_COLUMN && command->col_decl - && command->col_decl->as().type) + parse_alter_commands && command->type == ASTAlterCommand::MODIFY_COLUMN && command->remove_property.empty() && nullptr == command->settings_changes && nullptr == command->settings_resets) { MutationCommand res; res.ast = command->ptr(); From f109c141b0cffd9c27c935774a88a16ad071bac1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Aug 2024 13:30:41 +0000 Subject: [PATCH 094/259] Add safety check --- src/Storages/MutationCommands.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index 8276e9de232..75440aeac59 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -125,6 +125,8 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.ast = command->ptr(); res.type = MutationCommand::Type::READ_COLUMN; const auto & ast_col_decl = command->col_decl->as(); + if (nullptr == ast_col_decl.type) + throw Exception(ErrorCodes::LOGICAL_ERROR, "MODIFY COLUMN mutation command doesn't specify type: {}", serializeAST(*command)); res.column_name = ast_col_decl.name; res.data_type = DataTypeFactory::instance().get(ast_col_decl.type); return res; From 5b3ca6b2b916b63d12ddb402c0726d0c2e29c1b1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 28 Aug 2024 22:14:43 +0800 Subject: [PATCH 095/259] allow unsligned arrays in arrayZip --- src/Functions/array/arrayZip.cpp | 114 +++++++++++++++++++++++-------- 1 file changed, 85 insertions(+), 29 deletions(-) diff --git a/src/Functions/array/arrayZip.cpp b/src/Functions/array/arrayZip.cpp index 6c6fff5926b..39c04264c84 100644 --- a/src/Functions/array/arrayZip.cpp +++ b/src/Functions/array/arrayZip.cpp @@ -1,7 +1,8 @@ -#include #include -#include +#include +#include #include +#include #include #include #include @@ -12,23 +13,22 @@ namespace DB namespace ErrorCodes { - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int SIZES_OF_ARRAYS_DONT_MATCH; - extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; - extern const int ILLEGAL_COLUMN; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int SIZES_OF_ARRAYS_DONT_MATCH; +extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; +extern const int ILLEGAL_COLUMN; } /// arrayZip(['a', 'b', 'c'], ['d', 'e', 'f']) = [('a', 'd'), ('b', 'e'), ('c', 'f')] +/// arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e']) = [('a', 'd'), ('b', 'e'), ('c', null)] +template class FunctionArrayZip : public IFunction { public: - static constexpr auto name = "arrayZip"; + static constexpr auto name = allow_unaligned ? "arrayZipUnaligned" : "arrayZip"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } - String getName() const override - { - return name; - } + String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } @@ -39,8 +39,11 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, - "Function {} needs at least one argument; passed {}." , getName(), arguments.size()); + throw Exception( + ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, + "Function {} needs at least one argument; passed {}.", + getName(), + arguments.size()); DataTypes arguments_types; for (size_t index = 0; index < arguments.size(); ++index) @@ -48,16 +51,24 @@ public: const DataTypeArray * array_type = checkAndGetDataType(arguments[index].type.get()); if (!array_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument {} of function {} must be array. Found {} instead.", - toString(index + 1), getName(), arguments[0].type->getName()); + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument {} of function {} must be array. Found {} instead.", + toString(index + 1), + getName(), + arguments[0].type->getName()); - arguments_types.emplace_back(array_type->getNestedType()); + auto nested_type = array_type->getNestedType(); + if constexpr (allow_unaligned) + nested_type = makeNullable(nested_type); + arguments_types.emplace_back(nested_type); } return std::make_shared(std::make_shared(arguments_types)); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr + executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override { size_t num_arguments = arguments.size(); @@ -68,12 +79,19 @@ public: { /// Constant columns cannot be inside tuple. It's only possible to have constant tuple as a whole. ColumnPtr holder = arguments[i].column->convertToFullColumnIfConst(); - const ColumnArray * column_array = checkAndGetColumn(holder.get()); - if (!column_array) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Argument {} of function {} must be array. Found column {} instead.", - i + 1, getName(), holder->getName()); + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Argument {} of function {} must be array. Found column {} instead.", + i + 1, + getName(), + holder->getName()); + + tuple_columns[i] = column_array->getDataPtr(); + + if constexpr (allow_unaligned) + tuple_columns[i] = makeNullable(tuple_columns[i]); if (i == 0) { @@ -81,23 +99,61 @@ public: } else if (!column_array->hasEqualOffsets(static_cast(*first_array_column))) { - throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, - "The argument 1 and argument {} of function {} have different array sizes", - i + 1, getName()); + if constexpr (allow_unaligned) + return executeUnaligned(static_cast(*first_array_column), *column_array, input_rows_count); + else + throw Exception( + ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, + "The argument 1 and argument {} of function {} have different array sizes", + i + 1, + getName()); } - - tuple_columns[i] = column_array->getDataPtr(); } return ColumnArray::create( - ColumnTuple::create(tuple_columns), static_cast(*first_array_column).getOffsetsPtr()); + ColumnTuple::create(std::move(tuple_columns)), static_cast(*first_array_column).getOffsetsPtr()); + } + +private: + ColumnPtr + executeUnaligned(const ColumnArray & first_array_colmn, const ColumnArray & second_array_column, size_t input_rows_count) const + { + const auto & first_data = first_array_colmn.getDataPtr(); + const auto & second_data = second_array_column.getDataPtr(); + const auto & nullable_first_data = makeNullable(first_data); + const auto & nullable_second_data = makeNullable(second_data); + auto res_first_data = nullable_first_data->cloneEmpty(); + auto res_second_data = nullable_second_data->cloneEmpty(); + auto res_offsets_column = ColumnArray::ColumnOffsets::create(input_rows_count); + auto & res_offsets = assert_cast(*res_offsets_column).getData(); + + const auto & first_offsets = first_array_colmn.getOffsets(); + const auto & second_offsets = second_array_column.getOffsets(); + for (size_t i = 0; i < input_rows_count; ++i) + { + size_t first_size = first_offsets[i] - first_offsets[i - 1]; + size_t second_size = second_offsets[i] - second_offsets[i - 1]; + + res_first_data->insertRangeFrom(*nullable_first_data, first_offsets[i - 1], first_size); + res_second_data->insertRangeFrom(*nullable_second_data, second_offsets[i - 1], second_size); + + if (first_size < second_size) + res_first_data->insertManyDefaults(second_size - first_size); + else if (first_size > second_size) + res_second_data->insertManyDefaults(first_size - second_size); + + res_offsets[i] = std::max(first_size, second_size); + } + + Columns tuple_columns{std::move(res_first_data), std::move(res_second_data)}; + return ColumnArray::create(ColumnTuple::create(std::move(tuple_columns)), std::move(res_offsets_column)); } }; REGISTER_FUNCTION(ArrayZip) { - factory.registerFunction(); + factory.registerFunction>(); + factory.registerFunction>(); } } - From a1c9cc471d5b8afbc21a4aa6f8d44791c896eb10 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 28 Aug 2024 16:51:45 +0200 Subject: [PATCH 096/259] Apply suggestions from code review MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- .../AggregateFunctionDistinctJSONPaths.cpp | 20 +++++++++---------- src/Columns/ColumnDynamic.cpp | 2 +- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp index 6100bd57515..98996aac2f7 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -49,8 +49,8 @@ struct AggregateFunctionDistinctJSONPathsData /// Iterate over paths in shared data in this row. const auto [shared_data_paths, _] = column.getSharedDataPathsAndValues(); const auto & shared_data_offsets = column.getSharedDataOffsets(); - size_t start = shared_data_offsets[static_cast(row_num) - 1]; - size_t end = shared_data_offsets[static_cast(row_num)]; + const size_t start = shared_data_offsets[static_cast(row_num) - 1]; + const size_t end = shared_data_offsets[static_cast(row_num)]; for (size_t i = start; i != end; ++i) data.insert(shared_data_paths->getDataAt(i).toString()); } @@ -137,8 +137,8 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData /// Iterate over paths om shared data in this row and decode the data types. const auto [shared_data_paths, shared_data_values] = column.getSharedDataPathsAndValues(); const auto & shared_data_offsets = column.getSharedDataOffsets(); - size_t start = shared_data_offsets[static_cast(row_num) - 1]; - size_t end = shared_data_offsets[static_cast(row_num)]; + const size_t start = shared_data_offsets[static_cast(row_num) - 1]; + const size_t end = shared_data_offsets[static_cast(row_num)]; for (size_t i = start; i != end; ++i) { auto path = shared_data_paths->getDataAt(i).toString(); @@ -146,8 +146,8 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData ReadBufferFromMemory buf(value.data, value.size); auto type = decodeDataType(buf); /// We should not have Nulls here but let's check just in case. - if (!isNothing(type)) - data[path].insert(type->getName()); + chassert(!isNothingType(type)); + data[path].insert(type->getName()); } } @@ -172,8 +172,8 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData ReadBufferFromMemory buf(value.data, value.size); auto type = decodeDataType(buf); /// We should not have Nulls here but let's check just in case. - if (!isNothing(type)) - data[path].insert(type->getName()); + chassert(!isNothingType(type)); + data[path].insert(type->getName()); } } @@ -200,7 +200,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData size_t paths_size, types_size; readVarUInt(paths_size, buf); if (paths_size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, paths_size); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size for paths (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, paths_size); data.reserve(paths_size); String path, type; @@ -209,7 +209,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData readStringBinary(path, buf); readVarUInt(types_size, buf); if (types_size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, types_size); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size for types (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, types_size); data[path].reserve(types_size); for (size_t j = 0; j != types_size; ++j) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index b5378e983c6..eb85c6b0d08 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -983,7 +983,7 @@ ColumnPtr ColumnDynamic::compress() const String ColumnDynamic::getTypeNameAt(size_t row_num) const { const auto & variant_col = getVariantColumn(); - size_t discr = variant_col.globalDiscriminatorAt(row_num); + const size_t discr = variant_col.globalDiscriminatorAt(row_num); if (discr == ColumnVariant::NULL_DISCRIMINATOR) return ""; From 0aba986372504cbdc6eb28283f51a706ffdb7b2e Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 28 Aug 2024 15:06:05 +0000 Subject: [PATCH 097/259] Address review comments --- .../reference/distinctjsonpaths.md | 41 +++++++++++++++++++ .../AggregateFunctionDistinctDynamicTypes.cpp | 2 +- .../AggregateFunctionDistinctJSONPaths.cpp | 8 ++-- src/Columns/ColumnDynamic.cpp | 2 +- src/Columns/ColumnDynamic.h | 2 +- 5 files changed, 48 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md b/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md index f916734ca44..d88b2eb024b 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md +++ b/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md @@ -82,3 +82,44 @@ Result: │ {'a':['Int64'],'b':['Array(Nullable(Int64))','String'],'c.d.e':['Date'],'c.d.f':['Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))']} │ └───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` + +**Note** + +If JSON declaration contains paths with specified types, these paths will be always included in the result of `distinctJSONPaths/distinctJSONPathsAndTypes` functions even if input data didn't have values for these paths. + +```sql +DROP TABLE IF EXISTS test_json; +CREATE TABLE test_json(json JSON(a UInt32)) ENGINE = Memory; +INSERT INTO test_json VALUES ('{"b" : "Hello"}'), ('{"b" : "World", "c" : [1, 2, 3]}'); +``` + +```sql +SELECT json FROM test_json; +``` + +```text +┌─json──────────────────────────────────┐ +│ {"a":0,"b":"Hello"} │ +│ {"a":0,"b":"World","c":["1","2","3"]} │ +└───────────────────────────────────────┘ +``` + +```sql +SELECT distinctJSONPaths(json) FROM test_json; +``` + +```text +┌─distinctJSONPaths(json)─┐ +│ ['a','b','c'] │ +└─────────────────────────┘ +``` + +```sql +SELECT distinctJSONPathsAndTypes(json) FROM test_json; +``` + +```text +┌─distinctJSONPathsAndTypes(json)────────────────────────────────┐ +│ {'a':['UInt32'],'b':['String'],'c':['Array(Nullable(Int64))']} │ +└────────────────────────────────────────────────────────────────┘ +``` \ No newline at end of file diff --git a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp index 17e32b20a99..57f7aecd316 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp @@ -106,7 +106,7 @@ public: /// In this case we can avoid iterating over all rows because we can get all types /// in Dynamic column in a more efficient way. else - assert_cast(*columns[0]).getAllTypeNames(data(place).data); + assert_cast(*columns[0]).getAllTypeNamesInto(data(place).data); } void addManyDefaults( diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp index 98996aac2f7..4e60e6fe60b 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -146,7 +146,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData ReadBufferFromMemory buf(value.data, value.size); auto type = decodeDataType(buf); /// We should not have Nulls here but let's check just in case. - chassert(!isNothingType(type)); + chassert(!isNothing(type)); data[path].insert(type->getName()); } } @@ -160,7 +160,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData /// Add dynamic path only if it has at least one non-null value. /// getNumberOfDefaultRows for Dynamic column is O(1). if (dynamic_column->getNumberOfDefaultRows() != dynamic_column->size()) - dynamic_column->getAllTypeNames(data[path]); + dynamic_column->getAllTypeNamesInto(data[path]); } /// Iterate over all paths in shared data and decode the data types. @@ -172,7 +172,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData ReadBufferFromMemory buf(value.data, value.size); auto type = decodeDataType(buf); /// We should not have Nulls here but let's check just in case. - chassert(!isNothingType(type)); + chassert(!isNothing(type)); data[path].insert(type->getName()); } } @@ -255,7 +255,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData } }; -/// Calculates the list of distinct data types in Dynamic column. +/// Calculates the list of distinct paths or pairs (path, type) in JSON column. template class AggregateFunctionDistinctJSONPathsAndTypes final : public IAggregateFunctionDataHelper> { diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index eb85c6b0d08..269c8455e2f 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -997,7 +997,7 @@ String ColumnDynamic::getTypeNameAt(size_t row_num) const return variant_info.variant_names[discr]; } -void ColumnDynamic::getAllTypeNames(std::unordered_set & names) const +void ColumnDynamic::getAllTypeNamesInto(std::unordered_set & names) const { auto shared_variant_discr = getSharedVariantDiscriminator(); for (size_t i = 0; i != variant_info.variant_names.size(); ++i) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index c06c31bb8c9..5789b80a2de 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -431,7 +431,7 @@ public: const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type) const { return getVariantSerialization(variant_type, variant_type->getName()); } String getTypeNameAt(size_t row_num) const; - void getAllTypeNames(std::unordered_set & names) const; + void getAllTypeNamesInto(std::unordered_set & names) const; private: void createVariantInfo(const DataTypePtr & variant_type); From 82eae9f09f58959c2d9727dce66092132168de66 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 28 Aug 2024 16:18:57 +0000 Subject: [PATCH 098/259] tune --- tests/ci/integration_tests_runner.py | 2 +- tests/integration/test_async_load_databases/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index a7d5a8c4cf6..96e1ddc9bff 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -33,7 +33,7 @@ CLICKHOUSE_BINARY_PATH = "usr/bin/clickhouse" CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-odbc-bridge" CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-library-bridge" -FLAKY_TRIES_COUNT = 5 # run whole pytest several times +FLAKY_TRIES_COUNT = 3 # run whole pytest several times FLAKY_REPEAT_COUNT = 5 # runs test case in single module several times MAX_TIME_SECONDS = 3600 diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py index f36cff76ea2..94aba46c713 100644 --- a/tests/integration/test_async_load_databases/test.py +++ b/tests/integration/test_async_load_databases/test.py @@ -182,4 +182,4 @@ def test_multiple_tables(started_cluster): for i in order: assert query(f"select count() from test.table_{i}") == "100\n" for i in range(tables_count): - query(f"drop table test.table_{i}") + query(f"drop table test.table_{i} sync") From 3f9f70d27d7d796ee8d4037a6498f23e812ef125 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 27 Aug 2024 21:10:47 +0200 Subject: [PATCH 099/259] Revert "Fix strange trash" This reverts commit aee9612e5127f9cbe0a05c12a08480fc5072ff44. --- src/Core/SettingsFields.cpp | 2 +- src/Core/SettingsFields.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 278b1101c71..86e247722c8 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -237,7 +237,7 @@ SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f) String SettingFieldMaxThreads::toString() const { if (is_auto) - return "auto(" + ::DB::toString(value) + ")"; + return "'auto(" + ::DB::toString(value) + ")'"; else return ::DB::toString(value); } diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 533d69f3fbb..266141815e3 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -153,7 +153,7 @@ struct SettingFieldMaxThreads operator UInt64() const { return value; } /// NOLINT explicit operator Field() const { return value; } - /// Writes "auto()" instead of simple "" if `is_auto == true`. + /// Writes "auto()" instead of simple "" if `is_auto==true`. String toString() const; void parseFromString(const String & str); From 0dacd16947742be39be53ec72460673438e2e984 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 27 Aug 2024 21:14:04 +0200 Subject: [PATCH 100/259] Update SettingsFields.h --- src/Core/SettingsFields.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 266141815e3..533d69f3fbb 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -153,7 +153,7 @@ struct SettingFieldMaxThreads operator UInt64() const { return value; } /// NOLINT explicit operator Field() const { return value; } - /// Writes "auto()" instead of simple "" if `is_auto==true`. + /// Writes "auto()" instead of simple "" if `is_auto == true`. String toString() const; void parseFromString(const String & str); From ab567bad1354edb199baa9c8dc478c1617d6521b Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 01:28:27 +0200 Subject: [PATCH 101/259] What if I just strip quotes --- src/Core/SettingsFields.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 86e247722c8..ca84299ec5b 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -210,7 +210,7 @@ namespace { UInt64 stringToMaxThreads(const String & str) { - if (startsWith(str, "auto")) + if (startsWith(str, "auto") || startsWith(str, "'auto")) return 0; return parseFromString(str); } From 0c23257aa8220ebc8bd9c040088cfb3b26ff6a05 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 15:09:59 +0200 Subject: [PATCH 102/259] Revert "Fix integration test" This reverts commit 0f3506bf7af20231a290ed97a59b0e233fbc7c6e. --- tests/integration/test_cgroup_limit/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_cgroup_limit/test.py b/tests/integration/test_cgroup_limit/test.py index 5d56135d9ff..e77b0f70960 100644 --- a/tests/integration/test_cgroup_limit/test.py +++ b/tests/integration/test_cgroup_limit/test.py @@ -46,7 +46,7 @@ def test_cgroup_cpu_limit(): "clickhouse local -q \"select value from system.settings where name='max_threads'\"", num_cpus, ) - expect_output = (r"auto({})".format(math.ceil(num_cpus))).encode() + expect_output = (r"\'auto({})\'".format(math.ceil(num_cpus))).encode() assert ( result.strip() == expect_output ), f"fail for cpu limit={num_cpus}, result={result.strip()}, expect={expect_output}" From 896b1726e756bbc4ff3dd9eb417b00e13263f04e Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 15:11:19 +0200 Subject: [PATCH 103/259] Add a comment --- src/Core/SettingsFields.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index ca84299ec5b..47baedfa9db 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -237,6 +237,7 @@ SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f) String SettingFieldMaxThreads::toString() const { if (is_auto) + /// Removing quotes this will introduce incompatibility between replicas with different version. return "'auto(" + ::DB::toString(value) + ")'"; else return ::DB::toString(value); From 0815a85e2bece8e9b5d22e4217a9a505ff9a231a Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 16:15:10 +0200 Subject: [PATCH 104/259] Lint --- src/Core/SettingsFields.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 47baedfa9db..5c073707b9e 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -237,7 +237,7 @@ SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f) String SettingFieldMaxThreads::toString() const { if (is_auto) - /// Removing quotes this will introduce incompatibility between replicas with different version. + /// Removing these quotes this will introduce incompatibility between replicas with different versions. return "'auto(" + ::DB::toString(value) + ")'"; else return ::DB::toString(value); From 30eca0f4a04437cd5edc72a86864cd73bba9a17a Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 16:16:09 +0200 Subject: [PATCH 105/259] Lint harder --- src/Core/SettingsFields.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 5c073707b9e..930eedb8d70 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -237,7 +237,7 @@ SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f) String SettingFieldMaxThreads::toString() const { if (is_auto) - /// Removing these quotes this will introduce incompatibility between replicas with different versions. + /// Removing quotes here will introduce an incompatibility between replicas with different versions. return "'auto(" + ::DB::toString(value) + ")'"; else return ::DB::toString(value); From 57ba0f0b32ac8d6a092ce7a56b825265fa34dbc1 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Thu, 29 Aug 2024 00:31:33 +0000 Subject: [PATCH 106/259] format gtest_Parser.cpp --- src/Parsers/tests/gtest_Parser.cpp | 98 +++++++++++++++--------------- 1 file changed, 49 insertions(+), 49 deletions(-) diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 1b50d4480d7..47f7a54389b 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -225,55 +225,55 @@ INSTANTIATE_TEST_SUITE_P(ParserCreateDatabaseQuery, ParserTest, ::testing::Combine( ::testing::Values(std::make_shared()), ::testing::ValuesIn(std::initializer_list{ - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw')", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')" - }, - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl`\n(PARTITION BY toYYYYMM(created))", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n PARTITION BY toYYYYMM(created)\n)" - }, - { - "CREATE DATABASE db ENGINE=Foo TABLE OVERRIDE `tbl` (), TABLE OVERRIDE a (COLUMNS (_created DateTime MATERIALIZED now())), TABLE OVERRIDE b (PARTITION BY rand())", - "CREATE DATABASE db\nENGINE = Foo\nTABLE OVERRIDE tbl\n(\n\n),\nTABLE OVERRIDE a\n(\n COLUMNS\n (\n `_created` DateTime MATERIALIZED now()\n )\n),\nTABLE OVERRIDE b\n(\n PARTITION BY rand()\n)" - }, - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE tbl (COLUMNS (id UUID) PARTITION BY toYYYYMM(created))", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `id` UUID\n )\n PARTITION BY toYYYYMM(created)\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", - "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n INDEX foo foo TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created))\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", - "CREATE DATABASE db\nTABLE OVERRIDE t1\n(\n TTL inserted + toIntervalMonth(1)\n),\nTABLE OVERRIDE t2\n(\n TTL inserted + toIntervalMonth(2)\n)" - }, - { - "CREATE DATABASE db ENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw') SETTINGS allows_query_when_mysql_lost = 1 TABLE OVERRIDE tab3 (COLUMNS (_staged UInt8 MATERIALIZED 1) PARTITION BY (c3) TTL c3 + INTERVAL 10 minute), TABLE OVERRIDE tab5 (PARTITION BY (c3) TTL c3 + INTERVAL 10 minute)", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw')\nSETTINGS allows_query_when_mysql_lost = 1\nTABLE OVERRIDE tab3\n(\n COLUMNS\n (\n `_staged` UInt8 MATERIALIZED 1\n )\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n),\nTABLE OVERRIDE tab5\n(\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE tbl (PARTITION BY toYYYYMM(created) COLUMNS (created DateTime CODEC(Delta)))", - "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `created` DateTime CODEC(Delta)\n )\n PARTITION BY toYYYYMM(created)\n)" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 TABLE OVERRIDE a (ORDER BY (id, version))", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 COMMENT 'db comment' TABLE OVERRIDE a (ORDER BY (id, version))", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)\nCOMMENT 'db comment'" - } - }))); + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw')", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')" + }, + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl`\n(PARTITION BY toYYYYMM(created))", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db ENGINE=Foo TABLE OVERRIDE `tbl` (), TABLE OVERRIDE a (COLUMNS (_created DateTime MATERIALIZED now())), TABLE OVERRIDE b (PARTITION BY rand())", + "CREATE DATABASE db\nENGINE = Foo\nTABLE OVERRIDE tbl\n(\n\n),\nTABLE OVERRIDE a\n(\n COLUMNS\n (\n `_created` DateTime MATERIALIZED now()\n )\n),\nTABLE OVERRIDE b\n(\n PARTITION BY rand()\n)" + }, + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE tbl (COLUMNS (id UUID) PARTITION BY toYYYYMM(created))", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `id` UUID\n )\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", + "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n INDEX foo foo TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created))\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", + "CREATE DATABASE db\nTABLE OVERRIDE t1\n(\n TTL inserted + toIntervalMonth(1)\n),\nTABLE OVERRIDE t2\n(\n TTL inserted + toIntervalMonth(2)\n)" + }, + { + "CREATE DATABASE db ENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw') SETTINGS allows_query_when_mysql_lost = 1 TABLE OVERRIDE tab3 (COLUMNS (_staged UInt8 MATERIALIZED 1) PARTITION BY (c3) TTL c3 + INTERVAL 10 minute), TABLE OVERRIDE tab5 (PARTITION BY (c3) TTL c3 + INTERVAL 10 minute)", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw')\nSETTINGS allows_query_when_mysql_lost = 1\nTABLE OVERRIDE tab3\n(\n COLUMNS\n (\n `_staged` UInt8 MATERIALIZED 1\n )\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n),\nTABLE OVERRIDE tab5\n(\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE tbl (PARTITION BY toYYYYMM(created) COLUMNS (created DateTime CODEC(Delta)))", + "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `created` DateTime CODEC(Delta)\n )\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 TABLE OVERRIDE a (ORDER BY (id, version))", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 COMMENT 'db comment' TABLE OVERRIDE a (ORDER BY (id, version))", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)\nCOMMENT 'db comment'" + } +}))); INSTANTIATE_TEST_SUITE_P(ParserCreateUserQuery, ParserTest, ::testing::Combine( From 7d56c8bd43c70f396d4a1e0aa0c8887ca4ee94b9 Mon Sep 17 00:00:00 2001 From: "baolin.hbl" Date: Tue, 20 Aug 2024 03:28:26 +0000 Subject: [PATCH 107/259] Avoid detached covered-by-broken part duplicates Problem: When a broken part is found during the startup, it will clone the parts which are covered by the broken part, to the detached directory (with the 'covered-by-broken' prefix). A part may be covered by multiple merged parts, which will result in multiple clones, which leads to path conflicts and further attempts to clone to the try-n directory. If n exceeds 9, the clone is abandoned and the table is marked as read-only. pull#41981 tried fixed the problem, but the fix is incomplete. The metadata_version.txt file is deleted during covered-by-broken clone. As a result, looksLikeBrokenDetachedPartHasTheSameContent finds differences during part comparison. Fix: covered-by-broken retain metadata_version.txt file when cloning --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 1 + .../test_covered_by_broken_exists/__init__.py | 0 .../test_covered_by_broken_exists/test.py | 103 ++++++++++++++++++ 3 files changed, 104 insertions(+) create mode 100644 tests/integration/test_covered_by_broken_exists/__init__.py create mode 100644 tests/integration/test_covered_by_broken_exists/test.py diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 195aa4fdc10..22bb188f74a 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -2063,6 +2063,7 @@ DataPartStoragePtr IMergeTreeDataPart::makeCloneInDetached(const String & prefix IDataPartStorage::ClonePartParams params { .copy_instead_of_hardlink = isStoredOnRemoteDiskWithZeroCopySupport() && storage.supportsReplication() && storage_settings->allow_remote_fs_zero_copy_replication, + .keep_metadata_version = prefix == "covered-by-broken", .make_source_readonly = true, .external_transaction = disk_transaction }; diff --git a/tests/integration/test_covered_by_broken_exists/__init__.py b/tests/integration/test_covered_by_broken_exists/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_covered_by_broken_exists/test.py b/tests/integration/test_covered_by_broken_exists/test.py new file mode 100644 index 00000000000..b6d1f55f133 --- /dev/null +++ b/tests/integration/test_covered_by_broken_exists/test.py @@ -0,0 +1,103 @@ +import pytest +import logging +import time +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance("node1", stay_alive=True, with_zookeeper=True) +node2 = cluster.add_instance("node2", with_zookeeper=True) + +instance = node1 +q = node1.query + +path_to_data = "/var/lib/clickhouse/" + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def wait_merged_part(table, part_name, retries=100): + q("OPTIMIZE TABLE {} FINAL".format(table)) + for i in range(retries): + result = q( + "SELECT name FROM system.parts where table='{}' AND name='{}'".format( + table, part_name + ) + ) + if result: + return True + time.sleep(0.5) + else: + return False + + +def test_make_clone_covered_by_broken_detached_dir_exists(started_cluster): + q("DROP TABLE IF EXISTS test_make_clone_cvbdde") + + q( + "CREATE TABLE test_make_clone_cvbdde(n int, m String) ENGINE=ReplicatedMergeTree('/test_make_clone_cvbdde', '1') ORDER BY n SETTINGS old_parts_lifetime=3600, min_age_to_force_merge_seconds=1, min_age_to_force_merge_on_partition_only=0" + ) + path = path_to_data + "data/default/test_make_clone_cvbdde/" + + q("INSERT INTO test_make_clone_cvbdde VALUES (0, 'hbl')") + + q("INSERT INTO test_make_clone_cvbdde VALUES (1, 'hbl')") + if not (wait_merged_part("test_make_clone_cvbdde", "all_0_1_1")): + assert False, "Part all_0_1_1 doesn't appeared in system.parts" + + q("INSERT INTO test_make_clone_cvbdde VALUES (2, 'hbl')") + if not (wait_merged_part("test_make_clone_cvbdde", "all_0_2_2")): + assert False, "Part all_0_2_2 doesn't appeared in system.parts" + + q("INSERT INTO test_make_clone_cvbdde VALUES (3, 'hbl')") + if not (wait_merged_part("test_make_clone_cvbdde", "all_0_3_3")): + assert False, "Part all_0_3_3 doesn't appeared in system.parts" + + res = str(instance.exec_in_container(["ls", path]).strip().split("\n")) + + # broke the merged parts + instance.exec_in_container( + [ + "bash", + "-c", + "echo 'broken' > {}".format(path + "all_0_1_1/data.bin"), + ] + ) + + instance.exec_in_container( + [ + "bash", + "-c", + "echo 'broken' > {}".format(path + "all_0_2_2/data.bin"), + ] + ) + + instance.exec_in_container( + [ + "bash", + "-c", + "echo 'broken' > {}".format(path + "all_0_3_3/data.bin"), + ] + ) + + instance.restart_clickhouse(kill=True) + + assert [ + "broken-on-start_all_0_1_1", + "broken-on-start_all_0_2_2", + "broken-on-start_all_0_3_3", + "covered-by-broken_all_0_0_0", + "covered-by-broken_all_1_1_0", + "covered-by-broken_all_2_2_0", + "covered-by-broken_all_3_3_0", + ] == sorted( + instance.exec_in_container(["ls", path + "detached/"]).strip().split("\n") + ) From d88aa3952d34eeecbca363a1a66df111c5b3e587 Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 29 Aug 2024 10:56:26 +0000 Subject: [PATCH 108/259] 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 d533c24321a8a1c8d7a937bbce6756e91a2230b8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 29 Aug 2024 13:14:43 +0200 Subject: [PATCH 109/259] Reduce test size --- .../0_stateless/03227_distinct_dynamic_types_json_paths.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql index 6930f5a3d44..3af911e27fa 100644 --- a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql @@ -4,10 +4,11 @@ set allow_experimental_dynamic_type = 1; set allow_experimental_json_type = 1; set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; +set max_block_size = 10000; drop table if exists test_json_dynamic_aggregate_functions; create table test_json_dynamic_aggregate_functions (json JSON(a1 String, max_dynamic_paths=2, max_dynamic_types=2)) engine=Memory; -insert into test_json_dynamic_aggregate_functions select toJSONString(map('a' || number % 13, multiIf(number % 5 == 0, NULL, number % 5 == 1, number::UInt32, number % 5 == 2, 'str_' || number, number % 5 == 3, range(number % 5), toBool(number % 2)))) from numbers(200000); +insert into test_json_dynamic_aggregate_functions select toJSONString(map('a' || number % 13, multiIf(number % 5 == 0, NULL, number % 5 == 1, number::UInt32, number % 5 == 2, 'str_' || number, number % 5 == 3, range(number % 5), toBool(number % 2)))) from numbers(100000); select arrayJoin(distinctJSONPaths(json)) from test_json_dynamic_aggregate_functions; select arrayJoin(distinctJSONPathsAndTypes(json)) from test_json_dynamic_aggregate_functions; select arrayJoin(distinctDynamicTypes(json.a2)) from test_json_dynamic_aggregate_functions; From b6a6d9315217a783f61671e9a0c0ef997e1cd7e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 29 Aug 2024 11:28:56 +0000 Subject: [PATCH 110/259] Remove completely invalid queries from test --- ...lter_no_properties_before_remove_modify_reset.sql | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql b/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql index 13ad11bb139..0b98c605ccf 100644 --- a/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql +++ b/tests/queries/0_stateless/03231_alter_no_properties_before_remove_modify_reset.sql @@ -4,7 +4,6 @@ CREATE TABLE a (x Int64, y Int64 MATERIALIZED 1 SETTINGS (max_compress_block_siz SELECT 'REMOVE'; ALTER TABLE a MODIFY COLUMN y Int64 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } -ALTER TABLE a MODIFY COLUMN y NOT NULL REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y DEFAULT 2 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } @@ -17,7 +16,6 @@ ALTER TABLE a MODIFY COLUMN y SETTINGS (max_compress_block_size = 20000) REMOVE ALTER TABLE a MODIFY COLUMN y PRIMARY KEY REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } SELECT 'The same, but with type'; -ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR } @@ -31,7 +29,6 @@ ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY REMOVE MATERIALIZED; -- { client SELECT 'MODIFY SETTING'; ALTER TABLE a MODIFY COLUMN y Int64 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } -ALTER TABLE a MODIFY COLUMN y NOT NULL MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } @@ -44,7 +41,6 @@ ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) MODIFY SETTING max_com ALTER TABLE a MODIFY COLUMN y PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } SELECT 'The same, but with type'; -ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR } @@ -58,7 +54,6 @@ ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY MODIFY SETTING max_compress_bloc SELECT 'RESET SETTING'; ALTER TABLE a MODIFY COLUMN y Int64 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } -ALTER TABLE a MODIFY COLUMN y NOT NULL RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y DEFAULT 2 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } @@ -71,7 +66,6 @@ ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) RESET SETTING max_comp ALTER TABLE a MODIFY COLUMN y PRIMARY KEY RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } SELECT 'The same, but with type'; -ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR } @@ -89,7 +83,6 @@ SELECT 'All the above, but on server side'; SELECT 'REMOVE'; SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } -SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y NOT NULL REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } @@ -102,7 +95,6 @@ SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (max_compress_block_s SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } SELECT 'The same, but with type'; -SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR } @@ -116,7 +108,6 @@ SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY REMOVE MATER SELECT 'MODIFY SETTING'; SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } -SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y NOT NULL MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } @@ -129,7 +120,6 @@ SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) MO SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } SELECT 'The same, but with type'; -SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR } @@ -143,7 +133,6 @@ SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY MODIFY SETTI SELECT 'RESET SETTING'; SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } -SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y NOT NULL RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } @@ -156,7 +145,6 @@ SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) RE SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } SELECT 'The same, but with type'; -SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 NOT NULL RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR } From 0400dcb03eefdb6604d04f5ed3c70c179032f84d Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 29 Aug 2024 12:34:58 +0000 Subject: [PATCH 111/259] 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 4499f20ca8af46f88da72f6eaeb2880ded348efa Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 29 Aug 2024 14:38:59 +0000 Subject: [PATCH 112/259] better code in AsynchronousInsertQueue --- src/Interpreters/AsynchronousInsertQueue.cpp | 70 ++++++++------------ src/Interpreters/AsynchronousInsertQueue.h | 7 -- 2 files changed, 27 insertions(+), 50 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 5ed0d6d6257..c619ea80c7c 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -805,12 +805,10 @@ try if (async_insert_log) log_elements.reserve(data->entries.size()); - auto add_entry_to_asynchronous_insert_log = [&]( + auto add_entry_to_asynchronous_insert_log = [&, query_by_format = NameToNameMap{}]( const InsertData::EntryPtr & entry, - const NameToNameMap & query_by_format, const String & parsing_exception, - size_t num_rows, - size_t num_bytes) + size_t num_rows) mutable { if (!async_insert_log) return; @@ -822,15 +820,29 @@ try elem.table = query_table; elem.format = entry->format; elem.query_id = entry->query_id; - elem.bytes = num_bytes; + elem.bytes = entry->chunk.byteSize(); elem.rows = num_rows; elem.exception = parsing_exception; elem.data_kind = entry->chunk.getDataKind(); elem.timeout_milliseconds = data->timeout_ms.count(); elem.flush_query_id = insert_query_id; - auto it = query_by_format.find(entry->format); - elem.query_for_logging = it != query_by_format.end() ? it->second : key.query_str; + auto get_query_by_format = [&](const String & format) -> const String & + { + auto [it, inserted] = query_by_format.try_emplace(format); + if (!inserted) + return it->second; + + auto query = key.query->clone(); + assert_cast(*query).format = format; + it->second = serializeQuery(*query, insert_context->getSettingsRef().log_queries_cut_to_length); + return it->second; + }; + + if (entry->chunk.getDataKind() == DataKind::Parsed) + elem.query_for_logging = key.query_str; + else + elem.query_for_logging = get_query_by_format(entry->format); /// If there was a parsing error, /// the entry won't be flushed anyway, @@ -843,7 +855,7 @@ try else { elem.status = AsynchronousInsertLogElement::Ok; - log_elements.push_back(elem); + log_elements.push_back(std::move(elem)); } }; @@ -878,9 +890,8 @@ try if (async_insert_log) { - auto query_by_format = getQueriesByFormat(key.query, data->entries, insert_context); for (const auto & entry : data->entries) - add_entry_to_asynchronous_insert_log(entry, query_by_format, "", 0, entry->chunk.byteSize()); + add_entry_to_asynchronous_insert_log(entry, /*parsing_exception=*/ "", /*num_rows=*/ 0); auto exception = getCurrentExceptionMessage(false); auto flush_time = std::chrono::system_clock::now(); @@ -919,13 +930,13 @@ try if (key.data_kind == DataKind::Parsed) chunk = processEntriesWithParsing(key, data, header, insert_context, log, add_entry_to_asynchronous_insert_log); else - chunk = processPreprocessedEntries(key, data, header, insert_context, add_entry_to_asynchronous_insert_log); + chunk = processPreprocessedEntries(data, header, add_entry_to_asynchronous_insert_log); ProfileEvents::increment(ProfileEvents::AsyncInsertRows, chunk.getNumRows()); if (chunk.getNumRows() == 0) { - finish_entries(0, 0); + finish_entries(/*num_rows=*/ 0, /*num_bytes=*/ 0); return; } @@ -1012,7 +1023,6 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( StreamingFormatExecutor executor(header, format, std::move(on_error), std::move(adding_defaults_transform)); auto chunk_info = std::make_shared(); - auto query_for_logging = serializeQuery(*key.query, insert_context->getSettingsRef().log_queries_cut_to_length); for (const auto & entry : data->entries) { @@ -1025,15 +1035,13 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( auto buffer = std::make_unique(*bytes); - size_t num_bytes = bytes->size(); size_t num_rows = executor.execute(*buffer); - total_rows += num_rows; + chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, {}, current_exception, num_rows, num_bytes); - + add_to_async_insert_log(entry, current_exception, num_rows); current_exception.clear(); entry->resetChunk(); } @@ -1045,18 +1053,14 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( template Chunk AsynchronousInsertQueue::processPreprocessedEntries( - const InsertQuery & key, const InsertDataPtr & data, const Block & header, - const ContextPtr & insert_context, LogFunc && add_to_async_insert_log) { size_t total_rows = 0; auto chunk_info = std::make_shared(); auto result_columns = header.cloneEmptyColumns(); - auto query_by_format = getQueriesByFormat(key.query, data->entries, insert_context); - for (const auto & entry : data->entries) { const auto * block = entry->chunk.asBlock(); @@ -1073,10 +1077,11 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); total_rows += block_to_insert.rows(); + chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, query_by_format, "", block_to_insert.rows(), block_to_insert.bytes()); + add_to_async_insert_log(entry, /*parsing_exception=*/ "", block_to_insert.rows()); entry->resetChunk(); } @@ -1085,27 +1090,6 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( return chunk; } -NameToNameMap AsynchronousInsertQueue::getQueriesByFormat( - const ASTPtr & query, - const std::list & entries, - const ContextPtr & insert_context) -{ - std::unordered_map format_to_query; - auto query_copy = query->clone(); - - for (const auto & entry : entries) - { - auto [it, inserted] = format_to_query.try_emplace(entry->format); - if (!inserted) - continue; - - assert_cast(*query_copy).format = entry->format; - it->second = serializeQuery(*query_copy, insert_context->getSettingsRef().log_queries_cut_to_length); - } - - return format_to_query; -} - template void AsynchronousInsertQueue::finishWithException( const ASTPtr & query, const std::list & entries, const E & exception) diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index 9a84fe8bb12..cbe998a2850 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -287,17 +287,10 @@ private: template static Chunk processPreprocessedEntries( - const InsertQuery & key, const InsertDataPtr & data, const Block & header, - const ContextPtr & insert_context, LogFunc && add_to_async_insert_log); - static NameToNameMap getQueriesByFormat( - const ASTPtr & query, - const std::list & entries, - const ContextPtr & insert_context); - template static void finishWithException(const ASTPtr & query, const std::list & entries, const E & exception); From a4ddca773b4fdc5cc8d2b003bb64ae2de4790439 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 29 Aug 2024 16:03:41 +0000 Subject: [PATCH 113/259] return back num_bytes --- src/Interpreters/AsynchronousInsertQueue.cpp | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index c619ea80c7c..9d7007d61aa 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -808,7 +808,8 @@ try auto add_entry_to_asynchronous_insert_log = [&, query_by_format = NameToNameMap{}]( const InsertData::EntryPtr & entry, const String & parsing_exception, - size_t num_rows) mutable + size_t num_rows, + size_t num_bytes) mutable { if (!async_insert_log) return; @@ -820,7 +821,7 @@ try elem.table = query_table; elem.format = entry->format; elem.query_id = entry->query_id; - elem.bytes = entry->chunk.byteSize(); + elem.bytes = num_bytes; elem.rows = num_rows; elem.exception = parsing_exception; elem.data_kind = entry->chunk.getDataKind(); @@ -891,7 +892,7 @@ try if (async_insert_log) { for (const auto & entry : data->entries) - add_entry_to_asynchronous_insert_log(entry, /*parsing_exception=*/ "", /*num_rows=*/ 0); + add_entry_to_asynchronous_insert_log(entry, /*parsing_exception=*/ "", /*num_rows=*/ 0, entry->chunk.byteSize()); auto exception = getCurrentExceptionMessage(false); auto flush_time = std::chrono::system_clock::now(); @@ -1035,13 +1036,15 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( auto buffer = std::make_unique(*bytes); + size_t num_bytes = bytes->size(); size_t num_rows = executor.execute(*buffer); + total_rows += num_rows; chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, current_exception, num_rows); + add_to_async_insert_log(entry, current_exception, num_rows, num_bytes); current_exception.clear(); entry->resetChunk(); } @@ -1081,7 +1084,7 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, /*parsing_exception=*/ "", block_to_insert.rows()); + add_to_async_insert_log(entry, /*parsing_exception=*/ "", block_to_insert.rows(), block_to_insert.bytes()); entry->resetChunk(); } From aad57f438cf4ab5d09c0f02b8946f54e44e7d8f8 Mon Sep 17 00:00:00 2001 From: sakulali Date: Fri, 30 Aug 2024 00:16:10 +0800 Subject: [PATCH 114/259] CREATE TABLE AS copy PRIMARY KEY, ORDER BY, and similar clauses. --- src/Interpreters/InterpreterCreateQuery.cpp | 13 ++++ ...te_table_as_with_sorting_clauses.reference | 70 +++++++++++++++++++ ...6_create_table_as_with_sorting_clauses.sql | 37 ++++++++++ 3 files changed, 120 insertions(+) create mode 100644 tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.reference create mode 100644 tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 80cb0510b35..e9f40bdbaf5 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -821,6 +821,19 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti { properties.indices = as_storage_metadata->getSecondaryIndices(); properties.projections = as_storage_metadata->getProjections().clone(); + + /// CREATE TABLE AS should copy PRIMARY KEY, ORDER BY, and similar clauses. + if (!create.storage->primary_key && as_storage_metadata->isPrimaryKeyDefined() && as_storage_metadata->hasPrimaryKey()) + create.storage->set(create.storage->primary_key, as_storage_metadata->getPrimaryKeyAST()->clone()); + + if (!create.storage->partition_by && as_storage_metadata->isPartitionKeyDefined() && as_storage_metadata->hasPartitionKey()) + create.storage->set(create.storage->partition_by, as_storage_metadata->getPartitionKeyAST()->clone()); + + if (!create.storage->order_by && as_storage_metadata->isSortingKeyDefined() && as_storage_metadata->hasSortingKey()) + create.storage->set(create.storage->order_by, as_storage_metadata->getSortingKeyAST()->clone()); + + if (!create.storage->sample_by && as_storage_metadata->isSamplingKeyDefined() && as_storage_metadata->hasSamplingKey()) + create.storage->set(create.storage->sample_by, as_storage_metadata->getSamplingKeyAST()->clone()); } else { diff --git a/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.reference b/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.reference new file mode 100644 index 00000000000..cebb99f005e --- /dev/null +++ b/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.reference @@ -0,0 +1,70 @@ +-------------- Test copy sorting clauses from source table -------------- +CREATE TABLE default.x +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(EventDate) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +SETTINGS index_granularity = 8192 +------------------------------------------------------------------------- +CREATE TABLE default.x_as +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(EventDate) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192 +-------------- Test copy sorting clauses from destination table (source table without the same type clauses) -------------- +CREATE TABLE default.x +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PRIMARY KEY (CounterID, EventDate, intHash32(UserID)) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SETTINGS index_granularity = 8192 +------------------------------------------------------------------------- +CREATE TABLE default.x_as +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(EventDate) +PRIMARY KEY (CounterID, EventDate, intHash32(UserID)) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192 +-------------- Test copy sorting clauses from destination table (source table with the same type clauses) -------------- +CREATE TABLE default.x +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +ORDER BY CounterID +SETTINGS index_granularity = 8192 +------------------------------------------------------------------------- +CREATE TABLE default.x_as +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(EventDate) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192 diff --git a/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.sql b/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.sql new file mode 100644 index 00000000000..96c2df54491 --- /dev/null +++ b/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.sql @@ -0,0 +1,37 @@ +DROP TABLE IF EXISTS x; +DROP TABLE IF EXISTS x_as; + +SELECT '-------------- Test copy sorting clauses from source table --------------'; +CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID); +CREATE TABLE x_as AS x ENGINE = MergeTree SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1; + +SHOW CREATE TABLE x FORMAT TSVRaw; +SELECT '-------------------------------------------------------------------------'; +SHOW CREATE TABLE x_as FORMAT TSVRaw; + +DROP TABLE x; +DROP TABLE x_as; + +SELECT '-------------- Test copy sorting clauses from destination table (source table without the same type clauses) --------------'; +CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree PRIMARY KEY (CounterID, EventDate, intHash32(UserID)); +CREATE TABLE x_as AS x ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1; + +SHOW CREATE TABLE x FORMAT TSVRaw; +SELECT '-------------------------------------------------------------------------'; +SHOW CREATE TABLE x_as FORMAT TSVRaw; + +DROP TABLE x; +DROP TABLE x_as; + +SELECT '-------------- Test copy sorting clauses from destination table (source table with the same type clauses) --------------'; +CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree ORDER BY (CounterID); +CREATE TABLE x_as AS x ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1; + +SHOW CREATE TABLE x FORMAT TSVRaw; +SELECT '-------------------------------------------------------------------------'; +SHOW CREATE TABLE x_as FORMAT TSVRaw; + +DROP TABLE x; +DROP TABLE x_as; + + From df16831cc8cd9e1af95116d73309a594fb42b9e5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 29 Aug 2024 18:56:37 +0200 Subject: [PATCH 115/259] Update tests/integration/test_covered_by_broken_exists/test.py --- tests/integration/test_covered_by_broken_exists/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_covered_by_broken_exists/test.py b/tests/integration/test_covered_by_broken_exists/test.py index b6d1f55f133..caa091fdd2d 100644 --- a/tests/integration/test_covered_by_broken_exists/test.py +++ b/tests/integration/test_covered_by_broken_exists/test.py @@ -40,7 +40,7 @@ def wait_merged_part(table, part_name, retries=100): def test_make_clone_covered_by_broken_detached_dir_exists(started_cluster): - q("DROP TABLE IF EXISTS test_make_clone_cvbdde") + q("DROP TABLE IF EXISTS test_make_clone_cvbdde SYNC") q( "CREATE TABLE test_make_clone_cvbdde(n int, m String) ENGINE=ReplicatedMergeTree('/test_make_clone_cvbdde', '1') ORDER BY n SETTINGS old_parts_lifetime=3600, min_age_to_force_merge_seconds=1, min_age_to_force_merge_on_partition_only=0" From 25f31f914904e197797ef9917ff97c6447846203 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 29 Aug 2024 18:15:05 +0000 Subject: [PATCH 116/259] Fix conversion for Dynamic, add more tests --- src/Functions/FunctionsConversion.cpp | 62 ++- ..._variant_dynamic_cast_or_default.reference | 484 ++++++++++++++++++ .../03212_variant_dynamic_cast_or_default.sql | 108 ++++ 3 files changed, 644 insertions(+), 10 deletions(-) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 96e2a3291b3..271daa99d0c 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4143,12 +4143,15 @@ private: /// requires quite a lot of work. By now let's simply use try/catch. /// First, check that we can create a wrapper. WrapperType wrapper = prepareUnpackDictionaries(from, to); - /// Second, check if we can perform a conversion on empty columns. - ColumnsWithTypeAndName column_from = {{from->createColumn(), from, "" }}; - wrapper(column_from, to, nullptr, 0); + /// Second, check if we can perform a conversion on column with default value. + /// (we cannot just check empty column as we do some checks only during iteration over rows). + auto test_col = from->createColumn(); + test_col->insertDefault(); + ColumnsWithTypeAndName column_from = {{test_col->getPtr(), from, "" }}; + wrapper(column_from, to, nullptr, 1); return wrapper; } - catch (...) + catch (const Exception &) { return {}; } @@ -4393,10 +4396,27 @@ private: casted_variant_columns.reserve(variant_types.size()); for (size_t i = 0; i != variant_types.size(); ++i) { + /// Skip shared variant, it will be processed later. + if (i == column_dynamic.getSharedVariantDiscriminator()) + { + casted_variant_columns.push_back(nullptr); + continue; + } + const auto & variant_col = variant_column.getVariantPtrByGlobalDiscriminator(i); ColumnsWithTypeAndName variant = {{variant_col, variant_types[i], ""}}; - auto variant_wrapper = prepareUnpackDictionaries(variant_types[i], result_type); - casted_variant_columns.push_back(variant_wrapper(variant, result_type, nullptr, variant_col->size())); + WrapperType variant_wrapper; + if (cast_type == CastType::accurateOrNull) + /// Create wrapper only if we support conversion from variant to the resulting type. + variant_wrapper = createWrapperIfCanConvert(variant_types[i], result_type); + else + variant_wrapper = prepareUnpackDictionaries(variant_types[i], result_type); + + ColumnPtr casted_variant; + /// Check if we have wrapper for this variant. + if (variant_wrapper) + casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); + casted_variant_columns.push_back(casted_variant); } /// Second, collect all variants stored in shared variant and cast them to result type. @@ -4452,8 +4472,18 @@ private: for (size_t i = 0; i != variant_types_from_shared_variant.size(); ++i) { ColumnsWithTypeAndName variant = {{variant_columns_from_shared_variant[i]->getPtr(), variant_types_from_shared_variant[i], ""}}; - auto variant_wrapper = prepareUnpackDictionaries(variant_types_from_shared_variant[i], result_type); - casted_shared_variant_columns.push_back(variant_wrapper(variant, result_type, nullptr, variant_columns_from_shared_variant[i]->size())); + WrapperType variant_wrapper; + if (cast_type == CastType::accurateOrNull) + /// Create wrapper only if we support conversion from variant to the resulting type. + variant_wrapper = createWrapperIfCanConvert(variant_types_from_shared_variant[i], result_type); + else + variant_wrapper = prepareUnpackDictionaries(variant_types_from_shared_variant[i], result_type); + + ColumnPtr casted_variant; + /// Check if we have wrapper for this variant. + if (variant_wrapper) + casted_variant = variant_wrapper(variant, result_type, nullptr, variant_columns_from_shared_variant[i]->size()); + casted_shared_variant_columns.push_back(casted_variant); } /// Construct result column from all casted variants. @@ -4463,11 +4493,23 @@ private: { auto global_discr = variant_column.globalDiscriminatorByLocal(local_discriminators[i]); if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + { res->insertDefault(); + } else if (global_discr == shared_variant_discr) - res->insertFrom(*casted_shared_variant_columns[shared_variant_indexes[i]], shared_variant_offsets[i]); + { + if (casted_shared_variant_columns[shared_variant_indexes[i]]) + res->insertFrom(*casted_shared_variant_columns[shared_variant_indexes[i]], shared_variant_offsets[i]); + else + res->insertDefault(); + } else - res->insertFrom(*casted_variant_columns[global_discr], offsets[i]); + { + if (casted_variant_columns[global_discr]) + res->insertFrom(*casted_variant_columns[global_discr], offsets[i]); + else + res->insertDefault(); + } } return res; diff --git a/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference index 8b1a342181c..fd16d020019 100644 --- a/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference +++ b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.reference @@ -30,3 +30,487 @@ 5 5 \N str_6 \N [0,1,2,3,4,5,6] +-128 +-127 +-1 +0 +1 +2 +3 +126 +127 +0 +1 +2 +3 +126 +127 +254 +255 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +3232235521 +4294967294 +4294967295 +-9223372036854775808 +-9223372036854775807 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +3232235521 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +18446744073709551614 +18446744073709551615 +-170141183460469231731687303715884105728 +-170141183460469231731687303715884105727 +-9223372036854775808 +-9223372036854775807 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +18446744073709551614 +18446744073709551615 +170141183460469231731687303715884105726 +170141183460469231731687303715884105727 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +3232235521 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +18446744073709551614 +18446744073709551615 +170141183460469231731687303715884105726 +170141183460469231731687303715884105727 +296245801836096677496328508227807879401 +340282366920938463463374607431768211454 +340282366920938463463374607431768211455 +-57896044618658097711785492504343953926634992332820282019728792003956564819968 +-57896044618658097711785492504343953926634992332820282019728792003956564819967 +-170141183460469231731687303715884105728 +-170141183460469231731687303715884105727 +-9223372036854775808 +-9223372036854775807 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +18446744073709551614 +18446744073709551615 +170141183460469231731687303715884105726 +170141183460469231731687303715884105727 +340282366920938463463374607431768211454 +340282366920938463463374607431768211455 +57896044618658097711785492504343953926634992332820282019728792003956564819966 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +0 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +3232235521 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +18446744073709551614 +18446744073709551615 +170141183460469231731687303715884105726 +170141183460469231731687303715884105727 +340282366920938463463374607431768211454 +340282366920938463463374607431768211455 +57896044618658097711785492504343953926634992332820282019728792003956564819966 +57896044618658097711785492504343953926634992332820282019728792003956564819967 +115792089237316195423570985008687907853269984665640564039457584007913129639934 +115792089237316195423570985008687907853269984665640564039457584007913129639935 +-inf +-3.4028233e38 +-1.7014118e38 +-9223372000000000000 +-2147483600 +-32768 +-32767 +-128 +-127 +-1 +-1.1754942e-38 +-1e-45 +0 +1e-45 +1.1754942e-38 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +3.4028233e38 +inf +nan +-inf +-1.7976931348623157e308 +-5.78960446186581e76 +-3.40282347e38 +-3.4028232635611926e38 +-1.7014118346046923e38 +-9223372036854776000 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +-1.1754943499999998e-38 +-1.1754942106924411e-38 +-1.401298464324817e-45 +-1.3999999999999999e-45 +-2.2250738585072014e-308 +0 +2.2250738585072014e-308 +1.3999999999999999e-45 +1.401298464324817e-45 +1.1754942106924411e-38 +1.1754943499999998e-38 +1 +2 +3 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +4294967294 +4294967295 +3.4028232635611926e38 +3.40282347e38 +1.7976931348623157e308 +inf +nan +-32768 +-32767 +-128 +-127 +-1 +0 +1 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +4294967294 +4294967295 +-9223372036854775808 +-9223372036854775807 +-18446744073709551.616 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +18446744073709551614 +18446744073709551615 +-340282347000000000977176926486249829565.415 +-9223372036854775808 +-9223372036854775807 +-18446744073709551.616 +-2147483648 +-2147483647 +-32768 +-32767 +-128 +-127 +-1 +0 +1 +126 +127 +254 +255 +32766 +32767 +65534 +65535 +2147483646 +2147483647 +4294967294 +4294967295 +9223372036854775806 +9223372036854775807 +18446744073709551614 +18446744073709551615 +340282347000000000977176926486249829565.415 +1970-01-01 +1970-01-02 +1970-01-03 +1970-01-04 +1970-05-07 +1970-05-08 +1970-09-12 +1970-09-13 +2038-01-19 +2059-09-17 +2059-09-18 +2106-02-07 +2149-06-05 +2149-06-06 +2299-12-31 +2299-12-31 +1900-01-01 +1969-08-26 +1969-08-27 +1969-12-30 +1969-12-31 +1970-01-01 +1970-01-02 +1970-01-03 +1970-01-04 +1970-05-07 +1970-05-08 +1970-09-12 +1970-09-13 +2038-01-19 +2059-09-17 +2059-09-18 +2106-02-07 +2149-06-05 +2149-06-06 +2299-12-31 +1970-01-01 00:00:00 +1970-01-01 00:00:01 +1970-01-01 00:00:02 +1970-01-01 00:00:03 +1970-01-01 00:02:06 +1970-01-01 00:02:07 +1970-01-01 00:04:14 +1970-01-01 00:04:15 +1970-01-01 09:06:06 +1970-01-01 09:06:07 +1970-01-01 18:12:14 +1970-01-01 18:12:15 +2038-01-19 03:14:06 +2038-01-19 03:14:07 +2106-02-07 06:28:14 +2106-02-07 06:28:15 +0.0.0.0 +192.168.0.1 +:: +::1 +::ffff:192.168.0.1 +00000000-0000-0000-0000-000000000000 +dededdb6-7835-4ce4-8d11-b5de6f2820e9 diff --git a/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql index 1e71e36780c..f227bbdac77 100644 --- a/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql +++ b/tests/queries/0_stateless/03212_variant_dynamic_cast_or_default.sql @@ -1,9 +1,117 @@ set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; set allow_experimental_dynamic_type = 1; +set allow_suspicious_low_cardinality_types = 1; +set session_timezone = 'UTC'; select accurateCastOrDefault(variant, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number)) as variant from numbers(8); select accurateCastOrNull(variant, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number)) as variant from numbers(8); select accurateCastOrDefault(dynamic, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number))::Dynamic as dynamic from numbers(8); select accurateCastOrNull(dynamic, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number))::Dynamic as dynamic from numbers(8); + +drop table if exists t; +create table t (d Dynamic) engine=MergeTree order by tuple(); + +-- Integer types: signed and unsigned integers (UInt8, UInt16, UInt32, UInt64, UInt128, UInt256, Int8, Int16, Int32, Int64, Int128, Int256) +INSERT INTO t VALUES (-128::Int8), (-127::Int8), (-1::Int8), (0::Int8), (1::Int8), (126::Int8), (127::Int8); +INSERT INTO t VALUES (-128::Int8), (-127::Int8), (-1::Int8), (0::Int8), (1::Int8), (126::Int8), (127::Int8); +INSERT INTO t VALUES (-128::Int8), (-127::Int8), (-1::Int8), (0::Int8), (1::Int8), (126::Int8), (127::Int8); +INSERT INTO t VALUES (-32768::Int16), (-32767::Int16), (-1::Int16), (0::Int16), (1::Int16), (32766::Int16), (32767::Int16); +INSERT INTO t VALUES (-2147483648::Int32), (-2147483647::Int32), (-1::Int32), (0::Int32), (1::Int32), (2147483646::Int32), (2147483647::Int32); +INSERT INTO t VALUES (-9223372036854775808::Int64), (-9223372036854775807::Int64), (-1::Int64), (0::Int64), (1::Int64), (9223372036854775806::Int64), (9223372036854775807::Int64); +INSERT INTO t VALUES (-170141183460469231731687303715884105728::Int128), (-170141183460469231731687303715884105727::Int128), (-1::Int128), (0::Int128), (1::Int128), (170141183460469231731687303715884105726::Int128), (170141183460469231731687303715884105727::Int128); +INSERT INTO t VALUES (-57896044618658097711785492504343953926634992332820282019728792003956564819968::Int256), (-57896044618658097711785492504343953926634992332820282019728792003956564819967::Int256), (-1::Int256), (0::Int256), (1::Int256), (57896044618658097711785492504343953926634992332820282019728792003956564819966::Int256), (57896044618658097711785492504343953926634992332820282019728792003956564819967::Int256); + +INSERT INTO t VALUES (0::UInt8), (1::UInt8), (254::UInt8), (255::UInt8); +INSERT INTO t VALUES (0::UInt16), (1::UInt16), (65534::UInt16), (65535::UInt16); +INSERT INTO t VALUES (0::UInt32), (1::UInt32), (4294967294::UInt32), (4294967295::UInt32); +INSERT INTO t VALUES (0::UInt64), (1::UInt64), (18446744073709551614::UInt64), (18446744073709551615::UInt64); +INSERT INTO t VALUES (0::UInt128), (1::UInt128), (340282366920938463463374607431768211454::UInt128), (340282366920938463463374607431768211455::UInt128); +INSERT INTO t VALUES (0::UInt256), (1::UInt256), (115792089237316195423570985008687907853269984665640564039457584007913129639934::UInt256), (115792089237316195423570985008687907853269984665640564039457584007913129639935::UInt256); + +-- Floating-point numbers: floats(Float32 and Float64) values +INSERT INTO t VALUES (1.17549435e-38::Float32), (3.40282347e+38::Float32), (-3.40282347e+38::Float32), (-1.17549435e-38::Float32), (1.4e-45::Float32), (-1.4e-45::Float32); +INSERT INTO t VALUES (inf::Float32), (-inf::Float32), (nan::Float32); +INSERT INTO t VALUES (inf::FLOAT(12)), (-inf::FLOAT(12)), (nan::FLOAT(12)); +INSERT INTO t VALUES (inf::FLOAT(15,22)), (-inf::FLOAT(15,22)), (nan::FLOAT(15,22)); + +INSERT INTO t VALUES (1.17549435e-38::Float64), (3.40282347e+38::Float64), (-3.40282347e+38::Float64), (-1.17549435e-38::Float64), (1.4e-45::Float64), (-1.4e-45::Float64); +INSERT INTO t VALUES (2.2250738585072014e-308::Float64), (1.7976931348623157e+308::Float64), (-1.7976931348623157e+308::Float64), (-2.2250738585072014e-308::Float64); +INSERT INTO t VALUES (inf::Float64), (-inf::Float64), (nan::Float64); +INSERT INTO t VALUES (inf::DOUBLE(12)), (-inf::DOUBLE(12)), (nan::DOUBLE(12)); +INSERT INTO t VALUES (inf::DOUBLE(15,22)), (-inf::DOUBLE(15,22)), (nan::DOUBLE(15,22)); + +-- Strings: String and FixedString +INSERT INTO t VALUES ('string'::String), ('1'::FixedString(1)), ('1'::FixedString(2)), ('1'::FixedString(10)); --(''::String), + +-- Boolean +INSERT INTO t VALUES ('1'::Bool), (0::Bool); + +-- UUID +INSERT INTO t VALUES ('dededdb6-7835-4ce4-8d11-b5de6f2820e9'::UUID); +INSERT INTO t VALUES ('00000000-0000-0000-0000-000000000000'::UUID); + +-- LowCardinality +INSERT INTO t VALUES ('1'::LowCardinality(String)), ('1'::LowCardinality(String)), (0::LowCardinality(UInt16)); + +-- Arrays +INSERT INTO t VALUES ([]::Array(Dynamic)), ([[]]::Array(Array(Dynamic))), ([[[]]]::Array(Array(Array(Dynamic)))); + +-- Tuple +INSERT INTO t VALUES (()::Tuple(Dynamic)), ((())::Tuple(Tuple(Dynamic))), (((()))::Tuple(Tuple(Tuple(Dynamic)))); + +-- Map. +INSERT INTO t VALUES (map(11::Dynamic, 'v1'::Dynamic, '22'::Dynamic, 1::Dynamic)); + +-- SimpleAggregateFunction +INSERT INTO t VALUES ([1,2]::SimpleAggregateFunction(anyLast, Array(Int16))); + +-- IPs +INSERT INTO t VALUES (toIPv4('192.168.0.1')), (toIPv6('::1')); + +-- Geo +INSERT INTO t VALUES ((1.23, 4.56)::Point), (([(1.23, 4.56)::Point, (2.34, 5.67)::Point])::Ring); +INSERT INTO t VALUES ([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]]::MultiPolygon); + +-- Interval +INSERT INTO t VALUES (interval '1' day), (interval '2' month), (interval '3' year); + +-- Nested +INSERT INTO t VALUES ([(1, 'aa'), (2, 'bb')]::Nested(x UInt32, y String)); +INSERT INTO t VALUES ([(1, (2, ['aa', 'bb']), [(3, 'cc'), (4, 'dd')]), (5, (6, ['ee', 'ff']), [(7, 'gg'), (8, 'hh')])]::Nested(x UInt32, y Tuple(y1 UInt32, y2 Array(String)), z Nested(z1 UInt32, z2 String))); + +optimize table t final; + +select distinct toInt8OrDefault(d) as res from t order by res; +select distinct toUInt8OrDefault(d) as res from t order by res; +select distinct toInt16OrDefault(d) as res from t order by res; +select distinct toUInt16OrDefault(d) as res from t order by res; +select distinct toInt32OrDefault(d) as res from t order by res; +select distinct toUInt32OrDefault(d) as res from t order by res; +select distinct toInt64OrDefault(d) as res from t order by res; +select distinct toUInt64OrDefault(d) as res from t order by res; +select distinct toInt128OrDefault(d) as res from t order by res; +select distinct toUInt128OrDefault(d) as res from t order by res; +select distinct toInt256OrDefault(d) as res from t order by res; +select distinct toUInt256OrDefault(d) as res from t order by res; + +select distinct toFloat32OrDefault(d) as res from t order by res; +select distinct toFloat64OrDefault(d) as res from t order by res; + +select distinct toDecimal32OrDefault(d, 3) as res from t order by res; +select distinct toDecimal64OrDefault(d, 3) as res from t order by res; +select distinct toDecimal128OrDefault(d, 3) as res from t order by res; +select distinct toDecimal256OrDefault(d, 3) as res from t order by res; + +select distinct toDateOrDefault(d) as res from t order by res; +select distinct toDate32OrDefault(d) as res from t order by res; +select distinct toDateTimeOrDefault(d) as res from t order by res; + +select distinct toIPv4OrDefault(d) as res from t order by res; +select distinct toIPv6OrDefault(d) as res from t order by res; + +select distinct toUUIDOrDefault(d) as res from t order by res; + +drop table t; + From 79c01e717f56132c1af6503b93fc7094dcd47637 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Thu, 29 Aug 2024 23:15:39 +0400 Subject: [PATCH 117/259] 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 f7fda2471cba5f5e975cef247f10060084734fcb Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 30 Aug 2024 14:09:31 +0800 Subject: [PATCH 118/259] fix typo --- src/Interpreters/ActionsDAG.cpp | 4 ++-- src/Interpreters/ActionsDAG.h | 4 ++-- .../QueryPlan/Optimizations/useDataParallelAggregation.cpp | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 2a594839c6a..9ebc73260b6 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -396,7 +396,7 @@ const ActionsDAG::Node * ActionsDAG::tryFindInOutputs(const std::string & name) return nullptr; } -ActionsDAG::NodeRawConstPtrs ActionsDAG::findInOutpus(const Names & names) const +ActionsDAG::NodeRawConstPtrs ActionsDAG::findInOutputs(const Names & names) const { NodeRawConstPtrs required_nodes; required_nodes.reserve(names.size()); @@ -524,7 +524,7 @@ void ActionsDAG::removeUnusedActions(const NameSet & required_names, bool allow_ void ActionsDAG::removeUnusedActions(const Names & required_names, bool allow_remove_inputs, bool allow_constant_folding) { - auto required_nodes = findInOutpus(required_names); + auto required_nodes = findInOutputs(required_names); outputs.swap(required_nodes); removeUnusedActions(allow_remove_inputs, allow_constant_folding); } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index ee2b3fbf4f2..2459878ce20 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -156,7 +156,7 @@ public: const Node * tryFindInOutputs(const std::string & name) const; /// Same, but for the list of names. - NodeRawConstPtrs findInOutpus(const Names & names) const; + NodeRawConstPtrs findInOutputs(const Names & names) const; /// Find first node with the same name in output nodes and replace it. /// If was not found, add node to outputs end. @@ -436,7 +436,7 @@ public: /// Returns a list of nodes representing atomic predicates. static NodeRawConstPtrs extractConjunctionAtoms(const Node * predicate); - /// Get a list of nodes. For every node, check if it can be compused using allowed subset of inputs. + /// Get a list of nodes. For every node, check if it can be computed using allowed subset of inputs. /// Returns only those nodes from the list which can be computed. static NodeRawConstPtrs filterNodesByAllowedInputs( NodeRawConstPtrs nodes, diff --git a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp index 0eeaec9bde7..d943fea785b 100644 --- a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp +++ b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp @@ -155,7 +155,7 @@ bool isPartitionKeySuitsGroupByKey( return false; /// We are interested only in calculations required to obtain group by keys (and not aggregate function arguments for example). - auto key_nodes = group_by_actions.findInOutpus(aggregating.getParams().keys); + auto key_nodes = group_by_actions.findInOutputs(aggregating.getParams().keys); auto group_by_key_actions = ActionsDAG::cloneSubDAG(key_nodes, /*remove_aliases=*/ true); const auto & gb_key_required_columns = group_by_key_actions.getRequiredColumnsNames(); From 6ad8e5558a99b43e3452c057346b9c44e8e27517 Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 30 Aug 2024 07:25:25 +0000 Subject: [PATCH 119/259] 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 120/259] 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 121/259] 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 cf87893758b0172f19e54b8bc7fd962bf258c96e Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 30 Aug 2024 15:28:53 +0000 Subject: [PATCH 122/259] Don't infer Bool type from String in CSV when input_format_csv_try_infer_numbers_from_strings=1 --- src/Formats/EscapingRuleUtils.cpp | 8 ++++++-- .../03231_csv_dont_infer_bool_from_string.reference | 4 ++++ .../0_stateless/03231_csv_dont_infer_bool_from_string.sql | 4 ++++ 3 files changed, 14 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.reference create mode 100644 tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.sql diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 5429d8b7e0d..50a46d2334d 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -302,8 +302,12 @@ DataTypePtr tryInferDataTypeByEscapingRule(const String & field, const FormatSet /// Try to determine the type of value inside quotes auto type = tryInferDataTypeForSingleField(data, format_settings); - /// If we couldn't infer any type or it's a number and csv.try_infer_numbers_from_strings = 0, we determine it as a string. - if (!type || (format_settings.csv.try_infer_strings_from_quoted_tuples && isTuple(type)) || (!format_settings.csv.try_infer_numbers_from_strings && isNumber(type))) + /// Return String type if one of the following conditions apply + /// - we couldn't infer any type + /// - it's a number and csv.try_infer_numbers_from_strings = 0 + /// - it's a tuple and try_infer_strings_from_quoted_tuples = 0 + /// - it's a Bool type (we don't allow reading bool values from strings) + if (!type || (format_settings.csv.try_infer_strings_from_quoted_tuples && isTuple(type)) || (!format_settings.csv.try_infer_numbers_from_strings && isNumber(type)) || isBool(type)) return std::make_shared(); return type; diff --git a/tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.reference b/tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.reference new file mode 100644 index 00000000000..d23e2d2cbf3 --- /dev/null +++ b/tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.reference @@ -0,0 +1,4 @@ +c1 Nullable(Int64) +c2 Nullable(Float64) +c3 Nullable(String) +42 42.42 True diff --git a/tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.sql b/tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.sql new file mode 100644 index 00000000000..e3cf77249eb --- /dev/null +++ b/tests/queries/0_stateless/03231_csv_dont_infer_bool_from_string.sql @@ -0,0 +1,4 @@ +set input_format_csv_try_infer_numbers_from_strings = 1; +desc format(CSV, '"42","42.42","True"'); +select * from format(CSV, '"42","42.42","True"'); + From 199aaff7118ca04c8c7c32b9aaf9ea5cd660c700 Mon Sep 17 00:00:00 2001 From: sakulali Date: Sat, 31 Aug 2024 18:13:07 +0800 Subject: [PATCH 123/259] Kindly ping CI and try to reproduce failed CI tests From 6fd7656aebd0d747ddd06a069139cb935031492e Mon Sep 17 00:00:00 2001 From: m4xxx1m Date: Sat, 31 Aug 2024 18:22:03 +0300 Subject: [PATCH 124/259] Added Poco::Util::AbstractConfiguration::getHost method --- .../include/Poco/Util/AbstractConfiguration.h | 40 +++++ base/poco/Util/src/AbstractConfiguration.cpp | 153 ++++++++++++++++++ 2 files changed, 193 insertions(+) diff --git a/base/poco/Util/include/Poco/Util/AbstractConfiguration.h b/base/poco/Util/include/Poco/Util/AbstractConfiguration.h index 926ac3ba8a9..7f58df905c0 100644 --- a/base/poco/Util/include/Poco/Util/AbstractConfiguration.h +++ b/base/poco/Util/include/Poco/Util/AbstractConfiguration.h @@ -241,6 +241,20 @@ namespace Util /// If the value contains references to other properties (${}), these /// are expanded. + std::string getHost(const std::string & key) const; + /// Returns the string value of the host property with the given name. + /// Throws a NotFoundException if the key does not exist. + /// Throws a SyntaxException if the property is not a valid host (IP address or domain). + /// If the value contains references to other properties (${}), these + /// are expanded. + + std::string getHost(const std::string & key, const std::string & defaultValue) const; + /// If a property with the given key exists, returns the host property's string value, + /// otherwise returns the given default value. + /// Throws a SyntaxException if the property is not a valid host (IP address or domain). + /// If the value contains references to other properties (${}), these + /// are expanded. + virtual void setString(const std::string & key, const std::string & value); /// Sets the property with the given key to the given value. /// An already existing value for the key is overwritten. @@ -339,12 +353,38 @@ namespace Util static bool parseBool(const std::string & value); void setRawWithEvent(const std::string & key, std::string value); + static void checkHostValidity(const std::string & value); + /// Throws a SyntaxException if the value is not a valid host (IP address or domain). + virtual ~AbstractConfiguration(); private: std::string internalExpand(const std::string & value) const; std::string uncheckedExpand(const std::string & value) const; + static bool isValidIPv4Address(const std::string & value); + /// A string value is considered to be a valid IPv4 address if it matches + /// "x1.x2.x3.x4", where xi - integer in range 0..255 and may have leading zeroes + + static bool isValidIPv6Address(const std::string & value); + /// A string value is considered to be a valid IPv6 address if it matches + /// "x1:x2:x3:x4:x5:x6:x7:x8", where xi is hexadecimal integer and consist of 4 + /// characters or less (but at least 1), xi may have leading zeroes. + /// Letters in hexadecimal representation can be in upper case or lower case. + /// One or more consecutive hextets of zeroes can be replaced with "::", but + /// "::" can appear only once in a valid IPv6 address. + + static bool isValidDomainName(const std::string & value); + /// ::= [ "." ] + /// ::=