From 40adaeef92eaa9519f0d125f2cc1d070488d8238 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 18 Nov 2022 16:15:37 +0000 Subject: [PATCH 01/49] Fix aggregate functions optimization --- ...egateFunctionsArithmericOperationsPass.cpp | 12 +-- ...optimize_aggregation_arithmetics.reference | 81 +++++++++++++++++++ ...02481_optimize_aggregation_arithmetics.sql | 26 ++++++ 3 files changed, 114 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02481_optimize_aggregation_arithmetics.reference create mode 100644 tests/queries/0_stateless/02481_optimize_aggregation_arithmetics.sql diff --git a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp index dcf386b2988..7420321d3ee 100644 --- a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp +++ b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp @@ -116,8 +116,9 @@ public: if (!function_name_if_constant_is_negative.empty() && left_argument_constant_value_literal < zeroField(left_argument_constant_value_literal)) { - resolveAggregateFunctionNode(*aggregate_function_node, function_name_if_constant_is_negative); + lower_function_name = function_name_if_constant_is_negative; } + resolveAggregateFunctionNode(*aggregate_function_node, inner_function_arguments_nodes[1], lower_function_name); auto inner_function = aggregate_function_arguments_nodes[0]; auto inner_function_right_argument = std::move(inner_function_arguments_nodes[1]); @@ -132,8 +133,9 @@ public: if (!function_name_if_constant_is_negative.empty() && right_argument_constant_value_literal < zeroField(right_argument_constant_value_literal)) { - resolveAggregateFunctionNode(*aggregate_function_node, function_name_if_constant_is_negative); + lower_function_name = function_name_if_constant_is_negative; } + resolveAggregateFunctionNode(*aggregate_function_node, inner_function_arguments_nodes[0], function_name_if_constant_is_negative); auto inner_function = aggregate_function_arguments_nodes[0]; auto inner_function_left_argument = std::move(inner_function_arguments_nodes[0]); @@ -144,16 +146,16 @@ public: } private: - static inline void resolveAggregateFunctionNode(FunctionNode & function_node, const String & aggregate_function_name) + static inline void resolveAggregateFunctionNode(FunctionNode & function_node, QueryTreeNodePtr & argument, const String & aggregate_function_name) { - auto function_result_type = function_node.getResultType(); auto function_aggregate_function = function_node.getAggregateFunction(); AggregateFunctionProperties properties; auto aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, - function_aggregate_function->getArgumentTypes(), + { argument->getResultType() }, function_aggregate_function->getParameters(), properties); + auto function_result_type = aggregate_function->getReturnType(); function_node.resolveAsAggregateFunction(std::move(aggregate_function), std::move(function_result_type)); } diff --git a/tests/queries/0_stateless/02481_optimize_aggregation_arithmetics.reference b/tests/queries/0_stateless/02481_optimize_aggregation_arithmetics.reference new file mode 100644 index 00000000000..1cc6fc5d4b1 --- /dev/null +++ b/tests/queries/0_stateless/02481_optimize_aggregation_arithmetics.reference @@ -0,0 +1,81 @@ +-- { echoOn } +EXPLAIN actions=1 + ( + SELECT round(avg(log(2) * number), 6) AS k + FROM numbers(10000000) + GROUP BY number % 3, number % 2 + ) +SETTINGS allow_experimental_analyzer=1; +Expression ((Project names + Projection)) +Actions: INPUT : 0 -> avg(number_0) Float64 : 0 + COLUMN Const(Float64) -> 0.6931471805599453_Float64 Float64 : 1 + COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 2 + FUNCTION multiply(0.6931471805599453_Float64 :: 1, avg(number_0) :: 0) -> multiply(0.6931471805599453_Float64, avg(number_0)) Float64 : 3 + FUNCTION round(multiply(0.6931471805599453_Float64, avg(number_0)) :: 3, 6_UInt8 :: 2) -> round(multiply(0.6931471805599453_Float64, avg(number_0)), 6_UInt8) Float64 : 0 + ALIAS round(multiply(0.6931471805599453_Float64, avg(number_0)), 6_UInt8) :: 0 -> k Float64 : 2 +Positions: 2 + Aggregating + Keys: modulo(number_0, 3_UInt8), modulo(number_0, 2_UInt8) + Aggregates: + avg(number_0) + Function: avg(UInt64) → Float64 + Arguments: number_0 + Expression ((Before GROUP BY + Change column names to column identifiers)) + Actions: INPUT : 0 -> number UInt64 : 0 + COLUMN Const(UInt8) -> 3_UInt8 UInt8 : 1 + COLUMN Const(UInt8) -> 2_UInt8 UInt8 : 2 + ALIAS number :: 0 -> number_0 UInt64 : 3 + FUNCTION modulo(number_0 : 3, 3_UInt8 :: 1) -> modulo(number_0, 3_UInt8) UInt8 : 0 + FUNCTION modulo(number_0 : 3, 2_UInt8 :: 2) -> modulo(number_0, 2_UInt8) UInt8 : 1 + Positions: 0 1 3 + ReadFromStorage (SystemNumbers) +EXPLAIN actions=1 + ( + SELECT round(log(2) * avg(number), 6) AS k + FROM numbers(10000000) + GROUP BY number % 3, number % 2 + ) +SETTINGS allow_experimental_analyzer=1; +Expression ((Project names + Projection)) +Actions: INPUT : 0 -> avg(number_0) Float64 : 0 + COLUMN Const(Float64) -> 0.6931471805599453_Float64 Float64 : 1 + COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 2 + FUNCTION multiply(0.6931471805599453_Float64 :: 1, avg(number_0) :: 0) -> multiply(0.6931471805599453_Float64, avg(number_0)) Float64 : 3 + FUNCTION round(multiply(0.6931471805599453_Float64, avg(number_0)) :: 3, 6_UInt8 :: 2) -> round(multiply(0.6931471805599453_Float64, avg(number_0)), 6_UInt8) Float64 : 0 + ALIAS round(multiply(0.6931471805599453_Float64, avg(number_0)), 6_UInt8) :: 0 -> k Float64 : 2 +Positions: 2 + Aggregating + Keys: modulo(number_0, 3_UInt8), modulo(number_0, 2_UInt8) + Aggregates: + avg(number_0) + Function: avg(UInt64) → Float64 + Arguments: number_0 + Expression ((Before GROUP BY + Change column names to column identifiers)) + Actions: INPUT : 0 -> number UInt64 : 0 + COLUMN Const(UInt8) -> 3_UInt8 UInt8 : 1 + COLUMN Const(UInt8) -> 2_UInt8 UInt8 : 2 + ALIAS number :: 0 -> number_0 UInt64 : 3 + FUNCTION modulo(number_0 : 3, 3_UInt8 :: 1) -> modulo(number_0, 3_UInt8) UInt8 : 0 + FUNCTION modulo(number_0 : 3, 2_UInt8 :: 2) -> modulo(number_0, 2_UInt8) UInt8 : 1 + Positions: 0 1 3 + ReadFromStorage (SystemNumbers) +SELECT round(avg(log(2) * number), 6) AS k +FROM numbers(10000000) +GROUP BY number % 3, number % 2 +SETTINGS allow_experimental_analyzer=1; +3465734.516505 +3465735.209653 +3465735.9028 +3465736.595947 +3465735.209653 +3465735.9028 +SELECT round(log(2) * avg(number), 6) AS k +FROM numbers(10000000) +GROUP BY number % 3, number % 2 +SETTINGS allow_experimental_analyzer=0; +3465734.516505 +3465735.209653 +3465735.9028 +3465736.595947 +3465735.209653 +3465735.9028 diff --git a/tests/queries/0_stateless/02481_optimize_aggregation_arithmetics.sql b/tests/queries/0_stateless/02481_optimize_aggregation_arithmetics.sql new file mode 100644 index 00000000000..5fec5a79813 --- /dev/null +++ b/tests/queries/0_stateless/02481_optimize_aggregation_arithmetics.sql @@ -0,0 +1,26 @@ +-- { echoOn } +EXPLAIN actions=1 + ( + SELECT round(avg(log(2) * number), 6) AS k + FROM numbers(10000000) + GROUP BY number % 3, number % 2 + ) +SETTINGS allow_experimental_analyzer=1; + +EXPLAIN actions=1 + ( + SELECT round(log(2) * avg(number), 6) AS k + FROM numbers(10000000) + GROUP BY number % 3, number % 2 + ) +SETTINGS allow_experimental_analyzer=1; + +SELECT round(avg(log(2) * number), 6) AS k +FROM numbers(10000000) +GROUP BY number % 3, number % 2 +SETTINGS allow_experimental_analyzer=1; + +SELECT round(log(2) * avg(number), 6) AS k +FROM numbers(10000000) +GROUP BY number % 3, number % 2 +SETTINGS allow_experimental_analyzer=0; From cdc3912743bea2022aa9e1b6d482d1685af38c6f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 20 Dec 2022 22:44:27 +0100 Subject: [PATCH 02/49] fix incorrect usages of getPartName() --- .../BackupCoordinationReplicatedTables.cpp | 4 +-- src/Storages/MergeTree/ActiveDataPartSet.cpp | 4 +-- src/Storages/MergeTree/DropPartsRanges.cpp | 2 +- .../MergeTree/FutureMergedMutatedPart.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 26 +++++++++---------- .../MergeTree/MergeTreeDataWriter.cpp | 2 +- .../MergeTree/MergeTreeDeduplicationLog.cpp | 4 +-- src/Storages/MergeTree/MergeTreePartInfo.cpp | 20 +++++++++++++- src/Storages/MergeTree/MergeTreePartInfo.h | 4 ++- src/Storages/MergeTree/MergeTreeSink.cpp | 2 +- .../PartMovesBetweenShardsOrchestrator.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 15 ++++++----- 14 files changed, 57 insertions(+), 34 deletions(-) diff --git a/src/Backups/BackupCoordinationReplicatedTables.cpp b/src/Backups/BackupCoordinationReplicatedTables.cpp index 910719b5365..27977445641 100644 --- a/src/Backups/BackupCoordinationReplicatedTables.cpp +++ b/src/Backups/BackupCoordinationReplicatedTables.cpp @@ -78,9 +78,9 @@ public: throw Exception( ErrorCodes::CANNOT_BACKUP_TABLE, "Intersected parts detected: {} on replica {} and {} on replica {}", - part.info.getPartName(), + part.info.getPartNameForLogs(), *part.replica_name, - new_part_info.getPartName(), + new_part_info.getPartNameForLogs(), *replica_name); } ++last_it; diff --git a/src/Storages/MergeTree/ActiveDataPartSet.cpp b/src/Storages/MergeTree/ActiveDataPartSet.cpp index 13976f6ec45..b438f18c1b8 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -47,7 +47,7 @@ bool ActiveDataPartSet::addImpl(const MergeTreePartInfo & part_info, const Strin if (!part_info.contains(it->first)) { if (!part_info.isDisjoint(it->first)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} intersects previous part {}. It is a bug or a result of manual intervention in the ZooKeeper data.", name, it->first.getPartName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} intersects previous part {}. It is a bug or a result of manual intervention in the ZooKeeper data.", name, it->first.getPartNameForLogs()); ++it; break; } @@ -70,7 +70,7 @@ bool ActiveDataPartSet::addImpl(const MergeTreePartInfo & part_info, const Strin } if (it != part_info_to_name.end() && !part_info.isDisjoint(it->first)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} intersects part {}. It is a bug or a result of manual intervention in the ZooKeeper data.", name, it->first.getPartName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} intersects part {}. It is a bug or a result of manual intervention in the ZooKeeper data.", name, it->first.getPartNameForLogs()); part_info_to_name.emplace(part_info, name); return true; diff --git a/src/Storages/MergeTree/DropPartsRanges.cpp b/src/Storages/MergeTree/DropPartsRanges.cpp index d467a7cac3d..bc4f20a3471 100644 --- a/src/Storages/MergeTree/DropPartsRanges.cpp +++ b/src/Storages/MergeTree/DropPartsRanges.cpp @@ -19,7 +19,7 @@ bool DropPartsRanges::isAffectedByDropRange(const std::string & new_part_name, s { if (!drop_range.isDisjoint(entry_info)) { - postpone_reason = fmt::format("Has DROP RANGE affecting entry {} producing part {}. Will postpone it's execution.", drop_range.getPartName(), new_part_name); + postpone_reason = fmt::format("Has DROP RANGE affecting entry {} producing part {}. Will postpone it's execution.", drop_range.getPartNameForLogs(), new_part_name); return true; } } diff --git a/src/Storages/MergeTree/FutureMergedMutatedPart.cpp b/src/Storages/MergeTree/FutureMergedMutatedPart.cpp index 019b24f6916..ffd444b7135 100644 --- a/src/Storages/MergeTree/FutureMergedMutatedPart.cpp +++ b/src/Storages/MergeTree/FutureMergedMutatedPart.cpp @@ -81,7 +81,7 @@ void FutureMergedMutatedPart::assign(MergeTreeData::DataPartsVector parts_, Merg name = part_info.getPartNameV0(min_date, max_date); } else - name = part_info.getPartName(); + name = part_info.getPartNameV1(); } void FutureMergedMutatedPart::updatePath(const MergeTreeData & storage, const IReservation * reservation) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 02a7a2ae641..3ed5dc4a710 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -375,7 +375,7 @@ String IMergeTreeDataPart::getNewName(const MergeTreePartInfo & new_part_info) c return new_part_info.getPartNameV0(min_date, max_date); } else - return new_part_info.getPartName(); + return new_part_info.getPartNameV1(); } std::optional IMergeTreeDataPart::getColumnPosition(const String & column_name) const diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b92fada821c..214f0fc2e77 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2034,7 +2034,7 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t MergeTreePartInfo range_info = part->info; range_info.level = static_cast(range_info.max_block - range_info.min_block); range_info.mutation = 0; - independent_ranges_set.addImpl(range_info, range_info.getPartName()); + independent_ranges_set.addImpl(range_info, range_info.getPartNameV1()); } auto independent_ranges_infos = independent_ranges_set.getPartInfos(); @@ -2050,7 +2050,7 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t parts_in_range.push_back(part); sum_of_ranges += parts_in_range.size(); - LOG_TRACE(log, "Scheduling removal of {} parts in blocks range {}", parts_in_range.size(), range.getPartName()); + LOG_TRACE(log, "Scheduling removal of {} parts in blocks range {}", parts_in_range.size(), range.getPartNameForLogs()); pool.scheduleOrThrowOnError([&part_names_mutex, part_names_succeed, thread_group = CurrentThread::getGroup(), batch = std::move(parts_in_range)] { @@ -3335,7 +3335,7 @@ MergeTreeData::PartsToRemoveFromZooKeeper MergeTreeData::removePartsInRangeFromW DataPartsVector parts_to_remove; if (drop_range.min_block > drop_range.max_block) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid drop range: {}", drop_range.getPartName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid drop range: {}", drop_range.getPartNameForLogs()); auto partition_range = getVisibleDataPartsVectorInPartition(txn, drop_range.partition_id, &lock); @@ -3367,7 +3367,7 @@ MergeTreeData::PartsToRemoveFromZooKeeper MergeTreeData::removePartsInRangeFromW bool is_covered_by_min_max_block = part->info.min_block <= drop_range.min_block && part->info.max_block >= drop_range.max_block && part->info.getMutationVersion() >= drop_range.getMutationVersion(); if (is_covered_by_min_max_block) { - LOG_INFO(log, "Skipping drop range for part {} because covering part {} already exists", drop_range.getPartName(), part->name); + LOG_INFO(log, "Skipping drop range for part {} because covering part {} already exists", drop_range.getPartNameForLogs(), part->name); return {}; } } @@ -3378,7 +3378,7 @@ MergeTreeData::PartsToRemoveFromZooKeeper MergeTreeData::removePartsInRangeFromW { /// Intersect left border throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected merged part {} intersecting drop range {}", - part->name, drop_range.getPartName()); + part->name, drop_range.getPartNameForLogs()); } continue; @@ -3392,7 +3392,7 @@ MergeTreeData::PartsToRemoveFromZooKeeper MergeTreeData::removePartsInRangeFromW { /// Intersect right border throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected merged part {} intersecting drop range {}", - part->name, drop_range.getPartName()); + part->name, drop_range.getPartNameForLogs()); } parts_to_remove.emplace_back(part); @@ -4150,8 +4150,8 @@ void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String & { auto part_info = MergeTreePartInfo::fromPartName(partition_id, format_version); parts.push_back(getActiveContainingPart(part_info)); - if (!parts.back() || parts.back()->name != part_info.getPartName()) - throw Exception("Part " + partition_id + " is not exists or not active", ErrorCodes::NO_SUCH_DATA_PART); + if (!parts.back() || parts.back()->name != part_info.getPartNameAndCheckFormat(format_version)) + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "Part {} is not exists or not active", partition_id); } else parts = getVisibleDataPartsVectorInPartition(local_context, partition_id); @@ -4192,18 +4192,18 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String { auto part_info = MergeTreePartInfo::fromPartName(partition_id, format_version); parts.emplace_back(getActiveContainingPart(part_info)); - if (!parts.back() || parts.back()->name != part_info.getPartName()) - throw Exception("Part " + partition_id + " is not exists or not active", ErrorCodes::NO_SUCH_DATA_PART); + if (!parts.back() || parts.back()->name != part_info.getPartNameAndCheckFormat(format_version)) + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "Part {} is not exists or not active", partition_id); } else parts = getVisibleDataPartsVectorInPartition(local_context, partition_id); auto volume = getStoragePolicy()->getVolumeByName(name); if (!volume) - throw Exception("Volume " + name + " does not exists on policy " + getStoragePolicy()->getName(), ErrorCodes::UNKNOWN_DISK); + throw Exception(ErrorCodes::UNKNOWN_DISK, "Volume {} does not exists on policy {}", name, getStoragePolicy()->getName()); if (parts.empty()) - throw Exception("Nothing to move (check that the partition exists).", ErrorCodes::NO_SUCH_DATA_PART); + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "Nothing to move (check that the partition exists)."); std::erase_if(parts, [&](auto part_ptr) { @@ -4570,7 +4570,7 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const void MergeTreeData::restorePartFromBackup(std::shared_ptr restored_parts_holder, const MergeTreePartInfo & part_info, const String & part_path_in_backup) const { - String part_name = part_info.getPartName(); + String part_name = part_info.getPartNameAndCheckFormat(format_version); auto backup = restored_parts_holder->getBackup(); UInt64 total_size_of_part = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index c50c01ea356..7d664e69703 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -362,7 +362,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart( part_name = new_part_info.getPartNameV0(min_date, max_date); } else - part_name = new_part_info.getPartName(); + part_name = new_part_info.getPartNameV1(); String part_dir = TMP_PREFIX + part_name; temp_part.temporary_directory_lock = data.getTemporaryPartDirectoryHolder(part_dir); diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index d0f4d8b3604..b843ce6a078 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -232,7 +232,7 @@ std::pair MergeTreeDeduplicationLog::addPart(const std: /// Create new record MergeTreeDeduplicationLogRecord record; record.operation = MergeTreeDeduplicationOp::ADD; - record.part_name = part_info.getPartName(); + record.part_name = part_info.getPartNameAndCheckFormat(format_version); record.block_id = block_id; /// Write it to disk writeRecord(record, *current_writer); @@ -269,7 +269,7 @@ void MergeTreeDeduplicationLog::dropPart(const MergeTreePartInfo & drop_part_inf /// Create drop record MergeTreeDeduplicationLogRecord record; record.operation = MergeTreeDeduplicationOp::DROP; - record.part_name = part_info.getPartName(); + record.part_name = part_info.getPartNameAndCheckFormat(format_version); record.block_id = itr->key; /// Write it to disk writeRecord(record, *current_writer); diff --git a/src/Storages/MergeTree/MergeTreePartInfo.cpp b/src/Storages/MergeTree/MergeTreePartInfo.cpp index 8c518e4d17f..038bf26ca91 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -167,7 +167,25 @@ bool MergeTreePartInfo::contains(const String & outer_part_name, const String & } -String MergeTreePartInfo::getPartName() const +String MergeTreePartInfo::getPartNameAndCheckFormat(MergeTreeDataFormatVersion format_version) const +{ + if (format_version == MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + getPartNameV1(); + + /// We cannot just call getPartNameV0 because it requires extra arguments, but at least we can warn about it. + chassert(false); /// Catch it in CI. Feel free to remove this line. + throw Exception(ErrorCodes::BAD_DATA_PART_NAME, "Trying to get part name in new format for old format version." + "Either some new feature is incompatible with deprecated *MergeTree definition syntax or it's a bug."); +} + + +String MergeTreePartInfo::getPartNameForLogs() const +{ + /// We don't care about format version here + return getPartNameV1(); +} + +String MergeTreePartInfo::getPartNameV1() const { WriteBufferFromOwnString wb; diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index 60c7e4e8822..cad851fb882 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -103,7 +103,9 @@ struct MergeTreePartInfo return level == MergeTreePartInfo::MAX_LEVEL || level == another_max_level; } - String getPartName() const; + String getPartNameAndCheckFormat(MergeTreeDataFormatVersion format_version) const; + String getPartNameForLogs() const; + String getPartNameV1() const; String getPartNameV0(DayNum left_date, DayNum right_date) const; UInt64 getBlocksCount() const { diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 0de71e94ea8..99f6b1855e4 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -156,7 +156,7 @@ void MergeTreeSink::finishDelayedChunk() if (!res.second) { ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks); - LOG_INFO(storage.log, "Block with ID {} already exists as part {}; ignoring it", block_id, res.first.getPartName()); + LOG_INFO(storage.log, "Block with ID {} already exists as part {}; ignoring it", block_id, res.first.getPartNameForLogs()); continue; } } diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index d5f35ea1b3c..560d9f17a07 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -473,7 +473,7 @@ PartMovesBetweenShardsOrchestrator::Entry PartMovesBetweenShardsOrchestrator::st log_entry.log_entry_id = attach_log_entry_barrier_path; log_entry.part_checksum = part->checksums.getTotalChecksumHex(); log_entry.create_time = std::time(nullptr); - log_entry.new_part_name = part_info.getPartName(); + log_entry.new_part_name = part_info.getPartNameAndCheckFormat(storage.format_version); ops.emplace_back(zkutil::makeCreateRequest(attach_log_entry_barrier_path, log_entry.toString(), -1)); ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 9ff022d5d57..d6d74228d68 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1504,7 +1504,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( entry.znode_name, entry.typeToString(), entry.new_part_name, - info.getPartName()); + info.getPartNameForLogs()); LOG_TRACE(log, fmt::runtime(out_postpone_reason)); return false; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8c422b52a3c..79a8253a9de 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1484,8 +1484,11 @@ String StorageReplicatedMergeTree::getChecksumsForZooKeeper(const MergeTreeDataP MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFoundValidPart(const LogEntry& entry) const { + if (format_version != MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + return {}; + const MergeTreePartInfo actual_part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version); - const String part_new_name = actual_part_info.getPartName(); + const String part_new_name = actual_part_info.getPartNameV1(); for (const DiskPtr & disk : getStoragePolicy()->getDisks()) { @@ -1496,7 +1499,7 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo if (!part_info || part_info->partition_id != actual_part_info.partition_id) continue; - const String part_old_name = part_info->getPartName(); + const String part_old_name = part_info->getPartNameV1(); const VolumePtr volume = std::make_shared("volume_" + part_old_name, disk); @@ -3849,7 +3852,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) bool StorageReplicatedMergeTree::partIsInsertingWithParallelQuorum(const MergeTreePartInfo & part_info) const { auto zookeeper = getZooKeeper(); - return zookeeper->exists(fs::path(zookeeper_path) / "quorum" / "parallel" / part_info.getPartName()); + return zookeeper->exists(fs::path(zookeeper_path) / "quorum" / "parallel" / part_info.getPartNameAndCheckFormat(format_version)); } @@ -3871,7 +3874,7 @@ bool StorageReplicatedMergeTree::partIsLastQuorumPart(const MergeTreePartInfo & if (partition_it == parts_with_quorum.added_parts.end()) return false; - return partition_it->second == part_info.getPartName(); + return partition_it->second == part_info.getPartNameAndCheckFormat(format_version); } @@ -5048,7 +5051,7 @@ String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const return part_info.getPartNameV0(left_date, right_date); } - return part_info.getPartName(); + return part_info.getPartNameV1(); } bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const String & partition_id, MergeTreePartInfo & part_info, @@ -7540,7 +7543,7 @@ void StorageReplicatedMergeTree::enqueuePartForCheck(const String & part_name, t if (queue.hasDropRange(MergeTreePartInfo::fromPartName(part_name, format_version), &covering_drop_range)) { LOG_WARNING(log, "Do not enqueue part {} for check because it's covered by DROP_RANGE {} and going to be removed", - part_name, covering_drop_range.getPartName()); + part_name, covering_drop_range.getPartNameForLogs()); return; } part_check_thread.enqueuePart(part_name, delay_to_check_seconds); From 1ce69371fb5da17528ed4655e76841b9004caea4 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 28 Dec 2022 21:46:08 +0000 Subject: [PATCH 03/49] Infer UInt64 in case of Int64 overflow --- src/DataTypes/transformTypesRecursively.cpp | 4 +- src/DataTypes/transformTypesRecursively.h | 2 +- src/Formats/JSONUtils.cpp | 4 + src/Formats/SchemaInferenceUtils.cpp | 146 +++++++++++++++--- ...uint64_in_case_of_int64_overflow.reference | 12 ++ ..._infer_uint64_in_case_of_int64_overflow.sh | 18 +++ 6 files changed, 160 insertions(+), 26 deletions(-) create mode 100644 tests/queries/0_stateless/02517_infer_uint64_in_case_of_int64_overflow.reference create mode 100755 tests/queries/0_stateless/02517_infer_uint64_in_case_of_int64_overflow.sh diff --git a/src/DataTypes/transformTypesRecursively.cpp b/src/DataTypes/transformTypesRecursively.cpp index da3af0beee7..05f82a08abe 100644 --- a/src/DataTypes/transformTypesRecursively.cpp +++ b/src/DataTypes/transformTypesRecursively.cpp @@ -8,7 +8,7 @@ namespace DB { -void transformTypesRecursively(DataTypes & types, std::function transform_simple_types, std::function transform_complex_types) +void transformTypesRecursively(DataTypes & types, std::function transform_simple_types, std::function transform_complex_types) { TypeIndexesSet type_indexes; for (const auto & type : types) @@ -156,7 +156,7 @@ void transformTypesRecursively(DataTypes & types, std::function callback) { DataTypes types = {type}; - transformTypesRecursively(types, [callback](auto & data_types, const TypeIndexesSet &){ callback(data_types[0]); }, {}); + transformTypesRecursively(types, [callback](auto & data_types, TypeIndexesSet &){ callback(data_types[0]); }, {}); } } diff --git a/src/DataTypes/transformTypesRecursively.h b/src/DataTypes/transformTypesRecursively.h index 2cf8664f920..f9c776b4205 100644 --- a/src/DataTypes/transformTypesRecursively.h +++ b/src/DataTypes/transformTypesRecursively.h @@ -12,7 +12,7 @@ namespace DB /// If not all types are the same complex type (Array/Map/Tuple), this function won't be called to nested types. /// Function transform_simple_types will be applied to resulting simple types after all recursive calls. /// Function transform_complex_types will be applied to complex types (Array/Map/Tuple) after recursive call to their nested types. -void transformTypesRecursively(DataTypes & types, std::function transform_simple_types, std::function transform_complex_types); +void transformTypesRecursively(DataTypes & types, std::function transform_simple_types, std::function transform_complex_types); void callOnNestedSimpleTypes(DataTypePtr & type, std::function callback); diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index 16f275ed6b8..384619dba1d 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -131,6 +131,7 @@ namespace JSONUtils { skipWhitespaceIfAny(in); assertChar('{', in); + skipWhitespaceIfAny(in); bool first = true; NamesAndTypesList names_and_types; String field; @@ -144,6 +145,7 @@ namespace JSONUtils auto name = readFieldName(in); auto type = tryInferDataTypeForSingleJSONField(in, settings, inference_info); names_and_types.emplace_back(name, type); + skipWhitespaceIfAny(in); } if (in.eof()) @@ -157,6 +159,7 @@ namespace JSONUtils { skipWhitespaceIfAny(in); assertChar('[', in); + skipWhitespaceIfAny(in); bool first = true; DataTypes types; String field; @@ -168,6 +171,7 @@ namespace JSONUtils first = false; auto type = tryInferDataTypeForSingleJSONField(in, settings, inference_info); types.push_back(std::move(type)); + skipWhitespaceIfAny(in); } if (in.eof()) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 9d40ac98964..3d00f67884d 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -44,9 +44,16 @@ namespace return true; } + void updateTypeIndexes(DataTypes & data_types, TypeIndexesSet & type_indexes) + { + type_indexes.clear(); + for (const auto & type : data_types) + type_indexes.insert(type->getTypeId()); + } + /// If we have both Nothing and non Nothing types, convert all Nothing types to the first non Nothing. /// For example if we have types [Nothing, String, Nothing] we change it to [String, String, String] - void transformNothingSimpleTypes(DataTypes & data_types, const TypeIndexesSet & type_indexes) + void transformNothingSimpleTypes(DataTypes & data_types, TypeIndexesSet & type_indexes) { /// Check if we have both Nothing and non Nothing types. if (!type_indexes.contains(TypeIndex::Nothing) || type_indexes.size() <= 1) @@ -67,12 +74,32 @@ namespace if (isNothing(type)) type = not_nothing_type; } + + type_indexes.erase(TypeIndex::Nothing); + } + + /// If we have both Int64 and UInt64, convert all Int64 to UInt64, + /// because UInt64 is inferred only in case of Int64 overflow. + void transformIntegers(DataTypes & data_types, TypeIndexesSet & type_indexes) + { + if (!type_indexes.contains(TypeIndex::Int64) || !type_indexes.contains(TypeIndex::UInt64)) + return; + + for (auto & type : data_types) + { + if (WhichDataType(type).isInt64()) + type = std::make_shared(); + } + + type_indexes.erase(TypeIndex::Int64); } /// If we have both Int64 and Float64 types, convert all Int64 to Float64. - void transformIntegersAndFloatsToFloats(DataTypes & data_types, const TypeIndexesSet & type_indexes) + void transformIntegersAndFloatsToFloats(DataTypes & data_types, TypeIndexesSet & type_indexes) { - if (!type_indexes.contains(TypeIndex::Int64) || !type_indexes.contains(TypeIndex::Float64)) + bool have_floats = type_indexes.contains(TypeIndex::Float64); + bool have_integers = type_indexes.contains(TypeIndex::Int64) || type_indexes.contains(TypeIndex::UInt64); + if (!have_integers || !have_floats) return; for (auto & type : data_types) @@ -80,11 +107,14 @@ namespace if (isInteger(type)) type = std::make_shared(); } + + type_indexes.erase(TypeIndex::Int64); + type_indexes.erase(TypeIndex::UInt64); } /// If we have only Date and DateTime types, convert Date to DateTime, /// otherwise, convert all Date and DateTime to String. - void transformDatesAndDateTimes(DataTypes & data_types, const TypeIndexesSet & type_indexes) + void transformDatesAndDateTimes(DataTypes & data_types, TypeIndexesSet & type_indexes) { bool have_dates = type_indexes.contains(TypeIndex::Date); bool have_datetimes = type_indexes.contains(TypeIndex::DateTime64); @@ -98,6 +128,8 @@ namespace type = std::make_shared(); } + type_indexes.erase(TypeIndex::Date); + type_indexes.erase(TypeIndex::DateTime); return; } @@ -108,16 +140,18 @@ namespace if (isDate(type)) type = std::make_shared(9); } + + type_indexes.erase(TypeIndex::Date); } } - /// If we have numbers (Int64/Float64) and String types and numbers were parsed from String, + /// If we have numbers (Int64/UInt64/Float64) and String types and numbers were parsed from String, /// convert all numbers to String. void transformJSONNumbersBackToString( - DataTypes & data_types, const FormatSettings & settings, const TypeIndexesSet & type_indexes, JSONInferenceInfo * json_info) + DataTypes & data_types, const FormatSettings & settings, TypeIndexesSet & type_indexes, JSONInferenceInfo * json_info) { bool have_strings = type_indexes.contains(TypeIndex::String); - bool have_numbers = type_indexes.contains(TypeIndex::Int64) || type_indexes.contains(TypeIndex::Float64); + bool have_numbers = type_indexes.contains(TypeIndex::Int64) || type_indexes.contains(TypeIndex::UInt64) || type_indexes.contains(TypeIndex::Float64); if (!have_strings || !have_numbers) return; @@ -128,36 +162,43 @@ namespace || json_info->numbers_parsed_from_json_strings.contains(type.get()))) type = std::make_shared(); } + + updateTypeIndexes(data_types, type_indexes); } - /// If we have both Bool and number (Int64/Float64) types, - /// convert all Bool to Int64/Float64. - void transformBoolsAndNumbersToNumbers(DataTypes & data_types, const TypeIndexesSet & type_indexes) + /// If we have both Bool and number (Int64/UInt64/Float64) types, + /// convert all Bool to Int64/UInt64/Float64. + void transformBoolsAndNumbersToNumbers(DataTypes & data_types, TypeIndexesSet & type_indexes) { bool have_floats = type_indexes.contains(TypeIndex::Float64); - bool have_integers = type_indexes.contains(TypeIndex::Int64); + bool have_signed_integers = type_indexes.contains(TypeIndex::Int64); + bool have_unsigned_integers = type_indexes.contains(TypeIndex::UInt64); bool have_bools = type_indexes.contains(TypeIndex::UInt8); /// Check if we have both Bool and Integer/Float. - if (!have_bools || (!have_integers && !have_floats)) + if (!have_bools || (!have_signed_integers && !have_unsigned_integers && !have_floats)) return; for (auto & type : data_types) { if (isBool(type)) { - if (have_integers) + if (have_signed_integers) type = std::make_shared(); + else if (have_unsigned_integers) + type = std::make_shared(); else type = std::make_shared(); } } + + type_indexes.erase(TypeIndex::UInt8); } /// If we have type Nothing/Nullable(Nothing) and some other non Nothing types, /// convert all Nothing/Nullable(Nothing) types to the first non Nothing. /// For example, when we have [Nothing, Array(Int64)] it will convert it to [Array(Int64), Array(Int64)] /// (it can happen when transforming complex nested types like [Array(Nothing), Array(Array(Int64))]) - void transformNothingComplexTypes(DataTypes & data_types) + void transformNothingComplexTypes(DataTypes & data_types, TypeIndexesSet & type_indexes) { bool have_nothing = false; DataTypePtr not_nothing_type = nullptr; @@ -177,10 +218,12 @@ namespace if (isNothing(removeNullable(type))) type = not_nothing_type; } + + updateTypeIndexes(data_types, type_indexes); } /// If we have both Nullable and non Nullable types, make all types Nullable - void transformNullableTypes(DataTypes & data_types, const TypeIndexesSet & type_indexes) + void transformNullableTypes(DataTypes & data_types, TypeIndexesSet & type_indexes) { if (!type_indexes.contains(TypeIndex::Nullable)) return; @@ -190,6 +233,8 @@ namespace if (type->canBeInsideNullable()) type = makeNullable(type); } + + updateTypeIndexes(data_types, type_indexes); } /// If we have Tuple with the same nested types like Tuple(Int64, Int64), @@ -197,11 +242,12 @@ namespace /// For example when we had type Tuple(Int64, Nullable(Nothing)) and we /// transformed it to Tuple(Nullable(Int64), Nullable(Int64)) we will /// also transform it to Array(Nullable(Int64)) - void transformTuplesWithEqualNestedTypesToArrays(DataTypes & data_types, const TypeIndexesSet & type_indexes) + void transformTuplesWithEqualNestedTypesToArrays(DataTypes & data_types, TypeIndexesSet & type_indexes) { if (!type_indexes.contains(TypeIndex::Tuple)) return; + bool remove_tuple_index = true; for (auto & type : data_types) { if (isTuple(type)) @@ -209,8 +255,13 @@ namespace const auto * tuple_type = assert_cast(type.get()); if (checkIfTypesAreEqual(tuple_type->getElements())) type = std::make_shared(tuple_type->getElements().back()); + else + remove_tuple_index = false; } } + + if (remove_tuple_index) + type_indexes.erase(TypeIndex::Tuple); } template @@ -221,7 +272,7 @@ namespace /// For example, if we have [Tuple(Nullable(Nothing), String), Array(Date), Tuple(Date, String)] /// it will convert them all to Array(String) void transformJSONTuplesAndArraysToArrays( - DataTypes & data_types, const FormatSettings & settings, const TypeIndexesSet & type_indexes, JSONInferenceInfo * json_info) + DataTypes & data_types, const FormatSettings & settings, TypeIndexesSet & type_indexes, JSONInferenceInfo * json_info) { if (!type_indexes.contains(TypeIndex::Tuple)) return; @@ -266,12 +317,14 @@ namespace if (isArray(type) || isTuple(type)) type = std::make_shared(nested_types.back()); } + + type_indexes.erase(TypeIndex::Tuple); } } /// If we have Map and Object(JSON) types, convert all Map types to Object(JSON). /// If we have Map types with different value types, convert all Map types to Object(JSON) - void transformMapsAndObjectsToObjects(DataTypes & data_types, const TypeIndexesSet & type_indexes) + void transformMapsAndObjectsToObjects(DataTypes & data_types, TypeIndexesSet & type_indexes) { if (!type_indexes.contains(TypeIndex::Map)) return; @@ -298,9 +351,11 @@ namespace if (isMap(type)) type = std::make_shared("json", true); } + + type_indexes.erase(TypeIndex::Map); } - void transformMapsObjectsAndStringsToStrings(DataTypes & data_types, const TypeIndexesSet & type_indexes) + void transformMapsObjectsAndStringsToStrings(DataTypes & data_types, TypeIndexesSet & type_indexes) { bool have_maps = type_indexes.contains(TypeIndex::Map); bool have_objects = type_indexes.contains(TypeIndex::Object); @@ -315,19 +370,26 @@ namespace if (isMap(type) || isObject(type)) type = std::make_shared(); } + + type_indexes.erase(TypeIndex::Map); + type_indexes.erase(TypeIndex::Object); } template void transformInferredTypesIfNeededImpl(DataTypes & types, const FormatSettings & settings, JSONInferenceInfo * json_info) { - auto transform_simple_types = [&](DataTypes & data_types, const TypeIndexesSet & type_indexes) + auto transform_simple_types = [&](DataTypes & data_types, TypeIndexesSet & type_indexes) { /// Remove all Nothing type if possible. transformNothingSimpleTypes(data_types, type_indexes); - /// Transform integers to floats if needed. if (settings.try_infer_integers) + { + /// Transform Int64 to UInt64 if needed. + transformIntegers(data_types, type_indexes); + /// Transform integers to floats if needed. transformIntegersAndFloatsToFloats(data_types, type_indexes); + } /// Transform Date to DateTime or both to String if needed. if (settings.try_infer_dates || settings.try_infer_datetimes) @@ -347,14 +409,14 @@ namespace transformBoolsAndNumbersToNumbers(data_types, type_indexes); }; - auto transform_complex_types = [&](DataTypes & data_types, const TypeIndexesSet & type_indexes) + auto transform_complex_types = [&](DataTypes & data_types, TypeIndexesSet & type_indexes) { /// Make types Nullable if needed. transformNullableTypes(data_types, type_indexes); /// If we have type Nothing, it means that we had empty Array/Map while inference. /// If there is at least one non Nothing type, change all Nothing types to it. - transformNothingComplexTypes(data_types); + transformNothingComplexTypes(data_types, type_indexes); if constexpr (!is_json) return; @@ -571,10 +633,28 @@ namespace char * int_end = buf.position(); /// We cam safely get back to the start of the number, because we read from a string and we didn't reach eof. buf.position() = number_start; + + bool read_uint = false; + char * uint_end = nullptr; + /// In case of Int64 overflow we can try to infer UInt64. + if (!read_int) + { + UInt64 tmp_uint; + read_uint = tryReadIntText(tmp_uint, buf); + /// If we reached eof, it cannot be float (it requires no less data than integer) + if (buf.eof()) + return read_uint ? std::make_shared() : nullptr; + + uint_end = buf.position(); + buf.position() = number_start; + } + if (tryReadFloatText(tmp_float, buf)) { if (read_int && buf.position() == int_end) return std::make_shared(); + if (read_uint && buf.position() == uint_end) + return std::make_shared(); return std::make_shared(); } @@ -590,6 +670,19 @@ namespace bool read_int = tryReadIntText(tmp_int, peekable_buf); auto * int_end = peekable_buf.position(); peekable_buf.rollbackToCheckpoint(true); + + bool read_uint = false; + char * uint_end = nullptr; + /// In case of Int64 overflow we can try to infer UInt64. + if (!read_int) + { + PeekableReadBufferCheckpoint new_checkpoint(peekable_buf); + UInt64 tmp_uint; + read_uint = tryReadIntText(tmp_uint, peekable_buf); + uint_end = peekable_buf.position(); + peekable_buf.rollbackToCheckpoint(true); + } + if (tryReadFloatText(tmp_float, peekable_buf)) { /// Float parsing reads no fewer bytes than integer parsing, @@ -597,6 +690,8 @@ namespace /// If it's the same, then it's integer. if (read_int && peekable_buf.position() == int_end) return std::make_shared(); + if (read_uint && peekable_buf.position() == uint_end) + return std::make_shared(); return std::make_shared(); } } @@ -874,6 +969,11 @@ DataTypePtr tryInferNumberFromString(std::string_view field, const FormatSetting Int64 tmp_int; if (tryReadIntText(tmp_int, buf) && buf.eof()) return std::make_shared(); + + /// In case of Int64 overflow, try to infer UInt64 + UInt64 tmp_uint; + if (tryReadIntText(tmp_uint, buf) && buf.eof()) + return std::make_shared(); } /// We cam safely get back to the start of buffer, because we read from a string and we didn't reach eof. diff --git a/tests/queries/0_stateless/02517_infer_uint64_in_case_of_int64_overflow.reference b/tests/queries/0_stateless/02517_infer_uint64_in_case_of_int64_overflow.reference new file mode 100644 index 00000000000..96a50d75eee --- /dev/null +++ b/tests/queries/0_stateless/02517_infer_uint64_in_case_of_int64_overflow.reference @@ -0,0 +1,12 @@ +c1 Nullable(UInt64) +c1 Array(Nullable(UInt64)) +c1 Nullable(UInt64) +c1 Nullable(UInt64) +c1 Array(Nullable(UInt64)) +c1 Nullable(UInt64) +number Nullable(UInt64) +number Array(Nullable(UInt64)) +number Array(Nullable(UInt64)) +number Nullable(UInt64) +number Nullable(UInt64) +number Nullable(UInt64) diff --git a/tests/queries/0_stateless/02517_infer_uint64_in_case_of_int64_overflow.sh b/tests/queries/0_stateless/02517_infer_uint64_in_case_of_int64_overflow.sh new file mode 100755 index 00000000000..4019d2b7a78 --- /dev/null +++ b/tests/queries/0_stateless/02517_infer_uint64_in_case_of_int64_overflow.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo -ne "18446744073709551615" | $CLICKHOUSE_LOCAL --table=test --input-format=CSV -q "desc test"; +echo -ne '"[18446744073709551615, 10, 11]"' | $CLICKHOUSE_LOCAL --table=test --input-format=CSV -q "desc test"; +echo -ne "18446744073709551615\n10\n11" | $CLICKHOUSE_LOCAL --table=test --input-format=CSV -q "desc test"; +echo -ne "18446744073709551615" | $CLICKHOUSE_LOCAL --table=test --input-format=TSV -q "desc test"; +echo -ne "[18446744073709551615, 10, 11]" | $CLICKHOUSE_LOCAL --table=test --input-format=TSV -q "desc test"; +echo -ne "18446744073709551615\n10\n11" | $CLICKHOUSE_LOCAL --table=test --input-format=TSV -q "desc test"; +echo -ne '{"number" : 18446744073709551615}' | $CLICKHOUSE_LOCAL --table=test --input-format=JSONEachRow -q "desc test"; +echo -ne '{"number" : [18446744073709551615, 10, 11]}'| $CLICKHOUSE_LOCAL --table=test --input-format=JSONEachRow -q "desc test"; +echo -ne '{"number" : [18446744073709551615, true, 11]}'| $CLICKHOUSE_LOCAL --table=test --input-format=JSONEachRow -q "desc test"; +echo -ne '{"number" : 18446744073709551615}, {"number" : 10}, {"number" : 11}' | $CLICKHOUSE_LOCAL --table=test --input-format=JSONEachRow -q "desc test"; +echo -ne '{"number" : 18446744073709551615}, {"number" : false}, {"number" : 11}' | $CLICKHOUSE_LOCAL --table=test --input-format=JSONEachRow -q "desc test"; +echo -ne '{"number" : "18446744073709551615"}' | $CLICKHOUSE_LOCAL --table=test --input-format=JSONEachRow -q "desc test"; From 73fecae5ffed84ceca78f902e8f6967447b3863a Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 2 Jan 2023 15:31:07 +0000 Subject: [PATCH 04/49] Fix comments --- src/Formats/JSONUtils.cpp | 4 ++-- src/Formats/SchemaInferenceUtils.cpp | 5 +++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index 384619dba1d..574759b0c07 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -138,7 +138,7 @@ namespace JSONUtils while (!in.eof() && *in.position() != '}') { if (!first) - skipComma(in); + assertChar(',', in); else first = false; @@ -166,7 +166,7 @@ namespace JSONUtils while (!in.eof() && *in.position() != ']') { if (!first) - skipComma(in); + assertChar(',', in); else first = false; auto type = tryInferDataTypeForSingleJSONField(in, settings, inference_info); diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 4bcbae1e9ea..77ef2e8f27a 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -104,7 +104,8 @@ namespace for (auto & type : data_types) { - if (isInteger(type)) + WhichDataType which(type); + if (which.isFloat64() || which.isInt64() || which.isUInt64()) type = std::make_shared(); } @@ -631,7 +632,7 @@ namespace return read_int ? std::make_shared() : nullptr; char * int_end = buf.position(); - /// We cam safely get back to the start of the number, because we read from a string and we didn't reach eof. + /// We can safely get back to the start of the number, because we read from a string and we didn't reach eof. buf.position() = number_start; bool read_uint = false; From 6f0c0252f03dc504b454c6a36d792ea1f4da2363 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 4 Jan 2023 14:55:32 +0000 Subject: [PATCH 05/49] Fix tests --- tests/queries/0_stateless/02500_numbers_inference.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02500_numbers_inference.reference b/tests/queries/0_stateless/02500_numbers_inference.reference index bff7211f66a..7e1bb6510bb 100644 --- a/tests/queries/0_stateless/02500_numbers_inference.reference +++ b/tests/queries/0_stateless/02500_numbers_inference.reference @@ -16,5 +16,5 @@ c1 Nullable(Float64) c1 Nullable(Float64) c1 Array(Nullable(Float64)) c1 Array(Nullable(Float64)) -c1 Array(Nullable(Float64)) -c1 Array(Nullable(Float64)) +c1 Nullable(String) +c1 Nullable(String) From 1f3d75cbf256c493b248dcddfb729fd0e4fb55fc Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 4 Jan 2023 14:58:17 +0000 Subject: [PATCH 06/49] Better --- src/Formats/SchemaInferenceUtils.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 5db9b04a6c1..6d0853f6169 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -98,14 +98,14 @@ namespace void transformIntegersAndFloatsToFloats(DataTypes & data_types, TypeIndexesSet & type_indexes) { bool have_floats = type_indexes.contains(TypeIndex::Float64); - bool have_integers = type_indexes.contains(TypeIndex::Int64) || type_indexes.contains(TypeIndex::UInt64); + bool have_integers = type_indexes.contains(TypeIndex::Int64) || type_indexes.contains(TypeIndex::UInt64); if (!have_integers || !have_floats) return; for (auto & type : data_types) { WhichDataType which(type); - if (which.isFloat64() || which.isInt64() || which.isUInt64()) + if (which.isInt64() || which.isUInt64()) type = std::make_shared(); } From 5cde7762ad0574c2c01d8f0e988c3f2f63ba27a0 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 5 Jan 2023 15:54:44 -0500 Subject: [PATCH 07/49] WIP --- .../table-engines/integrations/deltalake.md | 33 ++++ .../table-functions/deltalake.md | 184 ++++++++++++++++++ 2 files changed, 217 insertions(+) create mode 100644 docs/en/engines/table-engines/integrations/deltalake.md create mode 100644 docs/en/sql-reference/table-functions/deltalake.md diff --git a/docs/en/engines/table-engines/integrations/deltalake.md b/docs/en/engines/table-engines/integrations/deltalake.md new file mode 100644 index 00000000000..44407e34e38 --- /dev/null +++ b/docs/en/engines/table-engines/integrations/deltalake.md @@ -0,0 +1,33 @@ +--- +slug: /en/engines/table-engines/integrations/deltalake +sidebar_label: DeltaLake +--- + +# DeltaLake Table Engine + +This engine provides a read-only integration with existing Delta Lake tables in Amazon S3. + +## Create Table + +Note that the Delta Lake table must already exist in S3, this command does not take DDL parameters to create a new table. + +``` sql +CREATE TABLE deltalake + ENGINE = DeltaLake(path, [aws_access_key_id, aws_secret_access_key,]) +``` + +**Engine parameters** + +- `path` — Bucket url with path to the existing Delta Lake table. +- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). + +**Example** + +```sql +CREATE TABLE deltalake ENGINE=DeltaLake('http://mars-doc-test.s3.amazonaws.com/clickhouse-bucket-3/test_table/', 'ABC123', 'Abc+123') +``` + +## See also + +- [DeltaLake table function](../../../sql-reference/table-functions/deltalake.md) + diff --git a/docs/en/sql-reference/table-functions/deltalake.md b/docs/en/sql-reference/table-functions/deltalake.md new file mode 100644 index 00000000000..7e3fffe4d8b --- /dev/null +++ b/docs/en/sql-reference/table-functions/deltalake.md @@ -0,0 +1,184 @@ +--- +slug: /en/sql-reference/table-functions/deltalake +sidebar_label: DeltLake +--- + +# DeltaLake Table Function + +Provides a read-only table-like interface to [Delta Lake](https://github.com/delta-io/delta) tables in [Amazon S3](https://aws.amazon.com/s3/). + +For example, to query an existing Delta Lake table named `deltalake` in S3: +```sql +CREATE TABLE dl_hits + ENGINE = DeltaLake('https://clickhouse-public-datasets.s3.amazonaws.com/delta_lake/hits/','',''); + +SHOW TABLES; + +DESCRIBE dl_hits; + +SELECT URL, Referer, UserAgent FROM dl_hits WHERE URL IS NOT NULL LIMIT 10; + +SELECT URL, Referer, UserAgent FROM deltaLake('https://clickhouse-public-datasets.s3.amazonaws.com/delta_lake/hits/') WHERE URL IS NOT NULL LIMIT 10; + +``` + +**Syntax** + +``` sql +s3(path [,aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,compression]) +``` + +**Arguments** + +- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [here](../../engines/table-engines/integrations/s3.md#wildcards-in-path). +- `format` — The [format](../../interfaces/formats.md#formats) of the file. +- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. +- `compression` — Parameter is optional. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. By default, it will autodetect compression by file extension. + +**Returned value** + +A table with the specified structure for reading or writing data in the specified file. + +**Examples** + +Selecting the first two rows from the table from S3 file `https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/data.csv`: + +``` sql +SELECT * +FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/data.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') +LIMIT 2; +``` + +``` text +┌─column1─┬─column2─┬─column3─┐ +│ 1 │ 2 │ 3 │ +│ 3 │ 2 │ 1 │ +└─────────┴─────────┴─────────┘ +``` + +The similar but from file with `gzip` compression: + +``` sql +SELECT * +FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/data.csv.gz', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32', 'gzip') +LIMIT 2; +``` + +``` text +┌─column1─┬─column2─┬─column3─┐ +│ 1 │ 2 │ 3 │ +│ 3 │ 2 │ 1 │ +└─────────┴─────────┴─────────┘ +``` + +## Usage + +Suppose that we have several files with following URIs on S3: + +- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/some_prefix/some_file_1.csv' +- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/some_prefix/some_file_2.csv' +- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/some_prefix/some_file_3.csv' +- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/some_prefix/some_file_4.csv' +- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/another_prefix/some_file_1.csv' +- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/another_prefix/some_file_2.csv' +- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/another_prefix/some_file_3.csv' +- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/another_prefix/some_file_4.csv' + +Count the amount of rows in files ending with numbers from 1 to 3: + +``` sql +SELECT count(*) +FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/{some,another}_prefix/some_file_{1..3}.csv', 'CSV', 'name String, value UInt32') +``` + +``` text +┌─count()─┐ +│ 18 │ +└─────────┘ +``` + +Count the total amount of rows in all files in these two directories: + +``` sql +SELECT count(*) +FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/{some,another}_prefix/*', 'CSV', 'name String, value UInt32') +``` + +``` text +┌─count()─┐ +│ 24 │ +└─────────┘ +``` + +:::warning +If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. +::: + +Count the total amount of rows in files named `file-000.csv`, `file-001.csv`, … , `file-999.csv`: + +``` sql +SELECT count(*) +FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/big_prefix/file-{000..999}.csv', 'CSV', 'name String, value UInt32'); +``` + +``` text +┌─count()─┐ +│ 12 │ +└─────────┘ +``` + +Insert data into file `test-data.csv.gz`: + +``` sql +INSERT INTO FUNCTION s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip') +VALUES ('test-data', 1), ('test-data-2', 2); +``` + +Insert data into file `test-data.csv.gz` from existing table: + +``` sql +INSERT INTO FUNCTION s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip') +SELECT name, value FROM existing_table; +``` + +Glob ** can be used for recursive directory traversal. Consider the below example, it will fetch all files from `my-test-bucket-768` directory recursively: + +``` sql +SELECT * FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/**', 'CSV', 'name String, value UInt32', 'gzip'); +``` + +The below get data from all `test-data.csv.gz` files from any folder inside `my-test-bucket` directory recursively: + +``` sql +SELECT * FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/**/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip'); +``` + +## Partitioned Write + +If you specify `PARTITION BY` expression when inserting data into `S3` table, a separate file is created for each partition value. Splitting the data into separate files helps to improve reading operations efficiency. + +**Examples** + +1. Using partition ID in a key creates separate files: + +```sql +INSERT INTO TABLE FUNCTION + s3('http://bucket.amazonaws.com/my_bucket/file_{_partition_id}.csv', 'CSV', 'a String, b UInt32, c UInt32') + PARTITION BY a VALUES ('x', 2, 3), ('x', 4, 5), ('y', 11, 12), ('y', 13, 14), ('z', 21, 22), ('z', 23, 24); +``` +As a result, the data is written into three files: `file_x.csv`, `file_y.csv`, and `file_z.csv`. + +2. Using partition ID in a bucket name creates files in different buckets: + +```sql +INSERT INTO TABLE FUNCTION + s3('http://bucket.amazonaws.com/my_bucket_{_partition_id}/file.csv', 'CSV', 'a UInt32, b UInt32, c UInt32') + PARTITION BY a VALUES (1, 2, 3), (1, 4, 5), (10, 11, 12), (10, 13, 14), (20, 21, 22), (20, 23, 24); +``` +As a result, the data is written into three files in different buckets: `my_bucket_1/file.csv`, `my_bucket_10/file.csv`, and `my_bucket_20/file.csv`. + +**See Also** + +- [S3 engine](../../engines/table-engines/integrations/s3.md) + +[Original article](https://clickhouse.com/docs/en/sql-reference/table-functions/s3/) From 20c7c0b1eff01457fcc9ad4933c64f40c84341f6 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 9 Jan 2023 18:21:31 +0800 Subject: [PATCH 08/49] change as request --- .../functions/date-time-functions.md | 11 ++++++-- src/Common/DateLUTImpl.h | 26 +++++++++++++++++++ src/Functions/DateTimeTransforms.h | 20 ++++++++------ src/Functions/toDayOfWeek.cpp | 5 ++-- .../02521_to_custom_day_of_week.reference | 7 +++++ .../02521_to_custom_day_of_week.sql | 10 +++++++ 6 files changed, 67 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02521_to_custom_day_of_week.reference create mode 100644 tests/queries/0_stateless/02521_to_custom_day_of_week.sql diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 89fa72de8bf..4ff89414e0a 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -207,9 +207,16 @@ Converts a date or date with time to a UInt8 number containing the number of the Aliases: `DAYOFMONTH`, `DAY`. -## toDayOfWeek +## toDayOfWeek(date\[,mode\]) -Converts a date or date with time to a UInt8 number containing the number of the day of the week (Monday is 1, and Sunday is 7). +Converts a date or date with time to a UInt8 number containing the number of the day of the week. The two-argument form of toDayOfWeek() enables you to specify whether the week starts on Monday or Sunday, and whether the return value should be in the range from 0 to 6 or from 1-7. If the mode argument is ommited, the default mode is 0. + +| Mode | First day of week | Range | +|------|-------------------|------------------------------------------------| +| 0 | Monday | 1-7, Monday = 1, Tuesday = 2, ..., Sunday = 7 | +| 1 | Monday | 0-6, Monday = 0, Tuesday = 1, ..., Sunday = 6 | +| 2 | Sunday | 0-6, Sunday = 0, Monday = 1, ..., Saturday = 6 | +| 3 | Sunday | 1-7, Sunday = 1, Monday = 2, ..., Saturday = 7 | Alias: `DAYOFWEEK`. diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index 84f063f9555..6bf530008dc 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -622,6 +622,25 @@ public: template inline UInt8 toDayOfWeek(DateOrTime v) const { return lut[toLUTIndex(v)].day_of_week; } + template + inline UInt8 toDayOfWeek(DateOrTime v, UInt8 week_day_mode) const + { + /// 0: Sun = 7, Mon = 1 + /// 1: Sun = 6, Mon = 0 + /// 2: Sun = 0, Mon = 1 + /// 3: Sun = 1, Mon = 2 + week_day_mode = check_week_day_mode(week_day_mode); + auto res = toDayOfWeek(v); + + bool start_from_sunday = week_day_mode & (1 << 1); + bool zero_based = (week_day_mode == 1 || week_day_mode == 2); + if (start_from_sunday) + res = res % 7 + 1; + if (zero_based) + --res; + return res; + } + template inline UInt8 toDayOfMonth(DateOrTime v) const { return lut[toLUTIndex(v)].day_of_month; } @@ -844,6 +863,13 @@ public: return week_format; } + /// Check and change mode to effective. + inline UInt8 check_week_day_mode(UInt8 mode) const /// NOLINT + { + return mode & 3; + } + + /** Calculate weekday from d. * Returns 0 for monday, 1 for tuesday... */ diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index f4163a336ef..56e4a0e2668 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -786,21 +786,25 @@ struct ToDayOfWeekImpl { static constexpr auto name = "toDayOfWeek"; - static inline UInt8 execute(Int64 t, const DateLUTImpl & time_zone) + static inline UInt8 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(t); } + static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(t); } + static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(ExtendedDayNum(d)); } + static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(DayNum(d)); } + static inline UInt8 execute(Int64 t, UInt8 week_day_mode, const DateLUTImpl & time_zone) { - return time_zone.toDayOfWeek(t); + return time_zone.toDayOfWeek(t, week_day_mode); } - static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) + static inline UInt8 execute(UInt32 t, UInt8 week_day_mode, const DateLUTImpl & time_zone) { - return time_zone.toDayOfWeek(t); + return time_zone.toDayOfWeek(t, week_day_mode); } - static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone) + static inline UInt8 execute(Int32 d, UInt8 week_day_mode, const DateLUTImpl & time_zone) { - return time_zone.toDayOfWeek(ExtendedDayNum(d)); + return time_zone.toDayOfWeek(ExtendedDayNum(d), week_day_mode); } - static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) + static inline UInt8 execute(UInt16 d, UInt8 week_day_mode, const DateLUTImpl & time_zone) { - return time_zone.toDayOfWeek(DayNum(d)); + return time_zone.toDayOfWeek(DayNum(d), week_day_mode); } using FactorTransform = ToMondayImpl; diff --git a/src/Functions/toDayOfWeek.cpp b/src/Functions/toDayOfWeek.cpp index 354d4dea894..09271cbe55d 100644 --- a/src/Functions/toDayOfWeek.cpp +++ b/src/Functions/toDayOfWeek.cpp @@ -1,13 +1,14 @@ #include #include -#include #include +#include + namespace DB { -using FunctionToDayOfWeek = FunctionDateOrDateTimeToSomething; +using FunctionToDayOfWeek = FunctionCustomWeekToSomething; REGISTER_FUNCTION(ToDayOfWeek) { diff --git a/tests/queries/0_stateless/02521_to_custom_day_of_week.reference b/tests/queries/0_stateless/02521_to_custom_day_of_week.reference new file mode 100644 index 00000000000..660dff37b72 --- /dev/null +++ b/tests/queries/0_stateless/02521_to_custom_day_of_week.reference @@ -0,0 +1,7 @@ +1 7 +1 7 +0 6 +1 0 +2 1 +1 7 +0 6 diff --git a/tests/queries/0_stateless/02521_to_custom_day_of_week.sql b/tests/queries/0_stateless/02521_to_custom_day_of_week.sql new file mode 100644 index 00000000000..5475e15a984 --- /dev/null +++ b/tests/queries/0_stateless/02521_to_custom_day_of_week.sql @@ -0,0 +1,10 @@ + +with toDate('2023-01-09') as date_mon, date_mon - 1 as date_sun select toDayOfWeek(date_mon), toDayOfWeek(date_sun); +with toDate('2023-01-09') as date_mon, date_mon - 1 as date_sun select toDayOfWeek(date_mon, 0), toDayOfWeek(date_sun, 0); +with toDate('2023-01-09') as date_mon, date_mon - 1 as date_sun select toDayOfWeek(date_mon, 1), toDayOfWeek(date_sun, 1); +with toDate('2023-01-09') as date_mon, date_mon - 1 as date_sun select toDayOfWeek(date_mon, 2), toDayOfWeek(date_sun, 2); +with toDate('2023-01-09') as date_mon, date_mon - 1 as date_sun select toDayOfWeek(date_mon, 3), toDayOfWeek(date_sun, 3); +with toDate('2023-01-09') as date_mon, date_mon - 1 as date_sun select toDayOfWeek(date_mon, 4), toDayOfWeek(date_sun, 4); +with toDate('2023-01-09') as date_mon, date_mon - 1 as date_sun select toDayOfWeek(date_mon, 5), toDayOfWeek(date_sun, 5); + +select toDayOfWeek(today(), -1); -- { serverError 43 } From 64bbdee8248e87c436f5af0447e608f9fb68e48c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 9 Jan 2023 19:36:51 +0800 Subject: [PATCH 09/49] fix stule --- src/Functions/toDayOfWeek.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Functions/toDayOfWeek.cpp b/src/Functions/toDayOfWeek.cpp index 09271cbe55d..06343714b9d 100644 --- a/src/Functions/toDayOfWeek.cpp +++ b/src/Functions/toDayOfWeek.cpp @@ -3,8 +3,6 @@ #include #include - - namespace DB { From 4bc2c614068055df675262eeb1fc9f2f56b8c635 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 9 Jan 2023 20:37:16 +0800 Subject: [PATCH 10/49] change as request --- src/Common/DateLUTImpl.h | 23 ++++++++++++++--------- src/Functions/DateTimeTransforms.h | 4 ---- src/Functions/dateName.cpp | 2 +- src/Functions/formatDateTime.cpp | 8 ++++---- 4 files changed, 19 insertions(+), 18 deletions(-) diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index 6bf530008dc..1e7f11d3c9e 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -39,6 +39,15 @@ enum class WeekModeFlag : UInt8 }; using YearWeek = std::pair; +/// Modes for toDayOfWeek() function. +enum class WeekDayMode +{ + WeekStartsMonday1 = 0, + WeekStartsMonday0 = 1, + WeekStartsSunday0 = 2, + WeekStartsSunday1 = 3 +}; + /** Lookup table to conversion of time to date, and to month / year / day of week / day of month and so on. * First time was implemented for OLAPServer, that needed to do billions of such transformations. */ @@ -625,15 +634,11 @@ public: template inline UInt8 toDayOfWeek(DateOrTime v, UInt8 week_day_mode) const { - /// 0: Sun = 7, Mon = 1 - /// 1: Sun = 6, Mon = 0 - /// 2: Sun = 0, Mon = 1 - /// 3: Sun = 1, Mon = 2 - week_day_mode = check_week_day_mode(week_day_mode); + WeekDayMode mode = check_week_day_mode(week_day_mode); auto res = toDayOfWeek(v); - bool start_from_sunday = week_day_mode & (1 << 1); - bool zero_based = (week_day_mode == 1 || week_day_mode == 2); + bool start_from_sunday = (mode == WeekDayMode::WeekStartsSunday0 || mode == WeekDayMode::WeekStartsSunday1); + bool zero_based = (mode == WeekDayMode::WeekStartsMonday0 || mode == WeekDayMode::WeekStartsSunday0); if (start_from_sunday) res = res % 7 + 1; if (zero_based) @@ -864,9 +869,9 @@ public: } /// Check and change mode to effective. - inline UInt8 check_week_day_mode(UInt8 mode) const /// NOLINT + inline WeekDayMode check_week_day_mode(UInt8 mode) const /// NOLINT { - return mode & 3; + return static_cast(mode & 3); } diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 56e4a0e2668..56a7a960ac9 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -786,10 +786,6 @@ struct ToDayOfWeekImpl { static constexpr auto name = "toDayOfWeek"; - static inline UInt8 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(t); } - static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(t); } - static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(ExtendedDayNum(d)); } - static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(DayNum(d)); } static inline UInt8 execute(Int64 t, UInt8 week_day_mode, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(t, week_day_mode); diff --git a/src/Functions/dateName.cpp b/src/Functions/dateName.cpp index 36c0be49190..bfb190b9a08 100644 --- a/src/Functions/dateName.cpp +++ b/src/Functions/dateName.cpp @@ -276,7 +276,7 @@ private: { static inline void write(WriteBuffer & buffer, Time source, const DateLUTImpl & timezone) { - const auto day = ToDayOfWeekImpl::execute(source, timezone); + const auto day = ToDayOfWeekImpl::execute(source, 0, timezone); static constexpr std::string_view day_names[] = { "Monday", diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index e7c9a1b5103..c01f32f68ae 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -344,13 +344,13 @@ private: static size_t mysqlDayOfWeek(char * dest, Time source, UInt64, UInt32, const DateLUTImpl & timezone) { - *dest = '0' + ToDayOfWeekImpl::execute(source, timezone); + *dest = '0' + ToDayOfWeekImpl::execute(source, 0, timezone); return 1; } static size_t mysqlDayOfWeek0To6(char * dest, Time source, UInt64, UInt32, const DateLUTImpl & timezone) { - auto day = ToDayOfWeekImpl::execute(source, timezone); + auto day = ToDayOfWeekImpl::execute(source, 0, timezone); *dest = '0' + (day == 7 ? 0 : day); return 1; } @@ -499,13 +499,13 @@ private: static size_t jodaDayOfWeek1Based(size_t min_represent_digits, char * dest, Time source, UInt64, UInt32, const DateLUTImpl & timezone) { - auto week_day = ToDayOfWeekImpl::execute(source, timezone); + auto week_day = ToDayOfWeekImpl::execute(source, 0, timezone); return writeNumberWithPadding(dest, week_day, min_represent_digits); } static size_t jodaDayOfWeekText(size_t min_represent_digits, char * dest, Time source, UInt64, UInt32, const DateLUTImpl & timezone) { - auto week_day = ToDayOfWeekImpl::execute(source, timezone); + auto week_day = ToDayOfWeekImpl::execute(source, 0, timezone); if (week_day == 7) week_day = 0; From a8da7b4c20964a08eccb83b56afaa56df41146f2 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 9 Jan 2023 20:39:46 +0800 Subject: [PATCH 11/49] change as request --- src/Common/DateLUTImpl.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index 1e7f11d3c9e..3d496e088bb 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -628,6 +628,7 @@ public: template inline Int16 toYear(DateOrTime v) const { return lut[toLUTIndex(v)].year; } + /// 1-based, starts on Monday template inline UInt8 toDayOfWeek(DateOrTime v) const { return lut[toLUTIndex(v)].day_of_week; } @@ -635,7 +636,7 @@ public: inline UInt8 toDayOfWeek(DateOrTime v, UInt8 week_day_mode) const { WeekDayMode mode = check_week_day_mode(week_day_mode); - auto res = toDayOfWeek(v); + UInt8 res = toDayOfWeek(v); bool start_from_sunday = (mode == WeekDayMode::WeekStartsSunday0 || mode == WeekDayMode::WeekStartsSunday1); bool zero_based = (mode == WeekDayMode::WeekStartsMonday0 || mode == WeekDayMode::WeekStartsSunday0); From 4236bc32ee29c34af6b6d25d4a64c547208fe3f7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 22 Dec 2022 11:06:38 +0100 Subject: [PATCH 12/49] Analyzer duplicate alias crash fix --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 17 ++++------------- ...analyzer_duplicate_alias_crash_fix.reference | 0 ...02513_analyzer_duplicate_alias_crash_fix.sql | 4 ++++ 3 files changed, 8 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/02513_analyzer_duplicate_alias_crash_fix.reference create mode 100644 tests/queries/0_stateless/02513_analyzer_duplicate_alias_crash_fix.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index e93548d34ed..d7a686d4dfa 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2020,7 +2020,10 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con StorageID storage_id(database_name, table_name); storage_id = context->resolveStorageID(storage_id); - auto storage = DatabaseCatalog::instance().getTable(storage_id, context); + auto storage = DatabaseCatalog::instance().tryGetTable(storage_id, context); + if (!storage) + return {}; + auto storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); auto storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); @@ -4084,8 +4087,6 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto & in_second_argument = function_in_arguments_nodes[1]; auto * table_node = in_second_argument->as(); auto * table_function_node = in_second_argument->as(); - auto * query_node = in_second_argument->as(); - auto * union_node = in_second_argument->as(); if (table_node && dynamic_cast(table_node->getStorage().get()) != nullptr) { @@ -4118,16 +4119,6 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi in_second_argument = std::move(in_second_argument_query_node); } - else if (query_node || union_node) - { - IdentifierResolveScope subquery_scope(in_second_argument, &scope /*parent_scope*/); - subquery_scope.subquery_depth = scope.subquery_depth + 1; - - if (query_node) - resolveQuery(in_second_argument, subquery_scope); - else if (union_node) - resolveUnion(in_second_argument, subquery_scope); - } } /// Initialize function argument columns diff --git a/tests/queries/0_stateless/02513_analyzer_duplicate_alias_crash_fix.reference b/tests/queries/0_stateless/02513_analyzer_duplicate_alias_crash_fix.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02513_analyzer_duplicate_alias_crash_fix.sql b/tests/queries/0_stateless/02513_analyzer_duplicate_alias_crash_fix.sql new file mode 100644 index 00000000000..fb50ea2c4ca --- /dev/null +++ b/tests/queries/0_stateless/02513_analyzer_duplicate_alias_crash_fix.sql @@ -0,0 +1,4 @@ +SET allow_experimental_analyzer = 1; + +SELECT toUInt64(NULL) AS x FROM (SELECT 1) HAVING x IN + (SELECT NULL FROM (SELECT x IN (SELECT x IN (SELECT 1), x IN (SELECT 1) FROM (SELECT 1 WHERE x IN (SELECT NULL FROM (SELECT NULL)))))); From 09c1cecb01f2b4ad339f4d0641351b3c0363ad3a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 10 Jan 2023 10:56:13 +0800 Subject: [PATCH 13/49] fix build error --- src/Functions/dateDiff.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index 60668f81edf..d43ef2d4caf 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -225,8 +225,8 @@ public: } else if constexpr (std::is_same_v>>) { - auto x_day_of_week = TransformDateTime64(transform_x.getScaleMultiplier()).execute(x, timezone_x); - auto y_day_of_week = TransformDateTime64(transform_y.getScaleMultiplier()).execute(y, timezone_y); + auto x_day_of_week = TransformDateTime64(transform_x.getScaleMultiplier()).execute(x, 0, timezone_x); + auto y_day_of_week = TransformDateTime64(transform_y.getScaleMultiplier()).execute(y, 0, timezone_y); if ((x_day_of_week > y_day_of_week) || ((x_day_of_week == y_day_of_week) && (a_comp.time.hour > b_comp.time.hour)) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) From c95925a5abae54412ea20746beae712d838d2825 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 10 Jan 2023 11:16:29 +0100 Subject: [PATCH 14/49] Fixed tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 51 ++++++++++++------- .../02337_analyzer_columns_basic.sql | 2 +- 2 files changed, 33 insertions(+), 20 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index d7a686d4dfa..4aa6422b6b8 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2870,7 +2870,10 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const if (resolved_identifier) { - bool is_cte = resolved_identifier->as() && resolved_identifier->as()->isCTE(); + auto * subquery_node = resolved_identifier->as(); + auto * union_node = resolved_identifier->as(); + + bool is_cte = (subquery_node && subquery_node->isCTE()) || (union_node && union_node->isCTE()); /** From parent scopes we can resolve table identifiers only as CTE. * Example: SELECT (SELECT 1 FROM a) FROM test_table AS a; @@ -4119,6 +4122,10 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi in_second_argument = std::move(in_second_argument_query_node); } + else + { + resolveExpressionNode(in_second_argument, scope, false /*allow_lambda_expression*/, true /*allow_table_expression*/); + } } /// Initialize function argument columns @@ -4708,13 +4715,29 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id { node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::TABLE_EXPRESSION}, scope).resolved_identifier; - /// If table identifier is resolved as CTE clone it - bool resolved_as_cte = node && node->as() && node->as()->isCTE(); + /// If table identifier is resolved as CTE clone it and resolve + auto * subquery_node = node->as(); + auto * union_node = node->as(); + bool resolved_as_cte = (subquery_node && subquery_node->isCTE()) || (union_node && union_node->isCTE()); if (resolved_as_cte) { node = node->clone(); - node->as().setIsCTE(false); + subquery_node = node->as(); + union_node = node->as(); + + if (subquery_node) + subquery_node->setIsCTE(false); + else + union_node->setIsCTE(false); + + IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); + subquery_scope.subquery_depth = scope.subquery_depth + 1; + + if (subquery_node) + resolveQuery(node, subquery_scope); + else + resolveUnion(node, subquery_scope); } } @@ -4830,6 +4853,9 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); subquery_scope.subquery_depth = scope.subquery_depth + 1; + ++subquery_counter; + std::string projection_name = "_subquery_" + std::to_string(subquery_counter); + if (node_type == QueryTreeNodeType::QUERY) resolveQuery(node, subquery_scope); else @@ -4838,9 +4864,8 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id if (!allow_table_expression) evaluateScalarSubqueryIfNeeded(node, subquery_scope.subquery_depth, subquery_scope.context); - ++subquery_counter; if (result_projection_names.empty()) - result_projection_names.push_back("_subquery_" + std::to_string(subquery_counter)); + result_projection_names.push_back(std::move(projection_name)); break; } @@ -5187,11 +5212,6 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod if (resolved_identifier_query_node || resolved_identifier_union_node) { - if (resolved_identifier_query_node) - resolved_identifier_query_node->setIsCTE(false); - else - resolved_identifier_union_node->setIsCTE(false); - if (table_expression_modifiers.has_value()) { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, @@ -5428,14 +5448,7 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, [[fallthrough]]; case QueryTreeNodeType::UNION: { - IdentifierResolveScope subquery_scope(join_tree_node, &scope); - subquery_scope.subquery_depth = scope.subquery_depth + 1; - - if (from_node_type == QueryTreeNodeType::QUERY) - resolveQuery(join_tree_node, subquery_scope); - else if (from_node_type == QueryTreeNodeType::UNION) - resolveUnion(join_tree_node, subquery_scope); - + resolveExpressionNode(join_tree_node, scope, false /*allow_lambda_expression*/, true /*allow_table_expression*/); break; } case QueryTreeNodeType::TABLE_FUNCTION: diff --git a/tests/queries/0_stateless/02337_analyzer_columns_basic.sql b/tests/queries/0_stateless/02337_analyzer_columns_basic.sql index 76f9f8b25e4..368a5670d17 100644 --- a/tests/queries/0_stateless/02337_analyzer_columns_basic.sql +++ b/tests/queries/0_stateless/02337_analyzer_columns_basic.sql @@ -31,7 +31,7 @@ INSERT INTO test_table VALUES (0, 'Value'); SELECT 'Table access without table name qualification'; SELECT test_id FROM test_table; -- { serverError 47 } -SELECT test_id FROM test_unknown_table; -- { serverError 60 } +SELECT test_id FROM test_unknown_table; -- { serverError 47 } DESCRIBE (SELECT id FROM test_table); SELECT id FROM test_table; From 4673b3fe1de44a030ca53ced88bd8d0efe9f94d6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Jan 2023 16:31:01 +0100 Subject: [PATCH 15/49] Revert "Revert "Custom reading for mutation"" --- src/Interpreters/Context.h | 6 - src/Interpreters/MutationsInterpreter.cpp | 418 +++++++++++++----- src/Interpreters/MutationsInterpreter.h | 72 ++- .../QueryPlan/ReadFromMergeTree.cpp | 1 - .../Sources/ThrowingExceptionSource.h | 32 ++ src/Storages/IStorage.h | 2 + src/Storages/MergeTree/MergeTreeData.h | 2 + .../MergeTree/MergeTreeDataSelectExecutor.h | 14 +- .../MergeTree/MergeTreeSequentialSource.cpp | 112 ++++- .../MergeTree/MergeTreeSequentialSource.h | 13 + src/Storages/MergeTree/MutateTask.cpp | 9 +- 11 files changed, 539 insertions(+), 142 deletions(-) create mode 100644 src/Processors/Sources/ThrowingExceptionSource.h diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 4b7d0685ba3..58478ab79b8 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -377,9 +377,6 @@ private: inline static ContextPtr global_context_instance; - /// A flag, used to mark if reader needs to apply deleted rows mask. - bool apply_deleted_mask = true; - /// Temporary data for query execution accounting. TemporaryDataOnDiskScopePtr temp_data_on_disk; public: @@ -973,9 +970,6 @@ public: bool isInternalQuery() const { return is_internal_query; } void setInternalQuery(bool internal) { is_internal_query = internal; } - bool applyDeletedMask() const { return apply_deleted_mask; } - void setApplyDeletedMask(bool apply) { apply_deleted_mask = apply; } - ActionLocksManagerPtr getActionLocksManager() const; enum class ApplicationType diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 3960e0759d6..f8627f1ff85 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -30,6 +30,9 @@ #include #include #include +#include +#include +#include namespace DB @@ -190,7 +193,8 @@ ColumnDependencies getAllColumnDependencies(const StorageMetadataPtr & metadata_ bool isStorageTouchedByMutations( - const StoragePtr & storage, + MergeTreeData & storage, + MergeTreeData::DataPartPtr source_part, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, ContextMutablePtr context_copy) @@ -199,19 +203,15 @@ bool isStorageTouchedByMutations( return false; bool all_commands_can_be_skipped = true; - auto storage_from_merge_tree_data_part = std::dynamic_pointer_cast(storage); for (const MutationCommand & command : commands) { if (!command.predicate) /// The command touches all rows. return true; - if (command.partition && !storage_from_merge_tree_data_part) - throw Exception("ALTER UPDATE/DELETE ... IN PARTITION is not supported for non-MergeTree tables", ErrorCodes::NOT_IMPLEMENTED); - - if (command.partition && storage_from_merge_tree_data_part) + if (command.partition) { - const String partition_id = storage_from_merge_tree_data_part->getPartitionIDFromQuery(command.partition, context_copy); - if (partition_id == storage_from_merge_tree_data_part->getPartitionId()) + const String partition_id = storage.getPartitionIDFromQuery(command.partition, context_copy); + if (partition_id == source_part->info.partition_id) all_commands_can_be_skipped = false; } else @@ -229,13 +229,15 @@ bool isStorageTouchedByMutations( context_copy->setSetting("allow_asynchronous_read_from_io_pool_for_merge_tree", false); context_copy->setSetting("max_streams_for_merge_tree_reading", Field(0)); - ASTPtr select_query = prepareQueryAffectedAST(commands, storage, context_copy); + ASTPtr select_query = prepareQueryAffectedAST(commands, storage.shared_from_this(), context_copy); + + auto storage_from_part = std::make_shared(source_part); /// Interpreter must be alive, when we use result of execute() method. /// For some reason it may copy context and give it into ExpressionTransform /// after that we will use context from destroyed stack frame in our stream. InterpreterSelectQuery interpreter( - select_query, context_copy, storage, metadata_snapshot, SelectQueryOptions().ignoreLimits().ignoreProjections()); + select_query, context_copy, storage_from_part, metadata_snapshot, SelectQueryOptions().ignoreLimits().ignoreProjections()); auto io = interpreter.execute(); PullingPipelineExecutor executor(io.pipeline); @@ -288,6 +290,57 @@ ASTPtr getPartitionAndPredicateExpressionForMutationCommand( return command.predicate ? command.predicate->clone() : partition_predicate_as_ast_func; } +MutationsInterpreter::Source::Source(StoragePtr storage_) : storage(std::move(storage_)) +{ +} + +MutationsInterpreter::Source::Source(MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_) + : data(&storage_), part(std::move(source_part_)) +{ +} + +StorageSnapshotPtr MutationsInterpreter::Source::getStorageSnapshot(const StorageMetadataPtr & snapshot_, const ContextPtr & context_) const +{ + if (data) + return data->getStorageSnapshot(snapshot_, context_); + + return storage->getStorageSnapshot(snapshot_, context_); +} + +StoragePtr MutationsInterpreter::Source::getStorage() const +{ + if (data) + return data->shared_from_this(); + + return storage; +} + +const MergeTreeData * MutationsInterpreter::Source::getMergeTreeData() const +{ + if (data) + return data; + + return dynamic_cast(storage.get()); +} + +bool MutationsInterpreter::Source::supportsLightweightDelete() const +{ + if (part) + return part->supportLightweightDeleteMutate(); + + return storage->supportsLightweightDelete(); +} + + +bool MutationsInterpreter::Source::hasLightweightDeleteMask() const +{ + return part && part->hasLightweightDelete(); +} + +bool MutationsInterpreter::Source::materializeTTLRecalculateOnly() const +{ + return data && data->getSettings()->materialize_ttl_recalculate_only; +} MutationsInterpreter::MutationsInterpreter( StoragePtr storage_, @@ -297,7 +350,45 @@ MutationsInterpreter::MutationsInterpreter( bool can_execute_, bool return_all_columns_, bool return_deleted_rows_) - : storage(std::move(storage_)) + : MutationsInterpreter( + Source(std::move(storage_)), + metadata_snapshot_, std::move(commands_), std::move(context_), + can_execute_, return_all_columns_, return_deleted_rows_) +{ + if (can_execute_ && dynamic_cast(source.getStorage().get())) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot execute mutation for {}. Mutation should be applied to every part separately.", + source.getStorage()->getName()); + } +} + +MutationsInterpreter::MutationsInterpreter( + MergeTreeData & storage_, + MergeTreeData::DataPartPtr source_part_, + const StorageMetadataPtr & metadata_snapshot_, + MutationCommands commands_, + ContextPtr context_, + bool can_execute_, + bool return_all_columns_, + bool return_deleted_rows_) + : MutationsInterpreter( + Source(storage_, std::move(source_part_)), + metadata_snapshot_, std::move(commands_), std::move(context_), + can_execute_, return_all_columns_, return_deleted_rows_) +{ +} + +MutationsInterpreter::MutationsInterpreter( + Source source_, + const StorageMetadataPtr & metadata_snapshot_, + MutationCommands commands_, + ContextPtr context_, + bool can_execute_, + bool return_all_columns_, + bool return_deleted_rows_) + : source(std::move(source_)) , metadata_snapshot(metadata_snapshot_) , commands(std::move(commands_)) , context(Context::createCopy(context_)) @@ -306,12 +397,12 @@ MutationsInterpreter::MutationsInterpreter( , return_all_columns(return_all_columns_) , return_deleted_rows(return_deleted_rows_) { - mutation_ast = prepare(!can_execute); + prepare(!can_execute); } -static NameSet getKeyColumns(const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot) +static NameSet getKeyColumns(const MutationsInterpreter::Source & source, const StorageMetadataPtr & metadata_snapshot) { - const MergeTreeData * merge_tree_data = dynamic_cast(storage.get()); + const MergeTreeData * merge_tree_data = source.getMergeTreeData(); if (!merge_tree_data) return {}; @@ -333,21 +424,12 @@ static NameSet getKeyColumns(const StoragePtr & storage, const StorageMetadataPt return key_columns; } -static bool materializeTTLRecalculateOnly(const StoragePtr & storage) -{ - auto storage_from_merge_tree_data_part = std::dynamic_pointer_cast(storage); - if (!storage_from_merge_tree_data_part) - return false; - - return storage_from_merge_tree_data_part->materializeTTLRecalculateOnly(); -} - static void validateUpdateColumns( - const StoragePtr & storage, + const MutationsInterpreter::Source & source, const StorageMetadataPtr & metadata_snapshot, const NameSet & updated_columns, const std::unordered_map & column_to_affected_materialized) { - NameSet key_columns = getKeyColumns(storage, metadata_snapshot); + NameSet key_columns = getKeyColumns(source, metadata_snapshot); for (const String & column_name : updated_columns) { @@ -364,7 +446,7 @@ static void validateUpdateColumns( /// Allow to override value of lightweight delete filter virtual column if (!found && column_name == LightweightDeleteDescription::FILTER_COLUMN.name) { - if (!storage->supportsLightweightDelete()) + if (!source.supportsLightweightDelete()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Lightweight delete is not supported for table"); found = true; } @@ -427,7 +509,7 @@ static std::optional> getExpressionsOfUpdatedNestedSubcolumn return res; } -ASTPtr MutationsInterpreter::prepare(bool dry_run) +void MutationsInterpreter::prepare(bool dry_run) { if (is_prepared) throw Exception("MutationsInterpreter is already prepared. It is a bug.", ErrorCodes::LOGICAL_ERROR); @@ -448,7 +530,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } NameSet updated_columns; - bool materialize_ttl_recalculate_only = materializeTTLRecalculateOnly(storage); + bool materialize_ttl_recalculate_only = source.materializeTTLRecalculateOnly(); for (const MutationCommand & command : commands) { @@ -481,7 +563,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) } } - validateUpdateColumns(storage, metadata_snapshot, updated_columns, column_to_affected_materialized); + validateUpdateColumns(source, metadata_snapshot, updated_columns, column_to_affected_materialized); } dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns); @@ -778,15 +860,10 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) stages_copy.back().filters = stage.filters; } - const ASTPtr select_query = prepareInterpreterSelectQuery(stages_copy, /* dry_run = */ true); - InterpreterSelectQuery interpreter{ - select_query, context, storage, metadata_snapshot, - SelectQueryOptions().analyze(/* dry_run = */ false).ignoreLimits().ignoreProjections()}; + prepareMutationStages(stages_copy, true); - auto first_stage_header = interpreter.getSampleBlock(); QueryPlan plan; - auto source = std::make_shared(first_stage_header); - plan.addStep(std::make_unique(Pipe(std::move(source)))); + initQueryPlan(stages_copy.front(), plan); auto pipeline = addStreamsForLaterStages(stages_copy, plan); updated_header = std::make_unique(pipeline.getHeader()); } @@ -801,21 +878,18 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) is_prepared = true; - return prepareInterpreterSelectQuery(stages, dry_run); + prepareMutationStages(stages, dry_run); } -ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & prepared_stages, bool dry_run) +void MutationsInterpreter::prepareMutationStages(std::vector & prepared_stages, bool dry_run) { - auto storage_snapshot = storage->getStorageSnapshot(metadata_snapshot, context); + auto storage_snapshot = source.getStorageSnapshot(metadata_snapshot, context); auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects(); auto all_columns = storage_snapshot->getColumns(options); /// Add _row_exists column if it is present in the part - if (auto part_storage = dynamic_pointer_cast(storage)) - { - if (part_storage->hasLightweightDeletedMask()) - all_columns.push_back({LightweightDeleteDescription::FILTER_COLUMN}); - } + if (source.hasLightweightDeleteMask()) + all_columns.push_back({LightweightDeleteDescription::FILTER_COLUMN}); /// Next, for each stage calculate columns changed by this and previous stages. for (size_t i = 0; i < prepared_stages.size(); ++i) @@ -839,7 +913,7 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & /// Now, calculate `expressions_chain` for each stage except the first. /// Do it backwards to propagate information about columns required as input for a stage to the previous stage. - for (size_t i = prepared_stages.size() - 1; i > 0; --i) + for (int64_t i = prepared_stages.size() - 1; i >= 0; --i) { auto & stage = prepared_stages[i]; @@ -859,7 +933,7 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & /// e.g. ALTER referencing the same table in scalar subquery bool execute_scalar_subqueries = !dry_run; auto syntax_result = TreeRewriter(context).analyze( - all_asts, all_columns, storage, storage_snapshot, + all_asts, all_columns, source.getStorage(), storage_snapshot, false, true, execute_scalar_subqueries); if (execute_scalar_subqueries && context->hasQueryContext()) @@ -897,6 +971,9 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & } } + if (i == 0 && actions_chain.steps.empty()) + actions_chain.lastStep(syntax_result->required_source_columns); + /// Remove all intermediate columns. actions_chain.addStep(); actions_chain.getLastStep().required_output.clear(); @@ -908,49 +985,198 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & actions_chain.finalize(); - /// Propagate information about columns needed as input. - for (const auto & column : actions_chain.steps.front()->getRequiredColumns()) - prepared_stages[i - 1].output_columns.insert(column.name); - } - - /// Execute first stage as a SELECT statement. - - auto select = std::make_shared(); - - select->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); - for (const auto & column_name : prepared_stages[0].output_columns) - select->select()->children.push_back(std::make_shared(column_name)); - - /// Don't let select list be empty. - if (select->select()->children.empty()) - select->select()->children.push_back(std::make_shared(Field(0))); - - if (!prepared_stages[0].filters.empty()) - { - ASTPtr where_expression; - if (prepared_stages[0].filters.size() == 1) - where_expression = prepared_stages[0].filters[0]; - else + if (i) { - auto coalesced_predicates = std::make_shared(); - coalesced_predicates->name = "and"; - coalesced_predicates->arguments = std::make_shared(); - coalesced_predicates->children.push_back(coalesced_predicates->arguments); - coalesced_predicates->arguments->children = prepared_stages[0].filters; - where_expression = std::move(coalesced_predicates); + /// Propagate information about columns needed as input. + for (const auto & column : actions_chain.steps.front()->getRequiredColumns()) + prepared_stages[i - 1].output_columns.insert(column.name); + } + } +} + +/// This structure re-implements adding virtual columns while reading from MergeTree part. +/// It would be good to unify it with IMergeTreeSelectAlgorithm. +struct VirtualColumns +{ + struct ColumnAndPosition + { + ColumnWithTypeAndName column; + size_t position; + }; + + using Columns = std::vector; + + Columns virtuals; + Names columns_to_read; + + VirtualColumns(Names required_columns, const MergeTreeData::DataPartPtr & part) : columns_to_read(std::move(required_columns)) + { + for (size_t i = 0; i < columns_to_read.size(); ++i) + { + if (columns_to_read[i] == LightweightDeleteDescription::FILTER_COLUMN.name) + { + LoadedMergeTreeDataPartInfoForReader part_info_reader(part); + if (!part_info_reader.getColumns().contains(LightweightDeleteDescription::FILTER_COLUMN.name)) + { + ColumnWithTypeAndName mask_column; + mask_column.type = LightweightDeleteDescription::FILTER_COLUMN.type; + mask_column.column = mask_column.type->createColumnConst(0, 1); + mask_column.name = std::move(columns_to_read[i]); + + virtuals.emplace_back(ColumnAndPosition{.column = std::move(mask_column), .position = i}); + } + } + else if (columns_to_read[i] == "_partition_id") + { + ColumnWithTypeAndName column; + column.type = std::make_shared(); + column.column = column.type->createColumnConst(0, part->info.partition_id); + column.name = std::move(columns_to_read[i]); + + virtuals.emplace_back(ColumnAndPosition{.column = std::move(column), .position = i}); + } + } + + if (!virtuals.empty()) + { + Names columns_no_virtuals; + columns_no_virtuals.reserve(columns_to_read.size()); + size_t next_virtual = 0; + for (size_t i = 0; i < columns_to_read.size(); ++i) + { + if (next_virtual < virtuals.size() && i == virtuals[next_virtual].position) + ++next_virtual; + else + columns_no_virtuals.emplace_back(std::move(columns_to_read[i])); + } + + columns_to_read.swap(columns_no_virtuals); } - select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where_expression)); } - return select; + void addVirtuals(QueryPlan & plan) + { + auto dag = std::make_unique(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + + for (auto & column : virtuals) + { + const auto & adding_const = dag->addColumn(std::move(column.column)); + auto & outputs = dag->getOutputs(); + outputs.insert(outputs.begin() + column.position, &adding_const); + } + + auto step = std::make_unique(plan.getCurrentDataStream(), std::move(dag)); + plan.addStep(std::move(step)); + } +}; + +void MutationsInterpreter::Source::read( + Stage & first_stage, + QueryPlan & plan, + const StorageMetadataPtr & snapshot_, + const ContextPtr & context_, + bool apply_deleted_mask_, + bool can_execute_) const +{ + auto required_columns = first_stage.expressions_chain.steps.front()->getRequiredColumns().getNames(); + auto storage_snapshot = getStorageSnapshot(snapshot_, context_); + + if (!can_execute_) + { + auto header = storage_snapshot->getSampleBlockForColumns(required_columns); + auto callback = []() + { + return DB::Exception(ErrorCodes::LOGICAL_ERROR, "Cannot execute a mutation because can_execute flag set to false"); + }; + + Pipe pipe(std::make_shared(header, callback)); + + auto read_from_pipe = std::make_unique(std::move(pipe)); + plan.addStep(std::move(read_from_pipe)); + return; + } + + if (data) + { + const auto & steps = first_stage.expressions_chain.steps; + const auto & names = first_stage.filter_column_names; + size_t num_filters = names.size(); + + ActionsDAGPtr filter; + if (!first_stage.filter_column_names.empty()) + { + + ActionsDAG::NodeRawConstPtrs nodes(num_filters); + for (size_t i = 0; i < num_filters; ++i) + nodes[i] = &steps[i]->actions()->findInOutputs(names[i]); + + filter = ActionsDAG::buildFilterActionsDAG(nodes, {}, context_); + } + + VirtualColumns virtual_columns(std::move(required_columns), part); + + createMergeTreeSequentialSource( + plan, *data, storage_snapshot, part, std::move(virtual_columns.columns_to_read), apply_deleted_mask_, filter, context_, + &Poco::Logger::get("MutationsInterpreter")); + + virtual_columns.addVirtuals(plan); + } + else + { + auto select = std::make_shared(); + + select->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); + for (const auto & column_name : first_stage.output_columns) + select->select()->children.push_back(std::make_shared(column_name)); + + /// Don't let select list be empty. + if (select->select()->children.empty()) + select->select()->children.push_back(std::make_shared(Field(0))); + + if (!first_stage.filters.empty()) + { + ASTPtr where_expression; + if (first_stage.filters.size() == 1) + where_expression = first_stage.filters[0]; + else + { + auto coalesced_predicates = std::make_shared(); + coalesced_predicates->name = "and"; + coalesced_predicates->arguments = std::make_shared(); + coalesced_predicates->children.push_back(coalesced_predicates->arguments); + coalesced_predicates->arguments->children = first_stage.filters; + where_expression = std::move(coalesced_predicates); + } + select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where_expression)); + } + + SelectQueryInfo query_info; + query_info.query = std::move(select); + + size_t max_block_size = context_->getSettingsRef().max_block_size; + size_t max_streams = 1; + storage->read(plan, required_columns, storage_snapshot, query_info, context_, QueryProcessingStage::FetchColumns, max_block_size, max_streams); + + if (!plan.isInitialized()) + { + /// It may be possible when there is nothing to read from storage. + auto header = storage_snapshot->getSampleBlockForColumns(required_columns); + auto read_from_pipe = std::make_unique(Pipe(std::make_shared(header))); + plan.addStep(std::move(read_from_pipe)); + } + } +} + +void MutationsInterpreter::initQueryPlan(Stage & first_stage, QueryPlan & plan) +{ + source.read(first_stage, plan, metadata_snapshot, context, apply_deleted_mask, can_execute); + addCreatingSetsStep(plan, first_stage.analyzer->getPreparedSets(), context); } QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const { - for (size_t i_stage = 1; i_stage < prepared_stages.size(); ++i_stage) + for (const Stage & stage : prepared_stages) { - const Stage & stage = prepared_stages[i_stage]; - for (size_t i = 0; i < stage.expressions_chain.steps.size(); ++i) { const auto & step = stage.expressions_chain.steps[i]; @@ -988,14 +1214,11 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v void MutationsInterpreter::validate() { - if (!select_interpreter) - select_interpreter = std::make_unique(mutation_ast, context, storage, metadata_snapshot, select_limits); - const Settings & settings = context->getSettingsRef(); /// For Replicated* storages mutations cannot employ non-deterministic functions /// because that produces inconsistencies between replicas - if (startsWith(storage->getName(), "Replicated") && !settings.allow_nondeterministic_mutations) + if (startsWith(source.getStorage()->getName(), "Replicated") && !settings.allow_nondeterministic_mutations) { for (const auto & command : commands) { @@ -1012,7 +1235,7 @@ void MutationsInterpreter::validate() } QueryPlan plan; - select_interpreter->buildQueryPlan(plan); + initQueryPlan(stages.front(), plan); auto pipeline = addStreamsForLaterStages(stages, plan); } @@ -1021,23 +1244,8 @@ QueryPipelineBuilder MutationsInterpreter::execute() if (!can_execute) throw Exception("Cannot execute mutations interpreter because can_execute flag set to false", ErrorCodes::LOGICAL_ERROR); - if (!select_interpreter) - { - /// Skip to apply deleted mask for MutateSomePartColumn cases when part has lightweight delete. - if (!apply_deleted_mask) - { - auto context_for_reading = Context::createCopy(context); - context_for_reading->setApplyDeletedMask(apply_deleted_mask); - select_interpreter = std::make_unique(mutation_ast, context_for_reading, storage, metadata_snapshot, select_limits); - } - else - select_interpreter = std::make_unique(mutation_ast, context, storage, metadata_snapshot, select_limits); - } - - QueryPlan plan; - select_interpreter->buildQueryPlan(plan); - + initQueryPlan(stages.front(), plan); auto builder = addStreamsForLaterStages(stages, plan); /// Sometimes we update just part of columns (for example UPDATE mutation) @@ -1069,11 +1277,7 @@ const ColumnDependencies & MutationsInterpreter::getColumnDependencies() const size_t MutationsInterpreter::evaluateCommandsSize() { - for (const MutationCommand & command : commands) - if (unlikely(!command.predicate && !command.partition)) /// The command touches all rows. - return mutation_ast->size(); - - return std::max(prepareQueryAffectedAST(commands, storage, context)->size(), mutation_ast->size()); + return prepareQueryAffectedAST(commands, source.getStorage(), context)->size(); } std::optional MutationsInterpreter::getStorageSortDescriptionIfPossible(const Block & header) const @@ -1096,7 +1300,7 @@ std::optional MutationsInterpreter::getStorageSortDescriptionIf ASTPtr MutationsInterpreter::getPartitionAndPredicateExpressionForMutationCommand(const MutationCommand & command) const { - return DB::getPartitionAndPredicateExpressionForMutationCommand(command, storage, context); + return DB::getPartitionAndPredicateExpressionForMutationCommand(command, source.getStorage(), context); } bool MutationsInterpreter::Stage::isAffectingAllColumns(const Names & storage_columns) const diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 336c5f11162..fbcb56fac6f 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -19,7 +19,8 @@ using QueryPipelineBuilderPtr = std::unique_ptr; /// Return false if the data isn't going to be changed by mutations. bool isStorageTouchedByMutations( - const StoragePtr & storage, + MergeTreeData & storage, + MergeTreeData::DataPartPtr source_part, const StorageMetadataPtr & metadata_snapshot, const std::vector & commands, ContextMutablePtr context_copy @@ -35,6 +36,8 @@ ASTPtr getPartitionAndPredicateExpressionForMutationCommand( /// to this data. class MutationsInterpreter { + struct Stage; + public: /// Storage to mutate, array of mutations commands and context. If you really want to execute mutation /// use can_execute = true, in other cases (validation, amount of commands) it can be false @@ -47,8 +50,18 @@ public: bool return_all_columns_ = false, bool return_deleted_rows_ = false); - void validate(); + /// Special case for MergeTree + MutationsInterpreter( + MergeTreeData & storage_, + MergeTreeData::DataPartPtr source_part_, + const StorageMetadataPtr & metadata_snapshot_, + MutationCommands commands_, + ContextPtr context_, + bool can_execute_, + bool return_all_columns_ = false, + bool return_deleted_rows_ = false); + void validate(); size_t evaluateCommandsSize(); /// The resulting stream will return blocks containing only changed columns and columns, that we need to recalculate indices. @@ -82,19 +95,60 @@ public: void setApplyDeletedMask(bool apply) { apply_deleted_mask = apply; } + /// Internal class which represents a data part for MergeTree + /// or just storage for other storages. + /// The main idea is to create a dedicated reading from MergeTree part. + /// Additionally we propagate some storage properties. + struct Source + { + StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & snapshot_, const ContextPtr & context_) const; + StoragePtr getStorage() const; + const MergeTreeData * getMergeTreeData() const; + + bool supportsLightweightDelete() const; + bool hasLightweightDeleteMask() const; + bool materializeTTLRecalculateOnly() const; + + void read( + Stage & first_stage, + QueryPlan & plan, + const StorageMetadataPtr & snapshot_, + const ContextPtr & context_, + bool apply_deleted_mask_, + bool can_execute_) const; + + explicit Source(StoragePtr storage_); + Source(MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_); + + private: + StoragePtr storage; + + /// Special case for MergeTree. + MergeTreeData * data = nullptr; + MergeTreeData::DataPartPtr part; + }; + private: - ASTPtr prepare(bool dry_run); + MutationsInterpreter( + Source source_, + const StorageMetadataPtr & metadata_snapshot_, + MutationCommands commands_, + ContextPtr context_, + bool can_execute_, + bool return_all_columns_, + bool return_deleted_rows_); - struct Stage; + void prepare(bool dry_run); - ASTPtr prepareInterpreterSelectQuery(std::vector &prepared_stages, bool dry_run); + void initQueryPlan(Stage & first_stage, QueryPlan & query_plan); + void prepareMutationStages(std::vector &prepared_stages, bool dry_run); QueryPipelineBuilder addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const; std::optional getStorageSortDescriptionIfPossible(const Block & header) const; ASTPtr getPartitionAndPredicateExpressionForMutationCommand(const MutationCommand & command) const; - StoragePtr storage; + Source source; StorageMetadataPtr metadata_snapshot; MutationCommands commands; ContextPtr context; @@ -103,12 +157,6 @@ private: bool apply_deleted_mask = true; - ASTPtr mutation_ast; - - /// We have to store interpreter because it use own copy of context - /// and some streams from execute method may use it. - std::unique_ptr select_interpreter; - /// A sequence of mutation commands is executed as a sequence of stages. Each stage consists of several /// filters, followed by updating values of some columns. Commands can reuse expressions calculated by the /// previous commands in the same stage, but at the end of each stage intermediate columns are thrown away diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 0d8fe84f9d3..4765b2cbfbe 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -64,7 +64,6 @@ static MergeTreeReaderSettings getMergeTreeReaderSettings( .save_marks_in_cache = true, .checksum_on_read = settings.checksum_on_read, .read_in_order = query_info.input_order_info != nullptr, - .apply_deleted_mask = context->applyDeletedMask(), .use_asynchronous_read_from_pool = settings.allow_asynchronous_read_from_io_pool_for_merge_tree && (settings.max_streams_to_max_threads_ratio > 1 || settings.max_streams_for_merge_tree_reading > 1), }; diff --git a/src/Processors/Sources/ThrowingExceptionSource.h b/src/Processors/Sources/ThrowingExceptionSource.h new file mode 100644 index 00000000000..5abebd89d07 --- /dev/null +++ b/src/Processors/Sources/ThrowingExceptionSource.h @@ -0,0 +1,32 @@ +#pragma once +#include + + +namespace DB +{ + +/// This source is throwing exception at the first attempt to read from it. +/// Can be used as a additional check that pipeline (or its part) is never executed. +class ThrowingExceptionSource : public ISource +{ +public: + + using CallBack = std::function; + + explicit ThrowingExceptionSource(Block header, CallBack callback_) + : ISource(std::move(header)) + , callback(std::move(callback_)) + {} + + String getName() const override { return "ThrowingExceptionSource"; } + +protected: + Chunk generate() override + { + throw callback(); + } + + CallBack callback; +}; + +} diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index cdf273b47df..7d927b51e5f 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -110,6 +110,8 @@ public: /// The name of the table. StorageID getStorageID() const; + virtual bool isMergeTree() const { return false; } + /// Returns true if the storage receives data from a remote server or servers. virtual bool isRemote() const { return false; } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 670c755cf72..19efd8f908a 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -424,6 +424,8 @@ public: StoragePolicyPtr getStoragePolicy() const override; + bool isMergeTree() const override { return true; } + bool supportsPrewhere() const override { return true; } bool supportsFinal() const override; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index e302663597d..30d09312245 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -66,6 +66,13 @@ public: size_t num_streams, std::shared_ptr max_block_numbers_to_read = nullptr) const; + static MarkRanges markRangesFromPKRange( + const MergeTreeData::DataPartPtr & part, + const StorageMetadataPtr & metadata_snapshot, + const KeyCondition & key_condition, + const Settings & settings, + Poco::Logger * log); + private: const MergeTreeData & data; Poco::Logger * log; @@ -78,13 +85,6 @@ private: const Settings & settings, Poco::Logger * log); - static MarkRanges markRangesFromPKRange( - const MergeTreeData::DataPartPtr & part, - const StorageMetadataPtr & metadata_snapshot, - const KeyCondition & key_condition, - const Settings & settings, - Poco::Logger * log); - static MarkRanges filterMarksUsingIndex( MergeTreeIndexPtr index_helper, MergeTreeIndexConditionPtr condition, diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 9e0c96fd88a..4539e0b36c5 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -1,9 +1,14 @@ #include #include #include +#include #include +#include +#include #include #include +#include +#include namespace DB { @@ -25,6 +30,8 @@ public: const StorageSnapshotPtr & storage_snapshot_, MergeTreeData::DataPartPtr data_part_, Names columns_to_read_, + std::optional mark_ranges_, + bool apply_deleted_mask, bool read_with_direct_io_, bool take_column_types_from_storage, bool quiet = false); @@ -56,6 +63,8 @@ private: Poco::Logger * log = &Poco::Logger::get("MergeTreeSequentialSource"); + std::optional mark_ranges; + std::shared_ptr mark_cache; using MergeTreeReaderPtr = std::unique_ptr; MergeTreeReaderPtr reader; @@ -76,6 +85,8 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( const StorageSnapshotPtr & storage_snapshot_, MergeTreeData::DataPartPtr data_part_, Names columns_to_read_, + std::optional mark_ranges_, + bool apply_deleted_mask, bool read_with_direct_io_, bool take_column_types_from_storage, bool quiet) @@ -85,6 +96,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( , data_part(std::move(data_part_)) , columns_to_read(std::move(columns_to_read_)) , read_with_direct_io(read_with_direct_io_) + , mark_ranges(std::move(mark_ranges_)) , mark_cache(storage.getContext()->getMarkCache()) { if (!quiet) @@ -126,11 +138,15 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( MergeTreeReaderSettings reader_settings = { .read_settings = read_settings, - .save_marks_in_cache = false + .save_marks_in_cache = false, + .apply_deleted_mask = apply_deleted_mask, }; + if (!mark_ranges) + mark_ranges.emplace(MarkRanges{MarkRange(0, data_part->getMarksCount())}); + reader = data_part->getReader(columns_for_reader, storage_snapshot->metadata, - MarkRanges{MarkRange(0, data_part->getMarksCount())}, + *mark_ranges, /* uncompressed_cache = */ nullptr, mark_cache.get(), reader_settings, {}, {}); } @@ -224,8 +240,10 @@ Pipe createMergeTreeSequentialSource( if (need_to_filter_deleted_rows) columns.emplace_back(LightweightDeleteDescription::FILTER_COLUMN.name); + bool apply_deleted_mask = false; + auto column_part_source = std::make_shared( - storage, storage_snapshot, data_part, columns, read_with_direct_io, take_column_types_from_storage, quiet); + storage, storage_snapshot, data_part, columns, std::optional{}, apply_deleted_mask, read_with_direct_io, take_column_types_from_storage, quiet); Pipe pipe(std::move(column_part_source)); @@ -242,4 +260,92 @@ Pipe createMergeTreeSequentialSource( return pipe; } +/// A Query Plan step to read from a single Merge Tree part +/// using Merge Tree Sequential Source (which reads strictly sequentially in a single thread). +/// This step is used for mutations because the usual reading is too tricky. +/// Previously, sequential reading was achieved by changing some settings like max_threads, +/// however, this approach lead to data corruption after some new settings were introduced. +class ReadFromPart final : public ISourceStep +{ +public: + ReadFromPart( + const MergeTreeData & storage_, + const StorageSnapshotPtr & storage_snapshot_, + MergeTreeData::DataPartPtr data_part_, + Names columns_to_read_, + bool apply_deleted_mask_, + ActionsDAGPtr filter_, + ContextPtr context_, + Poco::Logger * log_) + : ISourceStep(DataStream{.header = storage_snapshot_->getSampleBlockForColumns(columns_to_read_)}) + , storage(storage_) + , storage_snapshot(storage_snapshot_) + , data_part(std::move(data_part_)) + , columns_to_read(std::move(columns_to_read_)) + , apply_deleted_mask(apply_deleted_mask_) + , filter(std::move(filter_)) + , context(std::move(context_)) + , log(log_) + { + } + + String getName() const override { return fmt::format("ReadFromPart({})", data_part->name); } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override + { + std::optional mark_ranges; + + const auto & metadata_snapshot = storage_snapshot->metadata; + if (filter && metadata_snapshot->hasPrimaryKey()) + { + const auto & primary_key = storage_snapshot->metadata->getPrimaryKey(); + const Names & primary_key_column_names = primary_key.column_names; + KeyCondition key_condition(filter, context, primary_key_column_names, primary_key.expression, NameSet{}); + LOG_DEBUG(log, "Key condition: {}", key_condition.toString()); + + if (!key_condition.alwaysFalse()) + mark_ranges = MergeTreeDataSelectExecutor::markRangesFromPKRange( + data_part, metadata_snapshot, key_condition, context->getSettingsRef(), log); + + if (mark_ranges && mark_ranges->empty()) + { + pipeline.init(Pipe(std::make_unique(output_stream->header))); + return; + } + } + + auto source = std::make_unique( + storage, storage_snapshot, data_part, columns_to_read, std::move(mark_ranges), apply_deleted_mask, false, true); + + pipeline.init(Pipe(std::move(source))); + } + +private: + const MergeTreeData & storage; + StorageSnapshotPtr storage_snapshot; + MergeTreeData::DataPartPtr data_part; + Names columns_to_read; + bool apply_deleted_mask; + ActionsDAGPtr filter; + ContextPtr context; + Poco::Logger * log; +}; + +void createMergeTreeSequentialSource( + QueryPlan & plan, + const MergeTreeData & storage, + const StorageSnapshotPtr & storage_snapshot, + MergeTreeData::DataPartPtr data_part, + Names columns_to_read, + bool apply_deleted_mask, + ActionsDAGPtr filter, + ContextPtr context, + Poco::Logger * log) +{ + auto reading = std::make_unique( + storage, storage_snapshot, std::move(data_part), std::move(columns_to_read), apply_deleted_mask, filter, std::move(context), log); + + plan.addStep(std::move(reading)); +} + } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index c6c29f9d49a..fb249568e8f 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -20,4 +20,17 @@ Pipe createMergeTreeSequentialSource( bool quiet, std::shared_ptr> filtered_rows_count); +class QueryPlan; + +void createMergeTreeSequentialSource( + QueryPlan & plan, + const MergeTreeData & storage, + const StorageSnapshotPtr & storage_snapshot, + MergeTreeData::DataPartPtr data_part, + Names columns_to_read, + bool apply_deleted_mask, + ActionsDAGPtr filter, + ContextPtr context, + Poco::Logger * log); + } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index de68cb6f0ba..3ecb790243d 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -714,8 +714,6 @@ struct MutationContext FutureMergedMutatedPartPtr future_part; MergeTreeData::DataPartPtr source_part; - - StoragePtr storage_from_source_part; StorageMetadataPtr metadata_snapshot; MutationCommandsConstPtr commands; @@ -1478,10 +1476,9 @@ MutateTask::MutateTask( ctx->storage_columns = metadata_snapshot_->getColumns().getAllPhysical(); ctx->txn = txn; ctx->source_part = ctx->future_part->parts[0]; - ctx->storage_from_source_part = std::make_shared(ctx->source_part); ctx->need_prefix = need_prefix_; - auto storage_snapshot = ctx->storage_from_source_part->getStorageSnapshot(ctx->metadata_snapshot, context_); + auto storage_snapshot = ctx->data->getStorageSnapshot(ctx->metadata_snapshot, context_); extendObjectColumns(ctx->storage_columns, storage_snapshot->object_columns, /*with_subcolumns=*/ false); } @@ -1554,7 +1551,7 @@ bool MutateTask::prepare() } if (ctx->source_part->isStoredOnDisk() && !isStorageTouchedByMutations( - ctx->storage_from_source_part, ctx->metadata_snapshot, ctx->commands_for_part, Context::createCopy(context_for_reading))) + *ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->commands_for_part, Context::createCopy(context_for_reading))) { NameSet files_to_copy_instead_of_hardlinks; auto settings_ptr = ctx->data->getSettings(); @@ -1597,7 +1594,7 @@ bool MutateTask::prepare() if (!ctx->for_interpreter.empty()) { ctx->interpreter = std::make_unique( - ctx->storage_from_source_part, ctx->metadata_snapshot, ctx->for_interpreter, context_for_reading, true); + *ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->for_interpreter, context_for_reading, true); ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices(); ctx->materialized_projections = ctx->interpreter->grabMaterializedProjections(); ctx->mutation_kind = ctx->interpreter->getMutationKind(); From d945b72d6fe01c216e32f505c5bd85220382b4fb Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 10 Jan 2023 15:40:31 +0000 Subject: [PATCH 16/49] Pull SQLancer image before check run --- tests/ci/sqlancer_check.py | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/tests/ci/sqlancer_check.py b/tests/ci/sqlancer_check.py index ce6d89a7267..b286d1a63bc 100644 --- a/tests/ci/sqlancer_check.py +++ b/tests/ci/sqlancer_check.py @@ -29,6 +29,11 @@ from rerun_helper import RerunHelper IMAGE_NAME = "clickhouse/sqlancer-test" +def get_pull_command(docker_image): + return ( + f"docker pull --network=host {docker_image}" + ) + def get_run_command(download_url, workspace_path, image): return ( f"docker run " @@ -92,6 +97,21 @@ if __name__ == "__main__": if not os.path.exists(workspace_path): os.makedirs(workspace_path) + pull_command = get_pull_command(docker_image) + + logging.info("Going to pull image %s", pull_command) + + pull_log_path = os.path.join(workspace_path, "pull.log") + with open(pull_log_path, "w", encoding="utf-8") as log: + with subprocess.Popen( + pull_command, shell=True, stderr=log, stdout=log + ) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Pull successfully") + else: + logging.info("Pull failed") + run_command = get_run_command(build_url, workspace_path, docker_image) logging.info("Going to run %s", run_command) @@ -124,6 +144,7 @@ if __name__ == "__main__": paths = [ run_log_path, + pull_log_path, os.path.join(workspace_path, "clickhouse-server.log"), os.path.join(workspace_path, "stderr.log"), os.path.join(workspace_path, "stdout.log"), From fbba28b31e349450456f144704a3795f78d53707 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 9 Jan 2023 11:34:47 +0100 Subject: [PATCH 17/49] Analyzer aggregation without column fix --- src/Interpreters/ExpressionActions.cpp | 8 +-- src/Interpreters/ExpressionActions.h | 2 +- src/Interpreters/TreeRewriter.cpp | 2 +- src/Planner/PlannerJoinTree.cpp | 61 ++++++++++++++++++- .../QueryPlan/ReadFromMergeTree.cpp | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageMerge.cpp | 4 +- ...lyzer_aggregation_without_column.reference | 1 + ...21_analyzer_aggregation_without_column.sql | 15 +++++ 10 files changed, 85 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/02521_analyzer_aggregation_without_column.reference create mode 100644 tests/queries/0_stateless/02521_analyzer_aggregation_without_column.sql diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index d89be9f3e2e..5ea29615942 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -790,10 +790,10 @@ void ExpressionActions::assertDeterministic() const } -std::string ExpressionActions::getSmallestColumn(const NamesAndTypesList & columns) +NameAndTypePair ExpressionActions::getSmallestColumn(const NamesAndTypesList & columns) { std::optional min_size; - String res; + NameAndTypePair result; for (const auto & column : columns) { @@ -807,14 +807,14 @@ std::string ExpressionActions::getSmallestColumn(const NamesAndTypesList & colum if (!min_size || size < *min_size) { min_size = size; - res = column.name; + result = column; } } if (!min_size) throw Exception("No available columns", ErrorCodes::LOGICAL_ERROR); - return res; + return result; } std::string ExpressionActions::dumpActions() const diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index be63b9e0d78..faefe0985f7 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -111,7 +111,7 @@ public: std::string dumpActions() const; JSONBuilder::ItemPtr toTree() const; - static std::string getSmallestColumn(const NamesAndTypesList & columns); + static NameAndTypePair getSmallestColumn(const NamesAndTypesList & columns); /// Check if column is always zero. True if it's definite, false if we can't say for sure. /// Call it only after subqueries for sets were executed. diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 20c14b8d7b6..a1b3c8011cd 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1146,7 +1146,7 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select required.insert(std::min_element(columns.begin(), columns.end())->name); else if (!source_columns.empty()) /// If we have no information about columns sizes, choose a column of minimum size of its data type. - required.insert(ExpressionActions::getSmallestColumn(source_columns)); + required.insert(ExpressionActions::getSmallestColumn(source_columns).name); } else if (is_select && storage_snapshot && !columns_context.has_array_join) { diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 3584c9d4caa..999aa32d850 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -81,6 +81,63 @@ void checkAccessRights(const TableNode & table_node, const Names & column_names, query_context->checkAccess(AccessType::SELECT, storage_id, column_names); } +NameAndTypePair chooseSmallestColumnToReadFromStorage(const StoragePtr & storage, const StorageSnapshotPtr & storage_snapshot) +{ + /** We need to read at least one column to find the number of rows. + * We will find a column with minimum . + * Because it is the column that is cheapest to read. + */ + class ColumnWithSize + { + public: + ColumnWithSize(NameAndTypePair column_, ColumnSize column_size_) + : column(std::move(column_)) + , compressed_size(column_size_.data_compressed) + , uncompressed_size(column_size_.data_uncompressed) + , type_size(column.type->haveMaximumSizeOfValue() ? column.type->getMaximumSizeOfValueInMemory() : 100) + { + } + + bool operator<(const ColumnWithSize & rhs) const + { + return std::tie(compressed_size, type_size, uncompressed_size) + < std::tie(rhs.compressed_size, rhs.type_size, rhs.uncompressed_size); + } + + NameAndTypePair column; + size_t compressed_size = 0; + size_t uncompressed_size = 0; + size_t type_size = 0; + }; + + std::vector columns_with_sizes; + + auto column_sizes = storage->getColumnSizes(); + auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::AllPhysical).withSubcolumns()); + + if (!column_sizes.empty()) + { + for (auto & column_name_and_type : column_names_and_types) + { + auto it = column_sizes.find(column_name_and_type.name); + if (it == column_sizes.end()) + continue; + + columns_with_sizes.emplace_back(column_name_and_type, it->second); + } + } + + NameAndTypePair result; + + if (!columns_with_sizes.empty()) + result = std::min_element(columns_with_sizes.begin(), columns_with_sizes.end())->column; + else + /// If we have no information about columns sizes, choose a column of minimum size of its data type + result = ExpressionActions::getSmallestColumn(column_names_and_types); + + return result; +} + QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, SelectQueryInfo & select_query_info, const SelectQueryOptions & select_query_options, @@ -127,9 +184,7 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, if (columns_names.empty()) { - auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); - auto additional_column_to_read = column_names_and_types.front(); - + auto additional_column_to_read = chooseSmallestColumnToReadFromStorage(storage, storage_snapshot); const auto & column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(additional_column_to_read, table_expression); columns_names.push_back(additional_column_to_read.name); table_expression_data.addColumn(additional_column_to_read, column_identifier); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 0d8fe84f9d3..e5ad2729e6c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1023,7 +1023,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( if (result.column_names_to_read.empty()) { NamesAndTypesList available_real_columns = metadata_snapshot->getColumns().getAllPhysical(); - result.column_names_to_read.push_back(ExpressionActions::getSmallestColumn(available_real_columns)); + result.column_names_to_read.push_back(ExpressionActions::getSmallestColumn(available_real_columns).name); } // storage_snapshot->check(result.column_names_to_read); diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index bbabd523c45..c7008a317c3 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -599,7 +599,7 @@ Pipe StorageHDFS::read( { return std::any_of(virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col){ return col == virtual_col.name; }); }); if (fetch_columns.empty()) - fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical())); + fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns); block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 95bd0e7c53e..922754c2d8c 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -706,7 +706,7 @@ Pipe StorageFile::read( }); if (fetch_columns.empty()) - fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical())); + fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns); } else diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 79efab9e9d7..3e279b408d7 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -488,7 +488,7 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu column_names_as_aliases = alias_actions->getRequiredColumns().getNames(); if (column_names_as_aliases.empty()) - column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical())); + column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical()).name); } auto source_pipeline = createSources( @@ -574,7 +574,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( { /// If there are only virtual columns in query, you must request at least one other column. if (real_column_names.empty()) - real_column_names.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical())); + real_column_names.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); QueryPlan plan; if (StorageView * view = dynamic_cast(storage.get())) diff --git a/tests/queries/0_stateless/02521_analyzer_aggregation_without_column.reference b/tests/queries/0_stateless/02521_analyzer_aggregation_without_column.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02521_analyzer_aggregation_without_column.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02521_analyzer_aggregation_without_column.sql b/tests/queries/0_stateless/02521_analyzer_aggregation_without_column.sql new file mode 100644 index 00000000000..105bce6711c --- /dev/null +++ b/tests/queries/0_stateless/02521_analyzer_aggregation_without_column.sql @@ -0,0 +1,15 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + c0 String ALIAS c1, + c1 String, + c2 String, +) ENGINE = MergeTree ORDER BY c1; + +INSERT INTO test_table VALUES ('a', 'b'); + +SELECT MAX(1) FROM test_table; + +DROP TABLE test_table; From 4571c74fdd4524fb4c7b92ff3b21e40765c4c8fb Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 10 Jan 2023 12:22:33 +0100 Subject: [PATCH 18/49] Fixed build --- src/Storages/StorageS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index edd60a364af..9cb992bd24f 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1057,7 +1057,7 @@ Pipe StorageS3::read( { return std::any_of(virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col){ return col == virtual_col.name; }); }); if (fetch_columns.empty()) - fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical())); + fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns); block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); From ee86afb1256567bb9259106ce1b2116169925a6d Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 10 Jan 2023 11:14:12 -0500 Subject: [PATCH 19/49] add deltalake --- .../table-functions/deltalake.md | 181 +++--------------- 1 file changed, 24 insertions(+), 157 deletions(-) diff --git a/docs/en/sql-reference/table-functions/deltalake.md b/docs/en/sql-reference/table-functions/deltalake.md index 7e3fffe4d8b..af944d70426 100644 --- a/docs/en/sql-reference/table-functions/deltalake.md +++ b/docs/en/sql-reference/table-functions/deltalake.md @@ -3,182 +3,49 @@ slug: /en/sql-reference/table-functions/deltalake sidebar_label: DeltLake --- -# DeltaLake Table Function +# deltaLake Table Function -Provides a read-only table-like interface to [Delta Lake](https://github.com/delta-io/delta) tables in [Amazon S3](https://aws.amazon.com/s3/). +Provides a read-only table-like interface to [Delta Lake](https://github.com/delta-io/delta) tables in Amazon S3. -For example, to query an existing Delta Lake table named `deltalake` in S3: -```sql -CREATE TABLE dl_hits - ENGINE = DeltaLake('https://clickhouse-public-datasets.s3.amazonaws.com/delta_lake/hits/','',''); - -SHOW TABLES; - -DESCRIBE dl_hits; - -SELECT URL, Referer, UserAgent FROM dl_hits WHERE URL IS NOT NULL LIMIT 10; - -SELECT URL, Referer, UserAgent FROM deltaLake('https://clickhouse-public-datasets.s3.amazonaws.com/delta_lake/hits/') WHERE URL IS NOT NULL LIMIT 10; - -``` - -**Syntax** +## Syntax ``` sql -s3(path [,aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,compression]) +deltaLake(path [,aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,compression]) ``` -**Arguments** +## Arguments -- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [here](../../engines/table-engines/integrations/s3.md#wildcards-in-path). -- `format` — The [format](../../interfaces/formats.md#formats) of the file. -- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. -- `compression` — Parameter is optional. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. By default, it will autodetect compression by file extension. +- `path` — Bucket url with path to existing Delta Lake table in S3. +- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. These parameters are optional. If credentials are not specified, they are used from the ClickHouse configuration. For more information see [Using S3 for Data Storage](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3). +- `format` — The [format](/docs/en/interfaces/formats.md/#formats) of the file. +- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. +- `compression` — Parameter is optional. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. By default, compression will be autodetected by the file extension. **Returned value** -A table with the specified structure for reading or writing data in the specified file. +A table with the specified structure for reading data in the specified Delta Lake table in S3. **Examples** -Selecting the first two rows from the table from S3 file `https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/data.csv`: +Selecting rows from the table in S3 `https://clickhouse-public-datasets.s3.amazonaws.com/delta_lake/hits/`: ``` sql -SELECT * -FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/data.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') -LIMIT 2; +SELECT + URL, + UserAgent +FROM deltaLake('https://clickhouse-public-datasets.s3.amazonaws.com/delta_lake/hits/') +WHERE URL IS NOT NULL +LIMIT 2 ``` -``` text -┌─column1─┬─column2─┬─column3─┐ -│ 1 │ 2 │ 3 │ -│ 3 │ 2 │ 1 │ -└─────────┴─────────┴─────────┘ +``` response +┌─URL───────────────────────────────────────────────────────────────────┬─UserAgent─┐ +│ http://auto.ria.ua/search/index.kz/jobinmoscow/detail/55089/hasimages │ 1 │ +│ http://auto.ria.ua/search/index.kz/jobinmoscow.ru/gosushi │ 1 │ +└───────────────────────────────────────────────────────────────────────┴───────────┘ ``` -The similar but from file with `gzip` compression: - -``` sql -SELECT * -FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/data.csv.gz', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32', 'gzip') -LIMIT 2; -``` - -``` text -┌─column1─┬─column2─┬─column3─┐ -│ 1 │ 2 │ 3 │ -│ 3 │ 2 │ 1 │ -└─────────┴─────────┴─────────┘ -``` - -## Usage - -Suppose that we have several files with following URIs on S3: - -- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/some_prefix/some_file_1.csv' -- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/some_prefix/some_file_2.csv' -- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/some_prefix/some_file_3.csv' -- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/some_prefix/some_file_4.csv' -- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/another_prefix/some_file_1.csv' -- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/another_prefix/some_file_2.csv' -- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/another_prefix/some_file_3.csv' -- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/another_prefix/some_file_4.csv' - -Count the amount of rows in files ending with numbers from 1 to 3: - -``` sql -SELECT count(*) -FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/{some,another}_prefix/some_file_{1..3}.csv', 'CSV', 'name String, value UInt32') -``` - -``` text -┌─count()─┐ -│ 18 │ -└─────────┘ -``` - -Count the total amount of rows in all files in these two directories: - -``` sql -SELECT count(*) -FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/{some,another}_prefix/*', 'CSV', 'name String, value UInt32') -``` - -``` text -┌─count()─┐ -│ 24 │ -└─────────┘ -``` - -:::warning -If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. -::: - -Count the total amount of rows in files named `file-000.csv`, `file-001.csv`, … , `file-999.csv`: - -``` sql -SELECT count(*) -FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/big_prefix/file-{000..999}.csv', 'CSV', 'name String, value UInt32'); -``` - -``` text -┌─count()─┐ -│ 12 │ -└─────────┘ -``` - -Insert data into file `test-data.csv.gz`: - -``` sql -INSERT INTO FUNCTION s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip') -VALUES ('test-data', 1), ('test-data-2', 2); -``` - -Insert data into file `test-data.csv.gz` from existing table: - -``` sql -INSERT INTO FUNCTION s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip') -SELECT name, value FROM existing_table; -``` - -Glob ** can be used for recursive directory traversal. Consider the below example, it will fetch all files from `my-test-bucket-768` directory recursively: - -``` sql -SELECT * FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/**', 'CSV', 'name String, value UInt32', 'gzip'); -``` - -The below get data from all `test-data.csv.gz` files from any folder inside `my-test-bucket` directory recursively: - -``` sql -SELECT * FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/**/test-data.csv.gz', 'CSV', 'name String, value UInt32', 'gzip'); -``` - -## Partitioned Write - -If you specify `PARTITION BY` expression when inserting data into `S3` table, a separate file is created for each partition value. Splitting the data into separate files helps to improve reading operations efficiency. - -**Examples** - -1. Using partition ID in a key creates separate files: - -```sql -INSERT INTO TABLE FUNCTION - s3('http://bucket.amazonaws.com/my_bucket/file_{_partition_id}.csv', 'CSV', 'a String, b UInt32, c UInt32') - PARTITION BY a VALUES ('x', 2, 3), ('x', 4, 5), ('y', 11, 12), ('y', 13, 14), ('z', 21, 22), ('z', 23, 24); -``` -As a result, the data is written into three files: `file_x.csv`, `file_y.csv`, and `file_z.csv`. - -2. Using partition ID in a bucket name creates files in different buckets: - -```sql -INSERT INTO TABLE FUNCTION - s3('http://bucket.amazonaws.com/my_bucket_{_partition_id}/file.csv', 'CSV', 'a UInt32, b UInt32, c UInt32') - PARTITION BY a VALUES (1, 2, 3), (1, 4, 5), (10, 11, 12), (10, 13, 14), (20, 21, 22), (20, 23, 24); -``` -As a result, the data is written into three files in different buckets: `my_bucket_1/file.csv`, `my_bucket_10/file.csv`, and `my_bucket_20/file.csv`. - **See Also** -- [S3 engine](../../engines/table-engines/integrations/s3.md) +- [deltaLake engine](/docs/en/engines/table-engines/integrations/deltalake.md) -[Original article](https://clickhouse.com/docs/en/sql-reference/table-functions/s3/) From 7cb3e174191e04f288de69e7fc3e4bea16058335 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 10 Jan 2023 16:17:59 +0000 Subject: [PATCH 20/49] black --- tests/ci/sqlancer_check.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/ci/sqlancer_check.py b/tests/ci/sqlancer_check.py index b286d1a63bc..5b268141484 100644 --- a/tests/ci/sqlancer_check.py +++ b/tests/ci/sqlancer_check.py @@ -30,9 +30,8 @@ IMAGE_NAME = "clickhouse/sqlancer-test" def get_pull_command(docker_image): - return ( - f"docker pull --network=host {docker_image}" - ) + return f"docker pull --network=host {docker_image}" + def get_run_command(download_url, workspace_path, image): return ( From 879ee05218905d1baa1f96e9de2ae4107883c417 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 10 Jan 2023 11:18:33 -0500 Subject: [PATCH 21/49] fix case of names --- docs/en/engines/table-engines/integrations/deltalake.md | 2 +- docs/en/sql-reference/table-functions/deltalake.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/deltalake.md b/docs/en/engines/table-engines/integrations/deltalake.md index 44407e34e38..5ce044680d4 100644 --- a/docs/en/engines/table-engines/integrations/deltalake.md +++ b/docs/en/engines/table-engines/integrations/deltalake.md @@ -29,5 +29,5 @@ CREATE TABLE deltalake ENGINE=DeltaLake('http://mars-doc-test.s3.amazonaws.com/c ## See also -- [DeltaLake table function](../../../sql-reference/table-functions/deltalake.md) +- [deltaLake table function](../../../sql-reference/table-functions/deltalake.md) diff --git a/docs/en/sql-reference/table-functions/deltalake.md b/docs/en/sql-reference/table-functions/deltalake.md index af944d70426..6468e51d757 100644 --- a/docs/en/sql-reference/table-functions/deltalake.md +++ b/docs/en/sql-reference/table-functions/deltalake.md @@ -47,5 +47,5 @@ LIMIT 2 **See Also** -- [deltaLake engine](/docs/en/engines/table-engines/integrations/deltalake.md) +- [DeltaLake engine](/docs/en/engines/table-engines/integrations/deltalake.md) From 71333afd5a9c3d5c62c19066a24c705e5019d382 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Jan 2023 16:40:10 +0000 Subject: [PATCH 22/49] Fixing a test. --- src/Interpreters/MutationsInterpreter.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index f8627f1ff85..e95e53db41b 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -523,11 +523,8 @@ void MutationsInterpreter::prepare(bool dry_run) NamesAndTypesList all_columns = columns_desc.getAllPhysical(); /// Add _row_exists column if it is physically present in the part - if (auto part_storage = dynamic_pointer_cast(storage)) - { - if (part_storage->hasLightweightDeletedMask()) + if (source.hasLightweightDeleteMask()) all_columns.push_back({LightweightDeleteDescription::FILTER_COLUMN}); - } NameSet updated_columns; bool materialize_ttl_recalculate_only = source.materializeTTLRecalculateOnly(); From 7701dc571e84626c0bf81ee67e1c4daccd5efaf7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Jan 2023 18:19:38 +0100 Subject: [PATCH 23/49] Update MutationsInterpreter.cpp --- src/Interpreters/MutationsInterpreter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index e95e53db41b..cec03863c69 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -524,7 +524,7 @@ void MutationsInterpreter::prepare(bool dry_run) /// Add _row_exists column if it is physically present in the part if (source.hasLightweightDeleteMask()) - all_columns.push_back({LightweightDeleteDescription::FILTER_COLUMN}); + all_columns.push_back({LightweightDeleteDescription::FILTER_COLUMN}); NameSet updated_columns; bool materialize_ttl_recalculate_only = source.materializeTTLRecalculateOnly(); From 9a81f27fb22a21be79d36be35b1e28e3be334ed7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Jan 2023 17:32:26 +0000 Subject: [PATCH 24/49] Fix additional_table_filters with minmax/count projection. --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++++ .../01710_projection_additional_filters.reference | 1 + .../0_stateless/01710_projection_additional_filters.sql | 6 ++++++ 3 files changed, 11 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 358d527ae28..6bcfe5a35bd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5960,6 +5960,10 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg if (settings.parallel_replicas_count > 1 || settings.max_parallel_replicas > 1) return std::nullopt; + /// Cannot use projections in case of additional filter. + if (query_info.additional_filter_ast) + return std::nullopt; + auto query_ptr = query_info.original_query; auto * select_query = query_ptr->as(); if (!select_query) diff --git a/tests/queries/0_stateless/01710_projection_additional_filters.reference b/tests/queries/0_stateless/01710_projection_additional_filters.reference index 06b63ea6c2f..31b14cf6359 100644 --- a/tests/queries/0_stateless/01710_projection_additional_filters.reference +++ b/tests/queries/0_stateless/01710_projection_additional_filters.reference @@ -1 +1,2 @@ 0 0 0 +3 diff --git a/tests/queries/0_stateless/01710_projection_additional_filters.sql b/tests/queries/0_stateless/01710_projection_additional_filters.sql index 1633b48ba7e..f12d3e2766b 100644 --- a/tests/queries/0_stateless/01710_projection_additional_filters.sql +++ b/tests/queries/0_stateless/01710_projection_additional_filters.sql @@ -7,3 +7,9 @@ INSERT INTO t SELECT number % 10, number FROM numbers(10000); SELECT count(), min(a), max(a) FROM t SETTINGS additional_table_filters = {'t' : '0'}; DROP TABLE t; + +drop table if exists atf_p; +create table atf_p (x UInt64) engine = MergeTree order by tuple(); +insert into atf_p select number from numbers(10); +select count() from atf_p settings additional_table_filters = {'atf_p': 'x <= 2'}; +drop table atf_p; From 8fa1b070c6a02021b7b3cb858cf7185526720ace Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 10 Jan 2023 19:55:06 +0100 Subject: [PATCH 25/49] minor cleanup in stress/run.sh --- docker/test/stress/run.sh | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index f994e6c2269..7f3e551edbc 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -128,18 +128,12 @@ EOL function stop() { + local max_tries="${1:-90}" local pid # Preserve the pid, since the server can hung after the PID will be deleted. pid="$(cat /var/run/clickhouse-server/clickhouse-server.pid)" - clickhouse stop $max_tries --do-not-kill && return - - if [ -n "$1" ] - then - # temporarily disable it in BC check - clickhouse stop --force - return - fi + clickhouse stop --max-tries "$max_tries" --do-not-kill && return # We failed to stop the server with SIGTERM. Maybe it hang, let's collect stacktraces. kill -TERM "$(pidof gdb)" ||: @@ -465,7 +459,8 @@ if [ "$DISABLE_BC_CHECK" -ne "1" ]; then clickhouse stop --force ) - stop 1 + # Use bigger timeout for previous version + stop 300 mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.stress.log # Start new server From 2af17ec1c0c1217ab5fda4cc95248b7e8852ed8b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 10 Jan 2023 22:19:58 +0300 Subject: [PATCH 26/49] Update clickhouse-test --- tests/clickhouse-test | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 2709ad1eecf..a5c6b3e0bb8 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1544,8 +1544,11 @@ def check_server_started(args): print(" OK") sys.stdout.flush() return True - except (ConnectionError, http.client.ImproperConnectionState): - print(".", end="") + except (ConnectionError, http.client.ImproperConnectionState) as e: + if args.hung_check: + print("Connection error, will retry: ", str(e)) + else: + print(".", end="") sys.stdout.flush() retry_count -= 1 sleep(0.5) From 5cf1b1f61df98fb13b84db39a2e159b80061e1a6 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 10 Jan 2023 16:09:59 -0500 Subject: [PATCH 27/49] feedback --- docs/en/engines/table-engines/integrations/deltalake.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/deltalake.md b/docs/en/engines/table-engines/integrations/deltalake.md index 5ce044680d4..eb4d8e934a7 100644 --- a/docs/en/engines/table-engines/integrations/deltalake.md +++ b/docs/en/engines/table-engines/integrations/deltalake.md @@ -5,7 +5,7 @@ sidebar_label: DeltaLake # DeltaLake Table Engine -This engine provides a read-only integration with existing Delta Lake tables in Amazon S3. +This engine provides a read-only integration with existing [Delta Lake](https://github.com/delta-io/delta) tables in Amazon S3. ## Create Table From 75c04945bd5cd80328837a3ed3b9b28efd43f103 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 10 Jan 2023 16:18:50 -0500 Subject: [PATCH 28/49] spelling --- docs/en/sql-reference/table-functions/deltalake.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/deltalake.md b/docs/en/sql-reference/table-functions/deltalake.md index 6468e51d757..10e7c20e17a 100644 --- a/docs/en/sql-reference/table-functions/deltalake.md +++ b/docs/en/sql-reference/table-functions/deltalake.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/table-functions/deltalake -sidebar_label: DeltLake +sidebar_label: DeltaLake --- # deltaLake Table Function From 563e0e76f929e1366971ab895db7230657eaf802 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 10 Jan 2023 16:59:34 -0500 Subject: [PATCH 29/49] init --- .../table-engines/integrations/hudi.md | 33 +++++++++++++++++++ docs/en/sql-reference/table-functions/hudi.md | 31 +++++++++++++++++ 2 files changed, 64 insertions(+) create mode 100644 docs/en/engines/table-engines/integrations/hudi.md create mode 100644 docs/en/sql-reference/table-functions/hudi.md diff --git a/docs/en/engines/table-engines/integrations/hudi.md b/docs/en/engines/table-engines/integrations/hudi.md new file mode 100644 index 00000000000..6da1634ba5a --- /dev/null +++ b/docs/en/engines/table-engines/integrations/hudi.md @@ -0,0 +1,33 @@ +--- +slug: /en/engines/table-engines/integrations/hudi +sidebar_label: Hudi +--- + +# Hudi Table Engine + +This engine provides a read-only integration with existing Apache [Hudi](https://hudi.apache.org/) tables in Amazon S3. + +## Create Table + +Note that the Hudi table must already exist in S3, this command does not take DDL parameters to create a new table. + +``` sql +CREATE TABLE hudi_table + ENGINE = Hudi(path, [aws_access_key_id, aws_secret_access_key,]) +``` + +**Engine parameters** + +- `path` — Bucket url with the path to an existing Hudi table. +- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). + +**Example** + +```sql +CREATE TABLE hudi_table ENGINE=Hudi('http://mars-doc-test.s3.amazonaws.com/clickhouse-bucket-3/test_table/', 'ABC123', 'Abc+123') +``` + +## See also + +- [hudi table function](/docs/en/sql-reference/table-functions/hudi.md) + diff --git a/docs/en/sql-reference/table-functions/hudi.md b/docs/en/sql-reference/table-functions/hudi.md new file mode 100644 index 00000000000..c1ccd0cda2f --- /dev/null +++ b/docs/en/sql-reference/table-functions/hudi.md @@ -0,0 +1,31 @@ +--- +slug: /en/sql-reference/table-functions/hudi +sidebar_label: Hudi +--- + +# hudi Table Function + +Provides a read-only table-like interface to Apache [Hudi](https://hudi.apache.org/) tables in Amazon S3. + +## Syntax + +``` sql +hudi(path [,aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,compression]) +``` + +## Arguments + +- `path` — Bucket url with the path to an existing Hudi table in S3. +- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. These parameters are optional. If credentials are not specified, they are used from the ClickHouse configuration. For more information see [Using S3 for Data Storage](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3). +- `format` — The [format](/docs/en/interfaces/formats.md/#formats) of the file. +- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. +- `compression` — Parameter is optional. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. By default, compression will be autodetected by the file extension. + +**Returned value** + +A table with the specified structure for reading data in the specified Hudi table in S3. + +**See Also** + +- [Hudi engine](/docs/en/engines/table-engines/integrations/hudi.md) + From a704cf804a222da2c9c1b9a2219659e7fbe7cff7 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 10 Jan 2023 22:17:28 +0000 Subject: [PATCH 30/49] fix --- tests/ci/sqlancer_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/sqlancer_check.py b/tests/ci/sqlancer_check.py index 5b268141484..0e328122b9d 100644 --- a/tests/ci/sqlancer_check.py +++ b/tests/ci/sqlancer_check.py @@ -30,7 +30,7 @@ IMAGE_NAME = "clickhouse/sqlancer-test" def get_pull_command(docker_image): - return f"docker pull --network=host {docker_image}" + return f"docker pull {docker_image}" def get_run_command(download_url, workspace_path, image): From 6027b8ee4df1d2327a9b97d712fbd4e9e12fb2a8 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 10 Jan 2023 23:23:07 +0000 Subject: [PATCH 31/49] Remove redundant code --- .../Passes/AggregateFunctionsArithmericOperationsPass.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp index c25cff117d2..01072e0b3fc 100644 --- a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp +++ b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp @@ -156,7 +156,6 @@ private: { argument->getResultType() }, function_aggregate_function->getParameters(), properties); - auto function_result_type = aggregate_function->getReturnType(); function_node.resolveAsAggregateFunction(std::move(aggregate_function)); } From be4d79e92426e7d0ac899698986d2e08d7d07c3a Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Wed, 14 Dec 2022 13:25:05 +0800 Subject: [PATCH 32/49] Deallocate memory of profile events out of critical section To further shrink the critical section for releasing memory of the profile events (ProfileEventsCountersAndMemory), this commit puts the dealloaction out of the critical section while keeping the memory move under lock. This change could mitigate the contention for ThreadGroupStatus::mutex. --- src/Interpreters/ThreadStatusExt.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index e96a8a4b188..4b757e0be7e 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -342,11 +342,14 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) query_id.clear(); query_context.reset(); + /// The memory of thread_group->finished_threads_counters_memory is temporarily moved to this vector, which is deallocated out of critical section. + std::vector move_to_temp; + /// Avoid leaking of ThreadGroupStatus::finished_threads_counters_memory /// (this is in case someone uses system thread but did not call getProfileEventsCountersAndMemoryForThreads()) { std::lock_guard guard(thread_group->mutex); - auto stats = std::move(thread_group->finished_threads_counters_memory); + move_to_temp = std::move(thread_group->finished_threads_counters_memory); } thread_group.reset(); From 67943676630fb45e8c68b5dfbe199c80031a2cbb Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 11 Jan 2023 10:38:17 +0800 Subject: [PATCH 33/49] fix uts --- .../0_stateless/00189_time_zones_long.sql | 10 +- ...00921_datetime64_compatibility_long.python | 136 +++++++++--------- 2 files changed, 77 insertions(+), 69 deletions(-) diff --git a/tests/queries/0_stateless/00189_time_zones_long.sql b/tests/queries/0_stateless/00189_time_zones_long.sql index cf1b9e9ae1d..5760f6c0447 100644 --- a/tests/queries/0_stateless/00189_time_zones_long.sql +++ b/tests/queries/0_stateless/00189_time_zones_long.sql @@ -120,11 +120,11 @@ SELECT toDayOfMonth(toDateTime(1412106600), 'Pacific/Pitcairn'); /* toDayOfWeek */ SELECT 'toDayOfWeek'; -SELECT toDayOfWeek(toDateTime(1412106600), 'Asia/Istanbul'); -SELECT toDayOfWeek(toDateTime(1412106600), 'Europe/Paris'); -SELECT toDayOfWeek(toDateTime(1412106600), 'Europe/London'); -SELECT toDayOfWeek(toDateTime(1412106600), 'Asia/Tokyo'); -SELECT toDayOfWeek(toDateTime(1412106600), 'Pacific/Pitcairn'); +SELECT toDayOfWeek(toDateTime(1412106600), 0, 'Asia/Istanbul'); +SELECT toDayOfWeek(toDateTime(1412106600), 0, 'Europe/Paris'); +SELECT toDayOfWeek(toDateTime(1412106600), 0, 'Europe/London'); +SELECT toDayOfWeek(toDateTime(1412106600), 0, 'Asia/Tokyo'); +SELECT toDayOfWeek(toDateTime(1412106600), 0, 'Pacific/Pitcairn'); /* toHour */ diff --git a/tests/queries/0_stateless/00921_datetime64_compatibility_long.python b/tests/queries/0_stateless/00921_datetime64_compatibility_long.python index e3cd7ee6d36..2706c0f5b12 100644 --- a/tests/queries/0_stateless/00921_datetime64_compatibility_long.python +++ b/tests/queries/0_stateless/00921_datetime64_compatibility_long.python @@ -7,14 +7,14 @@ import sys import argparse # Create SQL statement to verify dateTime64 is accepted as argument to functions taking DateTime. -FUNCTIONS=""" +FUNCTIONS = """ toTimeZone(N, 'UTC') toYear(N, 'Asia/Istanbul') toQuarter(N, 'Asia/Istanbul') toMonth(N, 'Asia/Istanbul') toDayOfYear(N, 'Asia/Istanbul') toDayOfMonth(N, 'Asia/Istanbul') -toDayOfWeek(N, 'Asia/Istanbul') +toDayOfWeek(N, 0, 'Asia/Istanbul') toHour(N, 'Asia/Istanbul') toMinute(N, 'Asia/Istanbul') toSecond(N, 'Asia/Istanbul') @@ -90,68 +90,51 @@ formatDateTime(N, '%C %d %D %e %F %H %I %j %m %M %p %R %S %T %u %V %w %y %Y %%', extra_ops = [ # With same type: ( - ['N {op} N'], + ["N {op} N"], { - 'op': - [ - '- ', # does not work, but should it? - '+ ', # does not work, but should it? - '!=', '==', # equality and inequality supposed to take sub-second part in account - '< ', - '<=', - '> ', - '>=' + "op": [ + "- ", # does not work, but should it? + "+ ", # does not work, but should it? + "!=", + "==", # equality and inequality supposed to take sub-second part in account + "< ", + "<=", + "> ", + ">=", ] - } + }, ), # With other DateTime types: ( - [ - 'N {op} {arg}', - '{arg} {op} N' - ], + ["N {op} {arg}", "{arg} {op} N"], { - 'op': - [ - '-', # does not work, but should it? - '!=', '==', + "op": [ + "-", # does not work, but should it? + "!=", + "==", # these are naturally expected to work, but they don't: - '< ', - '<=', - '> ', - '>=' + "< ", + "<=", + "> ", + ">=", ], - 'arg': ['DT', 'D', 'DT64'], - } + "arg": ["DT", "D", "DT64"], + }, ), # With arithmetic types ( - [ - 'N {op} {arg}', - '{arg} {op} N' - ], + ["N {op} {arg}", "{arg} {op} N"], { - 'op': - [ - '+ ', - '- ', - '==', - '!=', - '< ', - '<=', - '> ', - '>=' - ], - 'arg': - [ - 'toUInt8(1)', - 'toInt8(-1)', - 'toUInt16(1)', - 'toInt16(-1)', - 'toUInt32(1)', - 'toInt32(-1)', - 'toUInt64(1)', - 'toInt64(-1)' + "op": ["+ ", "- ", "==", "!=", "< ", "<=", "> ", ">="], + "arg": [ + "toUInt8(1)", + "toInt8(-1)", + "toUInt16(1)", + "toInt16(-1)", + "toUInt32(1)", + "toInt32(-1)", + "toUInt64(1)", + "toInt64(-1)", ], }, ), @@ -167,14 +150,17 @@ for funcs, args in extra_ops: # filter out empty lines and commented out lines COMMENTED_OUT_LINE_RE = re.compile(r"^\s*#") -FUNCTIONS = list([f for f in FUNCTIONS if len(f) != 0 and COMMENTED_OUT_LINE_RE.match(f) == None]) -TYPES = ['D', 'DT', 'DT64'] +FUNCTIONS = list( + [f for f in FUNCTIONS if len(f) != 0 and COMMENTED_OUT_LINE_RE.match(f) == None] +) +TYPES = ["D", "DT", "DT64"] + def escape_string(s): if sys.version_info[0] > 2: - return s.encode('unicode_escape').decode('utf-8').replace("'", "\\'") + return s.encode("unicode_escape").decode("utf-8").replace("'", "\\'") else: - return s.encode('string-escape').decode('utf-8') + return s.encode("string-escape").decode("utf-8") def execute_functions_for_types(functions, types): @@ -186,18 +172,39 @@ def execute_functions_for_types(functions, types): WITH \ toDateTime64('2019-09-16 19:20:11.234', 3, 'Europe/Minsk') as DT64, \ toDateTime('2019-09-16 19:20:11', 'Europe/Minsk') as DT, \ -toDate('2019-09-16') as D, {X} as N".format(X=dt) - print(("""{prologue} SELECT toTypeName(r), {func} as r FORMAT CSV;""".format(prologue=prologue, func=func))) +toDate('2019-09-16') as D, {X} as N".format( + X=dt + ) + print( + ( + """{prologue} SELECT toTypeName(r), {func} as r FORMAT CSV;""".format( + prologue=prologue, func=func + ) + ) + ) print("""SELECT '------------------------------------------';""") + def main(): def parse_args(): parser = argparse.ArgumentParser() - parser.add_argument('--functions_re', type=re.compile, help="RE to enable functions", default=None) - parser.add_argument('--types_re', - type=lambda s: re.compile('^(' + s + ')$'), - help="RE to enable types, supported types: " + ",".join(TYPES), default=None) - parser.add_argument('--list_functions', action='store_true', help="List all functions to be tested and exit") + parser.add_argument( + "--functions_re", + type=re.compile, + help="RE to enable functions", + default=None, + ) + parser.add_argument( + "--types_re", + type=lambda s: re.compile("^(" + s + ")$"), + help="RE to enable types, supported types: " + ",".join(TYPES), + default=None, + ) + parser.add_argument( + "--list_functions", + action="store_true", + help="List all functions to be tested and exit", + ) return parser.parse_args() args = parse_args() @@ -223,5 +230,6 @@ def main(): execute_functions_for_types(functions, types) -if __name__ == '__main__': + +if __name__ == "__main__": exit(main()) From af0c3d751fb602a807dc51716c30ff9fbb0d1999 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 11 Jan 2023 14:32:28 +0800 Subject: [PATCH 34/49] fix uts --- .../0_stateless/00921_datetime64_compatibility_long.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference b/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference index 8d28a69ff3d..8a168ed0e9e 100644 --- a/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference +++ b/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference @@ -28,7 +28,7 @@ SELECT toDayOfMonth(N, \'Asia/Istanbul\') "UInt8",16 "UInt8",16 ------------------------------------------ -SELECT toDayOfWeek(N, \'Asia/Istanbul\') +SELECT toDayOfWeek(N, 0, \'Asia/Istanbul\') "UInt8",1 "UInt8",1 "UInt8",1 From 4dd628cd86577f7f0640fd5b50d0f9040fad1f88 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 11 Jan 2023 12:05:31 +0100 Subject: [PATCH 35/49] Clean trash from changelog for v22.3.16.1190-lts --- docs/changelogs/v22.3.16.1190-lts.md | 159 --------------------------- 1 file changed, 159 deletions(-) diff --git a/docs/changelogs/v22.3.16.1190-lts.md b/docs/changelogs/v22.3.16.1190-lts.md index 1b22d9a88be..a43d34551ca 100644 --- a/docs/changelogs/v22.3.16.1190-lts.md +++ b/docs/changelogs/v22.3.16.1190-lts.md @@ -7,186 +7,27 @@ sidebar_label: 2023 ### ClickHouse release v22.3.16.1190-lts (bb4e0934e5a) FIXME as compared to v22.10.1.1877-stable (98ab5a3c189) -#### Backward Incompatible Change -* JSONExtract family of functions will now attempt to coerce to the request type. [#41502](https://github.com/ClickHouse/ClickHouse/pull/41502) ([Márcio Martins](https://github.com/marcioapm)). -* Backported in [#43484](https://github.com/ClickHouse/ClickHouse/issues/43484): Fixed backward incompatibility in (de)serialization of states of `min`, `max`, `any*`, `argMin`, `argMax` aggregate functions with `String` argument. The incompatibility was introduced in https://github.com/ClickHouse/ClickHouse/pull/41431 and affects 22.9, 22.10 and 22.11 branches (fixed since 22.9.6, 22.10.4 and 22.11.2 correspondingly). Some minor releases of 22.3, 22.7 and 22.8 branches are also affected: 22.3.13...22.3.14 (fixed since 22.3.15), 22.8.6...22.8.9 (fixed since 22.8.10), 22.7.6 and newer (will not be fixed in 22.7, we recommend to upgrade from 22.7.* to 22.8.10 or newer). This release note does not concern users that have never used affected versions. Incompatible versions append extra `'\0'` to strings when reading states of the aggregate functions mentioned above. For example, if an older version saved state of `anyState('foobar')` to `state_column` then incompatible version will print `'foobar\0'` on `anyMerge(state_column)`. Also incompatible versions write states of the aggregate functions without trailing `'\0'`. Newer versions (that have the fix) can correctly read data written by all versions including incompatible versions, except one corner case. If an incompatible version saved a state with a string that actually ends with null character, then newer version will trim trailing `'\0'` when reading state of affected aggregate function. For example, if an incompatible version saved state of `anyState('abrac\0dabra\0')` to `state_column` then newer versions will print `'abrac\0dabra'` on `anyMerge(state_column)`. The issue also affects distributed queries when an incompatible version works in a cluster together with older or newer versions. [#43038](https://github.com/ClickHouse/ClickHouse/pull/43038) ([Raúl Marín](https://github.com/Algunenano)). - -#### New Feature -* - Add function `displayName`, closes [#36770](https://github.com/ClickHouse/ClickHouse/issues/36770). [#37681](https://github.com/ClickHouse/ClickHouse/pull/37681) ([hongbin](https://github.com/xlwh)). -* Add Hudi and DeltaLake table engines, read-only, only for tables on S3. [#41054](https://github.com/ClickHouse/ClickHouse/pull/41054) ([Daniil Rubin](https://github.com/rubin-do)). -* Add 4LW command `csnp` for manually creating snapshots. Additionally, `lgif` was added to get Raft information for a specific node (e.g. index of last created snapshot, last committed log index). [#41766](https://github.com/ClickHouse/ClickHouse/pull/41766) ([JackyWoo](https://github.com/JackyWoo)). -* Add function ascii like in spark: https://spark.apache.org/docs/latest/api/sql/#ascii. [#42670](https://github.com/ClickHouse/ClickHouse/pull/42670) ([李扬](https://github.com/taiyang-li)). -* Published function `formatReadableDecimalSize`. [#42774](https://github.com/ClickHouse/ClickHouse/pull/42774) ([Alejandro](https://github.com/alexon1234)). - -#### Performance Improvement -* Currently, the only saturable operators are And and Or, and their code paths are affected by this change. [#42214](https://github.com/ClickHouse/ClickHouse/pull/42214) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). -* `match` function can use the index if it's a condition on string prefix. This closes [#37333](https://github.com/ClickHouse/ClickHouse/issues/37333). [#42458](https://github.com/ClickHouse/ClickHouse/pull/42458) ([clarkcaoliu](https://github.com/Clark0)). -* Support parallel parsing for LineAsString input format. This improves performance just slightly. This closes [#42502](https://github.com/ClickHouse/ClickHouse/issues/42502). [#42780](https://github.com/ClickHouse/ClickHouse/pull/42780) ([Kruglov Pavel](https://github.com/Avogar)). -* Keeper performance improvement: improve commit performance for cases when many different nodes have uncommitted states. This should help with cases when a follower node can't sync fast enough. [#42926](https://github.com/ClickHouse/ClickHouse/pull/42926) ([Antonio Andelic](https://github.com/antonio2368)). - #### Improvement -* Support type `Object` inside other types, e.g. `Array(JSON)`. [#36969](https://github.com/ClickHouse/ClickHouse/pull/36969) ([Anton Popov](https://github.com/CurtizJ)). * Backported in [#42527](https://github.com/ClickHouse/ClickHouse/issues/42527): Fix issue with passing MySQL timeouts for MySQL database engine and MySQL table function. Closes [#34168](https://github.com/ClickHouse/ClickHouse/issues/34168)?notification_referrer_id=NT_kwDOAzsV57MzMDMxNjAzNTY5OjU0MjAzODc5. [#40751](https://github.com/ClickHouse/ClickHouse/pull/40751) ([Kseniia Sumarokova](https://github.com/kssenii)). -* ClickHouse Client and ClickHouse Local will show progress by default even in non-interactive mode. If `/dev/tty` is available, the progress will be rendered directly to the terminal, without writing to stderr. It allows to get progress even if stderr is redirected to a file, and the file will not be polluted by terminal escape sequences. The progress can be disabled by `--progress false`. This closes [#32238](https://github.com/ClickHouse/ClickHouse/issues/32238). [#42003](https://github.com/ClickHouse/ClickHouse/pull/42003) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* 1. Add, subtract and negate operations are now available on Intervals. In case when the types of Intervals are different they will be transformed into the Tuple of those types. 2. A tuple of intervals can be added to or subtracted from a Date/DateTime field. 3. Added parsing of Intervals with different types, for example: `INTERVAL '1 HOUR 1 MINUTE 1 SECOND'`. [#42195](https://github.com/ClickHouse/ClickHouse/pull/42195) ([Nikolay Degterinsky](https://github.com/evillique)). -* - Add `notLike` to key condition atom map, so condition like `NOT LIKE 'prefix%'` can use primary index. [#42209](https://github.com/ClickHouse/ClickHouse/pull/42209) ([Duc Canh Le](https://github.com/canhld94)). -* Add support for FixedString input to base64 coding functions. [#42285](https://github.com/ClickHouse/ClickHouse/pull/42285) ([ltrk2](https://github.com/ltrk2)). -* Add columns `bytes_on_disk` and `path` to `system.detached_parts`. Closes [#42264](https://github.com/ClickHouse/ClickHouse/issues/42264). [#42303](https://github.com/ClickHouse/ClickHouse/pull/42303) ([chen](https://github.com/xiedeyantu)). -* Added ** glob support for recursive directory traversal to filesystem and S3. resolves [#36316](https://github.com/ClickHouse/ClickHouse/issues/36316). [#42376](https://github.com/ClickHouse/ClickHouse/pull/42376) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). -* Mask passwords and secret keys both in `system.query_log` and `/var/log/clickhouse-server/*.log` and also in error messages. [#42484](https://github.com/ClickHouse/ClickHouse/pull/42484) ([Vitaly Baranov](https://github.com/vitlibar)). -* Add a new variable call `limit` in query_info, indicating whether this query is a limit-trivial query. If so, we will adjust the approximate total rows for later estimation. Closes [#7071](https://github.com/ClickHouse/ClickHouse/issues/7071). [#42580](https://github.com/ClickHouse/ClickHouse/pull/42580) ([Han Fei](https://github.com/hanfei1991)). -* Implement `ATTACH` of `MergeTree` table for `s3_plain` disk (plus some fixes for `s3_plain`). [#42628](https://github.com/ClickHouse/ClickHouse/pull/42628) ([Azat Khuzhin](https://github.com/azat)). -* Fix no progress indication on INSERT FROM INFILE. Closes [#42548](https://github.com/ClickHouse/ClickHouse/issues/42548). [#42634](https://github.com/ClickHouse/ClickHouse/pull/42634) ([chen](https://github.com/xiedeyantu)). -* Add `min_age_to_force_merge_on_partition_only` setting to optimize old parts for the entire partition only. [#42659](https://github.com/ClickHouse/ClickHouse/pull/42659) ([Antonio Andelic](https://github.com/antonio2368)). -* Throttling algorithm changed to token bucket. [#42665](https://github.com/ClickHouse/ClickHouse/pull/42665) ([Sergei Trifonov](https://github.com/serxa)). -* Added new field allow_readonly in system.table_functions to allow using table functions in readonly mode resolves [#42414](https://github.com/ClickHouse/ClickHouse/issues/42414) Implementation: * Added a new field allow_readonly to table system.table_functions. * Updated to use new field allow_readonly to allow using table functions in readonly mode. Testing: * Added a test for filesystem tests/queries/0_stateless/02473_functions_in_readonly_mode.sh Documentation: * Updated the english documentation for Table Functions. [#42708](https://github.com/ClickHouse/ClickHouse/pull/42708) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). -* Allow to use Date32 arguments for formatDateTime and FROM_UNIXTIME functions. [#42737](https://github.com/ClickHouse/ClickHouse/pull/42737) ([Roman Vasin](https://github.com/rvasin)). -* Backported in [#42839](https://github.com/ClickHouse/ClickHouse/issues/42839): Update tzdata to 2022f. Mexico will no longer observe DST except near the US border: https://www.timeanddate.com/news/time/mexico-abolishes-dst-2022.html. Chihuahua moves to year-round UTC-6 on 2022-10-30. Fiji no longer observes DST. See https://github.com/google/cctz/pull/235 and https://bugs.launchpad.net/ubuntu/+source/tzdata/+bug/1995209. [#42796](https://github.com/ClickHouse/ClickHouse/pull/42796) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Update tzdata to 2022f. Mexico will no longer observe DST except near the US border: https://www.timeanddate.com/news/time/mexico-abolishes-dst-2022.html. Chihuahua moves to year-round UTC-6 on 2022-10-30. Fiji no longer observes DST. See https://github.com/google/cctz/pull/235 and https://bugs.launchpad.net/ubuntu/+source/tzdata/+bug/1995209. [#42796](https://github.com/ClickHouse/ClickHouse/pull/42796) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add `FailedAsyncInsertQuery` event metric for async inserts. [#42814](https://github.com/ClickHouse/ClickHouse/pull/42814) ([Krzysztof Góralski](https://github.com/kgoralski)). -* Increase the size of upload part exponentially for backup to S3. [#42833](https://github.com/ClickHouse/ClickHouse/pull/42833) ([Vitaly Baranov](https://github.com/vitlibar)). #### Bug Fix * Backported in [#43829](https://github.com/ClickHouse/ClickHouse/issues/43829): Updated normaliser to clone the alias ast. resolves [#42452](https://github.com/ClickHouse/ClickHouse/issues/42452) Implementation: * Updated QueryNormalizer to clone alias ast, when its replaced. Previously just assigning the same leads to exception in LogicalExpressinsOptimizer as it would be the same parent being inserted again. * This bug is not seen with new analyser (allow_experimental_analyzer), so no changes for it. I added a test for the same. [#42827](https://github.com/ClickHouse/ClickHouse/pull/42827) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). #### Build/Testing/Packaging Improvement -* Run SQLancer for each pull request and commit to master. [SQLancer](https://github.com/sqlancer/sqlancer) is an OpenSource fuzzer that focuses on automatic detection of logical bugs. [#42397](https://github.com/ClickHouse/ClickHouse/pull/42397) ([Ilya Yatsishin](https://github.com/qoega)). -* Update to latest zlib-ng. [#42463](https://github.com/ClickHouse/ClickHouse/pull/42463) ([Boris Kuschel](https://github.com/bkuschel)). -* use llvm `l64.lld` in macOS suppress ld warnings, close [#42282](https://github.com/ClickHouse/ClickHouse/issues/42282). [#42470](https://github.com/ClickHouse/ClickHouse/pull/42470) ([Lloyd-Pottiger](https://github.com/Lloyd-Pottiger)). -* * Improve bugfix validation check: fix bug with skipping the check, port separate status in CI, run after check labels and style check. Close [#40349](https://github.com/ClickHouse/ClickHouse/issues/40349). [#42702](https://github.com/ClickHouse/ClickHouse/pull/42702) ([Vladimir C](https://github.com/vdimir)). -* Backported in [#43050](https://github.com/ClickHouse/ClickHouse/issues/43050): Wait for all files are in sync before archiving them in integration tests. [#42891](https://github.com/ClickHouse/ClickHouse/pull/42891) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Wait for all files are in sync before archiving them in integration tests. [#42891](https://github.com/ClickHouse/ClickHouse/pull/42891) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Use https://github.com/matus-chochlik/ctcache for clang-tidy results caching. [#42913](https://github.com/ClickHouse/ClickHouse/pull/42913) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Backported in [#42963](https://github.com/ClickHouse/ClickHouse/issues/42963): Before the fix, the user-defined config was preserved by RPM in `$file.rpmsave`. The PR fixes it and won't replace the user's files from packages. [#42936](https://github.com/ClickHouse/ClickHouse/pull/42936) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Before the fix, the user-defined config was preserved by RPM in `$file.rpmsave`. The PR fixes it and won't replace the user's files from packages. [#42936](https://github.com/ClickHouse/ClickHouse/pull/42936) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Backported in [#43039](https://github.com/ClickHouse/ClickHouse/issues/43039): Add a CI step to mark commits as ready for release; soft-forbid launching a release script from branches but master. [#43017](https://github.com/ClickHouse/ClickHouse/pull/43017) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Add a CI step to mark commits as ready for release; soft-forbid launching a release script from branches but master. [#43017](https://github.com/ClickHouse/ClickHouse/pull/43017) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#44109](https://github.com/ClickHouse/ClickHouse/issues/44109): Bring sha512 sums back to the building step. [#44017](https://github.com/ClickHouse/ClickHouse/pull/44017) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#44431](https://github.com/ClickHouse/ClickHouse/issues/44431): Kill stress tests after 2.5h in case of hanging process. [#44214](https://github.com/ClickHouse/ClickHouse/pull/44214) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Backported in [#44557](https://github.com/ClickHouse/ClickHouse/issues/44557): Retry the integration tests on compressing errors. [#44529](https://github.com/ClickHouse/ClickHouse/pull/44529) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). #### Bug Fix (user-visible misbehavior in official stable or prestable release) -* Fix schema inference in s3Cluster and improve in hdfsCluster. [#41979](https://github.com/ClickHouse/ClickHouse/pull/41979) ([Kruglov Pavel](https://github.com/Avogar)). -* Fix retries while reading from http table engines / table function. (retrtiable errors could be retries more times than needed, non-retrialble errors resulted in failed assertion in code). [#42224](https://github.com/ClickHouse/ClickHouse/pull/42224) ([Kseniia Sumarokova](https://github.com/kssenii)). -* A segmentation fault related to DNS & c-ares has been reported. The below error ocurred in multiple threads: ``` 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008088 [ 356 ] {} BaseDaemon: ######################################## 2022-09-28 15:41:19.008,"2022.09.28 15:41:19.008147 [ 356 ] {} BaseDaemon: (version 22.8.5.29 (official build), build id: 92504ACA0B8E2267) (from thread 353) (no query) Received signal Segmentation fault (11)" 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008196 [ 356 ] {} BaseDaemon: Address: 0xf Access: write. Address not mapped to object. 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008216 [ 356 ] {} BaseDaemon: Stack trace: 0x188f8212 0x1626851b 0x1626a69e 0x16269b3f 0x16267eab 0x13cf8284 0x13d24afc 0x13c5217e 0x14ec2495 0x15ba440f 0x15b9d13b 0x15bb2699 0x1891ccb3 0x1891e00d 0x18ae0769 0x18ade022 0x7f76aa985609 0x7f76aa8aa133 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008274 [ 356 ] {} BaseDaemon: 2. Poco::Net::IPAddress::family() const @ 0x188f8212 in /usr/bin/clickhouse 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008297 [ 356 ] {} BaseDaemon: 3. ? @ 0x1626851b in /usr/bin/clickhouse 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008309 [ 356 ] {} BaseDaemon: 4. ? @ 0x1626a69e in /usr/bin/clickhouse ```. [#42234](https://github.com/ClickHouse/ClickHouse/pull/42234) ([Arthur Passos](https://github.com/arthurpassos)). -* Fix `LOGICAL_ERROR` `Arguments of 'plus' have incorrect data types` which may happen in PK analysis (monotonicity check). Fix invalid PK analysis for monotonic binary functions with first constant argument. [#42410](https://github.com/ClickHouse/ClickHouse/pull/42410) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix incorrect key analysis when key types cannot be inside Nullable. This fixes [#42456](https://github.com/ClickHouse/ClickHouse/issues/42456). [#42469](https://github.com/ClickHouse/ClickHouse/pull/42469) ([Amos Bird](https://github.com/amosbird)). -* Fix typo in setting name that led to bad usage of schema inference cache while using setting `input_format_csv_use_best_effort_in_schema_inference`. Closes [#41735](https://github.com/ClickHouse/ClickHouse/issues/41735). [#42536](https://github.com/ClickHouse/ClickHouse/pull/42536) ([Kruglov Pavel](https://github.com/Avogar)). -* Fix create Set with wrong header when data type is LowCardinality. Closes [#42460](https://github.com/ClickHouse/ClickHouse/issues/42460). [#42579](https://github.com/ClickHouse/ClickHouse/pull/42579) ([flynn](https://github.com/ucasfl)). -* `(U)Int128` and `(U)Int256` values are correctly checked in `PREWHERE`. [#42605](https://github.com/ClickHouse/ClickHouse/pull/42605) ([Antonio Andelic](https://github.com/antonio2368)). -* Fix a bug in ParserFunction that could have led to a segmentation fault. [#42724](https://github.com/ClickHouse/ClickHouse/pull/42724) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fix truncate table does not hold lock correctly. [#42728](https://github.com/ClickHouse/ClickHouse/pull/42728) ([flynn](https://github.com/ucasfl)). -* Fix possible SIGSEGV for web disks when file does not exists (or `OPTIMIZE TABLE FINAL`, that also can got the same error eventually). [#42767](https://github.com/ClickHouse/ClickHouse/pull/42767) ([Azat Khuzhin](https://github.com/azat)). -* Fix `auth_type` mapping in `system.session_log`, by including `SSL_CERTIFICATE` for the enum values. [#42782](https://github.com/ClickHouse/ClickHouse/pull/42782) ([Miel Donkers](https://github.com/mdonkers)). -* Fix stack-use-after-return under ASAN build in ParserCreateUserQuery. [#42804](https://github.com/ClickHouse/ClickHouse/pull/42804) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fix lowerUTF8()/upperUTF8() in case of symbol was in between 16-byte boundary (very frequent case of you have strings > 16 bytes long). [#42812](https://github.com/ClickHouse/ClickHouse/pull/42812) ([Azat Khuzhin](https://github.com/azat)). -* Additional bound check was added to lz4 decompression routine to fix misbehaviour in case of malformed input. [#42868](https://github.com/ClickHouse/ClickHouse/pull/42868) ([Nikita Taranov](https://github.com/nickitat)). -* Fix rare possible hung on query cancellation. [#42874](https://github.com/ClickHouse/ClickHouse/pull/42874) ([Azat Khuzhin](https://github.com/azat)). -* * Fix incorrect saved_block_sample with multiple disjuncts in hash join, close [#42832](https://github.com/ClickHouse/ClickHouse/issues/42832). [#42876](https://github.com/ClickHouse/ClickHouse/pull/42876) ([Vladimir C](https://github.com/vdimir)). -* A null pointer will be generated when select if as from ‘three table join’ , For example, the SQL:. [#42883](https://github.com/ClickHouse/ClickHouse/pull/42883) ([zzsmdfj](https://github.com/zzsmdfj)). -* Fix memory sanitizer report in ClusterDiscovery, close [#42763](https://github.com/ClickHouse/ClickHouse/issues/42763). [#42905](https://github.com/ClickHouse/ClickHouse/pull/42905) ([Vladimir C](https://github.com/vdimir)). -* Fix rare NOT_FOUND_COLUMN_IN_BLOCK error when projection is possible to use but there is no projection available. This fixes [#42771](https://github.com/ClickHouse/ClickHouse/issues/42771) . The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/25563. [#42938](https://github.com/ClickHouse/ClickHouse/pull/42938) ([Amos Bird](https://github.com/amosbird)). -* Fix ATTACH TABLE in PostgreSQL database engine if the table contains DATETIME data type. Closes [#42817](https://github.com/ClickHouse/ClickHouse/issues/42817). [#42960](https://github.com/ClickHouse/ClickHouse/pull/42960) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix lambda parsing. Closes [#41848](https://github.com/ClickHouse/ClickHouse/issues/41848). [#42979](https://github.com/ClickHouse/ClickHouse/pull/42979) ([Nikolay Degterinsky](https://github.com/evillique)). * Backported in [#43512](https://github.com/ClickHouse/ClickHouse/issues/43512): - Fix several buffer over-reads. [#43159](https://github.com/ClickHouse/ClickHouse/pull/43159) ([Raúl Marín](https://github.com/Algunenano)). * Backported in [#43750](https://github.com/ClickHouse/ClickHouse/issues/43750): An issue with the following exception has been reported while trying to read a Parquet file from S3 into ClickHouse:. [#43297](https://github.com/ClickHouse/ClickHouse/pull/43297) ([Arthur Passos](https://github.com/arthurpassos)). -* Backported in [#43427](https://github.com/ClickHouse/ClickHouse/issues/43427): Fixed queries with `SAMPLE BY` with prewhere optimization on tables using `Merge` engine. [#43315](https://github.com/ClickHouse/ClickHouse/pull/43315) ([Antonio Andelic](https://github.com/antonio2368)). * Backported in [#43616](https://github.com/ClickHouse/ClickHouse/issues/43616): Fix sumMap() for Nullable(Decimal()). [#43414](https://github.com/ClickHouse/ClickHouse/pull/43414) ([Azat Khuzhin](https://github.com/azat)). -* Backported in [#43720](https://github.com/ClickHouse/ClickHouse/issues/43720): Fixed primary key analysis with conditions involving `toString(enum)`. [#43596](https://github.com/ClickHouse/ClickHouse/pull/43596) ([Nikita Taranov](https://github.com/nickitat)). * Backported in [#43885](https://github.com/ClickHouse/ClickHouse/issues/43885): Fixed `ALTER ... RESET SETTING` with `ON CLUSTER`. It could be applied to one replica only. Fixes [#43843](https://github.com/ClickHouse/ClickHouse/issues/43843). [#43848](https://github.com/ClickHouse/ClickHouse/pull/43848) ([Elena Torró](https://github.com/elenatorro)). * Backported in [#44179](https://github.com/ClickHouse/ClickHouse/issues/44179): Fix undefined behavior in the `quantiles` function, which might lead to uninitialized memory. Found by fuzzer. This closes [#44066](https://github.com/ClickHouse/ClickHouse/issues/44066). [#44067](https://github.com/ClickHouse/ClickHouse/pull/44067) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Backported in [#44283](https://github.com/ClickHouse/ClickHouse/issues/44283): Prevent `ReadonlyReplica` metric from having negative values. [#44220](https://github.com/ClickHouse/ClickHouse/pull/44220) ([Antonio Andelic](https://github.com/antonio2368)). -#### Build Improvement - -* ... Add support for format ipv6 on s390x. [#42412](https://github.com/ClickHouse/ClickHouse/pull/42412) ([Suzy Wang](https://github.com/SuzyWangIBMer)). - #### NO CL ENTRY -* NO CL ENTRY: 'Revert "Sonar Cloud Workflow"'. [#42725](https://github.com/ClickHouse/ClickHouse/pull/42725) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * NO CL ENTRY: 'Fix multipart upload for large S3 object, backport to 22.3'. [#44217](https://github.com/ClickHouse/ClickHouse/pull/44217) ([ianton-ru](https://github.com/ianton-ru)). -#### NOT FOR CHANGELOG / INSIGNIFICANT - -* Build with libcxx(abi) 15 [#42513](https://github.com/ClickHouse/ClickHouse/pull/42513) ([Robert Schulze](https://github.com/rschu1ze)). -* Sonar Cloud Workflow [#42534](https://github.com/ClickHouse/ClickHouse/pull/42534) ([Julio Jimenez](https://github.com/juliojimenez)). -* Invalid type in where for Merge table (logical error) [#42576](https://github.com/ClickHouse/ClickHouse/pull/42576) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix frequent memory drift message and clarify things in comments [#42582](https://github.com/ClickHouse/ClickHouse/pull/42582) ([Azat Khuzhin](https://github.com/azat)). -* Try to save `IDataPartStorage` interface [#42618](https://github.com/ClickHouse/ClickHouse/pull/42618) ([Anton Popov](https://github.com/CurtizJ)). -* Analyzer change setting into allow_experimental_analyzer [#42649](https://github.com/ClickHouse/ClickHouse/pull/42649) ([Maksim Kita](https://github.com/kitaisreal)). -* Analyzer IQueryTreeNode remove getName method [#42651](https://github.com/ClickHouse/ClickHouse/pull/42651) ([Maksim Kita](https://github.com/kitaisreal)). -* Minor fix iotest_nonblock build [#42658](https://github.com/ClickHouse/ClickHouse/pull/42658) ([Jordi Villar](https://github.com/jrdi)). -* Add tests and doc for some url-related functions [#42664](https://github.com/ClickHouse/ClickHouse/pull/42664) ([Vladimir C](https://github.com/vdimir)). -* Update version_date.tsv and changelogs after v22.10.1.1875-stable [#42676](https://github.com/ClickHouse/ClickHouse/pull/42676) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Fix error handling in clickhouse_helper.py [#42678](https://github.com/ClickHouse/ClickHouse/pull/42678) ([Ilya Yatsishin](https://github.com/qoega)). -* Fix execution of version_helper.py to use git tweaks [#42679](https://github.com/ClickHouse/ClickHouse/pull/42679) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* MergeTree indexes use RPNBuilderTree [#42681](https://github.com/ClickHouse/ClickHouse/pull/42681) ([Maksim Kita](https://github.com/kitaisreal)). -* Always run `BuilderReport` and `BuilderSpecialReport` in all CI types [#42684](https://github.com/ClickHouse/ClickHouse/pull/42684) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Always run `BuilderReport` and `BuilderSpecialReport` in all CI types [#42684](https://github.com/ClickHouse/ClickHouse/pull/42684) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Update version after release [#42699](https://github.com/ClickHouse/ClickHouse/pull/42699) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Update version_date.tsv and changelogs after v22.10.1.1877-stable [#42700](https://github.com/ClickHouse/ClickHouse/pull/42700) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* OrderByLimitByDuplicateEliminationPass improve performance [#42704](https://github.com/ClickHouse/ClickHouse/pull/42704) ([Maksim Kita](https://github.com/kitaisreal)). -* Analyzer improve subqueries representation [#42705](https://github.com/ClickHouse/ClickHouse/pull/42705) ([Maksim Kita](https://github.com/kitaisreal)). -* Update version_date.tsv and changelogs after v22.9.4.32-stable [#42712](https://github.com/ClickHouse/ClickHouse/pull/42712) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Update version_date.tsv and changelogs after v22.8.7.34-lts [#42713](https://github.com/ClickHouse/ClickHouse/pull/42713) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Update version_date.tsv and changelogs after v22.7.7.24-stable [#42714](https://github.com/ClickHouse/ClickHouse/pull/42714) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Move SonarCloud Job to nightly [#42718](https://github.com/ClickHouse/ClickHouse/pull/42718) ([Julio Jimenez](https://github.com/juliojimenez)). -* Update version_date.tsv and changelogs after v22.8.8.3-lts [#42738](https://github.com/ClickHouse/ClickHouse/pull/42738) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Minor fix implicit cast CaresPTRResolver [#42747](https://github.com/ClickHouse/ClickHouse/pull/42747) ([Jordi Villar](https://github.com/jrdi)). -* Fix build on master [#42752](https://github.com/ClickHouse/ClickHouse/pull/42752) ([Igor Nikonov](https://github.com/devcrafter)). -* Update version_date.tsv and changelogs after v22.3.14.18-lts [#42759](https://github.com/ClickHouse/ClickHouse/pull/42759) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Fix anchor links [#42760](https://github.com/ClickHouse/ClickHouse/pull/42760) ([Sergei Trifonov](https://github.com/serxa)). -* Update version_date.tsv and changelogs after v22.3.14.23-lts [#42764](https://github.com/ClickHouse/ClickHouse/pull/42764) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Update README.md [#42783](https://github.com/ClickHouse/ClickHouse/pull/42783) ([Yuko Takagi](https://github.com/yukotakagi)). -* Slightly better code with projections [#42794](https://github.com/ClickHouse/ClickHouse/pull/42794) ([Anton Popov](https://github.com/CurtizJ)). -* Fix some races in MergeTree [#42805](https://github.com/ClickHouse/ClickHouse/pull/42805) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Fix typo in comments [#42809](https://github.com/ClickHouse/ClickHouse/pull/42809) ([Gabriel](https://github.com/Gabriel39)). -* Fix compilation of LLVM with cmake cache [#42816](https://github.com/ClickHouse/ClickHouse/pull/42816) ([Azat Khuzhin](https://github.com/azat)). -* Fix link in docs [#42821](https://github.com/ClickHouse/ClickHouse/pull/42821) ([Sergei Trifonov](https://github.com/serxa)). -* Link to proper place in docs [#42822](https://github.com/ClickHouse/ClickHouse/pull/42822) ([Sergei Trifonov](https://github.com/serxa)). -* Fix argument type check in AggregateFunctionAnalysisOfVariance [#42823](https://github.com/ClickHouse/ClickHouse/pull/42823) ([Vladimir C](https://github.com/vdimir)). -* Tests/lambda analyzer [#42824](https://github.com/ClickHouse/ClickHouse/pull/42824) ([Denny Crane](https://github.com/den-crane)). -* Fix Missing Quotes - Sonar Nightly [#42831](https://github.com/ClickHouse/ClickHouse/pull/42831) ([Julio Jimenez](https://github.com/juliojimenez)). -* Add exclusions from the Snyk scan [#42834](https://github.com/ClickHouse/ClickHouse/pull/42834) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix Missing Env Vars - Sonar Nightly [#42843](https://github.com/ClickHouse/ClickHouse/pull/42843) ([Julio Jimenez](https://github.com/juliojimenez)). -* Fix typo [#42855](https://github.com/ClickHouse/ClickHouse/pull/42855) ([GoGoWen](https://github.com/GoGoWen)). -* Add timezone to 02458_datediff_date32 [#42857](https://github.com/ClickHouse/ClickHouse/pull/42857) ([Vladimir C](https://github.com/vdimir)). -* Adjust cancel and rerun workflow names to the actual [#42862](https://github.com/ClickHouse/ClickHouse/pull/42862) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Analyzer subquery in JOIN TREE with aggregation [#42865](https://github.com/ClickHouse/ClickHouse/pull/42865) ([Maksim Kita](https://github.com/kitaisreal)). -* Fix getauxval for sanitizer builds [#42866](https://github.com/ClickHouse/ClickHouse/pull/42866) ([Amos Bird](https://github.com/amosbird)). -* Update version_date.tsv and changelogs after v22.10.2.11-stable [#42871](https://github.com/ClickHouse/ClickHouse/pull/42871) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Validate Query Tree in debug [#42879](https://github.com/ClickHouse/ClickHouse/pull/42879) ([Dmitry Novik](https://github.com/novikd)). -* changed type name for s3 plain storage [#42890](https://github.com/ClickHouse/ClickHouse/pull/42890) ([Aleksandr](https://github.com/AVMusorin)). -* Cleanup implementation of regexpReplace(All|One) [#42907](https://github.com/ClickHouse/ClickHouse/pull/42907) ([Robert Schulze](https://github.com/rschu1ze)). -* Do not show status for Bugfix validate check in non bugfix PRs [#42932](https://github.com/ClickHouse/ClickHouse/pull/42932) ([Vladimir C](https://github.com/vdimir)). -* fix(typo): Passible -> Possible [#42933](https://github.com/ClickHouse/ClickHouse/pull/42933) ([Yakko Majuri](https://github.com/yakkomajuri)). -* Pin the cryptography version to not break lambdas [#42934](https://github.com/ClickHouse/ClickHouse/pull/42934) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Fix: bad cast from type DB::ColumnLowCardinality to DB::ColumnString [#42937](https://github.com/ClickHouse/ClickHouse/pull/42937) ([Igor Nikonov](https://github.com/devcrafter)). -* Attach thread pool for loading parts to the query [#42947](https://github.com/ClickHouse/ClickHouse/pull/42947) ([Azat Khuzhin](https://github.com/azat)). -* Fix macOS M1 builds due to sprintf deprecation [#42962](https://github.com/ClickHouse/ClickHouse/pull/42962) ([Jordi Villar](https://github.com/jrdi)). -* Less use of CH-specific bit_cast() [#42968](https://github.com/ClickHouse/ClickHouse/pull/42968) ([Robert Schulze](https://github.com/rschu1ze)). -* Remove some utils [#42972](https://github.com/ClickHouse/ClickHouse/pull/42972) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix a bug in CAST function parser [#42980](https://github.com/ClickHouse/ClickHouse/pull/42980) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fix old bug to remove `refs/head` from ref name [#42981](https://github.com/ClickHouse/ClickHouse/pull/42981) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Add debug information to nightly builds [#42997](https://github.com/ClickHouse/ClickHouse/pull/42997) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Add `on: workflow_call` to debug CI [#43000](https://github.com/ClickHouse/ClickHouse/pull/43000) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Simple fixes for restart replica description [#43004](https://github.com/ClickHouse/ClickHouse/pull/43004) ([Igor Nikonov](https://github.com/devcrafter)). -* Cleanup match code [#43006](https://github.com/ClickHouse/ClickHouse/pull/43006) ([Robert Schulze](https://github.com/rschu1ze)). -* Fix TSan errors (correctly ignore _exit interception) [#43009](https://github.com/ClickHouse/ClickHouse/pull/43009) ([Azat Khuzhin](https://github.com/azat)). -* fix bandwidth throttlers initialization order [#43015](https://github.com/ClickHouse/ClickHouse/pull/43015) ([Sergei Trifonov](https://github.com/serxa)). -* Add test for issue [#42520](https://github.com/ClickHouse/ClickHouse/issues/42520) [#43027](https://github.com/ClickHouse/ClickHouse/pull/43027) ([Robert Schulze](https://github.com/rschu1ze)). -* Fix msan warning [#43065](https://github.com/ClickHouse/ClickHouse/pull/43065) ([Raúl Marín](https://github.com/Algunenano)). -* Update SECURITY.md on new stable tags [#43365](https://github.com/ClickHouse/ClickHouse/pull/43365) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Use all parameters with prefixes from ssm [#43467](https://github.com/ClickHouse/ClickHouse/pull/43467) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Temporarily disable `test_hive_query` [#43542](https://github.com/ClickHouse/ClickHouse/pull/43542) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Do not checkout submodules recursively [#43637](https://github.com/ClickHouse/ClickHouse/pull/43637) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Use docker images cache from merged PRs in master and release branches [#43664](https://github.com/ClickHouse/ClickHouse/pull/43664) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Yet another fix for AggregateFunctionMinMaxAny [#43778](https://github.com/ClickHouse/ClickHouse/pull/43778) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Fix tags workflow [#43942](https://github.com/ClickHouse/ClickHouse/pull/43942) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Assign release PRs [#44055](https://github.com/ClickHouse/ClickHouse/pull/44055) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Fix another bug in AggregateFunctionMinMaxAny [#44091](https://github.com/ClickHouse/ClickHouse/pull/44091) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Bump libdivide (to gain some new optimizations) [#44132](https://github.com/ClickHouse/ClickHouse/pull/44132) ([Azat Khuzhin](https://github.com/azat)). -* Add check for submodules sanity [#44386](https://github.com/ClickHouse/ClickHouse/pull/44386) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Implement a custom central checkout action [#44399](https://github.com/ClickHouse/ClickHouse/pull/44399) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - From b156209ed6b84b874a31621699e6a61159da2ade Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 11 Jan 2023 14:17:44 +0300 Subject: [PATCH 36/49] Update test.py --- tests/integration/test_storage_rabbitmq/test.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 63b8d1215aa..43c964d9d93 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1,3 +1,10 @@ +import pytest + +# FIXME This test is too flaky +# https://github.com/ClickHouse/ClickHouse/issues/45160 + +pytestmark = pytest.mark.skip + import json import os.path as p import random @@ -9,7 +16,6 @@ from random import randrange import math import pika -import pytest from google.protobuf.internal.encoder import _VarintBytes from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster, check_rabbitmq_is_available From 21573028ea02b3c0a9cfe6dc2420a0360c2d3731 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Wed, 11 Jan 2023 07:38:41 -0500 Subject: [PATCH 37/49] Update docs/en/engines/table-engines/integrations/deltalake.md --- docs/en/engines/table-engines/integrations/deltalake.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/deltalake.md b/docs/en/engines/table-engines/integrations/deltalake.md index eb4d8e934a7..251d2fef52e 100644 --- a/docs/en/engines/table-engines/integrations/deltalake.md +++ b/docs/en/engines/table-engines/integrations/deltalake.md @@ -18,7 +18,7 @@ CREATE TABLE deltalake **Engine parameters** -- `path` — Bucket url with path to the existing Delta Lake table. +- `url` — Bucket url with path to the existing Delta Lake table. - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). **Example** From 73ef2657dd906974e5a4178e55099faec553ee77 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Wed, 11 Jan 2023 07:40:10 -0500 Subject: [PATCH 38/49] Update docs/en/engines/table-engines/integrations/hudi.md --- docs/en/engines/table-engines/integrations/hudi.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/hudi.md b/docs/en/engines/table-engines/integrations/hudi.md index 6da1634ba5a..75b1969101b 100644 --- a/docs/en/engines/table-engines/integrations/hudi.md +++ b/docs/en/engines/table-engines/integrations/hudi.md @@ -18,7 +18,7 @@ CREATE TABLE hudi_table **Engine parameters** -- `path` — Bucket url with the path to an existing Hudi table. +- `url` — Bucket url with the path to an existing Hudi table. - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). **Example** From 367d4fc4bf0cfc992f48ab5297f38da5017efb73 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Wed, 11 Jan 2023 07:40:52 -0500 Subject: [PATCH 39/49] Update docs/en/sql-reference/table-functions/hudi.md --- docs/en/sql-reference/table-functions/hudi.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/hudi.md b/docs/en/sql-reference/table-functions/hudi.md index c1ccd0cda2f..b8d0724a7b9 100644 --- a/docs/en/sql-reference/table-functions/hudi.md +++ b/docs/en/sql-reference/table-functions/hudi.md @@ -10,7 +10,7 @@ Provides a read-only table-like interface to Apache [Hudi](https://hudi.apache.o ## Syntax ``` sql -hudi(path [,aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,compression]) +hudi(url [,aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,compression]) ``` ## Arguments From d4c4f84161014f9434fc4535cd545b950ba2bd5d Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Wed, 11 Jan 2023 07:41:36 -0500 Subject: [PATCH 40/49] Update docs/en/sql-reference/table-functions/hudi.md --- docs/en/sql-reference/table-functions/hudi.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/hudi.md b/docs/en/sql-reference/table-functions/hudi.md index b8d0724a7b9..5a97b2401b4 100644 --- a/docs/en/sql-reference/table-functions/hudi.md +++ b/docs/en/sql-reference/table-functions/hudi.md @@ -15,7 +15,7 @@ hudi(url [,aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,co ## Arguments -- `path` — Bucket url with the path to an existing Hudi table in S3. +- `url` — Bucket url with the path to an existing Hudi table in S3. - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. These parameters are optional. If credentials are not specified, they are used from the ClickHouse configuration. For more information see [Using S3 for Data Storage](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3). - `format` — The [format](/docs/en/interfaces/formats.md/#formats) of the file. - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. From 02261b0e2a83bab18dcff08f48246ef122ab1449 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Wed, 11 Jan 2023 07:42:15 -0500 Subject: [PATCH 41/49] Update docs/en/engines/table-engines/integrations/hudi.md --- docs/en/engines/table-engines/integrations/hudi.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/hudi.md b/docs/en/engines/table-engines/integrations/hudi.md index 75b1969101b..4e335e6c075 100644 --- a/docs/en/engines/table-engines/integrations/hudi.md +++ b/docs/en/engines/table-engines/integrations/hudi.md @@ -13,7 +13,7 @@ Note that the Hudi table must already exist in S3, this command does not take DD ``` sql CREATE TABLE hudi_table - ENGINE = Hudi(path, [aws_access_key_id, aws_secret_access_key,]) + ENGINE = Hudi(url, [aws_access_key_id, aws_secret_access_key,]) ``` **Engine parameters** From 1ddc9c3bb60195db72dac5fb966d7870e9b0dbc8 Mon Sep 17 00:00:00 2001 From: rfraposa Date: Wed, 11 Jan 2023 05:44:51 -0700 Subject: [PATCH 42/49] Update merge-tree-settings.md --- .../settings/merge-tree-settings.md | 103 ++++++++++++++++++ 1 file changed, 103 insertions(+) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index ec492605930..ed4ee37fc37 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -643,3 +643,106 @@ Default value: `0` (limit never applied). ``` xml 10 ``` + +## ratio_of_defaults_for_sparse_serialization {#ratio_of_defaults_for_sparse_serialization} + +Minimal ratio of the number of _default_ values to the number of _all_ values in a column. Setting this value causes the column to be stored using sparse serializations. + +If a column is sparse (contains mostly zeros), ClickHouse can encode it in a sparse format and automatically optimize calculations - the data does not require full decompression during queries. To enable this sparse serialization, define the `ratio_of_defaults_for_sparse_serialization` setting to be less than 1.0. If the value is greater than or equal to 1.0 (the default), then the columns will be always written using the normal full serialization. + +Possible values: + +- Float between 0 and 1 to enable sparse serialization +- 1.0 (or greater) if you do not want to use sparse serialization + +Default value: `1.0` (sparse serialization is disabled) + +**Example** + +Notice the `s` column in the following table is an empty string for 95% of the rows. In `my_regular_table` we do not use sparse serialization, and in `my_sparse_table` we set `ratio_of_defaults_for_sparse_serialization` to 0.95: + +```sql +CREATE TABLE my_regular_table +( + `id` UInt64, + `s` String +) +ENGINE = MergeTree +ORDER BY id; + +INSERT INTO my_regular_table +SELECT + number AS id, + number % 20 = 0 ? toString(number): '' AS s +FROM + numbers(10000000); + + +CREATE TABLE my_sparse_table +( + `id` UInt64, + `s` String +) +ENGINE = MergeTree +ORDER BY id +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.95; + +INSERT INTO my_sparse_table +SELECT + number, + number % 20 = 0 ? toString(number): '' +FROM + numbers(10000000); +``` + +Notice the `s` column in `my_sparse_table` uses less storage space on disk: + +```sql +SELECT table, name, data_compressed_bytes, data_uncompressed_bytes FROM system.columns +WHERE table LIKE 'my_%_table'; +``` + +```response +┌─table────────────┬─name─┬─data_compressed_bytes─┬─data_uncompressed_bytes─┐ +│ my_regular_table │ id │ 37790741 │ 75488328 │ +│ my_regular_table │ s │ 2451377 │ 12683106 │ +│ my_sparse_table │ id │ 37790741 │ 75488328 │ +│ my_sparse_table │ s │ 2283454 │ 9855751 │ +└──────────────────┴──────┴───────────────────────┴─────────────────────────┘ +``` + +You can verify if a column is using the sparse encoding by viewing the `serialization_kind` column of the `system.parts_columns` table: + +```sql +SELECT column, serialization_kind FROM system.parts_columns +WHERE table LIKE 'my_sparse_table'; +``` + +You can see which parts of `s` were stored using the sparse serialization: + +```response +┌─column─┬─serialization_kind─┐ +│ id │ Default │ +│ s │ Default │ +│ id │ Default │ +│ s │ Default │ +│ id │ Default │ +│ s │ Sparse │ +│ id │ Default │ +│ s │ Sparse │ +│ id │ Default │ +│ s │ Sparse │ +│ id │ Default │ +│ s │ Sparse │ +│ id │ Default │ +│ s │ Sparse │ +│ id │ Default │ +│ s │ Sparse │ +│ id │ Default │ +│ s │ Sparse │ +│ id │ Default │ +│ s │ Sparse │ +│ id │ Default │ +│ s │ Sparse │ +└────────┴────────────────────┘ +``` \ No newline at end of file From 6e9669cfaebf4a0a8c8c5a15f9ead146833627e1 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Wed, 11 Jan 2023 07:53:37 -0500 Subject: [PATCH 43/49] Apply suggestions from code review --- docs/en/engines/table-engines/integrations/deltalake.md | 2 +- docs/en/sql-reference/table-functions/deltalake.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/deltalake.md b/docs/en/engines/table-engines/integrations/deltalake.md index 251d2fef52e..83526ac944d 100644 --- a/docs/en/engines/table-engines/integrations/deltalake.md +++ b/docs/en/engines/table-engines/integrations/deltalake.md @@ -13,7 +13,7 @@ Note that the Delta Lake table must already exist in S3, this command does not t ``` sql CREATE TABLE deltalake - ENGINE = DeltaLake(path, [aws_access_key_id, aws_secret_access_key,]) + ENGINE = DeltaLake(url, [aws_access_key_id, aws_secret_access_key,]) ``` **Engine parameters** diff --git a/docs/en/sql-reference/table-functions/deltalake.md b/docs/en/sql-reference/table-functions/deltalake.md index 10e7c20e17a..f1cc4659a2a 100644 --- a/docs/en/sql-reference/table-functions/deltalake.md +++ b/docs/en/sql-reference/table-functions/deltalake.md @@ -10,12 +10,12 @@ Provides a read-only table-like interface to [Delta Lake](https://github.com/del ## Syntax ``` sql -deltaLake(path [,aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,compression]) +deltaLake(url [,aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,compression]) ``` ## Arguments -- `path` — Bucket url with path to existing Delta Lake table in S3. +- `url` — Bucket url with path to existing Delta Lake table in S3. - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. These parameters are optional. If credentials are not specified, they are used from the ClickHouse configuration. For more information see [Using S3 for Data Storage](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3). - `format` — The [format](/docs/en/interfaces/formats.md/#formats) of the file. - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. From a79f6d19fa0d8cad4a38255f7ae547e1bfee02bf Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 11 Jan 2023 13:04:05 +0000 Subject: [PATCH 44/49] add docs for `system.moves` table --- docs/en/operations/system-tables/moves.md | 42 +++++++++++++++++++++++ 1 file changed, 42 insertions(+) create mode 100644 docs/en/operations/system-tables/moves.md diff --git a/docs/en/operations/system-tables/moves.md b/docs/en/operations/system-tables/moves.md new file mode 100644 index 00000000000..e790946a15f --- /dev/null +++ b/docs/en/operations/system-tables/moves.md @@ -0,0 +1,42 @@ +--- +slug: /en/operations/system-tables/moves +--- +# moves + +The table contains information about in-progress [data part moves](/docs/en/sql-reference/statements/alter/partition#move-partitionpart) of [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) tables. Each data part movement is represented by a single row. + +Columns: + +- `database` ([String](/docs/en/sql-reference/data-types/string.md)) — Name of the database. + +- `table` ([String](/docs/en/sql-reference/data-types/string.md)) — Name of the table containing moving data part. + +- `elapsed` ([Float64](../../sql-reference/data-types/float.md)) — Time elapsed (in seconds) since data part movement started. + +- `target_disk_name` ([String](disks.md)) — Name of [disk](/docs/en/operations/system-tables/disks/) to which the data part is moving. + +- `target_disk_path` ([String](disks.md)) — Path to the mount point of the [disk](/docs/en/operations/system-tables/disks/) in the file system. + +- `part_name` ([String](/docs/en/sql-reference/data-types/string.md)) — Name of the data part being moved. + +- `part_size` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Data part size. + +- `thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Identifier of a thread performing the movement. + +**Example** + +```sql +SELECT * FROM system.moves +``` + +```text +┌─database─┬─table─┬─────elapsed─┬─target_disk_name─┬─target_disk_path─┬─part_name─┬─part_size─┬─thread_id─┐ +│ default │ test2 │ 1.668056039 │ s3 │ ./disks/s3/ │ all_3_3_0 │ 136 │ 296146 │ +└──────────┴───────┴─────────────┴──────────────────┴──────────────────┴───────────┴───────────┴───────────┘ +``` + +**See Also** + +- [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) table engine +- [Using Multiple Block Devices for Data Storage](/docs/en/engines/table-engines/mergetree-family/mergetree#table_engine-mergetree-multiple-volumes) +- [ALTER TABLE ... MOVE PART](/docs/en/sql-reference/statements/alter/partition#move-partitionpart) command From 4767147745787180647243cafd76b939ed09dc25 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 11 Jan 2023 08:23:44 -0500 Subject: [PATCH 45/49] format query response --- docs/en/operations/system-tables/moves.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/moves.md b/docs/en/operations/system-tables/moves.md index e790946a15f..54f07540507 100644 --- a/docs/en/operations/system-tables/moves.md +++ b/docs/en/operations/system-tables/moves.md @@ -29,7 +29,7 @@ Columns: SELECT * FROM system.moves ``` -```text +```response ┌─database─┬─table─┬─────elapsed─┬─target_disk_name─┬─target_disk_path─┬─part_name─┬─part_size─┬─thread_id─┐ │ default │ test2 │ 1.668056039 │ s3 │ ./disks/s3/ │ all_3_3_0 │ 136 │ 296146 │ └──────────┴───────┴─────────────┴──────────────────┴──────────────────┴───────────┴───────────┴───────────┘ From 8d099a44172837e8c54c149d3d811322c757132c Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 11 Jan 2023 13:43:51 +0000 Subject: [PATCH 46/49] make more SQL queries copyable from docs in one click --- docs/en/interfaces/formats.md | 19 ++--- docs/en/operations/system-tables/disks.md | 2 +- .../system-tables/merge_tree_settings.md | 2 +- docs/en/operations/system-tables/numbers.md | 2 +- .../en/operations/system-tables/numbers_mt.md | 2 +- docs/en/operations/system-tables/one.md | 2 +- docs/en/operations/system-tables/processes.md | 2 +- .../sql-reference/table-functions/format.md | 5 +- .../sql-reference/table-functions/format.md | 5 +- .../mergetree-family/summingmergetree.md | 4 +- docs/zh/operations/system-tables/disks.md | 2 +- .../system-tables/merge_tree_settings.md | 2 +- docs/zh/operations/system-tables/numbers.md | 2 +- docs/zh/operations/system-tables/one.md | 2 +- docs/zh/sql-reference/data-types/array.md | 69 +++++++-------- docs/zh/sql-reference/data-types/enum.md | 85 ++++++++++++------- .../data-types/special-data-types/nothing.md | 8 +- docs/zh/sql-reference/data-types/tuple.md | 36 ++++---- .../functions/functions-for-nulls.md | 85 ++++++++++--------- .../functions/other-functions.md | 75 ++++++++-------- .../sql-reference/functions/uuid-functions.md | 22 +++-- docs/zh/sql-reference/operators/index.md | 24 ++---- .../sql-reference/table-functions/format.md | 5 +- 23 files changed, 235 insertions(+), 227 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index c78b34e0b0d..75ef0ac3cc0 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1265,7 +1265,7 @@ For input it uses the following correspondence between BSON types and ClickHouse | `\x10` int32 | [Int32/UInt32](/docs/en/sql-reference/data-types/int-uint.md)/[Decimal32](/docs/en/sql-reference/data-types/decimal.md) | | `\x12` int64 | [Int64/UInt64](/docs/en/sql-reference/data-types/int-uint.md)/[Decimal64](/docs/en/sql-reference/data-types/decimal.md)/[DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | -Other BSON types are not supported. Also, it performs conversion between different integer types (for example, you can insert BSON int32 value into ClickHouse UInt8). +Other BSON types are not supported. Also, it performs conversion between different integer types (for example, you can insert BSON int32 value into ClickHouse UInt8). Big integers and decimals (Int128/UInt128/Int256/UInt256/Decimal128/Decimal256) can be parsed from BSON Binary value with `\x00` binary subtype. In this case this format will validate that the size of binary data equals the size of expected value. Note: this format don't work properly on Big-Endian platforms. @@ -2319,25 +2319,22 @@ INSERT INTO `test2` VALUES (1),(2),(3); Queries: ```sql -:) desc file(dump.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2' - -DESCRIBE TABLE file(dump.sql, MySQLDump) -SETTINGS input_format_mysql_dump_table_name = 'test2' - -Query id: 25e66c89-e10a-42a8-9b42-1ee8bbbde5ef +DESCRIBE TABLE file(dump.sql, MySQLDump) SETTINGS input_format_mysql_dump_table_name = 'test2' +``` +```text ┌─name─┬─type────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐ │ x │ Nullable(Int32) │ │ │ │ │ │ └──────┴─────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ +``` -:) select * from file(dump.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2' - +```sql SELECT * FROM file(dump.sql, MySQLDump) SETTINGS input_format_mysql_dump_table_name = 'test2' +``` -Query id: 17d59664-ebce-4053-bb79-d46a516fb590 - +```text ┌─x─┐ │ 1 │ │ 2 │ diff --git a/docs/en/operations/system-tables/disks.md b/docs/en/operations/system-tables/disks.md index 4096a8c765c..d492e42c2ec 100644 --- a/docs/en/operations/system-tables/disks.md +++ b/docs/en/operations/system-tables/disks.md @@ -17,7 +17,7 @@ Columns: **Example** ```sql -:) SELECT * FROM system.disks; +SELECT * FROM system.disks; ``` ```text diff --git a/docs/en/operations/system-tables/merge_tree_settings.md b/docs/en/operations/system-tables/merge_tree_settings.md index 9f8006d77a7..a05d4abccda 100644 --- a/docs/en/operations/system-tables/merge_tree_settings.md +++ b/docs/en/operations/system-tables/merge_tree_settings.md @@ -15,7 +15,7 @@ Columns: **Example** ```sql -:) SELECT * FROM system.merge_tree_settings LIMIT 4 FORMAT Vertical; +SELECT * FROM system.merge_tree_settings LIMIT 4 FORMAT Vertical; ``` ```text diff --git a/docs/en/operations/system-tables/numbers.md b/docs/en/operations/system-tables/numbers.md index f2204dbf0ba..0dc001ebb6f 100644 --- a/docs/en/operations/system-tables/numbers.md +++ b/docs/en/operations/system-tables/numbers.md @@ -12,7 +12,7 @@ Reads from this table are not parallelized. **Example** ```sql -:) SELECT * FROM system.numbers LIMIT 10; +SELECT * FROM system.numbers LIMIT 10; ``` ```text diff --git a/docs/en/operations/system-tables/numbers_mt.md b/docs/en/operations/system-tables/numbers_mt.md index deb7be7dc68..cc461b29ad0 100644 --- a/docs/en/operations/system-tables/numbers_mt.md +++ b/docs/en/operations/system-tables/numbers_mt.md @@ -10,7 +10,7 @@ Used for tests. **Example** ```sql -:) SELECT * FROM system.numbers_mt LIMIT 10; +SELECT * FROM system.numbers_mt LIMIT 10; ``` ```text diff --git a/docs/en/operations/system-tables/one.md b/docs/en/operations/system-tables/one.md index d71c82f5e94..ee2907a6d6d 100644 --- a/docs/en/operations/system-tables/one.md +++ b/docs/en/operations/system-tables/one.md @@ -12,7 +12,7 @@ This is similar to the `DUAL` table found in other DBMSs. **Example** ```sql -:) SELECT * FROM system.one LIMIT 10; +SELECT * FROM system.one LIMIT 10; ``` ```text diff --git a/docs/en/operations/system-tables/processes.md b/docs/en/operations/system-tables/processes.md index 2cf15a9bb2b..95c46f551ef 100644 --- a/docs/en/operations/system-tables/processes.md +++ b/docs/en/operations/system-tables/processes.md @@ -20,7 +20,7 @@ Columns: - `is_all_data_sent` (Int8) – Was all data sent to the client (in other words query had been finished on the server). ```sql -:) SELECT * FROM system.processes LIMIT 10 FORMAT Vertical; +SELECT * FROM system.processes LIMIT 10 FORMAT Vertical; ``` ```text diff --git a/docs/en/sql-reference/table-functions/format.md b/docs/en/sql-reference/table-functions/format.md index 78b67a47d4e..4a0ee58d758 100644 --- a/docs/en/sql-reference/table-functions/format.md +++ b/docs/en/sql-reference/table-functions/format.md @@ -27,7 +27,7 @@ A table with data parsed from `data` argument according specified format and ext **Query:** ``` sql -:) select * from format(JSONEachRow, +SELECT * FROM format(JSONEachRow, $$ {"a": "Hello", "b": 111} {"a": "World", "b": 123} @@ -49,8 +49,7 @@ $$) **Query:** ```sql - -:) desc format(JSONEachRow, +DESC format(JSONEachRow, $$ {"a": "Hello", "b": 111} {"a": "World", "b": 123} diff --git a/docs/ru/sql-reference/table-functions/format.md b/docs/ru/sql-reference/table-functions/format.md index 5dc463e5b27..a91b4ca2b1e 100644 --- a/docs/ru/sql-reference/table-functions/format.md +++ b/docs/ru/sql-reference/table-functions/format.md @@ -27,7 +27,7 @@ A table with data parsed from `data` argument according specified format and ext **Query:** ``` sql -:) select * from format(JSONEachRow, +SELECT * FROM format(JSONEachRow, $$ {"a": "Hello", "b": 111} {"a": "World", "b": 123} @@ -49,8 +49,7 @@ $$) **Query:** ```sql - -:) desc format(JSONEachRow, +DESC format(JSONEachRow, $$ {"a": "Hello", "b": 111} {"a": "World", "b": 123} diff --git a/docs/zh/engines/table-engines/mergetree-family/summingmergetree.md b/docs/zh/engines/table-engines/mergetree-family/summingmergetree.md index 620a56006db..f59d327b4ae 100644 --- a/docs/zh/engines/table-engines/mergetree-family/summingmergetree.md +++ b/docs/zh/engines/table-engines/mergetree-family/summingmergetree.md @@ -69,7 +69,9 @@ ORDER BY key 向其中插入数据: - :) INSERT INTO summtt Values(1,1),(1,2),(2,1) +``` sql +INSERT INTO summtt Values(1,1),(1,2),(2,1) +``` ClickHouse可能不会完整的汇总所有行([见下文](#data-processing)),因此我们在查询中使用了聚合函数 `sum` 和 `GROUP BY` 子句。 diff --git a/docs/zh/operations/system-tables/disks.md b/docs/zh/operations/system-tables/disks.md index 36f7e8de4f1..0e774632074 100644 --- a/docs/zh/operations/system-tables/disks.md +++ b/docs/zh/operations/system-tables/disks.md @@ -16,7 +16,7 @@ slug: /zh/operations/system-tables/disks **示例** ```sql -:) SELECT * FROM system.disks; +SELECT * FROM system.disks; ``` ```text diff --git a/docs/zh/operations/system-tables/merge_tree_settings.md b/docs/zh/operations/system-tables/merge_tree_settings.md index c3c424c01fe..c2bdcd14d24 100644 --- a/docs/zh/operations/system-tables/merge_tree_settings.md +++ b/docs/zh/operations/system-tables/merge_tree_settings.md @@ -16,7 +16,7 @@ slug: /zh/operations/system-tables/merge_tree_settings **示例** ```sql -:) SELECT * FROM system.merge_tree_settings LIMIT 4 FORMAT Vertical; +SELECT * FROM system.merge_tree_settings LIMIT 4 FORMAT Vertical; ``` ```text diff --git a/docs/zh/operations/system-tables/numbers.md b/docs/zh/operations/system-tables/numbers.md index f3db66f365b..8cb92351ae7 100644 --- a/docs/zh/operations/system-tables/numbers.md +++ b/docs/zh/operations/system-tables/numbers.md @@ -12,7 +12,7 @@ slug: /zh/operations/system-tables/numbers **示例** ```sql -:) SELECT * FROM system.numbers LIMIT 10; +SELECT * FROM system.numbers LIMIT 10; ``` ```text diff --git a/docs/zh/operations/system-tables/one.md b/docs/zh/operations/system-tables/one.md index 6929b1b4245..2e44a9bd89a 100644 --- a/docs/zh/operations/system-tables/one.md +++ b/docs/zh/operations/system-tables/one.md @@ -12,7 +12,7 @@ slug: /zh/operations/system-tables/one **示例** ```sql -:) SELECT * FROM system.one LIMIT 10; +SELECT * FROM system.one LIMIT 10; ``` ```text diff --git a/docs/zh/sql-reference/data-types/array.md b/docs/zh/sql-reference/data-types/array.md index e2f18a42de8..0c38eb86004 100644 --- a/docs/zh/sql-reference/data-types/array.md +++ b/docs/zh/sql-reference/data-types/array.md @@ -19,29 +19,25 @@ slug: /zh/sql-reference/data-types/array 创建数组示例: - :) SELECT array(1, 2) AS x, toTypeName(x) +```sql +SELECT array(1, 2) AS x, toTypeName(x) +``` - SELECT - [1, 2] AS x, - toTypeName(x) +```text +┌─x─────┬─toTypeName(array(1, 2))─┐ +│ [1,2] │ Array(UInt8) │ +└───────┴─────────────────────────┘ +``` - ┌─x─────┬─toTypeName(array(1, 2))─┐ - │ [1,2] │ Array(UInt8) │ - └───────┴─────────────────────────┘ +``` sql +SELECT [1, 2] AS x, toTypeName(x) +``` - 1 rows in set. Elapsed: 0.002 sec. - - :) SELECT [1, 2] AS x, toTypeName(x) - - SELECT - [1, 2] AS x, - toTypeName(x) - - ┌─x─────┬─toTypeName([1, 2])─┐ - │ [1,2] │ Array(UInt8) │ - └───────┴────────────────────┘ - - 1 rows in set. Elapsed: 0.002 sec. +``` text +┌─x─────┬─toTypeName([1, 2])─┐ +│ [1,2] │ Array(UInt8) │ +└───────┴────────────────────┘ +``` ## 使用数据类型 {#shi-yong-shu-ju-lei-xing} @@ -50,26 +46,23 @@ ClickHouse会自动检测数组元素,并根据元素计算出存储这些元素 如果 ClickHouse 无法确定数据类型,它将产生异常。当尝试同时创建一个包含字符串和数字的数组时会发生这种情况 (`SELECT array(1, 'a')`)。 自动数据类型检测示例: +```sql +SELECT array(1, 2, NULL) AS x, toTypeName(x) +``` - :) SELECT array(1, 2, NULL) AS x, toTypeName(x) - - SELECT - [1, 2, NULL] AS x, - toTypeName(x) - - ┌─x──────────┬─toTypeName(array(1, 2, NULL))─┐ - │ [1,2,NULL] │ Array(Nullable(UInt8)) │ - └────────────┴───────────────────────────────┘ - - 1 rows in set. Elapsed: 0.002 sec. +```text +┌─x──────────┬─toTypeName(array(1, 2, NULL))─┐ +│ [1,2,NULL] │ Array(Nullable(UInt8)) │ +└────────────┴───────────────────────────────┘ +``` 如果您尝试创建不兼容的数据类型数组,ClickHouse 将引发异常: - :) SELECT array(1, 'a') +```sql +SELECT array(1, 'a') +``` - SELECT [1, 'a'] - - Received exception from server (version 1.1.54388): - Code: 386. DB::Exception: Received from localhost:9000, 127.0.0.1. DB::Exception: There is no supertype for types UInt8, String because some of them are String/FixedString and some of them are not. - - 0 rows in set. Elapsed: 0.246 sec. +```text +Received exception from server (version 1.1.54388): +Code: 386. DB::Exception: Received from localhost:9000, 127.0.0.1. DB::Exception: There is no supertype for types UInt8, String because some of them are String/FixedString and some of them are not. +``` diff --git a/docs/zh/sql-reference/data-types/enum.md b/docs/zh/sql-reference/data-types/enum.md index 0cf8a02d76b..9832df3da02 100644 --- a/docs/zh/sql-reference/data-types/enum.md +++ b/docs/zh/sql-reference/data-types/enum.md @@ -20,49 +20,64 @@ slug: /zh/sql-reference/data-types/enum 这个 `x` 列只能存储类型定义中列出的值:`'hello'`或`'world'`。如果您尝试保存任何其他值,ClickHouse 抛出异常。 - :) INSERT INTO t_enum VALUES ('hello'), ('world'), ('hello') +```sql +INSERT INTO t_enum VALUES ('hello'), ('world'), ('hello') +``` - INSERT INTO t_enum VALUES +```text +Ok. - Ok. +3 rows in set. Elapsed: 0.002 sec. +``` - 3 rows in set. Elapsed: 0.002 sec. +```sql +INSERT INTO t_enum VALUES('a') +``` - :) insert into t_enum values('a') - - INSERT INTO t_enum VALUES - - - Exception on client: - Code: 49. DB::Exception: Unknown element 'a' for type Enum8('hello' = 1, 'world' = 2) +```text +Exception on client: +Code: 49. DB::Exception: Unknown element 'a' for type Enum8('hello' = 1, 'world' = 2) +``` 当您从表中查询数据时,ClickHouse 从 `Enum` 中输出字符串值。 - SELECT * FROM t_enum +```sql +SELECT * FROM t_enum +``` - ┌─x─────┐ - │ hello │ - │ world │ - │ hello │ - └───────┘ +```text +┌─x─────┐ +│ hello │ +│ world │ +│ hello │ +└───────┘ +``` 如果需要看到对应行的数值,则必须将 `Enum` 值转换为整数类型。 - SELECT CAST(x, 'Int8') FROM t_enum +```sql +SELECT CAST(x, 'Int8') FROM t_enum +``` - ┌─CAST(x, 'Int8')─┐ - │ 1 │ - │ 2 │ - │ 1 │ - └─────────────────┘ +```text +┌─CAST(x, 'Int8')─┐ +│ 1 │ +│ 2 │ +│ 1 │ +└─────────────────┘ +``` 在查询中创建枚举值,您还需要使用 `CAST`。 - SELECT toTypeName(CAST('a', 'Enum8(\'a\' = 1, \'b\' = 2)')) +```sql +SELECT toTypeName(CAST('a', 'Enum8(\'a\' = 1, \'b\' = 2)')) +``` - ┌─toTypeName(CAST('a', 'Enum8(\'a\' = 1, \'b\' = 2)'))─┐ - │ Enum8('a' = 1, 'b' = 2) │ - └──────────────────────────────────────────────────────┘ +```text +┌─toTypeName(CAST('a', 'Enum8(\'a\' = 1, \'b\' = 2)'))─┐ +│ Enum8('a' = 1, 'b' = 2) │ +└──────────────────────────────────────────────────────┘ +``` ## 规则及用法 {#gui-ze-ji-yong-fa} @@ -72,15 +87,19 @@ slug: /zh/sql-reference/data-types/enum `Enum` 包含在 [可为空](nullable.md) 类型中。因此,如果您使用此查询创建一个表 - CREATE TABLE t_enum_nullable - ( - x Nullable( Enum8('hello' = 1, 'world' = 2) ) - ) - ENGINE = TinyLog +```sql +CREATE TABLE t_enum_nullable +( + x Nullable( Enum8('hello' = 1, 'world' = 2) ) +) +ENGINE = TinyLog +``` 不仅可以存储 `'hello'` 和 `'world'` ,还可以存储 `NULL`。 - INSERT INTO t_enum_nullable Values('hello'),('world'),(NULL) +```sql +INSERT INTO t_enum_nullable Values('hello'),('world'),(NULL) +``` 在内存中,`Enum` 列的存储方式与相应数值的 `Int8` 或 `Int16` 相同。 diff --git a/docs/zh/sql-reference/data-types/special-data-types/nothing.md b/docs/zh/sql-reference/data-types/special-data-types/nothing.md index 2b10934f566..e123622edf6 100644 --- a/docs/zh/sql-reference/data-types/special-data-types/nothing.md +++ b/docs/zh/sql-reference/data-types/special-data-types/nothing.md @@ -9,11 +9,11 @@ slug: /zh/sql-reference/data-types/special-data-types/nothing `Nothing` 类型也可以用来表示空数组: -``` bash -:) SELECT toTypeName(array()) - -SELECT toTypeName([]) +```sql +SELECT toTypeName(array()) +``` +```text ┌─toTypeName(array())─┐ │ Array(Nothing) │ └─────────────────────┘ diff --git a/docs/zh/sql-reference/data-types/tuple.md b/docs/zh/sql-reference/data-types/tuple.md index e991fa7145a..905a872da24 100644 --- a/docs/zh/sql-reference/data-types/tuple.md +++ b/docs/zh/sql-reference/data-types/tuple.md @@ -17,17 +17,15 @@ slug: /zh/sql-reference/data-types/tuple 创建元组的示例: - :) SELECT tuple(1,'a') AS x, toTypeName(x) +```sql +SELECT tuple(1,'a') AS x, toTypeName(x) +``` - SELECT - (1, 'a') AS x, - toTypeName(x) - - ┌─x───────┬─toTypeName(tuple(1, 'a'))─┐ - │ (1,'a') │ Tuple(UInt8, String) │ - └─────────┴───────────────────────────┘ - - 1 rows in set. Elapsed: 0.021 sec. +```text +┌─x───────┬─toTypeName(tuple(1, 'a'))─┐ +│ (1,'a') │ Tuple(UInt8, String) │ +└─────────┴───────────────────────────┘ +``` ## 元组中的数据类型 {#yuan-zu-zhong-de-shu-ju-lei-xing} @@ -35,14 +33,12 @@ slug: /zh/sql-reference/data-types/tuple 自动数据类型检测示例: - SELECT tuple(1, NULL) AS x, toTypeName(x) +```sql +SELECT tuple(1, NULL) AS x, toTypeName(x) +``` - SELECT - (1, NULL) AS x, - toTypeName(x) - - ┌─x────────┬─toTypeName(tuple(1, NULL))──────┐ - │ (1,NULL) │ Tuple(UInt8, Nullable(Nothing)) │ - └──────────┴─────────────────────────────────┘ - - 1 rows in set. Elapsed: 0.002 sec. +```text +┌─x────────┬─toTypeName(tuple(1, NULL))──────┐ +│ (1,NULL) │ Tuple(UInt8, Nullable(Nothing)) │ +└──────────┴─────────────────────────────────┘ +``` diff --git a/docs/zh/sql-reference/functions/functions-for-nulls.md b/docs/zh/sql-reference/functions/functions-for-nulls.md index 1ae53f5ddc1..9ecf39e56c5 100644 --- a/docs/zh/sql-reference/functions/functions-for-nulls.md +++ b/docs/zh/sql-reference/functions/functions-for-nulls.md @@ -22,24 +22,24 @@ slug: /zh/sql-reference/functions/functions-for-nulls 存在以下内容的表 - ┌─x─┬────y─┐ - │ 1 │ ᴺᵁᴸᴸ │ - │ 2 │ 3 │ - └───┴──────┘ +```text +┌─x─┬────y─┐ +│ 1 │ ᴺᵁᴸᴸ │ +│ 2 │ 3 │ +└───┴──────┘ +``` 对其进行查询 - :) SELECT x FROM t_null WHERE isNull(y) +```sql +SELECT x FROM t_null WHERE isNull(y) +``` - SELECT x - FROM t_null - WHERE isNull(y) - - ┌─x─┐ - │ 1 │ - └───┘ - - 1 rows in set. Elapsed: 0.010 sec. +```text +┌─x─┐ +│ 1 │ +└───┘ +``` ## isNotNull {#isnotnull} @@ -60,24 +60,24 @@ slug: /zh/sql-reference/functions/functions-for-nulls 存在以下内容的表 - ┌─x─┬────y─┐ - │ 1 │ ᴺᵁᴸᴸ │ - │ 2 │ 3 │ - └───┴──────┘ +```text +┌─x─┬────y─┐ +│ 1 │ ᴺᵁᴸᴸ │ +│ 2 │ 3 │ +└───┴──────┘ +``` 对其进行查询 - :) SELECT x FROM t_null WHERE isNotNull(y) +```sql +SELECT x FROM t_null WHERE isNotNull(y) +``` - SELECT x - FROM t_null - WHERE isNotNull(y) - - ┌─x─┐ - │ 2 │ - └───┘ - - 1 rows in set. Elapsed: 0.010 sec. +```text +┌─x─┐ +│ 2 │ +└───┘ +``` ## 合并 {#coalesce} @@ -98,26 +98,27 @@ slug: /zh/sql-reference/functions/functions-for-nulls 考虑可以指定多种联系客户的方式的联系人列表。 - ┌─name─────┬─mail─┬─phone─────┬──icq─┐ - │ client 1 │ ᴺᵁᴸᴸ │ 123-45-67 │ 123 │ - │ client 2 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ - └──────────┴──────┴───────────┴──────┘ +```text +┌─name─────┬─mail─┬─phone─────┬──icq─┐ +│ client 1 │ ᴺᵁᴸᴸ │ 123-45-67 │ 123 │ +│ client 2 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +└──────────┴──────┴───────────┴──────┘ +``` `mail`和`phone`字段是String类型,但`icq`字段是`UInt32`,所以它需要转换为`String`。 从联系人列表中获取客户的第一个可用联系方式: - :) SELECT coalesce(mail, phone, CAST(icq,'Nullable(String)')) FROM aBook +```sql +SELECT coalesce(mail, phone, CAST(icq,'Nullable(String)')) FROM aBook +``` - SELECT coalesce(mail, phone, CAST(icq, 'Nullable(String)')) - FROM aBook - - ┌─name─────┬─coalesce(mail, phone, CAST(icq, 'Nullable(String)'))─┐ - │ client 1 │ 123-45-67 │ - │ client 2 │ ᴺᵁᴸᴸ │ - └──────────┴──────────────────────────────────────────────────────┘ - - 2 rows in set. Elapsed: 0.006 sec. +```text +┌─name─────┬─coalesce(mail, phone, CAST(icq, 'Nullable(String)'))─┐ +│ client 1 │ 123-45-67 │ +│ client 2 │ ᴺᵁᴸᴸ │ +└──────────┴──────────────────────────────────────────────────────┘ +``` ## ifNull {#ifnull} diff --git a/docs/zh/sql-reference/functions/other-functions.md b/docs/zh/sql-reference/functions/other-functions.md index 07acf8fdfe0..a5c67e94921 100644 --- a/docs/zh/sql-reference/functions/other-functions.md +++ b/docs/zh/sql-reference/functions/other-functions.md @@ -398,23 +398,25 @@ FROM **`toTypeName ' 与 ' toColumnTypeName`的区别示例** - :) select toTypeName(cast('2018-01-01 01:02:03' AS DateTime)) +```sql +SELECT toTypeName(CAST('2018-01-01 01:02:03', 'DateTime')) +``` - SELECT toTypeName(CAST('2018-01-01 01:02:03', 'DateTime')) +```text +┌─toTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ +│ DateTime │ +└─────────────────────────────────────────────────────┘ +``` - ┌─toTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ - │ DateTime │ - └─────────────────────────────────────────────────────┘ +```sql +SELECT toColumnTypeName(CAST('2018-01-01 01:02:03', 'DateTime')) +``` - 1 rows in set. Elapsed: 0.008 sec. - - :) select toColumnTypeName(cast('2018-01-01 01:02:03' AS DateTime)) - - SELECT toColumnTypeName(CAST('2018-01-01 01:02:03', 'DateTime')) - - ┌─toColumnTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ - │ Const(UInt32) │ - └───────────────────────────────────────────────────────────┘ +```text +┌─toColumnTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ +│ Const(UInt32) │ +└───────────────────────────────────────────────────────────┘ +``` 该示例显示`DateTime`数据类型作为`Const(UInt32)`存储在内存中。 @@ -460,26 +462,25 @@ FROM **示例** - :) SELECT defaultValueOfArgumentType( CAST(1 AS Int8) ) +```sql +SELECT defaultValueOfArgumentType(CAST(1, 'Int8')) +``` - SELECT defaultValueOfArgumentType(CAST(1, 'Int8')) +```text +┌─defaultValueOfArgumentType(CAST(1, 'Int8'))─┐ +│ 0 │ +└─────────────────────────────────────────────┘ +``` - ┌─defaultValueOfArgumentType(CAST(1, 'Int8'))─┐ - │ 0 │ - └─────────────────────────────────────────────┘ - - 1 rows in set. Elapsed: 0.002 sec. - - :) SELECT defaultValueOfArgumentType( CAST(1 AS Nullable(Int8) ) ) - - SELECT defaultValueOfArgumentType(CAST(1, 'Nullable(Int8)')) - - ┌─defaultValueOfArgumentType(CAST(1, 'Nullable(Int8)'))─┐ - │ ᴺᵁᴸᴸ │ - └───────────────────────────────────────────────────────┘ - - 1 rows in set. Elapsed: 0.002 sec. +```sql +SELECT defaultValueOfArgumentType(CAST(1, 'Nullable(Int8)')) +``` +```text +┌─defaultValueOfArgumentType(CAST(1, 'Nullable(Int8)'))─┐ +│ ᴺᵁᴸᴸ │ +└───────────────────────────────────────────────────────┘ +``` ## indexHint {#indexhint} 输出符合索引选择范围内的所有数据,同时不实用参数中的表达式进行过滤。 @@ -506,9 +507,11 @@ SELECT count() FROM ontime 对该表进行如下的查询: +```sql +SELECT FlightDate AS k, count() FROM ontime GROUP BY k ORDER BY k ``` -:) SELECT FlightDate AS k, count() FROM ontime GROUP BY k ORDER BY k +```text SELECT FlightDate AS k, count() @@ -530,9 +533,11 @@ ORDER BY k ASC 在这个查询中,由于没有使用索引,所以ClickHouse将处理整个表的所有数据(`Processed 4.28 million rows`)。使用下面的查询尝试使用索引进行查询: +```sql +SELECT FlightDate AS k, count() FROM ontime WHERE k = '2017-09-15' GROUP BY k ORDER BY k ``` -:) SELECT FlightDate AS k, count() FROM ontime WHERE k = '2017-09-15' GROUP BY k ORDER BY k +```text SELECT FlightDate AS k, count() @@ -552,9 +557,11 @@ ORDER BY k ASC 现在将表达式`k = '2017-09-15'`传递给`indexHint`函数: +```sql +SELECT FlightDate AS k, count() FROM ontime WHERE indexHint(k = '2017-09-15') GROUP BY k ORDER BY k ``` -:) SELECT FlightDate AS k, count() FROM ontime WHERE indexHint(k = '2017-09-15') GROUP BY k ORDER BY k +```text SELECT FlightDate AS k, count() diff --git a/docs/zh/sql-reference/functions/uuid-functions.md b/docs/zh/sql-reference/functions/uuid-functions.md index 8ee65dd52d0..e635fd4fba8 100644 --- a/docs/zh/sql-reference/functions/uuid-functions.md +++ b/docs/zh/sql-reference/functions/uuid-functions.md @@ -21,13 +21,13 @@ UUID类型的值。 此示例演示如何在表中创建UUID类型的列,并对其写入数据。 -``` sql -:) CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog - -:) INSERT INTO t_uuid SELECT generateUUIDv4() - -:) SELECT * FROM t_uuid +```sql +CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog +INSERT INTO t_uuid SELECT generateUUIDv4() +SELECT * FROM t_uuid +``` +```text ┌────────────────────────────────────x─┐ │ f4bf890f-f9dc-4332-ad5c-0c18e73f28e9 │ └──────────────────────────────────────┘ @@ -47,9 +47,11 @@ UUID类型的值 **使用示例** -``` sql -:) SELECT toUUID('61f0c404-5cb3-11e7-907b-a6006ad3dba0') AS uuid +```sql +SELECT toUUID('61f0c404-5cb3-11e7-907b-a6006ad3dba0') AS uuid +``` +```text ┌─────────────────────────────────uuid─┐ │ 61f0c404-5cb3-11e7-907b-a6006ad3dba0 │ └──────────────────────────────────────┘ @@ -70,10 +72,12 @@ UUIDStringToNum(String) **使用示例** ``` sql -:) SELECT +SELECT '612f3c40-5d3b-217e-707b-6a546a3d7b29' AS uuid, UUIDStringToNum(uuid) AS bytes +``` +```text ┌─uuid─────────────────────────────────┬─bytes────────────┐ │ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ a/<@];!~p{jTj={) │ └──────────────────────────────────────┴──────────────────┘ diff --git a/docs/zh/sql-reference/operators/index.md b/docs/zh/sql-reference/operators/index.md index 7e0bd9a9cfb..8544f9f5a91 100644 --- a/docs/zh/sql-reference/operators/index.md +++ b/docs/zh/sql-reference/operators/index.md @@ -226,18 +226,14 @@ ClickHouse 支持 `IS NULL` 和 `IS NOT NULL` 。 -``` bash -:) SELECT x+100 FROM t_null WHERE y IS NULL - -SELECT x + 100 -FROM t_null -WHERE isNull(y) +``` sql +SELECT x+100 FROM t_null WHERE y IS NULL +``` +``` text ┌─plus(x, 100)─┐ │ 101 │ └──────────────┘ - -1 rows in set. Elapsed: 0.002 sec. ``` ### IS NOT NULL {#is-not-null} @@ -249,16 +245,12 @@ WHERE isNull(y) -``` bash -:) SELECT * FROM t_null WHERE y IS NOT NULL - -SELECT * -FROM t_null -WHERE isNotNull(y) +``` sql +SELECT * FROM t_null WHERE y IS NOT NULL +``` +``` text ┌─x─┬─y─┐ │ 2 │ 3 │ └───┴───┘ - -1 rows in set. Elapsed: 0.002 sec. ``` diff --git a/docs/zh/sql-reference/table-functions/format.md b/docs/zh/sql-reference/table-functions/format.md index ea2087fde5e..bc017ccc3c7 100644 --- a/docs/zh/sql-reference/table-functions/format.md +++ b/docs/zh/sql-reference/table-functions/format.md @@ -27,7 +27,7 @@ A table with data parsed from `data` argument according specified format and ext **Query:** ``` sql -:) select * from format(JSONEachRow, +SELECT * FROM format(JSONEachRow, $$ {"a": "Hello", "b": 111} {"a": "World", "b": 123} @@ -49,8 +49,7 @@ $$) **Query:** ```sql - -:) desc format(JSONEachRow, +DESC format(JSONEachRow, $$ {"a": "Hello", "b": 111} {"a": "World", "b": 123} From d34a755cfb14fed23c00eb86ad22c4102682bec0 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 11 Jan 2023 09:59:23 -0500 Subject: [PATCH 47/49] switch text to response for query blocks --- docs/zh/operations/system-tables/numbers.md | 2 +- docs/zh/operations/system-tables/one.md | 2 +- docs/zh/sql-reference/data-types/array.md | 8 +++---- docs/zh/sql-reference/data-types/enum.md | 10 ++++---- .../data-types/special-data-types/nothing.md | 2 +- docs/zh/sql-reference/data-types/tuple.md | 4 ++-- .../functions/functions-for-nulls.md | 12 +++++----- .../functions/other-functions.md | 23 ++++++++++--------- .../sql-reference/functions/uuid-functions.md | 9 ++++---- docs/zh/sql-reference/operators/index.md | 8 +++---- .../sql-reference/table-functions/format.md | 4 ++-- 11 files changed, 43 insertions(+), 41 deletions(-) diff --git a/docs/zh/operations/system-tables/numbers.md b/docs/zh/operations/system-tables/numbers.md index 8cb92351ae7..801c43f8e91 100644 --- a/docs/zh/operations/system-tables/numbers.md +++ b/docs/zh/operations/system-tables/numbers.md @@ -15,7 +15,7 @@ slug: /zh/operations/system-tables/numbers SELECT * FROM system.numbers LIMIT 10; ``` -```text +```response ┌─number─┐ │ 0 │ │ 1 │ diff --git a/docs/zh/operations/system-tables/one.md b/docs/zh/operations/system-tables/one.md index 2e44a9bd89a..29dd25c5282 100644 --- a/docs/zh/operations/system-tables/one.md +++ b/docs/zh/operations/system-tables/one.md @@ -15,7 +15,7 @@ slug: /zh/operations/system-tables/one SELECT * FROM system.one LIMIT 10; ``` -```text +```response ┌─dummy─┐ │ 0 │ └───────┘ diff --git a/docs/zh/sql-reference/data-types/array.md b/docs/zh/sql-reference/data-types/array.md index 0c38eb86004..46c40b889ad 100644 --- a/docs/zh/sql-reference/data-types/array.md +++ b/docs/zh/sql-reference/data-types/array.md @@ -23,7 +23,7 @@ slug: /zh/sql-reference/data-types/array SELECT array(1, 2) AS x, toTypeName(x) ``` -```text +```response ┌─x─────┬─toTypeName(array(1, 2))─┐ │ [1,2] │ Array(UInt8) │ └───────┴─────────────────────────┘ @@ -33,7 +33,7 @@ SELECT array(1, 2) AS x, toTypeName(x) SELECT [1, 2] AS x, toTypeName(x) ``` -``` text +```response ┌─x─────┬─toTypeName([1, 2])─┐ │ [1,2] │ Array(UInt8) │ └───────┴────────────────────┘ @@ -50,7 +50,7 @@ ClickHouse会自动检测数组元素,并根据元素计算出存储这些元素 SELECT array(1, 2, NULL) AS x, toTypeName(x) ``` -```text +```response ┌─x──────────┬─toTypeName(array(1, 2, NULL))─┐ │ [1,2,NULL] │ Array(Nullable(UInt8)) │ └────────────┴───────────────────────────────┘ @@ -62,7 +62,7 @@ SELECT array(1, 2, NULL) AS x, toTypeName(x) SELECT array(1, 'a') ``` -```text +```response Received exception from server (version 1.1.54388): Code: 386. DB::Exception: Received from localhost:9000, 127.0.0.1. DB::Exception: There is no supertype for types UInt8, String because some of them are String/FixedString and some of them are not. ``` diff --git a/docs/zh/sql-reference/data-types/enum.md b/docs/zh/sql-reference/data-types/enum.md index 9832df3da02..496a4c5a78c 100644 --- a/docs/zh/sql-reference/data-types/enum.md +++ b/docs/zh/sql-reference/data-types/enum.md @@ -24,7 +24,7 @@ slug: /zh/sql-reference/data-types/enum INSERT INTO t_enum VALUES ('hello'), ('world'), ('hello') ``` -```text +```response Ok. 3 rows in set. Elapsed: 0.002 sec. @@ -34,7 +34,7 @@ Ok. INSERT INTO t_enum VALUES('a') ``` -```text +```response Exception on client: Code: 49. DB::Exception: Unknown element 'a' for type Enum8('hello' = 1, 'world' = 2) ``` @@ -45,7 +45,7 @@ Code: 49. DB::Exception: Unknown element 'a' for type Enum8('hello' = 1, 'world' SELECT * FROM t_enum ``` -```text +```response ┌─x─────┐ │ hello │ │ world │ @@ -59,7 +59,7 @@ SELECT * FROM t_enum SELECT CAST(x, 'Int8') FROM t_enum ``` -```text +```response ┌─CAST(x, 'Int8')─┐ │ 1 │ │ 2 │ @@ -73,7 +73,7 @@ SELECT CAST(x, 'Int8') FROM t_enum SELECT toTypeName(CAST('a', 'Enum8(\'a\' = 1, \'b\' = 2)')) ``` -```text +```response ┌─toTypeName(CAST('a', 'Enum8(\'a\' = 1, \'b\' = 2)'))─┐ │ Enum8('a' = 1, 'b' = 2) │ └──────────────────────────────────────────────────────┘ diff --git a/docs/zh/sql-reference/data-types/special-data-types/nothing.md b/docs/zh/sql-reference/data-types/special-data-types/nothing.md index e123622edf6..19a78cb540e 100644 --- a/docs/zh/sql-reference/data-types/special-data-types/nothing.md +++ b/docs/zh/sql-reference/data-types/special-data-types/nothing.md @@ -13,7 +13,7 @@ slug: /zh/sql-reference/data-types/special-data-types/nothing SELECT toTypeName(array()) ``` -```text +```response ┌─toTypeName(array())─┐ │ Array(Nothing) │ └─────────────────────┘ diff --git a/docs/zh/sql-reference/data-types/tuple.md b/docs/zh/sql-reference/data-types/tuple.md index 905a872da24..004c80ff916 100644 --- a/docs/zh/sql-reference/data-types/tuple.md +++ b/docs/zh/sql-reference/data-types/tuple.md @@ -21,7 +21,7 @@ slug: /zh/sql-reference/data-types/tuple SELECT tuple(1,'a') AS x, toTypeName(x) ``` -```text +```response ┌─x───────┬─toTypeName(tuple(1, 'a'))─┐ │ (1,'a') │ Tuple(UInt8, String) │ └─────────┴───────────────────────────┘ @@ -37,7 +37,7 @@ SELECT tuple(1,'a') AS x, toTypeName(x) SELECT tuple(1, NULL) AS x, toTypeName(x) ``` -```text +```response ┌─x────────┬─toTypeName(tuple(1, NULL))──────┐ │ (1,NULL) │ Tuple(UInt8, Nullable(Nothing)) │ └──────────┴─────────────────────────────────┘ diff --git a/docs/zh/sql-reference/functions/functions-for-nulls.md b/docs/zh/sql-reference/functions/functions-for-nulls.md index 9ecf39e56c5..4dd30970923 100644 --- a/docs/zh/sql-reference/functions/functions-for-nulls.md +++ b/docs/zh/sql-reference/functions/functions-for-nulls.md @@ -22,7 +22,7 @@ slug: /zh/sql-reference/functions/functions-for-nulls 存在以下内容的表 -```text +```response ┌─x─┬────y─┐ │ 1 │ ᴺᵁᴸᴸ │ │ 2 │ 3 │ @@ -35,7 +35,7 @@ slug: /zh/sql-reference/functions/functions-for-nulls SELECT x FROM t_null WHERE isNull(y) ``` -```text +```response ┌─x─┐ │ 1 │ └───┘ @@ -60,7 +60,7 @@ SELECT x FROM t_null WHERE isNull(y) 存在以下内容的表 -```text +```response ┌─x─┬────y─┐ │ 1 │ ᴺᵁᴸᴸ │ │ 2 │ 3 │ @@ -73,7 +73,7 @@ SELECT x FROM t_null WHERE isNull(y) SELECT x FROM t_null WHERE isNotNull(y) ``` -```text +```response ┌─x─┐ │ 2 │ └───┘ @@ -98,7 +98,7 @@ SELECT x FROM t_null WHERE isNotNull(y) 考虑可以指定多种联系客户的方式的联系人列表。 -```text +```response ┌─name─────┬─mail─┬─phone─────┬──icq─┐ │ client 1 │ ᴺᵁᴸᴸ │ 123-45-67 │ 123 │ │ client 2 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ @@ -113,7 +113,7 @@ SELECT x FROM t_null WHERE isNotNull(y) SELECT coalesce(mail, phone, CAST(icq,'Nullable(String)')) FROM aBook ``` -```text +```response ┌─name─────┬─coalesce(mail, phone, CAST(icq, 'Nullable(String)'))─┐ │ client 1 │ 123-45-67 │ │ client 2 │ ᴺᵁᴸᴸ │ diff --git a/docs/zh/sql-reference/functions/other-functions.md b/docs/zh/sql-reference/functions/other-functions.md index a5c67e94921..2eeaad63694 100644 --- a/docs/zh/sql-reference/functions/other-functions.md +++ b/docs/zh/sql-reference/functions/other-functions.md @@ -33,7 +33,7 @@ slug: /zh/sql-reference/functions/other-functions SELECT 'some/long/path/to/file' AS a, basename(a) ``` -``` text +```response ┌─a──────────────────────┬─basename('some\\long\\path\\to\\file')─┐ │ some\long\path\to\file │ file │ └────────────────────────┴────────────────────────────────────────┘ @@ -43,7 +43,7 @@ SELECT 'some/long/path/to/file' AS a, basename(a) SELECT 'some\\long\\path\\to\\file' AS a, basename(a) ``` -``` text +```response ┌─a──────────────────────┬─basename('some\\long\\path\\to\\file')─┐ │ some\long\path\to\file │ file │ └────────────────────────┴────────────────────────────────────────┘ @@ -53,7 +53,7 @@ SELECT 'some\\long\\path\\to\\file' AS a, basename(a) SELECT 'some-file-name' AS a, basename(a) ``` -``` text +```response ┌─a──────────────┬─basename('some-file-name')─┐ │ some-file-name │ some-file-name │ └────────────────┴────────────────────────────┘ @@ -402,7 +402,7 @@ FROM SELECT toTypeName(CAST('2018-01-01 01:02:03', 'DateTime')) ``` -```text +```response ┌─toTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ │ DateTime │ └─────────────────────────────────────────────────────┘ @@ -412,7 +412,7 @@ SELECT toTypeName(CAST('2018-01-01 01:02:03', 'DateTime')) SELECT toColumnTypeName(CAST('2018-01-01 01:02:03', 'DateTime')) ``` -```text +```response ┌─toColumnTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ │ Const(UInt32) │ └───────────────────────────────────────────────────────────┘ @@ -466,7 +466,7 @@ SELECT toColumnTypeName(CAST('2018-01-01 01:02:03', 'DateTime')) SELECT defaultValueOfArgumentType(CAST(1, 'Int8')) ``` -```text +```response ┌─defaultValueOfArgumentType(CAST(1, 'Int8'))─┐ │ 0 │ └─────────────────────────────────────────────┘ @@ -476,7 +476,7 @@ SELECT defaultValueOfArgumentType(CAST(1, 'Int8')) SELECT defaultValueOfArgumentType(CAST(1, 'Nullable(Int8)')) ``` -```text +```response ┌─defaultValueOfArgumentType(CAST(1, 'Nullable(Int8)'))─┐ │ ᴺᵁᴸᴸ │ └───────────────────────────────────────────────────────┘ @@ -497,7 +497,8 @@ SELECT defaultValueOfArgumentType(CAST(1, 'Nullable(Int8)')) ``` SELECT count() FROM ontime - +``` +```response ┌─count()─┐ │ 4276457 │ └─────────┘ @@ -511,7 +512,7 @@ SELECT count() FROM ontime SELECT FlightDate AS k, count() FROM ontime GROUP BY k ORDER BY k ``` -```text +```response SELECT FlightDate AS k, count() @@ -537,7 +538,7 @@ ORDER BY k ASC SELECT FlightDate AS k, count() FROM ontime WHERE k = '2017-09-15' GROUP BY k ORDER BY k ``` -```text +```response SELECT FlightDate AS k, count() @@ -561,7 +562,7 @@ ORDER BY k ASC SELECT FlightDate AS k, count() FROM ontime WHERE indexHint(k = '2017-09-15') GROUP BY k ORDER BY k ``` -```text +```response SELECT FlightDate AS k, count() diff --git a/docs/zh/sql-reference/functions/uuid-functions.md b/docs/zh/sql-reference/functions/uuid-functions.md index e635fd4fba8..57b75a6c889 100644 --- a/docs/zh/sql-reference/functions/uuid-functions.md +++ b/docs/zh/sql-reference/functions/uuid-functions.md @@ -27,7 +27,7 @@ INSERT INTO t_uuid SELECT generateUUIDv4() SELECT * FROM t_uuid ``` -```text +```response ┌────────────────────────────────────x─┐ │ f4bf890f-f9dc-4332-ad5c-0c18e73f28e9 │ └──────────────────────────────────────┘ @@ -51,7 +51,7 @@ UUID类型的值 SELECT toUUID('61f0c404-5cb3-11e7-907b-a6006ad3dba0') AS uuid ``` -```text +```response ┌─────────────────────────────────uuid─┐ │ 61f0c404-5cb3-11e7-907b-a6006ad3dba0 │ └──────────────────────────────────────┘ @@ -77,7 +77,7 @@ SELECT UUIDStringToNum(uuid) AS bytes ``` -```text +```response ┌─uuid─────────────────────────────────┬─bytes────────────┐ │ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ a/<@];!~p{jTj={) │ └──────────────────────────────────────┴──────────────────┘ @@ -101,7 +101,8 @@ UUIDNumToString(FixedString(16)) SELECT 'a/<@];!~p{jTj={)' AS bytes, UUIDNumToString(toFixedString(bytes, 16)) AS uuid - +``` +```response ┌─bytes────────────┬─uuid─────────────────────────────────┐ │ a/<@];!~p{jTj={) │ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ └──────────────────┴──────────────────────────────────────┘ diff --git a/docs/zh/sql-reference/operators/index.md b/docs/zh/sql-reference/operators/index.md index 8544f9f5a91..353386903c4 100644 --- a/docs/zh/sql-reference/operators/index.md +++ b/docs/zh/sql-reference/operators/index.md @@ -143,7 +143,7 @@ SELECT FROM test.Orders; ``` -``` text +``` response ┌─OrderYear─┬─OrderMonth─┬─OrderDay─┬─OrderHour─┬─OrderMinute─┬─OrderSecond─┐ │ 2008 │ 10 │ 11 │ 13 │ 23 │ 44 │ └───────────┴────────────┴──────────┴───────────┴─────────────┴─────────────┘ @@ -161,7 +161,7 @@ FROM test.Orders; SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR ``` -``` text +``` response ┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┐ │ 2019-10-23 11:16:28 │ 2019-10-27 14:16:28 │ └─────────────────────┴────────────────────────────────────────────────────────┘ @@ -230,7 +230,7 @@ ClickHouse 支持 `IS NULL` 和 `IS NOT NULL` 。 SELECT x+100 FROM t_null WHERE y IS NULL ``` -``` text +``` response ┌─plus(x, 100)─┐ │ 101 │ └──────────────┘ @@ -249,7 +249,7 @@ SELECT x+100 FROM t_null WHERE y IS NULL SELECT * FROM t_null WHERE y IS NOT NULL ``` -``` text +``` response ┌─x─┬─y─┐ │ 2 │ 3 │ └───┴───┘ diff --git a/docs/zh/sql-reference/table-functions/format.md b/docs/zh/sql-reference/table-functions/format.md index bc017ccc3c7..f84d047e599 100644 --- a/docs/zh/sql-reference/table-functions/format.md +++ b/docs/zh/sql-reference/table-functions/format.md @@ -38,7 +38,7 @@ $$) **Result:** -```text +```response ┌───b─┬─a─────┐ │ 111 │ Hello │ │ 123 │ World │ @@ -60,7 +60,7 @@ $$) **Result:** -```text +```response ┌─name─┬─type──────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐ │ b │ Nullable(Float64) │ │ │ │ │ │ │ a │ Nullable(String) │ │ │ │ │ │ From 7168c217b0bd26ff47fd13a8f0e8f8fcc01b2839 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 11 Jan 2023 10:08:11 -0500 Subject: [PATCH 48/49] switch text to response for query blocks --- docs/en/interfaces/formats.md | 28 +++++++++---------- docs/en/operations/system-tables/disks.md | 2 +- .../system-tables/merge_tree_settings.md | 2 +- docs/en/operations/system-tables/numbers.md | 2 +- .../en/operations/system-tables/numbers_mt.md | 2 +- docs/en/operations/system-tables/one.md | 2 +- docs/en/operations/system-tables/processes.md | 2 +- .../sql-reference/table-functions/format.md | 4 +-- .../sql-reference/table-functions/format.md | 4 +-- 9 files changed, 24 insertions(+), 24 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 75ef0ac3cc0..e28c486afca 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -101,7 +101,7 @@ The `TabSeparated` format supports outputting total values (when using WITH TOTA SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT TabSeparated ``` -``` text +``` response 2014-03-17 1406958 2014-03-18 1383658 2014-03-19 1405797 @@ -177,7 +177,7 @@ INSERT INTO nestedt Values ( 1, [1], ['a']) SELECT * FROM nestedt FORMAT TSV ``` -``` text +``` response 1 [1] ['a'] ``` @@ -761,7 +761,7 @@ SELECT * FROM json_as_string; Result: -``` text +``` response ┌─json──────────────────────────────┐ │ {"foo":{"bar":{"x":"y"},"baz":1}} │ │ {} │ @@ -782,7 +782,7 @@ SELECT * FROM json_square_brackets; Result: -```text +```response ┌─field──────────────────────┐ │ {"id": 1, "name": "name1"} │ │ {"id": 2, "name": "name2"} │ @@ -1118,7 +1118,7 @@ When inserting data with `input_format_defaults_for_omitted_fields = 1`, ClickHo Consider the `UserActivity` table as an example: -``` text +``` response ┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ │ 4324182021466249494 │ 5 │ 146 │ -1 │ │ 4324182021466249494 │ 6 │ 185 │ 1 │ @@ -1127,7 +1127,7 @@ Consider the `UserActivity` table as an example: The query `SELECT * FROM UserActivity FORMAT JSONEachRow` returns: -``` text +``` response {"UserID":"4324182021466249494","PageViews":5,"Duration":146,"Sign":-1} {"UserID":"4324182021466249494","PageViews":6,"Duration":185,"Sign":1} ``` @@ -1171,7 +1171,7 @@ Without this setting, ClickHouse throws an exception. SELECT name, value FROM system.settings WHERE name = 'input_format_import_nested_json' ``` -``` text +``` response ┌─name────────────────────────────┬─value─┐ │ input_format_import_nested_json │ 0 │ └─────────────────────────────────┴───────┘ @@ -1181,7 +1181,7 @@ SELECT name, value FROM system.settings WHERE name = 'input_format_import_nested INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n": {"s": ["abc", "def"], "i": [1, 23]}} ``` -``` text +``` response Code: 117. DB::Exception: Unknown field found while parsing JSONEachRow format: n: (at row 1) ``` @@ -1191,7 +1191,7 @@ INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n": {"s": ["abc", "def"], SELECT * FROM json_each_row_nested ``` -``` text +``` response ┌─n.s───────────┬─n.i────┐ │ ['abc','def'] │ [1,23] │ └───────────────┴────────┘ @@ -1300,7 +1300,7 @@ Example (shown for the [PrettyCompact](#prettycompact) format): SELECT * FROM t_null ``` -``` text +``` response ┌─x─┬────y─┐ │ 1 │ ᴺᵁᴸᴸ │ └───┴──────┘ @@ -1312,7 +1312,7 @@ Rows are not escaped in Pretty\* formats. Example is shown for the [PrettyCompac SELECT 'String with \'quotes\' and \t character' AS Escaping_test ``` -``` text +``` response ┌─Escaping_test────────────────────────┐ │ String with 'quotes' and character │ └──────────────────────────────────────┘ @@ -1327,7 +1327,7 @@ The Pretty format supports outputting total values (when using WITH TOTALS) and SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT PrettyCompact ``` -``` text +``` response ┌──EventDate─┬───────c─┐ │ 2014-03-17 │ 1406958 │ │ 2014-03-18 │ 1383658 │ @@ -1488,7 +1488,7 @@ Example: SELECT * FROM t_null FORMAT Vertical ``` -``` text +``` response Row 1: ────── x: 1 @@ -1501,7 +1501,7 @@ Rows are not escaped in Vertical format: SELECT 'string with \'quotes\' and \t with some special \n characters' AS test FORMAT Vertical ``` -``` text +``` response Row 1: ────── test: string with 'quotes' and with some special diff --git a/docs/en/operations/system-tables/disks.md b/docs/en/operations/system-tables/disks.md index d492e42c2ec..a079f3338d2 100644 --- a/docs/en/operations/system-tables/disks.md +++ b/docs/en/operations/system-tables/disks.md @@ -20,7 +20,7 @@ Columns: SELECT * FROM system.disks; ``` -```text +```response ┌─name────┬─path─────────────────┬───free_space─┬──total_space─┬─keep_free_space─┐ │ default │ /var/lib/clickhouse/ │ 276392587264 │ 490652508160 │ 0 │ └─────────┴──────────────────────┴──────────────┴──────────────┴─────────────────┘ diff --git a/docs/en/operations/system-tables/merge_tree_settings.md b/docs/en/operations/system-tables/merge_tree_settings.md index a05d4abccda..672c79e335b 100644 --- a/docs/en/operations/system-tables/merge_tree_settings.md +++ b/docs/en/operations/system-tables/merge_tree_settings.md @@ -18,7 +18,7 @@ Columns: SELECT * FROM system.merge_tree_settings LIMIT 4 FORMAT Vertical; ``` -```text +```response Row 1: ────── name: index_granularity diff --git a/docs/en/operations/system-tables/numbers.md b/docs/en/operations/system-tables/numbers.md index 0dc001ebb6f..68efeb2ee38 100644 --- a/docs/en/operations/system-tables/numbers.md +++ b/docs/en/operations/system-tables/numbers.md @@ -15,7 +15,7 @@ Reads from this table are not parallelized. SELECT * FROM system.numbers LIMIT 10; ``` -```text +```response ┌─number─┐ │ 0 │ │ 1 │ diff --git a/docs/en/operations/system-tables/numbers_mt.md b/docs/en/operations/system-tables/numbers_mt.md index cc461b29ad0..653a8d43cc9 100644 --- a/docs/en/operations/system-tables/numbers_mt.md +++ b/docs/en/operations/system-tables/numbers_mt.md @@ -13,7 +13,7 @@ Used for tests. SELECT * FROM system.numbers_mt LIMIT 10; ``` -```text +```response ┌─number─┐ │ 0 │ │ 1 │ diff --git a/docs/en/operations/system-tables/one.md b/docs/en/operations/system-tables/one.md index ee2907a6d6d..6d3519f0069 100644 --- a/docs/en/operations/system-tables/one.md +++ b/docs/en/operations/system-tables/one.md @@ -15,7 +15,7 @@ This is similar to the `DUAL` table found in other DBMSs. SELECT * FROM system.one LIMIT 10; ``` -```text +```response ┌─dummy─┐ │ 0 │ └───────┘ diff --git a/docs/en/operations/system-tables/processes.md b/docs/en/operations/system-tables/processes.md index 95c46f551ef..76219813ad7 100644 --- a/docs/en/operations/system-tables/processes.md +++ b/docs/en/operations/system-tables/processes.md @@ -23,7 +23,7 @@ Columns: SELECT * FROM system.processes LIMIT 10 FORMAT Vertical; ``` -```text +```response Row 1: ────── is_initial_query: 1 diff --git a/docs/en/sql-reference/table-functions/format.md b/docs/en/sql-reference/table-functions/format.md index 4a0ee58d758..3af48249e3c 100644 --- a/docs/en/sql-reference/table-functions/format.md +++ b/docs/en/sql-reference/table-functions/format.md @@ -38,7 +38,7 @@ $$) **Result:** -```text +```response ┌───b─┬─a─────┐ │ 111 │ Hello │ │ 123 │ World │ @@ -60,7 +60,7 @@ $$) **Result:** -```text +```response ┌─name─┬─type──────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐ │ b │ Nullable(Float64) │ │ │ │ │ │ │ a │ Nullable(String) │ │ │ │ │ │ diff --git a/docs/ru/sql-reference/table-functions/format.md b/docs/ru/sql-reference/table-functions/format.md index a91b4ca2b1e..204658914e0 100644 --- a/docs/ru/sql-reference/table-functions/format.md +++ b/docs/ru/sql-reference/table-functions/format.md @@ -38,7 +38,7 @@ $$) **Result:** -```text +```response ┌───b─┬─a─────┐ │ 111 │ Hello │ │ 123 │ World │ @@ -60,7 +60,7 @@ $$) **Result:** -```text +```response ┌─name─┬─type──────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐ │ b │ Nullable(Float64) │ │ │ │ │ │ │ a │ Nullable(String) │ │ │ │ │ │ From 806dd1357ca44bc3995a4af80039567e70aefca2 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 11 Jan 2023 10:13:31 -0500 Subject: [PATCH 49/49] switch text to response for query blocks --- docs/zh/operations/system-tables/merge_tree_settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/merge_tree_settings.md b/docs/zh/operations/system-tables/merge_tree_settings.md index c2bdcd14d24..3118d6b7530 100644 --- a/docs/zh/operations/system-tables/merge_tree_settings.md +++ b/docs/zh/operations/system-tables/merge_tree_settings.md @@ -19,7 +19,7 @@ slug: /zh/operations/system-tables/merge_tree_settings SELECT * FROM system.merge_tree_settings LIMIT 4 FORMAT Vertical; ``` -```text +```response Row 1: ────── name: index_granularity