diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index a6cda0016a8..22a868f218e 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -693,11 +693,10 @@ MergeTreeDataMergerMutator::getColumnsForNewDataPart( } } - bool is_wide_part = isWidePart(source_part); SerializationInfoByName new_serialization_infos; for (const auto & [name, info] : serialization_infos) { - if (is_wide_part && removed_columns.count(name)) + if (removed_columns.count(name)) continue; auto it = renamed_columns_from_to.find(name); @@ -709,7 +708,7 @@ MergeTreeDataMergerMutator::getColumnsForNewDataPart( /// In compact parts we read all columns, because they all stored in a /// single file - if (!is_wide_part) + if (!isWidePart(source_part)) return {updated_header.getNamesAndTypesList(), new_serialization_infos}; Names source_column_names = source_part->getColumns().getNames(); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index c001b319c99..1fe701c54ae 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -422,10 +422,11 @@ void finalizeMutatedPart( const CompressionCodecPtr & codec) { auto disk = new_data_part->volume->getDisk(); + auto part_path = fs::path(new_data_part->getFullRelativePath()); if (new_data_part->uuid != UUIDHelpers::Nil) { - auto out = disk->writeFile(new_data_part->getFullRelativePath() + IMergeTreeDataPart::UUID_FILE_NAME, 4096); + auto out = disk->writeFile(part_path / IMergeTreeDataPart::UUID_FILE_NAME, 4096); HashingWriteBuffer out_hashing(*out); writeUUIDText(new_data_part->uuid, out_hashing); new_data_part->checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_size = out_hashing.count(); @@ -435,27 +436,36 @@ void finalizeMutatedPart( if (execute_ttl_type != ExecuteTTLType::NONE) { /// Write a file with ttl infos in json format. - auto out_ttl = disk->writeFile(fs::path(new_data_part->getFullRelativePath()) / "ttl.txt", 4096); + auto out_ttl = disk->writeFile(part_path / "ttl.txt", 4096); HashingWriteBuffer out_hashing(*out_ttl); new_data_part->ttl_infos.write(out_hashing); new_data_part->checksums.files["ttl.txt"].file_size = out_hashing.count(); new_data_part->checksums.files["ttl.txt"].file_hash = out_hashing.getHash(); } + if (!new_data_part->getSerializationInfos().empty()) + { + auto out = disk->writeFile(part_path / IMergeTreeDataPart::SERIALIZATION_FILE_NAME, 4096); + HashingWriteBuffer out_hashing(*out); + new_data_part->getSerializationInfos().writeJSON(out_hashing); + new_data_part->checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_size = out_hashing.count(); + new_data_part->checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_hash = out_hashing.getHash(); + } + { /// Write file with checksums. - auto out_checksums = disk->writeFile(fs::path(new_data_part->getFullRelativePath()) / "checksums.txt", 4096); + auto out_checksums = disk->writeFile(part_path / "checksums.txt", 4096); new_data_part->checksums.write(*out_checksums); } /// close fd { - auto out = disk->writeFile(new_data_part->getFullRelativePath() + IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME, 4096); + auto out = disk->writeFile(part_path / IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME, 4096); DB::writeText(queryToString(codec->getFullCodecDesc()), *out); } { /// Write a file with a description of columns. - auto out_columns = disk->writeFile(fs::path(new_data_part->getFullRelativePath()) / "columns.txt", 4096); + auto out_columns = disk->writeFile(part_path / "columns.txt", 4096); new_data_part->getColumns().writeText(*out_columns); } /// close fd @@ -466,7 +476,7 @@ void finalizeMutatedPart( new_data_part->modification_time = time(nullptr); new_data_part->loadProjections(false, false); new_data_part->setBytesOnDisk( - MergeTreeData::DataPart::calculateTotalSizeOnDisk(new_data_part->volume->getDisk(), new_data_part->getFullRelativePath())); + MergeTreeData::DataPart::calculateTotalSizeOnDisk(new_data_part->volume->getDisk(), part_path)); new_data_part->default_codec = codec; new_data_part->calculateColumnsAndSecondaryIndicesSizesOnDisk(); new_data_part->storage.lockSharedData(*new_data_part); @@ -1308,14 +1318,13 @@ bool MutateTask::prepare() ctx->mrk_extension = ctx->source_part->index_granularity_info.is_adaptive ? getAdaptiveMrkExtension(ctx->new_data_part->getType()) : getNonAdaptiveMrkExtension(); - const auto data_settings = ctx->data-> getSettings(); + const auto data_settings = ctx->data->getSettings(); ctx->need_sync = needSyncPart(ctx->source_part->rows_count, ctx->source_part->getBytesOnDisk(), *data_settings); ctx->execute_ttl_type = ExecuteTTLType::NONE; if (ctx->mutating_pipeline.initialized()) ctx->execute_ttl_type = MergeTreeDataMergerMutator::shouldExecuteTTL(ctx->metadata_snapshot, ctx->interpreter->getColumnDependencies()); - /// All columns from part are changed and may be some more that were missing before in part /// TODO We can materialize compact part without copying data if (!isWidePart(ctx->source_part) @@ -1325,7 +1334,6 @@ bool MutateTask::prepare() } else /// TODO: check that we modify only non-key columns in this case. { - /// We will modify only some of the columns. Other columns and key values can be copied as-is. for (const auto & name_type : ctx->updated_header.getNamesAndTypesList()) ctx->updated_columns.emplace(name_type.name); diff --git a/tests/queries/0_stateless/01780_column_sparse_alter.reference b/tests/queries/0_stateless/01780_column_sparse_alter.reference index 4fb0122db96..38fa0bf446a 100644 --- a/tests/queries/0_stateless/01780_column_sparse_alter.reference +++ b/tests/queries/0_stateless/01780_column_sparse_alter.reference @@ -5,3 +5,6 @@ u Sparse id Default t Sparse 182 +id Default +t Sparse +182 diff --git a/tests/queries/0_stateless/01780_column_sparse_alter.sql b/tests/queries/0_stateless/01780_column_sparse_alter.sql index 7f9558bfc18..4fb43bc814a 100644 --- a/tests/queries/0_stateless/01780_column_sparse_alter.sql +++ b/tests/queries/0_stateless/01780_column_sparse_alter.sql @@ -5,7 +5,6 @@ DROP TABLE IF EXISTS t_sparse_alter; CREATE TABLE t_sparse_alter (id UInt64, u UInt64, s String) ENGINE = MergeTree ORDER BY id SETTINGS ratio_of_defaults_for_sparse_serialization = 0.5; - INSERT INTO t_sparse_alter SELECT number, if (number % 11 = 0, number, 0), @@ -23,4 +22,10 @@ SELECT column, serialization_kind FROM system.parts_columns WHERE database = cur SELECT uniqExact(t) FROM t_sparse_alter; +DETACH TABLE t_sparse_alter; +ATTACH TABLE t_sparse_alter; + +SELECT column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_sparse_alter' AND active ORDER BY column; +SELECT uniqExact(t) FROM t_sparse_alter; + DROP TABLE t_sparse_alter;