From f8f67a788e4c8dc41b59d6f22631172fb4a431df Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 25 Jun 2020 19:55:45 +0300 Subject: [PATCH 001/298] allow to turn on fsync on inserts and merges --- src/Disks/DiskLocal.cpp | 17 +++++++- src/Disks/DiskLocal.h | 2 + src/Disks/DiskMemory.cpp | 5 +++ src/Disks/DiskMemory.h | 2 + src/Disks/IDisk.h | 3 ++ src/Disks/S3/DiskS3.cpp | 5 +++ src/Disks/S3/DiskS3.h | 2 + .../MergeTree/IMergeTreeDataPartWriter.cpp | 11 +++-- .../MergeTree/IMergeTreeDataPartWriter.h | 6 +-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 41 +++++++++++++------ .../MergeTree/MergeTreeDataMergerMutator.h | 6 ++- .../MergeTreeDataPartWriterCompact.cpp | 4 +- .../MergeTreeDataPartWriterCompact.h | 2 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 4 +- .../MergeTree/MergeTreeDataPartWriterWide.h | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 7 +++- src/Storages/MergeTree/MergeTreeSettings.h | 3 ++ .../MergeTree/MergedBlockOutputStream.cpp | 7 ++-- .../MergeTree/MergedBlockOutputStream.h | 1 + .../MergedColumnOnlyOutputStream.cpp | 9 ++-- .../MergeTree/MergedColumnOnlyOutputStream.h | 2 +- 21 files changed, 108 insertions(+), 33 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 68f5ee99a7a..c67bac7ffe2 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -8,7 +8,7 @@ #include #include - +#include namespace DB { @@ -19,6 +19,9 @@ namespace ErrorCodes extern const int EXCESSIVE_ELEMENT_IN_CONFIG; extern const int PATH_ACCESS_DENIED; extern const int INCORRECT_DISK_INDEX; + extern const int FILE_DOESNT_EXIST; + extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_FSYNC; } std::mutex DiskLocal::reservation_mutex; @@ -188,6 +191,18 @@ void DiskLocal::moveDirectory(const String & from_path, const String & to_path) Poco::File(disk_path + from_path).renameTo(disk_path + to_path); } +void DiskLocal::sync(const String & path) const +{ + String full_path = disk_path + path; + int fd = ::open(full_path.c_str(), O_RDONLY); + if (-1 == fd) + throwFromErrnoWithPath("Cannot open file " + full_path, full_path, + errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE); + + if (-1 == fsync(fd)) + throwFromErrnoWithPath("Cannot fsync " + full_path, full_path, ErrorCodes::CANNOT_FSYNC); +} + DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) { return std::make_unique(disk_path, path); diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 61a3994b655..743ba2ceb10 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -59,6 +59,8 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; + void sync(const String & path) const override; + DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 3e43d159ba5..5b3350e40f7 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -261,6 +261,11 @@ void DiskMemory::moveDirectory(const String & /*from_path*/, const String & /*to throw Exception("Method moveDirectory is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); } +void DiskMemory::sync(const String & /*path*/) const +{ + throw Exception("Method sync is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + DiskDirectoryIteratorPtr DiskMemory::iterateDirectory(const String & path) { std::lock_guard lock(mutex); diff --git a/src/Disks/DiskMemory.h b/src/Disks/DiskMemory.h index b0c1d30c61d..8a3ddf05aa7 100644 --- a/src/Disks/DiskMemory.h +++ b/src/Disks/DiskMemory.h @@ -52,6 +52,8 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; + void sync(const String & path) const override; + DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 011c75402f4..8de77a560d1 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -105,6 +105,9 @@ public: /// Move directory from `from_path` to `to_path`. virtual void moveDirectory(const String & from_path, const String & to_path) = 0; + /// Do fsync on directory. + virtual void sync(const String & path) const = 0; + /// Return iterator to the contents of the specified directory. virtual DiskDirectoryIteratorPtr iterateDirectory(const String & path) = 0; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 71b5991f770..292f6567df4 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -466,6 +466,11 @@ void DiskS3::clearDirectory(const String & path) remove(it->path()); } +void DiskS3::sync(const String & /*path*/) const +{ + throw Exception("Method sync is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + void DiskS3::moveFile(const String & from_path, const String & to_path) { if (exists(to_path)) diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 5fa8e8358a6..09132367ae8 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -58,6 +58,8 @@ public: void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); } + void sync(const String & path) const override; + DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void moveFile(const String & from_path, const String & to_path) override; diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 73ac7fc0064..03ae2166504 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -308,7 +308,8 @@ void IMergeTreeDataPartWriter::calculateAndSerializeSkipIndices( skip_index_data_mark = skip_index_current_data_mark; } -void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums) +void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization( + MergeTreeData::DataPart::Checksums & checksums, bool sync) { bool write_final_mark = (with_final_mark && data_written); if (write_final_mark && compute_granularity) @@ -330,12 +331,14 @@ void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization(MergeTreeData::Da index_stream->next(); checksums.files["primary.idx"].file_size = index_stream->count(); checksums.files["primary.idx"].file_hash = index_stream->getHash(); - index_stream = nullptr; + if (sync) + index_stream->sync(); + index_stream.reset(); } } void IMergeTreeDataPartWriter::finishSkipIndicesSerialization( - MergeTreeData::DataPart::Checksums & checksums) + MergeTreeData::DataPart::Checksums & checksums, bool sync) { for (size_t i = 0; i < skip_indices.size(); ++i) { @@ -348,6 +351,8 @@ void IMergeTreeDataPartWriter::finishSkipIndicesSerialization( { stream->finalize(); stream->addToChecksums(checksums); + if (sync) + stream->sync(); } skip_indices_streams.clear(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 2f849e7c895..eebdb880a66 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -102,9 +102,9 @@ public: void initSkipIndices(); void initPrimaryIndex(); - virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) = 0; - void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums); - void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums); + virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) = 0; + void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums, bool sync); + void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksum, bool sync); protected: /// Count index_granularity for block and store in `index_granularity` diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 00830dd78c2..ccd7f234925 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -576,6 +576,13 @@ public: } }; +static bool needSyncPart(const size_t input_rows, size_t input_bytes, const MergeTreeSettings & settings) +{ + return ((settings.min_rows_to_sync_after_merge && input_rows >= settings.min_rows_to_sync_after_merge) + || (settings.min_compressed_bytes_to_sync_after_merge && input_bytes >= settings.min_compressed_bytes_to_sync_after_merge)); +} + + /// parts should be sorted. MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( const FutureMergedMutatedPart & future_part, @@ -648,6 +655,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor } size_t sum_input_rows_upper_bound = merge_entry->total_rows_count; + size_t sum_compressed_bytes_upper_bound = merge_entry->total_size_bytes_compressed; MergeAlgorithm merge_alg = chooseMergeAlgorithm(parts, sum_input_rows_upper_bound, gathering_columns, deduplicate, need_remove_expired_values); LOG_DEBUG(log, "Selected MergeAlgorithm: {}", ((merge_alg == MergeAlgorithm::Vertical) ? "Vertical" : "Horizontal")); @@ -803,7 +811,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (need_remove_expired_values) merged_stream = std::make_shared(merged_stream, data, metadata_snapshot, new_data_part, time_of_merge, force_ttl); - if (metadata_snapshot->hasSecondaryIndices()) { const auto & indices = metadata_snapshot->getSecondaryIndices(); @@ -863,6 +870,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (need_remove_expired_values && ttl_merges_blocker.isCancelled()) throw Exception("Cancelled merging parts with expired TTL", ErrorCodes::ABORTED); + bool need_sync = needSyncPart(sum_input_rows_upper_bound, sum_compressed_bytes_upper_bound, *data_settings); MergeTreeData::DataPart::Checksums checksums_gathered_columns; /// Gather ordinary columns @@ -942,7 +950,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); column_gathered_stream.readSuffix(); - auto changed_checksums = column_to.writeSuffixAndGetChecksums(new_data_part, checksums_gathered_columns); + auto changed_checksums = column_to.writeSuffixAndGetChecksums(new_data_part, checksums_gathered_columns, need_sync); checksums_gathered_columns.add(std::move(changed_checksums)); if (rows_written != column_elems_written) @@ -979,9 +987,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor } if (merge_alg != MergeAlgorithm::Vertical) - to.writeSuffixAndFinalizePart(new_data_part); + to.writeSuffixAndFinalizePart(new_data_part, need_sync); else - to.writeSuffixAndFinalizePart(new_data_part, &storage_columns, &checksums_gathered_columns); + to.writeSuffixAndFinalizePart(new_data_part, need_sync, &storage_columns, &checksums_gathered_columns); + + if (need_sync) + new_data_part->volume->getDisk()->sync(new_part_tmp_path); return new_data_part; } @@ -1081,7 +1092,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor /// Don't change granularity type while mutating subset of columns auto mrk_extension = source_part->index_granularity_info.is_adaptive ? getAdaptiveMrkExtension(new_data_part->getType()) : getNonAdaptiveMrkExtension(); - + bool need_sync = needSyncPart(source_part->rows_count, source_part->getBytesOnDisk(), *data_settings); bool need_remove_expired_values = false; if (in && shouldExecuteTTL(metadata_snapshot, in->getHeader().getNamesAndTypesList().getNames(), commands_for_part)) @@ -1099,7 +1110,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor time_of_mutation, compression_codec, merge_entry, - need_remove_expired_values); + need_remove_expired_values, + need_sync); /// no finalization required, because mutateAllPartColumns use /// MergedBlockOutputStream which finilaze all part fields itself @@ -1154,7 +1166,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor time_of_mutation, compression_codec, merge_entry, - need_remove_expired_values); + need_remove_expired_values, + need_sync); } for (const auto & [rename_from, rename_to] : files_to_rename) @@ -1174,6 +1187,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor finalizeMutatedPart(source_part, new_data_part, need_remove_expired_values); } + if (need_sync) + new_data_part->volume->getDisk()->sync(new_part_tmp_path); + return new_data_part; } @@ -1599,7 +1615,8 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const + bool need_remove_expired_values, + bool need_sync) const { if (mutating_stream == nullptr) throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR); @@ -1637,7 +1654,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( new_data_part->minmax_idx = std::move(minmax_idx); mutating_stream->readSuffix(); - out.writeSuffixAndFinalizePart(new_data_part); + out.writeSuffixAndFinalizePart(new_data_part, need_sync); } void MergeTreeDataMergerMutator::mutateSomePartColumns( @@ -1650,7 +1667,8 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( time_t time_of_mutation, const CompressionCodecPtr & compression_codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const + bool need_remove_expired_values, + bool need_sync) const { if (mutating_stream == nullptr) throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR); @@ -1684,10 +1702,9 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( mutating_stream->readSuffix(); - auto changed_checksums = out.writeSuffixAndGetChecksums(new_data_part, new_data_part->checksums); + auto changed_checksums = out.writeSuffixAndGetChecksums(new_data_part, new_data_part->checksums, need_sync); new_data_part->checksums.add(std::move(changed_checksums)); - } void MergeTreeDataMergerMutator::finalizeMutatedPart( diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 121cc770d51..23b8d7f681b 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -189,7 +189,8 @@ private: time_t time_of_mutation, const CompressionCodecPtr & codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const; + bool need_remove_expired_values, + bool need_sync) const; /// Mutate some columns of source part with mutation_stream void mutateSomePartColumns( @@ -202,7 +203,8 @@ private: time_t time_of_mutation, const CompressionCodecPtr & codec, MergeListEntry & merge_entry, - bool need_remove_expired_values) const; + bool need_remove_expired_values, + bool need_sync) const; /// Initialize and write to disk new part fields like checksums, columns, /// etc. diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index f7a3ad75cf5..79800204a3b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -141,7 +141,7 @@ void MergeTreeDataPartWriterCompact::writeColumnSingleGranule(const ColumnWithTy column.type->serializeBinaryBulkStateSuffix(serialize_settings, state); } -void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) +void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) { if (columns_buffer.size() != 0) writeBlock(header.cloneWithColumns(columns_buffer.releaseColumns())); @@ -158,6 +158,8 @@ void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart: stream->finalize(); stream->addToChecksums(checksums); + if (sync) + stream->sync(); stream.reset(); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 8183c038c4c..dde7deafc58 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -20,7 +20,7 @@ public: void write(const Block & block, const IColumn::Permutation * permutation, const Block & primary_key_block, const Block & skip_indexes_block) override; - void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; protected: void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index e71ea4d4b94..fcd0249b10c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -264,7 +264,7 @@ void MergeTreeDataPartWriterWide::writeColumn( next_index_offset = current_row - total_rows; } -void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) +void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) { const auto & global_settings = storage.global_context.getSettingsRef(); IDataType::SerializeBinaryBulkSettings serialize_settings; @@ -295,6 +295,8 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Ch { stream.second->finalize(); stream.second->addToChecksums(checksums); + if (sync) + stream.second->sync(); } column_streams.clear(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index f5a9d17f63c..4286065a3ca 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -23,7 +23,7 @@ public: void write(const Block & block, const IColumn::Permutation * permutation, const Block & primary_key_block, const Block & skip_indexes_block) override; - void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenOffsetColumns & offset_columns); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 099480aca2f..cf8860b7f04 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -303,10 +303,15 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); + bool sync_on_insert = data.getSettings()->sync_after_insert; out.writePrefix(); out.writeWithPermutation(block, perm_ptr); - out.writeSuffixAndFinalizePart(new_data_part); + out.writeSuffixAndFinalizePart(new_data_part, sync_on_insert); + + /// Sync part directory. + if (sync_on_insert) + new_data_part->volume->getDisk()->sync(full_path); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterRows, block.rows()); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterUncompressedBytes, block.bytes()); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index f2d2a7cc3d4..da2c9ee49ee 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -43,6 +43,9 @@ struct MergeTreeSettings : public SettingsCollection M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(SettingSeconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ + M(SettingUInt64, min_rows_to_sync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_sync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingBool, sync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ \ /** Inserts settings. */ \ M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index e776a35f21f..5e15084aa7d 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -85,6 +85,7 @@ void MergedBlockOutputStream::writeSuffix() void MergedBlockOutputStream::writeSuffixAndFinalizePart( MergeTreeData::MutableDataPartPtr & new_part, + bool sync, const NamesAndTypesList * total_columns_list, MergeTreeData::DataPart::Checksums * additional_column_checksums) { @@ -95,9 +96,9 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( checksums = std::move(*additional_column_checksums); /// Finish columns serialization. - writer->finishDataSerialization(checksums); - writer->finishPrimaryIndexSerialization(checksums); - writer->finishSkipIndicesSerialization(checksums); + writer->finishDataSerialization(checksums, sync); + writer->finishPrimaryIndexSerialization(checksums, sync); + writer->finishSkipIndicesSerialization(checksums, sync); NamesAndTypesList part_columns; if (!total_columns_list) diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 1a8bf9da822..002ef78a9af 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -46,6 +46,7 @@ public: /// Finilize writing part and fill inner structures void writeSuffixAndFinalizePart( MergeTreeData::MutableDataPartPtr & new_part, + bool sync = false, const NamesAndTypesList * total_columns_list = nullptr, MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 1faadd0d720..e767fb3f155 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -63,12 +63,15 @@ void MergedColumnOnlyOutputStream::writeSuffix() } MergeTreeData::DataPart::Checksums -MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums) +MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums( + MergeTreeData::MutableDataPartPtr & new_part, + MergeTreeData::DataPart::Checksums & all_checksums, + bool sync) { /// Finish columns serialization. MergeTreeData::DataPart::Checksums checksums; - writer->finishDataSerialization(checksums); - writer->finishSkipIndicesSerialization(checksums); + writer->finishDataSerialization(checksums, sync); + writer->finishSkipIndicesSerialization(checksums, sync); auto columns = new_part->getColumns(); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index 902138ced9d..507a964ede0 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -27,7 +27,7 @@ public: void write(const Block & block) override; void writeSuffix() override; MergeTreeData::DataPart::Checksums - writeSuffixAndGetChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums); + writeSuffixAndGetChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums, bool sync = false); private: Block header; From b2aa565a37076230af2ceaa32ee21fa351d37931 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 27 Jun 2020 00:55:48 +0300 Subject: [PATCH 002/298] allow to turn on fsync on inserts, merges and fetches --- src/Common/FileSyncGuard.h | 41 +++++++++++++++++++ src/Disks/DiskLocal.cpp | 35 ++++++++++------ src/Disks/DiskLocal.h | 6 ++- src/Disks/DiskMemory.cpp | 20 ++++++--- src/Disks/DiskMemory.h | 6 ++- src/Disks/IDisk.h | 12 ++++-- src/Disks/S3/DiskS3.cpp | 21 +++++++--- src/Disks/S3/DiskS3.h | 6 ++- src/Storages/MergeTree/DataPartsExchange.cpp | 16 +++++++- src/Storages/MergeTree/DataPartsExchange.h | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 +++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 15 ++++--- .../MergeTree/MergeTreeDataWriter.cpp | 12 +++--- src/Storages/MergeTree/MergeTreeSettings.h | 2 + 14 files changed, 154 insertions(+), 44 deletions(-) create mode 100644 src/Common/FileSyncGuard.h diff --git a/src/Common/FileSyncGuard.h b/src/Common/FileSyncGuard.h new file mode 100644 index 00000000000..5ec9b1d0c98 --- /dev/null +++ b/src/Common/FileSyncGuard.h @@ -0,0 +1,41 @@ +#pragma once + +#include + +namespace DB +{ + +/// Helper class, that recieves file descriptor and does fsync for it in destructor. +/// It's used to keep descriptor open, while doing some operations with it, and do fsync at the end. +/// Guaranties of sequence 'close-reopen-fsync' may depend on kernel version. +/// Source: linux-fsdevel mailing-list https://marc.info/?l=linux-fsdevel&m=152535409207496 +class FileSyncGuard +{ +public: + /// NOTE: If you have already opened descriptor, it's preffered to use + /// this constructor instead of construnctor with path. + FileSyncGuard(const DiskPtr & disk_, int fd_) : disk(disk_), fd(fd_) {} + + FileSyncGuard(const DiskPtr & disk_, const String & path) + : disk(disk_), fd(disk_->open(path, O_RDONLY)) {} + + ~FileSyncGuard() + { + try + { + disk->sync(fd); + disk->close(fd); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + +private: + DiskPtr disk; + int fd = -1; +}; + +} + diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index c67bac7ffe2..f85b69baf5e 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; extern const int CANNOT_OPEN_FILE; extern const int CANNOT_FSYNC; + extern const int CANNOT_CLOSE_FILE; } std::mutex DiskLocal::reservation_mutex; @@ -191,18 +192,6 @@ void DiskLocal::moveDirectory(const String & from_path, const String & to_path) Poco::File(disk_path + from_path).renameTo(disk_path + to_path); } -void DiskLocal::sync(const String & path) const -{ - String full_path = disk_path + path; - int fd = ::open(full_path.c_str(), O_RDONLY); - if (-1 == fd) - throwFromErrnoWithPath("Cannot open file " + full_path, full_path, - errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE); - - if (-1 == fsync(fd)) - throwFromErrnoWithPath("Cannot fsync " + full_path, full_path, ErrorCodes::CANNOT_FSYNC); -} - DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) { return std::make_unique(disk_path, path); @@ -299,6 +288,28 @@ void DiskLocal::copy(const String & from_path, const std::shared_ptr & to IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } +int DiskLocal::open(const String & path, mode_t mode) const +{ + String full_path = disk_path + path; + int fd = ::open(full_path.c_str(), mode); + if (-1 == fd) + throwFromErrnoWithPath("Cannot open file " + full_path, full_path, + errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE); + return fd; +} + +void DiskLocal::close(int fd) const +{ + if (-1 == ::close(fd)) + throw Exception("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE); +} + +void DiskLocal::sync(int fd) const +{ + if (-1 == ::fsync(fd)) + throw Exception("Cannot fsync", ErrorCodes::CANNOT_FSYNC); +} + DiskPtr DiskLocalReservation::getDisk(size_t i) const { if (i != 0) diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 743ba2ceb10..d70ac06c18b 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -59,8 +59,6 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; - void sync(const String & path) const override; - DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; @@ -101,6 +99,10 @@ public: void createHardLink(const String & src_path, const String & dst_path) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; + private: bool tryReserve(UInt64 bytes); diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 5b3350e40f7..a7f1df04e1f 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -261,11 +261,6 @@ void DiskMemory::moveDirectory(const String & /*from_path*/, const String & /*to throw Exception("Method moveDirectory is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); } -void DiskMemory::sync(const String & /*path*/) const -{ - throw Exception("Method sync is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); -} - DiskDirectoryIteratorPtr DiskMemory::iterateDirectory(const String & path) { std::lock_guard lock(mutex); @@ -413,6 +408,21 @@ void DiskMemory::setReadOnly(const String &) throw Exception("Method setReadOnly is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); } +int DiskMemory::open(const String & /*path*/, mode_t /*mode*/) const +{ + throw Exception("Method open is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskMemory::close(int /*fd*/) const +{ + throw Exception("Method close is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskMemory::sync(int /*fd*/) const +{ + throw Exception("Method sync is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); +} + using DiskMemoryPtr = std::shared_ptr; diff --git a/src/Disks/DiskMemory.h b/src/Disks/DiskMemory.h index 8a3ddf05aa7..7f111fe5e7d 100644 --- a/src/Disks/DiskMemory.h +++ b/src/Disks/DiskMemory.h @@ -52,8 +52,6 @@ public: void moveDirectory(const String & from_path, const String & to_path) override; - void sync(const String & path) const override; - DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void createFile(const String & path) override; @@ -92,6 +90,10 @@ public: void createHardLink(const String & src_path, const String & dst_path) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; + private: void createDirectoriesImpl(const String & path); void replaceFileImpl(const String & from_path, const String & to_path); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 8de77a560d1..bc5c9381643 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -105,9 +105,6 @@ public: /// Move directory from `from_path` to `to_path`. virtual void moveDirectory(const String & from_path, const String & to_path) = 0; - /// Do fsync on directory. - virtual void sync(const String & path) const = 0; - /// Return iterator to the contents of the specified directory. virtual DiskDirectoryIteratorPtr iterateDirectory(const String & path) = 0; @@ -174,6 +171,15 @@ public: /// Create hardlink from `src_path` to `dst_path`. virtual void createHardLink(const String & src_path, const String & dst_path) = 0; + + /// Wrapper for POSIX open + virtual int open(const String & path, mode_t mode) const = 0; + + /// Wrapper for POSIX close + virtual void close(int fd) const = 0; + + /// Wrapper for POSIX fsync + virtual void sync(int fd) const = 0; }; using DiskPtr = std::shared_ptr; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 292f6567df4..3e0fb05ed6f 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -29,6 +29,7 @@ namespace ErrorCodes extern const int CANNOT_SEEK_THROUGH_FILE; extern const int UNKNOWN_FORMAT; extern const int INCORRECT_DISK_INDEX; + extern const int NOT_IMPLEMENTED; } namespace @@ -466,11 +467,6 @@ void DiskS3::clearDirectory(const String & path) remove(it->path()); } -void DiskS3::sync(const String & /*path*/) const -{ - throw Exception("Method sync is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); -} - void DiskS3::moveFile(const String & from_path, const String & to_path) { if (exists(to_path)) @@ -669,6 +665,21 @@ void DiskS3::setReadOnly(const String & path) Poco::File(metadata_path + path).setReadOnly(true); } +int DiskS3::open(const String & /*path*/, mode_t /*mode*/) const +{ + throw Exception("Method open is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskS3::close(int /*fd*/) const +{ + throw Exception("Method close is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + +void DiskS3::sync(int /*fd*/) const +{ + throw Exception("Method sync is not implemented for S3 disks", ErrorCodes::NOT_IMPLEMENTED); +} + DiskS3Reservation::~DiskS3Reservation() { try diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 09132367ae8..cbf161da561 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -58,8 +58,6 @@ public: void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); } - void sync(const String & path) const override; - DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; void moveFile(const String & from_path, const String & to_path) override; @@ -98,6 +96,10 @@ public: void setReadOnly(const String & path) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; + private: bool tryReserve(UInt64 bytes); diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 6796e630ff2..e7bb8206cd9 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -224,9 +225,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( int server_protocol_version = parse(in.getResponseCookie("server_protocol_version", "0")); ReservationPtr reservation; + size_t sum_files_size = 0; if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE) { - size_t sum_files_size; readBinary(sum_files_size, in); if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS) { @@ -247,7 +248,10 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( reservation = data.makeEmptyReservationOnLargestDisk(); } - return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, std::move(reservation), in); + bool sync = (data_settings->min_compressed_bytes_to_sync_after_fetch + && sum_files_size >= data_settings->min_compressed_bytes_to_sync_after_fetch); + + return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, sync, std::move(reservation), in); } MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( @@ -255,6 +259,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( const String & replica_path, bool to_detached, const String & tmp_prefix_, + bool sync, const ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in) { @@ -276,6 +281,10 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( disk->createDirectories(part_download_path); + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, part_download_path); + MergeTreeData::DataPart::Checksums checksums; for (size_t i = 0; i < files; ++i) { @@ -316,6 +325,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( if (file_name != "checksums.txt" && file_name != "columns.txt") checksums.addFile(file_name, file_size, expected_hash); + + if (sync) + hashing_out.sync(); } assertEOF(in); diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index c1aff6bdba5..e983d6deecf 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -71,6 +71,7 @@ private: const String & replica_path, bool to_detached, const String & tmp_prefix_, + bool sync, const ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 61dfeed6b7c..ab9bb7879aa 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -664,6 +665,10 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ String from = getFullRelativePath(); String to = storage.relative_data_path + new_relative_path + "/"; + std::optional sync_guard; + if (storage.getSettings()->sync_part_directory) + sync_guard.emplace(volume->getDisk(), to); + if (!volume->getDisk()->exists(from)) throw Exception("Part directory " + fullPath(volume->getDisk(), from) + " doesn't exist. Most likely it is logical error.", ErrorCodes::FILE_DOESNT_EXIST); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index ccd7f234925..9c8c4e3c1d5 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -695,6 +696,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor gathering_column_names.clear(); } + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, new_part_tmp_path); + /** Read from all parts, merge and write into a new one. * In passing, we calculate expression for sorting. */ @@ -991,9 +996,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor else to.writeSuffixAndFinalizePart(new_data_part, need_sync, &storage_columns, &checksums_gathered_columns); - if (need_sync) - new_data_part->volume->getDisk()->sync(new_part_tmp_path); - return new_data_part; } @@ -1089,6 +1091,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor disk->createDirectories(new_part_tmp_path); + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, new_part_tmp_path); + /// Don't change granularity type while mutating subset of columns auto mrk_extension = source_part->index_granularity_info.is_adaptive ? getAdaptiveMrkExtension(new_data_part->getType()) : getNonAdaptiveMrkExtension(); @@ -1187,9 +1193,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor finalizeMutatedPart(source_part, new_data_part, need_remove_expired_values); } - if (need_sync) - new_data_part->volume->getDisk()->sync(new_part_tmp_path); - return new_data_part; } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index cf8860b7f04..01f0b086cea 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace ProfileEvents @@ -259,7 +260,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->volume->getDisk()->removeRecursive(full_path); } - new_data_part->volume->getDisk()->createDirectories(full_path); + const auto disk = new_data_part->volume->getDisk(); + disk->createDirectories(full_path); + + std::optional sync_guard; + if (data.getSettings()->sync_part_directory) + sync_guard.emplace(disk, full_path); /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) @@ -309,10 +315,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa out.writeWithPermutation(block, perm_ptr); out.writeSuffixAndFinalizePart(new_data_part, sync_on_insert); - /// Sync part directory. - if (sync_on_insert) - new_data_part->volume->getDisk()->sync(full_path); - ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterRows, block.rows()); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterUncompressedBytes, block.bytes()); ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterCompressedBytes, new_data_part->getBytesOnDisk()); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index da2c9ee49ee..c559ce2804e 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -45,7 +45,9 @@ struct MergeTreeSettings : public SettingsCollection M(SettingSeconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ M(SettingUInt64, min_rows_to_sync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ M(SettingUInt64, min_compressed_bytes_to_sync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_sync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ M(SettingBool, sync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ + M(SettingBool, sync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ \ /** Inserts settings. */ \ M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ From ca346ea13cd0ad0f02a29d59302584c826b52298 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 3 Jul 2020 02:41:37 +0300 Subject: [PATCH 003/298] rename fsync-related settings --- src/Storages/MergeTree/DataPartsExchange.cpp | 6 +++--- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeSettings.h | 10 +++++----- 5 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index e7bb8206cd9..72b478cf587 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -248,8 +248,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( reservation = data.makeEmptyReservationOnLargestDisk(); } - bool sync = (data_settings->min_compressed_bytes_to_sync_after_fetch - && sum_files_size >= data_settings->min_compressed_bytes_to_sync_after_fetch); + bool sync = (data_settings->min_compressed_bytes_to_fsync_after_fetch + && sum_files_size >= data_settings->min_compressed_bytes_to_fsync_after_fetch); return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, sync, std::move(reservation), in); } @@ -282,7 +282,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( disk->createDirectories(part_download_path); std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, part_download_path); MergeTreeData::DataPart::Checksums checksums; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index ab9bb7879aa..3d8cb6b7fc5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -666,7 +666,7 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ String to = storage.relative_data_path + new_relative_path + "/"; std::optional sync_guard; - if (storage.getSettings()->sync_part_directory) + if (storage.getSettings()->fsync_part_directory) sync_guard.emplace(volume->getDisk(), to); if (!volume->getDisk()->exists(from)) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 9c8c4e3c1d5..c39d1981031 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -579,8 +579,8 @@ public: static bool needSyncPart(const size_t input_rows, size_t input_bytes, const MergeTreeSettings & settings) { - return ((settings.min_rows_to_sync_after_merge && input_rows >= settings.min_rows_to_sync_after_merge) - || (settings.min_compressed_bytes_to_sync_after_merge && input_bytes >= settings.min_compressed_bytes_to_sync_after_merge)); + return ((settings.min_rows_to_fsync_after_merge && input_rows >= settings.min_rows_to_fsync_after_merge) + || (settings.min_compressed_bytes_to_fsync_after_merge && input_bytes >= settings.min_compressed_bytes_to_fsync_after_merge)); } @@ -697,7 +697,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor } std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, new_part_tmp_path); /** Read from all parts, merge and write into a new one. @@ -1092,7 +1092,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor disk->createDirectories(new_part_tmp_path); std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, new_part_tmp_path); /// Don't change granularity type while mutating subset of columns diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 01f0b086cea..23210fc604e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -264,7 +264,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa disk->createDirectories(full_path); std::optional sync_guard; - if (data.getSettings()->sync_part_directory) + if (data.getSettings()->fsync_part_directory) sync_guard.emplace(disk, full_path); /// If we need to calculate some columns to sort. @@ -309,7 +309,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); - bool sync_on_insert = data.getSettings()->sync_after_insert; + bool sync_on_insert = data.getSettings()->fsync_after_insert; out.writePrefix(); out.writeWithPermutation(block, perm_ptr); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c559ce2804e..eeee0c4b1e1 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -43,11 +43,11 @@ struct MergeTreeSettings : public SettingsCollection M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(SettingSeconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ - M(SettingUInt64, min_rows_to_sync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ - M(SettingUInt64, min_compressed_bytes_to_sync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ - M(SettingUInt64, min_compressed_bytes_to_sync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ - M(SettingBool, sync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ - M(SettingBool, sync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ + M(SettingUInt64, min_rows_to_fsync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_fsync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ + M(SettingUInt64, min_compressed_bytes_to_fsync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ + M(SettingBool, fsync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ + M(SettingBool, fsync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ \ /** Inserts settings. */ \ M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ From d6434f61dc7b08072862d4d10ea6fa9da781b6c1 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 7 Jul 2020 03:15:02 +0300 Subject: [PATCH 004/298] support codecs in compact parts --- .../MergeTreeDataPartWriterCompact.cpp | 80 +++++++++++++------ .../MergeTreeDataPartWriterCompact.h | 21 ++++- .../01375_compact_parts_codecs.reference | 3 + .../01375_compact_parts_codecs.sql | 31 +++++++ 4 files changed, 109 insertions(+), 26 deletions(-) create mode 100644 tests/queries/0_stateless/01375_compact_parts_codecs.reference create mode 100644 tests/queries/0_stateless/01375_compact_parts_codecs.sql diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index f7a3ad75cf5..696197aa4ca 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -14,19 +14,23 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const MergeTreeIndexGranularity & index_granularity_) : IMergeTreeDataPartWriter( data_part_, columns_list_, metadata_snapshot_, indices_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_) + , plain_file(data_part->volume->getDisk()->writeFile( + part_path + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, + settings.max_compress_block_size, + WriteMode::Rewrite, + settings.estimated_size, + settings.aio_threshold)) + , plain_hashing(*plain_file) + , marks_file(data_part->volume->getDisk()->writeFile( + part_path + MergeTreeDataPartCompact::DATA_FILE_NAME + marks_file_extension_, + 4096, + WriteMode::Rewrite)) + , marks(*marks_file) { - using DataPart = MergeTreeDataPartCompact; - String data_file_name = DataPart::DATA_FILE_NAME; - - stream = std::make_unique( - data_file_name, - data_part->volume->getDisk(), - part_path + data_file_name, DataPart::DATA_FILE_EXTENSION, - part_path + data_file_name, marks_file_extension, - default_codec, - settings.max_compress_block_size, - settings.estimated_size, - settings.aio_threshold); + const auto & storage_columns = metadata_snapshot->getColumns(); + for (const auto & column : columns_list) + compressed_streams[column.name] = std::make_unique( + plain_hashing, storage_columns.getCodecOrDefault(column.name, default_codec)); } void MergeTreeDataPartWriterCompact::write( @@ -98,14 +102,13 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) for (const auto & column : columns_list) { - /// There could already be enough data to compress into the new block. - if (stream->compressed.offset() >= settings.min_compress_block_size) - stream->compressed.next(); + auto & stream = compressed_streams[column.name]; - writeIntBinary(stream->plain_hashing.count(), stream->marks); - writeIntBinary(stream->compressed.offset(), stream->marks); + writeIntBinary(plain_hashing.count(), marks); + writeIntBinary(UInt64(0), marks); writeColumnSingleGranule(block.getByName(column.name), current_row, rows_to_write); + stream->hashing_buf.next(); } ++from_mark; @@ -120,7 +123,7 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) index_granularity.appendMark(rows_written); } - writeIntBinary(rows_to_write, stream->marks); + writeIntBinary(rows_to_write, marks); } next_index_offset = 0; @@ -132,7 +135,7 @@ void MergeTreeDataPartWriterCompact::writeColumnSingleGranule(const ColumnWithTy IDataType::SerializeBinaryBulkStatePtr state; IDataType::SerializeBinaryBulkSettings serialize_settings; - serialize_settings.getter = [this](IDataType::SubstreamPath) -> WriteBuffer * { return &stream->compressed; }; + serialize_settings.getter = [this, &column](IDataType::SubstreamPath) -> WriteBuffer * { return &compressed_streams.at(column.name)->hashing_buf; }; serialize_settings.position_independent_encoding = true; serialize_settings.low_cardinality_max_dictionary_size = 0; @@ -150,15 +153,15 @@ void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart: { for (size_t i = 0; i < columns_list.size(); ++i) { - writeIntBinary(stream->plain_hashing.count(), stream->marks); - writeIntBinary(stream->compressed.offset(), stream->marks); + writeIntBinary(plain_hashing.count(), marks); + writeIntBinary(UInt64(0), marks); } - writeIntBinary(0ULL, stream->marks); + writeIntBinary(UInt64(0), marks); } - stream->finalize(); - stream->addToChecksums(checksums); - stream.reset(); + plain_file->next(); + marks.next(); + addToChecksums(checksums); } static void fillIndexGranularityImpl( @@ -199,6 +202,33 @@ void MergeTreeDataPartWriterCompact::fillIndexGranularity(size_t index_granulari rows_in_block); } +void MergeTreeDataPartWriterCompact::addToChecksums(MergeTreeDataPartChecksums & checksums) +{ + using uint128 = CityHash_v1_0_2::uint128; + + String data_file_name = MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION; + String marks_file_name = MergeTreeDataPartCompact::DATA_FILE_NAME + marks_file_extension; + + checksums.files[data_file_name].is_compressed = true; + size_t uncompressed_size = 0; + uint128 uncompressed_hash{0, 0}; + + for (const auto & [_, stream] : compressed_streams) + { + uncompressed_size += stream->hashing_buf.count(); + uncompressed_hash = CityHash_v1_0_2::CityHash128WithSeed( + reinterpret_cast(&uncompressed_hash), sizeof(uncompressed_hash), uncompressed_hash); + } + + checksums.files[data_file_name].uncompressed_size = uncompressed_size; + checksums.files[data_file_name].uncompressed_hash = uncompressed_hash; + checksums.files[data_file_name].file_size = plain_hashing.count(); + checksums.files[data_file_name].file_hash = plain_hashing.getHash(); + + checksums.files[marks_file_name].file_size = marks.count(); + checksums.files[marks_file_name].file_hash = marks.getHash(); +} + void MergeTreeDataPartWriterCompact::ColumnsBuffer::add(MutableColumns && columns) { if (accumulated_columns.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 8183c038c4c..a5bfd8a16cc 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -34,7 +34,7 @@ private: void writeBlock(const Block & block); - StreamPtr stream; + void addToChecksums(MergeTreeDataPartChecksums & checksumns); Block header; @@ -53,6 +53,25 @@ private: }; ColumnsBuffer columns_buffer; + + /// compressed -> compressed_buf -> plain_hashing -> plain_file + std::unique_ptr plain_file; + HashingWriteBuffer plain_hashing; + + struct CompressedStream + { + CompressedWriteBuffer compressed_buf; + HashingWriteBuffer hashing_buf; + + CompressedStream(WriteBuffer & buf, const CompressionCodecPtr & codec) + : compressed_buf(buf, codec), hashing_buf(compressed_buf) {} + }; + + std::unordered_map> compressed_streams; + + /// marks -> marks_file + std::unique_ptr marks_file; + HashingWriteBuffer marks; }; } diff --git a/tests/queries/0_stateless/01375_compact_parts_codecs.reference b/tests/queries/0_stateless/01375_compact_parts_codecs.reference new file mode 100644 index 00000000000..982c45a26e3 --- /dev/null +++ b/tests/queries/0_stateless/01375_compact_parts_codecs.reference @@ -0,0 +1,3 @@ +12000 11890 +11965 11890 +5858 11890 diff --git a/tests/queries/0_stateless/01375_compact_parts_codecs.sql b/tests/queries/0_stateless/01375_compact_parts_codecs.sql new file mode 100644 index 00000000000..467745c6fa2 --- /dev/null +++ b/tests/queries/0_stateless/01375_compact_parts_codecs.sql @@ -0,0 +1,31 @@ +DROP TABLE IF EXISTS codecs; + +CREATE TABLE codecs (id UInt32, val UInt32, s String) + ENGINE = MergeTree ORDER BY id + SETTINGS min_rows_for_wide_part = 10000; +INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); +SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) + FROM system.parts + WHERE table = 'codecs' AND database = currentDatabase(); + +DROP TABLE codecs; + +CREATE TABLE codecs (id UInt32 CODEC(NONE), val UInt32 CODEC(NONE), s String CODEC(NONE)) + ENGINE = MergeTree ORDER BY id + SETTINGS min_rows_for_wide_part = 10000; +INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); +SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) + FROM system.parts + WHERE table = 'codecs' AND database = currentDatabase(); + +DROP TABLE codecs; + +CREATE TABLE codecs (id UInt32, val UInt32 CODEC(Delta, ZSTD), s String CODEC(ZSTD)) + ENGINE = MergeTree ORDER BY id + SETTINGS min_rows_for_wide_part = 10000; +INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); +SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) + FROM system.parts + WHERE table = 'codecs' AND database = currentDatabase(); + +DROP TABLE codecs; From 80a62977f1aa430144a6bdfae0b7e37605eb5b20 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 9 Jul 2020 21:26:54 +0300 Subject: [PATCH 005/298] fix hashing in DataPartWriterCompact --- src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 69e581a6299..ac697e1b212 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -217,8 +217,9 @@ void MergeTreeDataPartWriterCompact::addToChecksums(MergeTreeDataPartChecksums & for (const auto & [_, stream] : compressed_streams) { uncompressed_size += stream->hashing_buf.count(); + auto stream_hash = stream->hashing_buf.getHash(); uncompressed_hash = CityHash_v1_0_2::CityHash128WithSeed( - reinterpret_cast(&uncompressed_hash), sizeof(uncompressed_hash), uncompressed_hash); + reinterpret_cast(&stream_hash), sizeof(stream_hash), uncompressed_hash); } checksums.files[data_file_name].uncompressed_size = uncompressed_size; From 24f627e52c5f6f461cd1bc42b2306725ad0491b8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 10 Jul 2020 18:57:10 +0300 Subject: [PATCH 006/298] fix reading from compact parts with different codecs --- .../CachedCompressedReadBuffer.cpp | 14 +++- src/Compression/CachedCompressedReadBuffer.h | 4 +- .../CompressedReadBufferFromFile.cpp | 6 ++ .../CompressedReadBufferFromFile.h | 1 + .../MergeTree/MergeTreeReaderCompact.cpp | 80 ++++++++++--------- .../MergeTree/MergeTreeReaderCompact.h | 21 ++++- 6 files changed, 81 insertions(+), 45 deletions(-) diff --git a/src/Compression/CachedCompressedReadBuffer.cpp b/src/Compression/CachedCompressedReadBuffer.cpp index 1b083c004c0..beb13d15f01 100644 --- a/src/Compression/CachedCompressedReadBuffer.cpp +++ b/src/Compression/CachedCompressedReadBuffer.cpp @@ -12,6 +12,7 @@ namespace DB namespace ErrorCodes { extern const int SEEK_POSITION_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } @@ -19,8 +20,9 @@ void CachedCompressedReadBuffer::initInput() { if (!file_in) { - file_in = file_in_creator(); - compressed_in = file_in.get(); + file_in_holder = file_in_creator(); + file_in = file_in_holder.get(); + compressed_in = file_in; if (profile_callback) file_in->setProfileCallback(profile_callback, clock_type); @@ -71,6 +73,14 @@ bool CachedCompressedReadBuffer::nextImpl() return true; } +CachedCompressedReadBuffer::CachedCompressedReadBuffer( + const std::string & path_, ReadBufferFromFileBase * file_in_, UncompressedCache * cache_) + : ReadBuffer(nullptr, 0), file_in(file_in_), cache(cache_), path(path_), file_pos(0) +{ + if (file_in == nullptr) + throw Exception("Neither file_in nor file_in_creator is initialized in CachedCompressedReadBuffer", ErrorCodes::LOGICAL_ERROR); +} + CachedCompressedReadBuffer::CachedCompressedReadBuffer( const std::string & path_, std::function()> file_in_creator_, UncompressedCache * cache_) : ReadBuffer(nullptr, 0), file_in_creator(std::move(file_in_creator_)), cache(cache_), path(path_), file_pos(0) diff --git a/src/Compression/CachedCompressedReadBuffer.h b/src/Compression/CachedCompressedReadBuffer.h index 88bcec8197d..2c5aa4920bd 100644 --- a/src/Compression/CachedCompressedReadBuffer.h +++ b/src/Compression/CachedCompressedReadBuffer.h @@ -22,7 +22,8 @@ class CachedCompressedReadBuffer : public CompressedReadBufferBase, public ReadB private: std::function()> file_in_creator; UncompressedCache * cache; - std::unique_ptr file_in; + std::unique_ptr file_in_holder; + ReadBufferFromFileBase * file_in; const std::string path; size_t file_pos; @@ -38,6 +39,7 @@ private: clockid_t clock_type {}; public: + CachedCompressedReadBuffer(const std::string & path_, ReadBufferFromFileBase * file_in_, UncompressedCache * cache_); CachedCompressedReadBuffer(const std::string & path, std::function()> file_in_creator, UncompressedCache * cache_); void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index ddd8bba686f..2927ee1b399 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -37,6 +37,12 @@ bool CompressedReadBufferFromFile::nextImpl() return true; } +CompressedReadBufferFromFile::CompressedReadBufferFromFile(ReadBufferFromFileBase & file_in_) + : BufferWithOwnMemory(0), file_in(file_in_) +{ + compressed_in = &file_in; +} + CompressedReadBufferFromFile::CompressedReadBufferFromFile(std::unique_ptr buf) : BufferWithOwnMemory(0), p_file_in(std::move(buf)), file_in(*p_file_in) { diff --git a/src/Compression/CompressedReadBufferFromFile.h b/src/Compression/CompressedReadBufferFromFile.h index 1729490f606..1de28062e41 100644 --- a/src/Compression/CompressedReadBufferFromFile.h +++ b/src/Compression/CompressedReadBufferFromFile.h @@ -28,6 +28,7 @@ private: bool nextImpl() override; public: + CompressedReadBufferFromFile(ReadBufferFromFileBase & buf); CompressedReadBufferFromFile(std::unique_ptr buf); CompressedReadBufferFromFile( diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 4357ee66a6e..920f171d7f9 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -45,40 +45,31 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( { size_t buffer_size = settings.max_read_buffer_size; const String full_data_path = data_part->getFullRelativePath() + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION; + file_in = data_part->volume->getDisk()->readFile( + full_data_path, buffer_size, 0, + settings.min_bytes_to_use_direct_io, + settings.min_bytes_to_use_mmap_io); - if (uncompressed_cache) + auto full_path = fullPath(data_part->volume->getDisk(), full_data_path); + for (const auto & column : columns) { - auto buffer = std::make_unique( - fullPath(data_part->volume->getDisk(), full_data_path), - [this, full_data_path, buffer_size]() - { - return data_part->volume->getDisk()->readFile( - full_data_path, - buffer_size, - 0, - settings.min_bytes_to_use_direct_io, - settings.min_bytes_to_use_mmap_io); - }, - uncompressed_cache); + + std::unique_ptr cached_buffer; + std::unique_ptr non_cached_buffer; + if (uncompressed_cache) + { + cached_buffer = std::make_unique(full_path, file_in.get(), uncompressed_cache); + if (profile_callback_) + cached_buffer->setProfileCallback(profile_callback_, clock_type_); + } + else + { + non_cached_buffer = std::make_unique(*file_in); + if (profile_callback_) + non_cached_buffer->setProfileCallback(profile_callback_, clock_type_); + } - if (profile_callback_) - buffer->setProfileCallback(profile_callback_, clock_type_); - - cached_buffer = std::move(buffer); - data_buffer = cached_buffer.get(); - } - else - { - auto buffer = - std::make_unique( - data_part->volume->getDisk()->readFile( - full_data_path, buffer_size, 0, settings.min_bytes_to_use_direct_io, settings.min_bytes_to_use_mmap_io)); - - if (profile_callback_) - buffer->setProfileCallback(profile_callback_, clock_type_); - - non_cached_buffer = std::move(buffer); - data_buffer = non_cached_buffer.get(); + column_streams[column.name] = ColumnStream{std::move(cached_buffer), std::move(non_cached_buffer)}; } size_t columns_num = columns.size(); @@ -181,15 +172,16 @@ void MergeTreeReaderCompact::readData( const String & name, IColumn & column, const IDataType & type, size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets) { + auto & stream = column_streams[name]; if (!isContinuousReading(from_mark, column_position)) - seekToMark(from_mark, column_position); + seekToMark(stream, from_mark, column_position); auto buffer_getter = [&](const IDataType::SubstreamPath & substream_path) -> ReadBuffer * { if (only_offsets && (substream_path.size() != 1 || substream_path[0].type != IDataType::Substream::ArraySizes)) return nullptr; - return data_buffer; + return stream.data_buffer; }; IDataType::DeserializeBinaryBulkSettings deserialize_settings; @@ -209,15 +201,15 @@ void MergeTreeReaderCompact::readData( } -void MergeTreeReaderCompact::seekToMark(size_t row_index, size_t column_index) +void MergeTreeReaderCompact::seekToMark(ColumnStream & stream, size_t row_index, size_t column_index) { MarkInCompressedFile mark = marks_loader.getMark(row_index, column_index); try { - if (cached_buffer) - cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); - if (non_cached_buffer) - non_cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); + if (stream.cached_buffer) + stream.cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); + if (stream.non_cached_buffer) + stream.non_cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); } catch (Exception & e) { @@ -239,4 +231,16 @@ bool MergeTreeReaderCompact::isContinuousReading(size_t mark, size_t column_posi || (mark == last_mark + 1 && column_position == 0 && last_column == data_part->getColumns().size() - 1); } +MergeTreeReaderCompact::ColumnStream::ColumnStream( + std::unique_ptr cached_buffer_, + std::unique_ptr non_cached_buffer_) + : cached_buffer(std::move(cached_buffer_)) + , non_cached_buffer(std::move(non_cached_buffer_)) +{ + if (cached_buffer) + data_buffer = cached_buffer.get(); + else + data_buffer = non_cached_buffer.get(); +} + } diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index 0457b4b6a50..41682f8b0bd 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -35,9 +36,21 @@ public: private: bool isContinuousReading(size_t mark, size_t column_position); - ReadBuffer * data_buffer; - std::unique_ptr cached_buffer; - std::unique_ptr non_cached_buffer; + std::unique_ptr file_in; + + struct ColumnStream + { + std::unique_ptr cached_buffer; + std::unique_ptr non_cached_buffer; + ReadBuffer * data_buffer; + + ColumnStream() = default; + ColumnStream( + std::unique_ptr cached_buffer_, + std::unique_ptr non_cached_buffer_); + }; + + std::unordered_map column_streams; MergeTreeMarksLoader marks_loader; @@ -49,7 +62,7 @@ private: size_t next_mark = 0; std::optional> last_read_granule; - void seekToMark(size_t row_index, size_t column_index); + void seekToMark(ColumnStream & stream, size_t row_index, size_t column_index); void readData(const String & name, IColumn & column, const IDataType & type, size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets = false); From fbec940e0fa7246ca7b42e056de7d0cea50640d0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 11 Jul 2020 02:33:36 +0300 Subject: [PATCH 007/298] fix reading and check query for compact parts with different codecs --- .../CachedCompressedReadBuffer.cpp | 2 ++ src/Compression/CachedCompressedReadBuffer.h | 2 +- .../MergeTreeDataPartWriterCompact.cpp | 4 ++-- .../MergeTree/MergeTreeReaderCompact.cpp | 5 +++-- src/Storages/MergeTree/checkDataPart.cpp | 22 ++++++++++++++----- .../01390_check_table_codec.reference | 2 ++ .../0_stateless/01390_check_table_codec.sql | 15 +++++++++++++ 7 files changed, 41 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/01390_check_table_codec.reference create mode 100644 tests/queries/0_stateless/01390_check_table_codec.sql diff --git a/src/Compression/CachedCompressedReadBuffer.cpp b/src/Compression/CachedCompressedReadBuffer.cpp index beb13d15f01..218925f8eae 100644 --- a/src/Compression/CachedCompressedReadBuffer.cpp +++ b/src/Compression/CachedCompressedReadBuffer.cpp @@ -79,6 +79,8 @@ CachedCompressedReadBuffer::CachedCompressedReadBuffer( { if (file_in == nullptr) throw Exception("Neither file_in nor file_in_creator is initialized in CachedCompressedReadBuffer", ErrorCodes::LOGICAL_ERROR); + + compressed_in = file_in; } CachedCompressedReadBuffer::CachedCompressedReadBuffer( diff --git a/src/Compression/CachedCompressedReadBuffer.h b/src/Compression/CachedCompressedReadBuffer.h index 2c5aa4920bd..89bf66a3e2c 100644 --- a/src/Compression/CachedCompressedReadBuffer.h +++ b/src/Compression/CachedCompressedReadBuffer.h @@ -23,7 +23,7 @@ private: std::function()> file_in_creator; UncompressedCache * cache; std::unique_ptr file_in_holder; - ReadBufferFromFileBase * file_in; + ReadBufferFromFileBase * file_in = nullptr; const std::string path; size_t file_pos; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index ac697e1b212..d15bba232d6 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -17,7 +17,7 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( default_codec_, settings_, index_granularity_) , plain_file(data_part->volume->getDisk()->writeFile( part_path + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, - settings.max_compress_block_size, + settings.max_compress_block_size, WriteMode::Rewrite, settings.estimated_size, settings.aio_threshold)) @@ -31,7 +31,7 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const auto & storage_columns = metadata_snapshot->getColumns(); for (const auto & column : columns_list) compressed_streams[column.name] = std::make_unique( - plain_hashing, storage_columns.getCodecOrDefault(column.name, default_codec)); + plain_hashing, storage_columns.getCodecOrDefault(column.name, default_codec)); } void MergeTreeDataPartWriterCompact::write( diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 920f171d7f9..89ca8b96dba 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -53,7 +53,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( auto full_path = fullPath(data_part->volume->getDisk(), full_data_path); for (const auto & column : columns) { - + std::unique_ptr cached_buffer; std::unique_ptr non_cached_buffer; if (uncompressed_cache) @@ -69,7 +69,8 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( non_cached_buffer->setProfileCallback(profile_callback_, clock_type_); } - column_streams[column.name] = ColumnStream{std::move(cached_buffer), std::move(non_cached_buffer)}; + auto column_from_part = getColumnFromPart(column); + column_streams[column_from_part.name] = ColumnStream{std::move(cached_buffer), std::move(non_cached_buffer)}; } size_t columns_num = columns.size(); diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 793bddc88c0..790a250d831 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -96,11 +96,24 @@ IMergeTreeDataPart::Checksums checkDataPart( }; }; + /// This function calculates only checksum of file content (compressed or uncompressed). + auto checksum_file = [](const DiskPtr & disk_, const String & file_path) + { + auto file_buf = disk_->readFile(file_path); + HashingReadBuffer hashing_buf(*file_buf); + hashing_buf.tryIgnore(std::numeric_limits::max()); + return IMergeTreeDataPart::Checksums::Checksum{hashing_buf.count(), hashing_buf.getHash()}; + }; + + bool check_uncompressed = true; /// First calculate checksums for columns data if (part_type == MergeTreeDataPartType::COMPACT) { const auto & file_name = MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION; - checksums_data.files[file_name] = checksum_compressed_file(disk, path + file_name); + checksums_data.files[file_name] = checksum_file(disk, path + file_name); + /// Uncompressed checksums in compact parts are computed in a complex way. + /// We check only checksum of compressed file. + check_uncompressed = false; } else if (part_type == MergeTreeDataPartType::WIDE) { @@ -141,10 +154,7 @@ IMergeTreeDataPart::Checksums checkDataPart( if (txt_checksum_it == checksum_files_txt.end() || txt_checksum_it->second.uncompressed_size == 0) { /// The file is not compressed. - auto file_buf = disk->readFile(it->path()); - HashingReadBuffer hashing_buf(*file_buf); - hashing_buf.tryIgnore(std::numeric_limits::max()); - checksums_data.files[file_name] = IMergeTreeDataPart::Checksums::Checksum(hashing_buf.count(), hashing_buf.getHash()); + checksums_data.files[file_name] = checksum_file(disk, it->path()); } else /// If we have both compressed and uncompressed in txt, than calculate them { @@ -157,7 +167,7 @@ IMergeTreeDataPart::Checksums checkDataPart( return {}; if (require_checksums || !checksums_txt.files.empty()) - checksums_txt.checkEqual(checksums_data, true); + checksums_txt.checkEqual(checksums_data, check_uncompressed); return checksums_data; } diff --git a/tests/queries/0_stateless/01390_check_table_codec.reference b/tests/queries/0_stateless/01390_check_table_codec.reference new file mode 100644 index 00000000000..3025e6463d8 --- /dev/null +++ b/tests/queries/0_stateless/01390_check_table_codec.reference @@ -0,0 +1,2 @@ +all_1_1_0 1 +all_1_1_0 1 diff --git a/tests/queries/0_stateless/01390_check_table_codec.sql b/tests/queries/0_stateless/01390_check_table_codec.sql new file mode 100644 index 00000000000..639d5bea6e4 --- /dev/null +++ b/tests/queries/0_stateless/01390_check_table_codec.sql @@ -0,0 +1,15 @@ +SET check_query_single_value_result = 0; + +DROP TABLE IF EXISTS check_codec; + +CREATE TABLE check_codec(a Int, b Int CODEC(Delta, ZSTD)) ENGINE = MergeTree ORDER BY a SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO check_codec SELECT number, number * 2 FROM numbers(1000); +CHECK TABLE check_codec; + +DROP TABLE check_codec; + +CREATE TABLE check_codec(a Int, b Int CODEC(Delta, ZSTD)) ENGINE = MergeTree ORDER BY a SETTINGS min_bytes_for_wide_part = '10M'; +INSERT INTO check_codec SELECT number, number * 2 FROM numbers(1000); +CHECK TABLE check_codec; + +DROP TABLE check_codec; From 9384b6950b6c5311202788c8b38ed84dd53a13e8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 14 Jul 2020 15:10:20 +0300 Subject: [PATCH 008/298] Add some tests configs --- tests/ci/build_config.json | 153 +++++++++++++++++++++++ tests/ci/tests_config.json | 242 +++++++++++++++++++++++++++++++++++++ 2 files changed, 395 insertions(+) create mode 100644 tests/ci/build_config.json create mode 100644 tests/ci/tests_config.json diff --git a/tests/ci/build_config.json b/tests/ci/build_config.json new file mode 100644 index 00000000000..e4b9c1d6b75 --- /dev/null +++ b/tests/ci/build_config.json @@ -0,0 +1,153 @@ +[ + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "alien_pkgs": true, + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "performance", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "address", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "undefined", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "thread", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "memory", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "debug", + "sanitizer": "", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "deb", + "bundled": "unbundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "splitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10-darwin", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10-aarch64", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10-freebsd", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + } +] diff --git a/tests/ci/tests_config.json b/tests/ci/tests_config.json new file mode 100644 index 00000000000..481de51d08b --- /dev/null +++ b/tests/ci/tests_config.json @@ -0,0 +1,242 @@ +{ + "Functional stateful tests (address)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (thread)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (memory)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (ubsan)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "undefined", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (debug)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "debug", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (release)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (release, DatabaseAtomic)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (address)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (thread)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (memory)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (ubsan)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "undefined", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (debug)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "debug", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (release)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (unbundled)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "unbundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (release, polymorphic parts enabled)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (release, DatabaseAtomic)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (address)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (thread)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (undefined)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "undefined", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (memory)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + } +} From 230938d3a3082fbf241c9d873571231a69a5f450 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 11 Jul 2020 15:12:42 +0800 Subject: [PATCH 009/298] Refactor joinGet and implement multi-key lookup. --- src/Functions/FunctionJoinGet.cpp | 83 +++++++++---------- src/Functions/FunctionJoinGet.h | 11 +-- src/Interpreters/HashJoin.cpp | 69 ++++++++------- src/Interpreters/HashJoin.h | 10 +-- src/Interpreters/misc.h | 2 +- .../0_stateless/01080_join_get_null.reference | 2 +- .../0_stateless/01080_join_get_null.sql | 12 +-- .../01400_join_get_with_multi_keys.reference | 1 + .../01400_join_get_with_multi_keys.sql | 9 ++ 9 files changed, 104 insertions(+), 95 deletions(-) create mode 100644 tests/queries/0_stateless/01400_join_get_with_multi_keys.reference create mode 100644 tests/queries/0_stateless/01400_join_get_with_multi_keys.sql diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index a33b70684a5..1badc689c6a 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -1,10 +1,10 @@ #include +#include #include #include #include #include -#include #include @@ -16,19 +16,35 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +template +void ExecutableFunctionJoinGet::execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t) +{ + Block keys; + for (size_t i = 2; i < arguments.size(); ++i) + { + auto key = block.getByPosition(arguments[i]); + keys.insert(std::move(key)); + } + block.getByPosition(result) = join->joinGet(keys, result_block); +} + +template +ExecutableFunctionImplPtr FunctionJoinGet::prepare(const Block &, const ColumnNumbers &, size_t) const +{ + return std::make_unique>(join, Block{{return_type->createColumn(), return_type, attr_name}}); +} + static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & context) { - if (arguments.size() != 3) - throw Exception{"Function joinGet takes 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - String join_name; if (const auto * name_col = checkAndGetColumnConst(arguments[0].column.get())) { join_name = name_col->getValue(); } else - throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function joinGet, expected a const string.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception( + "Illegal type " + arguments[0].type->getName() + " of first argument of function joinGet, expected a const string.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); size_t dot = join_name.find('.'); String database_name; @@ -43,10 +59,12 @@ static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & co ++dot; } String table_name = join_name.substr(dot); + if (table_name.empty()) + throw Exception("joinGet does not allow empty table name", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); auto table = DatabaseCatalog::instance().getTable({database_name, table_name}, context); auto storage_join = std::dynamic_pointer_cast(table); if (!storage_join) - throw Exception{"Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception("Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); String attr_name; if (const auto * name_col = checkAndGetColumnConst(arguments[1].column.get())) @@ -54,57 +72,30 @@ static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & co attr_name = name_col->getValue(); } else - throw Exception{"Illegal type " + arguments[1].type->getName() - + " of second argument of function joinGet, expected a const string.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception( + "Illegal type " + arguments[1].type->getName() + " of second argument of function joinGet, expected a const string.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_pair(storage_join, attr_name); } template FunctionBaseImplPtr JoinGetOverloadResolver::build(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const { + if (arguments.size() < 3) + throw Exception( + "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + + ", should be greater or equal to 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); auto [storage_join, attr_name] = getJoin(arguments, context); auto join = storage_join->getJoin(); - DataTypes data_types(arguments.size()); - + DataTypes data_types(arguments.size() - 2); + for (size_t i = 2; i < arguments.size(); ++i) + data_types[i - 2] = arguments[i].type; + auto return_type = join->joinGetCheckAndGetReturnType(data_types, attr_name, or_null); auto table_lock = storage_join->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); - for (size_t i = 0; i < arguments.size(); ++i) - data_types[i] = arguments[i].type; - - auto return_type = join->joinGetReturnType(attr_name, or_null); return std::make_unique>(table_lock, storage_join, join, attr_name, data_types, return_type); } -template -DataTypePtr JoinGetOverloadResolver::getReturnType(const ColumnsWithTypeAndName & arguments) const -{ - auto [storage_join, attr_name] = getJoin(arguments, context); - auto join = storage_join->getJoin(); - return join->joinGetReturnType(attr_name, or_null); -} - - -template -void ExecutableFunctionJoinGet::execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - auto ctn = block.getByPosition(arguments[2]); - if (isColumnConst(*ctn.column)) - ctn.column = ctn.column->cloneResized(1); - ctn.name = ""; // make sure the key name never collide with the join columns - Block key_block = {ctn}; - join->joinGet(key_block, attr_name, or_null); - auto & result_ctn = key_block.getByPosition(1); - if (isColumnConst(*ctn.column)) - result_ctn.column = ColumnConst::create(result_ctn.column, input_rows_count); - block.getByPosition(result) = result_ctn; -} - -template -ExecutableFunctionImplPtr FunctionJoinGet::prepare(const Block &, const ColumnNumbers &, size_t) const -{ - return std::make_unique>(join, attr_name); -} - void registerFunctionJoinGet(FunctionFactory & factory) { // joinGet diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h index a82da589960..6b3b1202f60 100644 --- a/src/Functions/FunctionJoinGet.h +++ b/src/Functions/FunctionJoinGet.h @@ -13,14 +13,14 @@ template class ExecutableFunctionJoinGet final : public IExecutableFunctionImpl { public: - ExecutableFunctionJoinGet(HashJoinPtr join_, String attr_name_) - : join(std::move(join_)), attr_name(std::move(attr_name_)) {} + ExecutableFunctionJoinGet(HashJoinPtr join_, const Block & result_block_) + : join(std::move(join_)), result_block(result_block_) {} static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet"; bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForLowCardinalityColumns() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; @@ -28,7 +28,7 @@ public: private: HashJoinPtr join; - const String attr_name; + Block result_block; }; template @@ -77,13 +77,14 @@ public: String getName() const override { return name; } FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override; - DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const override; + DataTypePtr getReturnType(const ColumnsWithTypeAndName &) const override { return {}; } // Not used bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForLowCardinalityColumns() const override { return true; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } private: const Context & context; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 27294a57675..ffc806b9e88 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -42,6 +42,7 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; extern const int SET_SIZE_LIMIT_EXCEEDED; extern const int TYPE_MISMATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } namespace @@ -1109,27 +1110,34 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) block = block.cloneWithColumns(std::move(dst_columns)); } -static void checkTypeOfKey(const Block & block_left, const Block & block_right) -{ - const auto & [c1, left_type_origin, left_name] = block_left.safeGetByPosition(0); - const auto & [c2, right_type_origin, right_name] = block_right.safeGetByPosition(0); - auto left_type = removeNullable(left_type_origin); - auto right_type = removeNullable(right_type_origin); - if (!left_type->equals(*right_type)) - throw Exception("Type mismatch of columns to joinGet by: " - + left_name + " " + left_type->getName() + " at left, " - + right_name + " " + right_type->getName() + " at right", - ErrorCodes::TYPE_MISMATCH); -} - - -DataTypePtr HashJoin::joinGetReturnType(const String & column_name, bool or_null) const +DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const { std::shared_lock lock(data->rwlock); + size_t num_keys = data_types.size(); + if (right_table_keys.columns() != num_keys) + throw Exception( + "Number of arguments for function joinGet" + toString(or_null ? "OrNull" : "") + + " doesn't match: passed, should be equal to " + toString(num_keys), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (size_t i = 0; i < num_keys; ++i) + { + const auto & left_type_origin = data_types[i]; + const auto & [c2, right_type_origin, right_name] = right_table_keys.safeGetByPosition(i); + auto left_type = removeNullable(left_type_origin); + auto right_type = removeNullable(right_type_origin); + if (!left_type->equals(*right_type)) + throw Exception( + "Type mismatch in joinGet key " + toString(i) + ": found type " + left_type->getName() + ", while the needed type is " + + right_type->getName(), + ErrorCodes::TYPE_MISMATCH); + } + if (!sample_block_with_columns_to_add.has(column_name)) throw Exception("StorageJoin doesn't contain column " + column_name, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + auto elem = sample_block_with_columns_to_add.getByName(column_name); if (or_null) elem.type = makeNullable(elem.type); @@ -1138,34 +1146,33 @@ DataTypePtr HashJoin::joinGetReturnType(const String & column_name, bool or_null template -void HashJoin::joinGetImpl(Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const +ColumnWithTypeAndName HashJoin::joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const { - joinBlockImpl( - block, {block.getByPosition(0).name}, block_with_columns_to_add, maps_); + // Assemble the key block with correct names. + Block keys; + for (size_t i = 0; i < block.columns(); ++i) + { + auto key = block.getByPosition(i); + key.name = key_names_right[i]; + keys.insert(std::move(key)); + } + + joinBlockImpl( + keys, key_names_right, block_with_columns_to_add, maps_); + return keys.getByPosition(keys.columns() - 1); } -// TODO: support composite key // TODO: return multiple columns as named tuple // TODO: return array of values when strictness == ASTTableJoin::Strictness::All -void HashJoin::joinGet(Block & block, const String & column_name, bool or_null) const +ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const { std::shared_lock lock(data->rwlock); - if (key_names_right.size() != 1) - throw Exception("joinGet only supports StorageJoin containing exactly one key", ErrorCodes::UNSUPPORTED_JOIN_KEYS); - - checkTypeOfKey(block, right_table_keys); - - auto elem = sample_block_with_columns_to_add.getByName(column_name); - if (or_null) - elem.type = makeNullable(elem.type); - elem.column = elem.type->createColumn(); - if ((strictness == ASTTableJoin::Strictness::Any || strictness == ASTTableJoin::Strictness::RightAny) && kind == ASTTableJoin::Kind::Left) { - joinGetImpl(block, {elem}, std::get(data->maps)); + return joinGetImpl(block, block_with_columns_to_add, std::get(data->maps)); } else throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN); diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 67d83d27a6d..025f41ac28f 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -162,11 +162,11 @@ public: */ void joinBlock(Block & block, ExtraBlockPtr & not_processed) override; - /// Infer the return type for joinGet function - DataTypePtr joinGetReturnType(const String & column_name, bool or_null) const; + /// Check joinGet arguments and infer the return type. + DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const; - /// Used by joinGet function that turns StorageJoin into a dictionary - void joinGet(Block & block, const String & column_name, bool or_null) const; + /// Used by joinGet function that turns StorageJoin into a dictionary. + ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const; /** Keep "totals" (separate part of dataset, see WITH TOTALS) to use later. */ @@ -383,7 +383,7 @@ private: void joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const; template - void joinGetImpl(Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const; + ColumnWithTypeAndName joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const; static Type chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes); }; diff --git a/src/Interpreters/misc.h b/src/Interpreters/misc.h index 094dfbbbb81..cae2691ca1f 100644 --- a/src/Interpreters/misc.h +++ b/src/Interpreters/misc.h @@ -28,7 +28,7 @@ inline bool functionIsLikeOperator(const std::string & name) inline bool functionIsJoinGet(const std::string & name) { - return name == "joinGet" || startsWith(name, "dictGet"); + return startsWith(name, "joinGet"); } inline bool functionIsDictGet(const std::string & name) diff --git a/tests/queries/0_stateless/01080_join_get_null.reference b/tests/queries/0_stateless/01080_join_get_null.reference index bfde072a796..0cfbf08886f 100644 --- a/tests/queries/0_stateless/01080_join_get_null.reference +++ b/tests/queries/0_stateless/01080_join_get_null.reference @@ -1 +1 @@ -2 2 +2 diff --git a/tests/queries/0_stateless/01080_join_get_null.sql b/tests/queries/0_stateless/01080_join_get_null.sql index 71e7ddf8e75..9f782452d34 100644 --- a/tests/queries/0_stateless/01080_join_get_null.sql +++ b/tests/queries/0_stateless/01080_join_get_null.sql @@ -1,12 +1,12 @@ DROP TABLE IF EXISTS test_joinGet; -DROP TABLE IF EXISTS test_join_joinGet; -CREATE TABLE test_joinGet(id Int32, user_id Nullable(Int32)) Engine = Memory(); -CREATE TABLE test_join_joinGet(user_id Int32, name String) Engine = Join(ANY, LEFT, user_id); +CREATE TABLE test_joinGet(user_id Nullable(Int32), name String) Engine = Join(ANY, LEFT, user_id); -INSERT INTO test_join_joinGet VALUES (2, 'a'), (6, 'b'), (10, 'c'); +INSERT INTO test_joinGet VALUES (2, 'a'), (6, 'b'), (10, 'c'), (null, 'd'); -SELECT 2 id, toNullable(toInt32(2)) user_id WHERE joinGet(test_join_joinGet, 'name', user_id) != ''; +SELECT toNullable(toInt32(2)) user_id WHERE joinGet(test_joinGet, 'name', user_id) != ''; + +-- If the JOIN keys are Nullable fields, the rows where at least one of the keys has the value NULL are not joined. +SELECT cast(null AS Nullable(Int32)) user_id WHERE joinGet(test_joinGet, 'name', user_id) != ''; DROP TABLE test_joinGet; -DROP TABLE test_join_joinGet; diff --git a/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference b/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference new file mode 100644 index 00000000000..49d59571fbf --- /dev/null +++ b/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference @@ -0,0 +1 @@ +0.1 diff --git a/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql b/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql new file mode 100644 index 00000000000..73068270762 --- /dev/null +++ b/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS test_joinGet; + +CREATE TABLE test_joinGet(a String, b String, c Float64) ENGINE = Join(any, left, a, b); + +INSERT INTO test_joinGet VALUES ('ab', '1', 0.1), ('ab', '2', 0.2), ('cd', '3', 0.3); + +SELECT joinGet(test_joinGet, 'c', 'ab', '1'); + +DROP TABLE test_joinGet; From 40504f6a6e9b54bdcdb0c63a5724648bf5bc04f5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 31 Jul 2020 17:57:00 +0300 Subject: [PATCH 010/298] Simpler version of #12999 w/o `pos` changes --- programs/client/Client.cpp | 126 +++++++++++++----- ...06_insert_values_and_expressions.reference | 2 + .../00306_insert_values_and_expressions.sql | 9 ++ 3 files changed, 103 insertions(+), 34 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 797342a1b44..78a6d7fe2d9 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -908,74 +908,127 @@ private: return processMultiQuery(text); } - bool processMultiQuery(const String & text) + bool processMultiQuery(const String & all_queries_text) { const bool test_mode = config().has("testmode"); { /// disable logs if expects errors - TestHint test_hint(test_mode, text); + TestHint test_hint(test_mode, all_queries_text); if (test_hint.clientError() || test_hint.serverError()) processTextAsSingleQuery("SET send_logs_level = 'none'"); } /// Several queries separated by ';'. /// INSERT data is ended by the end of line, not ';'. + /// An exception is VALUES format where we also support semicolon in + /// addition to end of line. - const char * begin = text.data(); - const char * end = begin + text.size(); + const char * this_query_begin = all_queries_text.data(); + const char * all_queries_end = all_queries_text.data() + all_queries_text.size(); - while (begin < end) + while (this_query_begin < all_queries_end) { - const char * pos = begin; - ASTPtr orig_ast = parseQuery(pos, end, true); + // Use the token iterator to skip any whitespace, semicolons and + // comments at the beginning of the query. An example from regression + // tests: + // insert into table t values ('invalid'); -- { serverError 469 } + // select 1 + // Here the test hint comment gets parsed as a part of second query. + // We parse the `INSERT VALUES` up to the semicolon, and the rest + // looks like a two-line query: + // -- { serverError 469 } + // select 1 + // and we expect it to fail with error 469, but this hint is actually + // for the previous query. Test hints should go after the query, so + // we can fix this by skipping leading comments. Token iterator skips + // comments and whitespace by itself, so we only have to check for + // semicolons. + // The code block is to limit visibility of `tokens` because we have + // another such variable further down the code, and get warnings for + // that. + { + Tokens tokens(this_query_begin, all_queries_end); + IParser::Pos token_iterator(tokens, + context.getSettingsRef().max_parser_depth); + while (token_iterator->type == TokenType::Semicolon + && token_iterator.isValid()) + { + ++token_iterator; + } + this_query_begin = token_iterator->begin; + if (this_query_begin >= all_queries_end) + { + break; + } + } - if (!orig_ast) + // Try to parse the query. + const char * this_query_end = this_query_begin; + parsed_query = parseQuery(this_query_end, all_queries_end, true); + + if (!parsed_query) { if (ignore_error) { - Tokens tokens(begin, end); + Tokens tokens(this_query_begin, all_queries_end); IParser::Pos token_iterator(tokens, context.getSettingsRef().max_parser_depth); while (token_iterator->type != TokenType::Semicolon && token_iterator.isValid()) ++token_iterator; - begin = token_iterator->end; + this_query_begin = token_iterator->end; continue; } return true; } - auto * insert = orig_ast->as(); - - if (insert && insert->data) + // INSERT queries may have the inserted data in the query text + // that follow the query itself, e.g. "insert into t format CSV 1;2". + // They need special handling. First of all, here we find where the + // inserted data ends. In multy-query mode, it is delimited by a + // newline. + // The VALUES format needs even more handling -- we also allow the + // data to be delimited by semicolon. This case is handled later by + // the format parser itself. + auto * insert_ast = parsed_query->as(); + if (insert_ast && insert_ast->data) { - pos = find_first_symbols<'\n'>(insert->data, end); - insert->end = pos; + this_query_end = find_first_symbols<'\n'>(insert_ast->data, all_queries_end); + insert_ast->end = this_query_end; + query_to_send = all_queries_text.substr( + this_query_begin - all_queries_text.data(), + insert_ast->data - this_query_begin); + } + else + { + query_to_send = all_queries_text.substr( + this_query_begin - all_queries_text.data(), + this_query_end - this_query_begin); } - String str = text.substr(begin - text.data(), pos - begin); + // full_query is the query + inline INSERT data. + full_query = all_queries_text.substr( + this_query_begin - all_queries_text.data(), + this_query_end - this_query_begin); - begin = pos; - while (isWhitespaceASCII(*begin) || *begin == ';') - ++begin; - - TestHint test_hint(test_mode, str); + // Look for the hint in the text of query + insert data, if any. + // e.g. insert into t format CSV 'a' -- { serverError 123 }. + TestHint test_hint(test_mode, full_query); expected_client_error = test_hint.clientError(); expected_server_error = test_hint.serverError(); try { - auto ast_to_process = orig_ast; - if (insert && insert->data) + processParsedSingleQuery(); + + if (insert_ast && insert_ast->data) { - ast_to_process = nullptr; - processTextAsSingleQuery(str); - } - else - { - parsed_query = ast_to_process; - full_query = str; - query_to_send = str; - processParsedSingleQuery(); + // For VALUES format: use the end of inline data as reported + // by the format parser (it is saved in sendData()). This + // allows us to handle queries like: + // insert into t values (1); select 1 + //, where the inline data is delimited by semicolon and not + // by a newline. + this_query_end = parsed_query->as()->end; } } catch (...) @@ -983,7 +1036,7 @@ private: last_exception_received_from_server = std::make_unique(getCurrentExceptionMessage(true), getCurrentExceptionCode()); actual_client_error = last_exception_received_from_server->code(); if (!ignore_error && (!actual_client_error || actual_client_error != expected_client_error)) - std::cerr << "Error on processing query: " << str << std::endl << last_exception_received_from_server->message(); + std::cerr << "Error on processing query: " << full_query << std::endl << last_exception_received_from_server->message(); received_exception_from_server = true; } @@ -997,6 +1050,8 @@ private: else return false; } + + this_query_begin = this_query_end; } return true; @@ -1407,7 +1462,7 @@ private: void sendData(Block & sample, const ColumnsDescription & columns_description) { /// If INSERT data must be sent. - const auto * parsed_insert_query = parsed_query->as(); + auto * parsed_insert_query = parsed_query->as(); if (!parsed_insert_query) return; @@ -1416,6 +1471,9 @@ private: /// Send data contained in the query. ReadBufferFromMemory data_in(parsed_insert_query->data, parsed_insert_query->end - parsed_insert_query->data); sendDataFrom(data_in, sample, columns_description); + // Remember where the data ended. We use this info later to determine + // where the next query begins. + parsed_insert_query->end = data_in.buffer().begin() + data_in.count(); } else if (!is_interactive) { diff --git a/tests/queries/0_stateless/00306_insert_values_and_expressions.reference b/tests/queries/0_stateless/00306_insert_values_and_expressions.reference index 960773dc489..e80a28accf4 100644 --- a/tests/queries/0_stateless/00306_insert_values_and_expressions.reference +++ b/tests/queries/0_stateless/00306_insert_values_and_expressions.reference @@ -2,3 +2,5 @@ 2 Hello, world 00000000-0000-0000-0000-000000000000 2016-01-02 2016-01-02 03:04:00 [0,1] 3 hello, world! ab41bdd6-5cd4-11e7-907b-a6006ad3dba0 2016-01-03 2016-01-02 03:00:00 [] 4 World ab41bdd6-5cd4-11e7-907b-a6006ad3dba0 2016-01-04 2016-12-11 10:09:08 [3,2,1] +11111 +1 diff --git a/tests/queries/0_stateless/00306_insert_values_and_expressions.sql b/tests/queries/0_stateless/00306_insert_values_and_expressions.sql index a57e9e69fe6..10a1415f287 100644 --- a/tests/queries/0_stateless/00306_insert_values_and_expressions.sql +++ b/tests/queries/0_stateless/00306_insert_values_and_expressions.sql @@ -5,3 +5,12 @@ INSERT INTO insert VALUES (1, 'Hello', 'ab41bdd6-5cd4-11e7-907b-a6006ad3dba0', ' SELECT * FROM insert ORDER BY i; DROP TABLE insert; + +-- Test the case where the VALUES are delimited by semicolon and a query follows +-- w/o newline. With most formats the query in the same line would be ignored or +-- lead to an error, but VALUES are an exception and support semicolon delimiter, +-- in addition to the newline. +create table if not exists t_306 (a int) engine Memory; +insert into t_306 values (1); select 11111; +select * from t_306; +drop table if exists t_306; From 405a6fb08fa22a9e063dd5e48e7ee6060f718749 Mon Sep 17 00:00:00 2001 From: hexiaoting <“hewenting_ict@163.com”> Date: Mon, 17 Aug 2020 18:20:23 +0800 Subject: [PATCH 011/298] New feature: LineAsString format. #13630 --- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatFactory.h | 1 + .../Impl/LineAsStringRowInputFormat.cpp | 101 ++++++++++++++++++ .../Formats/Impl/LineAsStringRowInputFormat.h | 31 ++++++ src/Processors/ya.make | 1 + 5 files changed, 135 insertions(+) create mode 100644 src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/LineAsStringRowInputFormat.h diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 5256ab2b321..f996e3d8cf2 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -365,6 +365,7 @@ FormatFactory::FormatFactory() registerInputFormatProcessorMsgPack(*this); registerOutputFormatProcessorMsgPack(*this); registerInputFormatProcessorJSONAsString(*this); + registerInputFormatProcessorLineAsString(*this); registerFileSegmentationEngineTabSeparated(*this); registerFileSegmentationEngineCSV(*this); diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index ea4004c191f..610cf8105b8 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -210,5 +210,6 @@ void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory); void registerInputFormatProcessorCapnProto(FormatFactory & factory); void registerInputFormatProcessorRegexp(FormatFactory & factory); void registerInputFormatProcessorJSONAsString(FormatFactory & factory); +void registerInputFormatProcessorLineAsString(FormatFactory & factory); } diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp new file mode 100644 index 00000000000..a28b3903724 --- /dev/null +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -0,0 +1,101 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INCORRECT_DATA; +} + +LineAsStringRowInputFormat::LineAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) : + IRowInputFormat(header_, in_, std::move(params_)), buf(in) +{ + if (header_.columns() > 1 || header_.getDataTypes()[0]->getTypeId() != TypeIndex::String) + { + throw Exception("This input format is only suitable for tables with a single column of type String.", ErrorCodes::LOGICAL_ERROR); + } +} + +void LineAsStringRowInputFormat::resetParser() +{ + IRowInputFormat::resetParser(); + buf.reset(); +} + +void LineAsStringRowInputFormat::readLineObject(IColumn & column) +{ + PeekableReadBufferCheckpoint checkpoint{buf}; + size_t balance = 0; + + if (*buf.position() != '"') + throw Exception("Line object must begin with '\"'.", ErrorCodes::INCORRECT_DATA); + + ++buf.position(); + ++balance; + + char * pos; + + while (balance) + { + if (buf.eof()) + throw Exception("Unexpected end of file while parsing Line object.", ErrorCodes::INCORRECT_DATA); + + pos = find_last_symbols_or_null<'"', '\\'>(buf.position(), buf.buffer().end()); + buf.position() = pos; + if (buf.position() == buf.buffer().end()) + continue; + else if (*buf.position() == '"') + { + --balance; + ++buf.position(); + } + else if (*buf.position() == '\\') + { + ++buf.position(); + if (!buf.eof()) + { + ++buf.position(); + } + } + + } + buf.makeContinuousMemoryFromCheckpointToPos(); + char * end = buf.position(); + buf.rollbackToCheckpoint(); + column.insertData(buf.position(), end - buf.position()); + buf.position() = end; +} + +bool LineAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) +{ + skipWhitespaceIfAny(buf); + + if (!buf.eof()) + readLineObject(*columns[0]); + + skipWhitespaceIfAny(buf); + if (!buf.eof() && *buf.position() == ',') + ++buf.position(); + skipWhitespaceIfAny(buf); + + return !buf.eof(); +} + +void registerInputFormatProcessorLineAsString(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("LineAsString", []( + ReadBuffer & buf, + const Block & sample, + const RowInputFormatParams & params, + const FormatSettings &) + { + return std::make_shared(sample, buf, params); + }); +} + +} diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h new file mode 100644 index 00000000000..a31dce1cc4a --- /dev/null +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ReadBuffer; + +/// This format parses a sequence of Line objects separated by newlines, spaces and/or comma. +/// Each Line object is parsed as a whole to string. +/// This format can only parse a table with single field of type String. + +class LineAsStringRowInputFormat : public IRowInputFormat +{ +public: + LineAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_); + + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + String getName() const override { return "LineAsStringRowInputFormat"; } + void resetParser() override; + +private: + void readLineObject(IColumn & column); + + PeekableReadBuffer buf; +}; + +} diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 4c25ad5bf3f..081b1d5ba1f 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -23,6 +23,7 @@ SRCS( Formats/Impl/ConstantExpressionTemplate.cpp Formats/Impl/CSVRowInputFormat.cpp Formats/Impl/CSVRowOutputFormat.cpp + Formats/Impl/LineAsStringRowInputFormat.cpp Formats/Impl/JSONAsStringRowInputFormat.cpp Formats/Impl/JSONCompactEachRowRowInputFormat.cpp Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp From 0451d5729323b7f46d79336fea4f0982bb1662ae Mon Sep 17 00:00:00 2001 From: hexiaoting <“hewenting_ict@163.com”> Date: Tue, 18 Aug 2020 10:35:08 +0800 Subject: [PATCH 012/298] Add new feature: LineAsString Format --- src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp | 6 ++++++ .../0_stateless/01460_line_as_string_format.reference | 1 + tests/queries/0_stateless/01460_line_as_string_format.sql | 5 +++++ 3 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/01460_line_as_string_format.reference create mode 100644 tests/queries/0_stateless/01460_line_as_string_format.sql diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index a28b3903724..36844fa700b 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -32,6 +32,12 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) PeekableReadBufferCheckpoint checkpoint{buf}; size_t balance = 0; + if (*buf.position() == ';') { + ++buf.position(); + if(buf.eof()) + return; + } + if (*buf.position() != '"') throw Exception("Line object must begin with '\"'.", ErrorCodes::INCORRECT_DATA); diff --git a/tests/queries/0_stateless/01460_line_as_string_format.reference b/tests/queries/0_stateless/01460_line_as_string_format.reference new file mode 100644 index 00000000000..989f8ac0292 --- /dev/null +++ b/tests/queries/0_stateless/01460_line_as_string_format.reference @@ -0,0 +1 @@ +"I love apple","I love banana","I love pear" diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sql b/tests/queries/0_stateless/01460_line_as_string_format.sql new file mode 100644 index 00000000000..e5518a828d0 --- /dev/null +++ b/tests/queries/0_stateless/01460_line_as_string_format.sql @@ -0,0 +1,5 @@ +DROP TABLE IF EXISTS line_as_string; +CREATE TABLE line_as_string (field String) ENGINE = Memory; +INSERT INTO line_as_string FORMAT LineAsString "I love apple","I love banana","I love pear"; +SELECT * FROM line_as_string; +DROP TABLE line_as_string; From e9be2f14ea8ac45f11c7c65b6c36646b64a5b390 Mon Sep 17 00:00:00 2001 From: hexiaoting <“hewenting_ict@163.com”> Date: Wed, 19 Aug 2020 11:50:43 +0800 Subject: [PATCH 013/298] fix implementation for \n separated lines --- .../Impl/LineAsStringRowInputFormat.cpp | 45 ++++++------------- .../01460_line_as_string_format.reference | 7 ++- .../01460_line_as_string_format.sh | 19 ++++++++ .../01460_line_as_string_format.sql | 5 --- 4 files changed, 38 insertions(+), 38 deletions(-) create mode 100755 tests/queries/0_stateless/01460_line_as_string_format.sh delete mode 100644 tests/queries/0_stateless/01460_line_as_string_format.sql diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index 36844fa700b..27bc71d764d 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -30,35 +30,22 @@ void LineAsStringRowInputFormat::resetParser() void LineAsStringRowInputFormat::readLineObject(IColumn & column) { PeekableReadBufferCheckpoint checkpoint{buf}; - size_t balance = 0; - - if (*buf.position() == ';') { - ++buf.position(); - if(buf.eof()) - return; - } - - if (*buf.position() != '"') - throw Exception("Line object must begin with '\"'.", ErrorCodes::INCORRECT_DATA); - - ++buf.position(); - ++balance; + bool newline = true; + bool over = false; char * pos; - while (balance) + while (newline) { - if (buf.eof()) - throw Exception("Unexpected end of file while parsing Line object.", ErrorCodes::INCORRECT_DATA); - - pos = find_last_symbols_or_null<'"', '\\'>(buf.position(), buf.buffer().end()); + pos = find_first_symbols<'\n', '\\'>(buf.position(), buf.buffer().end()); buf.position() = pos; - if (buf.position() == buf.buffer().end()) - continue; - else if (*buf.position() == '"') + if (buf.position() == buf.buffer().end()) { + over = true; + break; + } + else if (*buf.position() == '\n') { - --balance; - ++buf.position(); + newline = false; } else if (*buf.position() == '\\') { @@ -70,25 +57,19 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) } } + buf.makeContinuousMemoryFromCheckpointToPos(); - char * end = buf.position(); + char * end = over ? buf.position(): ++buf.position(); buf.rollbackToCheckpoint(); - column.insertData(buf.position(), end - buf.position()); + column.insertData(buf.position(), end - (over ? 0 : 1) - buf.position()); buf.position() = end; } bool LineAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { - skipWhitespaceIfAny(buf); - if (!buf.eof()) readLineObject(*columns[0]); - skipWhitespaceIfAny(buf); - if (!buf.eof() && *buf.position() == ',') - ++buf.position(); - skipWhitespaceIfAny(buf); - return !buf.eof(); } diff --git a/tests/queries/0_stateless/01460_line_as_string_format.reference b/tests/queries/0_stateless/01460_line_as_string_format.reference index 989f8ac0292..dec67eb2e0a 100644 --- a/tests/queries/0_stateless/01460_line_as_string_format.reference +++ b/tests/queries/0_stateless/01460_line_as_string_format.reference @@ -1 +1,6 @@ -"I love apple","I love banana","I love pear" +"id" : 1, +"date" : "01.01.2020", +"string" : "123{{{\\"\\\\", +"array" : [1, 2, 3], + +Finally implement this new feature. diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sh b/tests/queries/0_stateless/01460_line_as_string_format.sh new file mode 100755 index 00000000000..a985bc207a8 --- /dev/null +++ b/tests/queries/0_stateless/01460_line_as_string_format.sh @@ -0,0 +1,19 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS line_as_string"; + +$CLICKHOUSE_CLIENT --query="CREATE TABLE line_as_string(field String) ENGINE = Memory"; + +echo '"id" : 1, +"date" : "01.01.2020", +"string" : "123{{{\"\\", +"array" : [1, 2, 3], + +Finally implement this new feature.' | $CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string FORMAT LineAsString"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM line_as_string"; +$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string" + diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sql b/tests/queries/0_stateless/01460_line_as_string_format.sql deleted file mode 100644 index e5518a828d0..00000000000 --- a/tests/queries/0_stateless/01460_line_as_string_format.sql +++ /dev/null @@ -1,5 +0,0 @@ -DROP TABLE IF EXISTS line_as_string; -CREATE TABLE line_as_string (field String) ENGINE = Memory; -INSERT INTO line_as_string FORMAT LineAsString "I love apple","I love banana","I love pear"; -SELECT * FROM line_as_string; -DROP TABLE line_as_string; From 4331158d3051437f44c7fa1271e4673272cf8cac Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Aug 2020 16:09:23 +0300 Subject: [PATCH 014/298] merge with master --- src/Disks/DiskDecorator.cpp | 15 +++++++++++++++ src/Disks/DiskDecorator.h | 3 +++ .../MergeTree/MergeTreeDataPartWriterInMemory.cpp | 2 +- .../MergeTree/MergeTreeDataPartWriterInMemory.h | 2 +- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 11 ++++++----- 5 files changed, 26 insertions(+), 7 deletions(-) diff --git a/src/Disks/DiskDecorator.cpp b/src/Disks/DiskDecorator.cpp index e55534e347f..7f2ea58d7cf 100644 --- a/src/Disks/DiskDecorator.cpp +++ b/src/Disks/DiskDecorator.cpp @@ -165,4 +165,19 @@ void DiskDecorator::truncateFile(const String & path, size_t size) delegate->truncateFile(path, size); } +int DiskDecorator::open(const String & path, mode_t mode) const +{ + return delegate->open(path, mode); +} + +void DiskDecorator::close(int fd) const +{ + delegate->close(fd); +} + +void DiskDecorator::sync(int fd) const +{ + delegate->sync(fd); +} + } diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index 71bb100c576..f1ddfff4952 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -42,6 +42,9 @@ public: void setReadOnly(const String & path) override; void createHardLink(const String & src_path, const String & dst_path) override; void truncateFile(const String & path, size_t size) override; + int open(const String & path, mode_t mode) const override; + void close(int fd) const override; + void sync(int fd) const override; const String getType() const override { return delegate->getType(); } protected: diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp index a7486158737..f0738a1130a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp @@ -70,7 +70,7 @@ void MergeTreeDataPartWriterInMemory::calculateAndSerializePrimaryIndex(const Bl } } -void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) +void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool) { /// If part is empty we still need to initialize block by empty columns. if (!part_in_memory->block) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h index 92e4228a90d..6e59cdd08a9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h @@ -18,7 +18,7 @@ public: void write(const Block & block, const IColumn::Permutation * permutation, const Block & primary_key_block, const Block & skip_indexes_block) override; - void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; void calculateAndSerializePrimaryIndex(const Block & primary_index_block) override; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index f3a72657be5..b05b970da3b 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -251,6 +251,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->minmax_idx = std::move(minmax_idx); new_data_part->is_temp = true; + std::optional sync_guard; if (new_data_part->isStoredOnDisk()) { /// The name could be non-unique in case of stale files from previous runs. @@ -262,12 +263,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->volume->getDisk()->removeRecursive(full_path); } - const auto disk = new_data_part->volume->getDisk(); - disk->createDirectories(full_path); + const auto disk = new_data_part->volume->getDisk(); + disk->createDirectories(full_path); - std::optional sync_guard; - if (data.getSettings()->fsync_part_directory) - sync_guard.emplace(disk, full_path); + if (data.getSettings()->fsync_part_directory) + sync_guard.emplace(disk, full_path); + } /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) From 4834bed35b251fee8f53d72fa7c2650fd473a195 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 Aug 2020 14:35:53 +0300 Subject: [PATCH 015/298] Add recompression TTL parser --- src/Parsers/ASTTTLElement.cpp | 5 +++++ src/Parsers/ASTTTLElement.h | 2 ++ src/Parsers/ExpressionElementParsers.cpp | 19 +++++++++++++++++++ src/Storages/TTLDescription.cpp | 15 ++++++++++++++- src/Storages/TTLDescription.h | 5 +++++ src/Storages/TTLMode.h | 3 ++- 6 files changed, 47 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ASTTTLElement.cpp b/src/Parsers/ASTTTLElement.cpp index 1635d376d30..f37631769b8 100644 --- a/src/Parsers/ASTTTLElement.cpp +++ b/src/Parsers/ASTTTLElement.cpp @@ -57,6 +57,11 @@ void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & st } } } + else if (mode == TTLMode::RECOMPRESS) + { + settings.ostr << " RECOMPRESS "; + recompression_codec->formatImpl(settings, state, frame); + } else if (mode == TTLMode::DELETE) { /// It would be better to output "DELETE" here but that will break compatibility with earlier versions. diff --git a/src/Parsers/ASTTTLElement.h b/src/Parsers/ASTTTLElement.h index 7ee1f4795ff..aadd019b59c 100644 --- a/src/Parsers/ASTTTLElement.h +++ b/src/Parsers/ASTTTLElement.h @@ -20,6 +20,8 @@ public: ASTs group_by_key; std::vector> group_by_aggregations; + ASTPtr recompression_codec; + ASTTTLElement(TTLMode mode_, DataDestinationType destination_type_, const String & destination_name_) : mode(mode_) , destination_type(destination_type_) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index e24bb9c4129..67c3737f6f0 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1506,6 +1506,8 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_where("WHERE"); ParserKeyword s_group_by("GROUP BY"); ParserKeyword s_set("SET"); + ParserKeyword s_recompress("RECOMPRESS"); + ParserKeyword s_codec("CODEC"); ParserToken s_comma(TokenType::Comma); ParserToken s_eq(TokenType::Equals); @@ -1513,6 +1515,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserStringLiteral parser_string_literal; ParserExpression parser_exp; ParserExpressionList parser_expression_list(false); + ParserCodec parser_codec; ASTPtr ttl_expr; if (!parser_exp.parse(pos, ttl_expr, expected)) @@ -1536,6 +1539,10 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { mode = TTLMode::GROUP_BY; } + else if (s_recompress.ignore(pos)) + { + mode = TTLMode::RECOMPRESS; + } else { s_delete.ignore(pos); @@ -1544,6 +1551,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr where_expr; ASTPtr ast_group_by_key; + ASTPtr recompression_codec; std::vector> group_by_aggregations; if (mode == TTLMode::MOVE) @@ -1587,6 +1595,14 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!parser_exp.parse(pos, where_expr, expected)) return false; } + else if (mode == TTLMode::RECOMPRESS) + { + if (!s_codec.ignore(pos)) + return false; + + if (!parser_codec.parse(pos, recompression_codec, expected)) + return false; + } auto ttl_element = std::make_shared(mode, destination_type, destination_name); ttl_element->setTTL(std::move(ttl_expr)); @@ -1599,6 +1615,9 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ttl_element->group_by_aggregations = std::move(group_by_aggregations); } + if (mode == TTLMode::RECOMPRESS) + ttl_element->recompression_codec = recompression_codec; + node = ttl_element; return true; } diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 4c9da095278..656baf39971 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -263,6 +264,12 @@ TTLDescription TTLDescription::getTTLFromAST( result.aggregate_descriptions.push_back(descr); } } + else if (ttl_element->mode == TTLMode::RECOMPRESS) + { + result.recompression_codec = + CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( + ttl_element->recompression_codec, {}, !context.getSettingsRef().allow_suspicious_codecs); + } } checkTTLExpression(result.expression, result.result_column); @@ -311,15 +318,21 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( for (const auto & ttl_element_ptr : definition_ast->children) { auto ttl = TTLDescription::getTTLFromAST(ttl_element_ptr, columns, context, primary_key); - if (ttl.destination_type == DataDestinationType::DELETE) + if (ttl.mode == TTLMode::DELETE) { if (seen_delete_ttl) throw Exception("More than one DELETE TTL expression is not allowed", ErrorCodes::BAD_TTL_EXPRESSION); result.rows_ttl = ttl; seen_delete_ttl = true; } + else if (ttl.mode == TTLMode::RECOMPRESS) + { + result.recompression_ttl.emplace_back(std::move(ttl)); + } else + { result.move_ttl.emplace_back(std::move(ttl)); + } } return result; } diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index f7769fd42e9..4b0d4370a70 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -75,6 +75,9 @@ struct TTLDescription /// Name of destination disk or volume String destination_name; + /// Codec name which will be used to recompress data + ASTPtr recompression_codec; + /// Parse TTL structure from definition. Able to parse both column and table /// TTLs. static TTLDescription getTTLFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context, const KeyDescription & primary_key); @@ -102,6 +105,8 @@ struct TTLTableDescription /// Moving data TTL (to other disks or volumes) TTLDescriptions move_ttl; + TTLDescriptions recompression_ttl; + TTLTableDescription() = default; TTLTableDescription(const TTLTableDescription & other); TTLTableDescription & operator=(const TTLTableDescription & other); diff --git a/src/Storages/TTLMode.h b/src/Storages/TTLMode.h index 0681f10fc17..7f5fe0315c6 100644 --- a/src/Storages/TTLMode.h +++ b/src/Storages/TTLMode.h @@ -8,7 +8,8 @@ enum class TTLMode { DELETE, MOVE, - GROUP_BY + GROUP_BY, + RECOMPRESS, }; } From 42c210fcba41d2e0ba657b38048278667ebf5963 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 Aug 2020 15:12:51 +0300 Subject: [PATCH 016/298] Recompress TTLs in memory metadata --- src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h | 5 ++++- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 5 +++++ src/Storages/StorageInMemoryMetadata.cpp | 10 ++++++++++ src/Storages/StorageInMemoryMetadata.h | 4 ++++ 4 files changed, 23 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h index 209d7181b66..d2e131d5650 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h @@ -38,13 +38,16 @@ struct MergeTreeDataPartTTLInfos MergeTreeDataPartTTLInfo table_ttl; /// `part_min_ttl` and `part_max_ttl` are TTLs which are used for selecting parts - /// to merge in order to remove expired rows. + /// to merge in order to remove expired rows. time_t part_min_ttl = 0; time_t part_max_ttl = 0; /// Order is important as it would be serialized and hashed for checksums std::map moves_ttl; + /// Order is important as it would be serialized and hashed for checksums + std::map recompression_ttl; + void read(ReadBuffer & in); void write(WriteBuffer & out) const; void update(const MergeTreeDataPartTTLInfos & other_infos); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 445a02b06f0..23569a13b85 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -234,6 +234,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa for (const auto & ttl_entry : move_ttl_entries) updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); + const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); + for (const auto & ttl_entry : recompression_ttl_entries) + updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); + + NamesAndTypesList columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr)); VolumePtr volume = data.getStoragePolicy()->getVolume(0); diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index b7f4565a55a..f611c1ec95d 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -157,6 +157,16 @@ bool StorageInMemoryMetadata::hasAnyMoveTTL() const return !table_ttl.move_ttl.empty(); } +TTLDescriptions StorageInMemoryMetadata::getRecompressionTTLs() const +{ + return table_ttl.recompression_ttl; +} + +bool StorageInMemoryMetadata::hasAnyRecompressionTTL() const +{ + return !table_ttl.recompression_ttl.empty(); +} + ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet & updated_columns) const { if (updated_columns.empty()) diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 4c78d72a9d1..3656edf71f4 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -114,6 +114,10 @@ struct StorageInMemoryMetadata TTLDescriptions getMoveTTLs() const; bool hasAnyMoveTTL() const; + // Just wrapper for table TTLs, return info about recompression ttl + TTLDescriptions getRecompressionTTLs() const; + bool hasAnyRecompressionTTL() const; + /// Returns columns, which will be needed to calculate dependencies (skip /// indices, TTL expressions) if we update @updated_columns set of columns. ColumnDependencies getColumnDependencies(const NameSet & updated_columns) const; From adc18f4d3f8915a1ad505ebc67cace8d98d81c04 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 Aug 2020 16:29:31 +0300 Subject: [PATCH 017/298] Write with recompression TTL --- .../MergeTree/MergeTreeDataPartTTLInfo.cpp | 34 +++++++++++++++++++ .../MergeTree/MergeTreeDataWriter.cpp | 9 +++-- 2 files changed, 38 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp index 37d036fc6fc..94a2b4269ef 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp @@ -67,6 +67,18 @@ void MergeTreeDataPartTTLInfos::read(ReadBuffer & in) moves_ttl.emplace(expression, ttl_info); } } + if (json.has("recompression")) + { + const JSON & moves = json["recompression"]; + for (auto move : moves) // NOLINT + { + MergeTreeDataPartTTLInfo ttl_info; + ttl_info.min = move["min"].getUInt(); + ttl_info.max = move["max"].getUInt(); + String expression = move["expression"].getString(); + recompression_ttl.emplace(expression, ttl_info); + } + } } @@ -122,6 +134,28 @@ void MergeTreeDataPartTTLInfos::write(WriteBuffer & out) const } writeString("]", out); } + if (!recompression_ttl.empty()) + { + if (!moves_ttl.empty() || !columns_ttl.empty() || table_ttl.min) + writeString(",", out); + + writeString(R"("recompression":[)", out); + for (auto it = recompression_ttl.begin(); it != recompression_ttl.end(); ++it) + { + if (it != recompression_ttl.begin()) + writeString(",", out); + + writeString(R"({"expression":)", out); + writeString(doubleQuoteString(it->first), out); + writeString(R"(,"min":)", out); + writeIntText(it->second.min, out); + writeString(R"(,"max":)", out); + writeIntText(it->second.max, out); + writeString("}", out); + } + writeString("]", out); + + } writeString("}", out); } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 23569a13b85..92bf5345d5a 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -234,11 +234,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa for (const auto & ttl_entry : move_ttl_entries) updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); - const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); - for (const auto & ttl_entry : recompression_ttl_entries) - updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); - - NamesAndTypesList columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr)); VolumePtr volume = data.getStoragePolicy()->getVolume(0); @@ -303,6 +298,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs()) updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true); + const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); + for (const auto & ttl_entry : recompression_ttl_entries) + updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.recompression_ttl[ttl_entry.result_column], block, false); + new_data_part->ttl_infos.update(move_ttl_infos); /// This effectively chooses minimal compression method: From b20a0bc254e769e66093e7c2a2a574b252b5a698 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 Aug 2020 16:42:42 +0300 Subject: [PATCH 018/298] Add recompression flag in ReplicatedEntry --- src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp | 10 ++++++++++ src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h | 1 + 2 files changed, 11 insertions(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index af6d980ad98..a4fc600d1b3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -36,6 +36,9 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const out << s << '\n'; out << "into\n" << new_part_name; out << "\ndeduplicate: " << deduplicate; + /// For backward compatibility write only if enabled + if (recompress) + out << "\nrecompress: " << recompress; break; case DROP_RANGE: @@ -149,7 +152,14 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) } in >> new_part_name; if (format_version >= 4) + { in >> "\ndeduplicate: " >> deduplicate; + in >> "\n"; + if (in.eof()) + trailing_newline_found = true; + else if (checkString("recompress\n", in)) + in >> recompress; + } } else if (type_str == "drop" || type_str == "detach") { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index ae5fad0b83c..62599c2c3a7 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -79,6 +79,7 @@ struct ReplicatedMergeTreeLogEntryData Strings source_parts; bool deduplicate = false; /// Do deduplicate on merge + bool recompress = false; /// Recompress parts on merge String column_name; String index_name; From 46f833b7df64f77d361f78d629d3075f83945ebb Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 Aug 2020 22:50:42 +0300 Subject: [PATCH 019/298] Some changes --- src/Storages/MergeTree/MergeTreeData.cpp | 30 +++++++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 3 ++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 1 + .../MergeTree/MergeTreeDataMergerMutator.h | 1 + .../MergeTree/MergeTreeDataPartTTLInfo.cpp | 7 +++++ 5 files changed, 42 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b6a495161f5..b721cf4afbf 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3055,6 +3055,36 @@ MergeTreeData::selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & tt return max_max_ttl ? *best_entry_it : std::optional(); } + +CompressionCodecPtr MergeTreeData::getCompressionCodecForPart(size_t part_size_compressed, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t current_time) const +{ + + time_t max_max_ttl = 0; + TTLDescriptions::const_iterator best_entry_it; + auto metadata_snapshot = getInMemoryMetadataPtr(); + + const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); + for (auto ttl_entry_it = recompression_ttl_entries.begin(); ttl_entry_it != recompression_ttl_entries.end(); ++ttl_entry_it) + { + auto ttl_info_it = ttl_infos.recompression_ttl.find(ttl_entry_it->result_column); + /// Prefer TTL rule which went into action last. + if (ttl_info_it != ttl_infos.recompression_ttl.end() + && ttl_info_it->second.max <= current_time + && max_max_ttl <= ttl_info_it->second.max) + { + best_entry_it = ttl_entry_it; + max_max_ttl = ttl_info_it->second.max; + } + } + + if (max_max_ttl) + return CompressionCodecFactory::instance().get(best_entry_it->recompression_codec, {}); + + return global_context.chooseCompressionCodec( + part_size_compressed, + static_cast(part_size_compressed) / getTotalActiveSizeInBytes()); +} + MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affordable_states) const { DataParts res; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e088a1c098b..ab115927e1e 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -669,6 +669,9 @@ public: std::optional selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const; + + CompressionCodecPtr getCompressionCodecForPart(size_t part_size_compressed, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t current_time) const; + /// Limiting parallel sends per one table, used in DataPartsExchange std::atomic_uint current_table_sends {0}; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 673ad02bfb6..8cece66dafb 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -284,6 +284,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( current_time, data_settings->merge_with_ttl_timeout, data_settings->ttl_only_drop_parts); + parts_to_merge = merge_selector.select(partitions, max_total_size_to_merge); } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index d5798fe3582..e13711f8064 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -249,6 +249,7 @@ private: /// Stores the next TTL merge due time for each partition (used only by TTLMergeSelector) TTLMergeSelector::PartitionIdToTTLs next_ttl_merge_times_by_partition; + /// Performing TTL merges independently for each partition guarantees that /// there is only a limited number of TTL merges and no partition stores data, that is too stale }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp index 94a2b4269ef..4b0a8bdfa9e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp @@ -16,6 +16,12 @@ void MergeTreeDataPartTTLInfos::update(const MergeTreeDataPartTTLInfos & other_i updatePartMinMaxTTL(ttl_info.min, ttl_info.max); } + for (const auto & [name, ttl_info] : other_infos.recompression_ttl) + { + recompression_ttl[name].update(ttl_info); + updatePartMinMaxTTL(ttl_info.min, ttl_info.max); + } + for (const auto & [expression, ttl_info] : other_infos.moves_ttl) { moves_ttl[expression].update(ttl_info); @@ -77,6 +83,7 @@ void MergeTreeDataPartTTLInfos::read(ReadBuffer & in) ttl_info.max = move["max"].getUInt(); String expression = move["expression"].getString(); recompression_ttl.emplace(expression, ttl_info); + updatePartMinMaxTTL(ttl_info.min, ttl_info.max); } } } From 25140b9bd5b6421b84ef8586827cc49b9d015e7b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Sep 2020 04:39:36 +0300 Subject: [PATCH 020/298] fsync MergeTree format file --- src/Storages/MergeTree/MergeTreeData.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bbefba70c58..bc668659b6a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -220,6 +220,8 @@ MergeTreeData::MergeTreeData( format_version = min_format_version; auto buf = version_file.second->writeFile(version_file.first); writeIntText(format_version.toUnderType(), *buf); + if (global_context.getSettingsRef().fsync_metadata) + buf->sync(); } else { From 927eb32e882d070ff5ff5446d5b9e0071e2c6f9d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Sep 2020 04:46:40 +0300 Subject: [PATCH 021/298] add test for durability (draft) --- utils/durability-test/create.sql | 1 + utils/durability-test/durability-test.sh | 154 +++++++++++++++++++++++ utils/durability-test/insert.sql | 1 + utils/durability-test/install.sh | 3 + utils/durability-test/sshd_config | 8 ++ utils/durability-test/startup.exp | 23 ++++ 6 files changed, 190 insertions(+) create mode 100644 utils/durability-test/create.sql create mode 100644 utils/durability-test/durability-test.sh create mode 100644 utils/durability-test/insert.sql create mode 100644 utils/durability-test/install.sh create mode 100644 utils/durability-test/sshd_config create mode 100755 utils/durability-test/startup.exp diff --git a/utils/durability-test/create.sql b/utils/durability-test/create.sql new file mode 100644 index 00000000000..1ec394100e2 --- /dev/null +++ b/utils/durability-test/create.sql @@ -0,0 +1 @@ +CREATE TABLE test (a Int, s String) ENGINE = MergeTree ORDER BY a; diff --git a/utils/durability-test/durability-test.sh b/utils/durability-test/durability-test.sh new file mode 100644 index 00000000000..1f47c900f49 --- /dev/null +++ b/utils/durability-test/durability-test.sh @@ -0,0 +1,154 @@ +#!/bin/bash + +URL=http://cloud-images.ubuntu.com/bionic/current +IMAGE=bionic-server-cloudimg-amd64.img +SSH_PORT=11022 +CLICKHOUSE_PORT=9090 +PASSWORD=root + +TABLE_NAME=$1 +CREATE_QUERY=$2 +INSERT_QUERY=$3 + +if [[ -z $TABLE_NAME || -z $CREATE_QUERY || -z $INSERT_QUERY ]]; then + echo "Required 3 arguments: table name, file with create query, file with insert query" + exit 1 +fi + +function run() +{ + sshpass -p $PASSWORD ssh -p $SSH_PORT root@localhost "$1" +} + +function copy() +{ + sshpass -p $PASSWORD scp -r -P $SSH_PORT $1 root@localhost:$2 +} + +function wait_vm_for_start() +{ + echo "Waiting until VM started..." + started=0 + for i in {0..100}; do + run "exit" + if [ $? -eq 0 ]; then + started=1 + break + fi + sleep 1s + done + + if ((started == 0)); then + echo "Can't start or connect to VM." + exit 1 + fi + + echo "Started VM" +} + +function wait_clickhouse_for_start() +{ + echo "Waiting until ClickHouse started..." + started=0 + for i in {0..15}; do + run "clickhouse client --query 'select 1'" + if [ $? -eq 0 ]; then + started=1 + break + fi + sleep 1s + done + + if ((started == 0)); then + echo "Can't start ClickHouse." + fi + + echo "Started ClickHouse" +} + +echo "Downloading image" +curl -O $URL/$IMAGE + +qemu-img resize $IMAGE +10G +virt-customize -a $IMAGE --root-password password:$PASSWORD +virt-copy-in -a $IMAGE sshd_config /etc/ssh + +echo "Starting VM" + +chmod +x ./startup.exp +./startup.exp > qemu.log 2>&1 & + +wait_vm_for_start + +echo "Preparing VM" + +# Resize partition +run "growpart /dev/sda 1 && resize2fs /dev/sda1" + +if [[ -z $CLICKHOUSE_BINARY ]]; then + CLICKHOUSE_BINARY=/usr/bin/clickhouse +fi + +if [[ -z $CLICKHOUSE_CONFIG_DIR ]]; then + CLICKHOUSE_CONFIG_DIR=/etc/clickhouse-server +fi + +echo "Using ClickHouse binary: " $CLICKHOUSE_BINARY +echo "Using ClickHouse config from: " $CLICKHOUSE_CONFIG_DIR + +copy $CLICKHOUSE_BINARY /usr/bin +copy $CLICKHOUSE_CONFIG_DIR /etc +run "mv /etc/$CLICKHOUSE_CONFIG_DIR /etc/clickhouse-server" + +echo "Prepared VM" +echo "Starting ClickHouse" + +run "clickhouse server --config-file=/etc/clickhouse-server/config.xml > clickhouse-server.log 2>&1" & + +wait_clickhouse_for_start + +echo "Started ClickHouse" + +query=`cat $CREATE_QUERY` +echo "Executing query:" $query +run "clickhouse client --query '$query'" + +query=`cat $INSERT_QUERY` +echo "Will run in a loop query: " $query +run "clickhouse benchmark <<< '$query'" & +echo "Running queries" + +pid=`pidof qemu-system-x86_64` +sec=$(( (RANDOM % 3) + 25 )) + +ms=$(( RANDOM % 1000 )) + +echo "Will kill VM in $sec.$ms sec" + +sleep $sec.$ms +kill -9 $pid + +echo "Restarting" + +./startup.exp > qemu.log 2>&1 & +wait_vm_for_start + +run "rm -r *data/system" +run "clickhouse server --config-file=/etc/clickhouse-server/config.xml > clickhouse-server.log 2>&1" & +wait_clickhouse_for_start + +result=`run "grep $TABLE_NAME clickhouse-server.log | grep 'Caught exception while loading metadata'"` +if [[ -n $result ]]; then + echo "FAIL. Can't attach table:" + echo $result + exit 1 +fi + +result=`run "grep $TABLE_NAME clickhouse-server.log | grep 'Considering to remove broken part'"` +if [[ -n $result ]]; then + echo "FAIL. Have broken parts:" + echo $result + exit 1 +fi + +echo OK diff --git a/utils/durability-test/insert.sql b/utils/durability-test/insert.sql new file mode 100644 index 00000000000..8982ad47228 --- /dev/null +++ b/utils/durability-test/insert.sql @@ -0,0 +1 @@ +INSERT INTO test SELECT number, toString(number) FROM numbers(10) diff --git a/utils/durability-test/install.sh b/utils/durability-test/install.sh new file mode 100644 index 00000000000..526cde6743f --- /dev/null +++ b/utils/durability-test/install.sh @@ -0,0 +1,3 @@ +#!/bin/bash + +apt update && apt install qemu-kvm qemu virt-manager virt-viewer libguestfs-tools sshpass expect diff --git a/utils/durability-test/sshd_config b/utils/durability-test/sshd_config new file mode 100644 index 00000000000..6ed06d3d8ad --- /dev/null +++ b/utils/durability-test/sshd_config @@ -0,0 +1,8 @@ +PermitRootLogin yes +PasswordAuthentication yes +ChallengeResponseAuthentication no +UsePAM yes +X11Forwarding yes +PrintMotd no +AcceptEnv LANG LC_* +Subsystem sftp /usr/lib/openssh/sftp-server diff --git a/utils/durability-test/startup.exp b/utils/durability-test/startup.exp new file mode 100755 index 00000000000..540cfc0e4b8 --- /dev/null +++ b/utils/durability-test/startup.exp @@ -0,0 +1,23 @@ +#!/usr/bin/expect -f + +# Wait enough (forever) until a long-time boot +set timeout -1 + +spawn qemu-system-x86_64 \ + -hda bionic-server-cloudimg-amd64.img \ + -cpu qemu64,+ssse3,+sse4.1,+sse4.2,+popcnt -smp 8 \ + -net nic -net user,hostfwd=tcp::11022-:22 \ + -m 4096 -nographic + +expect "login: " +send "root\n" + +expect "Password: " +send "root\n" + +# Without it ssh is not working on guest machine for some reason +expect "# " +send "dhclient && ssh-keygen -A && systemctl restart sshd.service\n" + +# Wait forever +expect "########" From 602535396d0ac58c1885ef1d1d3e7c085335f059 Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 16:36:27 +0800 Subject: [PATCH 022/298] Refactor, move function declarations --- src/Formats/FormatFactory.cpp | 100 +++++++++++++++++++++++++++++----- src/Formats/FormatFactory.h | 69 ----------------------- 2 files changed, 85 insertions(+), 84 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 935d31d6541..871098e00c0 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -323,13 +323,85 @@ void FormatFactory::registerFileSegmentationEngine(const String & name, FileSegm target = std::move(file_segmentation_engine); } +/// File Segmentation Engines for parallel reading + +void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); +void registerFileSegmentationEngineCSV(FormatFactory & factory); +void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); +void registerFileSegmentationEngineRegexp(FormatFactory & factory); +void registerFileSegmentationEngineJSONAsString(FormatFactory & factory); + +/// Formats for both input/output. + +void registerInputFormatNative(FormatFactory & factory); +void registerOutputFormatNative(FormatFactory & factory); + +void registerInputFormatProcessorNative(FormatFactory & factory); +void registerOutputFormatProcessorNative(FormatFactory & factory); +void registerInputFormatProcessorRowBinary(FormatFactory & factory); +void registerOutputFormatProcessorRowBinary(FormatFactory & factory); +void registerInputFormatProcessorTabSeparated(FormatFactory & factory); +void registerOutputFormatProcessorTabSeparated(FormatFactory & factory); +void registerInputFormatProcessorValues(FormatFactory & factory); +void registerOutputFormatProcessorValues(FormatFactory & factory); +void registerInputFormatProcessorCSV(FormatFactory & factory); +void registerOutputFormatProcessorCSV(FormatFactory & factory); +void registerInputFormatProcessorTSKV(FormatFactory & factory); +void registerOutputFormatProcessorTSKV(FormatFactory & factory); +void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); +void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); +void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); +void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); +void registerInputFormatProcessorProtobuf(FormatFactory & factory); +void registerOutputFormatProcessorProtobuf(FormatFactory & factory); +void registerInputFormatProcessorTemplate(FormatFactory & factory); +void registerOutputFormatProcessorTemplate(FormatFactory & factory); +void registerInputFormatProcessorMsgPack(FormatFactory & factory); +void registerOutputFormatProcessorMsgPack(FormatFactory & factory); +void registerInputFormatProcessorORC(FormatFactory & factory); +void registerOutputFormatProcessorORC(FormatFactory & factory); +void registerInputFormatProcessorParquet(FormatFactory & factory); +void registerOutputFormatProcessorParquet(FormatFactory & factory); +void registerInputFormatProcessorArrow(FormatFactory & factory); +void registerOutputFormatProcessorArrow(FormatFactory & factory); +void registerInputFormatProcessorAvro(FormatFactory & factory); +void registerOutputFormatProcessorAvro(FormatFactory & factory); + +/// Output only (presentational) formats. + +void registerOutputFormatNull(FormatFactory & factory); + +void registerOutputFormatProcessorPretty(FormatFactory & factory); +void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory); +void registerOutputFormatProcessorPrettySpace(FormatFactory & factory); +void registerOutputFormatProcessorVertical(FormatFactory & factory); +void registerOutputFormatProcessorJSON(FormatFactory & factory); +void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); +void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); +void registerOutputFormatProcessorXML(FormatFactory & factory); +void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); +void registerOutputFormatProcessorNull(FormatFactory & factory); +void registerOutputFormatProcessorMySQLWire(FormatFactory & factory); +void registerOutputFormatProcessorMarkdown(FormatFactory & factory); +void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory); + +/// Input only formats. + +void registerInputFormatProcessorRegexp(FormatFactory & factory); +void registerInputFormatProcessorJSONAsString(FormatFactory & factory); +void registerInputFormatProcessorCapnProto(FormatFactory & factory); + FormatFactory::FormatFactory() { + registerFileSegmentationEngineTabSeparated(*this); + registerFileSegmentationEngineCSV(*this); + registerFileSegmentationEngineJSONEachRow(*this); + registerFileSegmentationEngineRegexp(*this); + registerFileSegmentationEngineJSONAsString(*this); + registerInputFormatNative(*this); registerOutputFormatNative(*this); - registerOutputFormatProcessorJSONEachRowWithProgress(*this); - registerInputFormatProcessorNative(*this); registerOutputFormatProcessorNative(*this); registerInputFormatProcessorRowBinary(*this); @@ -348,8 +420,11 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSONCompactEachRow(*this); registerInputFormatProcessorProtobuf(*this); registerOutputFormatProcessorProtobuf(*this); + registerInputFormatProcessorTemplate(*this); + registerOutputFormatProcessorTemplate(*this); + registerInputFormatProcessorMsgPack(*this); + registerOutputFormatProcessorMsgPack(*this); #if !defined(ARCADIA_BUILD) - registerInputFormatProcessorCapnProto(*this); registerInputFormatProcessorORC(*this); registerOutputFormatProcessorORC(*this); registerInputFormatProcessorParquet(*this); @@ -359,18 +434,6 @@ FormatFactory::FormatFactory() registerInputFormatProcessorAvro(*this); registerOutputFormatProcessorAvro(*this); #endif - registerInputFormatProcessorTemplate(*this); - registerOutputFormatProcessorTemplate(*this); - registerInputFormatProcessorRegexp(*this); - registerInputFormatProcessorMsgPack(*this); - registerOutputFormatProcessorMsgPack(*this); - registerInputFormatProcessorJSONAsString(*this); - - registerFileSegmentationEngineTabSeparated(*this); - registerFileSegmentationEngineCSV(*this); - registerFileSegmentationEngineJSONEachRow(*this); - registerFileSegmentationEngineRegexp(*this); - registerFileSegmentationEngineJSONAsString(*this); registerOutputFormatNull(*this); @@ -380,12 +443,19 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorVertical(*this); registerOutputFormatProcessorJSON(*this); registerOutputFormatProcessorJSONCompact(*this); + registerOutputFormatProcessorJSONEachRowWithProgress(*this); registerOutputFormatProcessorXML(*this); registerOutputFormatProcessorODBCDriver2(*this); registerOutputFormatProcessorNull(*this); registerOutputFormatProcessorMySQLWire(*this); registerOutputFormatProcessorMarkdown(*this); registerOutputFormatProcessorPostgreSQLWire(*this); + + registerInputFormatProcessorRegexp(*this); + registerInputFormatProcessorJSONAsString(*this); +#if !defined(ARCADIA_BUILD) + registerInputFormatProcessorCapnProto(*this); +#endif } FormatFactory & FormatFactory::instance() diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index f0d2b7826a0..54bff1eefc6 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -141,73 +141,4 @@ private: const Creators & getCreators(const String & name) const; }; -/// Formats for both input/output. - -void registerInputFormatNative(FormatFactory & factory); -void registerOutputFormatNative(FormatFactory & factory); - -void registerInputFormatProcessorNative(FormatFactory & factory); -void registerOutputFormatProcessorNative(FormatFactory & factory); -void registerInputFormatProcessorRowBinary(FormatFactory & factory); -void registerOutputFormatProcessorRowBinary(FormatFactory & factory); -void registerInputFormatProcessorTabSeparated(FormatFactory & factory); -void registerOutputFormatProcessorTabSeparated(FormatFactory & factory); -void registerInputFormatProcessorValues(FormatFactory & factory); -void registerOutputFormatProcessorValues(FormatFactory & factory); -void registerInputFormatProcessorCSV(FormatFactory & factory); -void registerOutputFormatProcessorCSV(FormatFactory & factory); -void registerInputFormatProcessorTSKV(FormatFactory & factory); -void registerOutputFormatProcessorTSKV(FormatFactory & factory); -void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); -void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); -void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); -void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); -void registerInputFormatProcessorParquet(FormatFactory & factory); -void registerOutputFormatProcessorParquet(FormatFactory & factory); -void registerInputFormatProcessorArrow(FormatFactory & factory); -void registerOutputFormatProcessorArrow(FormatFactory & factory); -void registerInputFormatProcessorProtobuf(FormatFactory & factory); -void registerOutputFormatProcessorProtobuf(FormatFactory & factory); -void registerInputFormatProcessorAvro(FormatFactory & factory); -void registerOutputFormatProcessorAvro(FormatFactory & factory); -void registerInputFormatProcessorTemplate(FormatFactory & factory); -void registerOutputFormatProcessorTemplate(FormatFactory & factory); -void registerInputFormatProcessorMsgPack(FormatFactory & factory); -void registerOutputFormatProcessorMsgPack(FormatFactory & factory); -void registerInputFormatProcessorORC(FormatFactory & factory); -void registerOutputFormatProcessorORC(FormatFactory & factory); - - -/// File Segmentation Engines for parallel reading - -void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); -void registerFileSegmentationEngineCSV(FormatFactory & factory); -void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); -void registerFileSegmentationEngineRegexp(FormatFactory & factory); -void registerFileSegmentationEngineJSONAsString(FormatFactory & factory); - -/// Output only (presentational) formats. - -void registerOutputFormatNull(FormatFactory & factory); - -void registerOutputFormatProcessorPretty(FormatFactory & factory); -void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory); -void registerOutputFormatProcessorPrettySpace(FormatFactory & factory); -void registerOutputFormatProcessorPrettyASCII(FormatFactory & factory); -void registerOutputFormatProcessorVertical(FormatFactory & factory); -void registerOutputFormatProcessorJSON(FormatFactory & factory); -void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); -void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); -void registerOutputFormatProcessorXML(FormatFactory & factory); -void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); -void registerOutputFormatProcessorNull(FormatFactory & factory); -void registerOutputFormatProcessorMySQLWire(FormatFactory & factory); -void registerOutputFormatProcessorMarkdown(FormatFactory & factory); -void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory); - -/// Input only formats. -void registerInputFormatProcessorCapnProto(FormatFactory & factory); -void registerInputFormatProcessorRegexp(FormatFactory & factory); -void registerInputFormatProcessorJSONAsString(FormatFactory & factory); - } From 86fa185bb6fbf8e1e6bc6044a7f4e523477e84db Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 17:06:28 +0800 Subject: [PATCH 023/298] Add JSONStrings formats --- src/Formats/FormatFactory.cpp | 6 + .../Impl/JSONStringsEachRowRowInputFormat.cpp | 245 ++++++++++++++++++ .../Impl/JSONStringsEachRowRowInputFormat.h | 54 ++++ .../JSONStringsEachRowRowOutputFormat.cpp | 117 +++++++++ .../Impl/JSONStringsEachRowRowOutputFormat.h | 45 ++++ .../Impl/JSONStringsRowOutputFormat.cpp | 93 +++++++ .../Formats/Impl/JSONStringsRowOutputFormat.h | 43 +++ src/Processors/ya.make | 3 + 8 files changed, 606 insertions(+) create mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h create mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp create mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h create mode 100644 src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp create mode 100644 src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 871098e00c0..cb378fbea96 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -352,6 +352,8 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); +void registerInputFormatProcessorJSONStringsEachRow(FormatFactory & factory); +void registerOutputFormatProcessorJSONStringsEachRow(FormatFactory & factory); void registerInputFormatProcessorProtobuf(FormatFactory & factory); void registerOutputFormatProcessorProtobuf(FormatFactory & factory); void registerInputFormatProcessorTemplate(FormatFactory & factory); @@ -378,6 +380,7 @@ void registerOutputFormatProcessorVertical(FormatFactory & factory); void registerOutputFormatProcessorJSON(FormatFactory & factory); void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); +void registerOutputFormatProcessorJSONStrings(FormatFactory & factory); void registerOutputFormatProcessorXML(FormatFactory & factory); void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); void registerOutputFormatProcessorNull(FormatFactory & factory); @@ -418,6 +421,8 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSONEachRow(*this); registerInputFormatProcessorJSONCompactEachRow(*this); registerOutputFormatProcessorJSONCompactEachRow(*this); + registerInputFormatProcessorJSONStringsEachRow(*this); + registerOutputFormatProcessorJSONStringsEachRow(*this); registerInputFormatProcessorProtobuf(*this); registerOutputFormatProcessorProtobuf(*this); registerInputFormatProcessorTemplate(*this); @@ -444,6 +449,7 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSON(*this); registerOutputFormatProcessorJSONCompact(*this); registerOutputFormatProcessorJSONEachRowWithProgress(*this); + registerOutputFormatProcessorJSONStrings(*this); registerOutputFormatProcessorXML(*this); registerOutputFormatProcessorODBCDriver2(*this); registerOutputFormatProcessorNull(*this); diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp new file mode 100644 index 00000000000..fff44a204fb --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp @@ -0,0 +1,245 @@ +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; + extern const int CANNOT_READ_ALL_DATA; +} + + +JSONStringsEachRowRowInputFormat::JSONStringsEachRowRowInputFormat(ReadBuffer & in_, + const Block & header_, + Params params_, + const FormatSettings & format_settings_, + bool with_names_) + : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_) +{ + const auto & sample = getPort().getHeader(); + size_t num_columns = sample.columns(); + + data_types.resize(num_columns); + column_indexes_by_names.reserve(num_columns); + + for (size_t i = 0; i < num_columns; ++i) + { + const auto & column_info = sample.getByPosition(i); + + data_types[i] = column_info.type; + column_indexes_by_names.emplace(column_info.name, i); + } +} + +void JSONStringsEachRowRowInputFormat::resetParser() +{ + IRowInputFormat::resetParser(); + column_indexes_for_input_fields.clear(); + not_seen_columns.clear(); +} + +void JSONStringsEachRowRowInputFormat::readPrefix() +{ + /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. + skipBOMIfExists(in); + + if (with_names) + { + size_t num_columns = getPort().getHeader().columns(); + read_columns.assign(num_columns, false); + + assertChar('[', in); + do + { + skipWhitespaceIfAny(in); + String column_name; + readJSONString(column_name, in); + addInputColumn(column_name); + skipWhitespaceIfAny(in); + } + while (checkChar(',', in)); + assertChar(']', in); + skipEndOfLine(); + + /// Type checking + assertChar('[', in); + for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i) + { + skipWhitespaceIfAny(in); + String data_type; + readJSONString(data_type, in); + + if (column_indexes_for_input_fields[i] && + data_types[*column_indexes_for_input_fields[i]]->getName() != data_type) + { + throw Exception( + "Type of '" + getPort().getHeader().getByPosition(*column_indexes_for_input_fields[i]).name + + "' must be " + data_types[*column_indexes_for_input_fields[i]]->getName() + + ", not " + data_type, + ErrorCodes::INCORRECT_DATA + ); + } + + if (i != column_indexes_for_input_fields.size() - 1) + assertChar(',', in); + skipWhitespaceIfAny(in); + } + assertChar(']', in); + } + else + { + size_t num_columns = getPort().getHeader().columns(); + read_columns.assign(num_columns, true); + column_indexes_for_input_fields.resize(num_columns); + + for (size_t i = 0; i < num_columns; ++i) + { + column_indexes_for_input_fields[i] = i; + } + } + + for (size_t i = 0; i < read_columns.size(); ++i) + { + if (!read_columns[i]) + { + not_seen_columns.emplace_back(i); + } + } +} + +void JSONStringsEachRowRowInputFormat::addInputColumn(const String & column_name) +{ + names_of_columns.emplace_back(column_name); + + const auto column_it = column_indexes_by_names.find(column_name); + if (column_it == column_indexes_by_names.end()) + { + if (format_settings.skip_unknown_fields) + { + column_indexes_for_input_fields.push_back(std::nullopt); + return; + } + + throw Exception( + "Unknown field found in JSONStringsEachRow header: '" + column_name + "' " + + "at position " + std::to_string(column_indexes_for_input_fields.size()) + + "\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed", + ErrorCodes::INCORRECT_DATA + ); + } + + const auto column_index = column_it->second; + + if (read_columns[column_index]) + throw Exception("Duplicate field found while parsing JSONStringsEachRow header: " + column_name, ErrorCodes::INCORRECT_DATA); + + read_columns[column_index] = true; + column_indexes_for_input_fields.emplace_back(column_index); +} + +bool JSONStringsEachRowRowInputFormat::readRow(DB::MutableColumns &columns, DB::RowReadExtension &ext) +{ + skipEndOfLine(); + + if (in.eof()) + return false; + + size_t num_columns = columns.size(); + + read_columns.assign(num_columns, false); + + assertChar('[', in); + for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column) + { + const auto & table_column = column_indexes_for_input_fields[file_column]; + if (table_column) + { + readField(*table_column, columns); + } + else + { + skipJSONField(in, StringRef(names_of_columns[file_column])); + } + + skipWhitespaceIfAny(in); + if (in.eof()) + throw Exception("Unexpected end of stream while parsing JSONStringsEachRow format", ErrorCodes::CANNOT_READ_ALL_DATA); + if (file_column + 1 != column_indexes_for_input_fields.size()) + { + assertChar(',', in); + skipWhitespaceIfAny(in); + } + } + assertChar(']', in); + + for (const auto & name : not_seen_columns) + columns[name]->insertDefault(); + + ext.read_columns = read_columns; + return true; +} + +void JSONStringsEachRowRowInputFormat::skipEndOfLine() +{ + skipWhitespaceIfAny(in); + if (!in.eof() && (*in.position() == ',' || *in.position() == ';')) + ++in.position(); + + skipWhitespaceIfAny(in); +} + +void JSONStringsEachRowRowInputFormat::readField(size_t index, MutableColumns & columns) +{ + try + { + read_columns[index] = true; + const auto & type = data_types[index]; + + String str; + readJSONString(str, in); + + ReadBufferFromString buf(str); + + type->deserializeAsWholeText(*columns[index], buf, format_settings); + } + catch (Exception & e) + { + e.addMessage("(while read the value of key " + getPort().getHeader().getByPosition(index).name + ")"); + throw; + } +} + +void JSONStringsEachRowRowInputFormat::syncAfterError() +{ + skipToUnescapedNextLineOrEOF(in); +} + +void registerInputFormatProcessorJSONStringsEachRow(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("JSONStringsEachRow", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, false); + }); + + factory.registerInputFormatProcessor("JSONStringsEachRowWithNamesAndTypes", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, true); + }); +} + +} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h new file mode 100644 index 00000000000..ec0a0f7bad9 --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h @@ -0,0 +1,54 @@ +#pragma once + +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class ReadBuffer; + +/** A stream for reading data in JSONStringsEachRow and JSONStringsEachRowWithNamesAndTypes formats +*/ +class JSONStringsEachRowRowInputFormat : public IRowInputFormat +{ +public: + JSONStringsEachRowRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_, bool with_names_); + + String getName() const override { return "JSONStringsEachRowRowInputFormat"; } + + + void readPrefix() override; + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + bool allowSyncAfterError() const override { return true; } + void syncAfterError() override; + void resetParser() override; + +private: + void addInputColumn(const String & column_name); + void skipEndOfLine(); + void readField(size_t index, MutableColumns & columns); + + const FormatSettings format_settings; + + using IndexesMap = std::unordered_map; + IndexesMap column_indexes_by_names; + + using OptionalIndexes = std::vector>; + OptionalIndexes column_indexes_for_input_fields; + + DataTypes data_types; + std::vector read_columns; + std::vector not_seen_columns; + + /// This is for the correct exceptions in skipping unknown fields. + std::vector names_of_columns; + + bool with_names; +}; + +} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp new file mode 100644 index 00000000000..75007ea236e --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp @@ -0,0 +1,117 @@ +#include +#include +#include +#include + + +namespace DB +{ + + +JSONStringsEachRowRowOutputFormat::JSONStringsEachRowRowOutputFormat(WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool with_names_) + : IRowOutputFormat(header_, out_, callback), settings(settings_), with_names(with_names_) +{ + const auto & sample = getPort(PortKind::Main).getHeader(); + NamesAndTypesList columns(sample.getNamesAndTypesList()); + fields.assign(columns.begin(), columns.end()); +} + + +void JSONStringsEachRowRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), out, settings); +} + + +void JSONStringsEachRowRowOutputFormat::writeFieldDelimiter() +{ + writeCString(", ", out); +} + + +void JSONStringsEachRowRowOutputFormat::writeRowStartDelimiter() +{ + writeChar('[', out); +} + + +void JSONStringsEachRowRowOutputFormat::writeRowEndDelimiter() +{ + writeCString("]\n", out); +} + +void JSONStringsEachRowRowOutputFormat::writeTotals(const Columns & columns, size_t row_num) +{ + writeChar('\n', out); + size_t num_columns = columns.size(); + writeChar('[', out); + for (size_t i = 0; i < num_columns; ++i) + { + if (i != 0) + JSONStringsEachRowRowOutputFormat::writeFieldDelimiter(); + + JSONStringsEachRowRowOutputFormat::writeField(*columns[i], *types[i], row_num); + } + writeCString("]\n", out); +} + +void JSONStringsEachRowRowOutputFormat::writePrefix() +{ + if (with_names) + { + writeChar('[', out); + for (size_t i = 0; i < fields.size(); ++i) + { + writeChar('\"', out); + writeString(fields[i].name, out); + writeChar('\"', out); + if (i != fields.size() - 1) + writeCString(", ", out); + } + writeCString("]\n[", out); + for (size_t i = 0; i < fields.size(); ++i) + { + writeJSONString(fields[i].type->getName(), out, settings); + if (i != fields.size() - 1) + writeCString(", ", out); + } + writeCString("]\n", out); + } +} + +void JSONStringsEachRowRowOutputFormat::consumeTotals(DB::Chunk chunk) +{ + if (with_names) + IRowOutputFormat::consumeTotals(std::move(chunk)); +} + +void registerOutputFormatProcessorJSONStringsEachRow(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("JSONStringsEachRow", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONStringsEachRowWithNamesAndTypes", []( + WriteBuffer &buf, + const Block &sample, + FormatFactory::WriteCallback callback, + const FormatSettings &format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); + }); +} + + +} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h new file mode 100644 index 00000000000..1d43a333da1 --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +/** The stream for outputting data in JSON format, by object per line. + * Does not validate UTF-8. + */ +class JSONStringsEachRowRowOutputFormat : public IRowOutputFormat +{ +public: + JSONStringsEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_, bool with_names); + + String getName() const override { return "JSONStringsEachRowRowOutputFormat"; } + + void writePrefix() override; + + void writeBeforeTotals() override {} + void writeTotals(const Columns & columns, size_t row_num) override; + void writeAfterTotals() override {} + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + +protected: + void consumeTotals(Chunk) override; + /// No extremes. + void consumeExtremes(Chunk) override {} + +private: + FormatSettings settings; + + NamesAndTypes fields; + + bool with_names; +}; +} diff --git a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp new file mode 100644 index 00000000000..6ccb315f73f --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp @@ -0,0 +1,93 @@ +#include +#include + +#include + + +namespace DB +{ + +JSONStringsRowOutputFormat::JSONStringsRowOutputFormat( + WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_) + : JSONRowOutputFormat(out_, header, callback, settings_) +{ +} + + +void JSONStringsRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), *ostr, settings); + ++field_number; +} + + +void JSONStringsRowOutputFormat::writeFieldDelimiter() +{ + writeCString(", ", *ostr); +} + +void JSONStringsRowOutputFormat::writeTotalsFieldDelimiter() +{ + writeCString(",", *ostr); +} + + +void JSONStringsRowOutputFormat::writeRowStartDelimiter() +{ + writeCString("\t\t[", *ostr); +} + + +void JSONStringsRowOutputFormat::writeRowEndDelimiter() +{ + writeChar(']', *ostr); + field_number = 0; + ++row_count; +} + +void JSONStringsRowOutputFormat::writeBeforeTotals() +{ + writeCString(",\n", *ostr); + writeChar('\n', *ostr); + writeCString("\t\"totals\": [", *ostr); +} + +void JSONStringsRowOutputFormat::writeAfterTotals() +{ + writeChar(']', *ostr); +} + +void JSONStringsRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num) +{ + writeCString("\t\t\"", *ostr); + writeCString(title, *ostr); + writeCString("\": [", *ostr); + + size_t extremes_columns = columns.size(); + for (size_t i = 0; i < extremes_columns; ++i) + { + if (i != 0) + writeTotalsFieldDelimiter(); + + writeField(*columns[i], *types[i], row_num); + } + + writeChar(']', *ostr); +} + +void registerOutputFormatProcessorJSONStrings(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("JSONStrings", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings); + }); +} + +} diff --git a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h new file mode 100644 index 00000000000..b221bc9ee36 --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h @@ -0,0 +1,43 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +struct FormatSettings; + +/** The stream for outputting data in the JSONStrings format. + */ +class JSONStringsRowOutputFormat : public JSONRowOutputFormat +{ +public: + JSONStringsRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_); + + String getName() const override { return "JSONStringsRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + + void writeBeforeTotals() override; + void writeAfterTotals() override; + +protected: + void writeExtremesElement(const char * title, const Columns & columns, size_t row_num) override; + + void writeTotalsField(const IColumn & column, const IDataType & type, size_t row_num) override + { + return writeField(column, type, row_num); + } + + void writeTotalsFieldDelimiter() override; + +}; + +} diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 30de38fedbd..27893674859 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -31,6 +31,9 @@ SRCS( Formats/Impl/JSONEachRowRowOutputFormat.cpp Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp Formats/Impl/JSONRowOutputFormat.cpp + Formats/Impl/JSONStringsEachRowRowInputFormat.cpp + Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp + Formats/Impl/JSONStringsRowOutputFormat.cpp Formats/Impl/MarkdownRowOutputFormat.cpp Formats/Impl/MsgPackRowInputFormat.cpp Formats/Impl/MsgPackRowOutputFormat.cpp From c25a99aaf58108651149930db5ef86e1313120c4 Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 17:47:37 +0800 Subject: [PATCH 024/298] Add tests --- .../01446_JSONStringsEachRow.reference | 47 ++++++++++++++ .../0_stateless/01446_JSONStringsEachRow.sql | 63 +++++++++++++++++++ .../0_stateless/01447_JSONStrings.reference | 43 +++++++++++++ .../queries/0_stateless/01447_JSONStrings.sql | 8 +++ 4 files changed, 161 insertions(+) create mode 100644 tests/queries/0_stateless/01446_JSONStringsEachRow.reference create mode 100644 tests/queries/0_stateless/01446_JSONStringsEachRow.sql create mode 100644 tests/queries/0_stateless/01447_JSONStrings.reference create mode 100644 tests/queries/0_stateless/01447_JSONStrings.sql diff --git a/tests/queries/0_stateless/01446_JSONStringsEachRow.reference b/tests/queries/0_stateless/01446_JSONStringsEachRow.reference new file mode 100644 index 00000000000..0b05f050b29 --- /dev/null +++ b/tests/queries/0_stateless/01446_JSONStringsEachRow.reference @@ -0,0 +1,47 @@ +1 +["1", "a"] +["2", "b"] +["3", "c"] +2 +["a", "1"] +["b", "1"] +["c", "1"] +3 +["value", "name"] +["UInt8", "String"] +["1", "a"] +["2", "b"] +["3", "c"] +4 +["name", "c"] +["String", "UInt64"] +["a", "1"] +["b", "1"] +["c", "1"] + +["", "3"] +5 +["first", "1", "2", "0"] +["second", "2", "0", "6"] +6 +["first", "1", "2", "0"] +["second", "2", "0", "6"] +7 +["16", "[15,16,17]", "['first','second','third']"] +8 +["first", "1", "2", "0"] +["second", "2", "0", "6"] +9 +["first", "1", "2", "0"] +["second", "2", "0", "6"] +10 +["first", "1", "16", "8"] +["second", "2", "32", "8"] +11 +["v1", "v2", "v3", "v4"] +["String", "UInt8", "UInt16", "UInt8"] +["", "2", "3", "1"] +12 +["v1", "n.id", "n.name"] +["UInt8", "Array(UInt8)", "Array(String)"] +["16", "[15,16,17]", "['first','second','third']"] diff --git a/tests/queries/0_stateless/01446_JSONStringsEachRow.sql b/tests/queries/0_stateless/01446_JSONStringsEachRow.sql new file mode 100644 index 00000000000..f461b217fe4 --- /dev/null +++ b/tests/queries/0_stateless/01446_JSONStringsEachRow.sql @@ -0,0 +1,63 @@ +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; +SELECT 1; +/* Check JSONStringsEachRow Output */ +CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value; +INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c'); +SELECT * FROM test_table FORMAT JSONStringsEachRow; +SELECT 2; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRow; +SELECT 3; +/* Check JSONStringsEachRowWithNamesAndTypes Output */ +SELECT * FROM test_table FORMAT JSONStringsEachRowWithNamesAndTypes; +SELECT 4; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRowWithNamesAndTypes; +DROP TABLE IF EXISTS test_table; +SELECT 5; +/* Check JSONStringsEachRow Input */ +CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; +INSERT INTO test_table FORMAT JSONStringsEachRow ["first", "1", "2", "NULL"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 6; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONStringsEachRow ["first", "1", "2", "ᴺᵁᴸᴸ"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 7; +/* Check Nested */ +CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; +INSERT INTO test_table_2 FORMAT JSONStringsEachRow ["16", "[15, 16, 17]", "['first', 'second', 'third']"]; +SELECT * FROM test_table_2 FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table_2; +SELECT 8; +/* Check JSONStringsEachRowWithNamesAndTypes Output */ +SET input_format_null_as_default = 0; +INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"]["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 9; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +SELECT 10; +/* Check Header */ +TRUNCATE TABLE test_table; +SET input_format_skip_unknown_fields = 1; +INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", "1", "32"]["second", "2", "64"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +SELECT 11; +TRUNCATE TABLE test_table; +INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"]["1", "2", "3"] +SELECT * FROM test_table FORMAT JSONStringsEachRowWithNamesAndTypes; +SELECT 12; +/* Check Nested */ +INSERT INTO test_table_2 FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"]["16", "[15, 16, 17]", "['first', 'second', 'third']"]; +SELECT * FROM test_table_2 FORMAT JSONStringsEachRowWithNamesAndTypes; + +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; diff --git a/tests/queries/0_stateless/01447_JSONStrings.reference b/tests/queries/0_stateless/01447_JSONStrings.reference new file mode 100644 index 00000000000..58af593dc77 --- /dev/null +++ b/tests/queries/0_stateless/01447_JSONStrings.reference @@ -0,0 +1,43 @@ +{ + "meta": + [ + { + "name": "1", + "type": "UInt8" + }, + { + "name": "'a'", + "type": "String" + }, + { + "name": "[1, 2, 3]", + "type": "Array(UInt8)" + }, + { + "name": "tuple(1, 'a')", + "type": "Tuple(UInt8, String)" + }, + { + "name": "NULL", + "type": "Nullable(Nothing)" + }, + { + "name": "nan", + "type": "Float64" + } + ], + + "data": + [ + ["1", "a", "[1,2,3]", "(1,'a')", "ᴺᵁᴸᴸ", "nan"] + ], + + "rows": 1, + + "statistics": + { + "elapsed": 0.00068988, + "rows_read": 1, + "bytes_read": 1 + } +} diff --git a/tests/queries/0_stateless/01447_JSONStrings.sql b/tests/queries/0_stateless/01447_JSONStrings.sql new file mode 100644 index 00000000000..7d89f0f5087 --- /dev/null +++ b/tests/queries/0_stateless/01447_JSONStrings.sql @@ -0,0 +1,8 @@ +SELECT + 1, + 'a', + [1, 2, 3], + (1, 'a'), + null, + nan +FORMAT JSONStrings; From 6d37c9d2a68ca3f6ae39c9bc0bb99424d7fc236e Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 17:48:03 +0800 Subject: [PATCH 025/298] Update docs about formats --- docs/en/interfaces/formats.md | 26 +++++++++++++++++++------- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 9d3965b4a9c..9c7c2dda8dc 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -26,7 +26,10 @@ The supported formats are: | [VerticalRaw](#verticalraw) | ✗ | ✔ | | [JSON](#json) | ✗ | ✔ | | [JSONCompact](#jsoncompact) | ✗ | ✔ | +| [JSONStrings](#jsonstrings) | ✗ | ✔ | | [JSONEachRow](#jsoneachrow) | ✔ | ✔ | +| [JSONCompactEachRow](#jsoncompacteachrow) | ✔ | ✔ | +| [JSONStringsEachRow](#jsonstringseachrow) | ✔ | ✔ | | [TSKV](#tskv) | ✔ | ✔ | | [Pretty](#pretty) | ✗ | ✔ | | [PrettyCompact](#prettycompact) | ✗ | ✔ | @@ -470,7 +473,7 @@ See also the [JSONEachRow](#jsoneachrow) format. ## JSONCompact {#jsoncompact} -Differs from JSON only in that data rows are output in arrays, not in objects. +Differs from JSON only in that data rows are output in arrays of any element type, not in objects. Example: @@ -514,17 +517,26 @@ Example: This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). See also the `JSONEachRow` format. -## JSONEachRow {#jsoneachrow} +## JSONStrings {#jsonstrings} -When using this format, ClickHouse outputs rows as separated, newline-delimited JSON objects, but the data as a whole is not valid JSON. +Differs from JSON and JSONCompact only in that data rows are output in arrays of strings. + +This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). +See also the `JSONEachRow` format. + +## JSONEachRow {#jsoneachrow} +## JSONCompactEachRow {#jsoncompacteachrow} +## JSONStringsEachRow {#jsonstringseachrow} + +When using these formats, ClickHouse outputs rows as separated, newline-delimited JSON values, but the data as a whole is not valid JSON. ``` json -{"SearchPhrase":"curtain designs","count()":"1064"} -{"SearchPhrase":"baku","count()":"1000"} -{"SearchPhrase":"","count()":"8267016"} +{"some_int":42,"some_str":"hello","some_tuple":[1,"a"]} // JSONEachRow +[42,"hello",[1,"a"]] // JSONCompactEachRow +["42","hello","(2,'a')"] // JSONStringsEachRow ``` -When inserting the data, you should provide a separate JSON object for each row. +When inserting the data, you should provide a separate JSON value for each row. ### Inserting Data {#inserting-data} From babd3beec09054d1dc4b1b8a35cf30da013f05af Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 17:48:19 +0800 Subject: [PATCH 026/298] Fix nullable data parsing --- src/DataTypes/DataTypeNullable.cpp | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 847047850fd..3318196b951 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -308,7 +308,10 @@ ReturnType DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer const DataTypePtr & nested_data_type) { return safeDeserialize(column, *nested_data_type, - [&istr] { return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); }, + [&istr] + { + return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); + }, [&nested_data_type, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsTextQuoted(nested, istr, settings); }); } @@ -316,7 +319,11 @@ ReturnType DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer void DataTypeNullable::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { safeDeserialize(column, *nested_data_type, - [&istr] { return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); }, + [&istr] + { + return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr) + || checkStringByFirstCharacterAndAssertTheRest("ᴺᵁᴸᴸ", istr); + }, [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsWholeText(nested, istr, settings); }); } From f0dc5a30853ff4b40d0097f07068fbb0f49eb714 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 1 Sep 2020 13:49:53 +0300 Subject: [PATCH 027/298] First working test --- src/DataStreams/TTLBlockInputStream.cpp | 28 +++++++++++++ src/DataStreams/TTLBlockInputStream.h | 2 + src/Interpreters/MutationsInterpreter.cpp | 8 ++++ src/Storages/MergeTree/MergeTreeData.cpp | 10 +++++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 41 +++++++++++++------ .../MergeTree/MergeTreeDataWriter.cpp | 7 +++- .../MergeTree/registerStorageMergeTree.cpp | 3 ++ src/Storages/StorageInMemoryMetadata.cpp | 5 ++- src/Storages/TTLDescription.cpp | 11 +++++ .../01465_ttl_recompression.reference | 10 +++++ .../0_stateless/01465_ttl_recompression.sql | 32 +++++++++++++++ 11 files changed, 142 insertions(+), 15 deletions(-) create mode 100644 tests/queries/0_stateless/01465_ttl_recompression.reference create mode 100644 tests/queries/0_stateless/01465_ttl_recompression.sql diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 6d80e784c03..e1586286678 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -134,6 +134,7 @@ Block TTLBlockInputStream::readImpl() removeValuesWithExpiredColumnTTL(block); updateMovesTTL(block); + updateRecompressionTTL(block); return block; } @@ -395,6 +396,33 @@ void TTLBlockInputStream::updateMovesTTL(Block & block) block.erase(column); } + +void TTLBlockInputStream::updateRecompressionTTL(Block & block) +{ + std::vector columns_to_remove; + for (const auto & ttl_entry : metadata_snapshot->getRecompressionTTLs()) + { + auto & new_ttl_info = new_ttl_infos.recompression_ttl[ttl_entry.result_column]; + + if (!block.has(ttl_entry.result_column)) + { + columns_to_remove.push_back(ttl_entry.result_column); + ttl_entry.expression->execute(block); + } + + const IColumn * ttl_column = block.getByName(ttl_entry.result_column).column.get(); + + for (size_t i = 0; i < block.rows(); ++i) + { + UInt32 cur_ttl = getTimestampByIndex(ttl_column, i); + new_ttl_info.update(cur_ttl); + } + } + + for (const String & column : columns_to_remove) + block.erase(column); +} + UInt32 TTLBlockInputStream::getTimestampByIndex(const IColumn * column, size_t ind) { if (const ColumnUInt16 * column_date = typeid_cast(column)) diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/DataStreams/TTLBlockInputStream.h index 3f37f35426c..18670021ec9 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/DataStreams/TTLBlockInputStream.h @@ -78,6 +78,8 @@ private: /// Updates TTL for moves void updateMovesTTL(Block & block); + void updateRecompressionTTL(Block & block); + UInt32 getTimestampByIndex(const IColumn * column, size_t ind); bool isTTLExpired(time_t ttl) const; }; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 9d35b339d94..3a397cb9b5a 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -533,8 +533,16 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) /// Special step to recalculate affected indices and TTL expressions. stages.emplace_back(context); for (const auto & column : unchanged_columns) + { + std::cerr << "ADDING UNCHANGED COLUMN TO STAGE:" << column << std::endl; stages.back().column_to_updated.emplace( column, std::make_shared(column)); + std::cerr << "OUTPUT COLUMNS:" << stages.back().output_columns.size() << std::endl; + for (const auto & col : stages.back().output_columns) + { + std::cerr << "OUTPUT COLUMN:" << col << std::endl; + } + } } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b721cf4afbf..536d72d327a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3064,8 +3064,10 @@ CompressionCodecPtr MergeTreeData::getCompressionCodecForPart(size_t part_size_c auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); + std::cerr << "RECOMPRESSION ENTRIES SIZE:" << recompression_ttl_entries.size() << std::endl; for (auto ttl_entry_it = recompression_ttl_entries.begin(); ttl_entry_it != recompression_ttl_entries.end(); ++ttl_entry_it) { + std::cerr << "RECOMPRESSION TTL SIZE:" << ttl_infos.recompression_ttl.size() << std::endl; auto ttl_info_it = ttl_infos.recompression_ttl.find(ttl_entry_it->result_column); /// Prefer TTL rule which went into action last. if (ttl_info_it != ttl_infos.recompression_ttl.end() @@ -3078,7 +3080,15 @@ CompressionCodecPtr MergeTreeData::getCompressionCodecForPart(size_t part_size_c } if (max_max_ttl) + { + std::cerr << "BEST ENTRY FOUND, MAX MAX:" << max_max_ttl << std::endl; + std::cerr << "RECOMPRESSION IS NULLPTR:" << (best_entry_it->recompression_codec == nullptr) << std::endl; return CompressionCodecFactory::instance().get(best_entry_it->recompression_codec, {}); + } + else + { + std::cerr << "NOT FOUND NEW RECOMPRESSION\n"; + } return global_context.chooseCompressionCodec( part_size_compressed, diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 8cece66dafb..9a77115e777 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -659,9 +659,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor /// (which is locked in shared mode when input streams are created) and when inserting new data /// the order is reverse. This annoys TSan even though one lock is locked in shared mode and thus /// deadlock is impossible. - auto compression_codec = data.global_context.chooseCompressionCodec( - merge_entry->total_size_bytes_compressed, - static_cast (merge_entry->total_size_bytes_compressed) / data.getTotalActiveSizeInBytes()); + auto compression_codec = data.getCompressionCodecForPart(merge_entry->total_size_bytes_compressed, new_data_part->ttl_infos, time_of_merge); /// TODO: Should it go through IDisk interface? String rows_sources_file_path; @@ -1082,15 +1080,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor auto disk = new_data_part->volume->getDisk(); String new_part_tmp_path = new_data_part->getFullRelativePath(); - /// Note: this is done before creating input streams, because otherwise data.data_parts_mutex - /// (which is locked in data.getTotalActiveSizeInBytes()) - /// (which is locked in shared mode when input streams are created) and when inserting new data - /// the order is reverse. This annoys TSan even though one lock is locked in shared mode and thus - /// deadlock is impossible. - auto compression_codec = context.chooseCompressionCodec( - source_part->getBytesOnDisk(), - static_cast(source_part->getBytesOnDisk()) / data.getTotalActiveSizeInBytes()); - disk->createDirectories(new_part_tmp_path); /// Don't change granularity type while mutating subset of columns @@ -1100,11 +1089,27 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor bool need_remove_expired_values = false; if (in && shouldExecuteTTL(metadata_snapshot, in->getHeader().getNamesAndTypesList().getNames(), commands_for_part)) + { + std::cerr << "GOING TO MATERIALIZE TTL\n"; need_remove_expired_values = true; + } + else + { + std::cerr << "NOT GOING TO MATERIALIZE TTL\n"; + std::cerr << "IN IS NULL:" << (in == nullptr) << std::endl; + } /// All columns from part are changed and may be some more that were missing before in part if (!isWidePart(source_part) || (interpreter && interpreter->isAffectingAllColumns())) { + std::cerr << "MUTATING ALL PART COLUMNS\n"; + /// Note: this is done before creating input streams, because otherwise data.data_parts_mutex + /// (which is locked in data.getTotalActiveSizeInBytes()) + /// (which is locked in shared mode when input streams are created) and when inserting new data + /// the order is reverse. This annoys TSan even though one lock is locked in shared mode and thus + /// deadlock is impossible. + auto compression_codec = data.getCompressionCodecForPart(source_part->getBytesOnDisk(), source_part->ttl_infos, time_of_mutation); + auto part_indices = getIndicesForNewDataPart(metadata_snapshot->getSecondaryIndices(), for_file_renames); mutateAllPartColumns( new_data_part, @@ -1121,6 +1126,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor } else /// TODO: check that we modify only non-key columns in this case. { + + std::cerr << "MUTATING SOME PART COLUMNS\n"; /// We will modify only some of the columns. Other columns and key values can be copied as-is. auto indices_to_recalc = getIndicesToRecalculate(in, updated_header.getNamesAndTypesList(), metadata_snapshot, context); @@ -1128,7 +1135,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor NameToNameVector files_to_rename = collectFilesForRenames(source_part, for_file_renames, mrk_extension); if (need_remove_expired_values) + { files_to_skip.insert("ttl.txt"); + } + for (const auto & name : files_to_skip) + { + std::cerr << "SKIPPING " << name << std::endl; + } /// Create hardlinks for unchanged files for (auto it = disk->iterateDirectory(source_part->getFullRelativePath()); it->isValid(); it->next()) @@ -1157,8 +1170,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor new_data_part->checksums = source_part->checksums; + auto compression_codec = source_part->default_codec; + if (in) { + std::cerr << "HEADER:" << updated_header.dumpStructure() << std::endl; + std::cerr << "IN HEADER:" << in->getHeader().dumpStructure() << std::endl; mutateSomePartColumns( source_part, metadata_snapshot, diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 92bf5345d5a..5115666066a 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -13,6 +13,7 @@ #include #include +#include namespace ProfileEvents { @@ -234,8 +235,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa for (const auto & ttl_entry : move_ttl_entries) updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); + time_t current_time = time(nullptr); NamesAndTypesList columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); - ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr)); + ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, current_time); VolumePtr volume = data.getStoragePolicy()->getVolume(0); auto new_data_part = data.createPart( @@ -306,7 +308,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa /// This effectively chooses minimal compression method: /// either default lz4 or compression method with zero thresholds on absolute and relative part size. - auto compression_codec = data.global_context.chooseCompressionCodec(0, 0); + auto compression_codec = data.getCompressionCodecForPart(0, new_data_part->ttl_infos, current_time); + std::cerr << "SELECTED CODEC:" << queryToString(compression_codec->getCodecDesc()) << std::endl; const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 4526b0d4f9b..8706c1f3b37 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -558,8 +558,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.sampling_key = KeyDescription::getKeyFromAST(args.storage_def->sample_by->ptr(), metadata.columns, args.context); if (args.storage_def->ttl_table) + { + std::cerr << "Parsing table ttl in description\n"; metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( args.storage_def->ttl_table->ptr(), metadata.columns, args.context, metadata.primary_key); + } if (args.query.columns_list && args.query.columns_list->indices) for (auto & index : args.query.columns_list->indices->children) diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index f611c1ec95d..f410fa34f59 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -124,7 +124,7 @@ TTLTableDescription StorageInMemoryMetadata::getTableTTLs() const bool StorageInMemoryMetadata::hasAnyTableTTL() const { - return hasAnyMoveTTL() || hasRowsTTL(); + return hasAnyMoveTTL() || hasRowsTTL() || hasAnyRecompressionTTL(); } TTLColumnsDescription StorageInMemoryMetadata::getColumnTTLs() const @@ -207,6 +207,9 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet } } + for (const auto & entry : getRecompressionTTLs()) + add_dependent_columns(entry.expression, required_ttl_columns); + for (const auto & [name, entry] : getColumnTTLs()) { if (add_dependent_columns(entry.expression, required_ttl_columns)) diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 656baf39971..ca5ea714dd9 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -89,6 +89,7 @@ TTLDescription::TTLDescription(const TTLDescription & other) , aggregate_descriptions(other.aggregate_descriptions) , destination_type(other.destination_type) , destination_name(other.destination_name) + , recompression_codec(other.recompression_codec) { if (other.expression) expression = std::make_shared(*other.expression); @@ -125,6 +126,12 @@ TTLDescription & TTLDescription::operator=(const TTLDescription & other) aggregate_descriptions = other.aggregate_descriptions; destination_type = other.destination_type; destination_name = other.destination_name; + + if (other.recompression_codec) + recompression_codec = other.recompression_codec->clone(); + else + recompression_codec.reset(); + return * this; } @@ -266,6 +273,7 @@ TTLDescription TTLDescription::getTTLFromAST( } else if (ttl_element->mode == TTLMode::RECOMPRESS) { + std::cerr << "GOT INTO RECOMPRESS\n"; result.recompression_codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( ttl_element->recompression_codec, {}, !context.getSettingsRef().allow_suspicious_codecs); @@ -283,6 +291,7 @@ TTLTableDescription::TTLTableDescription(const TTLTableDescription & other) : definition_ast(other.definition_ast ? other.definition_ast->clone() : nullptr) , rows_ttl(other.rows_ttl) , move_ttl(other.move_ttl) + , recompression_ttl(other.recompression_ttl) { } @@ -298,6 +307,7 @@ TTLTableDescription & TTLTableDescription::operator=(const TTLTableDescription & rows_ttl = other.rows_ttl; move_ttl = other.move_ttl; + recompression_ttl = other.recompression_ttl; return *this; } @@ -327,6 +337,7 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( } else if (ttl.mode == TTLMode::RECOMPRESS) { + std::cerr << "GOT RECOMPRESSIOn TTL\n"; result.recompression_ttl.emplace_back(std::move(ttl)); } else diff --git a/tests/queries/0_stateless/01465_ttl_recompression.reference b/tests/queries/0_stateless/01465_ttl_recompression.reference new file mode 100644 index 00000000000..2f1a2ea40b1 --- /dev/null +++ b/tests/queries/0_stateless/01465_ttl_recompression.reference @@ -0,0 +1,10 @@ +3000 +1_1_1_0 LZ4 +2_2_2_0 ZSTD(17) +3_3_3_0 LZ4HC(10) +1_1_1_0_4 LZ4 +2_2_2_0_4 ZSTD(17) +3_3_3_0_4 LZ4HC(10) +1_1_1_1_4 LZ4 +2_2_2_1_4 ZSTD(12) +3_3_3_1_4 ZSTD(12) diff --git a/tests/queries/0_stateless/01465_ttl_recompression.sql b/tests/queries/0_stateless/01465_ttl_recompression.sql new file mode 100644 index 00000000000..0c72000c624 --- /dev/null +++ b/tests/queries/0_stateless/01465_ttl_recompression.sql @@ -0,0 +1,32 @@ +DROP TABLE IF EXISTS recompression_table; + +CREATE TABLE recompression_table +( + dt DateTime, + key UInt64, + value String + +) ENGINE MergeTree() +ORDER BY tuple() +PARTITION BY key +TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(17)), dt + INTERVAL 1 YEAR RECOMPRESS CODEC(LZ4HC(10)); + +INSERT INTO recompression_table SELECT now(), 1, toString(number) from numbers(1000); + +INSERT INTO recompression_table SELECT now() - INTERVAL 2 MONTH, 2, toString(number) from numbers(1000, 1000); + +INSERT INTO recompression_table SELECT now() - INTERVAL 2 YEAR, 3, toString(number) from numbers(2000, 1000); + +SELECT COUNT() FROM recompression_table; + +SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name; + +ALTER TABLE recompression_table MODIFY TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(12)) SETTINGS mutations_sync = 2; + +SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name; + +OPTIMIZE TABLE recompression_table FINAL; + +SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name; + +DROP TABLE IF EXISTS recompression_table; From c37a19f7b095a8f05ab1e9b65c6181e0e1fb6605 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 1 Sep 2020 14:23:38 +0300 Subject: [PATCH 028/298] Better --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 9 --------- src/Storages/MergeTree/MergeTreeSettings.h | 2 ++ .../0_stateless/01465_ttl_recompression.reference | 2 ++ tests/queries/0_stateless/01465_ttl_recompression.sql | 8 ++++++++ 4 files changed, 12 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 9a77115e777..f46fb7a79ef 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1126,8 +1126,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor } else /// TODO: check that we modify only non-key columns in this case. { - - std::cerr << "MUTATING SOME PART COLUMNS\n"; /// We will modify only some of the columns. Other columns and key values can be copied as-is. auto indices_to_recalc = getIndicesToRecalculate(in, updated_header.getNamesAndTypesList(), metadata_snapshot, context); @@ -1138,11 +1136,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor { files_to_skip.insert("ttl.txt"); } - for (const auto & name : files_to_skip) - { - std::cerr << "SKIPPING " << name << std::endl; - } - /// Create hardlinks for unchanged files for (auto it = disk->iterateDirectory(source_part->getFullRelativePath()); it->isValid(); it->next()) { @@ -1174,8 +1167,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor if (in) { - std::cerr << "HEADER:" << updated_header.dumpStructure() << std::endl; - std::cerr << "IN HEADER:" << in->getHeader().dumpStructure() << std::endl; mutateSomePartColumns( source_part, metadata_snapshot, diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 085c441aa90..6ac262ed35a 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -33,8 +33,10 @@ struct Settings; M(UInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \ M(UInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ M(UInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + M(UInt64, max_replicated_recompressions_in_queue, 1, "How many tasks of recompressiong parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ + M(UInt64, number_of_free_entries_in_pool_to_execute_ttl_recompression, 10, "When there is less than specified number of free entries in pool, do not execute part recompression according to TTL. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ M(Seconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(Seconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(Seconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ diff --git a/tests/queries/0_stateless/01465_ttl_recompression.reference b/tests/queries/0_stateless/01465_ttl_recompression.reference index 2f1a2ea40b1..c03c003d5b8 100644 --- a/tests/queries/0_stateless/01465_ttl_recompression.reference +++ b/tests/queries/0_stateless/01465_ttl_recompression.reference @@ -1,7 +1,9 @@ +CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt64,\n `value` String\n)\nENGINE = MergeTree()\nPARTITION BY key\nORDER BY tuple()\nTTL dt + toIntervalMonth(1) RECOMPRESS CODEC(ZSTD(17)), dt + toIntervalYear(1) RECOMPRESS CODEC(LZ4HC(10))\nSETTINGS index_granularity = 8192 3000 1_1_1_0 LZ4 2_2_2_0 ZSTD(17) 3_3_3_0 LZ4HC(10) +CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt64,\n `value` String\n)\nENGINE = MergeTree()\nPARTITION BY key\nORDER BY tuple()\nTTL dt + toIntervalMonth(1) RECOMPRESS CODEC(ZSTD(12))\nSETTINGS index_granularity = 8192 1_1_1_0_4 LZ4 2_2_2_0_4 ZSTD(17) 3_3_3_0_4 LZ4HC(10) diff --git a/tests/queries/0_stateless/01465_ttl_recompression.sql b/tests/queries/0_stateless/01465_ttl_recompression.sql index 0c72000c624..92233f2d5cb 100644 --- a/tests/queries/0_stateless/01465_ttl_recompression.sql +++ b/tests/queries/0_stateless/01465_ttl_recompression.sql @@ -11,6 +11,10 @@ ORDER BY tuple() PARTITION BY key TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(17)), dt + INTERVAL 1 YEAR RECOMPRESS CODEC(LZ4HC(10)); +SHOW CREATE TABLE recompression_table; + +SYSTEM STOP TTL MERGES recompression_table; + INSERT INTO recompression_table SELECT now(), 1, toString(number) from numbers(1000); INSERT INTO recompression_table SELECT now() - INTERVAL 2 MONTH, 2, toString(number) from numbers(1000, 1000); @@ -23,8 +27,12 @@ SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompre ALTER TABLE recompression_table MODIFY TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(12)) SETTINGS mutations_sync = 2; +SHOW CREATE TABLE recompression_table; + SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name; +SYSTEM START TTL MERGES recompression_table; + OPTIMIZE TABLE recompression_table FINAL; SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name; From 3cadc9033ae63d7faa851b1707b3c6f9ce1a36aa Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Sep 2020 18:26:49 +0300 Subject: [PATCH 029/298] fsyncs for metadata files of part --- .../MergeTree/IMergeTreeDataPartWriter.h | 2 +- .../MergeTreeDataPartWriterOnDisk.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 13 +++++++-- .../MergeTree/MergedBlockOutputStream.h | 3 +- utils/durability-test/create_sync.sql | 1 + utils/durability-test/durability-test.sh | 28 ++++++++++--------- utils/durability-test/insert_sync.sql | 1 + 7 files changed, 32 insertions(+), 18 deletions(-) create mode 100644 utils/durability-test/create_sync.sql mode change 100644 => 100755 utils/durability-test/durability-test.sh create mode 100644 utils/durability-test/insert_sync.sql diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 4d3602e732e..4a42a58a65b 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -52,7 +52,7 @@ public: virtual void initPrimaryIndex() {} virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) = 0; - virtual void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & /* checksums */, bool /* sync */) {} + virtual void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & /* checksums */, bool /* sync */) {} virtual void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & /* checksums */, bool /* sync */) {} Columns releaseIndexColumns(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index dbe41144573..8295b881d87 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -332,7 +332,7 @@ void MergeTreeDataPartWriterOnDisk::finishPrimaryIndexSerialization( checksums.files["primary.idx"].file_size = index_stream->count(); checksums.files["primary.idx"].file_hash = index_stream->getHash(); if (sync) - index_stream->sync(); + index_file_stream->sync(); index_stream = nullptr; } } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index fdef5d69688..bdc6bade259 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -111,7 +111,7 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( part_columns = *total_columns_list; if (new_part->isStoredOnDisk()) - finalizePartOnDisk(new_part, part_columns, checksums); + finalizePartOnDisk(new_part, part_columns, checksums, sync); new_part->setColumns(part_columns); new_part->rows_count = rows_count; @@ -126,7 +126,8 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( void MergedBlockOutputStream::finalizePartOnDisk( const MergeTreeData::MutableDataPartPtr & new_part, NamesAndTypesList & part_columns, - MergeTreeData::DataPart::Checksums & checksums) + MergeTreeData::DataPart::Checksums & checksums, + bool sync) { if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part)) { @@ -143,6 +144,8 @@ void MergedBlockOutputStream::finalizePartOnDisk( count_out_hashing.next(); checksums.files["count.txt"].file_size = count_out_hashing.count(); checksums.files["count.txt"].file_hash = count_out_hashing.getHash(); + if (sync) + count_out->sync(); } if (!new_part->ttl_infos.empty()) @@ -153,6 +156,8 @@ void MergedBlockOutputStream::finalizePartOnDisk( new_part->ttl_infos.write(out_hashing); checksums.files["ttl.txt"].file_size = out_hashing.count(); checksums.files["ttl.txt"].file_hash = out_hashing.getHash(); + if (sync) + out->sync(); } removeEmptyColumnsFromPart(new_part, part_columns, checksums); @@ -161,12 +166,16 @@ void MergedBlockOutputStream::finalizePartOnDisk( /// Write a file with a description of columns. auto out = volume->getDisk()->writeFile(part_path + "columns.txt", 4096); part_columns.writeText(*out); + if (sync) + out->sync(); } { /// Write file with checksums. auto out = volume->getDisk()->writeFile(part_path + "checksums.txt", 4096); checksums.write(*out); + if (sync) + out->sync(); } } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 0b500b93f01..87ff9dd1ded 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -59,7 +59,8 @@ private: void finalizePartOnDisk( const MergeTreeData::MutableDataPartPtr & new_part, NamesAndTypesList & part_columns, - MergeTreeData::DataPart::Checksums & checksums); + MergeTreeData::DataPart::Checksums & checksums, + bool sync); private: NamesAndTypesList columns_list; diff --git a/utils/durability-test/create_sync.sql b/utils/durability-test/create_sync.sql new file mode 100644 index 00000000000..2cc88d2c943 --- /dev/null +++ b/utils/durability-test/create_sync.sql @@ -0,0 +1 @@ +CREATE TABLE test_sync (a Int, s String) ENGINE = MergeTree ORDER BY a SETTINGS fsync_after_insert = 1, min_compressed_bytes_to_fsync_after_merge = 1; diff --git a/utils/durability-test/durability-test.sh b/utils/durability-test/durability-test.sh old mode 100644 new mode 100755 index 1f47c900f49..c7f8936ec95 --- a/utils/durability-test/durability-test.sh +++ b/utils/durability-test/durability-test.sh @@ -17,12 +17,12 @@ fi function run() { - sshpass -p $PASSWORD ssh -p $SSH_PORT root@localhost "$1" + sshpass -p $PASSWORD ssh -p $SSH_PORT root@localhost "$1" 2>/dev/null } function copy() { - sshpass -p $PASSWORD scp -r -P $SSH_PORT $1 root@localhost:$2 + sshpass -p $PASSWORD scp -r -P $SSH_PORT $1 root@localhost:$2 2>/dev/null } function wait_vm_for_start() @@ -50,8 +50,8 @@ function wait_clickhouse_for_start() { echo "Waiting until ClickHouse started..." started=0 - for i in {0..15}; do - run "clickhouse client --query 'select 1'" + for i in {0..30}; do + run "clickhouse client --query 'select 1'" > /dev/null if [ $? -eq 0 ]; then started=1 break @@ -70,7 +70,7 @@ echo "Downloading image" curl -O $URL/$IMAGE qemu-img resize $IMAGE +10G -virt-customize -a $IMAGE --root-password password:$PASSWORD +virt-customize -a $IMAGE --root-password password:$PASSWORD > /dev/null 2>&1 virt-copy-in -a $IMAGE sshd_config /etc/ssh echo "Starting VM" @@ -93,8 +93,8 @@ if [[ -z $CLICKHOUSE_CONFIG_DIR ]]; then CLICKHOUSE_CONFIG_DIR=/etc/clickhouse-server fi -echo "Using ClickHouse binary: " $CLICKHOUSE_BINARY -echo "Using ClickHouse config from: " $CLICKHOUSE_CONFIG_DIR +echo "Using ClickHouse binary:" $CLICKHOUSE_BINARY +echo "Using ClickHouse config from:" $CLICKHOUSE_CONFIG_DIR copy $CLICKHOUSE_BINARY /usr/bin copy $CLICKHOUSE_CONFIG_DIR /etc @@ -104,23 +104,19 @@ echo "Prepared VM" echo "Starting ClickHouse" run "clickhouse server --config-file=/etc/clickhouse-server/config.xml > clickhouse-server.log 2>&1" & - wait_clickhouse_for_start -echo "Started ClickHouse" - query=`cat $CREATE_QUERY` echo "Executing query:" $query run "clickhouse client --query '$query'" query=`cat $INSERT_QUERY` echo "Will run in a loop query: " $query -run "clickhouse benchmark <<< '$query'" & +run "clickhouse benchmark <<< '$query' -c 8" & echo "Running queries" pid=`pidof qemu-system-x86_64` -sec=$(( (RANDOM % 3) + 25 )) - +sec=$(( (RANDOM % 5) + 25 )) ms=$(( RANDOM % 1000 )) echo "Will kill VM in $sec.$ms sec" @@ -130,6 +126,8 @@ kill -9 $pid echo "Restarting" +sleep 5s + ./startup.exp > qemu.log 2>&1 & wait_vm_for_start @@ -137,10 +135,12 @@ run "rm -r *data/system" run "clickhouse server --config-file=/etc/clickhouse-server/config.xml > clickhouse-server.log 2>&1" & wait_clickhouse_for_start +pid=`pidof qemu-system-x86_64` result=`run "grep $TABLE_NAME clickhouse-server.log | grep 'Caught exception while loading metadata'"` if [[ -n $result ]]; then echo "FAIL. Can't attach table:" echo $result + kill -9 $pid exit 1 fi @@ -148,7 +148,9 @@ result=`run "grep $TABLE_NAME clickhouse-server.log | grep 'Considering to remov if [[ -n $result ]]; then echo "FAIL. Have broken parts:" echo $result + kill -9 $pid exit 1 fi +kill -9 $pid echo OK diff --git a/utils/durability-test/insert_sync.sql b/utils/durability-test/insert_sync.sql new file mode 100644 index 00000000000..a1ad2ff4ea5 --- /dev/null +++ b/utils/durability-test/insert_sync.sql @@ -0,0 +1 @@ +INSERT INTO test_sync SELECT number, toString(number) FROM numbers(10) From ac5877e601714450a369062abbf80f84485bc6f5 Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 2 Sep 2020 00:58:39 +0800 Subject: [PATCH 030/298] Fix tests --- tests/queries/0_stateless/01447_JSONStrings.reference | 9 +-------- tests/queries/0_stateless/01447_JSONStrings.sql | 2 ++ 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01447_JSONStrings.reference b/tests/queries/0_stateless/01447_JSONStrings.reference index 58af593dc77..1c6f073c0d0 100644 --- a/tests/queries/0_stateless/01447_JSONStrings.reference +++ b/tests/queries/0_stateless/01447_JSONStrings.reference @@ -32,12 +32,5 @@ ["1", "a", "[1,2,3]", "(1,'a')", "ᴺᵁᴸᴸ", "nan"] ], - "rows": 1, - - "statistics": - { - "elapsed": 0.00068988, - "rows_read": 1, - "bytes_read": 1 - } + "rows": 1 } diff --git a/tests/queries/0_stateless/01447_JSONStrings.sql b/tests/queries/0_stateless/01447_JSONStrings.sql index 7d89f0f5087..45fc4a56d7a 100644 --- a/tests/queries/0_stateless/01447_JSONStrings.sql +++ b/tests/queries/0_stateless/01447_JSONStrings.sql @@ -1,3 +1,5 @@ +SET output_format_write_statistics = 0; + SELECT 1, 'a', From 26d75f76026303b6f3769ab4ea39ff639ebe836a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 2 Sep 2020 01:25:10 +0300 Subject: [PATCH 031/298] do fsync for WAL --- src/Storages/MergeTree/MergeTreeSettings.h | 2 ++ .../MergeTree/MergeTreeWriteAheadLog.cpp | 32 +++++++++++++++++-- .../MergeTree/MergeTreeWriteAheadLog.h | 10 +++++- 3 files changed, 41 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 1341526c38b..edf03710974 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -43,6 +43,8 @@ struct Settings; M(UInt64, min_compressed_bytes_to_fsync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ M(Bool, fsync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ M(Bool, fsync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ + M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Amount of bytes, accumulated in WAL to do fsync.", 0) \ + M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Interval in milliseconds after which fsync for WAL is being done.", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index eda8579c76a..6f220fc7d5d 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -16,17 +17,23 @@ namespace ErrorCodes extern const int CORRUPTED_DATA; } - MergeTreeWriteAheadLog::MergeTreeWriteAheadLog( - const MergeTreeData & storage_, + MergeTreeData & storage_, const DiskPtr & disk_, const String & name_) : storage(storage_) , disk(disk_) , name(name_) , path(storage.getRelativeDataPath() + name_) + , pool(storage.global_context.getSchedulePool()) { init(); + sync_task = pool.createTask("MergeTreeWriteAheadLog::sync", [this] + { + std::lock_guard lock(write_mutex); + out->sync(); + sync_scheduled = false; + }); } void MergeTreeWriteAheadLog::init() @@ -38,6 +45,7 @@ void MergeTreeWriteAheadLog::init() block_out = std::make_unique(*out, 0, Block{}); min_block_number = std::numeric_limits::max(); max_block_number = -1; + bytes_at_last_sync = 0; } void MergeTreeWriteAheadLog::addPart(const Block & block, const String & part_name) @@ -53,6 +61,7 @@ void MergeTreeWriteAheadLog::addPart(const Block & block, const String & part_na writeStringBinary(part_name, *out); block_out->write(block); block_out->flush(); + sync(lock); auto max_wal_bytes = storage.getSettings()->write_ahead_log_max_bytes; if (out->count() > max_wal_bytes) @@ -66,6 +75,7 @@ void MergeTreeWriteAheadLog::dropPart(const String & part_name) writeIntBinary(static_cast(0), *out); writeIntBinary(static_cast(ActionType::DROP_PART), *out); writeStringBinary(part_name, *out); + sync(lock); } void MergeTreeWriteAheadLog::rotate(const std::lock_guard &) @@ -175,6 +185,24 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor return result; } +void MergeTreeWriteAheadLog::sync(const std::lock_guard &) +{ + size_t bytes_to_sync = storage.getSettings()->write_ahead_log_bytes_to_fsync; + time_t time_to_sync = storage.getSettings()->write_ahead_log_interval_ms_to_fsync; + size_t current_bytes = out->count(); + + if (bytes_to_sync && current_bytes - bytes_at_last_sync > bytes_to_sync) + { + sync_task->schedule(); + bytes_at_last_sync = current_bytes; + } + else if (time_to_sync && !sync_scheduled) + { + sync_task->scheduleAfter(time_to_sync); + sync_scheduled = true; + } +} + std::optional MergeTreeWriteAheadLog::tryParseMinMaxBlockNumber(const String & filename) { diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 2cc3c2b4181..43abf3c04be 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -31,7 +32,7 @@ public: constexpr static auto WAL_FILE_EXTENSION = ".bin"; constexpr static auto DEFAULT_WAL_FILE_NAME = "wal.bin"; - MergeTreeWriteAheadLog(const MergeTreeData & storage_, const DiskPtr & disk_, + MergeTreeWriteAheadLog(MergeTreeData & storage_, const DiskPtr & disk_, const String & name = DEFAULT_WAL_FILE_NAME); void addPart(const Block & block, const String & part_name); @@ -44,6 +45,7 @@ public: private: void init(); void rotate(const std::lock_guard & lock); + void sync(const std::lock_guard & lock); const MergeTreeData & storage; DiskPtr disk; @@ -56,6 +58,12 @@ private: Int64 min_block_number = std::numeric_limits::max(); Int64 max_block_number = -1; + BackgroundSchedulePool & pool; + BackgroundSchedulePoolTaskHolder sync_task; + + size_t bytes_at_last_sync = 0; + bool sync_scheduled = false; + mutable std::mutex write_mutex; }; From a80c1adee81631f770f642ad4430a8ff44ff46af Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 2 Sep 2020 12:05:02 +0800 Subject: [PATCH 032/298] Add JSONCompactStrings formats --- docs/en/interfaces/formats.md | 288 +++++++++++------- src/Formats/FormatFactory.cpp | 6 - .../Impl/JSONCompactEachRowRowInputFormat.cpp | 49 ++- .../Impl/JSONCompactEachRowRowInputFormat.h | 11 +- .../JSONCompactEachRowRowOutputFormat.cpp | 37 ++- .../Impl/JSONCompactEachRowRowOutputFormat.h | 9 +- .../Impl/JSONCompactRowOutputFormat.cpp | 30 +- .../Formats/Impl/JSONCompactRowOutputFormat.h | 10 +- .../Impl/JSONEachRowRowInputFormat.cpp | 42 ++- .../Formats/Impl/JSONEachRowRowInputFormat.h | 9 +- .../Impl/JSONEachRowRowOutputFormat.cpp | 32 +- .../Formats/Impl/JSONEachRowRowOutputFormat.h | 10 +- ...JSONEachRowWithProgressRowOutputFormat.cpp | 11 +- .../Formats/Impl/JSONRowOutputFormat.cpp | 44 ++- .../Formats/Impl/JSONRowOutputFormat.h | 9 +- .../Impl/JSONStringsEachRowRowInputFormat.cpp | 245 --------------- .../Impl/JSONStringsEachRowRowInputFormat.h | 54 ---- .../JSONStringsEachRowRowOutputFormat.cpp | 117 ------- .../Impl/JSONStringsEachRowRowOutputFormat.h | 45 --- .../Impl/JSONStringsRowOutputFormat.cpp | 93 ------ .../Formats/Impl/JSONStringsRowOutputFormat.h | 43 --- .../0_stateless/01446_JSONStringsEachRow.sql | 63 ---- .../01446_json_strings_each_row.reference | 22 ++ .../01446_json_strings_each_row.sql | 38 +++ .../0_stateless/01447_json_strings.reference | 43 +++ ...JSONStrings.sql => 01447_json_strings.sql} | 0 ...8_json_compact_strings_each_row.reference} | 0 .../01448_json_compact_strings_each_row.sql | 63 ++++ ...e => 01449_json_compact_strings.reference} | 0 .../01449_json_compact_strings.sql | 10 + 30 files changed, 621 insertions(+), 812 deletions(-) delete mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp delete mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h delete mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp delete mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h delete mode 100644 src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp delete mode 100644 src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h delete mode 100644 tests/queries/0_stateless/01446_JSONStringsEachRow.sql create mode 100644 tests/queries/0_stateless/01446_json_strings_each_row.reference create mode 100644 tests/queries/0_stateless/01446_json_strings_each_row.sql create mode 100644 tests/queries/0_stateless/01447_json_strings.reference rename tests/queries/0_stateless/{01447_JSONStrings.sql => 01447_json_strings.sql} (100%) rename tests/queries/0_stateless/{01446_JSONStringsEachRow.reference => 01448_json_compact_strings_each_row.reference} (100%) create mode 100644 tests/queries/0_stateless/01448_json_compact_strings_each_row.sql rename tests/queries/0_stateless/{01447_JSONStrings.reference => 01449_json_compact_strings.reference} (100%) create mode 100644 tests/queries/0_stateless/01449_json_compact_strings.sql diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 9c7c2dda8dc..bfe5b6218e4 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -10,45 +10,51 @@ results of a `SELECT`, and to perform `INSERT`s into a file-backed table. The supported formats are: -| Format | Input | Output | -|-----------------------------------------------------------------|-------|--------| -| [TabSeparated](#tabseparated) | ✔ | ✔ | -| [TabSeparatedRaw](#tabseparatedraw) | ✔ | ✔ | -| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | -| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | -| [Template](#format-template) | ✔ | ✔ | -| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | -| [CSV](#csv) | ✔ | ✔ | -| [CSVWithNames](#csvwithnames) | ✔ | ✔ | -| [CustomSeparated](#format-customseparated) | ✔ | ✔ | -| [Values](#data-format-values) | ✔ | ✔ | -| [Vertical](#vertical) | ✗ | ✔ | -| [VerticalRaw](#verticalraw) | ✗ | ✔ | -| [JSON](#json) | ✗ | ✔ | -| [JSONCompact](#jsoncompact) | ✗ | ✔ | -| [JSONStrings](#jsonstrings) | ✗ | ✔ | -| [JSONEachRow](#jsoneachrow) | ✔ | ✔ | -| [JSONCompactEachRow](#jsoncompacteachrow) | ✔ | ✔ | -| [JSONStringsEachRow](#jsonstringseachrow) | ✔ | ✔ | -| [TSKV](#tskv) | ✔ | ✔ | -| [Pretty](#pretty) | ✗ | ✔ | -| [PrettyCompact](#prettycompact) | ✗ | ✔ | -| [PrettyCompactMonoBlock](#prettycompactmonoblock) | ✗ | ✔ | -| [PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ | -| [PrettySpace](#prettyspace) | ✗ | ✔ | -| [Protobuf](#protobuf) | ✔ | ✔ | -| [Avro](#data-format-avro) | ✔ | ✔ | -| [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ | -| [Parquet](#data-format-parquet) | ✔ | ✔ | -| [Arrow](#data-format-arrow) | ✔ | ✔ | -| [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ | -| [ORC](#data-format-orc) | ✔ | ✗ | -| [RowBinary](#rowbinary) | ✔ | ✔ | -| [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ | -| [Native](#native) | ✔ | ✔ | -| [Null](#null) | ✗ | ✔ | -| [XML](#xml) | ✗ | ✔ | -| [CapnProto](#capnproto) | ✔ | ✗ | +| Format | Input | Output | +|-----------------------------------------------------------------------------------------|-------|--------| +| [TabSeparated](#tabseparated) | ✔ | ✔ | +| [TabSeparatedRaw](#tabseparatedraw) | ✔ | ✔ | +| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | +| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | +| [Template](#format-template) | ✔ | ✔ | +| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | +| [CSV](#csv) | ✔ | ✔ | +| [CSVWithNames](#csvwithnames) | ✔ | ✔ | +| [CustomSeparated](#format-customseparated) | ✔ | ✔ | +| [Values](#data-format-values) | ✔ | ✔ | +| [Vertical](#vertical) | ✗ | ✔ | +| [VerticalRaw](#verticalraw) | ✗ | ✔ | +| [JSON](#json) | ✗ | ✔ | +| [JSONString](#jsonstring) | ✗ | ✔ | +| [JSONCompact](#jsoncompact) | ✗ | ✔ | +| [JSONCompactString](#jsoncompactstring) | ✗ | ✔ | +| [JSONEachRow](#jsoneachrow) | ✔ | ✔ | +| [JSONEachRowWithProgress](#jsoneachrowwithprogress) | ✗ | ✔ | +| [JSONStringEachRow](#jsonstringeachrow) | ✔ | ✔ | +| [JSONStringEachRowWithProgress](#jsonstringeachrowwithprogress) | ✗ | ✔ | +| [JSONCompactEachRow](#jsoncompacteachrow) | ✔ | ✔ | +| [JSONCompactEachRowWithNamesAndTypes](#jsoncompacteachrowwithnamesandtypes) | ✔ | ✔ | +| [JSONCompactStringEachRow](#jsoncompactstringeachrow) | ✔ | ✔ | +| [JSONCompactStringEachRowWithNamesAndTypes](#jsoncompactstringeachrowwithnamesandtypes) | ✔ | ✔ | +| [TSKV](#tskv) | ✔ | ✔ | +| [Pretty](#pretty) | ✗ | ✔ | +| [PrettyCompact](#prettycompact) | ✗ | ✔ | +| [PrettyCompactMonoBlock](#prettycompactmonoblock) | ✗ | ✔ | +| [PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ | +| [PrettySpace](#prettyspace) | ✗ | ✔ | +| [Protobuf](#protobuf) | ✔ | ✔ | +| [Avro](#data-format-avro) | ✔ | ✔ | +| [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ | +| [Parquet](#data-format-parquet) | ✔ | ✔ | +| [Arrow](#data-format-arrow) | ✔ | ✔ | +| [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ | +| [ORC](#data-format-orc) | ✔ | ✗ | +| [RowBinary](#rowbinary) | ✔ | ✔ | +| [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ | +| [Native](#native) | ✔ | ✔ | +| [Null](#null) | ✗ | ✔ | +| [XML](#xml) | ✗ | ✔ | +| [CapnProto](#capnproto) | ✔ | ✗ | You can control some format processing parameters with the ClickHouse settings. For more information read the [Settings](../operations/settings/settings.md) section. @@ -395,62 +401,41 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA "meta": [ { - "name": "SearchPhrase", + "name": "'hello'", "type": "String" }, { - "name": "c", + "name": "multiply(42, number)", "type": "UInt64" + }, + { + "name": "range(5)", + "type": "Array(UInt8)" } ], "data": [ { - "SearchPhrase": "", - "c": "8267016" + "'hello'": "hello", + "multiply(42, number)": "0", + "range(5)": [0,1,2,3,4] }, { - "SearchPhrase": "bathroom interior design", - "c": "2166" + "'hello'": "hello", + "multiply(42, number)": "42", + "range(5)": [0,1,2,3,4] }, { - "SearchPhrase": "yandex", - "c": "1655" - }, - { - "SearchPhrase": "spring 2014 fashion", - "c": "1549" - }, - { - "SearchPhrase": "freeform photos", - "c": "1480" + "'hello'": "hello", + "multiply(42, number)": "84", + "range(5)": [0,1,2,3,4] } ], - "totals": - { - "SearchPhrase": "", - "c": "8873898" - }, + "rows": 3, - "extremes": - { - "min": - { - "SearchPhrase": "", - "c": "1480" - }, - "max": - { - "SearchPhrase": "", - "c": "8267016" - } - }, - - "rows": 5, - - "rows_before_limit_at_least": 141137 + "rows_before_limit_at_least": 3 } ``` @@ -471,73 +456,166 @@ ClickHouse supports [NULL](../sql-reference/syntax.md), which is displayed as `n See also the [JSONEachRow](#jsoneachrow) format. -## JSONCompact {#jsoncompact} +## JSONString {#jsonstring} -Differs from JSON only in that data rows are output in arrays of any element type, not in objects. +Differs from JSON only in that data fields are output in strings, not in typed json values. Example: -``` json +```json { "meta": [ { - "name": "SearchPhrase", + "name": "'hello'", "type": "String" }, { - "name": "c", + "name": "multiply(42, number)", "type": "UInt64" + }, + { + "name": "range(5)", + "type": "Array(UInt8)" } ], "data": [ - ["", "8267016"], - ["bathroom interior design", "2166"], - ["yandex", "1655"], - ["fashion trends spring 2014", "1549"], - ["freeform photo", "1480"] + { + "'hello'": "hello", + "multiply(42, number)": "0", + "range(5)": "[0,1,2,3,4]" + }, + { + "'hello'": "hello", + "multiply(42, number)": "42", + "range(5)": "[0,1,2,3,4]" + }, + { + "'hello'": "hello", + "multiply(42, number)": "84", + "range(5)": "[0,1,2,3,4]" + } ], - "totals": ["","8873898"], + "rows": 3, - "extremes": - { - "min": ["","1480"], - "max": ["","8267016"] - }, - - "rows": 5, - - "rows_before_limit_at_least": 141137 + "rows_before_limit_at_least": 3 } ``` -This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). -See also the `JSONEachRow` format. +## JSONCompact {#jsoncompact} +## JSONCompactString {#jsoncompactstring} -## JSONStrings {#jsonstrings} +Differs from JSON only in that data rows are output in arrays, not in objects. -Differs from JSON and JSONCompact only in that data rows are output in arrays of strings. +Example: -This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). -See also the `JSONEachRow` format. +``` json +// JSONCompact +{ + "meta": + [ + { + "name": "'hello'", + "type": "String" + }, + { + "name": "multiply(42, number)", + "type": "UInt64" + }, + { + "name": "range(5)", + "type": "Array(UInt8)" + } + ], + + "data": + [ + ["hello", "0", [0,1,2,3,4]], + ["hello", "42", [0,1,2,3,4]], + ["hello", "84", [0,1,2,3,4]] + ], + + "rows": 3, + + "rows_before_limit_at_least": 3 +} +``` + +```json +// JSONCompactString +{ + "meta": + [ + { + "name": "'hello'", + "type": "String" + }, + { + "name": "multiply(42, number)", + "type": "UInt64" + }, + { + "name": "range(5)", + "type": "Array(UInt8)" + } + ], + + "data": + [ + ["hello", "0", "[0,1,2,3,4]"], + ["hello", "42", "[0,1,2,3,4]"], + ["hello", "84", "[0,1,2,3,4]"] + ], + + "rows": 3, + + "rows_before_limit_at_least": 3 +} +``` ## JSONEachRow {#jsoneachrow} +## JSONStringEachRow {#jsonstringeachrow} ## JSONCompactEachRow {#jsoncompacteachrow} -## JSONStringsEachRow {#jsonstringseachrow} +## JSONCompactStringEachRow {#jsoncompactstringeachrow} When using these formats, ClickHouse outputs rows as separated, newline-delimited JSON values, but the data as a whole is not valid JSON. ``` json {"some_int":42,"some_str":"hello","some_tuple":[1,"a"]} // JSONEachRow [42,"hello",[1,"a"]] // JSONCompactEachRow -["42","hello","(2,'a')"] // JSONStringsEachRow +["42","hello","(2,'a')"] // JSONCompactStringsEachRow ``` When inserting the data, you should provide a separate JSON value for each row. +## JSONEachRowWithProgress {#jsoneachrowwithprogress} +## JSONStringEachRowWithProgress {#jsonstringeachrowwithprogress} + +Differs from JSONEachRow/JSONStringEachRow in that ClickHouse will also yield progress information as JSON objects. + +```json +{"row":{"'hello'":"hello","multiply(42, number)":"0","range(5)":[0,1,2,3,4]}} +{"row":{"'hello'":"hello","multiply(42, number)":"42","range(5)":[0,1,2,3,4]}} +{"row":{"'hello'":"hello","multiply(42, number)":"84","range(5)":[0,1,2,3,4]}} +{"progress":{"read_rows":"3","read_bytes":"24","written_rows":"0","written_bytes":"0","total_rows_to_read":"3"}} +``` + +## JSONCompactEachRowWithNamesAndTypes {#jsoncompacteachrowwithnamesandtypes} +## JSONCompactStringEachRowWithNamesAndTypes {#jsoncompactstringeachrowwithnamesandtypes} + +Differs from JSONCompactEachRow/JSONCompactStringEachRow in that the column names and types are written as the first two rows. + +```json +["'hello'", "multiply(42, number)", "range(5)"] +["String", "UInt64", "Array(UInt8)"] +["hello", "0", [0,1,2,3,4]] +["hello", "42", [0,1,2,3,4]] +["hello", "84", [0,1,2,3,4]] +``` + ### Inserting Data {#inserting-data} ``` sql diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index cb378fbea96..871098e00c0 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -352,8 +352,6 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); -void registerInputFormatProcessorJSONStringsEachRow(FormatFactory & factory); -void registerOutputFormatProcessorJSONStringsEachRow(FormatFactory & factory); void registerInputFormatProcessorProtobuf(FormatFactory & factory); void registerOutputFormatProcessorProtobuf(FormatFactory & factory); void registerInputFormatProcessorTemplate(FormatFactory & factory); @@ -380,7 +378,6 @@ void registerOutputFormatProcessorVertical(FormatFactory & factory); void registerOutputFormatProcessorJSON(FormatFactory & factory); void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); -void registerOutputFormatProcessorJSONStrings(FormatFactory & factory); void registerOutputFormatProcessorXML(FormatFactory & factory); void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); void registerOutputFormatProcessorNull(FormatFactory & factory); @@ -421,8 +418,6 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSONEachRow(*this); registerInputFormatProcessorJSONCompactEachRow(*this); registerOutputFormatProcessorJSONCompactEachRow(*this); - registerInputFormatProcessorJSONStringsEachRow(*this); - registerOutputFormatProcessorJSONStringsEachRow(*this); registerInputFormatProcessorProtobuf(*this); registerOutputFormatProcessorProtobuf(*this); registerInputFormatProcessorTemplate(*this); @@ -449,7 +444,6 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSON(*this); registerOutputFormatProcessorJSONCompact(*this); registerOutputFormatProcessorJSONEachRowWithProgress(*this); - registerOutputFormatProcessorJSONStrings(*this); registerOutputFormatProcessorXML(*this); registerOutputFormatProcessorODBCDriver2(*this); registerOutputFormatProcessorNull(*this); diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index 82e3cb795bf..eb697ce5318 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -19,8 +20,9 @@ JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat(ReadBuffer & const Block & header_, Params params_, const FormatSettings & format_settings_, - bool with_names_) - : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_) + bool with_names_, + bool yield_strings_) + : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_), yield_strings(yield_strings_) { const auto & sample = getPort().getHeader(); size_t num_columns = sample.columns(); @@ -200,10 +202,25 @@ void JSONCompactEachRowRowInputFormat::readField(size_t index, MutableColumns & { read_columns[index] = true; const auto & type = data_types[index]; - if (format_settings.null_as_default && !type->isNullable()) - read_columns[index] = DataTypeNullable::deserializeTextJSON(*columns[index], in, format_settings, type); + + if (yield_strings) + { + // notice: null_as_default on "null" strings is not supported + + String str; + readJSONString(str, in); + + ReadBufferFromString buf(str); + + type->deserializeAsWholeText(*columns[index], buf, format_settings); + } else - type->deserializeAsTextJSON(*columns[index], in, format_settings); + { + if (format_settings.null_as_default && !type->isNullable()) + read_columns[index] = DataTypeNullable::deserializeTextJSON(*columns[index], in, format_settings, type); + else + type->deserializeAsTextJSON(*columns[index], in, format_settings); + } } catch (Exception & e) { @@ -225,7 +242,7 @@ void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, std::move(params), settings, false); + return std::make_shared(buf, sample, std::move(params), settings, false, false); }); factory.registerInputFormatProcessor("JSONCompactEachRowWithNamesAndTypes", []( @@ -234,7 +251,25 @@ void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, std::move(params), settings, true); + return std::make_shared(buf, sample, std::move(params), settings, true, false); + }); + + factory.registerInputFormatProcessor("JSONCompactStringsEachRow", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, false, true); + }); + + factory.registerInputFormatProcessor("JSONCompactStringsEachRowWithNamesAndTypes", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, true, true); }); } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index 5c864ebc751..593f297108c 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -12,12 +12,18 @@ namespace DB class ReadBuffer; -/** A stream for reading data in JSONCompactEachRow and JSONCompactEachRowWithNamesAndTypes formats +/** A stream for reading data in JSONCompactEachRow- formats */ class JSONCompactEachRowRowInputFormat : public IRowInputFormat { public: - JSONCompactEachRowRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_, bool with_names_); + JSONCompactEachRowRowInputFormat( + ReadBuffer & in_, + const Block & header_, + Params params_, + const FormatSettings & format_settings_, + bool with_names_, + bool yield_strings_); String getName() const override { return "JSONCompactEachRowRowInputFormat"; } @@ -49,6 +55,7 @@ private: std::vector names_of_columns; bool with_names; + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp index e155dcb4247..ab8fd164c3c 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -12,8 +12,9 @@ JSONCompactEachRowRowOutputFormat::JSONCompactEachRowRowOutputFormat(WriteBuffer const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_, - bool with_names_) - : IRowOutputFormat(header_, out_, callback), settings(settings_), with_names(with_names_) + bool with_names_, + bool yield_strings_) + : IRowOutputFormat(header_, out_, callback), settings(settings_), with_names(with_names_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); @@ -23,7 +24,15 @@ JSONCompactEachRowRowOutputFormat::JSONCompactEachRowRowOutputFormat(WriteBuffer void JSONCompactEachRowRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) { - type.serializeAsTextJSON(column, row_num, out, settings); + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), out, settings); + } + else + type.serializeAsTextJSON(column, row_num, out, settings); } @@ -97,7 +106,7 @@ void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings, false); + return std::make_shared(buf, sample, callback, format_settings, false, false); }); factory.registerOutputFormatProcessor("JSONCompactEachRowWithNamesAndTypes", []( @@ -106,7 +115,25 @@ void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings &format_settings) { - return std::make_shared(buf, sample, callback, format_settings, true); + return std::make_shared(buf, sample, callback, format_settings, true, false); + }); + + factory.registerOutputFormatProcessor("JSONCompactStringsEachRow", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, false, true); + }); + + factory.registerOutputFormatProcessor("JSONCompactStringsEachRowWithNamesAndTypes", []( + WriteBuffer &buf, + const Block &sample, + FormatFactory::WriteCallback callback, + const FormatSettings &format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true, true); }); } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h index a7857a82d2d..56936783e78 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h @@ -15,7 +15,13 @@ namespace DB class JSONCompactEachRowRowOutputFormat : public IRowOutputFormat { public: - JSONCompactEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_, bool with_names); + JSONCompactEachRowRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool with_names_, + bool yield_strings_); String getName() const override { return "JSONCompactEachRowRowOutputFormat"; } @@ -41,5 +47,6 @@ private: NamesAndTypes fields; bool with_names; + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp index 7e56a4643da..c36942cff09 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp @@ -8,15 +8,28 @@ namespace DB { JSONCompactRowOutputFormat::JSONCompactRowOutputFormat( - WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_) - : JSONRowOutputFormat(out_, header, callback, settings_) + WriteBuffer & out_, + const Block & header, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_) + : JSONRowOutputFormat(out_, header, callback, settings_, yield_strings_) { } void JSONCompactRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) { - type.serializeAsTextJSON(column, row_num, *ostr, settings); + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), *ostr, settings); + } + else + type.serializeAsTextJSON(column, row_num, *ostr, settings); + ++field_number; } @@ -83,7 +96,16 @@ void registerOutputFormatProcessorJSONCompact(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings); + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONCompactStrings", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h index f4002f74287..6585016c44f 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h @@ -11,12 +11,17 @@ namespace DB struct FormatSettings; -/** The stream for outputting data in the JSONCompact format. +/** The stream for outputting data in the JSONCompact- formats. */ class JSONCompactRowOutputFormat : public JSONRowOutputFormat { public: - JSONCompactRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_); + JSONCompactRowOutputFormat( + WriteBuffer & out_, + const Block & header, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_); String getName() const override { return "JSONCompactRowOutputFormat"; } @@ -37,7 +42,6 @@ protected: } void writeTotalsFieldDelimiter() override; - }; } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 6350db3b211..9ba82fbb009 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -29,8 +30,12 @@ enum JSONEachRowRowInputFormat::JSONEachRowRowInputFormat( - ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) - : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), name_map(header_.columns()) + ReadBuffer & in_, + const Block & header_, + Params params_, + const FormatSettings & format_settings_, + bool yield_strings_) + : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), name_map(header_.columns()), yield_strings(yield_strings_) { /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. skipBOMIfExists(in); @@ -138,10 +143,25 @@ void JSONEachRowRowInputFormat::readField(size_t index, MutableColumns & columns { seen_columns[index] = read_columns[index] = true; const auto & type = getPort().getHeader().getByPosition(index).type; - if (format_settings.null_as_default && !type->isNullable()) - read_columns[index] = DataTypeNullable::deserializeTextJSON(*columns[index], in, format_settings, type); + + if (yield_strings) + { + // notice: null_as_default on "null" strings is not supported + + String str; + readJSONString(str, in); + + ReadBufferFromString buf(str); + + type->deserializeAsWholeText(*columns[index], buf, format_settings); + } else - type->deserializeAsTextJSON(*columns[index], in, format_settings); + { + if (format_settings.null_as_default && !type->isNullable()) + read_columns[index] = DataTypeNullable::deserializeTextJSON(*columns[index], in, format_settings, type); + else + type->deserializeAsTextJSON(*columns[index], in, format_settings); + } } catch (Exception & e) { @@ -318,13 +338,23 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, std::move(params), settings); + return std::make_shared(buf, sample, std::move(params), settings, false); + }); + + factory.registerInputFormatProcessor("JSONStringsEachRow", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, true); }); } void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory) { factory.registerFileSegmentationEngine("JSONEachRow", &fileSegmentationEngineJSONEachRowImpl); + factory.registerFileSegmentationEngine("JSONStringsEachRow", &fileSegmentationEngineJSONEachRowImpl); } } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h index a0a4b735a3e..29a6ce6ecb8 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h @@ -20,7 +20,12 @@ class ReadBuffer; class JSONEachRowRowInputFormat : public IRowInputFormat { public: - JSONEachRowRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_); + JSONEachRowRowInputFormat( + ReadBuffer & in_, + const Block & header_, + Params params_, + const FormatSettings & format_settings_, + bool yield_strings_); String getName() const override { return "JSONEachRowRowInputFormat"; } @@ -75,6 +80,8 @@ private: bool data_in_square_brackets = false; bool allow_new_rows = true; + + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp index 910a9710de3..069499d99c1 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp @@ -8,8 +8,13 @@ namespace DB { -JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_) - : IRowOutputFormat(header_, out_, callback), settings(settings_) +JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_) + : IRowOutputFormat(header_, out_, callback), settings(settings_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -27,7 +32,17 @@ void JSONEachRowRowOutputFormat::writeField(const IColumn & column, const IDataT { writeString(fields[field_number], out); writeChar(':', out); - type.serializeAsTextJSON(column, row_num, out, settings); + + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), out, settings); + } + else + type.serializeAsTextJSON(column, row_num, out, settings); + ++field_number; } @@ -59,7 +74,16 @@ void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings); + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONStringsEachRow", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h index d2b6937cd01..5346a1ab19f 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h @@ -15,7 +15,12 @@ namespace DB class JSONEachRowRowOutputFormat : public IRowOutputFormat { public: - JSONEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_); + JSONEachRowRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_); String getName() const override { return "JSONEachRowRowOutputFormat"; } @@ -35,6 +40,9 @@ private: Names fields; FormatSettings settings; + +protected: + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp index a611b5a129b..35720df9672 100644 --- a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp @@ -36,7 +36,16 @@ void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factor FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings); + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONStringsEachRowWithProgress", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index b3255f2894e..7dd7eb9953a 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -7,8 +7,13 @@ namespace DB { -JSONRowOutputFormat::JSONRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_) - : IRowOutputFormat(header, out_, callback), settings(settings_) +JSONRowOutputFormat::JSONRowOutputFormat( + WriteBuffer & out_, + const Block & header, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_) + : IRowOutputFormat(header, out_, callback), settings(settings_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); @@ -71,7 +76,17 @@ void JSONRowOutputFormat::writeField(const IColumn & column, const IDataType & t writeCString("\t\t\t", *ostr); writeString(fields[field_number].name, *ostr); writeCString(": ", *ostr); - type.serializeAsTextJSON(column, row_num, *ostr, settings); + + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), *ostr, settings); + } + else + type.serializeAsTextJSON(column, row_num, *ostr, settings); + ++field_number; } @@ -80,7 +95,17 @@ void JSONRowOutputFormat::writeTotalsField(const IColumn & column, const IDataTy writeCString("\t\t", *ostr); writeString(fields[field_number].name, *ostr); writeCString(": ", *ostr); - type.serializeAsTextJSON(column, row_num, *ostr, settings); + + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), *ostr, settings); + } + else + type.serializeAsTextJSON(column, row_num, *ostr, settings); + ++field_number; } @@ -249,7 +274,16 @@ void registerOutputFormatProcessorJSON(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings); + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONStrings", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.h b/src/Processors/Formats/Impl/JSONRowOutputFormat.h index f9aea3a3e8b..4e9cceb717e 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.h @@ -16,7 +16,12 @@ namespace DB class JSONRowOutputFormat : public IRowOutputFormat { public: - JSONRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_); + JSONRowOutputFormat( + WriteBuffer & out_, + const Block & header, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_); String getName() const override { return "JSONRowOutputFormat"; } @@ -78,6 +83,8 @@ protected: Progress progress; Stopwatch watch; FormatSettings settings; + + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp deleted file mode 100644 index fff44a204fb..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp +++ /dev/null @@ -1,245 +0,0 @@ -#include -#include - -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INCORRECT_DATA; - extern const int CANNOT_READ_ALL_DATA; -} - - -JSONStringsEachRowRowInputFormat::JSONStringsEachRowRowInputFormat(ReadBuffer & in_, - const Block & header_, - Params params_, - const FormatSettings & format_settings_, - bool with_names_) - : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_) -{ - const auto & sample = getPort().getHeader(); - size_t num_columns = sample.columns(); - - data_types.resize(num_columns); - column_indexes_by_names.reserve(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - const auto & column_info = sample.getByPosition(i); - - data_types[i] = column_info.type; - column_indexes_by_names.emplace(column_info.name, i); - } -} - -void JSONStringsEachRowRowInputFormat::resetParser() -{ - IRowInputFormat::resetParser(); - column_indexes_for_input_fields.clear(); - not_seen_columns.clear(); -} - -void JSONStringsEachRowRowInputFormat::readPrefix() -{ - /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. - skipBOMIfExists(in); - - if (with_names) - { - size_t num_columns = getPort().getHeader().columns(); - read_columns.assign(num_columns, false); - - assertChar('[', in); - do - { - skipWhitespaceIfAny(in); - String column_name; - readJSONString(column_name, in); - addInputColumn(column_name); - skipWhitespaceIfAny(in); - } - while (checkChar(',', in)); - assertChar(']', in); - skipEndOfLine(); - - /// Type checking - assertChar('[', in); - for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i) - { - skipWhitespaceIfAny(in); - String data_type; - readJSONString(data_type, in); - - if (column_indexes_for_input_fields[i] && - data_types[*column_indexes_for_input_fields[i]]->getName() != data_type) - { - throw Exception( - "Type of '" + getPort().getHeader().getByPosition(*column_indexes_for_input_fields[i]).name - + "' must be " + data_types[*column_indexes_for_input_fields[i]]->getName() + - ", not " + data_type, - ErrorCodes::INCORRECT_DATA - ); - } - - if (i != column_indexes_for_input_fields.size() - 1) - assertChar(',', in); - skipWhitespaceIfAny(in); - } - assertChar(']', in); - } - else - { - size_t num_columns = getPort().getHeader().columns(); - read_columns.assign(num_columns, true); - column_indexes_for_input_fields.resize(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - column_indexes_for_input_fields[i] = i; - } - } - - for (size_t i = 0; i < read_columns.size(); ++i) - { - if (!read_columns[i]) - { - not_seen_columns.emplace_back(i); - } - } -} - -void JSONStringsEachRowRowInputFormat::addInputColumn(const String & column_name) -{ - names_of_columns.emplace_back(column_name); - - const auto column_it = column_indexes_by_names.find(column_name); - if (column_it == column_indexes_by_names.end()) - { - if (format_settings.skip_unknown_fields) - { - column_indexes_for_input_fields.push_back(std::nullopt); - return; - } - - throw Exception( - "Unknown field found in JSONStringsEachRow header: '" + column_name + "' " + - "at position " + std::to_string(column_indexes_for_input_fields.size()) + - "\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed", - ErrorCodes::INCORRECT_DATA - ); - } - - const auto column_index = column_it->second; - - if (read_columns[column_index]) - throw Exception("Duplicate field found while parsing JSONStringsEachRow header: " + column_name, ErrorCodes::INCORRECT_DATA); - - read_columns[column_index] = true; - column_indexes_for_input_fields.emplace_back(column_index); -} - -bool JSONStringsEachRowRowInputFormat::readRow(DB::MutableColumns &columns, DB::RowReadExtension &ext) -{ - skipEndOfLine(); - - if (in.eof()) - return false; - - size_t num_columns = columns.size(); - - read_columns.assign(num_columns, false); - - assertChar('[', in); - for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column) - { - const auto & table_column = column_indexes_for_input_fields[file_column]; - if (table_column) - { - readField(*table_column, columns); - } - else - { - skipJSONField(in, StringRef(names_of_columns[file_column])); - } - - skipWhitespaceIfAny(in); - if (in.eof()) - throw Exception("Unexpected end of stream while parsing JSONStringsEachRow format", ErrorCodes::CANNOT_READ_ALL_DATA); - if (file_column + 1 != column_indexes_for_input_fields.size()) - { - assertChar(',', in); - skipWhitespaceIfAny(in); - } - } - assertChar(']', in); - - for (const auto & name : not_seen_columns) - columns[name]->insertDefault(); - - ext.read_columns = read_columns; - return true; -} - -void JSONStringsEachRowRowInputFormat::skipEndOfLine() -{ - skipWhitespaceIfAny(in); - if (!in.eof() && (*in.position() == ',' || *in.position() == ';')) - ++in.position(); - - skipWhitespaceIfAny(in); -} - -void JSONStringsEachRowRowInputFormat::readField(size_t index, MutableColumns & columns) -{ - try - { - read_columns[index] = true; - const auto & type = data_types[index]; - - String str; - readJSONString(str, in); - - ReadBufferFromString buf(str); - - type->deserializeAsWholeText(*columns[index], buf, format_settings); - } - catch (Exception & e) - { - e.addMessage("(while read the value of key " + getPort().getHeader().getByPosition(index).name + ")"); - throw; - } -} - -void JSONStringsEachRowRowInputFormat::syncAfterError() -{ - skipToUnescapedNextLineOrEOF(in); -} - -void registerInputFormatProcessorJSONStringsEachRow(FormatFactory & factory) -{ - factory.registerInputFormatProcessor("JSONStringsEachRow", []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, std::move(params), settings, false); - }); - - factory.registerInputFormatProcessor("JSONStringsEachRowWithNamesAndTypes", []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, std::move(params), settings, true); - }); -} - -} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h deleted file mode 100644 index ec0a0f7bad9..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h +++ /dev/null @@ -1,54 +0,0 @@ -#pragma once - -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -class ReadBuffer; - -/** A stream for reading data in JSONStringsEachRow and JSONStringsEachRowWithNamesAndTypes formats -*/ -class JSONStringsEachRowRowInputFormat : public IRowInputFormat -{ -public: - JSONStringsEachRowRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_, bool with_names_); - - String getName() const override { return "JSONStringsEachRowRowInputFormat"; } - - - void readPrefix() override; - bool readRow(MutableColumns & columns, RowReadExtension & ext) override; - bool allowSyncAfterError() const override { return true; } - void syncAfterError() override; - void resetParser() override; - -private: - void addInputColumn(const String & column_name); - void skipEndOfLine(); - void readField(size_t index, MutableColumns & columns); - - const FormatSettings format_settings; - - using IndexesMap = std::unordered_map; - IndexesMap column_indexes_by_names; - - using OptionalIndexes = std::vector>; - OptionalIndexes column_indexes_for_input_fields; - - DataTypes data_types; - std::vector read_columns; - std::vector not_seen_columns; - - /// This is for the correct exceptions in skipping unknown fields. - std::vector names_of_columns; - - bool with_names; -}; - -} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp deleted file mode 100644 index 75007ea236e..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp +++ /dev/null @@ -1,117 +0,0 @@ -#include -#include -#include -#include - - -namespace DB -{ - - -JSONStringsEachRowRowOutputFormat::JSONStringsEachRowRowOutputFormat(WriteBuffer & out_, - const Block & header_, - FormatFactory::WriteCallback callback, - const FormatSettings & settings_, - bool with_names_) - : IRowOutputFormat(header_, out_, callback), settings(settings_), with_names(with_names_) -{ - const auto & sample = getPort(PortKind::Main).getHeader(); - NamesAndTypesList columns(sample.getNamesAndTypesList()); - fields.assign(columns.begin(), columns.end()); -} - - -void JSONStringsEachRowRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) -{ - WriteBufferFromOwnString buf; - - type.serializeAsText(column, row_num, buf, settings); - writeJSONString(buf.str(), out, settings); -} - - -void JSONStringsEachRowRowOutputFormat::writeFieldDelimiter() -{ - writeCString(", ", out); -} - - -void JSONStringsEachRowRowOutputFormat::writeRowStartDelimiter() -{ - writeChar('[', out); -} - - -void JSONStringsEachRowRowOutputFormat::writeRowEndDelimiter() -{ - writeCString("]\n", out); -} - -void JSONStringsEachRowRowOutputFormat::writeTotals(const Columns & columns, size_t row_num) -{ - writeChar('\n', out); - size_t num_columns = columns.size(); - writeChar('[', out); - for (size_t i = 0; i < num_columns; ++i) - { - if (i != 0) - JSONStringsEachRowRowOutputFormat::writeFieldDelimiter(); - - JSONStringsEachRowRowOutputFormat::writeField(*columns[i], *types[i], row_num); - } - writeCString("]\n", out); -} - -void JSONStringsEachRowRowOutputFormat::writePrefix() -{ - if (with_names) - { - writeChar('[', out); - for (size_t i = 0; i < fields.size(); ++i) - { - writeChar('\"', out); - writeString(fields[i].name, out); - writeChar('\"', out); - if (i != fields.size() - 1) - writeCString(", ", out); - } - writeCString("]\n[", out); - for (size_t i = 0; i < fields.size(); ++i) - { - writeJSONString(fields[i].type->getName(), out, settings); - if (i != fields.size() - 1) - writeCString(", ", out); - } - writeCString("]\n", out); - } -} - -void JSONStringsEachRowRowOutputFormat::consumeTotals(DB::Chunk chunk) -{ - if (with_names) - IRowOutputFormat::consumeTotals(std::move(chunk)); -} - -void registerOutputFormatProcessorJSONStringsEachRow(FormatFactory & factory) -{ - factory.registerOutputFormatProcessor("JSONStringsEachRow", []( - WriteBuffer & buf, - const Block & sample, - FormatFactory::WriteCallback callback, - const FormatSettings & format_settings) - { - return std::make_shared(buf, sample, callback, format_settings, false); - }); - - factory.registerOutputFormatProcessor("JSONStringsEachRowWithNamesAndTypes", []( - WriteBuffer &buf, - const Block &sample, - FormatFactory::WriteCallback callback, - const FormatSettings &format_settings) - { - return std::make_shared(buf, sample, callback, format_settings, true); - }); -} - - -} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h deleted file mode 100644 index 1d43a333da1..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h +++ /dev/null @@ -1,45 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -/** The stream for outputting data in JSON format, by object per line. - * Does not validate UTF-8. - */ -class JSONStringsEachRowRowOutputFormat : public IRowOutputFormat -{ -public: - JSONStringsEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_, bool with_names); - - String getName() const override { return "JSONStringsEachRowRowOutputFormat"; } - - void writePrefix() override; - - void writeBeforeTotals() override {} - void writeTotals(const Columns & columns, size_t row_num) override; - void writeAfterTotals() override {} - - void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; - void writeFieldDelimiter() override; - void writeRowStartDelimiter() override; - void writeRowEndDelimiter() override; - -protected: - void consumeTotals(Chunk) override; - /// No extremes. - void consumeExtremes(Chunk) override {} - -private: - FormatSettings settings; - - NamesAndTypes fields; - - bool with_names; -}; -} diff --git a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp deleted file mode 100644 index 6ccb315f73f..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp +++ /dev/null @@ -1,93 +0,0 @@ -#include -#include - -#include - - -namespace DB -{ - -JSONStringsRowOutputFormat::JSONStringsRowOutputFormat( - WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_) - : JSONRowOutputFormat(out_, header, callback, settings_) -{ -} - - -void JSONStringsRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) -{ - WriteBufferFromOwnString buf; - - type.serializeAsText(column, row_num, buf, settings); - writeJSONString(buf.str(), *ostr, settings); - ++field_number; -} - - -void JSONStringsRowOutputFormat::writeFieldDelimiter() -{ - writeCString(", ", *ostr); -} - -void JSONStringsRowOutputFormat::writeTotalsFieldDelimiter() -{ - writeCString(",", *ostr); -} - - -void JSONStringsRowOutputFormat::writeRowStartDelimiter() -{ - writeCString("\t\t[", *ostr); -} - - -void JSONStringsRowOutputFormat::writeRowEndDelimiter() -{ - writeChar(']', *ostr); - field_number = 0; - ++row_count; -} - -void JSONStringsRowOutputFormat::writeBeforeTotals() -{ - writeCString(",\n", *ostr); - writeChar('\n', *ostr); - writeCString("\t\"totals\": [", *ostr); -} - -void JSONStringsRowOutputFormat::writeAfterTotals() -{ - writeChar(']', *ostr); -} - -void JSONStringsRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num) -{ - writeCString("\t\t\"", *ostr); - writeCString(title, *ostr); - writeCString("\": [", *ostr); - - size_t extremes_columns = columns.size(); - for (size_t i = 0; i < extremes_columns; ++i) - { - if (i != 0) - writeTotalsFieldDelimiter(); - - writeField(*columns[i], *types[i], row_num); - } - - writeChar(']', *ostr); -} - -void registerOutputFormatProcessorJSONStrings(FormatFactory & factory) -{ - factory.registerOutputFormatProcessor("JSONStrings", []( - WriteBuffer & buf, - const Block & sample, - FormatFactory::WriteCallback callback, - const FormatSettings & format_settings) - { - return std::make_shared(buf, sample, callback, format_settings); - }); -} - -} diff --git a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h deleted file mode 100644 index b221bc9ee36..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h +++ /dev/null @@ -1,43 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -struct FormatSettings; - -/** The stream for outputting data in the JSONStrings format. - */ -class JSONStringsRowOutputFormat : public JSONRowOutputFormat -{ -public: - JSONStringsRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_); - - String getName() const override { return "JSONStringsRowOutputFormat"; } - - void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; - void writeFieldDelimiter() override; - void writeRowStartDelimiter() override; - void writeRowEndDelimiter() override; - - void writeBeforeTotals() override; - void writeAfterTotals() override; - -protected: - void writeExtremesElement(const char * title, const Columns & columns, size_t row_num) override; - - void writeTotalsField(const IColumn & column, const IDataType & type, size_t row_num) override - { - return writeField(column, type, row_num); - } - - void writeTotalsFieldDelimiter() override; - -}; - -} diff --git a/tests/queries/0_stateless/01446_JSONStringsEachRow.sql b/tests/queries/0_stateless/01446_JSONStringsEachRow.sql deleted file mode 100644 index f461b217fe4..00000000000 --- a/tests/queries/0_stateless/01446_JSONStringsEachRow.sql +++ /dev/null @@ -1,63 +0,0 @@ -DROP TABLE IF EXISTS test_table; -DROP TABLE IF EXISTS test_table_2; -SELECT 1; -/* Check JSONStringsEachRow Output */ -CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value; -INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c'); -SELECT * FROM test_table FORMAT JSONStringsEachRow; -SELECT 2; -/* Check Totals */ -SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRow; -SELECT 3; -/* Check JSONStringsEachRowWithNamesAndTypes Output */ -SELECT * FROM test_table FORMAT JSONStringsEachRowWithNamesAndTypes; -SELECT 4; -/* Check Totals */ -SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRowWithNamesAndTypes; -DROP TABLE IF EXISTS test_table; -SELECT 5; -/* Check JSONStringsEachRow Input */ -CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; -INSERT INTO test_table FORMAT JSONStringsEachRow ["first", "1", "2", "NULL"] ["second", "2", "null", "6"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -TRUNCATE TABLE test_table; -SELECT 6; -/* Check input_format_null_as_default = 1 */ -SET input_format_null_as_default = 1; -INSERT INTO test_table FORMAT JSONStringsEachRow ["first", "1", "2", "ᴺᵁᴸᴸ"] ["second", "2", "null", "6"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -TRUNCATE TABLE test_table; -SELECT 7; -/* Check Nested */ -CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; -INSERT INTO test_table_2 FORMAT JSONStringsEachRow ["16", "[15, 16, 17]", "['first', 'second', 'third']"]; -SELECT * FROM test_table_2 FORMAT JSONStringsEachRow; -TRUNCATE TABLE test_table_2; -SELECT 8; -/* Check JSONStringsEachRowWithNamesAndTypes Output */ -SET input_format_null_as_default = 0; -INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"]["second", "2", "null", "6"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -TRUNCATE TABLE test_table; -SELECT 9; -/* Check input_format_null_as_default = 1 */ -SET input_format_null_as_default = 1; -INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"] ["second", "2", "null", "6"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -SELECT 10; -/* Check Header */ -TRUNCATE TABLE test_table; -SET input_format_skip_unknown_fields = 1; -INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", "1", "32"]["second", "2", "64"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -SELECT 11; -TRUNCATE TABLE test_table; -INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"]["1", "2", "3"] -SELECT * FROM test_table FORMAT JSONStringsEachRowWithNamesAndTypes; -SELECT 12; -/* Check Nested */ -INSERT INTO test_table_2 FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"]["16", "[15, 16, 17]", "['first', 'second', 'third']"]; -SELECT * FROM test_table_2 FORMAT JSONStringsEachRowWithNamesAndTypes; - -DROP TABLE IF EXISTS test_table; -DROP TABLE IF EXISTS test_table_2; diff --git a/tests/queries/0_stateless/01446_json_strings_each_row.reference b/tests/queries/0_stateless/01446_json_strings_each_row.reference new file mode 100644 index 00000000000..84d41095b77 --- /dev/null +++ b/tests/queries/0_stateless/01446_json_strings_each_row.reference @@ -0,0 +1,22 @@ +1 +{"value":"1","name":"a"} +{"value":"2","name":"b"} +{"value":"3","name":"c"} +2 +{"name":"a","c":"1"} +{"name":"b","c":"1"} +{"name":"c","c":"1"} +3 +{"row":{"a":"1"}} +{"progress":{"read_rows":"1","read_bytes":"1","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} +4 +{"row":{"a":"1"}} +{"progress":{"read_rows":"1","read_bytes":"1","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} +5 +{"v1":"first","v2":"1","v3":"2","v4":"0"} +{"v1":"second","v2":"2","v3":"0","v4":"6"} +6 +{"v1":"first","v2":"1","v3":"2","v4":"0"} +{"v1":"second","v2":"2","v3":"0","v4":"6"} +7 +{"v1":"16","n.id":"[15,16,17]","n.name":"['first','second','third']"} diff --git a/tests/queries/0_stateless/01446_json_strings_each_row.sql b/tests/queries/0_stateless/01446_json_strings_each_row.sql new file mode 100644 index 00000000000..98bd3e3ab47 --- /dev/null +++ b/tests/queries/0_stateless/01446_json_strings_each_row.sql @@ -0,0 +1,38 @@ +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; +SELECT 1; +/* Check JSONStringsEachRow Output */ +CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value; +INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c'); +SELECT * FROM test_table FORMAT JSONStringsEachRow; +SELECT 2; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRow; +SELECT 3; +/* Check JSONStringsEachRowWithProgress Output */ +SELECT 1 as a FROM system.one FORMAT JSONStringsEachRowWithProgress; +SELECT 4; +/* Check Totals */ +SELECT 1 as a FROM system.one GROUP BY a WITH TOTALS ORDER BY a FORMAT JSONStringsEachRowWithProgress; +DROP TABLE IF EXISTS test_table; +SELECT 5; +/* Check JSONStringsEachRow Input */ +CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; +INSERT INTO test_table FORMAT JSONStringsEachRow {"v1": "first", "v2": "1", "v3": "2", "v4": "NULL"} {"v1": "second", "v2": "2", "v3": "null", "v4": "6"}; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 6; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONStringsEachRow {"v1": "first", "v2": "1", "v3": "2", "v4": "ᴺᵁᴸᴸ"} {"v1": "second", "v2": "2", "v3": "null", "v4": "6"}; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 7; +/* Check Nested */ +CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; +INSERT INTO test_table_2 FORMAT JSONStringsEachRow {"v1": "16", "n.id": "[15, 16, 17]", "n.name": "['first', 'second', 'third']"}; +SELECT * FROM test_table_2 FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table_2; + +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; diff --git a/tests/queries/0_stateless/01447_json_strings.reference b/tests/queries/0_stateless/01447_json_strings.reference new file mode 100644 index 00000000000..ab88e2f3696 --- /dev/null +++ b/tests/queries/0_stateless/01447_json_strings.reference @@ -0,0 +1,43 @@ +{ + "meta": + [ + { + "name": "1", + "type": "UInt8" + }, + { + "name": "'a'", + "type": "String" + }, + { + "name": "[1, 2, 3]", + "type": "Array(UInt8)" + }, + { + "name": "tuple(1, 'a')", + "type": "Tuple(UInt8, String)" + }, + { + "name": "NULL", + "type": "Nullable(Nothing)" + }, + { + "name": "nan", + "type": "Float64" + } + ], + + "data": + [ + { + "1": "1", + "'a'": "a", + "[1, 2, 3]": "[1,2,3]", + "tuple(1, 'a')": "(1,'a')", + "NULL": "ᴺᵁᴸᴸ", + "nan": "nan" + } + ], + + "rows": 1 +} diff --git a/tests/queries/0_stateless/01447_JSONStrings.sql b/tests/queries/0_stateless/01447_json_strings.sql similarity index 100% rename from tests/queries/0_stateless/01447_JSONStrings.sql rename to tests/queries/0_stateless/01447_json_strings.sql diff --git a/tests/queries/0_stateless/01446_JSONStringsEachRow.reference b/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference similarity index 100% rename from tests/queries/0_stateless/01446_JSONStringsEachRow.reference rename to tests/queries/0_stateless/01448_json_compact_strings_each_row.reference diff --git a/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql b/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql new file mode 100644 index 00000000000..c271de88434 --- /dev/null +++ b/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql @@ -0,0 +1,63 @@ +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; +SELECT 1; +/* Check JSONCompactStringsEachRow Output */ +CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value; +INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c'); +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +SELECT 2; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactStringsEachRow; +SELECT 3; +/* Check JSONCompactStringsEachRowWithNamesAndTypes Output */ +SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes; +SELECT 4; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactStringsEachRowWithNamesAndTypes; +DROP TABLE IF EXISTS test_table; +SELECT 5; +/* Check JSONCompactStringsEachRow Input */ +CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; +INSERT INTO test_table FORMAT JSONCompactStringsEachRow ["first", "1", "2", "NULL"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 6; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONCompactStringsEachRow ["first", "1", "2", "ᴺᵁᴸᴸ"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 7; +/* Check Nested */ +CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; +INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRow ["16", "[15, 16, 17]", "['first', 'second', 'third']"]; +SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRow; +TRUNCATE TABLE test_table_2; +SELECT 8; +/* Check JSONCompactStringsEachRowWithNamesAndTypes Output */ +SET input_format_null_as_default = 0; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"]["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 9; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +SELECT 10; +/* Check Header */ +TRUNCATE TABLE test_table; +SET input_format_skip_unknown_fields = 1; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", "1", "32"]["second", "2", "64"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +SELECT 11; +TRUNCATE TABLE test_table; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"]["1", "2", "3"] +SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes; +SELECT 12; +/* Check Nested */ +INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"]["16", "[15, 16, 17]", "['first', 'second', 'third']"]; +SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRowWithNamesAndTypes; + +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; diff --git a/tests/queries/0_stateless/01447_JSONStrings.reference b/tests/queries/0_stateless/01449_json_compact_strings.reference similarity index 100% rename from tests/queries/0_stateless/01447_JSONStrings.reference rename to tests/queries/0_stateless/01449_json_compact_strings.reference diff --git a/tests/queries/0_stateless/01449_json_compact_strings.sql b/tests/queries/0_stateless/01449_json_compact_strings.sql new file mode 100644 index 00000000000..5b676e30347 --- /dev/null +++ b/tests/queries/0_stateless/01449_json_compact_strings.sql @@ -0,0 +1,10 @@ +SET output_format_write_statistics = 0; + +SELECT + 1, + 'a', + [1, 2, 3], + (1, 'a'), + null, + nan +FORMAT JSONCompactStrings; From b08056fa8c0f84670bab96b5643dd36850db0d8a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Sep 2020 11:18:50 +0300 Subject: [PATCH 033/298] Better selection of Merges with TTL --- src/Storages/MergeTree/MergeList.cpp | 2 ++ src/Storages/MergeTree/MergeList.h | 2 ++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 32 +++++++++++++++---- .../MergeTree/MergeTreeDataMergerMutator.h | 7 ++-- src/Storages/MergeTree/MergeTreeSettings.h | 4 ++- src/Storages/MergeTree/MergeType.cpp | 27 ++++++++++++++++ src/Storages/MergeTree/MergeType.h | 17 ++++++++++ .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 13 ++++++++ .../MergeTree/ReplicatedMergeTreeLogEntry.h | 2 ++ .../MergeTree/ReplicatedMergeTreeQueue.cpp | 11 +++++-- .../MergeTree/ReplicatedMergeTreeQueue.h | 9 +++++- src/Storages/StorageMergeTree.cpp | 10 ++++-- src/Storages/StorageReplicatedMergeTree.cpp | 31 +++++++++++------- src/Storages/StorageReplicatedMergeTree.h | 3 +- src/Storages/System/StorageSystemMerges.cpp | 2 ++ 15 files changed, 145 insertions(+), 27 deletions(-) create mode 100644 src/Storages/MergeTree/MergeType.cpp create mode 100644 src/Storages/MergeTree/MergeType.h diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index e9d955f5395..5e7b7046c85 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -21,6 +21,7 @@ MergeListElement::MergeListElement(const std::string & database_, const std::str , result_data_version{future_part.part_info.getDataVersion()} , num_parts{future_part.parts.size()} , thread_id{getThreadId()} + , merge_type{toString(future_part.merge_type)} { for (const auto & source_part : future_part.parts) { @@ -70,6 +71,7 @@ MergeInfo MergeListElement::getInfo() const res.columns_written = columns_written.load(std::memory_order_relaxed); res.memory_usage = memory_tracker.get(); res.thread_id = thread_id; + res.merge_type = merge_type; for (const auto & source_part_name : source_part_names) res.source_part_names.emplace_back(source_part_name); diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 4ee8a75a868..e6ae0407ec0 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -45,6 +45,7 @@ struct MergeInfo UInt64 columns_written; UInt64 memory_usage; UInt64 thread_id; + std::string merge_type; }; struct FutureMergedMutatedPart; @@ -88,6 +89,7 @@ struct MergeListElement : boost::noncopyable UInt64 thread_id; + const std::string merge_type; MergeListElement(const std::string & database, const std::string & table, const FutureMergedMutatedPart & future_part); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 673ad02bfb6..a0ab7866402 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -158,15 +158,15 @@ MergeTreeDataMergerMutator::MergeTreeDataMergerMutator(MergeTreeData & data_, si } -UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge() +UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(MergeType merge_type) { size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); - return getMaxSourcePartsSizeForMerge(background_pool_size, busy_threads_in_pool == 0 ? 0 : busy_threads_in_pool - 1); /// 1 is current thread + return getMaxSourcePartsSizeForMerge(background_pool_size, busy_threads_in_pool == 0 ? 0 : busy_threads_in_pool - 1, merge_type); /// 1 is current thread } -UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used) +UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used, MergeType merge_type) { if (pool_used > pool_size) throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); @@ -178,14 +178,21 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz /// One entry is probably the entry where this function is executed. /// This will protect from bad settings. + + size_t lowering_setting; + if (merge_type == MergeType::TTL_DELETE) + lowering_setting = data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge_with_ttl; + else + lowering_setting = data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge; + UInt64 max_size = 0; - if (pool_used <= 1 || free_entries >= data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge) + if (pool_used <= 1 || free_entries >= lowering_setting) max_size = data_settings->max_bytes_to_merge_at_max_space_in_pool; else max_size = interpolateExponential( data_settings->max_bytes_to_merge_at_min_space_in_pool, data_settings->max_bytes_to_merge_at_max_space_in_pool, - static_cast(free_entries) / data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge); + static_cast(free_entries) / lowering_setting); return std::min(max_size, static_cast(data.getStoragePolicy()->getMaxUnreservedFreeSpace() / DISK_USAGE_COEFFICIENT_TO_SELECT)); } @@ -213,6 +220,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( bool aggressive, size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge_callback, + size_t max_total_size_to_merge_with_ttl, String * out_disable_reason) { MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector(); @@ -284,7 +292,9 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( current_time, data_settings->merge_with_ttl_timeout, data_settings->ttl_only_drop_parts); - parts_to_merge = merge_selector.select(partitions, max_total_size_to_merge); + + parts_to_merge = merge_selector.select(partitions, max_total_size_to_merge_with_ttl); + future_part.merge_type = MergeType::TTL_DELETE; } if (parts_to_merge.empty()) @@ -306,6 +316,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( *out_disable_reason = "There is no need to merge parts according to merge selector algorithm"; return false; } + future_part.merge_type = MergeType::NORMAL; } MergeTreeData::DataPartsVector parts; @@ -385,6 +396,12 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( LOG_DEBUG(log, "Selected {} parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name); future_part.assign(std::move(parts)); + + if (final) + future_part.merge_type = MergeType::FINAL; + else + future_part.merge_type = MergeType::NORMAL; + available_disk_space -= required_disk_space; return true; } @@ -634,6 +651,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor new_data_part->partition.assign(future_part.getPartition()); new_data_part->is_temp = true; + if (future_part.merge_type == MergeType::TTL_DELETE && ttl_merges_blocker.isCancelled()) + throw Exception("Cancelled merging parts with expired TTL", ErrorCodes::ABORTED); + bool need_remove_expired_values = false; for (const auto & part : parts) new_data_part->ttl_infos.update(part->ttl_infos); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index d5798fe3582..086a2a9cae2 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -22,6 +23,7 @@ struct FutureMergedMutatedPart MergeTreeDataPartType type; MergeTreePartInfo part_info; MergeTreeData::DataPartsVector parts; + MergeType merge_type = MergeType::NORMAL; const MergeTreePartition & getPartition() const { return parts.front()->partition; } @@ -57,12 +59,12 @@ public: /** Get maximum total size of parts to do merge, at current moment of time. * It depends on number of free threads in background_pool and amount of free space in disk. */ - UInt64 getMaxSourcePartsSizeForMerge(); + UInt64 getMaxSourcePartsSizeForMerge(MergeType merge_type); /** For explicitly passed size of pool and number of used tasks. * This method could be used to calculate threshold depending on number of tasks in replication queue. */ - UInt64 getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used); + UInt64 getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used, MergeType merge_type); /** Get maximum total size of parts to do mutation, at current moment of time. * It depends only on amount of free space in disk. @@ -81,6 +83,7 @@ public: bool aggressive, size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge, + size_t max_total_size_to_merge_with_ttl, String * out_disable_reason = nullptr); /** Select all the parts in the specified partition for merge, if possible. diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 085c441aa90..e5707ff837c 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -33,8 +33,10 @@ struct Settings; M(UInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \ M(UInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ M(UInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + M(UInt64, max_replicated_merges_with_ttl_in_queue, 1, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ + M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge_with_ttl, 14, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ M(Seconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(Seconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(Seconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ @@ -83,7 +85,7 @@ struct Settings; M(UInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).", 0) \ M(UInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).", 0) \ M(UInt64, min_index_granularity_bytes, 1024, "Minimum amount of bytes in single granule.", 1024) \ - M(Int64, merge_with_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with TTL can be repeated.", 0) \ + M(Int64, merge_with_ttl_timeout, 0, "Minimal time in seconds, when merge with TTL can be repeated.", 0) \ M(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \ M(Bool, write_final_mark, 1, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)", 0) \ M(Bool, enable_mixed_granularity_parts, 1, "Enable parts with adaptive and non adaptive granularity", 0) \ diff --git a/src/Storages/MergeTree/MergeType.cpp b/src/Storages/MergeTree/MergeType.cpp new file mode 100644 index 00000000000..b58a0de4093 --- /dev/null +++ b/src/Storages/MergeTree/MergeType.cpp @@ -0,0 +1,27 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +String toString(MergeType merge_type) +{ + switch (merge_type) + { + case MergeType::NORMAL: + return "NORMAL"; + case MergeType::FINAL: + return "FINAL"; + case MergeType::TTL_DELETE: + return "TTL_DELETE"; + } + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeType {}", static_cast(merge_type)); +} + +} diff --git a/src/Storages/MergeTree/MergeType.h b/src/Storages/MergeTree/MergeType.h new file mode 100644 index 00000000000..5d9abaa61b3 --- /dev/null +++ b/src/Storages/MergeTree/MergeType.h @@ -0,0 +1,17 @@ +#pragma once + +#include + +namespace DB +{ + +enum class MergeType +{ + NORMAL, + FINAL, + TTL_DELETE, +}; + +String toString(MergeType merge_type); + +} diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index af6d980ad98..de8dd7f6097 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -36,6 +36,8 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const out << s << '\n'; out << "into\n" << new_part_name; out << "\ndeduplicate: " << deduplicate; + if (merge_type != MergeType::NORMAL) + out <<"\nmerge_type: " << static_cast(merge_type); break; case DROP_RANGE: @@ -149,7 +151,18 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) } in >> new_part_name; if (format_version >= 4) + { in >> "\ndeduplicate: " >> deduplicate; + in >> "\n"; + if (in.eof()) + trailing_newline_found = true; + else if (checkString("merge_type: ", in)) + { + UInt64 value; + in >> value; + merge_type = static_cast(value); + } + } } else if (type_str == "drop" || type_str == "detach") { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index ae5fad0b83c..bea796ce015 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -79,6 +80,7 @@ struct ReplicatedMergeTreeLogEntryData Strings source_parts; bool deduplicate = false; /// Do deduplicate on merge + MergeType merge_type = MergeType::NORMAL; String column_name; String index_name; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 8e2c3752212..c9b366a9ec8 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1061,7 +1061,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( return false; } - UInt64 max_source_parts_size = entry.type == LogEntry::MERGE_PARTS ? merger_mutator.getMaxSourcePartsSizeForMerge() + UInt64 max_source_parts_size = entry.type == LogEntry::MERGE_PARTS ? merger_mutator.getMaxSourcePartsSizeForMerge(entry.merge_type) : merger_mutator.getMaxSourcePartSizeForMutation(); /** If there are enough free threads in background pool to do large merges (maximal size of merge is allowed), * then ignore value returned by getMaxSourcePartsSizeForMerge() and execute merge of any size, @@ -1312,21 +1312,26 @@ bool ReplicatedMergeTreeQueue::processEntry( } -std::pair ReplicatedMergeTreeQueue::countMergesAndPartMutations() const +ReplicatedMergeTreeQueue::OperationsInQueue ReplicatedMergeTreeQueue::countMergesAndPartMutations() const { std::lock_guard lock(state_mutex); size_t count_merges = 0; size_t count_mutations = 0; + size_t count_merges_with_ttl = 0; for (const auto & entry : queue) { if (entry->type == ReplicatedMergeTreeLogEntry::MERGE_PARTS) + { ++count_merges; + if (entry->merge_type == MergeType::TTL_DELETE) + ++count_merges_with_ttl; + } else if (entry->type == ReplicatedMergeTreeLogEntry::MUTATE_PART) ++count_mutations; } - return std::make_pair(count_merges, count_mutations); + return OperationsInQueue{count_merges, count_mutations, count_merges_with_ttl}; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 76f84da1ae8..c724701f1ff 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -46,6 +46,13 @@ private: } }; + struct OperationsInQueue + { + size_t merges = 0; + size_t mutations = 0; + size_t merges_with_ttl = 0; + }; + /// To calculate min_unprocessed_insert_time, max_processed_insert_time, for which the replica lag is calculated. using InsertsByTime = std::set; @@ -325,7 +332,7 @@ public: bool processEntry(std::function get_zookeeper, LogEntryPtr & entry, const std::function func); /// Count the number of merges and mutations of single parts in the queue. - std::pair countMergesAndPartMutations() const; + OperationsInQueue countMergesAndPartMutations() const; /// Count the total number of active mutations. size_t countMutations() const; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7e4318a32f6..05f2f5254f0 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -650,9 +650,14 @@ bool StorageMergeTree::merge( if (partition_id.empty()) { - UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(); + UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(MergeType::NORMAL); + UInt64 max_source_parts_size_with_ttl = 0; + + if (!aggressive) + max_source_parts_size_with_ttl = merger_mutator.getMaxSourcePartsSizeForMerge(MergeType::TTL_DELETE); + if (max_source_parts_size > 0) - selected = merger_mutator.selectPartsToMerge(future_part, aggressive, max_source_parts_size, can_merge, out_disable_reason); + selected = merger_mutator.selectPartsToMerge(future_part, aggressive, max_source_parts_size, can_merge, max_source_parts_size_with_ttl, out_disable_reason); else if (out_disable_reason) *out_disable_reason = "Current value of max_source_parts_size is zero"; } @@ -724,6 +729,7 @@ bool StorageMergeTree::merge( try { + std::cerr << "FUTURE PART MERGE TYPE:" << toString(future_part.merge_type) << std::endl; new_part = merger_mutator.mergePartsToTemporaryPart( future_part, metadata_snapshot, *merge_entry, table_lock_holder, time(nullptr), merging_tagger->reserved_space, deduplicate); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6058632d220..1c880c8c790 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2514,31 +2514,38 @@ void StorageReplicatedMergeTree::mergeSelectingTask() /// and in the same time, many small parts could be created and won't be merged. auto merges_and_mutations_queued = queue.countMergesAndPartMutations(); - size_t merges_and_mutations_sum = merges_and_mutations_queued.first + merges_and_mutations_queued.second; + size_t merges_and_mutations_sum = merges_and_mutations_queued.merges + merges_and_mutations_queued.mutations; if (merges_and_mutations_sum >= storage_settings_ptr->max_replicated_merges_in_queue) { LOG_TRACE(log, "Number of queued merges ({}) and part mutations ({})" " is greater than max_replicated_merges_in_queue ({}), so won't select new parts to merge or mutate.", - merges_and_mutations_queued.first, - merges_and_mutations_queued.second, + merges_and_mutations_queued.merges, + merges_and_mutations_queued.mutations, storage_settings_ptr->max_replicated_merges_in_queue); } else { UInt64 max_source_parts_size_for_merge = merger_mutator.getMaxSourcePartsSizeForMerge( - storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum); + storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum, MergeType::NORMAL); + + UInt64 max_source_parts_size_for_merge_with_ttl = 0; + if (merges_and_mutations_queued.merges_with_ttl < storage_settings_ptr->max_replicated_merges_with_ttl_in_queue) + max_source_parts_size_for_merge_with_ttl = merger_mutator.getMaxSourcePartsSizeForMerge( + storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum, MergeType::TTL_DELETE); + UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); FutureMergedMutatedPart future_merged_part; if (max_source_parts_size_for_merge > 0 && - merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, nullptr)) + merger_mutator.selectPartsToMerge(future_merged_part, false, + max_source_parts_size_for_merge, merge_pred, max_source_parts_size_for_merge_with_ttl, nullptr)) { create_result = createLogEntryToMergeParts(zookeeper, future_merged_part.parts, - future_merged_part.name, future_merged_part.type, deduplicate, nullptr, merge_pred.getVersion()); + future_merged_part.name, future_merged_part.type, deduplicate, nullptr, merge_pred.getVersion(), future_merged_part.merge_type); } /// If there are many mutations in queue, it may happen, that we cannot enqueue enough merges to merge all new parts else if (max_source_part_size_for_mutation > 0 && queue.countMutations() > 0 - && merges_and_mutations_queued.second < storage_settings_ptr->max_replicated_mutations_in_queue) + && merges_and_mutations_queued.mutations < storage_settings_ptr->max_replicated_mutations_in_queue) { /// Choose a part to mutate. DataPartsVector data_parts = getDataPartsVector(); @@ -2617,7 +2624,8 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c const MergeTreeDataPartType & merged_part_type, bool deduplicate, ReplicatedMergeTreeLogEntryData * out_log_entry, - int32_t log_version) + int32_t log_version, + MergeType merge_type) { std::vector> exists_futures; exists_futures.reserve(parts.size()); @@ -2649,6 +2657,7 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c entry.source_replica = replica_name; entry.new_part_name = merged_name; entry.new_part_type = merged_part_type; + entry.merge_type = merge_type; entry.deduplicate = deduplicate; entry.create_time = time(nullptr); @@ -3584,7 +3593,7 @@ bool StorageReplicatedMergeTree::optimize( CreateMergeEntryResult create_result = createLogEntryToMergeParts( zookeeper, future_merged_part.parts, future_merged_part.name, future_merged_part.type, deduplicate, - &merge_entry, can_merge.getVersion()); + &merge_entry, can_merge.getVersion(), future_merged_part.merge_type); if (create_result == CreateMergeEntryResult::MissingPart) return handle_noop("Can't create merge queue node in ZooKeeper, because some parts are missing"); @@ -3614,7 +3623,7 @@ bool StorageReplicatedMergeTree::optimize( if (!partition) { selected = merger_mutator.selectPartsToMerge( - future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, &disable_reason); + future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, 0, &disable_reason); } else { @@ -3639,7 +3648,7 @@ bool StorageReplicatedMergeTree::optimize( CreateMergeEntryResult create_result = createLogEntryToMergeParts( zookeeper, future_merged_part.parts, future_merged_part.name, future_merged_part.type, deduplicate, - &merge_entry, can_merge.getVersion()); + &merge_entry, can_merge.getVersion(), future_merged_part.merge_type); if (create_result == CreateMergeEntryResult::MissingPart) return handle_noop("Can't create merge queue node in ZooKeeper, because some parts are missing"); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index e9395f20f3f..2bc9265331d 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -450,7 +450,8 @@ private: const MergeTreeDataPartType & merged_part_type, bool deduplicate, ReplicatedMergeTreeLogEntryData * out_log_entry, - int32_t log_version); + int32_t log_version, + MergeType merge_type); CreateMergeEntryResult createLogEntryToMutatePart( const IMergeTreeDataPart & part, diff --git a/src/Storages/System/StorageSystemMerges.cpp b/src/Storages/System/StorageSystemMerges.cpp index 39d22bd00ca..b3bd8f77a89 100644 --- a/src/Storages/System/StorageSystemMerges.cpp +++ b/src/Storages/System/StorageSystemMerges.cpp @@ -30,6 +30,7 @@ NamesAndTypesList StorageSystemMerges::getNamesAndTypes() {"columns_written", std::make_shared()}, {"memory_usage", std::make_shared()}, {"thread_id", std::make_shared()}, + {"merge_type", std::make_shared()}, }; } @@ -65,6 +66,7 @@ void StorageSystemMerges::fillData(MutableColumns & res_columns, const Context & res_columns[i++]->insert(merge.columns_written); res_columns[i++]->insert(merge.memory_usage); res_columns[i++]->insert(merge.thread_id); + res_columns[i++]->insert(merge.merge_type); } } From fbb37c37df6c428579130772151492209742008e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Sep 2020 11:28:46 +0300 Subject: [PATCH 034/298] Simplier interface --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 10 +++++----- src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 6 +++--- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index a0ab7866402..31d566c4e0e 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -158,15 +158,15 @@ MergeTreeDataMergerMutator::MergeTreeDataMergerMutator(MergeTreeData & data_, si } -UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(MergeType merge_type) +UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(bool with_ttl) const { size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); - return getMaxSourcePartsSizeForMerge(background_pool_size, busy_threads_in_pool == 0 ? 0 : busy_threads_in_pool - 1, merge_type); /// 1 is current thread + return getMaxSourcePartsSizeForMerge(background_pool_size, busy_threads_in_pool == 0 ? 0 : busy_threads_in_pool - 1, with_ttl); /// 1 is current thread } -UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used, MergeType merge_type) +UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used, bool with_ttl) const { if (pool_used > pool_size) throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); @@ -180,7 +180,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz size_t lowering_setting; - if (merge_type == MergeType::TTL_DELETE) + if (with_ttl) lowering_setting = data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge_with_ttl; else lowering_setting = data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge; @@ -198,7 +198,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz } -UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation() +UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation() const { const auto data_settings = data.getSettings(); size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 086a2a9cae2..6b0e2e9be22 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -59,17 +59,17 @@ public: /** Get maximum total size of parts to do merge, at current moment of time. * It depends on number of free threads in background_pool and amount of free space in disk. */ - UInt64 getMaxSourcePartsSizeForMerge(MergeType merge_type); + UInt64 getMaxSourcePartsSizeForMerge(bool with_ttl) const; /** For explicitly passed size of pool and number of used tasks. * This method could be used to calculate threshold depending on number of tasks in replication queue. */ - UInt64 getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used, MergeType merge_type); + UInt64 getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used, bool with_ttl) const; /** Get maximum total size of parts to do mutation, at current moment of time. * It depends only on amount of free space in disk. */ - UInt64 getMaxSourcePartSizeForMutation(); + UInt64 getMaxSourcePartSizeForMutation() const; /** Selects which parts to merge. Uses a lot of heuristics. * diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index c9b366a9ec8..d1b4217401c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1061,7 +1061,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( return false; } - UInt64 max_source_parts_size = entry.type == LogEntry::MERGE_PARTS ? merger_mutator.getMaxSourcePartsSizeForMerge(entry.merge_type) + UInt64 max_source_parts_size = entry.type == LogEntry::MERGE_PARTS ? merger_mutator.getMaxSourcePartsSizeForMerge(entry.merge_type == MergeType::TTL_DELETE) : merger_mutator.getMaxSourcePartSizeForMutation(); /** If there are enough free threads in background pool to do large merges (maximal size of merge is allowed), * then ignore value returned by getMaxSourcePartsSizeForMerge() and execute merge of any size, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 05f2f5254f0..07e373ac93c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -650,11 +650,11 @@ bool StorageMergeTree::merge( if (partition_id.empty()) { - UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(MergeType::NORMAL); + UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(false); UInt64 max_source_parts_size_with_ttl = 0; if (!aggressive) - max_source_parts_size_with_ttl = merger_mutator.getMaxSourcePartsSizeForMerge(MergeType::TTL_DELETE); + max_source_parts_size_with_ttl = merger_mutator.getMaxSourcePartsSizeForMerge(true); if (max_source_parts_size > 0) selected = merger_mutator.selectPartsToMerge(future_part, aggressive, max_source_parts_size, can_merge, max_source_parts_size_with_ttl, out_disable_reason); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1c880c8c790..e01926d39d1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2526,12 +2526,12 @@ void StorageReplicatedMergeTree::mergeSelectingTask() else { UInt64 max_source_parts_size_for_merge = merger_mutator.getMaxSourcePartsSizeForMerge( - storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum, MergeType::NORMAL); + storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum, false); UInt64 max_source_parts_size_for_merge_with_ttl = 0; if (merges_and_mutations_queued.merges_with_ttl < storage_settings_ptr->max_replicated_merges_with_ttl_in_queue) max_source_parts_size_for_merge_with_ttl = merger_mutator.getMaxSourcePartsSizeForMerge( - storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum, MergeType::TTL_DELETE); + storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum, true); UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); From 06b38a4d44fc579e1635f201caf0babe1c7c74fe Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Sep 2020 13:31:31 +0300 Subject: [PATCH 035/298] Remove cerr --- src/Interpreters/MutationsInterpreter.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 3a397cb9b5a..ef95b25eb98 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -534,14 +534,14 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) stages.emplace_back(context); for (const auto & column : unchanged_columns) { - std::cerr << "ADDING UNCHANGED COLUMN TO STAGE:" << column << std::endl; + //std::cerr << "ADDING UNCHANGED COLUMN TO STAGE:" << column << std::endl; stages.back().column_to_updated.emplace( column, std::make_shared(column)); - std::cerr << "OUTPUT COLUMNS:" << stages.back().output_columns.size() << std::endl; - for (const auto & col : stages.back().output_columns) - { - std::cerr << "OUTPUT COLUMN:" << col << std::endl; - } + //std::cerr << "OUTPUT COLUMNS:" << stages.back().output_columns.size() << std::endl; + //for (const auto & col : stages.back().output_columns) + //{ + // std::cerr << "OUTPUT COLUMN:" << col << std::endl; + //} } } } From 128cb7ce22d9d763da462c5d41dbe90c237718f6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Sep 2020 15:16:12 +0300 Subject: [PATCH 036/298] Don't select already selected parts --- src/Storages/MergeTree/MergeSelector.h | 14 ++--- src/Storages/MergeTree/MergeTreeData.cpp | 59 +++---------------- src/Storages/MergeTree/MergeTreeData.h | 3 - .../MergeTree/MergeTreeDataMergerMutator.cpp | 28 ++++----- .../MergeTree/MergeTreeDataPartTTLInfo.cpp | 28 +++++++++ .../MergeTree/MergeTreeDataPartTTLInfo.h | 15 +++-- .../MergeTree/MergeTreePartsMover.cpp | 5 +- src/Storages/MergeTree/MergeType.cpp | 5 ++ src/Storages/MergeTree/MergeType.h | 2 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 4 +- src/Storages/MergeTree/TTLMergeSelector.cpp | 31 ++++++++-- src/Storages/MergeTree/TTLMergeSelector.h | 26 ++++++-- src/Storages/TTLDescription.cpp | 2 +- 13 files changed, 123 insertions(+), 99 deletions(-) diff --git a/src/Storages/MergeTree/MergeSelector.h b/src/Storages/MergeTree/MergeSelector.h index ae2c48fced1..285dc1a3660 100644 --- a/src/Storages/MergeTree/MergeSelector.h +++ b/src/Storages/MergeTree/MergeSelector.h @@ -4,6 +4,8 @@ #include #include #include +#include +#include namespace DB @@ -40,17 +42,9 @@ public: /// Opaque pointer to avoid dependencies (it is not possible to do forward declaration of typedef). const void * data; - /// Minimal time, when we need to delete some data from this part. - time_t min_delete_ttl; + MergeTreeDataPartTTLInfos ttl_infos; - /// Maximum time, when we will need to drop this part altogether because all rows in it are expired. - time_t max_delete_ttl; - - /// Minimal time, when we need to recompress this part. - time_t min_recompress_ttl; - - /// Maximum time, when we need to recompress this part. - time_t max_recompress_ttl; + ASTPtr compression_codec_desc; }; /// Parts are belong to partitions. Only parts within same partition could be merged. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 16a08b180f9..f535a040535 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -507,6 +507,7 @@ void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_meta if (new_table_ttl.definition_ast) { + std::cerr << "MOVE TTL SIZE:" << new_table_ttl.move_ttl.size() << std::endl; for (const auto & move_ttl : new_table_ttl.move_ttl) { if (!getDestinationForTTL(move_ttl)) @@ -2975,9 +2976,11 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ { expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size); + auto metadata_snapshot = getInMemoryMetadataPtr(); ReservationPtr reservation; - auto ttl_entry = selectTTLEntryForTTLInfos(ttl_infos, time_of_move); + auto ttl_entry = selectTTLEntryForTTLInfos(metadata_snapshot->getMoveTTLs(), ttl_infos.moves_ttl, time_of_move, true); + if (ttl_entry) { SpacePtr destination_ptr = getDestinationForTTL(*ttl_entry); @@ -3031,64 +3034,16 @@ bool MergeTreeData::isPartInTTLDestination(const TTLDescription & ttl, const IMe return false; } -std::optional -MergeTreeData::selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const -{ - time_t max_max_ttl = 0; - TTLDescriptions::const_iterator best_entry_it; - auto metadata_snapshot = getInMemoryMetadataPtr(); - - const auto & move_ttl_entries = metadata_snapshot->getMoveTTLs(); - for (auto ttl_entry_it = move_ttl_entries.begin(); ttl_entry_it != move_ttl_entries.end(); ++ttl_entry_it) - { - auto ttl_info_it = ttl_infos.moves_ttl.find(ttl_entry_it->result_column); - /// Prefer TTL rule which went into action last. - if (ttl_info_it != ttl_infos.moves_ttl.end() - && ttl_info_it->second.max <= time_of_move - && max_max_ttl <= ttl_info_it->second.max) - { - best_entry_it = ttl_entry_it; - max_max_ttl = ttl_info_it->second.max; - } - } - - return max_max_ttl ? *best_entry_it : std::optional(); -} - - CompressionCodecPtr MergeTreeData::getCompressionCodecForPart(size_t part_size_compressed, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t current_time) const { - time_t max_max_ttl = 0; - TTLDescriptions::const_iterator best_entry_it; auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); - //std::cerr << "RECOMPRESSION ENTRIES SIZE:" << recompression_ttl_entries.size() << std::endl; - for (auto ttl_entry_it = recompression_ttl_entries.begin(); ttl_entry_it != recompression_ttl_entries.end(); ++ttl_entry_it) - { - //std::cerr << "RECOMPRESSION TTL SIZE:" << ttl_infos.recompression_ttl.size() << std::endl; - auto ttl_info_it = ttl_infos.recompression_ttl.find(ttl_entry_it->result_column); - /// Prefer TTL rule which went into action last. - if (ttl_info_it != ttl_infos.recompression_ttl.end() - && ttl_info_it->second.max <= current_time - && max_max_ttl <= ttl_info_it->second.max) - { - best_entry_it = ttl_entry_it; - max_max_ttl = ttl_info_it->second.max; - } - } + auto best_ttl_entry = selectTTLEntryForTTLInfos(recompression_ttl_entries, ttl_infos.recompression_ttl, current_time, false); - if (max_max_ttl) - { - //std::cerr << "BEST ENTRY FOUND, MAX MAX:" << max_max_ttl << std::endl; - //std::cerr << "RECOMPRESSION IS NULLPTR:" << (best_entry_it->recompression_codec == nullptr) << std::endl; - return CompressionCodecFactory::instance().get(best_entry_it->recompression_codec, {}); - } - //else - //{ - // std::cerr << "NOT FOUND NEW RECOMPRESSION\n"; - //} + if (best_ttl_entry) + return CompressionCodecFactory::instance().get(best_ttl_entry->recompression_codec, {}); return global_context.chooseCompressionCodec( part_size_compressed, diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index ab115927e1e..14cefe9af1d 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -667,9 +667,6 @@ public: ExpressionActionsPtr getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; ExpressionActionsPtr getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; - std::optional selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const; - - CompressionCodecPtr getCompressionCodecForPart(size_t part_size_compressed, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t current_time) const; /// Limiting parallel sends per one table, used in DataPartsExchange diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index ce860a5b590..11bc6bbd46d 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -225,6 +225,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( { MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector(); const auto data_settings = data.getSettings(); + auto metadata_snapshot = data.getInMemoryMetadataPtr(); if (data_parts.empty()) { @@ -268,10 +269,8 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( part_info.age = current_time - part->modification_time; part_info.level = part->info.level; part_info.data = ∂ - part_info.min_delete_ttl = part->ttl_infos.part_min_ttl; - part_info.max_delete_ttl = part->ttl_infos.part_max_ttl; - part_info.min_recompress_ttl = part->ttl_infos.getMinRecompressionTTL(); - part_info.max_recompress_ttl = part->ttl_infos.getMaxRecompressionTTL(); + part_info.ttl_infos = part->ttl_infos; + part_info.compression_codec_desc = part->default_codec->getCodecDesc(); partitions.back().emplace_back(part_info); @@ -287,7 +286,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( IMergeSelector::PartsInPartition parts_to_merge; - if (!ttl_merges_blocker.isCancelled()) + if (!ttl_merges_blocker.isCancelled() && metadata_snapshot->hasAnyTTL()) { TTLDeleteMergeSelector delete_ttl_selector( next_ttl_merge_times_by_partition, @@ -298,12 +297,13 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( parts_to_merge = delete_ttl_selector.select(partitions, max_total_size_to_merge_with_ttl); if (!parts_to_merge.empty()) future_part.merge_type = MergeType::TTL_DELETE; - else + else if (metadata_snapshot->hasAnyRecompressionTTL()) { TTLRecompressMergeSelector recompress_ttl_selector( next_ttl_merge_times_by_partition, current_time, - data_settings->merge_with_ttl_timeout); + data_settings->merge_with_ttl_timeout, + metadata_snapshot->getRecompressionTTLs()); parts_to_merge = recompress_ttl_selector.select(partitions, max_total_size_to_merge_with_ttl); if (!parts_to_merge.empty()) @@ -665,7 +665,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor new_data_part->partition.assign(future_part.getPartition()); new_data_part->is_temp = true; - if (future_part.merge_type == MergeType::TTL_DELETE && ttl_merges_blocker.isCancelled()) + if (isTTLMergeType(future_part.merge_type) && ttl_merges_blocker.isCancelled()) throw Exception("Cancelled merging parts with expired TTL", ErrorCodes::ABORTED); bool need_remove_expired_values = false; @@ -840,8 +840,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (deduplicate) merged_stream = std::make_shared(merged_stream, sort_description, SizeLimits(), 0 /*limit_hint*/, Names()); - if (need_remove_expired_values) - merged_stream = std::make_shared(merged_stream, data, metadata_snapshot, new_data_part, time_of_merge, false); + if (need_remove_expired_values || (future_part.merge_type == MergeType::FINAL && !ttl_merges_blocker.isCancelled())) + merged_stream = std::make_shared(merged_stream, data, metadata_snapshot, new_data_part, time_of_merge, future_part.merge_type == MergeType::FINAL); if (metadata_snapshot->hasSecondaryIndices()) @@ -1123,19 +1123,19 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor if (in && shouldExecuteTTL(metadata_snapshot, in->getHeader().getNamesAndTypesList().getNames(), commands_for_part)) { - std::cerr << "GOING TO MATERIALIZE TTL\n"; + //std::cerr << "GOING TO MATERIALIZE TTL\n"; need_remove_expired_values = true; } else { - std::cerr << "NOT GOING TO MATERIALIZE TTL\n"; - std::cerr << "IN IS NULL:" << (in == nullptr) << std::endl; + //std::cerr << "NOT GOING TO MATERIALIZE TTL\n"; + //std::cerr << "IN IS NULL:" << (in == nullptr) << std::endl; } /// All columns from part are changed and may be some more that were missing before in part if (!isWidePart(source_part) || (interpreter && interpreter->isAffectingAllColumns())) { - std::cerr << "MUTATING ALL PART COLUMNS\n"; + //std::cerr << "MUTATING ALL PART COLUMNS\n"; /// Note: this is done before creating input streams, because otherwise data.data_parts_mutex /// (which is locked in data.getTotalActiveSizeInBytes()) /// (which is locked in shared mode when input streams are created) and when inserting new data diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp index 0664d3c5df0..42fc4be0fa5 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp @@ -186,4 +186,32 @@ time_t MergeTreeDataPartTTLInfos::getMaxRecompressionTTL() const return max; } + +std::optional selectTTLEntryForTTLInfos(const TTLDescriptions & descriptions, const TTLInfoMap & ttl_info_map, time_t current_time, bool use_max) +{ + time_t best_ttl_time = 0; + TTLDescriptions::const_iterator best_entry_it; + for (auto ttl_entry_it = descriptions.begin(); ttl_entry_it != descriptions.end(); ++ttl_entry_it) + { + auto ttl_info_it = ttl_info_map.find(ttl_entry_it->result_column); + time_t ttl_time; + + if (use_max) + ttl_time = ttl_info_it->second.max; + else + ttl_time = ttl_info_it->second.min; + + /// Prefer TTL rule which went into action last. + if (ttl_info_it != ttl_info_map.end() + && ttl_time <= current_time + && best_ttl_time <= ttl_time) + { + best_entry_it = ttl_entry_it; + best_ttl_time = ttl_time; + } + } + + return best_ttl_time ? *best_entry_it : std::optional(); +} + } diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h index 0f46b4f97e8..d0738053d1d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include #include @@ -30,11 +31,13 @@ struct MergeTreeDataPartTTLInfo } }; +/// Order is important as it would be serialized and hashed for checksums +using TTLInfoMap = std::map; + /// PartTTLInfo for all columns and table with minimal ttl for whole part struct MergeTreeDataPartTTLInfos { - /// Order is important as it would be serialized and hashed for checksums - std::map columns_ttl; + TTLInfoMap columns_ttl; MergeTreeDataPartTTLInfo table_ttl; /// `part_min_ttl` and `part_max_ttl` are TTLs which are used for selecting parts @@ -42,11 +45,9 @@ struct MergeTreeDataPartTTLInfos time_t part_min_ttl = 0; time_t part_max_ttl = 0; - /// Order is important as it would be serialized and hashed for checksums - std::map moves_ttl; + TTLInfoMap moves_ttl; - /// Order is important as it would be serialized and hashed for checksums - std::map recompression_ttl; + TTLInfoMap recompression_ttl; time_t getMinRecompressionTTL() const; time_t getMaxRecompressionTTL() const; @@ -70,4 +71,6 @@ struct MergeTreeDataPartTTLInfos } }; +std::optional selectTTLEntryForTTLInfos(const TTLDescriptions & descriptions, const TTLInfoMap & ttl_info_map, time_t current_time, bool use_max); + } diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index e84ff418bc3..92ea745c5df 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -121,6 +121,8 @@ bool MergeTreePartsMover::selectPartsForMove( time_t time_of_move = time(nullptr); + auto metadata_snapshot = data->getInMemoryMetadataPtr(); + for (const auto & part : data_parts) { String reason; @@ -128,7 +130,8 @@ bool MergeTreePartsMover::selectPartsForMove( if (!can_move(part, &reason)) continue; - auto ttl_entry = data->selectTTLEntryForTTLInfos(part->ttl_infos, time_of_move); + auto ttl_entry = selectTTLEntryForTTLInfos(metadata_snapshot->getMoveTTLs(), part->ttl_infos.moves_ttl, time_of_move, true); + auto to_insert = need_to_move.find(part->volume->getDisk()); ReservationPtr reservation; if (ttl_entry) diff --git a/src/Storages/MergeTree/MergeType.cpp b/src/Storages/MergeTree/MergeType.cpp index 69732877ad3..875a0a93f6b 100644 --- a/src/Storages/MergeTree/MergeType.cpp +++ b/src/Storages/MergeTree/MergeType.cpp @@ -26,4 +26,9 @@ String toString(MergeType merge_type) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeType {}", static_cast(merge_type)); } +bool isTTLMergeType(MergeType merge_type) +{ + return merge_type == MergeType::TTL_DELETE || merge_type == MergeType::TTL_RECOMPRESS; +} + } diff --git a/src/Storages/MergeTree/MergeType.h b/src/Storages/MergeTree/MergeType.h index 2bc88352bf2..0f4a0043c54 100644 --- a/src/Storages/MergeTree/MergeType.h +++ b/src/Storages/MergeTree/MergeType.h @@ -15,4 +15,6 @@ enum class MergeType String toString(MergeType merge_type); +bool isTTLMergeType(MergeType merge_type); + } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index d1b4217401c..206d8f93038 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1061,7 +1061,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( return false; } - UInt64 max_source_parts_size = entry.type == LogEntry::MERGE_PARTS ? merger_mutator.getMaxSourcePartsSizeForMerge(entry.merge_type == MergeType::TTL_DELETE) + UInt64 max_source_parts_size = entry.type == LogEntry::MERGE_PARTS ? merger_mutator.getMaxSourcePartsSizeForMerge(isTTLMergeType(entry.merge_type)) : merger_mutator.getMaxSourcePartSizeForMutation(); /** If there are enough free threads in background pool to do large merges (maximal size of merge is allowed), * then ignore value returned by getMaxSourcePartsSizeForMerge() and execute merge of any size, @@ -1324,7 +1324,7 @@ ReplicatedMergeTreeQueue::OperationsInQueue ReplicatedMergeTreeQueue::countMerge if (entry->type == ReplicatedMergeTreeLogEntry::MERGE_PARTS) { ++count_merges; - if (entry->merge_type == MergeType::TTL_DELETE) + if (isTTLMergeType(entry->merge_type)) ++count_merges_with_ttl; } else if (entry->type == ReplicatedMergeTreeLogEntry::MUTATE_PART) diff --git a/src/Storages/MergeTree/TTLMergeSelector.cpp b/src/Storages/MergeTree/TTLMergeSelector.cpp index 5c2d22ab11c..1bc5d563936 100644 --- a/src/Storages/MergeTree/TTLMergeSelector.cpp +++ b/src/Storages/MergeTree/TTLMergeSelector.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -39,7 +40,7 @@ IMergeSelector::PartsInPartition ITTLMergeSelector::select( { time_t ttl = getTTLForPart(*part_it); - if (ttl && (partition_to_merge_index == -1 || ttl < partition_to_merge_min_ttl)) + if (ttl && !isTTLAlreadySatisfied(*part_it) && (partition_to_merge_index == -1 || ttl < partition_to_merge_min_ttl)) { partition_to_merge_min_ttl = ttl; partition_to_merge_index = i; @@ -59,7 +60,7 @@ IMergeSelector::PartsInPartition ITTLMergeSelector::select( { time_t ttl = getTTLForPart(*best_begin); - if (!ttl || ttl > current_time + if (!ttl || isTTLAlreadySatisfied(*best_begin) || ttl > current_time || (max_total_size_to_merge && total_size > max_total_size_to_merge)) { ++best_begin; @@ -77,7 +78,7 @@ IMergeSelector::PartsInPartition ITTLMergeSelector::select( { time_t ttl = getTTLForPart(*best_end); - if (!ttl || ttl > current_time + if (!ttl || isTTLAlreadySatisfied(*best_end) || ttl > current_time || (max_total_size_to_merge && total_size > max_total_size_to_merge)) break; @@ -93,12 +94,32 @@ IMergeSelector::PartsInPartition ITTLMergeSelector::select( time_t TTLDeleteMergeSelector::getTTLForPart(const IMergeSelector::Part & part) const { - return only_drop_parts ? part.max_delete_ttl : part.min_delete_ttl; + return only_drop_parts ? part.ttl_infos.part_max_ttl : part.ttl_infos.part_min_ttl; } time_t TTLRecompressMergeSelector::getTTLForPart(const IMergeSelector::Part & part) const { - return part.min_recompress_ttl; + return part.ttl_infos.getMinRecompressionTTL(); +} + +bool TTLRecompressMergeSelector::isTTLAlreadySatisfied(const IMergeSelector::Part & part) const +{ + if (recompression_ttls.empty()) + return false; + + auto ttl_description = selectTTLEntryForTTLInfos(recompression_ttls, part.ttl_infos.recompression_ttl, current_time, false); + + if (!ttl_description) + return true; + + auto ast_to_str = [](ASTPtr query) -> String + { + if (!query) + return ""; + return queryToString(query); + }; + + return ast_to_str(ttl_description->recompression_codec) == ast_to_str(part.compression_codec_desc); } } diff --git a/src/Storages/MergeTree/TTLMergeSelector.h b/src/Storages/MergeTree/TTLMergeSelector.h index a7380aa87c9..de4cbc11a57 100644 --- a/src/Storages/MergeTree/TTLMergeSelector.h +++ b/src/Storages/MergeTree/TTLMergeSelector.h @@ -2,6 +2,7 @@ #include #include +#include #include @@ -21,9 +22,9 @@ public: using PartitionIdToTTLs = std::map; ITTLMergeSelector(PartitionIdToTTLs & merge_due_times_, time_t current_time_, Int64 merge_cooldown_time_) - : merge_due_times(merge_due_times_), - current_time(current_time_), - merge_cooldown_time(merge_cooldown_time_) + : current_time(current_time_) + , merge_due_times(merge_due_times_) + , merge_cooldown_time(merge_cooldown_time_) { } @@ -32,10 +33,13 @@ public: const size_t max_total_size_to_merge) override; virtual time_t getTTLForPart(const IMergeSelector::Part & part) const = 0; + virtual bool isTTLAlreadySatisfied(const IMergeSelector::Part & part) const = 0; + +protected: + time_t current_time; private: PartitionIdToTTLs & merge_due_times; - time_t current_time; Int64 merge_cooldown_time; }; @@ -51,6 +55,11 @@ public: time_t getTTLForPart(const IMergeSelector::Part & part) const override; + bool isTTLAlreadySatisfied(const IMergeSelector::Part &) const override + { + return false; + } + private: bool only_drop_parts; }; @@ -58,9 +67,16 @@ private: class TTLRecompressMergeSelector : public ITTLMergeSelector { public: - using ITTLMergeSelector::ITTLMergeSelector; + TTLRecompressMergeSelector(PartitionIdToTTLs & merge_due_times_, time_t current_time_, Int64 merge_cooldown_time_, const TTLDescriptions & recompression_ttls_) + : ITTLMergeSelector(merge_due_times_, current_time_, merge_cooldown_time_) + , recompression_ttls(recompression_ttls_) + {} time_t getTTLForPart(const IMergeSelector::Part & part) const override; + + bool isTTLAlreadySatisfied(const IMergeSelector::Part & part) const override; +private: + TTLDescriptions recompression_ttls; }; } diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index bc634fe67bd..07173d61ece 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -328,7 +328,7 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( for (const auto & ttl_element_ptr : definition_ast->children) { auto ttl = TTLDescription::getTTLFromAST(ttl_element_ptr, columns, context, primary_key); - if (ttl.mode == TTLMode::DELETE) + if (ttl.mode == TTLMode::DELETE || ttl.mode == TTLMode::GROUP_BY) { if (seen_delete_ttl) throw Exception("More than one DELETE TTL expression is not allowed", ErrorCodes::BAD_TTL_EXPRESSION); From dcbddbb8d04e7c9f16bc7555ac37a5fb0a60a382 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Sep 2020 19:15:41 +0300 Subject: [PATCH 037/298] Add recompression ttls tests and fix bugs --- src/Storages/MergeTree/MergeTreeData.cpp | 3 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 21 +-- .../MergeTree/MergeTreeDataPartTTLInfo.cpp | 11 +- .../MergeTree/MergeTreeDataPartTTLInfo.h | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 1 - src/Storages/MergeTree/MergeTreeSettings.cpp | 13 ++ src/Storages/MergeTree/MergeTreeSettings.h | 4 +- src/Storages/MergeTree/MergeType.cpp | 2 - src/Storages/MergeTree/MergeType.h | 1 - src/Storages/MergeTree/TTLMergeSelector.cpp | 2 + .../MergeTree/registerStorageMergeTree.cpp | 1 - src/Storages/StorageMergeTree.cpp | 1 - src/Storages/TTLDescription.cpp | 2 +- .../test_recompression_ttl/__init__.py | 0 .../configs/background_pool_config.xml | 9 ++ .../test_recompression_ttl/test.py | 131 ++++++++++++++++++ 16 files changed, 179 insertions(+), 25 deletions(-) create mode 100644 tests/integration/test_recompression_ttl/__init__.py create mode 100644 tests/integration/test_recompression_ttl/configs/background_pool_config.xml create mode 100644 tests/integration/test_recompression_ttl/test.py diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f535a040535..e5946619da5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -507,7 +507,6 @@ void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_meta if (new_table_ttl.definition_ast) { - std::cerr << "MOVE TTL SIZE:" << new_table_ttl.move_ttl.size() << std::endl; for (const auto & move_ttl : new_table_ttl.move_ttl) { if (!getDestinationForTTL(move_ttl)) @@ -3040,6 +3039,8 @@ CompressionCodecPtr MergeTreeData::getCompressionCodecForPart(size_t part_size_c auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); + LOG_DEBUG(log, "RECOMPRESSION ENTRIES SIZE {}", recompression_ttl_entries.size()); + LOG_DEBUG(log, "TTL INFOS SIZE {}", ttl_infos.recompression_ttl.size()); auto best_ttl_entry = selectTTLEntryForTTLInfos(recompression_ttl_entries, ttl_infos.recompression_ttl, current_time, false); if (best_ttl_entry) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 11bc6bbd46d..7c849e2a457 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -234,6 +234,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( return false; } + //LOG_DEBUG(log, "SELECTING PARTS TO MERGE"); time_t current_time = std::time(nullptr); IMergeSelector::Partitions partitions; @@ -270,7 +271,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( part_info.level = part->info.level; part_info.data = ∂ part_info.ttl_infos = part->ttl_infos; - part_info.compression_codec_desc = part->default_codec->getCodecDesc(); + part_info.compression_codec_desc = part->default_codec->getFullCodecDesc(); partitions.back().emplace_back(part_info); @@ -288,6 +289,8 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( if (!ttl_merges_blocker.isCancelled() && metadata_snapshot->hasAnyTTL()) { + + //LOG_DEBUG(log, "SELECTING WITH TTL"); TTLDeleteMergeSelector delete_ttl_selector( next_ttl_merge_times_by_partition, current_time, @@ -299,6 +302,8 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( future_part.merge_type = MergeType::TTL_DELETE; else if (metadata_snapshot->hasAnyRecompressionTTL()) { + + //LOG_DEBUG(log, "SELECTING WITH RECOMPRESSION"); TTLRecompressMergeSelector recompress_ttl_selector( next_ttl_merge_times_by_partition, current_time, @@ -307,7 +312,10 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( parts_to_merge = recompress_ttl_selector.select(partitions, max_total_size_to_merge_with_ttl); if (!parts_to_merge.empty()) + { + //LOG_DEBUG(log, "SELECTED PARTS: {}", parts_to_merge.size()); future_part.merge_type = MergeType::TTL_RECOMPRESS; + } } } @@ -410,11 +418,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( LOG_DEBUG(log, "Selected {} parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name); future_part.assign(std::move(parts)); - - if (final) - future_part.merge_type = MergeType::FINAL; - else - future_part.merge_type = MergeType::NORMAL; + future_part.merge_type = MergeType::NORMAL; available_disk_space -= required_disk_space; return true; @@ -693,6 +697,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor /// the order is reverse. This annoys TSan even though one lock is locked in shared mode and thus /// deadlock is impossible. auto compression_codec = data.getCompressionCodecForPart(merge_entry->total_size_bytes_compressed, new_data_part->ttl_infos, time_of_merge); + LOG_DEBUG(log, "CHOOSEN CODEC {} FOR PART {}", queryToString(compression_codec->getCodecDesc()), new_data_part->name); /// TODO: Should it go through IDisk interface? String rows_sources_file_path; @@ -840,8 +845,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (deduplicate) merged_stream = std::make_shared(merged_stream, sort_description, SizeLimits(), 0 /*limit_hint*/, Names()); - if (need_remove_expired_values || (future_part.merge_type == MergeType::FINAL && !ttl_merges_blocker.isCancelled())) - merged_stream = std::make_shared(merged_stream, data, metadata_snapshot, new_data_part, time_of_merge, future_part.merge_type == MergeType::FINAL); + if (need_remove_expired_values) + merged_stream = std::make_shared(merged_stream, data, metadata_snapshot, new_data_part, time_of_merge, false); if (metadata_snapshot->hasSecondaryIndices()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp index 42fc4be0fa5..1cecb2672fb 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp @@ -73,15 +73,14 @@ void MergeTreeDataPartTTLInfos::read(ReadBuffer & in) } if (json.has("recompression")) { - const JSON & moves = json["recompression"]; - for (auto move : moves) // NOLINT + const JSON & recompressions = json["recompression"]; + for (auto recompression : recompressions) // NOLINT { MergeTreeDataPartTTLInfo ttl_info; - ttl_info.min = move["min"].getUInt(); - ttl_info.max = move["max"].getUInt(); - String expression = move["expression"].getString(); + ttl_info.min = recompression["min"].getUInt(); + ttl_info.max = recompression["max"].getUInt(); + String expression = recompression["expression"].getString(); recompression_ttl.emplace(expression, ttl_info); - updatePartMinMaxTTL(ttl_info.min, ttl_info.max); } } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h index d0738053d1d..1176c036b8c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h @@ -67,7 +67,7 @@ struct MergeTreeDataPartTTLInfos bool empty() { - return !part_min_ttl && moves_ttl.empty(); + return !part_min_ttl && moves_ttl.empty() && recompression_ttl.empty(); } }; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 5115666066a..607535225a2 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -309,7 +309,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa /// This effectively chooses minimal compression method: /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getCompressionCodecForPart(0, new_data_part->ttl_infos, current_time); - std::cerr << "SELECTED CODEC:" << queryToString(compression_codec->getCodecDesc()) << std::endl; const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 7f537ec330a..337cad224a4 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -100,6 +100,19 @@ void MergeTreeSettings::sanityCheck(const Settings & query_settings) const number_of_free_entries_in_pool_to_lower_max_size_of_merge, query_settings.background_pool_size); } + + if (number_of_free_entries_in_pool_to_lower_max_size_of_merge_with_ttl >= query_settings.background_pool_size) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_lower_max_size_of_merge_with_ttl' setting" + " ({}) (default values are defined in section of config.xml" + " or the value can be specified per table in SETTINGS section of CREATE TABLE query)" + " is greater or equals to the value of 'background_pool_size'" + " ({}) (the value is defined in users.xml for default profile)." + " This indicates incorrect configuration because the maximum size of merge with TTL will be always lowered.", + number_of_free_entries_in_pool_to_lower_max_size_of_merge_with_ttl, + query_settings.background_pool_size); + } + } } diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index e5707ff837c..f2235bf94aa 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -33,10 +33,10 @@ struct Settings; M(UInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \ M(UInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ M(UInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ - M(UInt64, max_replicated_merges_with_ttl_in_queue, 1, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + M(UInt64, max_replicated_merges_with_ttl_in_queue, 1, "How many tasks of merging parts with TTL are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ - M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge_with_ttl, 14, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ + M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge_with_ttl, 14, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running TTL merges.", 0) \ M(Seconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(Seconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(Seconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ diff --git a/src/Storages/MergeTree/MergeType.cpp b/src/Storages/MergeTree/MergeType.cpp index 875a0a93f6b..b521d835b36 100644 --- a/src/Storages/MergeTree/MergeType.cpp +++ b/src/Storages/MergeTree/MergeType.cpp @@ -15,8 +15,6 @@ String toString(MergeType merge_type) { case MergeType::NORMAL: return "NORMAL"; - case MergeType::FINAL: - return "FINAL"; case MergeType::TTL_DELETE: return "TTL_DELETE"; case MergeType::TTL_RECOMPRESS: diff --git a/src/Storages/MergeTree/MergeType.h b/src/Storages/MergeTree/MergeType.h index 0f4a0043c54..26fb172f463 100644 --- a/src/Storages/MergeTree/MergeType.h +++ b/src/Storages/MergeTree/MergeType.h @@ -8,7 +8,6 @@ namespace DB enum class MergeType { NORMAL, - FINAL, TTL_DELETE, TTL_RECOMPRESS, }; diff --git a/src/Storages/MergeTree/TTLMergeSelector.cpp b/src/Storages/MergeTree/TTLMergeSelector.cpp index 1bc5d563936..2e71f3c5401 100644 --- a/src/Storages/MergeTree/TTLMergeSelector.cpp +++ b/src/Storages/MergeTree/TTLMergeSelector.cpp @@ -118,6 +118,8 @@ bool TTLRecompressMergeSelector::isTTLAlreadySatisfied(const IMergeSelector::Par return ""; return queryToString(query); }; + //LOG_DEBUG(&Poco::Logger::get("RECOMPRESS SELECTOR"), "PART CODEC: {}", ast_to_str(part.compression_codec_desc)); + //LOG_DEBUG(&Poco::Logger::get("RECOMPRESS SELECTOR"), "ENTRY CODEC: {}", ast_to_str(ttl_description->recompression_codec)); return ast_to_str(ttl_description->recompression_codec) == ast_to_str(part.compression_codec_desc); } diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 8706c1f3b37..b0c422bd79f 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -559,7 +559,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.storage_def->ttl_table) { - std::cerr << "Parsing table ttl in description\n"; metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( args.storage_def->ttl_table->ptr(), metadata.columns, args.context, metadata.primary_key); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 07e373ac93c..72dee939c9d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -729,7 +729,6 @@ bool StorageMergeTree::merge( try { - std::cerr << "FUTURE PART MERGE TYPE:" << toString(future_part.merge_type) << std::endl; new_part = merger_mutator.mergePartsToTemporaryPart( future_part, metadata_snapshot, *merge_entry, table_lock_holder, time(nullptr), merging_tagger->reserved_space, deduplicate); diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 07173d61ece..8a212074027 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -273,7 +273,7 @@ TTLDescription TTLDescription::getTTLFromAST( } else if (ttl_element->mode == TTLMode::RECOMPRESS) { - std::cerr << "GOT INTO RECOMPRESS\n"; + //std::cerr << "GOT INTO RECOMPRESS\n"; result.recompression_codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( ttl_element->recompression_codec, {}, !context.getSettingsRef().allow_suspicious_codecs); diff --git a/tests/integration/test_recompression_ttl/__init__.py b/tests/integration/test_recompression_ttl/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_recompression_ttl/configs/background_pool_config.xml b/tests/integration/test_recompression_ttl/configs/background_pool_config.xml new file mode 100644 index 00000000000..e62a0105907 --- /dev/null +++ b/tests/integration/test_recompression_ttl/configs/background_pool_config.xml @@ -0,0 +1,9 @@ + + 1 + 0 + 0.0 + 0 + 1 + 1 + 0 + diff --git a/tests/integration/test_recompression_ttl/test.py b/tests/integration/test_recompression_ttl/test.py new file mode 100644 index 00000000000..2bf36d79f05 --- /dev/null +++ b/tests/integration/test_recompression_ttl/test.py @@ -0,0 +1,131 @@ +import time +import pytest + +import helpers.client as client +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=['configs/background_pool_config.xml'], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=['configs/background_pool_config.xml'], with_zookeeper=True) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + except Exception as ex: + print ex + + finally: + cluster.shutdown() + + +def wait_part_in_parts(node, table, part_name, retries=40): + for i in range(retries): + result = node.query("SELECT name FROM system.parts where name = '{}' and table = '{}'".format(part_name, table)) + if result: + return True + time.sleep(0.5) + else: + return False + + +def optimize_final_table_until_success(node, table_name, retries=40): + for i in range(retries): + try: + node.query("OPTIMIZE TABLE {} FINAL".format(table_name), settings={"optimize_throw_if_noop": "1"}) + return True + except: + time.sleep(0.5) + else: + return False + + +def wait_part_and_get_compression_codec(node, table, part_name, retries=40): + if wait_part_in_parts(node, table, part_name, retries): + return node.query("SELECT default_compression_codec FROM system.parts where name = '{}' and table = '{}'".format(part_name, table)).strip() + return None + + +def test_recompression_simple(started_cluster): + node1.query("CREATE TABLE table_for_recompression (d DateTime, key UInt64, data String) ENGINE MergeTree() ORDER BY tuple() TTL d + INTERVAL 10 SECOND RECOMPRESS CODEC(ZSTD(10))") + node1.query("INSERT INTO table_for_recompression VALUES (now(), 1, '1')") + + assert node1.query("SELECT default_compression_codec FROM system.parts where name = 'all_1_1_0'") == "LZ4\n" + + codec = wait_part_and_get_compression_codec(node1, "table_for_recompression", "all_1_1_1") + if not codec: + assert False, "Part all_1_1_1 doesn't appeared in system.parts" + + assert codec == "ZSTD(10)" + + if wait_part_in_parts(node1, "table_for_recompression", "all_1_1_2", retries=20): + assert False, "Redundant merge were assigned for part all_1_1_1 -> all_1_1_2" + + optimize_final_table_until_success(node1, "table_for_recompression") + + assert node1.query("SELECT default_compression_codec FROM system.parts where name = 'all_1_1_2'") == "ZSTD(10)\n" + + +def test_recompression_multiple_ttls(started_cluster): + node2.query("CREATE TABLE table_for_recompression (d DateTime, key UInt64, data String) ENGINE MergeTree() ORDER BY tuple() \ + TTL d + INTERVAL 5 SECOND RECOMPRESS CODEC(ZSTD(10)), \ + d + INTERVAL 10 SECOND RECOMPRESS CODEC(ZSTD(11)), \ + d + INTERVAL 15 SECOND RECOMPRESS CODEC(ZSTD(12))") + + node2.query("INSERT INTO table_for_recompression VALUES (now(), 1, '1')") + + assert node2.query("SELECT default_compression_codec FROM system.parts where name = 'all_1_1_0'") == "LZ4\n" + + codec = wait_part_and_get_compression_codec(node2, "table_for_recompression", "all_1_1_1") + if not codec: + assert False, "Part all_1_1_1 doesn't appeared in system.parts" + + assert codec == "ZSTD(10)" + + codec = wait_part_and_get_compression_codec(node2, "table_for_recompression", "all_1_1_2") + if not codec: + assert False, "Part all_1_1_2 doesn't appeared in system.parts" + + assert codec == "ZSTD(11)" + + codec = wait_part_and_get_compression_codec(node2, "table_for_recompression", "all_1_1_3") + if not codec: + assert False, "Part all_1_1_3 doesn't appeared in system.parts" + + assert codec == "ZSTD(12)" + + if wait_part_in_parts(node2, "table_for_recompression", "all_1_1_4", retries=20): + assert False, "Redundant merge were assigned for part all_1_1_3 -> all_1_1_4" + + optimize_final_table_until_success(node2, "table_for_recompression") + + assert node2.query("SELECT default_compression_codec FROM system.parts where name = 'all_1_1_4'") == "ZSTD(12)\n" + + +def test_recompression_replicated(started_cluster): + for i, node in enumerate([node1, node2]): + node.query("CREATE TABLE recompression_replicated (d DateTime, key UInt64, data String) \ + ENGINE ReplicatedMergeTree('/test/rr', '{}') ORDER BY tuple() \ + TTL d + INTERVAL 10 SECOND RECOMPRESS CODEC(ZSTD(13))".format(i + 1)) + + node1.query("INSERT INTO recompression_replicated VALUES (now(), 1, '1')") + node2.query("SYSTEM SYNC REPLICA recompression_replicated", timeout=5) + + assert node1.query("SELECT default_compression_codec FROM system.parts where name = 'all_0_0_0' and table = 'recompression_replicated'") == "LZ4\n" + assert node2.query("SELECT default_compression_codec FROM system.parts where name = 'all_0_0_0' and table = 'recompression_replicated'") == "LZ4\n" + + codec1 = wait_part_and_get_compression_codec(node1, "recompression_replicated", "all_0_0_1") + if not codec1: + assert False, "Part all_0_0_1 doesn't appeared in system.parts on node1" + + codec2 = wait_part_and_get_compression_codec(node2, "recompression_replicated", "all_0_0_1") + if not codec2: + assert False, "Part all_0_0_1 doesn't appeared in system.parts on node2" + + assert codec1 == "ZSTD(13)" + assert codec2 == "ZSTD(13)" From 672f239f5b57ce073e1f8a1417152dbc14606251 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 3 Sep 2020 10:34:53 +0300 Subject: [PATCH 038/298] Add settings to test --- tests/integration/test_recompression_ttl/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_recompression_ttl/test.py b/tests/integration/test_recompression_ttl/test.py index 2bf36d79f05..5ea0f91d495 100644 --- a/tests/integration/test_recompression_ttl/test.py +++ b/tests/integration/test_recompression_ttl/test.py @@ -52,7 +52,7 @@ def wait_part_and_get_compression_codec(node, table, part_name, retries=40): def test_recompression_simple(started_cluster): - node1.query("CREATE TABLE table_for_recompression (d DateTime, key UInt64, data String) ENGINE MergeTree() ORDER BY tuple() TTL d + INTERVAL 10 SECOND RECOMPRESS CODEC(ZSTD(10))") + node1.query("CREATE TABLE table_for_recompression (d DateTime, key UInt64, data String) ENGINE MergeTree() ORDER BY tuple() TTL d + INTERVAL 10 SECOND RECOMPRESS CODEC(ZSTD(10)) SETTINGS merge_with_ttl_timeout = 0") node1.query("INSERT INTO table_for_recompression VALUES (now(), 1, '1')") assert node1.query("SELECT default_compression_codec FROM system.parts where name = 'all_1_1_0'") == "LZ4\n" @@ -75,7 +75,7 @@ def test_recompression_multiple_ttls(started_cluster): node2.query("CREATE TABLE table_for_recompression (d DateTime, key UInt64, data String) ENGINE MergeTree() ORDER BY tuple() \ TTL d + INTERVAL 5 SECOND RECOMPRESS CODEC(ZSTD(10)), \ d + INTERVAL 10 SECOND RECOMPRESS CODEC(ZSTD(11)), \ - d + INTERVAL 15 SECOND RECOMPRESS CODEC(ZSTD(12))") + d + INTERVAL 15 SECOND RECOMPRESS CODEC(ZSTD(12)) SETTINGS merge_with_ttl_timeout = 0") node2.query("INSERT INTO table_for_recompression VALUES (now(), 1, '1')") @@ -111,7 +111,7 @@ def test_recompression_replicated(started_cluster): for i, node in enumerate([node1, node2]): node.query("CREATE TABLE recompression_replicated (d DateTime, key UInt64, data String) \ ENGINE ReplicatedMergeTree('/test/rr', '{}') ORDER BY tuple() \ - TTL d + INTERVAL 10 SECOND RECOMPRESS CODEC(ZSTD(13))".format(i + 1)) + TTL d + INTERVAL 10 SECOND RECOMPRESS CODEC(ZSTD(13)) SETTINGS merge_with_ttl_timeout = 0".format(i + 1)) node1.query("INSERT INTO recompression_replicated VALUES (now(), 1, '1')") node2.query("SYSTEM SYNC REPLICA recompression_replicated", timeout=5) From f4c7ff03766ebf9f1f480ab2e8583db2b491f9b7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 3 Sep 2020 16:00:13 +0300 Subject: [PATCH 039/298] Add fixed size of Merge TTLS --- src/Storages/MergeTree/MergeList.cpp | 6 ++- src/Storages/MergeTree/MergeList.h | 4 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 25 ++++++++++- .../MergeTree/MergeTreeDataMergerMutator.h | 10 +++++ src/Storages/MergeTree/MergeTreeSettings.cpp | 13 ++++++ src/Storages/MergeTree/MergeTreeSettings.h | 2 + src/Storages/MergeTree/MergeType.cpp | 41 +++++++++++++++++++ src/Storages/MergeTree/MergeType.h | 20 +++++++++ .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 14 +++++++ .../MergeTree/ReplicatedMergeTreeLogEntry.h | 2 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 9 +++- .../MergeTree/ReplicatedMergeTreeQueue.h | 8 +++- src/Storages/StorageMergeTree.cpp | 15 ++++++- src/Storages/StorageReplicatedMergeTree.cpp | 26 +++++++----- src/Storages/StorageReplicatedMergeTree.h | 3 +- src/Storages/System/StorageSystemMerges.cpp | 5 +++ .../System/StorageSystemReplicationQueue.cpp | 6 +++ 17 files changed, 191 insertions(+), 18 deletions(-) create mode 100644 src/Storages/MergeTree/MergeType.cpp create mode 100644 src/Storages/MergeTree/MergeType.h diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index e9d955f5395..30324bd5d9e 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -15,12 +15,15 @@ namespace DB { MergeListElement::MergeListElement(const std::string & database_, const std::string & table_, const FutureMergedMutatedPart & future_part) - : database{database_}, table{table_}, partition_id{future_part.part_info.partition_id} + : database{database_} + , table{table_} + , partition_id{future_part.part_info.partition_id} , result_part_name{future_part.name} , result_part_path{future_part.path} , result_data_version{future_part.part_info.getDataVersion()} , num_parts{future_part.parts.size()} , thread_id{getThreadId()} + , merge_type{future_part.merge_type} { for (const auto & source_part : future_part.parts) { @@ -70,6 +73,7 @@ MergeInfo MergeListElement::getInfo() const res.columns_written = columns_written.load(std::memory_order_relaxed); res.memory_usage = memory_tracker.get(); res.thread_id = thread_id; + res.merge_type = toString(merge_type); for (const auto & source_part_name : source_part_names) res.source_part_names.emplace_back(source_part_name); diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 4ee8a75a868..0b41745a9ba 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -45,6 +46,7 @@ struct MergeInfo UInt64 columns_written; UInt64 memory_usage; UInt64 thread_id; + std::string merge_type; }; struct FutureMergedMutatedPart; @@ -87,7 +89,7 @@ struct MergeListElement : boost::noncopyable MemoryTracker * background_thread_memory_tracker_prev_parent = nullptr; UInt64 thread_id; - + MergeType merge_type; MergeListElement(const std::string & database, const std::string & table, const FutureMergedMutatedPart & future_part); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 673ad02bfb6..670f42d916d 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -208,11 +208,29 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation() } +UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMergeWithTTL() +{ + const auto data_settings = data.getSettings(); + size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); + + /// DataPart can be store only at one disk. Get maximum reservable free space at all disks. + UInt64 disk_space = data.getStoragePolicy()->getMaxUnreservedFreeSpace(); + + /// Allow merges with TTL only if there are enough threads, leave free threads for regular merges + if (busy_threads_in_pool <= 1 + || background_pool_size - busy_threads_in_pool >= data_settings->number_of_free_entries_in_pool_to_execute_merge_with_ttl) + return static_cast(disk_space / DISK_USAGE_COEFFICIENT_TO_RESERVE); + + return 0; + +} + bool MergeTreeDataMergerMutator::selectPartsToMerge( FutureMergedMutatedPart & future_part, bool aggressive, size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge_callback, + size_t max_total_size_to_merge_with_ttl, String * out_disable_reason) { MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector(); @@ -284,7 +302,9 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( current_time, data_settings->merge_with_ttl_timeout, data_settings->ttl_only_drop_parts); - parts_to_merge = merge_selector.select(partitions, max_total_size_to_merge); + parts_to_merge = merge_selector.select(partitions, max_total_size_to_merge_with_ttl); + if (!parts_to_merge.empty()) + future_part.merge_type = MergeType::TTL_DELETE; } if (parts_to_merge.empty()) @@ -593,6 +613,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (merges_blocker.isCancelled()) throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); + if (isTTLMergeType(future_part.merge_type) && ttl_merges_blocker.isCancelled()) + throw Exception("Cancelled merging parts with TTL", ErrorCodes::ABORTED); + const MergeTreeData::DataPartsVector & parts = future_part.parts; LOG_DEBUG(log, "Merging {} parts: from {} to {} into {}", parts.size(), parts.front()->name, parts.back()->name, future_part.type.toString()); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index d5798fe3582..a874c93e2f6 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -22,6 +23,7 @@ struct FutureMergedMutatedPart MergeTreeDataPartType type; MergeTreePartInfo part_info; MergeTreeData::DataPartsVector parts; + MergeType merge_type = MergeType::REGULAR; const MergeTreePartition & getPartition() const { return parts.front()->partition; } @@ -59,6 +61,13 @@ public: */ UInt64 getMaxSourcePartsSizeForMerge(); + /** Get maximum total size of parts to do merge with TTL, at current moment + * of time. If busy threads count is less than value specified by + * number_of_free_entries_in_pool_to_execute_merge_with_ttl than maximum + * size (available on disk) is allowed. + */ + UInt64 getMaxSourcePartsSizeForMergeWithTTL(); + /** For explicitly passed size of pool and number of used tasks. * This method could be used to calculate threshold depending on number of tasks in replication queue. */ @@ -81,6 +90,7 @@ public: bool aggressive, size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge, + size_t max_total_size_to_merge_with_ttl, String * out_disable_reason = nullptr); /** Select all the parts in the specified partition for merge, if possible. diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 7f537ec330a..44504bdec84 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -100,6 +100,19 @@ void MergeTreeSettings::sanityCheck(const Settings & query_settings) const number_of_free_entries_in_pool_to_lower_max_size_of_merge, query_settings.background_pool_size); } + + if (number_of_free_entries_in_pool_to_execute_merge_with_ttl >= query_settings.background_pool_size) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_execute_merge_with_ttl' setting" + " ({}) (default values are defined in section of config.xml" + " or the value can be specified per table in SETTINGS section of CREATE TABLE query)" + " is greater or equals to the value of 'background_pool_size'" + " ({}) (the value is defined in users.xml for default profile)." + " This indicates incorrect configuration because TTL cannot work with these settings.", + number_of_free_entries_in_pool_to_execute_merge_with_ttl, + query_settings.background_pool_size); + } + } } diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 085c441aa90..06fc21b24c3 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -33,8 +33,10 @@ struct Settings; M(UInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \ M(UInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ M(UInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + M(UInt64, max_replicated_merges_with_ttl_in_queue, 1, "How many tasks of merging parts with TTL are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ + M(UInt64, number_of_free_entries_in_pool_to_execute_merge_with_ttl, 12, "When there is less than specified number of free entries in pool, do not execute merge with TTL. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ M(Seconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(Seconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(Seconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ diff --git a/src/Storages/MergeTree/MergeType.cpp b/src/Storages/MergeTree/MergeType.cpp new file mode 100644 index 00000000000..d31197e8d4c --- /dev/null +++ b/src/Storages/MergeTree/MergeType.cpp @@ -0,0 +1,41 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + + +MergeType checkAndGetMergeType(UInt64 merge_type) +{ + if (merge_type == static_cast(MergeType::REGULAR)) + return MergeType::REGULAR; + else if (merge_type == static_cast(MergeType::TTL_DELETE)) + return MergeType::TTL_DELETE; + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeType {}", static_cast(merge_type)); +} + +String toString(MergeType merge_type) +{ + switch (merge_type) + { + case MergeType::REGULAR: + return "REGULAR"; + case MergeType::TTL_DELETE: + return "TTL_DELETE"; + } + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeType {}", static_cast(merge_type)); +} + +bool isTTLMergeType(MergeType merge_type) +{ + return merge_type == MergeType::TTL_DELETE; +} + +} diff --git a/src/Storages/MergeTree/MergeType.h b/src/Storages/MergeTree/MergeType.h new file mode 100644 index 00000000000..20fd7cd24af --- /dev/null +++ b/src/Storages/MergeTree/MergeType.h @@ -0,0 +1,20 @@ +#pragma once + +#include + +namespace DB +{ + +enum class MergeType +{ + REGULAR, + TTL_DELETE, +}; + +MergeType checkAndGetMergeType(UInt64 merge_type); + +String toString(MergeType merge_type); + +bool isTTLMergeType(MergeType merge_type); + +} diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index af6d980ad98..d95ae6b729d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -36,6 +36,8 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const out << s << '\n'; out << "into\n" << new_part_name; out << "\ndeduplicate: " << deduplicate; + if (merge_type != MergeType::REGULAR) + out <<"\nmerge_type: " << static_cast(merge_type); break; case DROP_RANGE: @@ -148,8 +150,20 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) source_parts.push_back(s); } in >> new_part_name; + if (format_version >= 4) + { in >> "\ndeduplicate: " >> deduplicate; + in >> "\n"; + if (in.eof()) + trailing_newline_found = true; + else if (checkString("merge_type: ", in)) + { + UInt64 value; + in >> value; + merge_type = checkAndGetMergeType(value); + } + } } else if (type_str == "drop" || type_str == "detach") { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index ae5fad0b83c..2f5d038291b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -79,6 +80,7 @@ struct ReplicatedMergeTreeLogEntryData Strings source_parts; bool deduplicate = false; /// Do deduplicate on merge + MergeType merge_type = MergeType::REGULAR; String column_name; String index_name; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 8e2c3752212..61d53c60128 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1312,21 +1312,26 @@ bool ReplicatedMergeTreeQueue::processEntry( } -std::pair ReplicatedMergeTreeQueue::countMergesAndPartMutations() const +ReplicatedMergeTreeQueue::OperationsInQueue ReplicatedMergeTreeQueue::countMergesAndPartMutations() const { std::lock_guard lock(state_mutex); size_t count_merges = 0; size_t count_mutations = 0; + size_t count_merges_with_ttl = 0; for (const auto & entry : queue) { if (entry->type == ReplicatedMergeTreeLogEntry::MERGE_PARTS) + { ++count_merges; + if (isTTLMergeType(entry->merge_type)) + ++count_merges_with_ttl; + } else if (entry->type == ReplicatedMergeTreeLogEntry::MUTATE_PART) ++count_mutations; } - return std::make_pair(count_merges, count_mutations); + return OperationsInQueue{count_merges, count_mutations, count_merges_with_ttl}; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 76f84da1ae8..c024cd53c0b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -49,6 +49,12 @@ private: /// To calculate min_unprocessed_insert_time, max_processed_insert_time, for which the replica lag is calculated. using InsertsByTime = std::set; + struct OperationsInQueue + { + size_t merges = 0; + size_t mutations = 0; + size_t merges_with_ttl = 0; + }; StorageReplicatedMergeTree & storage; MergeTreeDataFormatVersion format_version; @@ -325,7 +331,7 @@ public: bool processEntry(std::function get_zookeeper, LogEntryPtr & entry, const std::function func); /// Count the number of merges and mutations of single parts in the queue. - std::pair countMergesAndPartMutations() const; + OperationsInQueue countMergesAndPartMutations() const; /// Count the total number of active mutations. size_t countMutations() const; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7e4318a32f6..729263c3aaa 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -651,8 +651,21 @@ bool StorageMergeTree::merge( if (partition_id.empty()) { UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(); + UInt64 max_source_parts_size_with_ttl = merger_mutator.getMaxSourcePartsSizeForMergeWithTTL(); + + /// TTL requirements is much more strict than for regular merge, so + /// if regular not possible, than merge with ttl is not also not + /// possible. if (max_source_parts_size > 0) - selected = merger_mutator.selectPartsToMerge(future_part, aggressive, max_source_parts_size, can_merge, out_disable_reason); + { + selected = merger_mutator.selectPartsToMerge( + future_part, + aggressive, + max_source_parts_size, + can_merge, + max_source_parts_size_with_ttl, + out_disable_reason); + } else if (out_disable_reason) *out_disable_reason = "Current value of max_source_parts_size is zero"; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6058632d220..a5b293fd30b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2514,13 +2514,13 @@ void StorageReplicatedMergeTree::mergeSelectingTask() /// and in the same time, many small parts could be created and won't be merged. auto merges_and_mutations_queued = queue.countMergesAndPartMutations(); - size_t merges_and_mutations_sum = merges_and_mutations_queued.first + merges_and_mutations_queued.second; + size_t merges_and_mutations_sum = merges_and_mutations_queued.merges + merges_and_mutations_queued.mutations; if (merges_and_mutations_sum >= storage_settings_ptr->max_replicated_merges_in_queue) { LOG_TRACE(log, "Number of queued merges ({}) and part mutations ({})" " is greater than max_replicated_merges_in_queue ({}), so won't select new parts to merge or mutate.", - merges_and_mutations_queued.first, - merges_and_mutations_queued.second, + merges_and_mutations_queued.merges, + merges_and_mutations_queued.mutations, storage_settings_ptr->max_replicated_merges_in_queue); } else @@ -2529,16 +2529,20 @@ void StorageReplicatedMergeTree::mergeSelectingTask() storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum); UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); + UInt64 max_source_part_size_for_merge_with_ttl = 0; + if (merges_and_mutations_queued.merges_with_ttl < storage_settings_ptr->max_replicated_merges_with_ttl_in_queue) + max_source_part_size_for_merge_with_ttl = merger_mutator.getMaxSourcePartsSizeForMergeWithTTL(); + FutureMergedMutatedPart future_merged_part; if (max_source_parts_size_for_merge > 0 && - merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, nullptr)) + merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, max_source_part_size_for_merge_with_ttl, nullptr)) { create_result = createLogEntryToMergeParts(zookeeper, future_merged_part.parts, - future_merged_part.name, future_merged_part.type, deduplicate, nullptr, merge_pred.getVersion()); + future_merged_part.name, future_merged_part.type, deduplicate, nullptr, merge_pred.getVersion(), future_merged_part.merge_type); } /// If there are many mutations in queue, it may happen, that we cannot enqueue enough merges to merge all new parts else if (max_source_part_size_for_mutation > 0 && queue.countMutations() > 0 - && merges_and_mutations_queued.second < storage_settings_ptr->max_replicated_mutations_in_queue) + && merges_and_mutations_queued.mutations < storage_settings_ptr->max_replicated_mutations_in_queue) { /// Choose a part to mutate. DataPartsVector data_parts = getDataPartsVector(); @@ -2617,7 +2621,8 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c const MergeTreeDataPartType & merged_part_type, bool deduplicate, ReplicatedMergeTreeLogEntryData * out_log_entry, - int32_t log_version) + int32_t log_version, + MergeType merge_type) { std::vector> exists_futures; exists_futures.reserve(parts.size()); @@ -2650,6 +2655,7 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c entry.new_part_name = merged_name; entry.new_part_type = merged_part_type; entry.deduplicate = deduplicate; + entry.merge_type = merge_type; entry.create_time = time(nullptr); for (const auto & part : parts) @@ -3584,7 +3590,7 @@ bool StorageReplicatedMergeTree::optimize( CreateMergeEntryResult create_result = createLogEntryToMergeParts( zookeeper, future_merged_part.parts, future_merged_part.name, future_merged_part.type, deduplicate, - &merge_entry, can_merge.getVersion()); + &merge_entry, can_merge.getVersion(), future_merged_part.merge_type); if (create_result == CreateMergeEntryResult::MissingPart) return handle_noop("Can't create merge queue node in ZooKeeper, because some parts are missing"); @@ -3614,7 +3620,7 @@ bool StorageReplicatedMergeTree::optimize( if (!partition) { selected = merger_mutator.selectPartsToMerge( - future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, &disable_reason); + future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, &disable_reason); } else { @@ -3639,7 +3645,7 @@ bool StorageReplicatedMergeTree::optimize( CreateMergeEntryResult create_result = createLogEntryToMergeParts( zookeeper, future_merged_part.parts, future_merged_part.name, future_merged_part.type, deduplicate, - &merge_entry, can_merge.getVersion()); + &merge_entry, can_merge.getVersion(), future_merged_part.merge_type); if (create_result == CreateMergeEntryResult::MissingPart) return handle_noop("Can't create merge queue node in ZooKeeper, because some parts are missing"); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index e9395f20f3f..2bc9265331d 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -450,7 +450,8 @@ private: const MergeTreeDataPartType & merged_part_type, bool deduplicate, ReplicatedMergeTreeLogEntryData * out_log_entry, - int32_t log_version); + int32_t log_version, + MergeType merge_type); CreateMergeEntryResult createLogEntryToMutatePart( const IMergeTreeDataPart & part, diff --git a/src/Storages/System/StorageSystemMerges.cpp b/src/Storages/System/StorageSystemMerges.cpp index 39d22bd00ca..3b9e39c1ef8 100644 --- a/src/Storages/System/StorageSystemMerges.cpp +++ b/src/Storages/System/StorageSystemMerges.cpp @@ -30,6 +30,7 @@ NamesAndTypesList StorageSystemMerges::getNamesAndTypes() {"columns_written", std::make_shared()}, {"memory_usage", std::make_shared()}, {"thread_id", std::make_shared()}, + {"merge_type", std::make_shared()}, }; } @@ -65,6 +66,10 @@ void StorageSystemMerges::fillData(MutableColumns & res_columns, const Context & res_columns[i++]->insert(merge.columns_written); res_columns[i++]->insert(merge.memory_usage); res_columns[i++]->insert(merge.thread_id); + if (!merge.is_mutation) + res_columns[i++]->insert(merge.merge_type); + else + res_columns[i++]->insertDefault(); } } diff --git a/src/Storages/System/StorageSystemReplicationQueue.cpp b/src/Storages/System/StorageSystemReplicationQueue.cpp index f5e43abada0..f04d8759507 100644 --- a/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -42,6 +42,7 @@ NamesAndTypesList StorageSystemReplicationQueue::getNamesAndTypes() { "num_postponed", std::make_shared() }, { "postpone_reason", std::make_shared() }, { "last_postpone_time", std::make_shared() }, + { "merge_type", std::make_shared() }, }; } @@ -145,6 +146,11 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const res_columns[col_num++]->insert(entry.num_postponed); res_columns[col_num++]->insert(entry.postpone_reason); res_columns[col_num++]->insert(UInt64(entry.last_postpone_time)); + + if (entry.type == ReplicatedMergeTreeLogEntryData::Type::MERGE_PARTS) + res_columns[col_num++]->insert(toString(entry.merge_type)); + else + res_columns[col_num++]->insertDefault(); } } } From 13248a744b4409460cf458e075d37a27e9d9f3ef Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 3 Sep 2020 16:02:24 +0300 Subject: [PATCH 040/298] Fix bug in parts selection --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 32 +++++-- .../test_concurrent_ttl_merges/__init__.py | 0 .../configs/fast_background_pool.xml | 9 ++ .../test_concurrent_ttl_merges/test.py | 83 +++++++++++++++++++ 4 files changed, 116 insertions(+), 8 deletions(-) create mode 100644 tests/integration/test_concurrent_ttl_merges/__init__.py create mode 100644 tests/integration/test_concurrent_ttl_merges/configs/fast_background_pool.xml create mode 100644 tests/integration/test_concurrent_ttl_merges/test.py diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 670f42d916d..4c57fbc01ff 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -250,8 +250,20 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( const String * prev_partition_id = nullptr; /// Previous part only in boundaries of partition frame const MergeTreeData::DataPartPtr * prev_part = nullptr; + for (const MergeTreeData::DataPartPtr & part : data_parts) { + const String & partition_id = part->info.partition_id; + + if (!prev_partition_id || partition_id != *prev_partition_id) + { + if (partitions.empty() || !partitions.back().empty()) + partitions.emplace_back(); + /// New partition frame. + prev_partition_id = &partition_id; + prev_part = nullptr; + } + /// Check predicate only for first part in each partition. if (!prev_part) { @@ -262,15 +274,19 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( if (!can_merge_callback(nullptr, part, nullptr)) continue; } - - const String & partition_id = part->info.partition_id; - if (!prev_partition_id || partition_id != *prev_partition_id || (prev_part && !can_merge_callback(*prev_part, part, nullptr))) + else { - if (partitions.empty() || !partitions.back().empty()) - partitions.emplace_back(); - /// New partition frame. - prev_partition_id = &partition_id; - prev_part = nullptr; + /// If we cannot merge with previous part we had to start new parts + /// interval (in the same partition) + if (!can_merge_callback(*prev_part, part, nullptr)) + { + /// Starting new interval in the same partition + if (!partitions.back().empty()) + partitions.emplace_back(); + + /// Now we haven't previous part, but it affects only logging + prev_part = nullptr; + } } IMergeSelector::Part part_info; diff --git a/tests/integration/test_concurrent_ttl_merges/__init__.py b/tests/integration/test_concurrent_ttl_merges/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_concurrent_ttl_merges/configs/fast_background_pool.xml b/tests/integration/test_concurrent_ttl_merges/configs/fast_background_pool.xml new file mode 100644 index 00000000000..e62a0105907 --- /dev/null +++ b/tests/integration/test_concurrent_ttl_merges/configs/fast_background_pool.xml @@ -0,0 +1,9 @@ + + 1 + 0 + 0.0 + 0 + 1 + 1 + 0 + diff --git a/tests/integration/test_concurrent_ttl_merges/test.py b/tests/integration/test_concurrent_ttl_merges/test.py new file mode 100644 index 00000000000..9da56f8d3e0 --- /dev/null +++ b/tests/integration/test_concurrent_ttl_merges/test.py @@ -0,0 +1,83 @@ +import time +import pytest + +import helpers.client as client +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV +from helpers.test_tools import assert_eq_with_retry + + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=['configs/fast_background_pool.xml'], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=['configs/fast_background_pool.xml'], with_zookeeper=True) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def count_ttl_merges_in_queue(node, table): + result = node.query("SELECT count() FROM system.replication_queue WHERE merge_type = 'TTL_DELETE' and table = '{}'".format(table)) + if not result: + return 0 + return int(result.strip()) + + +def count_regular_merges_in_queue(node, table): + result = node.query("SELECT count() FROM system.replication_queue WHERE merge_type = 'REGULAR' and table = '{}'".format(table)) + if not result: + return 0 + return int(result.strip()) + + +def count_ttl_merges_in_background_pool(node, table): + result = node.query("SELECT count() FROM system.merges WHERE merge_type = 'TTL_DELETE' and table = '{}'".format(table)) + if not result: + return 0 + return int(result.strip()) + + +def count_regular_merges_in_background_pool(node, table): + result = node.query("SELECT count() FROM system.merges WHERE merge_type = 'REGULAR' and table = '{}'".format(table)) + if not result: + return 0 + return int(result.strip()) + + +def count_running_mutations(node, table): + result = node.query("SELECT count() FROM system.merges WHERE table = '{}' and is_mutation=1".format(table)) + if not result: + return 0 + return int(result.strip()) + + +def test_no_ttl_merges_in_busy_pool(started_cluster): + node1.query("CREATE TABLE test_ttl (d DateTime, key UInt64, data UInt64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 + sleepEachRow(1) MONTH SETTINGS merge_with_ttl_timeout = 0, number_of_free_entries_in_pool_to_execute_mutation = 0") + + node1.query("SYSTEM STOP TTL MERGES") + + for i in range(1, 7): + node1.query("INSERT INTO test_ttl SELECT now() - INTERVAL 1 MONTH + number - 1, {}, number FROM numbers(5)".format(i)) + + node1.query("ALTER TABLE test_ttl UPDATE data = data + 1 WHERE sleepEachRow(1) = 0") + + while count_running_mutations(node1, "test_ttl") < 6: + print "Mutations count", count_running_mutations(node1, "test_ttl") + assert count_ttl_merges_in_background_pool(node1, "test_ttl") == 0 + time.sleep(0.5) + + node1.query("SYSTEM START TTL MERGES") + + while count_running_mutations(node1, "test_ttl") == 6: + print "Mutations count after start TTL", count_running_mutations(node1, "test_ttl") + assert node1.query("SELECT count() FROM test_ttl") == "30\n" + time.sleep(0.5) + + assert_eq_with_retry(node1, "SELECT COUNT() FROM test_ttl", "0") From d5da58918e8245652269381df4e70eed346be2b8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 4 Sep 2020 01:04:46 +0300 Subject: [PATCH 041/298] create less compressed streams while writing compact parts --- src/Compression/CompressionCodecDelta.cpp | 6 ++++ src/Compression/CompressionCodecDelta.h | 3 ++ .../CompressionCodecDoubleDelta.cpp | 6 ++++ src/Compression/CompressionCodecDoubleDelta.h | 3 ++ src/Compression/CompressionCodecGorilla.cpp | 6 ++++ src/Compression/CompressionCodecGorilla.h | 3 ++ src/Compression/CompressionCodecLZ4.cpp | 5 ++++ src/Compression/CompressionCodecLZ4.h | 2 ++ src/Compression/CompressionCodecMultiple.cpp | 6 ++++ src/Compression/CompressionCodecMultiple.h | 3 ++ src/Compression/CompressionCodecNone.cpp | 5 ++++ src/Compression/CompressionCodecNone.h | 3 ++ src/Compression/CompressionCodecT64.cpp | 7 +++++ src/Compression/CompressionCodecT64.h | 2 ++ src/Compression/CompressionCodecZSTD.cpp | 5 ++++ src/Compression/CompressionCodecZSTD.h | 3 ++ src/Compression/ICompressionCodec.cpp | 7 +++++ src/Compression/ICompressionCodec.h | 5 ++++ .../MergeTreeDataPartWriterCompact.cpp | 29 ++++++++++++++----- .../MergeTreeDataPartWriterCompact.h | 16 +++++----- 20 files changed, 110 insertions(+), 15 deletions(-) diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index 51bd19f646b..dc866e527d6 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -36,6 +36,12 @@ ASTPtr CompressionCodecDelta::getCodecDesc() const return makeASTFunction("Delta", literal); } +void CompressionCodecDelta::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); + hash.update(delta_bytes_size); +} + namespace { diff --git a/src/Compression/CompressionCodecDelta.h b/src/Compression/CompressionCodecDelta.h index 5c3979e063e..a192fab051a 100644 --- a/src/Compression/CompressionCodecDelta.h +++ b/src/Compression/CompressionCodecDelta.h @@ -14,7 +14,10 @@ public: ASTPtr getCodecDesc() const override; + void updateHash(SipHash & hash) const override; + protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index 157e2df1a3f..dd2e95a916d 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -339,6 +339,12 @@ ASTPtr CompressionCodecDoubleDelta::getCodecDesc() const return std::make_shared("DoubleDelta"); } +void CompressionCodecDoubleDelta::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); + hash.update(data_bytes_size); +} + UInt32 CompressionCodecDoubleDelta::getMaxCompressedDataSize(UInt32 uncompressed_size) const { const auto result = 2 // common header diff --git a/src/Compression/CompressionCodecDoubleDelta.h b/src/Compression/CompressionCodecDoubleDelta.h index a2690d24414..30ef086077d 100644 --- a/src/Compression/CompressionCodecDoubleDelta.h +++ b/src/Compression/CompressionCodecDoubleDelta.h @@ -100,7 +100,10 @@ public: ASTPtr getCodecDesc() const override; + void updateHash(SipHash & hash) const override; + protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; diff --git a/src/Compression/CompressionCodecGorilla.cpp b/src/Compression/CompressionCodecGorilla.cpp index 042835f4a32..3d08734fe91 100644 --- a/src/Compression/CompressionCodecGorilla.cpp +++ b/src/Compression/CompressionCodecGorilla.cpp @@ -254,6 +254,12 @@ ASTPtr CompressionCodecGorilla::getCodecDesc() const return std::make_shared("Gorilla"); } +void CompressionCodecGorilla::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); + hash.update(data_bytes_size); +} + UInt32 CompressionCodecGorilla::getMaxCompressedDataSize(UInt32 uncompressed_size) const { const auto result = 2 // common header diff --git a/src/Compression/CompressionCodecGorilla.h b/src/Compression/CompressionCodecGorilla.h index 523add0700f..df0f329dc31 100644 --- a/src/Compression/CompressionCodecGorilla.h +++ b/src/Compression/CompressionCodecGorilla.h @@ -97,7 +97,10 @@ public: ASTPtr getCodecDesc() const override; + void updateHash(SipHash & hash) const override; + protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; diff --git a/src/Compression/CompressionCodecLZ4.cpp b/src/Compression/CompressionCodecLZ4.cpp index cf3622cd702..1370349d68d 100644 --- a/src/Compression/CompressionCodecLZ4.cpp +++ b/src/Compression/CompressionCodecLZ4.cpp @@ -35,6 +35,11 @@ ASTPtr CompressionCodecLZ4::getCodecDesc() const return std::make_shared("LZ4"); } +void CompressionCodecLZ4::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); +} + UInt32 CompressionCodecLZ4::getMaxCompressedDataSize(UInt32 uncompressed_size) const { return LZ4_COMPRESSBOUND(uncompressed_size); diff --git a/src/Compression/CompressionCodecLZ4.h b/src/Compression/CompressionCodecLZ4.h index 2f19af08185..229e25481e6 100644 --- a/src/Compression/CompressionCodecLZ4.h +++ b/src/Compression/CompressionCodecLZ4.h @@ -18,6 +18,8 @@ public: UInt32 getAdditionalSizeAtTheEndOfBuffer() const override { return LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER; } + void updateHash(SipHash & hash) const override; + protected: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; diff --git a/src/Compression/CompressionCodecMultiple.cpp b/src/Compression/CompressionCodecMultiple.cpp index 868df90825e..77f0fc132fe 100644 --- a/src/Compression/CompressionCodecMultiple.cpp +++ b/src/Compression/CompressionCodecMultiple.cpp @@ -37,6 +37,12 @@ ASTPtr CompressionCodecMultiple::getCodecDesc() const return result; } +void CompressionCodecMultiple::updateHash(SipHash & hash) const +{ + for (const auto & codec : codecs) + codec->updateHash(hash); +} + UInt32 CompressionCodecMultiple::getMaxCompressedDataSize(UInt32 uncompressed_size) const { UInt32 compressed_size = uncompressed_size; diff --git a/src/Compression/CompressionCodecMultiple.h b/src/Compression/CompressionCodecMultiple.h index cd50d3250e3..6bac189bdf7 100644 --- a/src/Compression/CompressionCodecMultiple.h +++ b/src/Compression/CompressionCodecMultiple.h @@ -19,7 +19,10 @@ public: static std::vector getCodecsBytesFromData(const char * source); + void updateHash(SipHash & hash) const override; + protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 decompressed_size) const override; diff --git a/src/Compression/CompressionCodecNone.cpp b/src/Compression/CompressionCodecNone.cpp index 50c19b2b547..f727c4b4860 100644 --- a/src/Compression/CompressionCodecNone.cpp +++ b/src/Compression/CompressionCodecNone.cpp @@ -17,6 +17,11 @@ ASTPtr CompressionCodecNone::getCodecDesc() const return std::make_shared("NONE"); } +void CompressionCodecNone::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); +} + UInt32 CompressionCodecNone::doCompressData(const char * source, UInt32 source_size, char * dest) const { memcpy(dest, source, source_size); diff --git a/src/Compression/CompressionCodecNone.h b/src/Compression/CompressionCodecNone.h index ed604063198..370ef301694 100644 --- a/src/Compression/CompressionCodecNone.h +++ b/src/Compression/CompressionCodecNone.h @@ -15,7 +15,10 @@ public: ASTPtr getCodecDesc() const override; + void updateHash(SipHash & hash) const override; + protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; diff --git a/src/Compression/CompressionCodecT64.cpp b/src/Compression/CompressionCodecT64.cpp index 16462e50ebd..30972a5fe1f 100644 --- a/src/Compression/CompressionCodecT64.cpp +++ b/src/Compression/CompressionCodecT64.cpp @@ -646,6 +646,13 @@ ASTPtr CompressionCodecT64::getCodecDesc() const return makeASTFunction("T64", literal); } +void CompressionCodecT64::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); + hash.update(type_idx); + hash.update(variant); +} + void registerCodecT64(CompressionCodecFactory & factory) { auto reg_func = [&](const ASTPtr & arguments, DataTypePtr type) -> CompressionCodecPtr diff --git a/src/Compression/CompressionCodecT64.h b/src/Compression/CompressionCodecT64.h index 11efbea0955..9671eb81ce1 100644 --- a/src/Compression/CompressionCodecT64.h +++ b/src/Compression/CompressionCodecT64.h @@ -35,6 +35,8 @@ public: ASTPtr getCodecDesc() const override; + void updateHash(SipHash & hash) const override; + protected: UInt32 doCompressData(const char * src, UInt32 src_size, char * dst) const override; void doDecompressData(const char * src, UInt32 src_size, char * dst, UInt32 uncompressed_size) const override; diff --git a/src/Compression/CompressionCodecZSTD.cpp b/src/Compression/CompressionCodecZSTD.cpp index ab48580533e..3b317884fec 100644 --- a/src/Compression/CompressionCodecZSTD.cpp +++ b/src/Compression/CompressionCodecZSTD.cpp @@ -32,6 +32,11 @@ ASTPtr CompressionCodecZSTD::getCodecDesc() const return makeASTFunction("ZSTD", literal); } +void CompressionCodecZSTD::updateHash(SipHash & hash) const +{ + getCodecDesc()->updateTreeHash(hash); +} + UInt32 CompressionCodecZSTD::getMaxCompressedDataSize(UInt32 uncompressed_size) const { return ZSTD_compressBound(uncompressed_size); diff --git a/src/Compression/CompressionCodecZSTD.h b/src/Compression/CompressionCodecZSTD.h index 2ad893083c3..3bfb6bb1d4d 100644 --- a/src/Compression/CompressionCodecZSTD.h +++ b/src/Compression/CompressionCodecZSTD.h @@ -21,7 +21,10 @@ public: UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override; + void updateHash(SipHash & hash) const override; + protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; diff --git a/src/Compression/ICompressionCodec.cpp b/src/Compression/ICompressionCodec.cpp index 4aafc298658..5de015b2680 100644 --- a/src/Compression/ICompressionCodec.cpp +++ b/src/Compression/ICompressionCodec.cpp @@ -35,6 +35,13 @@ ASTPtr ICompressionCodec::getFullCodecDesc() const return result; } +UInt64 ICompressionCodec::getHash() const +{ + SipHash hash; + updateHash(hash); + return hash.get64(); +} + UInt32 ICompressionCodec::compress(const char * source, UInt32 source_size, char * dest) const { assert(source != nullptr && dest != nullptr); diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index fa1f73ce4dd..8f72ba55200 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -36,6 +37,10 @@ public: /// "CODEC(LZ4,LZ4HC(5))" ASTPtr getFullCodecDesc() const; + /// Hash, that depends on codec ast and optional parameters like data type + virtual void updateHash(SipHash & hash) const = 0; + UInt64 getHash() const; + /// Compressed bytes from uncompressed source to dest. Dest should preallocate memory UInt32 compress(const char * source, UInt32 source_size, char * dest) const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 5e311e0a2f9..70beaec5e5e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -29,9 +29,18 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( , marks(*marks_file) { const auto & storage_columns = metadata_snapshot->getColumns(); + + /// Create compressed stream for every different codec. + std::unordered_map streams_by_codec; for (const auto & column : columns_list) - compressed_streams[column.name] = std::make_unique( - plain_hashing, storage_columns.getCodecOrDefault(column.name, default_codec)); + { + auto codec = storage_columns.getCodecOrDefault(column.name, default_codec); + auto & stream = streams_by_codec[codec->getHash()]; + if (!stream) + stream = std::make_shared(plain_hashing, codec); + + compressed_streams.push_back(stream); + } } void MergeTreeDataPartWriterCompact::write( @@ -101,14 +110,15 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) if (rows_to_write) data_written = true; - for (const auto & column : columns_list) + auto name_and_type = columns_list.begin(); + for (size_t i = 0; i < columns_list.size(); ++i, ++name_and_type) { - auto & stream = compressed_streams[column.name]; + auto & stream = compressed_streams[i]; writeIntBinary(plain_hashing.count(), marks); writeIntBinary(stream->hashing_buf.offset(), marks); - writeColumnSingleGranule(block.getByName(column.name), current_row, rows_to_write); + writeColumnSingleGranule(block.getByName(name_and_type->name), stream, current_row, rows_to_write); /// Write one compressed block per column in granule for more optimal reading. stream->hashing_buf.next(); @@ -133,12 +143,15 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) next_mark = from_mark; } -void MergeTreeDataPartWriterCompact::writeColumnSingleGranule(const ColumnWithTypeAndName & column, size_t from_row, size_t number_of_rows) const +void MergeTreeDataPartWriterCompact::writeColumnSingleGranule( + const ColumnWithTypeAndName & column, + const CompressedStreamPtr & stream, + size_t from_row, size_t number_of_rows) const { IDataType::SerializeBinaryBulkStatePtr state; IDataType::SerializeBinaryBulkSettings serialize_settings; - serialize_settings.getter = [this, &column](IDataType::SubstreamPath) -> WriteBuffer * { return &compressed_streams.at(column.name)->hashing_buf; }; + serialize_settings.getter = [&stream](IDataType::SubstreamPath) -> WriteBuffer * { return &stream->hashing_buf; }; serialize_settings.position_independent_encoding = true; serialize_settings.low_cardinality_max_dictionary_size = 0; @@ -213,7 +226,7 @@ void MergeTreeDataPartWriterCompact::addToChecksums(MergeTreeDataPartChecksums & size_t uncompressed_size = 0; CityHash_v1_0_2::uint128 uncompressed_hash{0, 0}; - for (const auto & [_, stream] : compressed_streams) + for (const auto & stream : compressed_streams) { uncompressed_size += stream->hashing_buf.count(); auto stream_hash = stream->hashing_buf.getHash(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 6206d6e867e..2f24c515fb3 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -26,12 +26,6 @@ protected: void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) override; private: - /// Write single granule of one column (rows between 2 marks) - void writeColumnSingleGranule( - const ColumnWithTypeAndName & column, - size_t from_row, - size_t number_of_rows) const; - void writeBlock(const Block & block); void addToChecksums(MergeTreeDataPartChecksums & checksumns); @@ -67,11 +61,19 @@ private: : compressed_buf(buf, codec), hashing_buf(compressed_buf) {} }; - std::unordered_map> compressed_streams; + using CompressedStreamPtr = std::shared_ptr; + std::vector compressed_streams; /// marks -> marks_file std::unique_ptr marks_file; HashingWriteBuffer marks; + + /// Write single granule of one column (rows between 2 marks) + void writeColumnSingleGranule( + const ColumnWithTypeAndName & column, + const CompressedStreamPtr & stream, + size_t from_row, + size_t number_of_rows) const; }; } From 6883ee7eea30329c48c25c83bee341b4fdd886d7 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 4 Sep 2020 01:38:17 +0300 Subject: [PATCH 042/298] create less compressed streams while writing compact parts --- src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp | 5 +---- src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h | 5 +++++ 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 70beaec5e5e..22df2abecf3 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -29,9 +29,6 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( , marks(*marks_file) { const auto & storage_columns = metadata_snapshot->getColumns(); - - /// Create compressed stream for every different codec. - std::unordered_map streams_by_codec; for (const auto & column : columns_list) { auto codec = storage_columns.getCodecOrDefault(column.name, default_codec); @@ -226,7 +223,7 @@ void MergeTreeDataPartWriterCompact::addToChecksums(MergeTreeDataPartChecksums & size_t uncompressed_size = 0; CityHash_v1_0_2::uint128 uncompressed_hash{0, 0}; - for (const auto & stream : compressed_streams) + for (const auto & [_, stream] : streams_by_codec) { uncompressed_size += stream->hashing_buf.count(); auto stream_hash = stream->hashing_buf.getHash(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 2f24c515fb3..a121554f4be 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -62,6 +62,11 @@ private: }; using CompressedStreamPtr = std::shared_ptr; + + /// Create compressed stream for every different codec. + std::unordered_map streams_by_codec; + + /// For better performance save pointer to stream by every column. std::vector compressed_streams; /// marks -> marks_file From 23b9677879a2a0618b35032439650ec08e760c57 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 4 Sep 2020 08:46:58 +0300 Subject: [PATCH 043/298] Added a script to import git repository to ClickHouse --- src/Common/ShellCommand.cpp | 4 + src/IO/ReadBufferFromFile.cpp | 3 + src/IO/WriteBufferFromFile.cpp | 3 + utils/CMakeLists.txt | 1 + utils/git-to-clickhouse/CMakeLists.txt | 2 + utils/git-to-clickhouse/git-to-clickhouse.cpp | 638 ++++++++++++++++++ 6 files changed, 651 insertions(+) create mode 100644 utils/git-to-clickhouse/CMakeLists.txt create mode 100644 utils/git-to-clickhouse/git-to-clickhouse.cpp diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 53ab2301a0a..127f95fef06 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -186,6 +186,10 @@ int ShellCommand::tryWait() { wait_called = true; + in.close(); + out.close(); + err.close(); + LOG_TRACE(getLogger(), "Will wait for shell command pid {}", pid); int status = 0; diff --git a/src/IO/ReadBufferFromFile.cpp b/src/IO/ReadBufferFromFile.cpp index 40f69625e68..226615c757e 100644 --- a/src/IO/ReadBufferFromFile.cpp +++ b/src/IO/ReadBufferFromFile.cpp @@ -77,6 +77,9 @@ ReadBufferFromFile::~ReadBufferFromFile() void ReadBufferFromFile::close() { + if (fd < 0) + return; + if (0 != ::close(fd)) throw Exception("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE); diff --git a/src/IO/WriteBufferFromFile.cpp b/src/IO/WriteBufferFromFile.cpp index b59a110edb4..4ade2e2c971 100644 --- a/src/IO/WriteBufferFromFile.cpp +++ b/src/IO/WriteBufferFromFile.cpp @@ -92,6 +92,9 @@ WriteBufferFromFile::~WriteBufferFromFile() /// Close file before destruction of object. void WriteBufferFromFile::close() { + if (fd < 0) + return; + next(); if (0 != ::close(fd)) diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 0dd95388e7d..dd03afe9fb8 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -29,6 +29,7 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) add_subdirectory (convert-month-partitioned-parts) add_subdirectory (checksum-for-compressed-block) add_subdirectory (wal-dump) + add_subdirectory (git-to-clickhouse) endif () if (ENABLE_CODE_QUALITY) diff --git a/utils/git-to-clickhouse/CMakeLists.txt b/utils/git-to-clickhouse/CMakeLists.txt new file mode 100644 index 00000000000..0e46b68d471 --- /dev/null +++ b/utils/git-to-clickhouse/CMakeLists.txt @@ -0,0 +1,2 @@ +add_executable (git-to-clickhouse git-to-clickhouse.cpp) +target_link_libraries(git-to-clickhouse PRIVATE dbms boost::program_options) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp new file mode 100644 index 00000000000..42920328ad7 --- /dev/null +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -0,0 +1,638 @@ +#include +#include +#include + +#include + +#include +#include +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; +} + +enum class LineType +{ + Empty, + Comment, + Punct, + Code, +}; + +void writeText(LineType type, WriteBuffer & out) +{ + switch (type) + { + case LineType::Empty: writeString("Empty", out); break; + case LineType::Comment: writeString("Comment", out); break; + case LineType::Punct: writeString("Punct", out); break; + case LineType::Code: writeString("Code", out); break; + } +} + +struct LineChange +{ + int8_t sign{}; /// 1 if added, -1 if deleted + uint16_t line_number_old{}; + uint16_t line_number_new{}; + uint16_t hunk_num{}; /// ordinal number of hunk in diff, starting with 0 + uint16_t hunk_start_line_number_old{}; + uint16_t hunk_start_line_number_new{}; + std::string hunk_context; /// The context (like a line with function name) as it is calculated by git + std::string line; /// Line content without leading whitespaces + uint8_t indent{}; /// The number of leading whitespaces or tabs * 4 + LineType line_type{}; + + void setLineInfo(std::string full_line) + { + indent = 0; + + const char * pos = full_line.data(); + const char * end = pos + full_line.size(); + + while (pos < end) + { + if (*pos == ' ') + ++indent; + else if (*pos == '\t') + indent += 4; + else + break; + ++pos; + } + + line.assign(pos, end); + + if (pos == end) + { + line_type = LineType::Empty; + } + else if (pos + 1 < end + && ((pos[0] == '/' && pos[1] == '/') + || (pos[0] == '*' && pos[1] == ' '))) /// This is not precise. + { + line_type = LineType::Comment; + } + else + { + while (pos < end) + { + if (isAlphaNumericASCII(*pos)) + { + line_type = LineType::Code; + break; + } + ++pos; + } + if (pos == end) + line_type = LineType::Punct; + } + } + + void writeTextWithoutNewline(WriteBuffer & out) const + { + writeText(sign, out); + writeChar('\t', out); + writeText(line_number_old, out); + writeChar('\t', out); + writeText(line_number_new, out); + writeChar('\t', out); + writeText(hunk_num, out); + writeChar('\t', out); + writeText(hunk_start_line_number_old, out); + writeChar('\t', out); + writeText(hunk_start_line_number_new, out); + writeChar('\t', out); + writeText(hunk_context, out); + writeChar('\t', out); + writeText(line, out); + writeChar('\t', out); + writeText(indent, out); + writeChar('\t', out); + writeText(line_type, out); + } +}; + +using LineChanges = std::vector; + +enum class FileChangeType +{ + Add, + Delete, + Modify, + Rename, + Copy, + Type, +}; + +void writeText(FileChangeType type, WriteBuffer & out) +{ + switch (type) + { + case FileChangeType::Add: writeString("Add", out); break; + case FileChangeType::Delete: writeString("Delete", out); break; + case FileChangeType::Modify: writeString("Modify", out); break; + case FileChangeType::Rename: writeString("Rename", out); break; + case FileChangeType::Copy: writeString("Copy", out); break; + case FileChangeType::Type: writeString("Type", out); break; + } +} + +struct FileChange +{ + FileChangeType change_type{}; + std::string new_file_path; + std::string old_file_path; + uint16_t lines_added{}; + uint16_t lines_deleted{}; + uint16_t hunks_added{}; + uint16_t hunks_removed{}; + uint16_t hunks_changed{}; + + void writeTextWithoutNewline(WriteBuffer & out) const + { + writeText(change_type, out); + writeChar('\t', out); + writeText(new_file_path, out); + writeChar('\t', out); + writeText(old_file_path, out); + writeChar('\t', out); + writeText(lines_added, out); + writeChar('\t', out); + writeText(lines_deleted, out); + writeChar('\t', out); + writeText(hunks_added, out); + writeChar('\t', out); + writeText(hunks_removed, out); + writeChar('\t', out); + writeText(hunks_changed, out); + } +}; + +struct FileChangeAndLineChanges +{ + FileChange file_change; + LineChanges line_changes; +}; + +struct Commit +{ + std::string hash; + std::string author_name; + std::string author_email; + time_t time{}; + std::string message; + uint32_t files_added{}; + uint32_t files_deleted{}; + uint32_t files_renamed{}; + uint32_t files_modified{}; + uint32_t lines_added{}; + uint32_t lines_deleted{}; + uint32_t hunks_added{}; + uint32_t hunks_removed{}; + uint32_t hunks_changed{}; + + void writeTextWithoutNewline(WriteBuffer & out) const + { + writeText(hash, out); + writeChar('\t', out); + writeText(author_name, out); + writeChar('\t', out); + writeText(author_email, out); + writeChar('\t', out); + writeText(time, out); + writeChar('\t', out); + writeText(message, out); + writeChar('\t', out); + writeText(files_added, out); + writeChar('\t', out); + writeText(files_deleted, out); + writeChar('\t', out); + writeText(files_renamed, out); + writeChar('\t', out); + writeText(files_modified, out); + writeChar('\t', out); + writeText(lines_added, out); + writeChar('\t', out); + writeText(lines_deleted, out); + writeChar('\t', out); + writeText(hunks_added, out); + writeChar('\t', out); + writeText(hunks_removed, out); + writeChar('\t', out); + writeText(hunks_changed, out); + } +}; + + +void skipUntilWhitespace(ReadBuffer & buf) +{ + while (!buf.eof()) + { + char * next_pos = find_first_symbols<'\t', '\n', ' '>(buf.position(), buf.buffer().end()); + buf.position() = next_pos; + + if (!buf.hasPendingData()) + continue; + + if (*buf.position() == '\t' || *buf.position() == '\n' || *buf.position() == ' ') + return; + } +} + +void skipUntilNextLine(ReadBuffer & buf) +{ + while (!buf.eof()) + { + char * next_pos = find_first_symbols<'\n'>(buf.position(), buf.buffer().end()); + buf.position() = next_pos; + + if (!buf.hasPendingData()) + continue; + + if (*buf.position() == '\n') + { + ++buf.position(); + return; + } + } +} + +void readStringUntilNextLine(std::string & s, ReadBuffer & buf) +{ + s.clear(); + while (!buf.eof()) + { + char * next_pos = find_first_symbols<'\n'>(buf.position(), buf.buffer().end()); + s.append(buf.position(), next_pos - buf.position()); + buf.position() = next_pos; + + if (!buf.hasPendingData()) + continue; + + if (*buf.position() == '\n') + { + ++buf.position(); + return; + } + } +} + + +struct Result +{ + WriteBufferFromFile commits{"commits.tsv"}; + WriteBufferFromFile file_changes{"file_changes.tsv"}; + WriteBufferFromFile line_changes{"line_changes.tsv"}; +}; + + +void processCommit(std::string hash, Result & result) +{ + std::string command = fmt::format( + "git show --raw --pretty='format:%at%x09%aN%x09%aE%x0A%s%x00' --patch --unified=0 {}", + hash); + + std::cerr << command << "\n"; + + auto commit_info = ShellCommand::execute(command); + auto & in = commit_info->out; + + Commit commit; + commit.hash = hash; + + readText(commit.time, in); + assertChar('\t', in); + readText(commit.author_name, in); + assertChar('\t', in); + readText(commit.author_email, in); + assertChar('\n', in); + readNullTerminated(commit.message, in); + + std::cerr << fmt::format("{}\t{}\n", toString(LocalDateTime(commit.time)), commit.message); + + if (!in.eof()) + assertChar('\n', in); + + /// File changes in form + /// :100644 100644 b90fe6bb94 3ffe4c380f M src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp + /// :100644 100644 828dedf6b5 828dedf6b5 R100 dbms/src/Functions/GeoUtils.h dbms/src/Functions/PolygonUtils.h + + std::map file_changes; + + while (checkChar(':', in)) + { + FileChange file_change; + + for (size_t i = 0; i < 4; ++i) + { + skipUntilWhitespace(in); + skipWhitespaceIfAny(in); + } + + char change_type; + readChar(change_type, in); + + int confidence; + switch (change_type) + { + case 'A': + file_change.change_type = FileChangeType::Add; + ++commit.files_added; + break; + case 'D': + file_change.change_type = FileChangeType::Delete; + ++commit.files_deleted; + break; + case 'M': + file_change.change_type = FileChangeType::Modify; + ++commit.files_modified; + break; + case 'R': + file_change.change_type = FileChangeType::Rename; + ++commit.files_renamed; + readText(confidence, in); + break; + case 'C': + file_change.change_type = FileChangeType::Copy; + readText(confidence, in); + break; + case 'T': + file_change.change_type = FileChangeType::Type; + break; + default: + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected file change type: {}", change_type); + } + + skipWhitespaceIfAny(in); + + if (change_type == 'R' || change_type == 'C') + { + readText(file_change.old_file_path, in); + skipWhitespaceIfAny(in); + readText(file_change.new_file_path, in); + } + else + { + readText(file_change.new_file_path, in); + } + + assertChar('\n', in); + + file_changes.emplace( + file_change.new_file_path, + FileChangeAndLineChanges{ file_change, {} }); + } + + if (!in.eof()) + { + assertChar('\n', in); + + /// Diffs for every file in form of + /// --- a/src/Storages/StorageReplicatedMergeTree.cpp + /// +++ b/src/Storages/StorageReplicatedMergeTree.cpp + /// @@ -1387,2 +1387 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) + /// - table_lock, entry.create_time, reserved_space, entry.deduplicate, + /// - entry.force_ttl); + /// + table_lock, entry.create_time, reserved_space, entry.deduplicate); + + std::string old_file_path; + std::string new_file_path; + FileChangeAndLineChanges * file_change_and_line_changes = nullptr; + LineChange line_change; + + while (!in.eof()) + { + if (checkString("@@ ", in)) + { + if (!file_change_and_line_changes) + { + auto file_name = new_file_path.empty() ? old_file_path : new_file_path; + auto it = file_changes.find(file_name); + if (file_changes.end() == it) + std::cerr << fmt::format("Warning: skipping bad file name {}\n", file_name); + else + file_change_and_line_changes = &it->second; + } + + if (file_change_and_line_changes) + { + uint16_t old_lines = 1; + uint16_t new_lines = 1; + + assertChar('-', in); + readText(line_change.hunk_start_line_number_old, in); + if (checkChar(',', in)) + readText(old_lines, in); + + assertString(" +", in); + readText(line_change.hunk_start_line_number_new, in); + if (checkChar(',', in)) + readText(new_lines, in); + + assertString(" @@", in); + if (checkChar(' ', in)) + readStringUntilNextLine(line_change.hunk_context, in); + else + assertChar('\n', in); + + ++line_change.hunk_num; + line_change.line_number_old = line_change.hunk_start_line_number_old; + line_change.line_number_new = line_change.hunk_start_line_number_new; + + if (old_lines && new_lines) + { + ++commit.hunks_changed; + ++file_change_and_line_changes->file_change.hunks_changed; + } + else if (old_lines) + { + ++commit.hunks_removed; + ++file_change_and_line_changes->file_change.hunks_removed; + } + else if (new_lines) + { + ++commit.hunks_added; + ++file_change_and_line_changes->file_change.hunks_added; + } + } + } + else if (checkChar('-', in)) + { + if (checkString("-- ", in)) + { + if (checkString("a/", in)) + { + readStringUntilNextLine(old_file_path, in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else if (checkString("/dev/null", in)) + { + old_file_path.clear(); + assertChar('\n', in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else + skipUntilNextLine(in); /// Actually it can be the line in diff. Skip it for simplicity. + } + else + { + if (file_change_and_line_changes) + { + ++commit.lines_deleted; + + line_change.sign = -1; + readStringUntilNextLine(line_change.line, in); + line_change.setLineInfo(line_change.line); + + file_change_and_line_changes->line_changes.push_back(line_change); + ++line_change.line_number_old; + } + } + } + else if (checkChar('+', in)) + { + if (checkString("++ ", in)) + { + if (checkString("b/", in)) + { + readStringUntilNextLine(new_file_path, in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else if (checkString("/dev/null", in)) + { + new_file_path.clear(); + assertChar('\n', in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else + skipUntilNextLine(in); /// Actually it can be the line in diff. Skip it for simplicity. + } + else + { + if (file_change_and_line_changes) + { + ++commit.lines_added; + + line_change.sign = 1; + readStringUntilNextLine(line_change.line, in); + line_change.setLineInfo(line_change.line); + + file_change_and_line_changes->line_changes.push_back(line_change); + ++line_change.line_number_new; + } + } + } + else + { + skipUntilNextLine(in); + } + } + } + + /// Write the result + + /// commits table + { + auto & out = result.commits; + + commit.writeTextWithoutNewline(out); + writeChar('\n', out); + } + + for (const auto & elem : file_changes) + { + const FileChange & file_change = elem.second.file_change; + + /// file_changes table + { + auto & out = result.file_changes; + + file_change.writeTextWithoutNewline(out); + writeChar('\t', out); + commit.writeTextWithoutNewline(out); + writeChar('\n', out); + } + + /// line_changes table + for (const auto & line_change : elem.second.line_changes) + { + auto & out = result.line_changes; + + line_change.writeTextWithoutNewline(out); + writeChar('\t', out); + file_change.writeTextWithoutNewline(out); + writeChar('\t', out); + commit.writeTextWithoutNewline(out); + writeChar('\n', out); + } + } +} + + +void processLog() +{ + Result result; + + std::string command = "git log --no-merges --pretty=%H"; + std::cerr << command << "\n"; + auto git_log = ShellCommand::execute(command); + + auto & in = git_log->out; + while (!in.eof()) + { + std::string hash; + readString(hash, in); + assertChar('\n', in); + + std::cerr << fmt::format("Processing commit {}\n", hash); + processCommit(std::move(hash), result); + } +} + + +} + +int main(int /*argc*/, char ** /*argv*/) +try +{ + using namespace DB; + +/* boost::program_options::options_description desc("Allowed options"); + desc.add_options()("help,h", "produce help message"); + + boost::program_options::variables_map options; + boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); + + if (options.count("help") || argc != 2) + { + std::cout << "Usage: " << argv[0] << std::endl; + std::cout << desc << std::endl; + return 1; + }*/ + + processLog(); + return 0; +} +catch (...) +{ + std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; + throw; +} From 338a6e20f60bb21c99ee2c4f261d96bc55ec4b97 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 4 Sep 2020 09:12:16 +0300 Subject: [PATCH 044/298] Added a script to import git repository to ClickHouse --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 97 +++++++++++++++++++ 1 file changed, 97 insertions(+) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 42920328ad7..314bba0d5b4 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -16,6 +16,101 @@ #include +/** How to use: + +DROP DATABASE IF EXISTS git; +CREATE DATABASE git; + +CREATE TABLE git.commits +( + hash String, + author_name LowCardinality(String), + author_email LowCardinality(String), + time DateTime, + message String, + files_added UInt32, + files_deleted UInt32, + files_renamed UInt32, + files_modified UInt32, + lines_added UInt32, + lines_deleted UInt32, + hunks_added UInt32, + hunks_removed UInt32, + hunks_changed UInt32 +) ENGINE = MergeTree ORDER BY time; + +CREATE TABLE git.file_changes +( + change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), + new_file_path LowCardinality(String), + old_file_path LowCardinality(String), + lines_added UInt16, + lines_deleted UInt16, + hunks_added UInt16, + hunks_removed UInt16, + hunks_changed UInt16, + + commit_hash String, + author_name LowCardinality(String), + author_email LowCardinality(String), + time DateTime, + commit_message String, + commit_files_added UInt32, + commit_files_deleted UInt32, + commit_files_renamed UInt32, + commit_files_modified UInt32, + commit_lines_added UInt32, + commit_lines_deleted UInt32, + commit_hunks_added UInt32, + commit_hunks_removed UInt32, + commit_hunks_changed UInt32 +) ENGINE = MergeTree ORDER BY time; + +CREATE TABLE git.line_changes +( + sign Int8, + line_number_old UInt16, + line_number_new UInt16, + hunk_num UInt16, + hunk_start_line_number_old UInt16, + hunk_start_line_number_new UInt16, + hunk_context LowCardinality(String), + line LowCardinality(String), + indent UInt8, + line_type Enum('Empty' = 0, 'Comment' = 1, 'Punct' = 2, 'Code' = 3), + + file_change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), + new_file_path LowCardinality(String), + old_file_path LowCardinality(String), + file_lines_added UInt16, + file_lines_deleted UInt16, + file_hunks_added UInt16, + file_hunks_removed UInt16, + file_hunks_changed UInt16, + + commit_hash String, + author_name LowCardinality(String), + author_email LowCardinality(String), + time DateTime, + commit_message String, + commit_files_added UInt32, + commit_files_deleted UInt32, + commit_files_renamed UInt32, + commit_files_modified UInt32, + commit_lines_added UInt32, + commit_lines_deleted UInt32, + commit_hunks_added UInt32, + commit_hunks_removed UInt32, + commit_hunks_changed UInt32 +) ENGINE = MergeTree ORDER BY time; + +clickhouse-client --query "INSERT INTO git.commits FORMAT TSV" < commits.tsv +clickhouse-client --query "INSERT INTO git.file_changes FORMAT TSV" < file_changes.tsv +clickhouse-client --query "INSERT INTO git.line_changes FORMAT TSV" < line_changes.tsv + + */ + + namespace DB { @@ -495,6 +590,7 @@ void processCommit(std::string hash, Result & result) if (file_change_and_line_changes) { ++commit.lines_deleted; + ++file_change_and_line_changes->file_change.lines_deleted; line_change.sign = -1; readStringUntilNextLine(line_change.line, in); @@ -530,6 +626,7 @@ void processCommit(std::string hash, Result & result) if (file_change_and_line_changes) { ++commit.lines_added; + ++file_change_and_line_changes->file_change.lines_added; line_change.sign = 1; readStringUntilNextLine(line_change.line, in); From 61ecaebcb1b8a306bfda2fec90a20171427d2164 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Sep 2020 09:55:19 +0300 Subject: [PATCH 045/298] Simplify settings for TTL merges --- src/Storages/MergeTree/MergeList.h | 14 ++++++++++- .../MergeTree/MergeTreeDataMergerMutator.cpp | 25 +++---------------- .../MergeTree/MergeTreeDataMergerMutator.h | 9 +------ src/Storages/MergeTree/MergeTreeSettings.cpp | 13 ---------- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- src/Storages/StorageMergeTree.cpp | 6 +++-- src/Storages/StorageReplicatedMergeTree.cpp | 10 ++++---- 7 files changed, 28 insertions(+), 51 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 0b41745a9ba..5af71b88341 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -132,6 +132,7 @@ class MergeList mutable std::mutex mutex; container_t merges; + std::atomic merges_with_ttl_counter = 0; public: using Entry = MergeListEntry; using EntryPtr = std::unique_ptr; @@ -140,7 +141,9 @@ public: EntryPtr insert(Args &&... args) { std::lock_guard lock{mutex}; - return std::make_unique(*this, merges.emplace(merges.end(), std::forward(args)...)); + auto entry = std::make_unique(*this, merges.emplace(merges.end(), std::forward(args)...)); + merges_with_ttl_counter += (*entry)->merge_type == MergeType::TTL_DELETE; + return entry; } info_container_t get() const @@ -163,12 +166,21 @@ public: merge_element.is_cancelled = true; } } + + size_t getExecutingMergesWithTTLCount() const + { + return merges_with_ttl_counter; + } }; inline MergeListEntry::~MergeListEntry() { std::lock_guard lock{list.mutex}; + + if (it->merge_type == MergeType::TTL_DELETE) + list.merges_with_ttl_counter--; + list.merges.erase(it); } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index ac86f483694..d25bb5224cf 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -207,34 +207,17 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation() return 0; } - -UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMergeWithTTL() -{ - const auto data_settings = data.getSettings(); - size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); - - /// DataPart can be store only at one disk. Get maximum reservable free space at all disks. - UInt64 disk_space = data.getStoragePolicy()->getMaxUnreservedFreeSpace(); - - /// Allow merges with TTL only if there are enough threads, leave free threads for regular merges - if (busy_threads_in_pool <= 1 - || background_pool_size - busy_threads_in_pool >= data_settings->number_of_free_entries_in_pool_to_execute_merge_with_ttl) - return static_cast(disk_space / DISK_USAGE_COEFFICIENT_TO_RESERVE); - - return 0; - -} - bool MergeTreeDataMergerMutator::selectPartsToMerge( FutureMergedMutatedPart & future_part, bool aggressive, size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge_callback, - size_t max_total_size_to_merge_with_ttl, + bool merge_with_ttl_allowed, String * out_disable_reason) { MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector(); const auto data_settings = data.getSettings(); + auto metadata_snapshot = data.getInMemoryMetadataPtr(); if (data_parts.empty()) { @@ -311,7 +294,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( IMergeSelector::PartsRange parts_to_merge; - if (!ttl_merges_blocker.isCancelled()) + if (metadata_snapshot->hasAnyTTL() && merge_with_ttl_allowed && !ttl_merges_blocker.isCancelled()) { TTLMergeSelector merge_selector( next_ttl_merge_times_by_partition, @@ -319,7 +302,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( data_settings->merge_with_ttl_timeout, data_settings->ttl_only_drop_parts); - parts_to_merge = merge_selector.select(parts_ranges, max_total_size_to_merge_with_ttl); + parts_to_merge = merge_selector.select(parts_ranges, max_total_size_to_merge); if (!parts_to_merge.empty()) future_part.merge_type = MergeType::TTL_DELETE; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index a874c93e2f6..492807fe39a 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -61,13 +61,6 @@ public: */ UInt64 getMaxSourcePartsSizeForMerge(); - /** Get maximum total size of parts to do merge with TTL, at current moment - * of time. If busy threads count is less than value specified by - * number_of_free_entries_in_pool_to_execute_merge_with_ttl than maximum - * size (available on disk) is allowed. - */ - UInt64 getMaxSourcePartsSizeForMergeWithTTL(); - /** For explicitly passed size of pool and number of used tasks. * This method could be used to calculate threshold depending on number of tasks in replication queue. */ @@ -90,7 +83,7 @@ public: bool aggressive, size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge, - size_t max_total_size_to_merge_with_ttl, + bool merge_with_ttl_allowed, String * out_disable_reason = nullptr); /** Select all the parts in the specified partition for merge, if possible. diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 44504bdec84..7f537ec330a 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -100,19 +100,6 @@ void MergeTreeSettings::sanityCheck(const Settings & query_settings) const number_of_free_entries_in_pool_to_lower_max_size_of_merge, query_settings.background_pool_size); } - - if (number_of_free_entries_in_pool_to_execute_merge_with_ttl >= query_settings.background_pool_size) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_execute_merge_with_ttl' setting" - " ({}) (default values are defined in section of config.xml" - " or the value can be specified per table in SETTINGS section of CREATE TABLE query)" - " is greater or equals to the value of 'background_pool_size'" - " ({}) (the value is defined in users.xml for default profile)." - " This indicates incorrect configuration because TTL cannot work with these settings.", - number_of_free_entries_in_pool_to_execute_merge_with_ttl, - query_settings.background_pool_size); - } - } } diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 06fc21b24c3..80236d227ba 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -36,7 +36,7 @@ struct Settings; M(UInt64, max_replicated_merges_with_ttl_in_queue, 1, "How many tasks of merging parts with TTL are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ M(UInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ - M(UInt64, number_of_free_entries_in_pool_to_execute_merge_with_ttl, 12, "When there is less than specified number of free entries in pool, do not execute merge with TTL. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ + M(UInt64, max_number_of_merges_with_ttl_in_pool, 2, "When there is more than specified number of merges with TTL entries in pool, do not assign new merge with TTL. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ M(Seconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ M(Seconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ M(Seconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 729263c3aaa..d6cce2e98ae 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -627,11 +627,13 @@ bool StorageMergeTree::merge( { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto metadata_snapshot = getInMemoryMetadataPtr(); + auto data_settings = getSettings(); FutureMergedMutatedPart future_part; /// You must call destructor with unlocked `currently_processing_in_background_mutex`. std::optional merging_tagger; + auto & merge_list = global_context.getMergeList(); { std::unique_lock lock(currently_processing_in_background_mutex); @@ -651,7 +653,7 @@ bool StorageMergeTree::merge( if (partition_id.empty()) { UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(); - UInt64 max_source_parts_size_with_ttl = merger_mutator.getMaxSourcePartsSizeForMergeWithTTL(); + bool merge_with_ttl_allowed = merge_list.getExecutingMergesWithTTLCount() < data_settings->max_number_of_merges_with_ttl_in_pool; /// TTL requirements is much more strict than for regular merge, so /// if regular not possible, than merge with ttl is not also not @@ -663,7 +665,7 @@ bool StorageMergeTree::merge( aggressive, max_source_parts_size, can_merge, - max_source_parts_size_with_ttl, + merge_with_ttl_allowed, out_disable_reason); } else if (out_disable_reason) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a5b293fd30b..6823d6c3129 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2525,17 +2525,17 @@ void StorageReplicatedMergeTree::mergeSelectingTask() } else { + const auto & merge_list = global_context.getMergeList(); UInt64 max_source_parts_size_for_merge = merger_mutator.getMaxSourcePartsSizeForMerge( storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum); UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); - UInt64 max_source_part_size_for_merge_with_ttl = 0; - if (merges_and_mutations_queued.merges_with_ttl < storage_settings_ptr->max_replicated_merges_with_ttl_in_queue) - max_source_part_size_for_merge_with_ttl = merger_mutator.getMaxSourcePartsSizeForMergeWithTTL(); + bool merge_with_ttl_allowed = merges_and_mutations_queued.merges_with_ttl < storage_settings_ptr->max_replicated_merges_with_ttl_in_queue && + merge_list.getExecutingMergesWithTTLCount() < storage_settings_ptr->max_number_of_merges_with_ttl_in_pool; FutureMergedMutatedPart future_merged_part; if (max_source_parts_size_for_merge > 0 && - merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, max_source_part_size_for_merge_with_ttl, nullptr)) + merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, merge_with_ttl_allowed, nullptr)) { create_result = createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, future_merged_part.type, deduplicate, nullptr, merge_pred.getVersion(), future_merged_part.merge_type); @@ -3620,7 +3620,7 @@ bool StorageReplicatedMergeTree::optimize( if (!partition) { selected = merger_mutator.selectPartsToMerge( - future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, &disable_reason); + future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, true, &disable_reason); } else { From 82c56349a5413311a4de51718567a776207d0c4f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Sep 2020 13:08:09 +0300 Subject: [PATCH 046/298] Some comments --- src/Storages/MergeTree/MergeTreeData.cpp | 5 ++ src/Storages/MergeTree/MergeTreeData.h | 5 ++ src/Storages/MergeTree/MergeType.h | 12 ++- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 29 +++++- src/Storages/StorageMergeTree.cpp | 9 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- .../test_concurrent_ttl_merges/test.py | 88 +++++++++++++++++-- 7 files changed, 135 insertions(+), 17 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b6a495161f5..5daecdbb3ef 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3666,4 +3666,9 @@ NamesAndTypesList MergeTreeData::getVirtuals() const }; } +size_t MergeTreeData::getTotalMergesWithTTLInMergeList() const +{ + return global_context.getMergeList().getExecutingMergesWithTTLCount(); +} + } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e088a1c098b..205700ecd64 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -646,6 +646,11 @@ public: /// TTL rule. bool isPartInTTLDestination(const TTLDescription & ttl, const IMergeTreeDataPart & part) const; + /// Get count of total merges with TTL in MergeList (system.merges) for all + /// tables (not only current table). + /// Method is cheap and doesn't require any locks. + size_t getTotalMergesWithTTLInMergeList() const; + using WriteAheadLogPtr = std::shared_ptr; WriteAheadLogPtr getWriteAheadLog(); diff --git a/src/Storages/MergeTree/MergeType.h b/src/Storages/MergeTree/MergeType.h index 20fd7cd24af..27efe1a8539 100644 --- a/src/Storages/MergeTree/MergeType.h +++ b/src/Storages/MergeTree/MergeType.h @@ -5,16 +5,24 @@ namespace DB { +/// Type of Merge. Used to control amount of different merges during merges +/// assignment. Also allows to apply special logic during merge process +/// (mergePartsToTemporaryPart). Stored in FutureMergedMutatedPart and +/// ReplicatedMergeTreeLogEntry. +/// +/// Order is important, don't try to change it. enum class MergeType { - REGULAR, - TTL_DELETE, + REGULAR = 1, + TTL_DELETE = 2, }; +/// Check parsed merge_type from raw int and get enum value. MergeType checkAndGetMergeType(UInt64 merge_type); String toString(MergeType merge_type); +/// Check this merge assigned with TTL bool isTTLMergeType(MergeType merge_type); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 61d53c60128..5fd15547ea4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1070,7 +1070,34 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( * because the leader replica does not assign merges of greater size (except OPTIMIZE PARTITION and OPTIMIZE FINAL). */ const auto data_settings = data.getSettings(); - bool ignore_max_size = (entry.type == LogEntry::MERGE_PARTS) && (max_source_parts_size == data_settings->max_bytes_to_merge_at_max_space_in_pool); + bool ignore_max_size = false; + if (entry.type == LogEntry::MERGE_PARTS) + { + ignore_max_size = max_source_parts_size == data_settings->max_bytes_to_merge_at_max_space_in_pool; + + if (isTTLMergeType(entry.merge_type)) + { + if (merger_mutator.ttl_merges_blocker.isCancelled()) + { + String reason = "Not executing log entry for part " + entry.new_part_name + " because merges with TTL is cancelled now."; + LOG_DEBUG(log, reason); + out_postpone_reason = reason; + return false; + } + size_t total_merges_with_ttl = data.getTotalMergesWithTTLInMergeList(); + if (total_merges_with_ttl >= data_settings->max_number_of_merges_with_ttl_in_pool) + { + const char * format_str = "Not executing log entry for part {}" + " because {} merges with TTL already executing, maximum {}."; + LOG_DEBUG(log, format_str, entry.new_part_name, total_merges_with_ttl, + data_settings->max_number_of_merges_with_ttl_in_pool); + + out_postpone_reason = fmt::format(format_str, entry.new_part_name, total_merges_with_ttl, + data_settings->max_number_of_merges_with_ttl_in_pool); + return false; + } + } + } if (!ignore_max_size && sum_parts_size_in_bytes > max_source_parts_size) { diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index d6cce2e98ae..347474753dc 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -633,7 +633,7 @@ bool StorageMergeTree::merge( /// You must call destructor with unlocked `currently_processing_in_background_mutex`. std::optional merging_tagger; - auto & merge_list = global_context.getMergeList(); + MergeList::EntryPtr merge_entry; { std::unique_lock lock(currently_processing_in_background_mutex); @@ -653,7 +653,7 @@ bool StorageMergeTree::merge( if (partition_id.empty()) { UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(); - bool merge_with_ttl_allowed = merge_list.getExecutingMergesWithTTLCount() < data_settings->max_number_of_merges_with_ttl_in_pool; + bool merge_with_ttl_allowed = getTotalMergesWithTTLInMergeList() < data_settings->max_number_of_merges_with_ttl_in_pool; /// TTL requirements is much more strict than for regular merge, so /// if regular not possible, than merge with ttl is not also not @@ -716,11 +716,10 @@ bool StorageMergeTree::merge( } merging_tagger.emplace(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, false); + auto table_id = getStorageID(); + merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); } - auto table_id = getStorageID(); - MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); - /// Logging Stopwatch stopwatch; MutableDataPartPtr new_part; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6823d6c3129..13438821ba6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1377,6 +1377,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) + backQuote(entry.new_part_name), ErrorCodes::BAD_DATA_PART_NAME); } future_merged_part.updatePath(*this, reserved_space); + future_merged_part.merge_type = entry.merge_type; auto table_id = getStorageID(); MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_merged_part); @@ -2525,13 +2526,12 @@ void StorageReplicatedMergeTree::mergeSelectingTask() } else { - const auto & merge_list = global_context.getMergeList(); UInt64 max_source_parts_size_for_merge = merger_mutator.getMaxSourcePartsSizeForMerge( storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum); UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); bool merge_with_ttl_allowed = merges_and_mutations_queued.merges_with_ttl < storage_settings_ptr->max_replicated_merges_with_ttl_in_queue && - merge_list.getExecutingMergesWithTTLCount() < storage_settings_ptr->max_number_of_merges_with_ttl_in_pool; + getTotalMergesWithTTLInMergeList() < storage_settings_ptr->max_number_of_merges_with_ttl_in_pool; FutureMergedMutatedPart future_merged_part; if (max_source_parts_size_for_merge > 0 && diff --git a/tests/integration/test_concurrent_ttl_merges/test.py b/tests/integration/test_concurrent_ttl_merges/test.py index a82da2a1a8b..1ca303a6dcc 100644 --- a/tests/integration/test_concurrent_ttl_merges/test.py +++ b/tests/integration/test_concurrent_ttl_merges/test.py @@ -30,13 +30,6 @@ def count_ttl_merges_in_queue(node, table): return int(result.strip()) -def count_regular_merges_in_queue(node, table): - result = node.query("SELECT count() FROM system.replication_queue WHERE merge_type = 'REGULAR' and table = '{}'".format(table)) - if not result: - return 0 - return int(result.strip()) - - def count_ttl_merges_in_background_pool(node, table): result = node.query("SELECT count() FROM system.merges WHERE merge_type = 'TTL_DELETE' and table = '{}'".format(table)) if not result: @@ -84,3 +77,84 @@ def test_no_ttl_merges_in_busy_pool(started_cluster): time.sleep(0.5) assert_eq_with_retry(node1, "SELECT COUNT() FROM test_ttl", "0") + + +def test_limited_ttl_merges_in_empty_pool(started_cluster): + node1.query("CREATE TABLE test_ttl_v2 (d DateTime, key UInt64, data UInt64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0") + + node1.query("SYSTEM STOP TTL MERGES") + + for i in range(100): + node1.query("INSERT INTO test_ttl_v2 SELECT now() - INTERVAL 1 MONTH, {}, number FROM numbers(1)".format(i)) + + assert node1.query("SELECT COUNT() FROM test_ttl_v2") == "100\n" + + node1.query("SYSTEM START TTL MERGES") + + merges_with_ttl_count = set({}) + while True: + merges_with_ttl_count.add(count_ttl_merges_in_background_pool(node1, "test_ttl_v2")) + time.sleep(0.01) + if node1.query("SELECT COUNT() FROM test_ttl_v2") == "0\n": + break + + assert max(merges_with_ttl_count) <= 2 + + +def test_limited_ttl_merges_in_empty_pool_replicated(started_cluster): + node1.query("CREATE TABLE replicated_ttl (d DateTime, key UInt64, data UInt64) ENGINE = ReplicatedMergeTree('/test/t', '1') ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0") + + node1.query("SYSTEM STOP TTL MERGES") + + for i in range(100): + node1.query("INSERT INTO replicated_ttl SELECT now() - INTERVAL 1 MONTH, {}, number FROM numbers(1)".format(i)) + + assert node1.query("SELECT COUNT() FROM replicated_ttl") == "100\n" + + node1.query("SYSTEM START TTL MERGES") + + merges_with_ttl_count = set({}) + entries_with_ttl_count = set({}) + while True: + merges_with_ttl_count.add(count_ttl_merges_in_background_pool(node1, "replicated_ttl")) + entries_with_ttl_count.add(count_ttl_merges_in_queue(node1, "replicated_ttl")) + time.sleep(0.01) + if node1.query("SELECT COUNT() FROM replicated_ttl") == "0\n": + break + + assert max(merges_with_ttl_count) <= 2 + assert max(entries_with_ttl_count) <= 1 + + +def test_limited_ttl_merges_two_replicas(started_cluster): + # Actually this test quite fast and often we cannot catch any merges. + # To check for sure just add some sleeps in mergePartsToTemporaryPart + node1.query("CREATE TABLE replicated_ttl_2 (d DateTime, key UInt64, data UInt64) ENGINE = ReplicatedMergeTree('/test/t2', '1') ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0") + node2.query("CREATE TABLE replicated_ttl_2 (d DateTime, key UInt64, data UInt64) ENGINE = ReplicatedMergeTree('/test/t2', '2') ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0") + + node1.query("SYSTEM STOP TTL MERGES") + node2.query("SYSTEM STOP TTL MERGES") + + for i in range(100): + node1.query("INSERT INTO replicated_ttl_2 SELECT now() - INTERVAL 1 MONTH, {}, number FROM numbers(10000)".format(i)) + + node2.query("SYSTEM SYNC REPLICA replicated_ttl_2", timeout=10) + assert node1.query("SELECT COUNT() FROM replicated_ttl_2") == "1000000\n" + assert node2.query("SELECT COUNT() FROM replicated_ttl_2") == "1000000\n" + + node1.query("SYSTEM START TTL MERGES") + node2.query("SYSTEM START TTL MERGES") + + merges_with_ttl_count_node1 = set({}) + merges_with_ttl_count_node2 = set({}) + while True: + merges_with_ttl_count_node1.add(count_ttl_merges_in_background_pool(node1, "replicated_ttl_2")) + merges_with_ttl_count_node2.add(count_ttl_merges_in_background_pool(node2, "replicated_ttl_2")) + if node1.query("SELECT COUNT() FROM replicated_ttl_2") == "0\n" and node2.query("SELECT COUNT() FROM replicated_ttl_2") == "0\n": + break + + # Both replicas can assign merges with TTL. If one will perform better than + # the other slow replica may have several merges in queue, so we don't + # check them + assert max(merges_with_ttl_count_node1) <= 2 + assert max(merges_with_ttl_count_node2) <= 2 From 69b31ab90dab9916c5b0c3200a3bf49168368f1b Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Sep 2020 13:29:55 +0300 Subject: [PATCH 047/298] More comments --- src/Storages/MergeTree/MergeType.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/MergeTree/MergeType.h b/src/Storages/MergeTree/MergeType.h index 27efe1a8539..7ade9ddddd5 100644 --- a/src/Storages/MergeTree/MergeType.h +++ b/src/Storages/MergeTree/MergeType.h @@ -13,7 +13,9 @@ namespace DB /// Order is important, don't try to change it. enum class MergeType { + /// Just regular merge REGULAR = 1, + /// Merge assigned to delete some data from parts (with TTLMergeSelector) TTL_DELETE = 2, }; From 6f5ba4d8e51e75f4e6ab0c39dbf6b18f12daa58e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Sep 2020 13:31:29 +0300 Subject: [PATCH 048/298] Fix ya.make --- src/Storages/ya.make | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 3054d5b9cc7..894085e8b91 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -84,6 +84,7 @@ SRCS( MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp MergeTree/MergeTreeWhereOptimizer.cpp MergeTree/MergeTreeWriteAheadLog.cpp + MergeTree/MergeType.cpp MergeTree/registerStorageMergeTree.cpp MergeTree/ReplicatedMergeTreeAddress.cpp MergeTree/ReplicatedMergeTreeAltersSequence.cpp From e42d0f60da6c228ac7b896fd3504dc7e500d68b4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Sep 2020 14:27:27 +0300 Subject: [PATCH 049/298] Fix several bugs --- src/Storages/MergeTree/MergeList.h | 5 +++-- src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp | 8 +++++--- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 5af71b88341..d0b01913058 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -142,7 +142,8 @@ public: { std::lock_guard lock{mutex}; auto entry = std::make_unique(*this, merges.emplace(merges.end(), std::forward(args)...)); - merges_with_ttl_counter += (*entry)->merge_type == MergeType::TTL_DELETE; + if (isTTLMergeType((*entry)->merge_type)) + ++merges_with_ttl_counter; return entry; } @@ -179,7 +180,7 @@ inline MergeListEntry::~MergeListEntry() std::lock_guard lock{list.mutex}; if (it->merge_type == MergeType::TTL_DELETE) - list.merges_with_ttl_counter--; + --list.merges_with_ttl_counter; list.merges.erase(it); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index d95ae6b729d..b79717fc54c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -154,15 +154,17 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) if (format_version >= 4) { in >> "\ndeduplicate: " >> deduplicate; + + /// Trying to be more backward compatible in >> "\n"; - if (in.eof()) - trailing_newline_found = true; - else if (checkString("merge_type: ", in)) + if (checkString("merge_type: ", in)) { UInt64 value; in >> value; merge_type = checkAndGetMergeType(value); } + else + trailing_newline_found = true; } } else if (type_str == "drop" || type_str == "detach") diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 13438821ba6..ad845537139 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3620,7 +3620,7 @@ bool StorageReplicatedMergeTree::optimize( if (!partition) { selected = merger_mutator.selectPartsToMerge( - future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, true, &disable_reason); + future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, false, &disable_reason); } else { From 68913eab62ea2ac13a2021ce9548b9b3e987f922 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 4 Sep 2020 15:48:55 +0300 Subject: [PATCH 050/298] better reading from compact parts with differents codecs --- .../CachedCompressedReadBuffer.cpp | 19 +---- src/Compression/CachedCompressedReadBuffer.h | 6 +- src/Compression/CompressedReadBufferBase.cpp | 46 +++++++--- src/Compression/CompressedReadBufferBase.h | 5 +- .../CompressedReadBufferFromFile.cpp | 12 +-- .../CompressedReadBufferFromFile.h | 6 +- .../MergeTree/MergeTreeReaderCompact.cpp | 84 +++++++++---------- .../MergeTree/MergeTreeReaderCompact.h | 20 +---- .../01375_compact_parts_codecs.reference | 3 + .../01375_compact_parts_codecs.sql | 6 ++ 10 files changed, 104 insertions(+), 103 deletions(-) diff --git a/src/Compression/CachedCompressedReadBuffer.cpp b/src/Compression/CachedCompressedReadBuffer.cpp index 218925f8eae..3fb45ab0948 100644 --- a/src/Compression/CachedCompressedReadBuffer.cpp +++ b/src/Compression/CachedCompressedReadBuffer.cpp @@ -12,7 +12,6 @@ namespace DB namespace ErrorCodes { extern const int SEEK_POSITION_OUT_OF_BOUND; - extern const int LOGICAL_ERROR; } @@ -20,9 +19,8 @@ void CachedCompressedReadBuffer::initInput() { if (!file_in) { - file_in_holder = file_in_creator(); - file_in = file_in_holder.get(); - compressed_in = file_in; + file_in = file_in_creator(); + compressed_in = file_in.get(); if (profile_callback) file_in->setProfileCallback(profile_callback, clock_type); @@ -74,19 +72,10 @@ bool CachedCompressedReadBuffer::nextImpl() } CachedCompressedReadBuffer::CachedCompressedReadBuffer( - const std::string & path_, ReadBufferFromFileBase * file_in_, UncompressedCache * cache_) - : ReadBuffer(nullptr, 0), file_in(file_in_), cache(cache_), path(path_), file_pos(0) -{ - if (file_in == nullptr) - throw Exception("Neither file_in nor file_in_creator is initialized in CachedCompressedReadBuffer", ErrorCodes::LOGICAL_ERROR); - - compressed_in = file_in; -} - -CachedCompressedReadBuffer::CachedCompressedReadBuffer( - const std::string & path_, std::function()> file_in_creator_, UncompressedCache * cache_) + const std::string & path_, std::function()> file_in_creator_, UncompressedCache * cache_, bool allow_different_codecs_) : ReadBuffer(nullptr, 0), file_in_creator(std::move(file_in_creator_)), cache(cache_), path(path_), file_pos(0) { + allow_different_codecs = allow_different_codecs_; } void CachedCompressedReadBuffer::seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block) diff --git a/src/Compression/CachedCompressedReadBuffer.h b/src/Compression/CachedCompressedReadBuffer.h index 5debdc006cc..c2338f6f841 100644 --- a/src/Compression/CachedCompressedReadBuffer.h +++ b/src/Compression/CachedCompressedReadBuffer.h @@ -21,9 +21,8 @@ class CachedCompressedReadBuffer : public CompressedReadBufferBase, public ReadB { private: std::function()> file_in_creator; - std::unique_ptr file_in_holder; - ReadBufferFromFileBase * file_in = nullptr; UncompressedCache * cache; + std::unique_ptr file_in; const std::string path; size_t file_pos; @@ -39,8 +38,7 @@ private: clockid_t clock_type {}; public: - CachedCompressedReadBuffer(const std::string & path_, ReadBufferFromFileBase * file_in_, UncompressedCache * cache_); - CachedCompressedReadBuffer(const std::string & path, std::function()> file_in_creator, UncompressedCache * cache_); + CachedCompressedReadBuffer(const std::string & path, std::function()> file_in_creator, UncompressedCache * cache_, bool allow_different_codecs_ = false); void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index a05b5cd7f64..be2f697e1b3 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -105,13 +105,24 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, uint8_t method = ICompressionCodec::readMethod(own_compressed_buffer.data()); if (!codec) + { codec = CompressionCodecFactory::instance().get(method); + } else if (method != codec->getMethodByte()) - throw Exception("Data compressed with different methods, given method byte 0x" - + getHexUIntLowercase(method) - + ", previous method byte 0x" - + getHexUIntLowercase(codec->getMethodByte()), - ErrorCodes::CANNOT_DECOMPRESS); + { + if (allow_different_codecs) + { + codec = CompressionCodecFactory::instance().get(method); + } + else + { + throw Exception("Data compressed with different methods, given method byte 0x" + + getHexUIntLowercase(method) + + ", previous method byte 0x" + + getHexUIntLowercase(codec->getMethodByte()), + ErrorCodes::CANNOT_DECOMPRESS); + } + } size_compressed_without_checksum = ICompressionCodec::readCompressedBlockSize(own_compressed_buffer.data()); size_decompressed = ICompressionCodec::readDecompressedBlockSize(own_compressed_buffer.data()); @@ -163,21 +174,32 @@ void CompressedReadBufferBase::decompress(char * to, size_t size_decompressed, s uint8_t method = ICompressionCodec::readMethod(compressed_buffer); if (!codec) + { codec = CompressionCodecFactory::instance().get(method); + } else if (codec->getMethodByte() != method) - throw Exception("Data compressed with different methods, given method byte " - + getHexUIntLowercase(method) - + ", previous method byte " - + getHexUIntLowercase(codec->getMethodByte()), - ErrorCodes::CANNOT_DECOMPRESS); + { + if (allow_different_codecs) + { + codec = CompressionCodecFactory::instance().get(method); + } + else + { + throw Exception("Data compressed with different methods, given method byte " + + getHexUIntLowercase(method) + + ", previous method byte " + + getHexUIntLowercase(codec->getMethodByte()), + ErrorCodes::CANNOT_DECOMPRESS); + } + } codec->decompress(compressed_buffer, size_compressed_without_checksum, to); } /// 'compressed_in' could be initialized lazily, but before first call of 'readCompressedData'. -CompressedReadBufferBase::CompressedReadBufferBase(ReadBuffer * in) - : compressed_in(in), own_compressed_buffer(0) +CompressedReadBufferBase::CompressedReadBufferBase(ReadBuffer * in, bool allow_different_codecs_) + : compressed_in(in), own_compressed_buffer(0), allow_different_codecs(allow_different_codecs_) { } diff --git a/src/Compression/CompressedReadBufferBase.h b/src/Compression/CompressedReadBufferBase.h index f44140dcd04..71dc5274d5b 100644 --- a/src/Compression/CompressedReadBufferBase.h +++ b/src/Compression/CompressedReadBufferBase.h @@ -26,6 +26,9 @@ protected: /// Don't checksum on decompressing. bool disable_checksum = false; + /// Allow reading data, compressed by different codecs from one file. + bool allow_different_codecs; + /// Read compressed data into compressed_buffer. Get size of decompressed data from block header. Checksum if need. /// Returns number of compressed bytes read. size_t readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum); @@ -34,7 +37,7 @@ protected: public: /// 'compressed_in' could be initialized lazily, but before first call of 'readCompressedData'. - CompressedReadBufferBase(ReadBuffer * in = nullptr); + CompressedReadBufferBase(ReadBuffer * in = nullptr, bool allow_different_codecs_ = false); ~CompressedReadBufferBase(); /** Disable checksums. diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index 852194bb81e..f3fa2d6bc10 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -36,26 +36,22 @@ bool CompressedReadBufferFromFile::nextImpl() return true; } -CompressedReadBufferFromFile::CompressedReadBufferFromFile(ReadBufferFromFileBase & file_in_) - : BufferWithOwnMemory(0), file_in(file_in_) -{ - compressed_in = &file_in; -} - -CompressedReadBufferFromFile::CompressedReadBufferFromFile(std::unique_ptr buf) +CompressedReadBufferFromFile::CompressedReadBufferFromFile(std::unique_ptr buf, bool allow_different_codecs_) : BufferWithOwnMemory(0), p_file_in(std::move(buf)), file_in(*p_file_in) { compressed_in = &file_in; + allow_different_codecs = allow_different_codecs_; } CompressedReadBufferFromFile::CompressedReadBufferFromFile( - const std::string & path, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, size_t buf_size) + const std::string & path, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, size_t buf_size, bool allow_different_codecs_) : BufferWithOwnMemory(0) , p_file_in(createReadBufferFromFileBase(path, estimated_size, aio_threshold, mmap_threshold, buf_size)) , file_in(*p_file_in) { compressed_in = &file_in; + allow_different_codecs = allow_different_codecs_; } diff --git a/src/Compression/CompressedReadBufferFromFile.h b/src/Compression/CompressedReadBufferFromFile.h index 1de28062e41..166b2595ef9 100644 --- a/src/Compression/CompressedReadBufferFromFile.h +++ b/src/Compression/CompressedReadBufferFromFile.h @@ -28,11 +28,11 @@ private: bool nextImpl() override; public: - CompressedReadBufferFromFile(ReadBufferFromFileBase & buf); - CompressedReadBufferFromFile(std::unique_ptr buf); + CompressedReadBufferFromFile(std::unique_ptr buf, bool allow_different_codecs_ = false); CompressedReadBufferFromFile( - const std::string & path, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + const std::string & path, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, bool allow_different_codecs_ = false); void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 93dc8372565..87b3f0a4329 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -73,31 +73,41 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( buffer_size = settings.max_read_buffer_size; const String full_data_path = data_part->getFullRelativePath() + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION; - file_in = data_part->volume->getDisk()->readFile( - full_data_path, buffer_size, 0, - settings.min_bytes_to_use_direct_io, - settings.min_bytes_to_use_mmap_io); - - auto full_path = fullPath(data_part->volume->getDisk(), full_data_path); - for (const auto & column : columns) + if (uncompressed_cache) { - std::unique_ptr cached_buffer; - std::unique_ptr non_cached_buffer; - if (uncompressed_cache) - { - cached_buffer = std::make_unique(full_path, file_in.get(), uncompressed_cache); - if (profile_callback_) - cached_buffer->setProfileCallback(profile_callback_, clock_type_); - } - else - { - non_cached_buffer = std::make_unique(*file_in); - if (profile_callback_) - non_cached_buffer->setProfileCallback(profile_callback_, clock_type_); - } + auto buffer = std::make_unique( + fullPath(data_part->volume->getDisk(), full_data_path), + [this, full_data_path, buffer_size]() + { + return data_part->volume->getDisk()->readFile( + full_data_path, + buffer_size, + 0, + settings.min_bytes_to_use_direct_io, + settings.min_bytes_to_use_mmap_io); + }, + uncompressed_cache, + /* allow_different_codecs = */ true); - auto column_from_part = getColumnFromPart(column); - column_streams[column_from_part.name] = ColumnStream{std::move(cached_buffer), std::move(non_cached_buffer)}; + if (profile_callback_) + buffer->setProfileCallback(profile_callback_, clock_type_); + + cached_buffer = std::move(buffer); + data_buffer = cached_buffer.get(); + } + else + { + auto buffer = + std::make_unique( + data_part->volume->getDisk()->readFile( + full_data_path, buffer_size, 0, settings.min_bytes_to_use_direct_io, settings.min_bytes_to_use_mmap_io), + /* allow_different_codecs = */ true); + + if (profile_callback_) + buffer->setProfileCallback(profile_callback_, clock_type_); + + non_cached_buffer = std::move(buffer); + data_buffer = non_cached_buffer.get(); } } catch (...) @@ -192,16 +202,15 @@ void MergeTreeReaderCompact::readData( const String & name, IColumn & column, const IDataType & type, size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets) { - auto & stream = column_streams[name]; if (!isContinuousReading(from_mark, column_position)) - seekToMark(stream, from_mark, column_position); + seekToMark(from_mark, column_position); auto buffer_getter = [&](const IDataType::SubstreamPath & substream_path) -> ReadBuffer * { if (only_offsets && (substream_path.size() != 1 || substream_path[0].type != IDataType::Substream::ArraySizes)) return nullptr; - return stream.data_buffer; + return data_buffer; }; IDataType::DeserializeBinaryBulkSettings deserialize_settings; @@ -221,15 +230,15 @@ void MergeTreeReaderCompact::readData( } -void MergeTreeReaderCompact::seekToMark(ColumnStream & stream, size_t row_index, size_t column_index) +void MergeTreeReaderCompact::seekToMark(size_t row_index, size_t column_index) { MarkInCompressedFile mark = marks_loader.getMark(row_index, column_index); try { - if (stream.cached_buffer) - stream.cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); - if (stream.non_cached_buffer) - stream.non_cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); + if (cached_buffer) + cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); + if (non_cached_buffer) + non_cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block); } catch (Exception & e) { @@ -241,7 +250,6 @@ void MergeTreeReaderCompact::seekToMark(ColumnStream & stream, size_t row_index, } } - bool MergeTreeReaderCompact::isContinuousReading(size_t mark, size_t column_position) { if (!last_read_granule) @@ -251,18 +259,6 @@ bool MergeTreeReaderCompact::isContinuousReading(size_t mark, size_t column_posi || (mark == last_mark + 1 && column_position == 0 && last_column == data_part->getColumns().size() - 1); } -MergeTreeReaderCompact::ColumnStream::ColumnStream( - std::unique_ptr cached_buffer_, - std::unique_ptr non_cached_buffer_) - : cached_buffer(std::move(cached_buffer_)) - , non_cached_buffer(std::move(non_cached_buffer_)) -{ - if (cached_buffer) - data_buffer = cached_buffer.get(); - else - data_buffer = non_cached_buffer.get(); -} - namespace { diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index ec765adbf0e..9ef88716579 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -39,21 +39,9 @@ public: private: bool isContinuousReading(size_t mark, size_t column_position); - std::unique_ptr file_in; - - struct ColumnStream - { - std::unique_ptr cached_buffer; - std::unique_ptr non_cached_buffer; - ReadBuffer * data_buffer; - - ColumnStream() = default; - ColumnStream( - std::unique_ptr cached_buffer_, - std::unique_ptr non_cached_buffer_); - }; - - std::unordered_map column_streams; + ReadBuffer * data_buffer; + std::unique_ptr cached_buffer; + std::unique_ptr non_cached_buffer; MergeTreeMarksLoader marks_loader; @@ -66,7 +54,7 @@ private: size_t next_mark = 0; std::optional> last_read_granule; - void seekToMark(ColumnStream & stream, size_t row_index, size_t column_index); + void seekToMark(size_t row_index, size_t column_index); void readData(const String & name, IColumn & column, const IDataType & type, size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets = false); diff --git a/tests/queries/0_stateless/01375_compact_parts_codecs.reference b/tests/queries/0_stateless/01375_compact_parts_codecs.reference index 982c45a26e3..24b3e22d9a6 100644 --- a/tests/queries/0_stateless/01375_compact_parts_codecs.reference +++ b/tests/queries/0_stateless/01375_compact_parts_codecs.reference @@ -1,3 +1,6 @@ 12000 11890 +499500 499500 999 11965 11890 +499500 499500 999 5858 11890 +499500 499500 999 diff --git a/tests/queries/0_stateless/01375_compact_parts_codecs.sql b/tests/queries/0_stateless/01375_compact_parts_codecs.sql index 467745c6fa2..4b285f5bcc1 100644 --- a/tests/queries/0_stateless/01375_compact_parts_codecs.sql +++ b/tests/queries/0_stateless/01375_compact_parts_codecs.sql @@ -8,6 +8,8 @@ SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) FROM system.parts WHERE table = 'codecs' AND database = currentDatabase(); +SELECT sum(id), sum(val), max(s) FROM codecs; + DROP TABLE codecs; CREATE TABLE codecs (id UInt32 CODEC(NONE), val UInt32 CODEC(NONE), s String CODEC(NONE)) @@ -18,6 +20,8 @@ SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) FROM system.parts WHERE table = 'codecs' AND database = currentDatabase(); +SELECT sum(id), sum(val), max(s) FROM codecs; + DROP TABLE codecs; CREATE TABLE codecs (id UInt32, val UInt32 CODEC(Delta, ZSTD), s String CODEC(ZSTD)) @@ -28,4 +32,6 @@ SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) FROM system.parts WHERE table = 'codecs' AND database = currentDatabase(); +SELECT sum(id), sum(val), max(s) FROM codecs; + DROP TABLE codecs; From ea7168580bfa630a2b985a1aee15161d1d1a56c3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Sep 2020 16:55:07 +0300 Subject: [PATCH 051/298] Fixes --- src/Storages/MergeTree/MergeList.h | 2 +- src/Storages/MergeTree/MergeTreeSettings.h | 1 + src/Storages/MergeTree/MergeType.cpp | 2 ++ src/Storages/StorageReplicatedMergeTree.cpp | 10 ++++++++++ 4 files changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index d0b01913058..4d080ff3569 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -179,7 +179,7 @@ inline MergeListEntry::~MergeListEntry() { std::lock_guard lock{list.mutex}; - if (it->merge_type == MergeType::TTL_DELETE) + if (isTTLMergeType(it->merge_type)) --list.merges_with_ttl_counter; list.merges.erase(it); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 692e07d9884..c4d8e7bd11f 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -54,6 +54,7 @@ struct Settings; M(UInt64, min_replicated_logs_to_keep, 10, "Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.", 0) \ M(Seconds, prefer_fetch_merged_part_time_threshold, 3600, "If time passed after replication log entry creation exceeds this threshold and sum size of parts is greater than \"prefer_fetch_merged_part_size_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ M(UInt64, prefer_fetch_merged_part_size_threshold, 10ULL * 1024 * 1024 * 1024, "If sum size of parts exceeds this threshold and time passed after replication log entry creation is greater than \"prefer_fetch_merged_part_time_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ + M(Seconds, try_fetch_recompressed_part_timeout, 7200, "Recompression works slow in most cases, so we don't start merge with recompression until this timeout and trying to fetch recompressed part from replica which assigned this merge with recompression.", 0) \ M(Bool, always_fetch_merged_part, 0, "If true, replica never merge parts and always download merged parts from other replicas.", 0) \ M(UInt64, max_suspicious_broken_parts, 10, "Max broken parts, if more - deny automatic deletion.", 0) \ M(UInt64, max_files_to_modify_in_alter_columns, 75, "Not apply ALTER if number of files for modification(deletion, addition) more than this.", 0) \ diff --git a/src/Storages/MergeTree/MergeType.cpp b/src/Storages/MergeTree/MergeType.cpp index 7f1495b14b3..e622eb33e31 100644 --- a/src/Storages/MergeTree/MergeType.cpp +++ b/src/Storages/MergeTree/MergeType.cpp @@ -15,6 +15,8 @@ MergeType checkAndGetMergeType(UInt64 merge_type) return MergeType::REGULAR; else if (merge_type == static_cast(MergeType::TTL_DELETE)) return MergeType::TTL_DELETE; + else if (merge_type == static_cast(MergeType::TTL_RECOMPRESS)) + return MergeType::TTL_RECOMPRESS; throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeType {}", static_cast(merge_type)); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index be03d541ac8..24ffa2bc410 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1308,6 +1308,16 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) return false; } + if (entry.merge_type == MergeType::TTL_RECOMPRESS && + (time(nullptr) - entry.create_time) <= storage_settings_ptr->try_fetch_recompressed_part_timeout.totalSeconds() && + entry.source_replica != replica_name) + { + LOG_INFO(log, "Will try to fetch part {} until '{}' because this part assigned to recompression merge. " + "Source replica {} will try to merge this part first", entry.new_part_name, + LocalDateTime(entry.create_time + storage_settings_ptr->try_fetch_recompressed_part_timeout.totalSeconds()), entry.source_replica); + return false; + } + DataPartsVector parts; bool have_all_parts = true; for (const String & name : entry.source_parts) From fecb2f13115a1776e5fb9b1cd0f1c3f91e2c5ca5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Sep 2020 17:08:43 +0300 Subject: [PATCH 052/298] Less copypaste --- src/DataStreams/TTLBlockInputStream.cpp | 34 +++++-------------- src/DataStreams/TTLBlockInputStream.h | 3 ++ src/Storages/MergeTree/MergeTreeData.cpp | 2 -- .../MergeTree/MergeTreeDataMergerMutator.cpp | 14 ++------ .../MergeTree/MergeTreeDataMergerMutator.h | 4 ++- src/Storages/MergeTree/MergeTreeSettings.h | 3 +- 6 files changed, 19 insertions(+), 41 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index e1586286678..85d9c7fead2 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -370,13 +370,12 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) block.erase(column); } -void TTLBlockInputStream::updateMovesTTL(Block & block) +void TTLBlockInputStream::updateTTLWithDescriptions(Block & block, const TTLDescriptions & descriptions, TTLInfoMap & ttl_info_map) { std::vector columns_to_remove; - for (const auto & ttl_entry : metadata_snapshot->getMoveTTLs()) + for (const auto & ttl_entry : descriptions) { - auto & new_ttl_info = new_ttl_infos.moves_ttl[ttl_entry.result_column]; - + auto & new_ttl_info = ttl_info_map[ttl_entry.result_column]; if (!block.has(ttl_entry.result_column)) { columns_to_remove.push_back(ttl_entry.result_column); @@ -396,31 +395,14 @@ void TTLBlockInputStream::updateMovesTTL(Block & block) block.erase(column); } +void TTLBlockInputStream::updateMovesTTL(Block & block) +{ + updateTTLWithDescriptions(block, metadata_snapshot->getMoveTTLs(), new_ttl_infos.moves_ttl); +} void TTLBlockInputStream::updateRecompressionTTL(Block & block) { - std::vector columns_to_remove; - for (const auto & ttl_entry : metadata_snapshot->getRecompressionTTLs()) - { - auto & new_ttl_info = new_ttl_infos.recompression_ttl[ttl_entry.result_column]; - - if (!block.has(ttl_entry.result_column)) - { - columns_to_remove.push_back(ttl_entry.result_column); - ttl_entry.expression->execute(block); - } - - const IColumn * ttl_column = block.getByName(ttl_entry.result_column).column.get(); - - for (size_t i = 0; i < block.rows(); ++i) - { - UInt32 cur_ttl = getTimestampByIndex(ttl_column, i); - new_ttl_info.update(cur_ttl); - } - } - - for (const String & column : columns_to_remove) - block.erase(column); + updateTTLWithDescriptions(block, metadata_snapshot->getRecompressionTTLs(), new_ttl_infos.recompression_ttl); } UInt32 TTLBlockInputStream::getTimestampByIndex(const IColumn * column, size_t ind) diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/DataStreams/TTLBlockInputStream.h index 18670021ec9..774b413ed1b 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/DataStreams/TTLBlockInputStream.h @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -75,6 +76,8 @@ private: /// Finalize agg_result into result_columns void finalizeAggregates(MutableColumns & result_columns); + void updateTTLWithDescriptions(Block & block, const TTLDescriptions & descriptions, TTLInfoMap & ttl_info_map); + /// Updates TTL for moves void updateMovesTTL(Block & block); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e876ecb8b34..8ba00f29d9d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3039,8 +3039,6 @@ CompressionCodecPtr MergeTreeData::getCompressionCodecForPart(size_t part_size_c auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); - LOG_DEBUG(log, "RECOMPRESSION ENTRIES SIZE {}", recompression_ttl_entries.size()); - LOG_DEBUG(log, "TTL INFOS SIZE {}", ttl_infos.recompression_ttl.size()); auto best_ttl_entry = selectTTLEntryForTTLInfos(recompression_ttl_entries, ttl_infos.recompression_ttl, current_time, false); if (best_ttl_entry) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 9c104d042d9..ddad80e1b76 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -225,7 +225,6 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( return false; } - //LOG_DEBUG(log, "SELECTING PARTS TO MERGE"); time_t current_time = std::time(nullptr); IMergeSelector::PartsRanges parts_ranges; @@ -296,10 +295,8 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( if (metadata_snapshot->hasAnyTTL() && merge_with_ttl_allowed && !ttl_merges_blocker.isCancelled()) { - - //LOG_DEBUG(log, "SELECTING WITH TTL"); TTLDeleteMergeSelector delete_ttl_selector( - next_ttl_merge_times_by_partition, + next_delete_ttl_merge_times_by_partition, current_time, data_settings->merge_with_ttl_timeout, data_settings->ttl_only_drop_parts); @@ -309,20 +306,15 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( future_part.merge_type = MergeType::TTL_DELETE; else if (metadata_snapshot->hasAnyRecompressionTTL()) { - - //LOG_DEBUG(log, "SELECTING WITH RECOMPRESSION"); TTLRecompressMergeSelector recompress_ttl_selector( - next_ttl_merge_times_by_partition, + next_recompress_ttl_merge_times_by_partition, current_time, - data_settings->merge_with_ttl_timeout, + data_settings->merge_with_recompression_ttl_timeout, metadata_snapshot->getRecompressionTTLs()); parts_to_merge = recompress_ttl_selector.select(parts_ranges, max_total_size_to_merge); if (!parts_to_merge.empty()) - { - //LOG_DEBUG(log, "SELECTED PARTS: {}", parts_to_merge.size()); future_part.merge_type = MergeType::TTL_RECOMPRESS; - } } } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 6ecfef728d2..5f6b9246d68 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -251,8 +251,10 @@ private: time_t disk_space_warning_time = 0; /// Stores the next TTL merge due time for each partition (used only by TTLMergeSelector) - ITTLMergeSelector::PartitionIdToTTLs next_ttl_merge_times_by_partition; + ITTLMergeSelector::PartitionIdToTTLs next_delete_ttl_merge_times_by_partition; + /// Stores the next TTL merge due time for each partition (used only by TTLMergeSelector) + ITTLMergeSelector::PartitionIdToTTLs next_recompress_ttl_merge_times_by_partition; /// Performing TTL merges independently for each partition guarantees that /// there is only a limited number of TTL merges and no partition stores data, that is too stale }; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c4d8e7bd11f..5e1d85e54d5 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -86,7 +86,8 @@ struct Settings; M(UInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).", 0) \ M(UInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).", 0) \ M(UInt64, min_index_granularity_bytes, 1024, "Minimum amount of bytes in single granule.", 1024) \ - M(Int64, merge_with_ttl_timeout, 0, "Minimal time in seconds, when merge with TTL can be repeated.", 0) \ + M(Int64, merge_with_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with delete TTL can be repeated.", 0) \ + M(Int64, merge_with_recompression_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with recompression TTL can be repeated.", 0) \ M(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \ M(Bool, write_final_mark, 1, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)", 0) \ M(Bool, enable_mixed_granularity_parts, 1, "Enable parts with adaptive and non adaptive granularity", 0) \ From 21de2f54f69d45007cd40fb42d28fb96193f14d5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Sep 2020 17:13:57 +0300 Subject: [PATCH 053/298] Fix tests --- tests/integration/test_recompression_ttl/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_recompression_ttl/test.py b/tests/integration/test_recompression_ttl/test.py index 5ea0f91d495..a581dd24e43 100644 --- a/tests/integration/test_recompression_ttl/test.py +++ b/tests/integration/test_recompression_ttl/test.py @@ -52,7 +52,7 @@ def wait_part_and_get_compression_codec(node, table, part_name, retries=40): def test_recompression_simple(started_cluster): - node1.query("CREATE TABLE table_for_recompression (d DateTime, key UInt64, data String) ENGINE MergeTree() ORDER BY tuple() TTL d + INTERVAL 10 SECOND RECOMPRESS CODEC(ZSTD(10)) SETTINGS merge_with_ttl_timeout = 0") + node1.query("CREATE TABLE table_for_recompression (d DateTime, key UInt64, data String) ENGINE MergeTree() ORDER BY tuple() TTL d + INTERVAL 10 SECOND RECOMPRESS CODEC(ZSTD(10)) SETTINGS merge_with_recompression_ttl_timeout = 0") node1.query("INSERT INTO table_for_recompression VALUES (now(), 1, '1')") assert node1.query("SELECT default_compression_codec FROM system.parts where name = 'all_1_1_0'") == "LZ4\n" @@ -75,7 +75,7 @@ def test_recompression_multiple_ttls(started_cluster): node2.query("CREATE TABLE table_for_recompression (d DateTime, key UInt64, data String) ENGINE MergeTree() ORDER BY tuple() \ TTL d + INTERVAL 5 SECOND RECOMPRESS CODEC(ZSTD(10)), \ d + INTERVAL 10 SECOND RECOMPRESS CODEC(ZSTD(11)), \ - d + INTERVAL 15 SECOND RECOMPRESS CODEC(ZSTD(12)) SETTINGS merge_with_ttl_timeout = 0") + d + INTERVAL 15 SECOND RECOMPRESS CODEC(ZSTD(12)) SETTINGS merge_with_recompression_ttl_timeout = 0") node2.query("INSERT INTO table_for_recompression VALUES (now(), 1, '1')") @@ -111,7 +111,7 @@ def test_recompression_replicated(started_cluster): for i, node in enumerate([node1, node2]): node.query("CREATE TABLE recompression_replicated (d DateTime, key UInt64, data String) \ ENGINE ReplicatedMergeTree('/test/rr', '{}') ORDER BY tuple() \ - TTL d + INTERVAL 10 SECOND RECOMPRESS CODEC(ZSTD(13)) SETTINGS merge_with_ttl_timeout = 0".format(i + 1)) + TTL d + INTERVAL 10 SECOND RECOMPRESS CODEC(ZSTD(13)) SETTINGS merge_with_recompression_ttl_timeout = 0".format(i + 1)) node1.query("INSERT INTO recompression_replicated VALUES (now(), 1, '1')") node2.query("SYSTEM SYNC REPLICA recompression_replicated", timeout=5) From f48d654d3515b2e8d9160c02ac038dbe08132fbd Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Sep 2020 17:15:34 +0300 Subject: [PATCH 054/298] Less garbage --- src/Interpreters/MutationsInterpreter.cpp | 8 -------- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 1 - 2 files changed, 9 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index ef95b25eb98..9d35b339d94 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -533,16 +533,8 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) /// Special step to recalculate affected indices and TTL expressions. stages.emplace_back(context); for (const auto & column : unchanged_columns) - { - //std::cerr << "ADDING UNCHANGED COLUMN TO STAGE:" << column << std::endl; stages.back().column_to_updated.emplace( column, std::make_shared(column)); - //std::cerr << "OUTPUT COLUMNS:" << stages.back().output_columns.size() << std::endl; - //for (const auto & col : stages.back().output_columns) - //{ - // std::cerr << "OUTPUT COLUMN:" << col << std::endl; - //} - } } } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index ddad80e1b76..977c6faace7 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -706,7 +706,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor /// the order is reverse. This annoys TSan even though one lock is locked in shared mode and thus /// deadlock is impossible. auto compression_codec = data.getCompressionCodecForPart(merge_entry->total_size_bytes_compressed, new_data_part->ttl_infos, time_of_merge); - LOG_DEBUG(log, "CHOOSEN CODEC {} FOR PART {}", queryToString(compression_codec->getCodecDesc()), new_data_part->name); /// TODO: Should it go through IDisk interface? String rows_sources_file_path; From c933f72adb8716316df8b37200fa0dd6f9e1a2e1 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 4 Sep 2020 17:17:27 +0300 Subject: [PATCH 055/298] Disable S3 requests processing during context shutdown to speed up termination process. --- src/Disks/IDisk.h | 3 +++ src/Disks/S3/DiskS3.cpp | 5 +++++ src/Disks/S3/DiskS3.h | 2 ++ src/Interpreters/Context.cpp | 7 +++++++ 4 files changed, 17 insertions(+) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 53dc4999dc4..17de6db3487 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -183,6 +183,9 @@ public: /// Return disk type - "local", "s3", etc. virtual const String getType() const = 0; + /// Invoked when Global Context is shutdown. + virtual void shutdown() { } + private: /// Returns executor to perform asynchronous operations. Executor & getExecutor() { return *executor; } diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 5aa57518c83..cff7cc3429a 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -746,4 +746,9 @@ void DiskS3::setReadOnly(const String & path) Poco::File(metadata_path + path).setReadOnly(true); } +void DiskS3::shutdown() +{ + client->DisableRequestProcessing(); +} + } diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 34f00af6439..db352feb063 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -102,6 +102,8 @@ public: const String getType() const override { return "s3"; } + void shutdown() override; + private: bool tryReserve(UInt64 bytes); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 1318f0331c4..3f65fdb8de0 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1973,6 +1973,13 @@ void Context::reloadConfig() const void Context::shutdown() { + auto disks = getDisksMap(); + for (auto & [disk_name, disk] : disks) + { + LOG_INFO(shared->log, "Shutdown disk {}", disk_name); + disk->shutdown(); + } + shared->shutdown(); } From 79ef72178eceb8e9ae85999226b55a805f8c46e5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Sep 2020 17:18:49 +0300 Subject: [PATCH 056/298] Less garbage --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 977c6faace7..b88d80dc41f 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1135,20 +1135,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor bool need_remove_expired_values = false; if (in && shouldExecuteTTL(metadata_snapshot, in->getHeader().getNamesAndTypesList().getNames(), commands_for_part)) - { - //std::cerr << "GOING TO MATERIALIZE TTL\n"; need_remove_expired_values = true; - } - else - { - //std::cerr << "NOT GOING TO MATERIALIZE TTL\n"; - //std::cerr << "IN IS NULL:" << (in == nullptr) << std::endl; - } /// All columns from part are changed and may be some more that were missing before in part if (!isWidePart(source_part) || (interpreter && interpreter->isAffectingAllColumns())) { - //std::cerr << "MUTATING ALL PART COLUMNS\n"; /// Note: this is done before creating input streams, because otherwise data.data_parts_mutex /// (which is locked in data.getTotalActiveSizeInBytes()) /// (which is locked in shared mode when input streams are created) and when inserting new data @@ -1179,9 +1170,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor NameToNameVector files_to_rename = collectFilesForRenames(source_part, for_file_renames, mrk_extension); if (need_remove_expired_values) - { files_to_skip.insert("ttl.txt"); - } /// Create hardlinks for unchanged files for (auto it = disk->iterateDirectory(source_part->getFullRelativePath()); it->isValid(); it->next()) { From 2d1042614eb74a33d97df90ba0f6a198e58ec5e9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Sep 2020 17:19:41 +0300 Subject: [PATCH 057/298] Less comments --- src/Storages/TTLDescription.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 8a212074027..7f55badf819 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -273,7 +273,6 @@ TTLDescription TTLDescription::getTTLFromAST( } else if (ttl_element->mode == TTLMode::RECOMPRESS) { - //std::cerr << "GOT INTO RECOMPRESS\n"; result.recompression_codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST( ttl_element->recompression_codec, {}, !context.getSettingsRef().allow_suspicious_codecs); @@ -337,7 +336,6 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( } else if (ttl.mode == TTLMode::RECOMPRESS) { - //std::cerr << "GOT RECOMPRESSIOn TTL\n"; result.recompression_ttl.emplace_back(std::move(ttl)); } else From 37a2bd0bfdf189814e00dbf36dd37c135ee1d81a Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Sep 2020 17:23:32 +0300 Subject: [PATCH 058/298] less debug --- src/Storages/MergeTree/TTLMergeSelector.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/MergeTree/TTLMergeSelector.cpp b/src/Storages/MergeTree/TTLMergeSelector.cpp index fccd0f28f84..bb7c001eae1 100644 --- a/src/Storages/MergeTree/TTLMergeSelector.cpp +++ b/src/Storages/MergeTree/TTLMergeSelector.cpp @@ -118,8 +118,6 @@ bool TTLRecompressMergeSelector::isTTLAlreadySatisfied(const IMergeSelector::Par return ""; return queryToString(query); }; - //LOG_DEBUG(&Poco::Logger::get("RECOMPRESS SELECTOR"), "PART CODEC: {}", ast_to_str(part.compression_codec_desc)); - //LOG_DEBUG(&Poco::Logger::get("RECOMPRESS SELECTOR"), "ENTRY CODEC: {}", ast_to_str(ttl_description->recompression_codec)); return ast_to_str(ttl_description->recompression_codec) == ast_to_str(part.compression_codec_desc); } From c73bb980024a43bcfdbdd633d3fb00a9e3099258 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 4 Sep 2020 18:07:17 +0300 Subject: [PATCH 059/298] fix clang-tidy --- src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 22df2abecf3..ab064689f47 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -143,7 +143,7 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) void MergeTreeDataPartWriterCompact::writeColumnSingleGranule( const ColumnWithTypeAndName & column, const CompressedStreamPtr & stream, - size_t from_row, size_t number_of_rows) const + size_t from_row, size_t number_of_rows) { IDataType::SerializeBinaryBulkStatePtr state; IDataType::SerializeBinaryBulkSettings serialize_settings; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index a121554f4be..fecf5ce40e8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -28,7 +28,7 @@ protected: private: void writeBlock(const Block & block); - void addToChecksums(MergeTreeDataPartChecksums & checksumns); + void addToChecksums(MergeTreeDataPartChecksums & checksums); Block header; @@ -74,11 +74,11 @@ private: HashingWriteBuffer marks; /// Write single granule of one column (rows between 2 marks) - void writeColumnSingleGranule( + static void writeColumnSingleGranule( const ColumnWithTypeAndName & column, const CompressedStreamPtr & stream, size_t from_row, - size_t number_of_rows) const; + size_t number_of_rows); }; } From dbeeb7d141e8c9fa8a6c208ba827cd86d1ad75cb Mon Sep 17 00:00:00 2001 From: yulu86 Date: Sat, 5 Sep 2020 01:52:23 +0800 Subject: [PATCH 060/298] optimize chinese tutorial docs to make it more human readable --- docs/zh/getting-started/tutorial.md | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/docs/zh/getting-started/tutorial.md b/docs/zh/getting-started/tutorial.md index 38d5a586806..a17adb2d163 100644 --- a/docs/zh/getting-started/tutorial.md +++ b/docs/zh/getting-started/tutorial.md @@ -1,6 +1,4 @@ --- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd toc_priority: 12 toc_title: "\u6559\u7A0B" --- @@ -9,25 +7,25 @@ toc_title: "\u6559\u7A0B" ## 从本教程中可以期待什么? {#what-to-expect-from-this-tutorial} -通过本教程,您将学习如何设置一个简单的ClickHouse集群。 它会很小,但容错和可扩展。 然后,我们将使用其中一个示例数据集来填充数据并执行一些演示查询。 +通过本教程,您将学习如何设置一个简单的ClickHouse集群。 它会很小,但却是容错和可扩展的。 然后,我们将使用其中一个示例数据集来填充数据并执行一些演示查询。 ## 单节点设置 {#single-node-setup} -为了推迟分布式环境的复杂性,我们将首先在单个服务器或虚拟机上部署ClickHouse。 ClickHouse通常是从安装 [黛布](install.md#install-from-deb-packages) 或 [rpm](install.md#from-rpm-packages) 包,但也有 [替代办法](install.md#from-docker-image) 对于不支持它们的操作系统。 +为了推迟分布式环境的复杂性,我们将首先在单个服务器或虚拟机上部署ClickHouse。 ClickHouse通常是从[deb](install.md#install-from-deb-packages) 或 [rpm](install.md#from-rpm-packages) 包安装,但对于不支持它们的操作系统也有 [替代方法](install.md#from-docker-image) 。 -例如,您选择了 `deb` 包和执行: +例如,您选择了从 `deb` 包安装,执行: ``` bash {% include 'install/deb.sh' %} ``` -我们在安装的软件包中有什么: +在我们安装的软件中包含这些包: -- `clickhouse-client` 包包含 [ツ环板clientョツ嘉ッツ偲](../interfaces/cli.md) 应用程序,交互式ClickHouse控制台客户端。 -- `clickhouse-common` 包包含一个ClickHouse可执行文件。 -- `clickhouse-server` 包包含要作为服务器运行ClickHouse的配置文件。 +- `clickhouse-client` 包,包含 [clickhouse-client](../interfaces/cli.md) 应用程序,它是交互式ClickHouse控制台客户端。 +- `clickhouse-common` 包,包含一个ClickHouse可执行文件。 +- `clickhouse-server` 包,包含要作为服务端运行的ClickHouse配置文件。 -服务器配置文件位于 `/etc/clickhouse-server/`. 在进一步讨论之前,请注意 `` 元素in `config.xml`. Path确定数据存储的位置,因此应该位于磁盘容量较大的卷上;默认值为 `/var/lib/clickhouse/`. 如果你想调整配置,直接编辑并不方便 `config.xml` 文件,考虑到它可能会在未来的软件包更新中被重写。 复盖配置元素的推荐方法是创建 [在配置文件。d目录](../operations/configuration-files.md) 它作为 “patches” 要配置。xml +服务端配置文件位于 `/etc/clickhouse-server/`。在进一步讨论之前,请注意 `config.xml`文件中的`` 元素. Path决定了数据存储的位置,因此该位置应该位于磁盘容量较大的卷上;默认值为 `/var/lib/clickhouse/`。如果你想调整配置,直接编辑并不方便 `config.xml` 文件,考虑到它可能会在未来的软件包更新中被重写。 复盖配置元素的推荐方法是创建 [在配置文件。d目录](../operations/configuration-files.md) 它作为 “patches” 要配置。xml 你可能已经注意到了, `clickhouse-server` 安装包后不会自动启动。 它也不会在更新后自动重新启动。 您启动服务器的方式取决于您的init系统,通常情况下,它是: From 9d11d4dd449a5bf3507e090e32243657bec7d0d5 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 4 Sep 2020 21:45:22 +0300 Subject: [PATCH 061/298] Variable inline. --- src/Interpreters/Context.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 3f65fdb8de0..ce6b622d9ef 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1973,8 +1973,7 @@ void Context::reloadConfig() const void Context::shutdown() { - auto disks = getDisksMap(); - for (auto & [disk_name, disk] : disks) + for (auto & [disk_name, disk] : getDisksMap()) { LOG_INFO(shared->log, "Shutdown disk {}", disk_name); disk->shutdown(); From 439bcab6d66d28127feac8cffcde10b627bd5513 Mon Sep 17 00:00:00 2001 From: yulu86 Date: Sat, 5 Sep 2020 10:36:35 +0800 Subject: [PATCH 062/298] update chinese tutorial to make it more human readable --- docs/zh/getting-started/tutorial.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/zh/getting-started/tutorial.md b/docs/zh/getting-started/tutorial.md index a17adb2d163..43c7ed0ec59 100644 --- a/docs/zh/getting-started/tutorial.md +++ b/docs/zh/getting-started/tutorial.md @@ -25,9 +25,9 @@ toc_title: "\u6559\u7A0B" - `clickhouse-common` 包,包含一个ClickHouse可执行文件。 - `clickhouse-server` 包,包含要作为服务端运行的ClickHouse配置文件。 -服务端配置文件位于 `/etc/clickhouse-server/`。在进一步讨论之前,请注意 `config.xml`文件中的`` 元素. Path决定了数据存储的位置,因此该位置应该位于磁盘容量较大的卷上;默认值为 `/var/lib/clickhouse/`。如果你想调整配置,直接编辑并不方便 `config.xml` 文件,考虑到它可能会在未来的软件包更新中被重写。 复盖配置元素的推荐方法是创建 [在配置文件。d目录](../operations/configuration-files.md) 它作为 “patches” 要配置。xml +服务端配置文件位于 `/etc/clickhouse-server/`。 在进一步讨论之前,请注意 `config.xml`文件中的`` 元素. Path决定了数据存储的位置,因此该位置应该位于磁盘容量较大的卷上;默认值为 `/var/lib/clickhouse/`。 如果你想调整配置,考虑到它可能会在未来的软件包更新中被重写,直接编辑`config.xml` 文件并不方便。 推荐的方法是在[配置文件](../operations/configuration-files.md)目录创建文件,作为config.xml文件的“补丁”,用以复写配置元素。 -你可能已经注意到了, `clickhouse-server` 安装包后不会自动启动。 它也不会在更新后自动重新启动。 您启动服务器的方式取决于您的init系统,通常情况下,它是: +你可能已经注意到了, `clickhouse-server` 安装后不会自动启动。 它也不会在更新后自动重新启动。 您启动服务端的方式取决于您的初始系统,通常情况下是这样: ``` bash sudo service clickhouse-server start @@ -39,13 +39,13 @@ sudo service clickhouse-server start sudo /etc/init.d/clickhouse-server start ``` -服务器日志的默认位置是 `/var/log/clickhouse-server/`. 服务器已准备好处理客户端连接一旦它记录 `Ready for connections` 消息 +服务端日志的默认位置是 `/var/log/clickhouse-server/`。当服务端在日志中记录 `Ready for connections` 消息,即表示服务端已准备好处理客户端连接。 -一旦 `clickhouse-server` 正在运行我们可以利用 `clickhouse-client` 连接到服务器并运行一些测试查询,如 `SELECT "Hello, world!";`. +一旦 `clickhouse-server` 启动并运行,我们可以利用 `clickhouse-client` 连接到服务端,并运行一些测试查询,如 `SELECT "Hello, world!";`.
-Clickhouse-客户端的快速提示 +Clickhouse-client的快速提示 交互模式: From 7b95e56e8c902578f8fcebc5d9edeccce1eb35ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 03:09:40 +0300 Subject: [PATCH 063/298] Advancements --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 187 +++++++++++++----- 1 file changed, 133 insertions(+), 54 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 314bba0d5b4..d6264a63978 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -1,6 +1,11 @@ #include #include #include +#include +#include +#include + +#include #include @@ -16,7 +21,8 @@ #include -/** How to use: +static constexpr auto documentation = R"( +Prepare the database by executing the following queries: DROP DATABASE IF EXISTS git; CREATE DATABASE git; @@ -44,11 +50,11 @@ CREATE TABLE git.file_changes change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), new_file_path LowCardinality(String), old_file_path LowCardinality(String), - lines_added UInt16, - lines_deleted UInt16, - hunks_added UInt16, - hunks_removed UInt16, - hunks_changed UInt16, + lines_added UInt32, + lines_deleted UInt32, + hunks_added UInt32, + hunks_removed UInt32, + hunks_changed UInt32, commit_hash String, author_name LowCardinality(String), @@ -69,11 +75,11 @@ CREATE TABLE git.file_changes CREATE TABLE git.line_changes ( sign Int8, - line_number_old UInt16, - line_number_new UInt16, - hunk_num UInt16, - hunk_start_line_number_old UInt16, - hunk_start_line_number_new UInt16, + line_number_old UInt32, + line_number_new UInt32, + hunk_num UInt32, + hunk_start_line_number_old UInt32, + hunk_start_line_number_new UInt32, hunk_context LowCardinality(String), line LowCardinality(String), indent UInt8, @@ -82,11 +88,11 @@ CREATE TABLE git.line_changes file_change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), new_file_path LowCardinality(String), old_file_path LowCardinality(String), - file_lines_added UInt16, - file_lines_deleted UInt16, - file_hunks_added UInt16, - file_hunks_removed UInt16, - file_hunks_changed UInt16, + file_lines_added UInt32, + file_lines_deleted UInt32, + file_hunks_added UInt32, + file_hunks_removed UInt32, + file_hunks_changed UInt32, commit_hash String, author_name LowCardinality(String), @@ -104,12 +110,15 @@ CREATE TABLE git.line_changes commit_hunks_changed UInt32 ) ENGINE = MergeTree ORDER BY time; +Insert the data with the following commands: + clickhouse-client --query "INSERT INTO git.commits FORMAT TSV" < commits.tsv clickhouse-client --query "INSERT INTO git.file_changes FORMAT TSV" < file_changes.tsv clickhouse-client --query "INSERT INTO git.line_changes FORMAT TSV" < line_changes.tsv - */ +)"; +namespace po = boost::program_options; namespace DB { @@ -141,11 +150,11 @@ void writeText(LineType type, WriteBuffer & out) struct LineChange { int8_t sign{}; /// 1 if added, -1 if deleted - uint16_t line_number_old{}; - uint16_t line_number_new{}; - uint16_t hunk_num{}; /// ordinal number of hunk in diff, starting with 0 - uint16_t hunk_start_line_number_old{}; - uint16_t hunk_start_line_number_new{}; + uint32_t line_number_old{}; + uint32_t line_number_new{}; + uint32_t hunk_num{}; /// ordinal number of hunk in diff, starting with 0 + uint32_t hunk_start_line_number_old{}; + uint32_t hunk_start_line_number_new{}; std::string hunk_context; /// The context (like a line with function name) as it is calculated by git std::string line; /// Line content without leading whitespaces uint8_t indent{}; /// The number of leading whitespaces or tabs * 4 @@ -251,11 +260,11 @@ struct FileChange FileChangeType change_type{}; std::string new_file_path; std::string old_file_path; - uint16_t lines_added{}; - uint16_t lines_deleted{}; - uint16_t hunks_added{}; - uint16_t hunks_removed{}; - uint16_t hunks_changed{}; + uint32_t lines_added{}; + uint32_t lines_deleted{}; + uint32_t hunks_added{}; + uint32_t hunks_removed{}; + uint32_t hunks_changed{}; void writeTextWithoutNewline(WriteBuffer & out) const { @@ -395,13 +404,38 @@ struct Result }; -void processCommit(std::string hash, Result & result) +struct Options +{ + bool skip_commits_without_parents = true; + std::optional skip_paths; + std::unordered_set skip_commits; + size_t diff_size_limit = 0; + + Options(const po::variables_map & options) + { + skip_commits_without_parents = options["skip-commits-without-parents"].as(); + if (options.count("skip-paths")) + { + skip_paths.emplace(options["skip-paths"].as()); + } + if (options.count("skip-commit")) + { + auto vec = options["skip-commit"].as>(); + skip_commits.insert(vec.begin(), vec.end()); + } + diff_size_limit = options["diff-size-limit"].as(); + } +}; + + +void processCommit( + const Options & options, size_t commit_num, size_t total_commits, std::string hash, Result & result) { std::string command = fmt::format( - "git show --raw --pretty='format:%at%x09%aN%x09%aE%x0A%s%x00' --patch --unified=0 {}", + "git show --raw --pretty='format:%at%x09%aN%x09%aE%x09%P%x0A%s%x00' --patch --unified=0 {}", hash); - std::cerr << command << "\n"; + //std::cerr << command << "\n"; auto commit_info = ShellCommand::execute(command); auto & in = commit_info->out; @@ -414,10 +448,23 @@ void processCommit(std::string hash, Result & result) readText(commit.author_name, in); assertChar('\t', in); readText(commit.author_email, in); + assertChar('\t', in); + std::string parent_hash; + readString(parent_hash, in); assertChar('\n', in); readNullTerminated(commit.message, in); - std::cerr << fmt::format("{}\t{}\n", toString(LocalDateTime(commit.time)), commit.message); + std::string message_to_print = commit.message; + std::replace_if(message_to_print.begin(), message_to_print.end(), [](char c){ return std::iscntrl(c); }, ' '); + + fmt::print("{}% {} {} {}\n", + commit_num * 100 / total_commits, toString(LocalDateTime(commit.time)), hash, message_to_print); + + if (options.skip_commits_without_parents && commit_num != 0 && parent_hash.empty()) + { + std::cerr << "Warning: skipping commit without parents\n"; + return; + } if (!in.eof()) assertChar('\n', in); @@ -487,9 +534,12 @@ void processCommit(std::string hash, Result & result) assertChar('\n', in); - file_changes.emplace( - file_change.new_file_path, - FileChangeAndLineChanges{ file_change, {} }); + if (!(options.skip_paths && re2_st::RE2::PartialMatch(file_change.new_file_path, *options.skip_paths))) + { + file_changes.emplace( + file_change.new_file_path, + FileChangeAndLineChanges{ file_change, {} }); + } } if (!in.eof()) @@ -517,16 +567,14 @@ void processCommit(std::string hash, Result & result) { auto file_name = new_file_path.empty() ? old_file_path : new_file_path; auto it = file_changes.find(file_name); - if (file_changes.end() == it) - std::cerr << fmt::format("Warning: skipping bad file name {}\n", file_name); - else + if (file_changes.end() != it) file_change_and_line_changes = &it->second; } if (file_change_and_line_changes) { - uint16_t old_lines = 1; - uint16_t new_lines = 1; + uint32_t old_lines = 1; + uint32_t new_lines = 1; assertChar('-', in); readText(line_change.hunk_start_line_number_old, in); @@ -644,6 +692,9 @@ void processCommit(std::string hash, Result & result) } } + if (commit.lines_added + commit.lines_deleted > options.diff_size_limit) + return; + /// Write the result /// commits table @@ -684,14 +735,20 @@ void processCommit(std::string hash, Result & result) } -void processLog() +void processLog(const Options & options) { Result result; - std::string command = "git log --no-merges --pretty=%H"; - std::cerr << command << "\n"; + std::string command = "git log --reverse --no-merges --pretty=%H"; + fmt::print("{}\n", command); auto git_log = ShellCommand::execute(command); + /// Collect hashes in memory. This is inefficient but allows to display beautiful progress. + /// The number of commits is in order of single millions for the largest repositories, + /// so don't care about potential waste of ~100 MB of memory. + + std::vector hashes; + auto & in = git_log->out; while (!in.eof()) { @@ -699,33 +756,55 @@ void processLog() readString(hash, in); assertChar('\n', in); - std::cerr << fmt::format("Processing commit {}\n", hash); - processCommit(std::move(hash), result); + if (!options.skip_commits.count(hash)) + hashes.emplace_back(std::move(hash)); + } + + size_t num_commits = hashes.size(); + fmt::print("Total {} commits to process.\n", num_commits); + + for (size_t i = 0; i < num_commits; ++i) + { + processCommit(options, i, num_commits, hashes[i], result); } } } -int main(int /*argc*/, char ** /*argv*/) +int main(int argc, char ** argv) try { using namespace DB; -/* boost::program_options::options_description desc("Allowed options"); - desc.add_options()("help,h", "produce help message"); + po::options_description desc("Allowed options"); + desc.add_options() + ("help,h", "produce help message") + ("skip-commits-without-parents", po::value()->default_value(true), + "Skip commits without parents (except the initial commit)." + " These commits are usually erroneous but they can make sense in very rare cases.") + ("skip-paths", po::value(), + "Skip paths that matches regular expression (re2 syntax).") + ("skip-commit", po::value>(), + "Skip commit with specified hash. The option can be specified multiple times.") + ("diff-size-limit", po::value()->default_value(0), + "Skip commits whose diff size (number of added + removed lines) is larger than specified threshold") + ; - boost::program_options::variables_map options; - boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); + po::variables_map options; + po::store(boost::program_options::parse_command_line(argc, argv, desc), options); - if (options.count("help") || argc != 2) + if (options.count("help")) { - std::cout << "Usage: " << argv[0] << std::endl; - std::cout << desc << std::endl; + std::cout << documentation << '\n' + << "Usage: " << argv[0] << '\n' + << desc << '\n' + << "\nExample:\n" + << "\n./git-to-clickhouse --diff-size-limit 100000 --skip-paths '^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/'\n"; return 1; - }*/ + } - processLog(); + processLog(options); return 0; } catch (...) From abe836a584aeaf71b0ba04b8c8cc670385519e94 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 03:13:39 +0300 Subject: [PATCH 064/298] Remove emails as they are mostly useless --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 20 ++++++------------- 1 file changed, 6 insertions(+), 14 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index d6264a63978..9203efb0043 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -30,8 +30,7 @@ CREATE DATABASE git; CREATE TABLE git.commits ( hash String, - author_name LowCardinality(String), - author_email LowCardinality(String), + author LowCardinality(String), time DateTime, message String, files_added UInt32, @@ -57,8 +56,7 @@ CREATE TABLE git.file_changes hunks_changed UInt32, commit_hash String, - author_name LowCardinality(String), - author_email LowCardinality(String), + author LowCardinality(String), time DateTime, commit_message String, commit_files_added UInt32, @@ -95,8 +93,7 @@ CREATE TABLE git.line_changes file_hunks_changed UInt32, commit_hash String, - author_name LowCardinality(String), - author_email LowCardinality(String), + author LowCardinality(String), time DateTime, commit_message String, commit_files_added UInt32, @@ -295,8 +292,7 @@ struct FileChangeAndLineChanges struct Commit { std::string hash; - std::string author_name; - std::string author_email; + std::string author; time_t time{}; std::string message; uint32_t files_added{}; @@ -313,9 +309,7 @@ struct Commit { writeText(hash, out); writeChar('\t', out); - writeText(author_name, out); - writeChar('\t', out); - writeText(author_email, out); + writeText(author, out); writeChar('\t', out); writeText(time, out); writeChar('\t', out); @@ -445,9 +439,7 @@ void processCommit( readText(commit.time, in); assertChar('\t', in); - readText(commit.author_name, in); - assertChar('\t', in); - readText(commit.author_email, in); + readText(commit.author, in); assertChar('\t', in); std::string parent_hash; readString(parent_hash, in); From 09978decbdf40c95e7cd8855ad804a2ad31cc09d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 03:47:00 +0300 Subject: [PATCH 065/298] Adjustments --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 52 +++++++++++++------ 1 file changed, 35 insertions(+), 17 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 9203efb0043..a81bc6679a7 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -47,8 +48,9 @@ CREATE TABLE git.commits CREATE TABLE git.file_changes ( change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), - new_file_path LowCardinality(String), - old_file_path LowCardinality(String), + path LowCardinality(String), + old_path LowCardinality(String), + file_extension LowCardinality(String), lines_added UInt32, lines_deleted UInt32, hunks_added UInt32, @@ -84,8 +86,9 @@ CREATE TABLE git.line_changes line_type Enum('Empty' = 0, 'Comment' = 1, 'Punct' = 2, 'Code' = 3), file_change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), - new_file_path LowCardinality(String), - old_file_path LowCardinality(String), + path LowCardinality(String), + old_path LowCardinality(String), + file_extension LowCardinality(String), file_lines_added UInt32, file_lines_deleted UInt32, file_hunks_added UInt32, @@ -255,8 +258,9 @@ void writeText(FileChangeType type, WriteBuffer & out) struct FileChange { FileChangeType change_type{}; - std::string new_file_path; - std::string old_file_path; + std::string path; + std::string old_path; + std::string file_extension; uint32_t lines_added{}; uint32_t lines_deleted{}; uint32_t hunks_added{}; @@ -267,9 +271,11 @@ struct FileChange { writeText(change_type, out); writeChar('\t', out); - writeText(new_file_path, out); + writeText(path, out); writeChar('\t', out); - writeText(old_file_path, out); + writeText(old_path, out); + writeChar('\t', out); + writeText(file_extension, out); writeChar('\t', out); writeText(lines_added, out); writeChar('\t', out); @@ -422,11 +428,20 @@ struct Options }; +/// Rough snapshot of repository calculated by application of diffs. It's used to calculate blame info. +struct File +{ + std::vector lines; +}; + +using Snapshot = std::map; + + void processCommit( - const Options & options, size_t commit_num, size_t total_commits, std::string hash, Result & result) + const Options & options, size_t commit_num, size_t total_commits, std::string hash, Snapshot & /*snapshot*/, Result & result) { std::string command = fmt::format( - "git show --raw --pretty='format:%at%x09%aN%x09%aE%x09%P%x0A%s%x00' --patch --unified=0 {}", + "git show --raw --pretty='format:%at%x09%aN%x09%P%x0A%s%x00' --patch --unified=0 {}", hash); //std::cerr << command << "\n"; @@ -515,21 +530,23 @@ void processCommit( if (change_type == 'R' || change_type == 'C') { - readText(file_change.old_file_path, in); + readText(file_change.old_path, in); skipWhitespaceIfAny(in); - readText(file_change.new_file_path, in); + readText(file_change.path, in); } else { - readText(file_change.new_file_path, in); + readText(file_change.path, in); } + file_change.file_extension = std::filesystem::path(file_change.path).extension(); + assertChar('\n', in); - if (!(options.skip_paths && re2_st::RE2::PartialMatch(file_change.new_file_path, *options.skip_paths))) + if (!(options.skip_paths && re2_st::RE2::PartialMatch(file_change.path, *options.skip_paths))) { file_changes.emplace( - file_change.new_file_path, + file_change.path, FileChangeAndLineChanges{ file_change, {} }); } } @@ -755,9 +772,10 @@ void processLog(const Options & options) size_t num_commits = hashes.size(); fmt::print("Total {} commits to process.\n", num_commits); + Snapshot snapshot; for (size_t i = 0; i < num_commits; ++i) { - processCommit(options, i, num_commits, hashes[i], result); + processCommit(options, i, num_commits, hashes[i], snapshot, result); } } @@ -792,7 +810,7 @@ try << "Usage: " << argv[0] << '\n' << desc << '\n' << "\nExample:\n" - << "\n./git-to-clickhouse --diff-size-limit 100000 --skip-paths '^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/'\n"; + << "\n./git-to-clickhouse --diff-size-limit 100000 --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/'\n"; return 1; } From d1f1326a1370abd5d837864d02851ef1b3b20745 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 07:02:17 +0300 Subject: [PATCH 066/298] Concurrent processing + history --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 411 ++++++++++++------ 1 file changed, 283 insertions(+), 128 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index a81bc6679a7..6686c1ac480 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include @@ -80,11 +82,17 @@ CREATE TABLE git.line_changes hunk_num UInt32, hunk_start_line_number_old UInt32, hunk_start_line_number_new UInt32, + hunk_lines_added UInt32, + hunk_lines_deleted UInt32, hunk_context LowCardinality(String), line LowCardinality(String), indent UInt8, line_type Enum('Empty' = 0, 'Comment' = 1, 'Punct' = 2, 'Code' = 3), + prev_commit_hash String, + prev_author LowCardinality(String), + prev_time DateTime, + file_change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), path LowCardinality(String), old_path LowCardinality(String), @@ -128,6 +136,112 @@ namespace ErrorCodes extern const int INCORRECT_DATA; } + +struct Commit +{ + std::string hash; + std::string author; + LocalDateTime time{}; + std::string message; + uint32_t files_added{}; + uint32_t files_deleted{}; + uint32_t files_renamed{}; + uint32_t files_modified{}; + uint32_t lines_added{}; + uint32_t lines_deleted{}; + uint32_t hunks_added{}; + uint32_t hunks_removed{}; + uint32_t hunks_changed{}; + + void writeTextWithoutNewline(WriteBuffer & out) const + { + writeText(hash, out); + writeChar('\t', out); + writeText(author, out); + writeChar('\t', out); + writeText(time, out); + writeChar('\t', out); + writeText(message, out); + writeChar('\t', out); + writeText(files_added, out); + writeChar('\t', out); + writeText(files_deleted, out); + writeChar('\t', out); + writeText(files_renamed, out); + writeChar('\t', out); + writeText(files_modified, out); + writeChar('\t', out); + writeText(lines_added, out); + writeChar('\t', out); + writeText(lines_deleted, out); + writeChar('\t', out); + writeText(hunks_added, out); + writeChar('\t', out); + writeText(hunks_removed, out); + writeChar('\t', out); + writeText(hunks_changed, out); + } +}; + + +enum class FileChangeType +{ + Add, + Delete, + Modify, + Rename, + Copy, + Type, +}; + +void writeText(FileChangeType type, WriteBuffer & out) +{ + switch (type) + { + case FileChangeType::Add: writeString("Add", out); break; + case FileChangeType::Delete: writeString("Delete", out); break; + case FileChangeType::Modify: writeString("Modify", out); break; + case FileChangeType::Rename: writeString("Rename", out); break; + case FileChangeType::Copy: writeString("Copy", out); break; + case FileChangeType::Type: writeString("Type", out); break; + } +} + +struct FileChange +{ + FileChangeType change_type{}; + std::string path; + std::string old_path; + std::string file_extension; + uint32_t lines_added{}; + uint32_t lines_deleted{}; + uint32_t hunks_added{}; + uint32_t hunks_removed{}; + uint32_t hunks_changed{}; + + void writeTextWithoutNewline(WriteBuffer & out) const + { + writeText(change_type, out); + writeChar('\t', out); + writeText(path, out); + writeChar('\t', out); + writeText(old_path, out); + writeChar('\t', out); + writeText(file_extension, out); + writeChar('\t', out); + writeText(lines_added, out); + writeChar('\t', out); + writeText(lines_deleted, out); + writeChar('\t', out); + writeText(hunks_added, out); + writeChar('\t', out); + writeText(hunks_removed, out); + writeChar('\t', out); + writeText(hunks_changed, out); + } +}; + + enum class LineType { Empty, @@ -155,10 +269,15 @@ struct LineChange uint32_t hunk_num{}; /// ordinal number of hunk in diff, starting with 0 uint32_t hunk_start_line_number_old{}; uint32_t hunk_start_line_number_new{}; + uint32_t hunk_lines_added{}; + uint32_t hunk_lines_deleted{}; std::string hunk_context; /// The context (like a line with function name) as it is calculated by git std::string line; /// Line content without leading whitespaces uint8_t indent{}; /// The number of leading whitespaces or tabs * 4 LineType line_type{}; + std::string prev_commit_hash; + std::string prev_author; + LocalDateTime prev_time{}; void setLineInfo(std::string full_line) { @@ -220,6 +339,10 @@ struct LineChange writeChar('\t', out); writeText(hunk_start_line_number_new, out); writeChar('\t', out); + writeText(hunk_lines_added, out); + writeChar('\t', out); + writeText(hunk_lines_deleted, out); + writeChar('\t', out); writeText(hunk_context, out); writeChar('\t', out); writeText(line, out); @@ -227,120 +350,17 @@ struct LineChange writeText(indent, out); writeChar('\t', out); writeText(line_type, out); + writeChar('\t', out); + writeText(prev_commit_hash, out); + writeChar('\t', out); + writeText(prev_author, out); + writeChar('\t', out); + writeText(prev_time, out); } }; using LineChanges = std::vector; -enum class FileChangeType -{ - Add, - Delete, - Modify, - Rename, - Copy, - Type, -}; - -void writeText(FileChangeType type, WriteBuffer & out) -{ - switch (type) - { - case FileChangeType::Add: writeString("Add", out); break; - case FileChangeType::Delete: writeString("Delete", out); break; - case FileChangeType::Modify: writeString("Modify", out); break; - case FileChangeType::Rename: writeString("Rename", out); break; - case FileChangeType::Copy: writeString("Copy", out); break; - case FileChangeType::Type: writeString("Type", out); break; - } -} - -struct FileChange -{ - FileChangeType change_type{}; - std::string path; - std::string old_path; - std::string file_extension; - uint32_t lines_added{}; - uint32_t lines_deleted{}; - uint32_t hunks_added{}; - uint32_t hunks_removed{}; - uint32_t hunks_changed{}; - - void writeTextWithoutNewline(WriteBuffer & out) const - { - writeText(change_type, out); - writeChar('\t', out); - writeText(path, out); - writeChar('\t', out); - writeText(old_path, out); - writeChar('\t', out); - writeText(file_extension, out); - writeChar('\t', out); - writeText(lines_added, out); - writeChar('\t', out); - writeText(lines_deleted, out); - writeChar('\t', out); - writeText(hunks_added, out); - writeChar('\t', out); - writeText(hunks_removed, out); - writeChar('\t', out); - writeText(hunks_changed, out); - } -}; - -struct FileChangeAndLineChanges -{ - FileChange file_change; - LineChanges line_changes; -}; - -struct Commit -{ - std::string hash; - std::string author; - time_t time{}; - std::string message; - uint32_t files_added{}; - uint32_t files_deleted{}; - uint32_t files_renamed{}; - uint32_t files_modified{}; - uint32_t lines_added{}; - uint32_t lines_deleted{}; - uint32_t hunks_added{}; - uint32_t hunks_removed{}; - uint32_t hunks_changed{}; - - void writeTextWithoutNewline(WriteBuffer & out) const - { - writeText(hash, out); - writeChar('\t', out); - writeText(author, out); - writeChar('\t', out); - writeText(time, out); - writeChar('\t', out); - writeText(message, out); - writeChar('\t', out); - writeText(files_added, out); - writeChar('\t', out); - writeText(files_deleted, out); - writeChar('\t', out); - writeText(files_renamed, out); - writeChar('\t', out); - writeText(files_modified, out); - writeChar('\t', out); - writeText(lines_added, out); - writeChar('\t', out); - writeText(lines_deleted, out); - writeChar('\t', out); - writeText(hunks_added, out); - writeChar('\t', out); - writeText(hunks_removed, out); - writeChar('\t', out); - writeText(hunks_changed, out); - } -}; - void skipUntilWhitespace(ReadBuffer & buf) { @@ -407,13 +427,15 @@ struct Result struct Options { bool skip_commits_without_parents = true; + size_t threads = 1; std::optional skip_paths; std::unordered_set skip_commits; - size_t diff_size_limit = 0; + std::optional diff_size_limit; Options(const po::variables_map & options) { skip_commits_without_parents = options["skip-commits-without-parents"].as(); + threads = options["threads"].as(); if (options.count("skip-paths")) { skip_paths.emplace(options["skip-paths"].as()); @@ -423,36 +445,123 @@ struct Options auto vec = options["skip-commit"].as>(); skip_commits.insert(vec.begin(), vec.end()); } - diff_size_limit = options["diff-size-limit"].as(); + if (options.count("diff-size-limit")) + { + diff_size_limit = options["diff-size-limit"].as(); + } } }; /// Rough snapshot of repository calculated by application of diffs. It's used to calculate blame info. -struct File +struct FileBlame { - std::vector lines; + using Lines = std::list; + Lines lines; + Lines::iterator it; + size_t current_idx = 1; + + FileBlame() + { + it = lines.begin(); + } + + FileBlame & operator=(const FileBlame & rhs) + { + lines = rhs.lines; + it = lines.begin(); + current_idx = 1; + return *this; + } + + FileBlame(const FileBlame & rhs) + { + *this = rhs; + } + + void walk(uint32_t num) + { + if (current_idx < num) + { + while (current_idx < num && it != lines.end()) + { + ++current_idx; + ++it; + } + } + else if (current_idx > num) + { + --current_idx; + --it; + } + } + + const Commit * find(uint32_t num) + { + walk(num); + + if (current_idx == num && it != lines.end()) + return &*it; + return {}; + } + + void addLine(uint32_t num, Commit commit) + { + walk(num); + + while (it == lines.end() && current_idx < num) + { + lines.emplace_back(); + ++current_idx; + } + if (it == lines.end()) + { + lines.emplace_back(); + --it; + } + + lines.insert(it, commit); + } + + void removeLine(uint32_t num) + { + walk(num); + + if (current_idx == num) + it = lines.erase(it); + } }; -using Snapshot = std::map; +using Snapshot = std::map; + +struct FileChangeAndLineChanges +{ + FileChangeAndLineChanges(FileChange file_change_) : file_change(file_change_) {} + + FileChange file_change; + LineChanges line_changes; + + std::map deleted_lines; +}; void processCommit( - const Options & options, size_t commit_num, size_t total_commits, std::string hash, Snapshot & /*snapshot*/, Result & result) + std::unique_ptr & commit_info, + const Options & options, + size_t commit_num, + size_t total_commits, + std::string hash, + Snapshot & snapshot, + Result & result) { - std::string command = fmt::format( - "git show --raw --pretty='format:%at%x09%aN%x09%P%x0A%s%x00' --patch --unified=0 {}", - hash); - - //std::cerr << command << "\n"; - - auto commit_info = ShellCommand::execute(command); auto & in = commit_info->out; Commit commit; commit.hash = hash; - readText(commit.time, in); + time_t commit_time; + readText(commit_time, in); + commit.time = commit_time; assertChar('\t', in); readText(commit.author, in); assertChar('\t', in); @@ -465,7 +574,7 @@ void processCommit( std::replace_if(message_to_print.begin(), message_to_print.end(), [](char c){ return std::iscntrl(c); }, ' '); fmt::print("{}% {} {} {}\n", - commit_num * 100 / total_commits, toString(LocalDateTime(commit.time)), hash, message_to_print); + commit_num * 100 / total_commits, toString(commit.time), hash, message_to_print); if (options.skip_commits_without_parents && commit_num != 0 && parent_hash.empty()) { @@ -533,6 +642,8 @@ void processCommit( readText(file_change.old_path, in); skipWhitespaceIfAny(in); readText(file_change.path, in); + + snapshot[file_change.path] = snapshot[file_change.old_path]; } else { @@ -547,7 +658,7 @@ void processCommit( { file_changes.emplace( file_change.path, - FileChangeAndLineChanges{ file_change, {} }); + FileChangeAndLineChanges(file_change)); } } @@ -601,6 +712,9 @@ void processCommit( else assertChar('\n', in); + line_change.hunk_lines_added = new_lines; + line_change.hunk_lines_deleted = old_lines; + ++line_change.hunk_num; line_change.line_number_old = line_change.hunk_start_line_number_old; line_change.line_number_new = line_change.hunk_start_line_number_new; @@ -653,6 +767,16 @@ void processCommit( readStringUntilNextLine(line_change.line, in); line_change.setLineInfo(line_change.line); + FileBlame & file_snapshot = snapshot[old_file_path]; + if (const Commit * prev_commit = file_snapshot.find(line_change.line_number_old)) + { + line_change.prev_commit_hash = prev_commit->hash; + line_change.prev_author = prev_commit->author; + line_change.prev_time = prev_commit->time; + file_change_and_line_changes->deleted_lines[line_change.line_number_old] = *prev_commit; + file_snapshot.removeLine(line_change.line_number_old); + } + file_change_and_line_changes->line_changes.push_back(line_change); ++line_change.line_number_old; } @@ -689,6 +813,16 @@ void processCommit( readStringUntilNextLine(line_change.line, in); line_change.setLineInfo(line_change.line); + FileBlame & file_snapshot = snapshot[new_file_path]; + if (file_change_and_line_changes->deleted_lines.count(line_change.line_number_new)) + { + const auto & prev_commit = file_change_and_line_changes->deleted_lines[line_change.line_number_new]; + line_change.prev_commit_hash = prev_commit.hash; + line_change.prev_author = prev_commit.author; + line_change.prev_time = prev_commit.time; + } + file_snapshot.addLine(line_change.line_number_new, commit); + file_change_and_line_changes->line_changes.push_back(line_change); ++line_change.line_number_new; } @@ -701,7 +835,7 @@ void processCommit( } } - if (commit.lines_added + commit.lines_deleted > options.diff_size_limit) + if (options.diff_size_limit && commit.lines_added + commit.lines_deleted > *options.diff_size_limit) return; /// Write the result @@ -744,6 +878,16 @@ void processCommit( } +auto gitShow(const std::string & hash) +{ + std::string command = fmt::format( + "git show --raw --pretty='format:%at%x09%aN%x09%P%x0A%s%x00' --patch --unified=0 {}", + hash); + + return ShellCommand::execute(command); +} + + void processLog(const Options & options) { Result result; @@ -772,10 +916,19 @@ void processLog(const Options & options) size_t num_commits = hashes.size(); fmt::print("Total {} commits to process.\n", num_commits); + /// Will run multiple processes in parallel + size_t num_threads = options.threads; + + std::vector> show_commands(num_threads); + for (size_t i = 0; i < num_commits && i < num_threads; ++i) + show_commands[i] = gitShow(hashes[i]); + Snapshot snapshot; for (size_t i = 0; i < num_commits; ++i) { - processCommit(options, i, num_commits, hashes[i], snapshot, result); + processCommit(show_commands[i % num_threads], options, i, num_commits, hashes[i], snapshot, result); + if (i + num_threads < num_commits) + show_commands[i % num_threads] = gitShow(hashes[i + num_threads]); } } @@ -797,8 +950,10 @@ try "Skip paths that matches regular expression (re2 syntax).") ("skip-commit", po::value>(), "Skip commit with specified hash. The option can be specified multiple times.") - ("diff-size-limit", po::value()->default_value(0), + ("diff-size-limit", po::value(), "Skip commits whose diff size (number of added + removed lines) is larger than specified threshold") + ("threads", po::value()->default_value(std::thread::hardware_concurrency()), + "Number of threads to interact with git") ; po::variables_map options; From 3f29453c02ef3d3716927d81258218516b183d7b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 09:38:48 +0300 Subject: [PATCH 067/298] Roughly working blame --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 179 ++++++++++++++---- 1 file changed, 137 insertions(+), 42 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 6686c1ac480..c1c27a82812 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -13,6 +14,7 @@ #include #include +#include #include #include #include @@ -427,19 +429,26 @@ struct Result struct Options { bool skip_commits_without_parents = true; + bool skip_commits_with_duplicate_diffs = true; size_t threads = 1; std::optional skip_paths; + std::optional skip_commits_with_messages; std::unordered_set skip_commits; std::optional diff_size_limit; Options(const po::variables_map & options) { skip_commits_without_parents = options["skip-commits-without-parents"].as(); + skip_commits_with_duplicate_diffs = options["skip-commits-with-duplicate-diffs"].as(); threads = options["threads"].as(); if (options.count("skip-paths")) { skip_paths.emplace(options["skip-paths"].as()); } + if (options.count("skip-commits-with-messages")) + { + skip_commits_with_messages.emplace(options["skip-commits-with-messages"].as()); + } if (options.count("skip-commit")) { auto vec = options["skip-commit"].as>(); @@ -481,15 +490,12 @@ struct FileBlame void walk(uint32_t num) { - if (current_idx < num) + while (current_idx < num && it != lines.end()) { - while (current_idx < num && it != lines.end()) - { - ++current_idx; - ++it; - } + ++current_idx; + ++it; } - else if (current_idx > num) + while (current_idx > num) { --current_idx; --it; @@ -500,6 +506,8 @@ struct FileBlame { walk(num); +// std::cerr << "current_idx: " << current_idx << ", num: " << num << "\n"; + if (current_idx == num && it != lines.end()) return &*it; return {}; @@ -514,20 +522,17 @@ struct FileBlame lines.emplace_back(); ++current_idx; } - if (it == lines.end()) - { - lines.emplace_back(); - --it; - } - lines.insert(it, commit); + it = lines.insert(it, commit); } void removeLine(uint32_t num) { +// std::cerr << "Removing line " << num << ", current_idx: " << current_idx << "\n"; + walk(num); - if (current_idx == num) + if (current_idx == num && it != lines.end()) it = lines.erase(it); } }; @@ -540,10 +545,10 @@ struct FileChangeAndLineChanges FileChange file_change; LineChanges line_changes; - - std::map deleted_lines; }; +using DiffHashes = std::unordered_set; + void processCommit( std::unique_ptr & commit_info, @@ -552,6 +557,7 @@ void processCommit( size_t total_commits, std::string hash, Snapshot & snapshot, + DiffHashes & diff_hashes, Result & result) { auto & in = commit_info->out; @@ -570,6 +576,9 @@ void processCommit( assertChar('\n', in); readNullTerminated(commit.message, in); + if (options.skip_commits_with_messages && re2_st::RE2::PartialMatch(commit.message, *options.skip_commits_with_messages)) + return; + std::string message_to_print = commit.message; std::replace_if(message_to_print.begin(), message_to_print.end(), [](char c){ return std::iscntrl(c); }, ' '); @@ -643,7 +652,10 @@ void processCommit( skipWhitespaceIfAny(in); readText(file_change.path, in); - snapshot[file_change.path] = snapshot[file_change.old_path]; +// std::cerr << "Move from " << file_change.old_path << " to " << file_change.path << "\n"; + + if (file_change.path != file_change.old_path) + snapshot[file_change.path] = snapshot[file_change.old_path]; } else { @@ -706,6 +718,9 @@ void processCommit( if (checkChar(',', in)) readText(new_lines, in); + if (line_change.hunk_start_line_number_new == 0) + line_change.hunk_start_line_number_new = 1; + assertString(" @@", in); if (checkChar(' ', in)) readStringUntilNextLine(line_change.hunk_context, in); @@ -767,16 +782,6 @@ void processCommit( readStringUntilNextLine(line_change.line, in); line_change.setLineInfo(line_change.line); - FileBlame & file_snapshot = snapshot[old_file_path]; - if (const Commit * prev_commit = file_snapshot.find(line_change.line_number_old)) - { - line_change.prev_commit_hash = prev_commit->hash; - line_change.prev_author = prev_commit->author; - line_change.prev_time = prev_commit->time; - file_change_and_line_changes->deleted_lines[line_change.line_number_old] = *prev_commit; - file_snapshot.removeLine(line_change.line_number_old); - } - file_change_and_line_changes->line_changes.push_back(line_change); ++line_change.line_number_old; } @@ -813,16 +818,6 @@ void processCommit( readStringUntilNextLine(line_change.line, in); line_change.setLineInfo(line_change.line); - FileBlame & file_snapshot = snapshot[new_file_path]; - if (file_change_and_line_changes->deleted_lines.count(line_change.line_number_new)) - { - const auto & prev_commit = file_change_and_line_changes->deleted_lines[line_change.line_number_new]; - line_change.prev_commit_hash = prev_commit.hash; - line_change.prev_author = prev_commit.author; - line_change.prev_time = prev_commit.time; - } - file_snapshot.addLine(line_change.line_number_new, commit); - file_change_and_line_changes->line_changes.push_back(line_change); ++line_change.line_number_new; } @@ -838,6 +833,99 @@ void processCommit( if (options.diff_size_limit && commit.lines_added + commit.lines_deleted > *options.diff_size_limit) return; + /// Calculate hash of diff and skip duplicates + if (options.skip_commits_with_duplicate_diffs) + { + SipHash hasher; + + for (auto & elem : file_changes) + { + hasher.update(elem.second.file_change.change_type); + hasher.update(elem.second.file_change.old_path.size()); + hasher.update(elem.second.file_change.old_path); + hasher.update(elem.second.file_change.path.size()); + hasher.update(elem.second.file_change.path); + + hasher.update(elem.second.line_changes.size()); + for (auto & line_change : elem.second.line_changes) + { + hasher.update(line_change.sign); + hasher.update(line_change.line_number_old); + hasher.update(line_change.line_number_new); + hasher.update(line_change.indent); + hasher.update(line_change.line.size()); + hasher.update(line_change.line); + } + } + + UInt128 hash_of_diff; + hasher.get128(hash_of_diff.low, hash_of_diff.high); + + if (!diff_hashes.insert(hash_of_diff).second) + return; + } + + /// Update snapshot and blame info + + for (auto & elem : file_changes) + { +// std::cerr << elem.first << "\n"; + + FileBlame & file_snapshot = snapshot[elem.first]; + std::unordered_map deleted_lines; + + /// Obtain blame info from previous state of the snapshot + + for (auto & line_change : elem.second.line_changes) + { + if (line_change.sign == -1) + { + if (const Commit * prev_commit = file_snapshot.find(line_change.line_number_old); + prev_commit && prev_commit->time <= commit.time) + { + line_change.prev_commit_hash = prev_commit->hash; + line_change.prev_author = prev_commit->author; + line_change.prev_time = prev_commit->time; + deleted_lines[line_change.line_number_old] = *prev_commit; + } + else + { + // std::cerr << "Did not find line " << line_change.line_number_old << " from file " << elem.first << ": " << line_change.line << "\n"; + } + } + else if (line_change.sign == 1) + { + uint32_t this_line_in_prev_commit = line_change.hunk_start_line_number_old + + (line_change.line_number_new - line_change.hunk_start_line_number_new); + + if (deleted_lines.count(this_line_in_prev_commit)) + { + const auto & prev_commit = deleted_lines[this_line_in_prev_commit]; + if (prev_commit.time <= commit.time) + { + line_change.prev_commit_hash = prev_commit.hash; + line_change.prev_author = prev_commit.author; + line_change.prev_time = prev_commit.time; + } + } + } + } + + /// Update the snapshot + + for (const auto & line_change : elem.second.line_changes) + { + if (line_change.sign == -1) + { + file_snapshot.removeLine(line_change.line_number_new); + } + else if (line_change.sign == 1) + { + file_snapshot.addLine(line_change.line_number_new, commit); + } + } + } + /// Write the result /// commits table @@ -881,7 +969,7 @@ void processCommit( auto gitShow(const std::string & hash) { std::string command = fmt::format( - "git show --raw --pretty='format:%at%x09%aN%x09%P%x0A%s%x00' --patch --unified=0 {}", + "git show --raw --pretty='format:%ct%x09%aN%x09%P%x0A%s%x00' --patch --unified=0 {}", hash); return ShellCommand::execute(command); @@ -924,9 +1012,11 @@ void processLog(const Options & options) show_commands[i] = gitShow(hashes[i]); Snapshot snapshot; + DiffHashes diff_hashes; + for (size_t i = 0; i < num_commits; ++i) { - processCommit(show_commands[i % num_threads], options, i, num_commits, hashes[i], snapshot, result); + processCommit(show_commands[i % num_threads], options, i, num_commits, hashes[i], snapshot, diff_hashes, result); if (i + num_threads < num_commits) show_commands[i % num_threads] = gitShow(hashes[i + num_threads]); } @@ -946,10 +1036,15 @@ try ("skip-commits-without-parents", po::value()->default_value(true), "Skip commits without parents (except the initial commit)." " These commits are usually erroneous but they can make sense in very rare cases.") - ("skip-paths", po::value(), - "Skip paths that matches regular expression (re2 syntax).") + ("skip-commits-with-duplicate-diffs", po::value()->default_value(true), + "Skip commits with duplicate diffs." + " These commits are usually results of cherry-pick or merge after rebase.") ("skip-commit", po::value>(), "Skip commit with specified hash. The option can be specified multiple times.") + ("skip-paths", po::value(), + "Skip paths that matches regular expression (re2 syntax).") + ("skip-commits-with-messages", po::value(), + "Skip commits whose messages matches regular expression (re2 syntax).") ("diff-size-limit", po::value(), "Skip commits whose diff size (number of added + removed lines) is larger than specified threshold") ("threads", po::value()->default_value(std::thread::hardware_concurrency()), @@ -965,7 +1060,7 @@ try << "Usage: " << argv[0] << '\n' << desc << '\n' << "\nExample:\n" - << "\n./git-to-clickhouse --diff-size-limit 100000 --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/'\n"; + << "\n./git-to-clickhouse --diff-size-limit 100000 --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/' --skip-commits-with-messages '^Merge branch '\n"; return 1; } From 99c33612d65c627bbb9fc31d9d97906195d3cf53 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 10:29:58 +0300 Subject: [PATCH 068/298] Better diagnostics --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index c1c27a82812..6b29708ead3 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -582,7 +582,7 @@ void processCommit( std::string message_to_print = commit.message; std::replace_if(message_to_print.begin(), message_to_print.end(), [](char c){ return std::iscntrl(c); }, ' '); - fmt::print("{}% {} {} {}\n", + std::cerr << fmt::format("{}% {} {} {}\n", commit_num * 100 / total_commits, toString(commit.time), hash, message_to_print); if (options.skip_commits_without_parents && commit_num != 0 && parent_hash.empty()) From 3ec9656aa21a3142d2898b7d259a4740a6691fd2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 10:38:39 +0300 Subject: [PATCH 069/298] Slightly more robust --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 6b29708ead3..f3653bb282f 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -568,12 +568,10 @@ void processCommit( time_t commit_time; readText(commit_time, in); commit.time = commit_time; - assertChar('\t', in); - readText(commit.author, in); - assertChar('\t', in); + assertChar('\0', in); + readNullTerminated(commit.author, in); std::string parent_hash; - readString(parent_hash, in); - assertChar('\n', in); + readNullTerminated(parent_hash, in); readNullTerminated(commit.message, in); if (options.skip_commits_with_messages && re2_st::RE2::PartialMatch(commit.message, *options.skip_commits_with_messages)) @@ -969,7 +967,7 @@ void processCommit( auto gitShow(const std::string & hash) { std::string command = fmt::format( - "git show --raw --pretty='format:%ct%x09%aN%x09%P%x0A%s%x00' --patch --unified=0 {}", + "git show --raw --pretty='format:%ct%x00%aN%x00%P%x00%s%x00' --patch --unified=0 {}", hash); return ShellCommand::execute(command); From 25ca5e91bd0f3074c8d7d0874e125d2dcc611889 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 16:09:32 -0700 Subject: [PATCH 070/298] AsynchronousMetricLog - add event_time_microseconds column --- src/Core/Field.h | 3 ++- src/Interpreters/AsynchronousMetricLog.cpp | 17 +++++++++++++---- src/Interpreters/AsynchronousMetricLog.h | 1 + 3 files changed, 16 insertions(+), 5 deletions(-) diff --git a/src/Core/Field.h b/src/Core/Field.h index 0bfdf597543..8973d106c0b 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -768,7 +768,8 @@ T & Field::get() // Disregard signedness when converting between int64 types. constexpr Field::Types::Which target = TypeToEnum>::value; assert(target == which - || (isInt64FieldType(target) && isInt64FieldType(which))); + || (isInt64FieldType(target) && isInt64FieldType(which)) + || target == Field::Types::Decimal64 /* DateTime64 fields */); #endif ValueType * MAY_ALIAS ptr = reinterpret_cast(&storage); diff --git a/src/Interpreters/AsynchronousMetricLog.cpp b/src/Interpreters/AsynchronousMetricLog.cpp index e4415773655..d2c81c9dfc5 100644 --- a/src/Interpreters/AsynchronousMetricLog.cpp +++ b/src/Interpreters/AsynchronousMetricLog.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -13,10 +14,11 @@ Block AsynchronousMetricLogElement::createBlock() { ColumnsWithTypeAndName columns; - columns.emplace_back(std::make_shared(), "event_date"); - columns.emplace_back(std::make_shared(), "event_time"); - columns.emplace_back(std::make_shared(), "name"); - columns.emplace_back(std::make_shared(), "value"); + columns.emplace_back(std::make_shared(), "event_date"); + columns.emplace_back(std::make_shared(), "event_time"); + columns.emplace_back(std::make_shared(6), "event_time_microseconds"); + columns.emplace_back(std::make_shared(), "name"); + columns.emplace_back(std::make_shared(), "value"); return Block(columns); } @@ -28,6 +30,7 @@ void AsynchronousMetricLogElement::appendToBlock(MutableColumns & columns) const columns[column_idx++]->insert(event_date); columns[column_idx++]->insert(event_time); + columns[column_idx++]->insert(event_time_microseconds); columns[column_idx++]->insert(metric_name); columns[column_idx++]->insert(value); } @@ -38,6 +41,11 @@ inline UInt64 time_in_milliseconds(std::chrono::time_point(timepoint.time_since_epoch()).count(); } +inline UInt64 time_in_microseconds(std::chrono::time_point timepoint) +{ + return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); +} + inline UInt64 time_in_seconds(std::chrono::time_point timepoint) { @@ -50,6 +58,7 @@ void AsynchronousMetricLog::addValues(const AsynchronousMetricValues & values) const auto now = std::chrono::system_clock::now(); element.event_time = time_in_seconds(now); + element.event_time_microseconds = time_in_microseconds(now); element.event_date = DateLUT::instance().toDayNum(element.event_time); for (const auto & [key, value] : values) diff --git a/src/Interpreters/AsynchronousMetricLog.h b/src/Interpreters/AsynchronousMetricLog.h index b7d6aab95b6..0c02244246e 100644 --- a/src/Interpreters/AsynchronousMetricLog.h +++ b/src/Interpreters/AsynchronousMetricLog.h @@ -22,6 +22,7 @@ struct AsynchronousMetricLogElement { UInt16 event_date; time_t event_time; + UInt64 event_time_microseconds; std::string metric_name; double value; From 6a5b885ac1167dd8bba4a0c8b091289dd4c1e79e Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 16:17:28 -0700 Subject: [PATCH 071/298] AsynchronousMetricLog - add tests for event_time_microseconds column --- .../0_stateless/01473_event_time_microseconds.reference | 2 ++ tests/queries/0_stateless/01473_event_time_microseconds.sql | 5 +++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/01473_event_time_microseconds.reference create mode 100644 tests/queries/0_stateless/01473_event_time_microseconds.sql diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference new file mode 100644 index 00000000000..6c3b6ec5c6c --- /dev/null +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -0,0 +1,2 @@ +'01473_asynchronous_metric_log_event_start_time_milliseconds_test' +ok \ No newline at end of file diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql new file mode 100644 index 00000000000..af38cf4ca70 --- /dev/null +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -0,0 +1,5 @@ +set log_queries = 1; + +select '01473_asynchronous_metric_log_event_start_time_milliseconds_test'; +system flush logs; +SELECT If((select count(event_time_microseconds) from system.asynchronous_metric_log) > 0, 'ok', 'fail'); -- success From 1c1f50c6b665b96d7ba5742a60c770081299213d Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 16:19:38 -0700 Subject: [PATCH 072/298] MetricLog - add event_time_microseconds column --- src/Interpreters/MetricLog.cpp | 14 +++++++++++--- src/Interpreters/MetricLog.h | 1 + 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/MetricLog.cpp b/src/Interpreters/MetricLog.cpp index 69fcc4917b9..ce5d5793b87 100644 --- a/src/Interpreters/MetricLog.cpp +++ b/src/Interpreters/MetricLog.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -11,9 +12,10 @@ Block MetricLogElement::createBlock() { ColumnsWithTypeAndName columns_with_type_and_name; - columns_with_type_and_name.emplace_back(std::make_shared(), "event_date"); - columns_with_type_and_name.emplace_back(std::make_shared(), "event_time"); - columns_with_type_and_name.emplace_back(std::make_shared(), "milliseconds"); + columns_with_type_and_name.emplace_back(std::make_shared(), "event_date"); + columns_with_type_and_name.emplace_back(std::make_shared(), "event_time"); + columns_with_type_and_name.emplace_back(std::make_shared(6), "event_time_microseconds"); + columns_with_type_and_name.emplace_back(std::make_shared(), "milliseconds"); for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) { @@ -41,6 +43,7 @@ void MetricLogElement::appendToBlock(MutableColumns & columns) const columns[column_idx++]->insert(DateLUT::instance().toDayNum(event_time)); columns[column_idx++]->insert(event_time); + columns[column_idx++]->insert(event_time_microseconds); columns[column_idx++]->insert(milliseconds); for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) @@ -80,6 +83,10 @@ inline UInt64 time_in_milliseconds(std::chrono::time_point(timepoint.time_since_epoch()).count(); } +inline UInt64 time_in_microseconds(std::chrono::time_point timepoint) +{ + return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); +} inline UInt64 time_in_seconds(std::chrono::time_point timepoint) { @@ -102,6 +109,7 @@ void MetricLog::metricThreadFunction() MetricLogElement elem; elem.event_time = std::chrono::system_clock::to_time_t(current_time); + elem.event_time_microseconds = time_in_microseconds(current_time); elem.milliseconds = time_in_milliseconds(current_time) - time_in_seconds(current_time) * 1000; elem.profile_events.resize(ProfileEvents::end()); diff --git a/src/Interpreters/MetricLog.h b/src/Interpreters/MetricLog.h index 7774a45d7e1..f52d078bdc9 100644 --- a/src/Interpreters/MetricLog.h +++ b/src/Interpreters/MetricLog.h @@ -18,6 +18,7 @@ namespace DB struct MetricLogElement { time_t event_time{}; + UInt64 event_time_microseconds{}; UInt64 milliseconds{}; std::vector profile_events; From ec0d0243cc994f91fcc8a6a0fd36bde22f853af5 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 16:21:51 -0700 Subject: [PATCH 073/298] MetricLog - add tests for event_time_microseconds field --- .../0_stateless/01473_event_time_microseconds.reference | 6 ++++-- tests/queries/0_stateless/01473_event_time_microseconds.sql | 4 ++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference index 6c3b6ec5c6c..cac87f32a29 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.reference +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -1,2 +1,4 @@ -'01473_asynchronous_metric_log_event_start_time_milliseconds_test' -ok \ No newline at end of file +01473_asynchronous_metric_log_event_start_time_milliseconds_test +ok +01473_metric_log_event_start_time_milliseconds_test +ok diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index af38cf4ca70..6a13d6e1543 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -3,3 +3,7 @@ set log_queries = 1; select '01473_asynchronous_metric_log_event_start_time_milliseconds_test'; system flush logs; SELECT If((select count(event_time_microseconds) from system.asynchronous_metric_log) > 0, 'ok', 'fail'); -- success + +select '01473_metric_log_event_start_time_milliseconds_test'; +system flush logs; +SELECT If((select count(event_time_microseconds) from system.metric_log) > 0, 'ok', 'fail'); -- success From 9de49d130f733c0fcc00d8f7bd85c4fb2eecbfc2 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 16:36:38 -0700 Subject: [PATCH 074/298] AsynchronousMetricLog & MetricLog - update docs with examples --- .../system-tables/asynchronous_metric_log.md | 25 ++++++------ .../en/operations/system-tables/metric_log.md | 40 +++++++++---------- 2 files changed, 33 insertions(+), 32 deletions(-) diff --git a/docs/en/operations/system-tables/asynchronous_metric_log.md b/docs/en/operations/system-tables/asynchronous_metric_log.md index 6b1d71e1ca6..75607cc30b0 100644 --- a/docs/en/operations/system-tables/asynchronous_metric_log.md +++ b/docs/en/operations/system-tables/asynchronous_metric_log.md @@ -6,6 +6,7 @@ Columns: - `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time. +- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Event time with microseconds resolution. - `name` ([String](../../sql-reference/data-types/string.md)) — Metric name. - `value` ([Float64](../../sql-reference/data-types/float.md)) — Metric value. @@ -16,18 +17,18 @@ SELECT * FROM system.asynchronous_metric_log LIMIT 10 ``` ``` text -┌─event_date─┬──────────event_time─┬─name─────────────────────────────────────┬────value─┐ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.arenas.all.pmuzzy │ 0 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.arenas.all.pdirty │ 4214 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.background_thread.run_intervals │ 0 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.background_thread.num_runs │ 0 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.retained │ 17657856 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.mapped │ 71471104 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.resident │ 61538304 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.metadata │ 6199264 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.allocated │ 38074336 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.epoch │ 2 │ -└────────────┴─────────────────────┴──────────────────────────────────────────┴──────────┘ +┌─event_date─┬──────────event_time─┬────event_time_microseconds─┬─name─────────────────────────────────────┬─────value─┐ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ CPUFrequencyMHz_0 │ 2120.9 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.arenas.all.pmuzzy │ 743 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.arenas.all.pdirty │ 26288 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.background_thread.run_intervals │ 0 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.background_thread.num_runs │ 0 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.retained │ 60694528 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.mapped │ 303161344 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.resident │ 260931584 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.metadata │ 12079488 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.allocated │ 133756128 │ +└────────────┴─────────────────────┴────────────────────────────┴──────────────────────────────────────────┴───────────┘ ``` **See Also** diff --git a/docs/en/operations/system-tables/metric_log.md b/docs/en/operations/system-tables/metric_log.md index 9ccf61291d2..063fe81923b 100644 --- a/docs/en/operations/system-tables/metric_log.md +++ b/docs/en/operations/system-tables/metric_log.md @@ -23,28 +23,28 @@ SELECT * FROM system.metric_log LIMIT 1 FORMAT Vertical; ``` text Row 1: ────── -event_date: 2020-02-18 -event_time: 2020-02-18 07:15:33 -milliseconds: 554 -ProfileEvent_Query: 0 -ProfileEvent_SelectQuery: 0 -ProfileEvent_InsertQuery: 0 -ProfileEvent_FileOpen: 0 -ProfileEvent_Seek: 0 -ProfileEvent_ReadBufferFromFileDescriptorRead: 1 -ProfileEvent_ReadBufferFromFileDescriptorReadFailed: 0 -ProfileEvent_ReadBufferFromFileDescriptorReadBytes: 0 -ProfileEvent_WriteBufferFromFileDescriptorWrite: 1 -ProfileEvent_WriteBufferFromFileDescriptorWriteFailed: 0 -ProfileEvent_WriteBufferFromFileDescriptorWriteBytes: 56 +event_date: 2020-09-05 +event_time: 2020-09-05 16:22:33 +event_time_microseconds: 2020-09-05 16:22:33.196807 +milliseconds: 196 +ProfileEvent_Query: 0 +ProfileEvent_SelectQuery: 0 +ProfileEvent_InsertQuery: 0 +ProfileEvent_FailedQuery: 0 +ProfileEvent_FailedSelectQuery: 0 ... -CurrentMetric_Query: 0 -CurrentMetric_Merge: 0 -CurrentMetric_PartMutation: 0 -CurrentMetric_ReplicatedFetch: 0 -CurrentMetric_ReplicatedSend: 0 -CurrentMetric_ReplicatedChecks: 0 ... +CurrentMetric_Revision: 54439 +CurrentMetric_VersionInteger: 20009001 +CurrentMetric_RWLockWaitingReaders: 0 +CurrentMetric_RWLockWaitingWriters: 0 +CurrentMetric_RWLockActiveReaders: 0 +CurrentMetric_RWLockActiveWriters: 0 +CurrentMetric_GlobalThread: 74 +CurrentMetric_GlobalThreadActive: 26 +CurrentMetric_LocalThread: 0 +CurrentMetric_LocalThreadActive: 0 +CurrentMetric_DistributedFilesToInsert: 0 ``` **See also** From db58fa15aaf202318e043549440589797b51aa0a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 02:24:31 +0300 Subject: [PATCH 075/298] Some tweaks --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index f3653bb282f..9e1ef14fcbf 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -435,6 +435,7 @@ struct Options std::optional skip_commits_with_messages; std::unordered_set skip_commits; std::optional diff_size_limit; + std::string stop_after_commit; Options(const po::variables_map & options) { @@ -458,6 +459,10 @@ struct Options { diff_size_limit = options["diff-size-limit"].as(); } + if (options.count("stop-after-commit")) + { + stop_after_commit = options["stop-after-commit"].as(); + } } }; @@ -828,7 +833,7 @@ void processCommit( } } - if (options.diff_size_limit && commit.lines_added + commit.lines_deleted > *options.diff_size_limit) + if (options.diff_size_limit && commit_num != 0 && commit.lines_added + commit.lines_deleted > *options.diff_size_limit) return; /// Calculate hash of diff and skip duplicates @@ -1015,6 +1020,10 @@ void processLog(const Options & options) for (size_t i = 0; i < num_commits; ++i) { processCommit(show_commands[i % num_threads], options, i, num_commits, hashes[i], snapshot, diff_hashes, result); + + if (!options.stop_after_commit.empty() && hashes[i] == options.stop_after_commit) + break; + if (i + num_threads < num_commits) show_commands[i % num_threads] = gitShow(hashes[i + num_threads]); } @@ -1043,10 +1052,12 @@ try "Skip paths that matches regular expression (re2 syntax).") ("skip-commits-with-messages", po::value(), "Skip commits whose messages matches regular expression (re2 syntax).") - ("diff-size-limit", po::value(), - "Skip commits whose diff size (number of added + removed lines) is larger than specified threshold") + ("diff-size-limit", po::value()->default_value(100000), + "Skip commits whose diff size (number of added + removed lines) is larger than specified threshold. Does not apply for initial commit.") + ("stop-after-commit", po::value(), + "Stop processing after specified commit hash.") ("threads", po::value()->default_value(std::thread::hardware_concurrency()), - "Number of threads to interact with git") + "Number of concurrent git subprocesses to spawn") ; po::variables_map options; @@ -1058,7 +1069,7 @@ try << "Usage: " << argv[0] << '\n' << desc << '\n' << "\nExample:\n" - << "\n./git-to-clickhouse --diff-size-limit 100000 --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/' --skip-commits-with-messages '^Merge branch '\n"; + << "\n./git-to-clickhouse --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/' --skip-commits-with-messages '^Merge branch '\n"; return 1; } From 684a910395cc37203453d1faa09ab839d3a4f32a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 03:17:26 +0300 Subject: [PATCH 076/298] Polish --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 803 ++++++++++-------- 1 file changed, 451 insertions(+), 352 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 9e1ef14fcbf..6e43853d6ba 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -277,10 +277,14 @@ struct LineChange std::string line; /// Line content without leading whitespaces uint8_t indent{}; /// The number of leading whitespaces or tabs * 4 LineType line_type{}; + /// Information from the history (blame). std::string prev_commit_hash; std::string prev_author; LocalDateTime prev_time{}; + /** Classify line to empty / code / comment / single punctuation char. + * Very rough and mostly suitable for our C++ style. + */ void setLineInfo(std::string full_line) { indent = 0; @@ -306,8 +310,9 @@ struct LineChange line_type = LineType::Empty; } else if (pos + 1 < end - && ((pos[0] == '/' && pos[1] == '/') - || (pos[0] == '*' && pos[1] == ' '))) /// This is not precise. + && ((pos[0] == '/' && (pos[1] == '/' || pos[1] == '*')) + || (pos[0] == '*' && pos[1] == ' ') /// This is not precise. + || (pos[0] == '#' && pos[1] == ' '))) { line_type = LineType::Comment; } @@ -363,6 +368,18 @@ struct LineChange using LineChanges = std::vector; +struct FileDiff +{ + FileDiff(FileChange file_change_) : file_change(file_change_) {} + + FileChange file_change; + LineChanges line_changes; +}; + +using CommitDiff = std::map; + + +/** Parsing helpers */ void skipUntilWhitespace(ReadBuffer & buf) { @@ -418,14 +435,57 @@ void readStringUntilNextLine(std::string & s, ReadBuffer & buf) } -struct Result +/** Writes the resulting tables to files that can be imported to ClickHouse. + */ +struct ResultWriter { WriteBufferFromFile commits{"commits.tsv"}; WriteBufferFromFile file_changes{"file_changes.tsv"}; WriteBufferFromFile line_changes{"line_changes.tsv"}; + + void appendCommit(const Commit & commit, const CommitDiff & files) + { + /// commits table + { + auto & out = commits; + + commit.writeTextWithoutNewline(out); + writeChar('\n', out); + } + + for (const auto & elem : files) + { + const FileChange & file_change = elem.second.file_change; + + /// file_changes table + { + auto & out = file_changes; + + file_change.writeTextWithoutNewline(out); + writeChar('\t', out); + commit.writeTextWithoutNewline(out); + writeChar('\n', out); + } + + /// line_changes table + for (const auto & line_change : elem.second.line_changes) + { + auto & out = line_changes; + + line_change.writeTextWithoutNewline(out); + writeChar('\t', out); + file_change.writeTextWithoutNewline(out); + writeChar('\t', out); + commit.writeTextWithoutNewline(out); + writeChar('\n', out); + } + } + } }; +/** See description in "main". + */ struct Options { bool skip_commits_without_parents = true; @@ -467,11 +527,23 @@ struct Options }; -/// Rough snapshot of repository calculated by application of diffs. It's used to calculate blame info. +/** Rough snapshot of repository calculated by application of diffs. It's used to calculate blame info. + * Represented by a list of lines. For every line it contains information about commit that modified this line the last time. + * + * Note that there are many cases when this info may become incorrect. + * The first reason is that git history is non-linear but we form this snapshot by application of commit diffs in some order + * that cannot give us correct results even theoretically. + * The second reason is that we don't process merge commits. But merge commits may contain differences for conflict resolution. + * + * We expect that the information will be mostly correct for the purpose of analytics. + * So, it can provide the expected "blame" info for the most of the lines. + */ struct FileBlame { using Lines = std::list; Lines lines; + + /// We walk through this list adding or removing lines. Lines::iterator it; size_t current_idx = 1; @@ -480,6 +552,7 @@ struct FileBlame it = lines.begin(); } + /// This is important when file was copied or renamed. FileBlame & operator=(const FileBlame & rhs) { lines = rhs.lines; @@ -493,6 +566,7 @@ struct FileBlame *this = rhs; } + /// Move iterator to requested line or stop at the end. void walk(uint32_t num) { while (current_idx < num && it != lines.end()) @@ -522,6 +596,7 @@ struct FileBlame { walk(num); + /// If the inserted line is over the end of file, we insert empty lines before it. while (it == lines.end() && current_idx < num) { lines.emplace_back(); @@ -542,334 +617,24 @@ struct FileBlame } }; +/// All files with their blame info. When file is renamed, we also rename it in snapshot. using Snapshot = std::map; -struct FileChangeAndLineChanges + +/** Enrich the line changes data with the history info from the snapshot + * - the author, time and commit of the previous change to every found line (blame). + * And update the snapshot. + */ +void updateSnapshot(Snapshot & snapshot, const Commit & commit, CommitDiff & file_changes) { - FileChangeAndLineChanges(FileChange file_change_) : file_change(file_change_) {} - - FileChange file_change; - LineChanges line_changes; -}; - -using DiffHashes = std::unordered_set; - - -void processCommit( - std::unique_ptr & commit_info, - const Options & options, - size_t commit_num, - size_t total_commits, - std::string hash, - Snapshot & snapshot, - DiffHashes & diff_hashes, - Result & result) -{ - auto & in = commit_info->out; - - Commit commit; - commit.hash = hash; - - time_t commit_time; - readText(commit_time, in); - commit.time = commit_time; - assertChar('\0', in); - readNullTerminated(commit.author, in); - std::string parent_hash; - readNullTerminated(parent_hash, in); - readNullTerminated(commit.message, in); - - if (options.skip_commits_with_messages && re2_st::RE2::PartialMatch(commit.message, *options.skip_commits_with_messages)) - return; - - std::string message_to_print = commit.message; - std::replace_if(message_to_print.begin(), message_to_print.end(), [](char c){ return std::iscntrl(c); }, ' '); - - std::cerr << fmt::format("{}% {} {} {}\n", - commit_num * 100 / total_commits, toString(commit.time), hash, message_to_print); - - if (options.skip_commits_without_parents && commit_num != 0 && parent_hash.empty()) + /// Renames and copies. + for (auto & elem : file_changes) { - std::cerr << "Warning: skipping commit without parents\n"; - return; + auto & file = elem.second.file_change; + if (file.path != file.old_path) + snapshot[file.path] = snapshot[file.old_path]; } - if (!in.eof()) - assertChar('\n', in); - - /// File changes in form - /// :100644 100644 b90fe6bb94 3ffe4c380f M src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp - /// :100644 100644 828dedf6b5 828dedf6b5 R100 dbms/src/Functions/GeoUtils.h dbms/src/Functions/PolygonUtils.h - - std::map file_changes; - - while (checkChar(':', in)) - { - FileChange file_change; - - for (size_t i = 0; i < 4; ++i) - { - skipUntilWhitespace(in); - skipWhitespaceIfAny(in); - } - - char change_type; - readChar(change_type, in); - - int confidence; - switch (change_type) - { - case 'A': - file_change.change_type = FileChangeType::Add; - ++commit.files_added; - break; - case 'D': - file_change.change_type = FileChangeType::Delete; - ++commit.files_deleted; - break; - case 'M': - file_change.change_type = FileChangeType::Modify; - ++commit.files_modified; - break; - case 'R': - file_change.change_type = FileChangeType::Rename; - ++commit.files_renamed; - readText(confidence, in); - break; - case 'C': - file_change.change_type = FileChangeType::Copy; - readText(confidence, in); - break; - case 'T': - file_change.change_type = FileChangeType::Type; - break; - default: - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected file change type: {}", change_type); - } - - skipWhitespaceIfAny(in); - - if (change_type == 'R' || change_type == 'C') - { - readText(file_change.old_path, in); - skipWhitespaceIfAny(in); - readText(file_change.path, in); - -// std::cerr << "Move from " << file_change.old_path << " to " << file_change.path << "\n"; - - if (file_change.path != file_change.old_path) - snapshot[file_change.path] = snapshot[file_change.old_path]; - } - else - { - readText(file_change.path, in); - } - - file_change.file_extension = std::filesystem::path(file_change.path).extension(); - - assertChar('\n', in); - - if (!(options.skip_paths && re2_st::RE2::PartialMatch(file_change.path, *options.skip_paths))) - { - file_changes.emplace( - file_change.path, - FileChangeAndLineChanges(file_change)); - } - } - - if (!in.eof()) - { - assertChar('\n', in); - - /// Diffs for every file in form of - /// --- a/src/Storages/StorageReplicatedMergeTree.cpp - /// +++ b/src/Storages/StorageReplicatedMergeTree.cpp - /// @@ -1387,2 +1387 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) - /// - table_lock, entry.create_time, reserved_space, entry.deduplicate, - /// - entry.force_ttl); - /// + table_lock, entry.create_time, reserved_space, entry.deduplicate); - - std::string old_file_path; - std::string new_file_path; - FileChangeAndLineChanges * file_change_and_line_changes = nullptr; - LineChange line_change; - - while (!in.eof()) - { - if (checkString("@@ ", in)) - { - if (!file_change_and_line_changes) - { - auto file_name = new_file_path.empty() ? old_file_path : new_file_path; - auto it = file_changes.find(file_name); - if (file_changes.end() != it) - file_change_and_line_changes = &it->second; - } - - if (file_change_and_line_changes) - { - uint32_t old_lines = 1; - uint32_t new_lines = 1; - - assertChar('-', in); - readText(line_change.hunk_start_line_number_old, in); - if (checkChar(',', in)) - readText(old_lines, in); - - assertString(" +", in); - readText(line_change.hunk_start_line_number_new, in); - if (checkChar(',', in)) - readText(new_lines, in); - - if (line_change.hunk_start_line_number_new == 0) - line_change.hunk_start_line_number_new = 1; - - assertString(" @@", in); - if (checkChar(' ', in)) - readStringUntilNextLine(line_change.hunk_context, in); - else - assertChar('\n', in); - - line_change.hunk_lines_added = new_lines; - line_change.hunk_lines_deleted = old_lines; - - ++line_change.hunk_num; - line_change.line_number_old = line_change.hunk_start_line_number_old; - line_change.line_number_new = line_change.hunk_start_line_number_new; - - if (old_lines && new_lines) - { - ++commit.hunks_changed; - ++file_change_and_line_changes->file_change.hunks_changed; - } - else if (old_lines) - { - ++commit.hunks_removed; - ++file_change_and_line_changes->file_change.hunks_removed; - } - else if (new_lines) - { - ++commit.hunks_added; - ++file_change_and_line_changes->file_change.hunks_added; - } - } - } - else if (checkChar('-', in)) - { - if (checkString("-- ", in)) - { - if (checkString("a/", in)) - { - readStringUntilNextLine(old_file_path, in); - line_change = LineChange{}; - file_change_and_line_changes = nullptr; - } - else if (checkString("/dev/null", in)) - { - old_file_path.clear(); - assertChar('\n', in); - line_change = LineChange{}; - file_change_and_line_changes = nullptr; - } - else - skipUntilNextLine(in); /// Actually it can be the line in diff. Skip it for simplicity. - } - else - { - if (file_change_and_line_changes) - { - ++commit.lines_deleted; - ++file_change_and_line_changes->file_change.lines_deleted; - - line_change.sign = -1; - readStringUntilNextLine(line_change.line, in); - line_change.setLineInfo(line_change.line); - - file_change_and_line_changes->line_changes.push_back(line_change); - ++line_change.line_number_old; - } - } - } - else if (checkChar('+', in)) - { - if (checkString("++ ", in)) - { - if (checkString("b/", in)) - { - readStringUntilNextLine(new_file_path, in); - line_change = LineChange{}; - file_change_and_line_changes = nullptr; - } - else if (checkString("/dev/null", in)) - { - new_file_path.clear(); - assertChar('\n', in); - line_change = LineChange{}; - file_change_and_line_changes = nullptr; - } - else - skipUntilNextLine(in); /// Actually it can be the line in diff. Skip it for simplicity. - } - else - { - if (file_change_and_line_changes) - { - ++commit.lines_added; - ++file_change_and_line_changes->file_change.lines_added; - - line_change.sign = 1; - readStringUntilNextLine(line_change.line, in); - line_change.setLineInfo(line_change.line); - - file_change_and_line_changes->line_changes.push_back(line_change); - ++line_change.line_number_new; - } - } - } - else - { - skipUntilNextLine(in); - } - } - } - - if (options.diff_size_limit && commit_num != 0 && commit.lines_added + commit.lines_deleted > *options.diff_size_limit) - return; - - /// Calculate hash of diff and skip duplicates - if (options.skip_commits_with_duplicate_diffs) - { - SipHash hasher; - - for (auto & elem : file_changes) - { - hasher.update(elem.second.file_change.change_type); - hasher.update(elem.second.file_change.old_path.size()); - hasher.update(elem.second.file_change.old_path); - hasher.update(elem.second.file_change.path.size()); - hasher.update(elem.second.file_change.path); - - hasher.update(elem.second.line_changes.size()); - for (auto & line_change : elem.second.line_changes) - { - hasher.update(line_change.sign); - hasher.update(line_change.line_number_old); - hasher.update(line_change.line_number_new); - hasher.update(line_change.indent); - hasher.update(line_change.line.size()); - hasher.update(line_change.line); - } - } - - UInt128 hash_of_diff; - hasher.get128(hash_of_diff.low, hash_of_diff.high); - - if (!diff_hashes.insert(hash_of_diff).second) - return; - } - - /// Update snapshot and blame info - for (auto & elem : file_changes) { // std::cerr << elem.first << "\n"; @@ -928,47 +693,379 @@ void processCommit( } } } +} - /// Write the result - /// commits table +/** Deduplication of commits with identical diffs. + */ +using DiffHashes = std::unordered_set; + +UInt128 diffHash(const CommitDiff & file_changes) +{ + SipHash hasher; + + for (auto & elem : file_changes) { - auto & out = result.commits; + hasher.update(elem.second.file_change.change_type); + hasher.update(elem.second.file_change.old_path.size()); + hasher.update(elem.second.file_change.old_path); + hasher.update(elem.second.file_change.path.size()); + hasher.update(elem.second.file_change.path); - commit.writeTextWithoutNewline(out); - writeChar('\n', out); + hasher.update(elem.second.line_changes.size()); + for (auto & line_change : elem.second.line_changes) + { + hasher.update(line_change.sign); + hasher.update(line_change.line_number_old); + hasher.update(line_change.line_number_new); + hasher.update(line_change.indent); + hasher.update(line_change.line.size()); + hasher.update(line_change.line); + } } - for (const auto & elem : file_changes) + UInt128 hash_of_diff; + hasher.get128(hash_of_diff.low, hash_of_diff.high); + + return hash_of_diff; +} + + +/** File changes in form + * :100644 100644 b90fe6bb94 3ffe4c380f M src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp + * :100644 100644 828dedf6b5 828dedf6b5 R100 dbms/src/Functions/GeoUtils.h dbms/src/Functions/PolygonUtils.h + * according to the output of 'git show --raw' + */ +void processFileChanges( + ReadBuffer & in, + const Options & options, + Commit & commit, + CommitDiff & file_changes) +{ + while (checkChar(':', in)) { - const FileChange & file_change = elem.second.file_change; + FileChange file_change; - /// file_changes table + /// We don't care about file mode and content hashes. + for (size_t i = 0; i < 4; ++i) { - auto & out = result.file_changes; - - file_change.writeTextWithoutNewline(out); - writeChar('\t', out); - commit.writeTextWithoutNewline(out); - writeChar('\n', out); + skipUntilWhitespace(in); + skipWhitespaceIfAny(in); } - /// line_changes table - for (const auto & line_change : elem.second.line_changes) - { - auto & out = result.line_changes; + char change_type; + readChar(change_type, in); - line_change.writeTextWithoutNewline(out); - writeChar('\t', out); - file_change.writeTextWithoutNewline(out); - writeChar('\t', out); - commit.writeTextWithoutNewline(out); - writeChar('\n', out); + /// For rename and copy there is a number called "score". We ignore it. + int score; + + switch (change_type) + { + case 'A': + file_change.change_type = FileChangeType::Add; + ++commit.files_added; + break; + case 'D': + file_change.change_type = FileChangeType::Delete; + ++commit.files_deleted; + break; + case 'M': + file_change.change_type = FileChangeType::Modify; + ++commit.files_modified; + break; + case 'R': + file_change.change_type = FileChangeType::Rename; + ++commit.files_renamed; + readText(score, in); + break; + case 'C': + file_change.change_type = FileChangeType::Copy; + readText(score, in); + break; + case 'T': + file_change.change_type = FileChangeType::Type; + break; + default: + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected file change type: {}", change_type); + } + + skipWhitespaceIfAny(in); + + if (change_type == 'R' || change_type == 'C') + { + readText(file_change.old_path, in); + skipWhitespaceIfAny(in); + readText(file_change.path, in); + } + else + { + readText(file_change.path, in); + } + + file_change.file_extension = std::filesystem::path(file_change.path).extension(); + /// It gives us extension in form of '.cpp'. There is a reason for it but we remove initial dot for simplicity. + if (!file_change.file_extension.empty() && file_change.file_extension.front() == '.') + file_change.file_extension = file_change.file_extension.substr(1, std::string::npos); + + assertChar('\n', in); + + if (!(options.skip_paths && re2_st::RE2::PartialMatch(file_change.path, *options.skip_paths))) + { + file_changes.emplace( + file_change.path, + FileDiff(file_change)); } } } +/** Process the list of diffs for every file from the result of "git show". + * Caveats: + * - changes in binary files can be ignored; + * - if a line content begins with '+' or '-' it will be skipped + * it means that if you store diffs in repository and "git show" will display diff-of-diff for you, + * it won't be processed correctly; + * - we expect some specific format of the diff; but it may actually depend on git config; + * - non-ASCII file names are not processed correctly (they will not be found and will be ignored). + */ +void processDiffs( + ReadBuffer & in, + std::optional size_limit, + Commit & commit, + CommitDiff & file_changes) +{ + std::string old_file_path; + std::string new_file_path; + FileDiff * file_change_and_line_changes = nullptr; + LineChange line_change; + + /// Diffs for every file in form of + /// --- a/src/Storages/StorageReplicatedMergeTree.cpp + /// +++ b/src/Storages/StorageReplicatedMergeTree.cpp + /// @@ -1387,2 +1387 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) + /// - table_lock, entry.create_time, reserved_space, entry.deduplicate, + /// - entry.force_ttl); + /// + table_lock, entry.create_time, reserved_space, entry.deduplicate); + + size_t diff_size = 0; + while (!in.eof()) + { + if (checkString("@@ ", in)) + { + if (!file_change_and_line_changes) + { + auto file_name = new_file_path.empty() ? old_file_path : new_file_path; + auto it = file_changes.find(file_name); + if (file_changes.end() != it) + file_change_and_line_changes = &it->second; + } + + if (file_change_and_line_changes) + { + uint32_t old_lines = 1; + uint32_t new_lines = 1; + + assertChar('-', in); + readText(line_change.hunk_start_line_number_old, in); + if (checkChar(',', in)) + readText(old_lines, in); + + assertString(" +", in); + readText(line_change.hunk_start_line_number_new, in); + if (checkChar(',', in)) + readText(new_lines, in); + + /// This is needed to simplify the logic of updating snapshot: + /// When all lines are removed we can treat it as repeated removal of line with number 1. + if (line_change.hunk_start_line_number_new == 0) + line_change.hunk_start_line_number_new = 1; + + assertString(" @@", in); + if (checkChar(' ', in)) + readStringUntilNextLine(line_change.hunk_context, in); + else + assertChar('\n', in); + + line_change.hunk_lines_added = new_lines; + line_change.hunk_lines_deleted = old_lines; + + ++line_change.hunk_num; + line_change.line_number_old = line_change.hunk_start_line_number_old; + line_change.line_number_new = line_change.hunk_start_line_number_new; + + if (old_lines && new_lines) + { + ++commit.hunks_changed; + ++file_change_and_line_changes->file_change.hunks_changed; + } + else if (old_lines) + { + ++commit.hunks_removed; + ++file_change_and_line_changes->file_change.hunks_removed; + } + else if (new_lines) + { + ++commit.hunks_added; + ++file_change_and_line_changes->file_change.hunks_added; + } + } + } + else if (checkChar('-', in)) + { + if (checkString("-- ", in)) + { + if (checkString("a/", in)) + { + readStringUntilNextLine(old_file_path, in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else if (checkString("/dev/null", in)) + { + old_file_path.clear(); + assertChar('\n', in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else + skipUntilNextLine(in); /// Actually it can be the line in diff. Skip it for simplicity. + } + else + { + ++diff_size; + if (file_change_and_line_changes) + { + ++commit.lines_deleted; + ++file_change_and_line_changes->file_change.lines_deleted; + + line_change.sign = -1; + readStringUntilNextLine(line_change.line, in); + line_change.setLineInfo(line_change.line); + + file_change_and_line_changes->line_changes.push_back(line_change); + ++line_change.line_number_old; + } + } + } + else if (checkChar('+', in)) + { + if (checkString("++ ", in)) + { + if (checkString("b/", in)) + { + readStringUntilNextLine(new_file_path, in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else if (checkString("/dev/null", in)) + { + new_file_path.clear(); + assertChar('\n', in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else + skipUntilNextLine(in); /// Actually it can be the line in diff. Skip it for simplicity. + } + else + { + ++diff_size; + if (file_change_and_line_changes) + { + ++commit.lines_added; + ++file_change_and_line_changes->file_change.lines_added; + + line_change.sign = 1; + readStringUntilNextLine(line_change.line, in); + line_change.setLineInfo(line_change.line); + + file_change_and_line_changes->line_changes.push_back(line_change); + ++line_change.line_number_new; + } + } + } + else + { + /// Unknown lines are ignored. + skipUntilNextLine(in); + } + + if (size_limit && diff_size > *size_limit) + return; + } +} + + +/** Process the "git show" result for a single commit. Append the result to tables. + */ +void processCommit( + ReadBuffer & in, + const Options & options, + size_t commit_num, + size_t total_commits, + std::string hash, + Snapshot & snapshot, + DiffHashes & diff_hashes, + ResultWriter & result) +{ + Commit commit; + commit.hash = hash; + + time_t commit_time; + readText(commit_time, in); + commit.time = commit_time; + assertChar('\0', in); + readNullTerminated(commit.author, in); + std::string parent_hash; + readNullTerminated(parent_hash, in); + readNullTerminated(commit.message, in); + + if (options.skip_commits_with_messages && re2_st::RE2::PartialMatch(commit.message, *options.skip_commits_with_messages)) + return; + + std::string message_to_print = commit.message; + std::replace_if(message_to_print.begin(), message_to_print.end(), [](char c){ return std::iscntrl(c); }, ' '); + + std::cerr << fmt::format("{}% {} {} {}\n", + commit_num * 100 / total_commits, toString(commit.time), hash, message_to_print); + + if (options.skip_commits_without_parents && commit_num != 0 && parent_hash.empty()) + { + std::cerr << "Warning: skipping commit without parents\n"; + return; + } + + if (!in.eof()) + assertChar('\n', in); + + CommitDiff file_changes; + processFileChanges(in, options, commit, file_changes); + + if (!in.eof()) + { + assertChar('\n', in); + processDiffs(in, commit_num != 0 ? options.diff_size_limit : std::nullopt, commit, file_changes); + } + + /// Skip commits with too large diffs. + if (options.diff_size_limit && commit_num != 0 && commit.lines_added + commit.lines_deleted > *options.diff_size_limit) + return; + + /// Calculate hash of diff and skip duplicates + if (options.skip_commits_with_duplicate_diffs && !diff_hashes.insert(diffHash(file_changes)).second) + return; + + /// Update snapshot and blame info + updateSnapshot(snapshot, commit, file_changes); + + /// Write the result + result.appendCommit(commit, file_changes); +} + + +/** Runs child process and allows to read the result. + * Multiple processes can be run for parallel processing. + */ auto gitShow(const std::string & hash) { std::string command = fmt::format( @@ -979,9 +1076,11 @@ auto gitShow(const std::string & hash) } +/** Obtain the list of commits and process them. + */ void processLog(const Options & options) { - Result result; + ResultWriter result; std::string command = "git log --reverse --no-merges --pretty=%H"; fmt::print("{}\n", command); @@ -1019,7 +1118,7 @@ void processLog(const Options & options) for (size_t i = 0; i < num_commits; ++i) { - processCommit(show_commands[i % num_threads], options, i, num_commits, hashes[i], snapshot, diff_hashes, result); + processCommit(show_commands[i % num_threads]->out, options, i, num_commits, hashes[i], snapshot, diff_hashes, result); if (!options.stop_after_commit.empty() && hashes[i] == options.stop_after_commit) break; From 94d49e4197b443a6bced0ac0d137ad646c1c1946 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 03:18:15 +0300 Subject: [PATCH 077/298] Minor modifications --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 6e43853d6ba..2add6813008 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -120,7 +120,9 @@ CREATE TABLE git.line_changes commit_hunks_changed UInt32 ) ENGINE = MergeTree ORDER BY time; -Insert the data with the following commands: +Run the tool. + +Then insert the data with the following commands: clickhouse-client --query "INSERT INTO git.commits FORMAT TSV" < commits.tsv clickhouse-client --query "INSERT INTO git.file_changes FORMAT TSV" < file_changes.tsv From 47ca6211604c6fcb7b2c4e137d739ebff88da975 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 03:25:06 +0300 Subject: [PATCH 078/298] Minor modifications --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 2add6813008..875da3ba0ac 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -993,7 +993,12 @@ void processDiffs( } if (size_limit && diff_size > *size_limit) + { + /// Drain to avoid "broken pipe" error in child process. + while (!in.eof()) + in.ignore(in.available()); return; + } } } From 6e0afbecf4fd0ccd04e9dbb82bff6a507545e8d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 04:02:35 +0300 Subject: [PATCH 079/298] Minor modifications --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 875da3ba0ac..b5488b0d69a 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -138,6 +138,7 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_DATA; + extern const int CHILD_WAS_NOT_EXITED_NORMALLY; } @@ -994,9 +995,6 @@ void processDiffs( if (size_limit && diff_size > *size_limit) { - /// Drain to avoid "broken pipe" error in child process. - while (!in.eof()) - in.ignore(in.available()); return; } } @@ -1127,6 +1125,19 @@ void processLog(const Options & options) { processCommit(show_commands[i % num_threads]->out, options, i, num_commits, hashes[i], snapshot, diff_hashes, result); + try + { + show_commands[i % num_threads]->wait(); + } + catch (const Exception & e) + { + /// For broken pipe when we stopped reading prematurally. + if (e.code() == ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY) + std::cerr << getCurrentExceptionMessage(false) << "\n"; + else + throw; + } + if (!options.stop_after_commit.empty() && hashes[i] == options.stop_after_commit) break; From 69ce9e1f7020df985d7ea6ee450bf0d4b3438a0d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 05:36:54 +0300 Subject: [PATCH 080/298] More documentation --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index b5488b0d69a..d3b6f77d3d7 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -27,6 +27,51 @@ static constexpr auto documentation = R"( +A tool to extract information from Git repository for analytics. + +It dumps the data for the following tables: +- commits - commits with statistics; +- file_changes - files changed in every commit with the info about the change and statistics; +- line_changes - every changed line in every changed file in every commit with full info about the line and the information about previous change of this line. + +The largest and the most important table is "line_changes". + +Allows to answer questions like: +- list files with maximum number of authors; +- show me the oldest lines of code in the repository; +- show me the files with longest history; +- list favorite files for author; +- list largest files with lowest number of authors; +- at what weekday the code has highest chance to stay in repository; +- the distribution of code age across repository; +- files sorted by average code age; +- quickly show file with blame info (rough); +- commits and lines of code distribution by time; by weekday, by author; for specific subdirectories; +- show history for every subdirectory, file, line of file, the number of changes (lines and commits) across time; how the number of contributors was changed across time; +- list files with most modifications; +- list files that were rewritten most number of time or by most of authors; +- what is percentage of code removal by other authors, across authors; +- the matrix of authors that shows what authors tends to rewrite another authors code; +- what is the worst time to write code in sense that the code has highest chance to be rewritten; +- the average time before code will be rewritten and the median (half-life of code decay); +- comments/code percentage change in time / by author / by location; +- who tend to write more tests / cpp code / comments. + +The data is intended for analytical purposes. It can be imprecise by many reasons but it should be good enough for its purpose. + +The data is not intended to provide any conclusions for managers, it is especially counter-indicative for any kinds of "performance review". Instead you can spend multiple days looking at various interesting statistics. + +Run this tool inside your git repository. It will create .tsv files that can be loaded into ClickHouse (or into other DBMS if you dare). + +The tool can process large enough repositories in a reasonable time. +It has been tested on: +- ClickHouse: 31 seconds; 3 million rows; +- LLVM: 8 minues; 62 million rows; +- Linux - 12 minutes; 85 million rows; +- Chromium - 67 minutes; 343 million rows; +(the numbers as of Sep 2020) + + Prepare the database by executing the following queries: DROP DATABASE IF EXISTS git; From 1dc48f66710c5a93e5376320ea7cf3c4a18046d5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 05:39:08 +0300 Subject: [PATCH 081/298] Better help --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index d3b6f77d3d7..6ef82ac3b6b 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -13,6 +13,7 @@ #include +#include #include #include #include @@ -1199,7 +1200,7 @@ try { using namespace DB; - po::options_description desc("Allowed options"); + po::options_description desc("Allowed options", getTerminalWidth()); desc.add_options() ("help,h", "produce help message") ("skip-commits-without-parents", po::value()->default_value(true), From 1400bdbf83c9ebf6e63eeda73966b7e7c0210d80 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 06:11:35 +0300 Subject: [PATCH 082/298] Fix unit tests --- src/Common/ShellCommand.cpp | 23 +++++++++++++++---- utils/git-to-clickhouse/git-to-clickhouse.cpp | 13 ----------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 127f95fef06..bbb8801f190 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -57,7 +57,16 @@ ShellCommand::~ShellCommand() LOG_WARNING(getLogger(), "Cannot kill shell command pid {} errno '{}'", pid, errnoToString(retcode)); } else if (!wait_called) - tryWait(); + { + try + { + tryWait(); + } + catch (...) + { + tryLogCurrentException(getLogger()); + } + } } void ShellCommand::logCommand(const char * filename, char * const argv[]) @@ -74,7 +83,8 @@ void ShellCommand::logCommand(const char * filename, char * const argv[]) LOG_TRACE(ShellCommand::getLogger(), "Will start shell command '{}' with arguments {}", filename, args.str()); } -std::unique_ptr ShellCommand::executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only, bool terminate_in_destructor) +std::unique_ptr ShellCommand::executeImpl( + const char * filename, char * const argv[], bool pipe_stdin_only, bool terminate_in_destructor) { logCommand(filename, argv); @@ -130,7 +140,8 @@ std::unique_ptr ShellCommand::executeImpl(const char * filename, c _exit(int(ReturnCodes::CANNOT_EXEC)); } - std::unique_ptr res(new ShellCommand(pid, pipe_stdin.fds_rw[1], pipe_stdout.fds_rw[0], pipe_stderr.fds_rw[0], terminate_in_destructor)); + std::unique_ptr res(new ShellCommand( + pid, pipe_stdin.fds_rw[1], pipe_stdout.fds_rw[0], pipe_stderr.fds_rw[0], terminate_in_destructor)); LOG_TRACE(getLogger(), "Started shell command '{}' with pid {}", filename, pid); @@ -143,7 +154,8 @@ std::unique_ptr ShellCommand::executeImpl(const char * filename, c } -std::unique_ptr ShellCommand::execute(const std::string & command, bool pipe_stdin_only, bool terminate_in_destructor) +std::unique_ptr ShellCommand::execute( + const std::string & command, bool pipe_stdin_only, bool terminate_in_destructor) { /// Arguments in non-constant chunks of memory (as required for `execv`). /// Moreover, their copying must be done before calling `vfork`, so after `vfork` do a minimum of things. @@ -157,7 +169,8 @@ std::unique_ptr ShellCommand::execute(const std::string & command, } -std::unique_ptr ShellCommand::executeDirect(const std::string & path, const std::vector & arguments, bool terminate_in_destructor) +std::unique_ptr ShellCommand::executeDirect( + const std::string & path, const std::vector & arguments, bool terminate_in_destructor) { size_t argv_sum_size = path.size() + 1; for (const auto & arg : arguments) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 6ef82ac3b6b..a081efa3f47 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -1171,19 +1171,6 @@ void processLog(const Options & options) { processCommit(show_commands[i % num_threads]->out, options, i, num_commits, hashes[i], snapshot, diff_hashes, result); - try - { - show_commands[i % num_threads]->wait(); - } - catch (const Exception & e) - { - /// For broken pipe when we stopped reading prematurally. - if (e.code() == ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY) - std::cerr << getCurrentExceptionMessage(false) << "\n"; - else - throw; - } - if (!options.stop_after_commit.empty() && hashes[i] == options.stop_after_commit) break; From d18e7adbc03e4e7d7ee268e8f90a14e73be7b021 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 06:22:47 +0300 Subject: [PATCH 083/298] Add git-import as a tool --- programs/CMakeLists.txt | 18 ++++++++++++++---- programs/config_tools.h.in | 1 + programs/git-import/CMakeLists.txt | 10 ++++++++++ programs/git-import/clickhouse-git-import.cpp | 2 ++ .../git-import/git-import.cpp | 4 ++-- programs/install/Install.cpp | 1 + programs/main.cpp | 6 ++++++ utils/CMakeLists.txt | 1 - utils/git-to-clickhouse/CMakeLists.txt | 2 -- 9 files changed, 36 insertions(+), 9 deletions(-) create mode 100644 programs/git-import/CMakeLists.txt create mode 100644 programs/git-import/clickhouse-git-import.cpp rename utils/git-to-clickhouse/git-to-clickhouse.cpp => programs/git-import/git-import.cpp (99%) delete mode 100644 utils/git-to-clickhouse/CMakeLists.txt diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 89220251cda..ae4a72ef62a 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -16,6 +16,7 @@ option (ENABLE_CLICKHOUSE_COMPRESSOR "Enable clickhouse-compressor" ${ENABLE_CLI option (ENABLE_CLICKHOUSE_COPIER "Enable clickhouse-copier" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_FORMAT "Enable clickhouse-format" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_OBFUSCATOR "Enable clickhouse-obfuscator" ${ENABLE_CLICKHOUSE_ALL}) +option (ENABLE_CLICKHOUSE_GIT_IMPORT "Enable clickhouse-git-import" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_ODBC_BRIDGE "Enable clickhouse-odbc-bridge" ${ENABLE_CLICKHOUSE_ALL}) if (CLICKHOUSE_SPLIT_BINARY) @@ -91,21 +92,22 @@ add_subdirectory (copier) add_subdirectory (format) add_subdirectory (obfuscator) add_subdirectory (install) +add_subdirectory (git-import) if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) add_subdirectory (odbc-bridge) endif () if (CLICKHOUSE_ONE_SHARED) - add_library(clickhouse-lib SHARED ${CLICKHOUSE_SERVER_SOURCES} ${CLICKHOUSE_CLIENT_SOURCES} ${CLICKHOUSE_LOCAL_SOURCES} ${CLICKHOUSE_BENCHMARK_SOURCES} ${CLICKHOUSE_COPIER_SOURCES} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_SOURCES} ${CLICKHOUSE_COMPRESSOR_SOURCES} ${CLICKHOUSE_FORMAT_SOURCES} ${CLICKHOUSE_OBFUSCATOR_SOURCES} ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) - target_link_libraries(clickhouse-lib ${CLICKHOUSE_SERVER_LINK} ${CLICKHOUSE_CLIENT_LINK} ${CLICKHOUSE_LOCAL_LINK} ${CLICKHOUSE_BENCHMARK_LINK} ${CLICKHOUSE_COPIER_LINK} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_LINK} ${CLICKHOUSE_COMPRESSOR_LINK} ${CLICKHOUSE_FORMAT_LINK} ${CLICKHOUSE_OBFUSCATOR_LINK} ${CLICKHOUSE_ODBC_BRIDGE_LINK}) - target_include_directories(clickhouse-lib ${CLICKHOUSE_SERVER_INCLUDE} ${CLICKHOUSE_CLIENT_INCLUDE} ${CLICKHOUSE_LOCAL_INCLUDE} ${CLICKHOUSE_BENCHMARK_INCLUDE} ${CLICKHOUSE_COPIER_INCLUDE} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_INCLUDE} ${CLICKHOUSE_COMPRESSOR_INCLUDE} ${CLICKHOUSE_FORMAT_INCLUDE} ${CLICKHOUSE_OBFUSCATOR_INCLUDE} ${CLICKHOUSE_ODBC_BRIDGE_INCLUDE}) + add_library(clickhouse-lib SHARED ${CLICKHOUSE_SERVER_SOURCES} ${CLICKHOUSE_CLIENT_SOURCES} ${CLICKHOUSE_LOCAL_SOURCES} ${CLICKHOUSE_BENCHMARK_SOURCES} ${CLICKHOUSE_COPIER_SOURCES} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_SOURCES} ${CLICKHOUSE_COMPRESSOR_SOURCES} ${CLICKHOUSE_FORMAT_SOURCES} ${CLICKHOUSE_OBFUSCATOR_SOURCES} ${CLICKHOUSE_GIT_IMPORT_SOURCES} ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) + target_link_libraries(clickhouse-lib ${CLICKHOUSE_SERVER_LINK} ${CLICKHOUSE_CLIENT_LINK} ${CLICKHOUSE_LOCAL_LINK} ${CLICKHOUSE_BENCHMARK_LINK} ${CLICKHOUSE_COPIER_LINK} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_LINK} ${CLICKHOUSE_COMPRESSOR_LINK} ${CLICKHOUSE_FORMAT_LINK} ${CLICKHOUSE_OBFUSCATOR_LINK} ${CLICKHOUSE_GIT_IMPORT_LINK} ${CLICKHOUSE_ODBC_BRIDGE_LINK}) + target_include_directories(clickhouse-lib ${CLICKHOUSE_SERVER_INCLUDE} ${CLICKHOUSE_CLIENT_INCLUDE} ${CLICKHOUSE_LOCAL_INCLUDE} ${CLICKHOUSE_BENCHMARK_INCLUDE} ${CLICKHOUSE_COPIER_INCLUDE} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_INCLUDE} ${CLICKHOUSE_COMPRESSOR_INCLUDE} ${CLICKHOUSE_FORMAT_INCLUDE} ${CLICKHOUSE_OBFUSCATOR_INCLUDE} ${CLICKHOUSE_GIT_IMPORT_INCLUDE} ${CLICKHOUSE_ODBC_BRIDGE_INCLUDE}) set_target_properties(clickhouse-lib PROPERTIES SOVERSION ${VERSION_MAJOR}.${VERSION_MINOR} VERSION ${VERSION_SO} OUTPUT_NAME clickhouse DEBUG_POSTFIX "") install (TARGETS clickhouse-lib LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} COMPONENT clickhouse) endif() if (CLICKHOUSE_SPLIT_BINARY) - set (CLICKHOUSE_ALL_TARGETS clickhouse-server clickhouse-client clickhouse-local clickhouse-benchmark clickhouse-extract-from-config clickhouse-compressor clickhouse-format clickhouse-obfuscator clickhouse-copier) + set (CLICKHOUSE_ALL_TARGETS clickhouse-server clickhouse-client clickhouse-local clickhouse-benchmark clickhouse-extract-from-config clickhouse-compressor clickhouse-format clickhouse-obfuscator clickhouse-git-import clickhouse-copier) if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-odbc-bridge) @@ -149,6 +151,9 @@ else () if (ENABLE_CLICKHOUSE_OBFUSCATOR) clickhouse_target_link_split_lib(clickhouse obfuscator) endif () + if (ENABLE_CLICKHOUSE_GIT_IMPORT) + clickhouse_target_link_split_lib(clickhouse git-import) + endif () if (ENABLE_CLICKHOUSE_INSTALL) clickhouse_target_link_split_lib(clickhouse install) endif () @@ -199,6 +204,11 @@ else () install (FILES ${CMAKE_CURRENT_BINARY_DIR}/clickhouse-obfuscator DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) list(APPEND CLICKHOUSE_BUNDLE clickhouse-obfuscator) endif () + if (ENABLE_CLICKHOUSE_GIT_IMPORT) + add_custom_target (clickhouse-git-import ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-git-import DEPENDS clickhouse) + install (FILES ${CMAKE_CURRENT_BINARY_DIR}/clickhouse-git-import DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) + list(APPEND CLICKHOUSE_BUNDLE clickhouse-git-import) + endif () if(ENABLE_CLICKHOUSE_ODBC_BRIDGE) list(APPEND CLICKHOUSE_BUNDLE clickhouse-odbc-bridge) endif() diff --git a/programs/config_tools.h.in b/programs/config_tools.h.in index 11386aca60e..7cb5a6d883a 100644 --- a/programs/config_tools.h.in +++ b/programs/config_tools.h.in @@ -12,5 +12,6 @@ #cmakedefine01 ENABLE_CLICKHOUSE_COMPRESSOR #cmakedefine01 ENABLE_CLICKHOUSE_FORMAT #cmakedefine01 ENABLE_CLICKHOUSE_OBFUSCATOR +#cmakedefine01 ENABLE_CLICKHOUSE_GIT_IMPORT #cmakedefine01 ENABLE_CLICKHOUSE_INSTALL #cmakedefine01 ENABLE_CLICKHOUSE_ODBC_BRIDGE diff --git a/programs/git-import/CMakeLists.txt b/programs/git-import/CMakeLists.txt new file mode 100644 index 00000000000..279bb35a272 --- /dev/null +++ b/programs/git-import/CMakeLists.txt @@ -0,0 +1,10 @@ +set (CLICKHOUSE_GIT_IMPORT_SOURCES git-import.cpp) + +set (CLICKHOUSE_GIT_IMPORT_LINK + PRIVATE + boost::program_options + dbms +) + +clickhouse_program_add(git-import) + diff --git a/programs/git-import/clickhouse-git-import.cpp b/programs/git-import/clickhouse-git-import.cpp new file mode 100644 index 00000000000..cfa06306604 --- /dev/null +++ b/programs/git-import/clickhouse-git-import.cpp @@ -0,0 +1,2 @@ +int mainEntryClickHouseGitImport(int argc, char ** argv); +int main(int argc_, char ** argv_) { return mainEntryClickHouseGitImport(argc_, argv_); } diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/programs/git-import/git-import.cpp similarity index 99% rename from utils/git-to-clickhouse/git-to-clickhouse.cpp rename to programs/git-import/git-import.cpp index a081efa3f47..f1ed4d28c6e 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/programs/git-import/git-import.cpp @@ -1182,7 +1182,7 @@ void processLog(const Options & options) } -int main(int argc, char ** argv) +int mainEntryClickHouseGitImport(int argc, char ** argv) try { using namespace DB; @@ -1219,7 +1219,7 @@ try << "Usage: " << argv[0] << '\n' << desc << '\n' << "\nExample:\n" - << "\n./git-to-clickhouse --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/' --skip-commits-with-messages '^Merge branch '\n"; + << "\nclickhouse git-import --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/' --skip-commits-with-messages '^Merge branch '\n"; return 1; } diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 7b7ab149447..bd60fbb63ba 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -205,6 +205,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) "clickhouse-benchmark", "clickhouse-copier", "clickhouse-obfuscator", + "clickhouse-git-import", "clickhouse-compressor", "clickhouse-format", "clickhouse-extract-from-config" diff --git a/programs/main.cpp b/programs/main.cpp index 3df5f9f683b..b91bd732f21 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -46,6 +46,9 @@ int mainEntryClickHouseClusterCopier(int argc, char ** argv); #if ENABLE_CLICKHOUSE_OBFUSCATOR int mainEntryClickHouseObfuscator(int argc, char ** argv); #endif +#if ENABLE_CLICKHOUSE_GIT_IMPORT +int mainEntryClickHouseGitImport(int argc, char ** argv); +#endif #if ENABLE_CLICKHOUSE_INSTALL int mainEntryClickHouseInstall(int argc, char ** argv); int mainEntryClickHouseStart(int argc, char ** argv); @@ -91,6 +94,9 @@ std::pair clickhouse_applications[] = #if ENABLE_CLICKHOUSE_OBFUSCATOR {"obfuscator", mainEntryClickHouseObfuscator}, #endif +#if ENABLE_CLICKHOUSE_GIT_IMPORT + {"git-import", mainEntryClickHouseGitImport}, +#endif #if ENABLE_CLICKHOUSE_INSTALL {"install", mainEntryClickHouseInstall}, {"start", mainEntryClickHouseStart}, diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 93490fba565..b4408a298c3 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -30,7 +30,6 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) add_subdirectory (checksum-for-compressed-block) add_subdirectory (db-generator) add_subdirectory (wal-dump) - add_subdirectory (git-to-clickhouse) endif () if (ENABLE_CODE_QUALITY) diff --git a/utils/git-to-clickhouse/CMakeLists.txt b/utils/git-to-clickhouse/CMakeLists.txt deleted file mode 100644 index 0e46b68d471..00000000000 --- a/utils/git-to-clickhouse/CMakeLists.txt +++ /dev/null @@ -1,2 +0,0 @@ -add_executable (git-to-clickhouse git-to-clickhouse.cpp) -target_link_libraries(git-to-clickhouse PRIVATE dbms boost::program_options) From ee54971c3d26ca1219da4909bd30f44bee77fd97 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 07:11:03 +0300 Subject: [PATCH 084/298] Fix build --- programs/git-import/git-import.cpp | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/programs/git-import/git-import.cpp b/programs/git-import/git-import.cpp index f1ed4d28c6e..d314969a1a8 100644 --- a/programs/git-import/git-import.cpp +++ b/programs/git-import/git-import.cpp @@ -184,7 +184,6 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_DATA; - extern const int CHILD_WAS_NOT_EXITED_NORMALLY; } @@ -419,7 +418,7 @@ using LineChanges = std::vector; struct FileDiff { - FileDiff(FileChange file_change_) : file_change(file_change_) {} + explicit FileDiff(FileChange file_change_) : file_change(file_change_) {} FileChange file_change; LineChanges line_changes; @@ -546,7 +545,7 @@ struct Options std::optional diff_size_limit; std::string stop_after_commit; - Options(const po::variables_map & options) + explicit Options(const po::variables_map & options) { skip_commits_without_parents = options["skip-commits-without-parents"].as(); skip_commits_with_duplicate_diffs = options["skip-commits-with-duplicate-diffs"].as(); @@ -753,7 +752,7 @@ UInt128 diffHash(const CommitDiff & file_changes) { SipHash hasher; - for (auto & elem : file_changes) + for (const auto & elem : file_changes) { hasher.update(elem.second.file_change.change_type); hasher.update(elem.second.file_change.old_path.size()); @@ -762,7 +761,7 @@ UInt128 diffHash(const CommitDiff & file_changes) hasher.update(elem.second.file_change.path); hasher.update(elem.second.line_changes.size()); - for (auto & line_change : elem.second.line_changes) + for (const auto & line_change : elem.second.line_changes) { hasher.update(line_change.sign); hasher.update(line_change.line_number_old); @@ -1159,6 +1158,8 @@ void processLog(const Options & options) /// Will run multiple processes in parallel size_t num_threads = options.threads; + if (num_threads == 0) + throw Exception("num-threads cannot be zero", ErrorCodes::INCORRECT_DATA); std::vector> show_commands(num_threads); for (size_t i = 0; i < num_commits && i < num_threads; ++i) @@ -1223,7 +1224,7 @@ try return 1; } - processLog(options); + processLog(Options(options)); return 0; } catch (...) From 04a69650068c3ff5967f3639c55082dbd34017cf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 09:40:01 +0300 Subject: [PATCH 085/298] Fix error with executable dictionary source --- docker/test/fasttest/run.sh | 3 +- docker/test/stateless/run.sh | 1 + docker/test/stateless_unbundled/run.sh | 1 + docker/test/stateless_with_coverage/run.sh | 1 + src/Common/tests/CMakeLists.txt | 3 + src/Common/tests/shell_command_inout.cpp | 47 +++++++ .../ExecutableDictionarySource.cpp | 119 ++++++++++-------- src/Dictionaries/ExecutableDictionarySource.h | 1 + tests/config/executable_dictionary.xml | 108 ++++++++++++++++ .../01474_executable_dictionary.reference | 3 + .../01474_executable_dictionary.sql | 3 + 11 files changed, 240 insertions(+), 50 deletions(-) create mode 100644 src/Common/tests/shell_command_inout.cpp create mode 100644 tests/config/executable_dictionary.xml create mode 100644 tests/queries/0_stateless/01474_executable_dictionary.reference create mode 100644 tests/queries/0_stateless/01474_executable_dictionary.sql diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 1f8d612a125..9f5a9b05219 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -11,7 +11,7 @@ stage=${stage:-} # A variable to pass additional flags to CMake. # Here we explicitly default it to nothing so that bash doesn't complain about -# it being undefined. Also read it as array so that we can pass an empty list +# it being undefined. Also read it as array so that we can pass an empty list # of additional variable to cmake properly, and it doesn't generate an extra # empty parameter. read -ra FASTTEST_CMAKE_FLAGS <<< "${FASTTEST_CMAKE_FLAGS:-}" @@ -128,6 +128,7 @@ ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-se ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/ #ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/ diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 2ff15ca9c6a..4a9ad891883 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -24,6 +24,7 @@ ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-se ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/ diff --git a/docker/test/stateless_unbundled/run.sh b/docker/test/stateless_unbundled/run.sh index 2ff15ca9c6a..4a9ad891883 100755 --- a/docker/test/stateless_unbundled/run.sh +++ b/docker/test/stateless_unbundled/run.sh @@ -24,6 +24,7 @@ ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-se ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/ diff --git a/docker/test/stateless_with_coverage/run.sh b/docker/test/stateless_with_coverage/run.sh index 64317ee62fd..c3ccb18659b 100755 --- a/docker/test/stateless_with_coverage/run.sh +++ b/docker/test/stateless_with_coverage/run.sh @@ -57,6 +57,7 @@ ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-se ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/ diff --git a/src/Common/tests/CMakeLists.txt b/src/Common/tests/CMakeLists.txt index f6c232cdd22..8de9424e044 100644 --- a/src/Common/tests/CMakeLists.txt +++ b/src/Common/tests/CMakeLists.txt @@ -84,3 +84,6 @@ target_link_libraries (procfs_metrics_provider_perf PRIVATE clickhouse_common_io add_executable (average average.cpp) target_link_libraries (average PRIVATE clickhouse_common_io) + +add_executable (shell_command_inout shell_command_inout.cpp) +target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io) diff --git a/src/Common/tests/shell_command_inout.cpp b/src/Common/tests/shell_command_inout.cpp new file mode 100644 index 00000000000..615700cd042 --- /dev/null +++ b/src/Common/tests/shell_command_inout.cpp @@ -0,0 +1,47 @@ +#include + +#include +#include + +#include +#include +#include + +/** This example shows how we can proxy stdin to ShellCommand and obtain stdout in streaming fashion. */ + +int main(int argc, char ** argv) +try +{ + using namespace DB; + + if (argc < 2) + { + std::cerr << "Usage: shell_command_inout 'command...' < in > out\n"; + return 1; + } + + auto command = ShellCommand::execute(argv[1]); + + ReadBufferFromFileDescriptor in(STDIN_FILENO); + WriteBufferFromFileDescriptor out(STDOUT_FILENO); + WriteBufferFromFileDescriptor err(STDERR_FILENO); + + /// Background thread sends data and foreground thread receives result. + + std::thread thread([&] + { + copyData(in, command->in); + command->in.close(); + }); + + copyData(command->out, out); + copyData(command->err, err); + + thread.join(); + return 0; +} +catch (...) +{ + std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; + throw; +} diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 918cf0732ab..74aab610e0d 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -1,12 +1,13 @@ #include "ExecutableDictionarySource.h" -#include -#include +#include #include #include #include #include #include +#include +#include #include #include #include @@ -16,6 +17,7 @@ #include "DictionaryStructure.h" #include "registerDictionaries.h" + namespace DB { static const UInt64 max_block_size = 8192; @@ -31,15 +33,23 @@ namespace /// Owns ShellCommand and calls wait for it. class ShellCommandOwningBlockInputStream : public OwningBlockInputStream { + private: + Poco::Logger * log; public: - ShellCommandOwningBlockInputStream(const BlockInputStreamPtr & impl, std::unique_ptr own_) - : OwningBlockInputStream(std::move(impl), std::move(own_)) + ShellCommandOwningBlockInputStream(Poco::Logger * log_, const BlockInputStreamPtr & impl, std::unique_ptr command_) + : OwningBlockInputStream(std::move(impl), std::move(command_)), log(log_) { } void readSuffix() override { OwningBlockInputStream::readSuffix(); + + std::string err; + readStringUntilEOF(err, own->err); + if (!err.empty()) + LOG_ERROR(log, "Having stderr: {}", err); + own->wait(); } }; @@ -80,7 +90,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadAll() LOG_TRACE(log, "loadAll {}", toString()); auto process = ShellCommand::execute(command); auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size); - return std::make_shared(input_stream, std::move(process)); + return std::make_shared(log, input_stream, std::move(process)); } BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll() @@ -95,67 +105,77 @@ BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll() LOG_TRACE(log, "loadUpdatedAll {}", command_with_update_field); auto process = ShellCommand::execute(command_with_update_field); auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size); - return std::make_shared(input_stream, std::move(process)); + return std::make_shared(log, input_stream, std::move(process)); } namespace { - /** A stream, that also runs and waits for background thread - * (that will feed data into pipe to be read from the other side of the pipe). + /** A stream, that runs child process and sends data to its stdin in background thread, + * and receives data from its stdout. */ class BlockInputStreamWithBackgroundThread final : public IBlockInputStream { public: BlockInputStreamWithBackgroundThread( - const BlockInputStreamPtr & stream_, std::unique_ptr && command_, std::packaged_task && task_) - : stream{stream_}, command{std::move(command_)}, task(std::move(task_)), thread([this] { - task(); - command->in.close(); - }) + const Context & context, + const std::string & format, + const Block & sample_block, + const std::string & command_str, + Poco::Logger * log_, + std::function && send_data_) + : log(log_), + command(ShellCommand::execute(command_str)), + send_data(std::move(send_data_)), + thread([this] { send_data(command->in); }) { - children.push_back(stream); + //WriteBufferFromFileDescriptor err(STDERR_FILENO); + //copyData(command->out, err); + //err.next(); + //thread.join(); + stream = context.getInputFormat(format, command->out, sample_block, max_block_size); } ~BlockInputStreamWithBackgroundThread() override { if (thread.joinable()) - { - try - { - readSuffix(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } + thread.join(); } - Block getHeader() const override { return stream->getHeader(); } + Block getHeader() const override + { + return stream->getHeader(); + } private: - Block readImpl() override { return stream->read(); } + Block readImpl() override + { + return stream->read(); + } + + void readPrefix() override + { + stream->readPrefix(); + } void readSuffix() override { - IBlockInputStream::readSuffix(); - if (!wait_called) - { - wait_called = true; - command->wait(); - } - thread.join(); - /// To rethrow an exception, if any. - task.get_future().get(); + stream->readSuffix(); + + std::string err; + readStringUntilEOF(err, command->err); + if (!err.empty()) + LOG_ERROR(log, "Having stderr: {}", err); + + command->wait(); } String getName() const override { return "WithBackgroundThread"; } + Poco::Logger * log; BlockInputStreamPtr stream; std::unique_ptr command; - std::packaged_task task; - ThreadFromGlobalPool thread; - bool wait_called = false; + std::function send_data; + mutable ThreadFromGlobalPool thread; }; } @@ -164,28 +184,29 @@ namespace BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector & ids) { LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size()); - auto process = ShellCommand::execute(command); - - auto output_stream = context.getOutputFormat(format, process->in, sample_block); - auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size); return std::make_shared( - input_stream, std::move(process), std::packaged_task([output_stream, &ids]() mutable { formatIDs(output_stream, ids); })); + context, format, sample_block, command, log, + [&ids, this](WriteBufferFromFile & out) mutable + { + auto output_stream = context.getOutputFormat(format, out, sample_block); + formatIDs(output_stream, ids); + out.close(); + }); } BlockInputStreamPtr ExecutableDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size()); - auto process = ShellCommand::execute(command); - - auto output_stream = context.getOutputFormat(format, process->in, sample_block); - auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size); return std::make_shared( - input_stream, std::move(process), std::packaged_task([output_stream, key_columns, &requested_rows, this]() mutable + context, format, sample_block, command, log, + [key_columns, &requested_rows, this](WriteBufferFromFile & out) mutable { + auto output_stream = context.getOutputFormat(format, out, sample_block); formatKeys(dict_struct, output_stream, key_columns, requested_rows); - })); + out.close(); + }); } bool ExecutableDictionarySource::isModified() const diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index f28d71ca5e3..b2aabf26323 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -14,6 +14,7 @@ namespace DB /// Allows loading dictionaries from executable class ExecutableDictionarySource final : public IDictionarySource { + friend class BlockInputStreamWithBackgroundThread; public: ExecutableDictionarySource( const DictionaryStructure & dict_struct_, diff --git a/tests/config/executable_dictionary.xml b/tests/config/executable_dictionary.xml new file mode 100644 index 00000000000..50df32e2ec6 --- /dev/null +++ b/tests/config/executable_dictionary.xml @@ -0,0 +1,108 @@ + + + + executable_complex + + + JSONEachRow + cd /; clickhouse-local --input-format JSONEachRow --output-format JSONEachRow --structure 'x UInt64, y UInt64' --query "SELECT x, y, x + y AS a, x * y AS b FROM table" + + + 0 + + + 1000 + + + + + + x + UInt64 + + + y + UInt64 + + + + a + UInt64 + 0 + + + b + UInt64 + 0 + + + + + + executable_simple + + + JSONEachRow + cd /; clickhouse-local --input-format JSONEachRow --output-format JSONEachRow --structure 'x UInt64' --query "SELECT x, x + x AS a, x * x AS b FROM table" + + + 0 + + + 1000 + + + + + x + + + a + UInt64 + 0 + + + b + UInt64 + 0 + + + + + + executable_complex_direct + + + JSONEachRow + cd /; clickhouse-local --input-format JSONEachRow --output-format JSONEachRow --structure 'x UInt64, y UInt64' --query "SELECT x, y, x + y AS a, x * y AS b FROM table" + + + 0 + + + + + + + x + UInt64 + + + y + UInt64 + + + + a + UInt64 + 0 + + + b + UInt64 + 0 + + + + + diff --git a/tests/queries/0_stateless/01474_executable_dictionary.reference b/tests/queries/0_stateless/01474_executable_dictionary.reference new file mode 100644 index 00000000000..4d0994b08c3 --- /dev/null +++ b/tests/queries/0_stateless/01474_executable_dictionary.reference @@ -0,0 +1,3 @@ +999999 1999998 999998000001 +999999 1999998 999998000001 +999999 1999998 999998000001 diff --git a/tests/queries/0_stateless/01474_executable_dictionary.sql b/tests/queries/0_stateless/01474_executable_dictionary.sql new file mode 100644 index 00000000000..727cf47f79f --- /dev/null +++ b/tests/queries/0_stateless/01474_executable_dictionary.sql @@ -0,0 +1,3 @@ +SELECT number, dictGet('executable_complex', 'a', (number, number)) AS a, dictGet('executable_complex', 'b', (number, number)) AS b FROM numbers(1000000) WHERE number = 999999; +SELECT number, dictGet('executable_complex_direct', 'a', (number, number)) AS a, dictGet('executable_complex_direct', 'b', (number, number)) AS b FROM numbers(1000000) WHERE number = 999999; +SELECT number, dictGet('executable_simple', 'a', number) AS a, dictGet('executable_simple', 'b', number) AS b FROM numbers(1000000) WHERE number = 999999; From 8dd98f74a5a5cd5c5cba804f96b3349c5f9a2e25 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 09:43:28 +0300 Subject: [PATCH 086/298] Remove debug output --- src/Dictionaries/ExecutableDictionarySource.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 74aab610e0d..0709be2420a 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -128,10 +128,6 @@ namespace send_data(std::move(send_data_)), thread([this] { send_data(command->in); }) { - //WriteBufferFromFileDescriptor err(STDERR_FILENO); - //copyData(command->out, err); - //err.next(); - //thread.join(); stream = context.getInputFormat(format, command->out, sample_block, max_block_size); } From 1f0d2be17adbc292fef91d4b7703d654871cb815 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 7 Sep 2020 09:44:32 +0300 Subject: [PATCH 087/298] Update ExecutableDictionarySource.h --- src/Dictionaries/ExecutableDictionarySource.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index b2aabf26323..f28d71ca5e3 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -14,7 +14,6 @@ namespace DB /// Allows loading dictionaries from executable class ExecutableDictionarySource final : public IDictionarySource { - friend class BlockInputStreamWithBackgroundThread; public: ExecutableDictionarySource( const DictionaryStructure & dict_struct_, From 3903794386c32d1894fa266d760eed07419a1d54 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 7 Sep 2020 09:45:01 +0300 Subject: [PATCH 088/298] Update ExecutableDictionarySource.cpp --- src/Dictionaries/ExecutableDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 0709be2420a..cc250727261 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -171,7 +171,7 @@ namespace BlockInputStreamPtr stream; std::unique_ptr command; std::function send_data; - mutable ThreadFromGlobalPool thread; + ThreadFromGlobalPool thread; }; } From d666d4c4497e90901e47b1c09c3f730f90f4c7c4 Mon Sep 17 00:00:00 2001 From: zhangshengyu Date: Mon, 7 Sep 2020 15:00:47 +0800 Subject: [PATCH 089/298] fix zh translate --- docs/zh/guides/apply-catboost-model.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/zh/guides/apply-catboost-model.md b/docs/zh/guides/apply-catboost-model.md index be21c372307..3657a947ad2 100644 --- a/docs/zh/guides/apply-catboost-model.md +++ b/docs/zh/guides/apply-catboost-model.md @@ -15,7 +15,7 @@ toc_title: "\u5E94\u7528CatBoost\u6A21\u578B" 1. [创建表](#create-table). 2. [将数据插入到表中](#insert-data-to-table). -3. [碌莽禄into拢Integrate010-68520682\](#integrate-catboost-into-clickhouse) (可选步骤)。 +3. [将CatBoost集成到ClickHouse中](#integrate-catboost-into-clickhouse) (可选步骤)。 4. [从SQL运行模型推理](#run-model-inference). 有关训练CatBoost模型的详细信息,请参阅 [培训和应用模型](https://catboost.ai/docs/features/training.html#training). @@ -119,12 +119,12 @@ FROM amazon_train +-------+ ``` -## 3. 碌莽禄into拢Integrate010-68520682\ {#integrate-catboost-into-clickhouse} +## 3. 将CatBoost集成到ClickHouse中 {#integrate-catboost-into-clickhouse} !!! note "注" **可选步骤。** Docker映像包含运行CatBoost和ClickHouse所需的所有内容。 -碌莽禄to拢integrate010-68520682\: +CatBoost集成到ClickHouse步骤: **1.** 构建评估库。 From 3942cc615f03ecb8e5b9e7437fdc5c57613c245d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 7 Sep 2020 10:09:42 +0300 Subject: [PATCH 090/298] Update git-import.cpp --- programs/git-import/git-import.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/programs/git-import/git-import.cpp b/programs/git-import/git-import.cpp index d314969a1a8..45bc47348e7 100644 --- a/programs/git-import/git-import.cpp +++ b/programs/git-import/git-import.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include @@ -539,8 +539,8 @@ struct Options bool skip_commits_without_parents = true; bool skip_commits_with_duplicate_diffs = true; size_t threads = 1; - std::optional skip_paths; - std::optional skip_commits_with_messages; + std::optional skip_paths; + std::optional skip_commits_with_messages; std::unordered_set skip_commits; std::optional diff_size_limit; std::string stop_after_commit; @@ -857,7 +857,7 @@ void processFileChanges( assertChar('\n', in); - if (!(options.skip_paths && re2_st::RE2::PartialMatch(file_change.path, *options.skip_paths))) + if (!(options.skip_paths && re2::RE2::PartialMatch(file_change.path, *options.skip_paths))) { file_changes.emplace( file_change.path, @@ -1070,7 +1070,7 @@ void processCommit( readNullTerminated(parent_hash, in); readNullTerminated(commit.message, in); - if (options.skip_commits_with_messages && re2_st::RE2::PartialMatch(commit.message, *options.skip_commits_with_messages)) + if (options.skip_commits_with_messages && re2::RE2::PartialMatch(commit.message, *options.skip_commits_with_messages)) return; std::string message_to_print = commit.message; From ba70de63f83431dcb116f0be24be5c5ef0822d23 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 7 Sep 2020 10:28:04 +0300 Subject: [PATCH 091/298] Fix diff --- .../MergeTree/MergeTreeDataWriter.cpp | 5 +-- .../01465_ttl_recompression.reference | 27 +++++++++----- .../0_stateless/01465_ttl_recompression.sql | 37 ++++++++++++++++++- 3 files changed, 55 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 607535225a2..1d7bf545009 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -235,9 +235,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa for (const auto & ttl_entry : move_ttl_entries) updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); - time_t current_time = time(nullptr); NamesAndTypesList columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); - ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, current_time); + ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr)); VolumePtr volume = data.getStoragePolicy()->getVolume(0); auto new_data_part = data.createPart( @@ -308,7 +307,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa /// This effectively chooses minimal compression method: /// either default lz4 or compression method with zero thresholds on absolute and relative part size. - auto compression_codec = data.getCompressionCodecForPart(0, new_data_part->ttl_infos, current_time); + auto compression_codec = data.global_context.chooseCompressionCodec(0, 0); const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); diff --git a/tests/queries/0_stateless/01465_ttl_recompression.reference b/tests/queries/0_stateless/01465_ttl_recompression.reference index c03c003d5b8..40d7ed8896b 100644 --- a/tests/queries/0_stateless/01465_ttl_recompression.reference +++ b/tests/queries/0_stateless/01465_ttl_recompression.reference @@ -1,12 +1,21 @@ -CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt64,\n `value` String\n)\nENGINE = MergeTree()\nPARTITION BY key\nORDER BY tuple()\nTTL dt + toIntervalMonth(1) RECOMPRESS CODEC(ZSTD(17)), dt + toIntervalYear(1) RECOMPRESS CODEC(LZ4HC(10))\nSETTINGS index_granularity = 8192 +CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt64,\n `value` String\n)\nENGINE = MergeTree()\nPARTITION BY key\nORDER BY tuple()\nTTL dt + toIntervalMonth(1) RECOMPRESS CODEC(ZSTD(17)), dt + toIntervalYear(1) RECOMPRESS CODEC(LZ4HC(10))\nSETTINGS min_rows_for_wide_part = 0, index_granularity = 8192 3000 1_1_1_0 LZ4 -2_2_2_0 ZSTD(17) -3_3_3_0 LZ4HC(10) -CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt64,\n `value` String\n)\nENGINE = MergeTree()\nPARTITION BY key\nORDER BY tuple()\nTTL dt + toIntervalMonth(1) RECOMPRESS CODEC(ZSTD(12))\nSETTINGS index_granularity = 8192 -1_1_1_0_4 LZ4 -2_2_2_0_4 ZSTD(17) -3_3_3_0_4 LZ4HC(10) +2_2_2_0 LZ4 +3_3_3_0 LZ4 +1_1_1_1 LZ4 +2_2_2_1 ZSTD(17) +3_3_3_1 LZ4HC(10) +CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt64,\n `value` String\n)\nENGINE = MergeTree()\nPARTITION BY key\nORDER BY tuple()\nTTL dt + toIntervalDay(1) RECOMPRESS CODEC(ZSTD(12))\nSETTINGS min_rows_for_wide_part = 0, index_granularity = 8192 1_1_1_1_4 LZ4 -2_2_2_1_4 ZSTD(12) -3_3_3_1_4 ZSTD(12) +2_2_2_1_4 ZSTD(17) +3_3_3_1_4 LZ4HC(10) +1_1_1_2_4 LZ4 +2_2_2_2_4 ZSTD(12) +3_3_3_2_4 ZSTD(12) +1_1_1_0 LZ4 +2_2_2_0 LZ4 +3_3_3_0 LZ4 +1_1_1_0_4 LZ4 +2_2_2_0_4 ZSTD(12) +3_3_3_0_4 ZSTD(12) diff --git a/tests/queries/0_stateless/01465_ttl_recompression.sql b/tests/queries/0_stateless/01465_ttl_recompression.sql index 92233f2d5cb..0683f971d5f 100644 --- a/tests/queries/0_stateless/01465_ttl_recompression.sql +++ b/tests/queries/0_stateless/01465_ttl_recompression.sql @@ -9,7 +9,8 @@ CREATE TABLE recompression_table ) ENGINE MergeTree() ORDER BY tuple() PARTITION BY key -TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(17)), dt + INTERVAL 1 YEAR RECOMPRESS CODEC(LZ4HC(10)); +TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(17)), dt + INTERVAL 1 YEAR RECOMPRESS CODEC(LZ4HC(10)) +SETTINGS min_rows_for_wide_part = 0; SHOW CREATE TABLE recompression_table; @@ -25,7 +26,11 @@ SELECT COUNT() FROM recompression_table; SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name; -ALTER TABLE recompression_table MODIFY TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(12)) SETTINGS mutations_sync = 2; +OPTIMIZE TABLE recompression_table FINAL; + +SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name; + +ALTER TABLE recompression_table MODIFY TTL dt + INTERVAL 1 DAY RECOMPRESS CODEC(ZSTD(12)) SETTINGS mutations_sync = 2; SHOW CREATE TABLE recompression_table; @@ -38,3 +43,31 @@ OPTIMIZE TABLE recompression_table FINAL; SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name; DROP TABLE IF EXISTS recompression_table; + +CREATE TABLE recompression_table_compact +( + dt DateTime, + key UInt64, + value String + +) ENGINE MergeTree() +ORDER BY tuple() +PARTITION BY key +TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(17)), dt + INTERVAL 1 YEAR RECOMPRESS CODEC(LZ4HC(10)) +SETTINGS min_rows_for_wide_part = 10000; + +SYSTEM STOP TTL MERGES recompression_table_compact; + +INSERT INTO recompression_table_compact SELECT now(), 1, toString(number) from numbers(1000); + +INSERT INTO recompression_table_compact SELECT now() - INTERVAL 2 MONTH, 2, toString(number) from numbers(1000, 1000); + +INSERT INTO recompression_table_compact SELECT now() - INTERVAL 2 YEAR, 3, toString(number) from numbers(2000, 1000); + +SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table_compact' and active = 1 and database = currentDatabase() ORDER BY name; + +ALTER TABLE recompression_table_compact MODIFY TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(12)) SETTINGS mutations_sync = 2; -- mutation affect all columns, so codec changes + +SELECT name, default_compression_codec FROM system.parts WHERE table = 'recompression_table_compact' and active = 1 and database = currentDatabase() ORDER BY name; + +DROP TABLE recompression_table_compact; From 4c3c1cdaf3e4064a5d65a40dea5383e522e8f2ee Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 7 Sep 2020 15:24:27 +0800 Subject: [PATCH 092/298] Fix Nullable String to Enum conversion. --- src/Functions/FunctionsConversion.h | 4 +--- .../01490_nullable_string_to_enum.reference | 1 + .../0_stateless/01490_nullable_string_to_enum.sql | 12 ++++++++++++ 3 files changed, 14 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01490_nullable_string_to_enum.reference create mode 100644 tests/queries/0_stateless/01490_nullable_string_to_enum.sql diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index a18139fd4c8..ffe7677afe7 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2260,9 +2260,7 @@ private: size_t nullable_pos = block.columns() - 1; nullable_col = typeid_cast(block.getByPosition(nullable_pos).column.get()); - if (!nullable_col) - throw Exception("Last column should be ColumnNullable", ErrorCodes::LOGICAL_ERROR); - if (col && nullable_col->size() != col->size()) + if (col && nullable_col && nullable_col->size() != col->size()) throw Exception("ColumnNullable is not compatible with original", ErrorCodes::LOGICAL_ERROR); } diff --git a/tests/queries/0_stateless/01490_nullable_string_to_enum.reference b/tests/queries/0_stateless/01490_nullable_string_to_enum.reference new file mode 100644 index 00000000000..ce013625030 --- /dev/null +++ b/tests/queries/0_stateless/01490_nullable_string_to_enum.reference @@ -0,0 +1 @@ +hello diff --git a/tests/queries/0_stateless/01490_nullable_string_to_enum.sql b/tests/queries/0_stateless/01490_nullable_string_to_enum.sql new file mode 100644 index 00000000000..e0624af4a7a --- /dev/null +++ b/tests/queries/0_stateless/01490_nullable_string_to_enum.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS t_enum; +DROP TABLE IF EXISTS t_source; + +CREATE TABLE t_enum(x Enum8('hello' = 1, 'world' = 2)) ENGINE = TinyLog; +CREATE TABLE t_source(x Nullable(String)) ENGINE = TinyLog; + +INSERT INTO t_source (x) VALUES ('hello'); +INSERT INTO t_enum(x) SELECT x from t_source WHERE x in ('hello', 'world'); +SELECT * FROM t_enum; + +DROP TABLE IF EXISTS t_enum; +DROP TABLE IF EXISTS t_source; From f274ffc9d15f446048e197d33d482ff0869684e7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 7 Sep 2020 10:59:14 +0300 Subject: [PATCH 093/298] Add comments --- src/DataStreams/TTLBlockInputStream.h | 3 +++ src/Storages/MergeTree/MergeSelector.h | 3 +++ src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.h | 2 ++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 8 +++++--- .../MergeTree/MergeTreeDataMergerMutator.h | 4 ++-- .../MergeTree/MergeTreeDataPartTTLInfo.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h | 6 +++++- src/Storages/MergeTree/MergeTreePartsMover.cpp | 2 +- src/Storages/MergeTree/TTLMergeSelector.cpp | 2 +- src/Storages/MergeTree/TTLMergeSelector.h | 15 +++++++++++++++ 11 files changed, 40 insertions(+), 11 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/DataStreams/TTLBlockInputStream.h index 774b413ed1b..1d3b69f61c5 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/DataStreams/TTLBlockInputStream.h @@ -76,11 +76,14 @@ private: /// Finalize agg_result into result_columns void finalizeAggregates(MutableColumns & result_columns); + /// Execute description expressions on block and update ttl's in + /// ttl_info_map with expression results. void updateTTLWithDescriptions(Block & block, const TTLDescriptions & descriptions, TTLInfoMap & ttl_info_map); /// Updates TTL for moves void updateMovesTTL(Block & block); + /// Update values for recompression TTL using data from block. void updateRecompressionTTL(Block & block); UInt32 getTimestampByIndex(const IColumn * column, size_t ind); diff --git a/src/Storages/MergeTree/MergeSelector.h b/src/Storages/MergeTree/MergeSelector.h index bc2dc81c486..9c043005312 100644 --- a/src/Storages/MergeTree/MergeSelector.h +++ b/src/Storages/MergeTree/MergeSelector.h @@ -42,8 +42,11 @@ public: /// Opaque pointer to avoid dependencies (it is not possible to do forward declaration of typedef). const void * data; + /// Information about different TTLs for part. Can be used by + /// TTLSelector to assign merges with TTL. MergeTreeDataPartTTLInfos ttl_infos; + /// Part compression codec definition. ASTPtr compression_codec_desc; }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8ba00f29d9d..9f00fee070e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2978,7 +2978,7 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ auto metadata_snapshot = getInMemoryMetadataPtr(); ReservationPtr reservation; - auto ttl_entry = selectTTLEntryForTTLInfos(metadata_snapshot->getMoveTTLs(), ttl_infos.moves_ttl, time_of_move, true); + auto ttl_entry = selectTTLDescriptionForTTLInfos(metadata_snapshot->getMoveTTLs(), ttl_infos.moves_ttl, time_of_move, true); if (ttl_entry) { @@ -3039,7 +3039,7 @@ CompressionCodecPtr MergeTreeData::getCompressionCodecForPart(size_t part_size_c auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); - auto best_ttl_entry = selectTTLEntryForTTLInfos(recompression_ttl_entries, ttl_infos.recompression_ttl, current_time, false); + auto best_ttl_entry = selectTTLDescriptionForTTLInfos(recompression_ttl_entries, ttl_infos.recompression_ttl, current_time, false); if (best_ttl_entry) return CompressionCodecFactory::instance().get(best_ttl_entry->recompression_codec, {}); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index c6c734f315a..82f118a4c0f 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -672,6 +672,8 @@ public: ExpressionActionsPtr getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; ExpressionActionsPtr getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; + /// Get compression codec for part according to TTL rules and + /// section from config.xml. CompressionCodecPtr getCompressionCodecForPart(size_t part_size_compressed, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t current_time) const; /// Limiting parallel sends per one table, used in DataPartsExchange diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index b88d80dc41f..a8f7e265f68 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -295,6 +295,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( if (metadata_snapshot->hasAnyTTL() && merge_with_ttl_allowed && !ttl_merges_blocker.isCancelled()) { + /// TTL delete is prefered to recompression TTLDeleteMergeSelector delete_ttl_selector( next_delete_ttl_merge_times_by_partition, current_time, @@ -303,7 +304,9 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( parts_to_merge = delete_ttl_selector.select(parts_ranges, max_total_size_to_merge); if (!parts_to_merge.empty()) + { future_part.merge_type = MergeType::TTL_DELETE; + } else if (metadata_snapshot->hasAnyRecompressionTTL()) { TTLRecompressMergeSelector recompress_ttl_selector( @@ -625,6 +628,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (merges_blocker.isCancelled()) throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); + /// We don't want to perform merge assigned with TTL as normal merge, so + /// throw exception if (isTTLMergeType(future_part.merge_type) && ttl_merges_blocker.isCancelled()) throw Exception("Cancelled merging parts with TTL", ErrorCodes::ABORTED); @@ -669,9 +674,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor new_data_part->partition.assign(future_part.getPartition()); new_data_part->is_temp = true; - if (isTTLMergeType(future_part.merge_type) && ttl_merges_blocker.isCancelled()) - throw Exception("Cancelled merging parts with expired TTL", ErrorCodes::ABORTED); - bool need_remove_expired_values = false; bool force_ttl = false; for (const auto & part : parts) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 5f6b9246d68..96ab14ba57b 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -250,10 +250,10 @@ private: /// When the last time you wrote to the log that the disk space was running out (not to write about this too often). time_t disk_space_warning_time = 0; - /// Stores the next TTL merge due time for each partition (used only by TTLMergeSelector) + /// Stores the next TTL delete merge due time for each partition (used only by TTLDeleteMergeSelector) ITTLMergeSelector::PartitionIdToTTLs next_delete_ttl_merge_times_by_partition; - /// Stores the next TTL merge due time for each partition (used only by TTLMergeSelector) + /// Stores the next TTL recompress merge due time for each partition (used only by TTLRecompressionMergeSelector) ITTLMergeSelector::PartitionIdToTTLs next_recompress_ttl_merge_times_by_partition; /// Performing TTL merges independently for each partition guarantees that /// there is only a limited number of TTL merges and no partition stores data, that is too stale diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp index 1cecb2672fb..33ed60c225a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp @@ -186,7 +186,7 @@ time_t MergeTreeDataPartTTLInfos::getMaxRecompressionTTL() const } -std::optional selectTTLEntryForTTLInfos(const TTLDescriptions & descriptions, const TTLInfoMap & ttl_info_map, time_t current_time, bool use_max) +std::optional selectTTLDescriptionForTTLInfos(const TTLDescriptions & descriptions, const TTLInfoMap & ttl_info_map, time_t current_time, bool use_max) { time_t best_ttl_time = 0; TTLDescriptions::const_iterator best_entry_it; diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h index 2ab571cf3ba..d9a10785738 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h @@ -49,7 +49,10 @@ struct MergeTreeDataPartTTLInfos TTLInfoMap recompression_ttl; + /// Return min recompression TTL value if any, otherwise return zero. time_t getMinRecompressionTTL() const; + + /// Return max recompression TTL value if any, otherwise return zero. time_t getMaxRecompressionTTL() const; void read(ReadBuffer & in); @@ -71,6 +74,7 @@ struct MergeTreeDataPartTTLInfos } }; -std::optional selectTTLEntryForTTLInfos(const TTLDescriptions & descriptions, const TTLInfoMap & ttl_info_map, time_t current_time, bool use_max); +/// Selects the most appropriate TTLDescription using TTL info and current time. +std::optional selectTTLDescriptionForTTLInfos(const TTLDescriptions & descriptions, const TTLInfoMap & ttl_info_map, time_t current_time, bool use_max); } diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 92ea745c5df..586c4393dfb 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -130,7 +130,7 @@ bool MergeTreePartsMover::selectPartsForMove( if (!can_move(part, &reason)) continue; - auto ttl_entry = selectTTLEntryForTTLInfos(metadata_snapshot->getMoveTTLs(), part->ttl_infos.moves_ttl, time_of_move, true); + auto ttl_entry = selectTTLDescriptionForTTLInfos(metadata_snapshot->getMoveTTLs(), part->ttl_infos.moves_ttl, time_of_move, true); auto to_insert = need_to_move.find(part->volume->getDisk()); ReservationPtr reservation; diff --git a/src/Storages/MergeTree/TTLMergeSelector.cpp b/src/Storages/MergeTree/TTLMergeSelector.cpp index bb7c001eae1..d46eb19815a 100644 --- a/src/Storages/MergeTree/TTLMergeSelector.cpp +++ b/src/Storages/MergeTree/TTLMergeSelector.cpp @@ -107,7 +107,7 @@ bool TTLRecompressMergeSelector::isTTLAlreadySatisfied(const IMergeSelector::Par if (recompression_ttls.empty()) return false; - auto ttl_description = selectTTLEntryForTTLInfos(recompression_ttls, part.ttl_infos.recompression_ttl, current_time, false); + auto ttl_description = selectTTLDescriptionForTTLInfos(recompression_ttls, part.ttl_infos.recompression_ttl, current_time, false); if (!ttl_description) return true; diff --git a/src/Storages/MergeTree/TTLMergeSelector.h b/src/Storages/MergeTree/TTLMergeSelector.h index eab4cdcd295..1d41b65f9fb 100644 --- a/src/Storages/MergeTree/TTLMergeSelector.h +++ b/src/Storages/MergeTree/TTLMergeSelector.h @@ -32,7 +32,12 @@ public: const PartsRanges & parts_ranges, const size_t max_total_size_to_merge) override; + /// Get TTL value for part, may depend on child type and some settings in + /// constructor. virtual time_t getTTLForPart(const IMergeSelector::Part & part) const = 0; + + /// Sometimes we can check that TTL already satisfied using information + /// stored in part and don't assign merge for such part. virtual bool isTTLAlreadySatisfied(const IMergeSelector::Part & part) const = 0; protected: @@ -44,6 +49,8 @@ private: }; +/// Select parts to merge using information about delete TTL. Depending on flag +/// only_drop_parts can use max or min TTL value. class TTLDeleteMergeSelector : public ITTLMergeSelector { public: @@ -55,6 +62,8 @@ public: time_t getTTLForPart(const IMergeSelector::Part & part) const override; + /// Delete TTL should be checked only by TTL time, there are no other ways + /// to satisfy it. bool isTTLAlreadySatisfied(const IMergeSelector::Part &) const override { return false; @@ -64,6 +73,8 @@ private: bool only_drop_parts; }; +/// Select parts to merge using information about recompression TTL and +/// compression codec of existing parts. class TTLRecompressMergeSelector : public ITTLMergeSelector { public: @@ -72,8 +83,12 @@ public: , recompression_ttls(recompression_ttls_) {} + /// Return part min recompression TTL. time_t getTTLForPart(const IMergeSelector::Part & part) const override; + /// Checks that part's codec is not already equal to required codec + /// according to recompression TTL. It doesn't make sence to assign such + /// merge. bool isTTLAlreadySatisfied(const IMergeSelector::Part & part) const override; private: TTLDescriptions recompression_ttls; From 99ebab706cf70a286a5a6b5b2ac6070085f1ebf0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 12:02:22 +0300 Subject: [PATCH 094/298] Fix "Arcadia" --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 707f91b0c93..16450efb26e 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -139,3 +139,4 @@ 01455_time_zones 01456_ast_optimizations_over_distributed 01460_DistributedFilesToInsert +01474_executable_dictionary From 9493532fdb1ecc20d0ed084a61f58a9b10e869f6 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 7 Sep 2020 04:05:30 +0300 Subject: [PATCH 095/298] rework test for redis dictionary --- .../runner/compose/docker_compose_redis.yml | 2 +- .../dictionary.py | 0 .../external_sources.py | 14 +- .../fake_cert.pem | 0 .../http_server.py | 0 .../test.py | 126 +------------ .../test_dictionaries_redis/__init__.py | 0 .../test_dictionaries_redis/test.py | 176 ++++++++++++++++++ 8 files changed, 184 insertions(+), 134 deletions(-) rename tests/integration/{test_dictionaries_all_layouts_and_sources => helpers}/dictionary.py (100%) rename tests/integration/{test_dictionaries_all_layouts_and_sources => helpers}/external_sources.py (97%) rename tests/integration/{test_dictionaries_all_layouts_and_sources => helpers}/fake_cert.pem (100%) rename tests/integration/{test_dictionaries_all_layouts_and_sources => helpers}/http_server.py (100%) create mode 100644 tests/integration/test_dictionaries_redis/__init__.py create mode 100644 tests/integration/test_dictionaries_redis/test.py diff --git a/docker/test/integration/runner/compose/docker_compose_redis.yml b/docker/test/integration/runner/compose/docker_compose_redis.yml index 2c9ace96d0c..72df99ec59b 100644 --- a/docker/test/integration/runner/compose/docker_compose_redis.yml +++ b/docker/test/integration/runner/compose/docker_compose_redis.yml @@ -5,4 +5,4 @@ services: restart: always ports: - 6380:6379 - command: redis-server --requirepass "clickhouse" + command: redis-server --requirepass "clickhouse" --databases 32 diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/dictionary.py b/tests/integration/helpers/dictionary.py similarity index 100% rename from tests/integration/test_dictionaries_all_layouts_and_sources/dictionary.py rename to tests/integration/helpers/dictionary.py diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/external_sources.py b/tests/integration/helpers/external_sources.py similarity index 97% rename from tests/integration/test_dictionaries_all_layouts_and_sources/external_sources.py rename to tests/integration/helpers/external_sources.py index fac7dcdea1e..5e8d420ff94 100644 --- a/tests/integration/test_dictionaries_all_layouts_and_sources/external_sources.py +++ b/tests/integration/helpers/external_sources.py @@ -477,13 +477,13 @@ class SourceCassandra(ExternalSource): class SourceRedis(ExternalSource): def __init__( - self, name, internal_hostname, internal_port, docker_hostname, docker_port, user, password, storage_type + self, name, internal_hostname, internal_port, docker_hostname, docker_port, user, password, db_index, storage_type ): super(SourceRedis, self).__init__( name, internal_hostname, internal_port, docker_hostname, docker_port, user, password ) self.storage_type = storage_type - self.db_index = 1 + self.db_index = db_index def get_source_str(self, table_name): return ''' @@ -513,21 +513,13 @@ class SourceRedis(ExternalSource): values = [] for name in self.ordered_names: values.append(str(row.data[name])) - print 'values: ', values if len(values) == 2: self.client.set(*values) - print 'kek: ', self.client.get(values[0]) else: self.client.hset(*values) def compatible_with_layout(self, layout): - if ( - layout.is_simple and self.storage_type == "simple" or - layout.is_complex and self.storage_type == "simple" and layout.name == "complex_key_hashed_one_key" or - layout.is_complex and self.storage_type == "hash_map" and layout.name == "complex_key_hashed_two_keys" - ): - return True - return False + return layout.is_simple and self.storage_type == "simple" or layout.is_complex and self.storage_type == "hash_map" class SourceAerospike(ExternalSource): def __init__(self, name, internal_hostname, internal_port, diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/fake_cert.pem b/tests/integration/helpers/fake_cert.pem similarity index 100% rename from tests/integration/test_dictionaries_all_layouts_and_sources/fake_cert.pem rename to tests/integration/helpers/fake_cert.pem diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/http_server.py b/tests/integration/helpers/http_server.py similarity index 100% rename from tests/integration/test_dictionaries_all_layouts_and_sources/http_server.py rename to tests/integration/helpers/http_server.py diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py index 3e11a544229..5a46498ce08 100644 --- a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py +++ b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py @@ -2,9 +2,9 @@ import pytest import os from helpers.cluster import ClickHouseCluster -from dictionary import Field, Row, Dictionary, DictionaryStructure, Layout -from external_sources import SourceMySQL, SourceClickHouse, SourceFile, SourceExecutableCache, SourceExecutableHashed -from external_sources import SourceMongo, SourceMongoURI, SourceHTTP, SourceHTTPS, SourceRedis, SourceCassandra +from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout +from helpers.external_sources import SourceMySQL, SourceClickHouse, SourceFile, SourceExecutableCache, SourceExecutableHashed +from helpers.external_sources import SourceMongo, SourceMongoURI, SourceHTTP, SourceHTTPS, SourceRedis, SourceCassandra import math SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -132,14 +132,6 @@ SOURCES = [ DICTIONARIES = [] -# Key-value dictionaries with only one possible field for key -SOURCES_KV = [ - SourceRedis("RedisSimple", "localhost", "6380", "redis1", "6379", "", "clickhouse", storage_type="simple"), - SourceRedis("RedisHash", "localhost", "6380", "redis1", "6379", "", "clickhouse", storage_type="hash_map"), -] - -DICTIONARIES_KV = [] - cluster = None node = None @@ -170,17 +162,6 @@ def setup_module(module): else: print "Source", source.name, "incompatible with layout", layout.name - for layout in LAYOUTS: - field_keys = list(filter(lambda x: x.is_key, FIELDS[layout.layout_type])) - for source in SOURCES_KV: - if not source.compatible_with_layout(layout): - print "Source", source.name, "incompatible with layout", layout.name - continue - - for field in FIELDS[layout.layout_type]: - if not (field.is_key or field.is_range or field.is_range_key): - DICTIONARIES_KV.append(get_dict(source, layout, field_keys + [field], field.name)) - cluster = ClickHouseCluster(__file__) main_configs = [] @@ -199,7 +180,7 @@ def setup_module(module): def started_cluster(): try: cluster.start() - for dictionary in DICTIONARIES + DICTIONARIES_KV: + for dictionary in DICTIONARIES: print "Preparing", dictionary.name dictionary.prepare_source(cluster) print "Prepared" @@ -363,102 +344,3 @@ def test_ranged_dictionaries(started_cluster, fold): for query, answer in queries_with_answers: print query assert node.query(query) == str(answer) + '\n' - - -@pytest.mark.parametrize("fold", list(range(10))) -def test_key_value_simple_dictionaries(started_cluster, fold): - fields = FIELDS["simple"] - values = VALUES["simple"] - data = [Row(fields, vals) for vals in values] - - all_simple_dicts = [d for d in DICTIONARIES_KV if d.structure.layout.layout_type == "simple"] - simple_dicts = get_dictionaries(fold, 10, all_simple_dicts) - - for dct in simple_dicts: - queries_with_answers = [] - local_data = [] - for row in data: - local_fields = dct.get_fields() - local_values = [row.get_value_by_name(field.name) for field in local_fields if row.has_field(field.name)] - local_data.append(Row(local_fields, local_values)) - - dct.load_data(local_data) - - node.query("system reload dictionary {}".format(dct.name)) - - print 'name: ', dct.name - - for row in local_data: - print dct.get_fields() - for field in dct.get_fields(): - print field.name, field.is_key - if not field.is_key: - for query in dct.get_select_get_queries(field, row): - queries_with_answers.append((query, row.get_value_by_name(field.name))) - - for query in dct.get_select_has_queries(field, row): - queries_with_answers.append((query, 1)) - - for query in dct.get_select_get_or_default_queries(field, row): - queries_with_answers.append((query, field.default_value_for_get)) - - if dct.structure.has_hierarchy: - for query in dct.get_hierarchical_queries(data[0]): - queries_with_answers.append((query, [1])) - - for query in dct.get_hierarchical_queries(data[1]): - queries_with_answers.append((query, [2, 1])) - - for query in dct.get_is_in_queries(data[0], data[1]): - queries_with_answers.append((query, 0)) - - for query in dct.get_is_in_queries(data[1], data[0]): - queries_with_answers.append((query, 1)) - - for query, answer in queries_with_answers: - print query - if isinstance(answer, list): - answer = str(answer).replace(' ', '') - assert node.query(query) == str(answer) + '\n' - - -@pytest.mark.parametrize("fold", list(range(10))) -def test_key_value_complex_dictionaries(started_cluster, fold): - fields = FIELDS["complex"] - values = VALUES["complex"] - data = [Row(fields, vals) for vals in values] - - all_complex_dicts = [d for d in DICTIONARIES_KV if d.structure.layout.layout_type == "complex"] - complex_dicts = get_dictionaries(fold, 10, all_complex_dicts) - for dct in complex_dicts: - dct.load_data(data) - - node.query("system reload dictionaries") - - for dct in complex_dicts: - queries_with_answers = [] - local_data = [] - for row in data: - local_fields = dct.get_fields() - local_values = [row.get_value_by_name(field.name) for field in local_fields if row.has_field(field.name)] - local_data.append(Row(local_fields, local_values)) - - dct.load_data(local_data) - - node.query("system reload dictionary {}".format(dct.name)) - - for row in local_data: - for field in dct.get_fields(): - if not field.is_key: - for query in dct.get_select_get_queries(field, row): - queries_with_answers.append((query, row.get_value_by_name(field.name))) - - for query in dct.get_select_has_queries(field, row): - queries_with_answers.append((query, 1)) - - for query in dct.get_select_get_or_default_queries(field, row): - queries_with_answers.append((query, field.default_value_for_get)) - - for query, answer in queries_with_answers: - print query - assert node.query(query) == str(answer) + '\n' diff --git a/tests/integration/test_dictionaries_redis/__init__.py b/tests/integration/test_dictionaries_redis/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_dictionaries_redis/test.py b/tests/integration/test_dictionaries_redis/test.py new file mode 100644 index 00000000000..1588efa3426 --- /dev/null +++ b/tests/integration/test_dictionaries_redis/test.py @@ -0,0 +1,176 @@ +import os +import pytest +import redis + +from helpers.cluster import ClickHouseCluster +from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout +from helpers.external_sources import SourceRedis + +cluster = None +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +dict_configs_path = os.path.join(SCRIPT_DIR, 'configs/dictionaries') +node = None + +KEY_FIELDS = { + "simple": [ + Field("KeyField", 'UInt64', is_key=True, default_value_for_get=9999999) + ], + "complex": [ + Field("KeyField1", 'UInt64', is_key=True, default_value_for_get=9999999), + Field("KeyField2", 'String', is_key=True, default_value_for_get='xxxxxxxxx'), + ] +} + +KEY_VALUES = { + "simple" : [ + [1], [2] + ], + "complex" : [ + [1, 'world'], [2, 'qwerty2'] + ] +} + +FIELDS = [ + Field("UInt8_", 'UInt8', default_value_for_get=55), + Field("UInt16_", 'UInt16', default_value_for_get=66), + Field("UInt32_", 'UInt32', default_value_for_get=77), + Field("UInt64_", 'UInt64', default_value_for_get=88), + Field("Int8_", 'Int8', default_value_for_get=-55), + Field("Int16_", 'Int16', default_value_for_get=-66), + Field("Int32_", 'Int32', default_value_for_get=-77), + Field("Int64_", 'Int64', default_value_for_get=-88), + Field("UUID_", 'UUID', default_value_for_get='550e8400-0000-0000-0000-000000000000'), + Field("Date_", 'Date', default_value_for_get='2018-12-30'), + Field("DateTime_", 'DateTime', default_value_for_get='2018-12-30 00:00:00'), + Field("String_", 'String', default_value_for_get='hi'), + Field("Float32_", 'Float32', default_value_for_get=555.11), + Field("Float64_", 'Float64', default_value_for_get=777.11), +] + +VALUES = [ + [22, 3], + [333, 4], + [4444, 5], + [55555, 6], + [-6, -7], + [-77, -8], + [-888, -9], + [-999, -10], + ['550e8400-e29b-41d4-a716-446655440003', '550e8400-e29b-41d4-a716-446655440002'], + ['1973-06-28', '1978-06-28'], + ['1985-02-28 23:43:25', '1986-02-28 23:42:25'], + ['hello', 'hello'], + [22.543, 21.543], + [3332154213.4, 3222154213.4], +] + +LAYOUTS = [ + Layout("flat"), + Layout("hashed"), + Layout("cache"), + Layout("complex_key_hashed"), + # Layout("complex_key_cache"), # Currently not supported + Layout("direct"), + # Layout("complex_key_direct") # Currently not supported +] + +DICTIONARIES = [] + +def get_dict(source, layout, fields, suffix_name=''): + global dict_configs_path + + structure = DictionaryStructure(layout, fields) + dict_name = source.name + "_" + layout.name + '_' + suffix_name + dict_path = os.path.join(dict_configs_path, dict_name + '.xml') + dictionary = Dictionary(dict_name, structure, source, dict_path, "table_" + dict_name, fields) + dictionary.generate_config() + return dictionary + + +def setup_module(module): + global DICTIONARIES + global cluster + global node + global dict_configs_path + + for f in os.listdir(dict_configs_path): + os.remove(os.path.join(dict_configs_path, f)) + + for i, field in enumerate(FIELDS): + DICTIONARIES.append([]) + sources = [] + sources.append(SourceRedis("RedisSimple", "localhost", "6380", "redis1", "6379", "", "clickhouse", i * 2, storage_type="simple")) + sources.append(SourceRedis("RedisHash", "localhost", "6380", "redis1", "6379", "", "clickhouse", i * 2 + 1, storage_type="hash_map")) + for source in sources: + for layout in LAYOUTS: + if not source.compatible_with_layout(layout): + print "Source", source.name, "incompatible with layout", layout.name + continue + + fields = KEY_FIELDS[layout.layout_type] + [field] + DICTIONARIES[i].append(get_dict(source, layout, fields, field.name)) + + main_configs = [] + dictionaries = [] + for fname in os.listdir(dict_configs_path): + dictionaries.append(os.path.join(dict_configs_path, fname)) + + cluster = ClickHouseCluster(__file__) + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_redis=True) + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + assert len(FIELDS) == len(VALUES) + for dicts in DICTIONARIES: + for dictionary in dicts: + print "Preparing", dictionary.name + dictionary.prepare_source(cluster) + print "Prepared" + + yield cluster + + finally: + cluster.shutdown() + +@pytest.mark.parametrize("id", range(len(FIELDS))) +def test_redis_dictionaries(started_cluster, id): + print 'id:', id + + dicts = DICTIONARIES[id] + values = VALUES[id] + field = FIELDS[id] + + node.query("system reload dictionaries") + + for dct in dicts: + data = [] + dict_type = dct.structure.layout.layout_type + key_fields = KEY_FIELDS[dict_type] + key_values = KEY_VALUES[dict_type] + + for key_value, value in zip(key_values, values): + data.append(Row(key_fields + [field], key_value + [value])) + + dct.load_data(data) + + queries_with_answers = [] + for row in data: + for query in dct.get_select_get_queries(field, row): + queries_with_answers.append((query, row.get_value_by_name(field.name))) + + for query in dct.get_select_has_queries(field, row): + queries_with_answers.append((query, 1)) + + for query in dct.get_select_get_or_default_queries(field, row): + queries_with_answers.append((query, field.default_value_for_get)) + + node.query("system reload dictionary {}".format(dct.name)) + + for query, answer in queries_with_answers: + print query + assert node.query(query) == str(answer) + '\n' + + # Checks, that dictionaries can be reloaded. + node.query("system reload dictionaries") From acbe21304a56cf840b0a54d2f1918d5a459bbcbb Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 7 Sep 2020 14:46:32 +0300 Subject: [PATCH 096/298] Fix test for compact parts --- tests/queries/0_stateless/01465_ttl_recompression.reference | 4 ++-- tests/queries/0_stateless/01465_ttl_recompression.sql | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01465_ttl_recompression.reference b/tests/queries/0_stateless/01465_ttl_recompression.reference index 40d7ed8896b..2f8815c62eb 100644 --- a/tests/queries/0_stateless/01465_ttl_recompression.reference +++ b/tests/queries/0_stateless/01465_ttl_recompression.reference @@ -1,4 +1,4 @@ -CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt64,\n `value` String\n)\nENGINE = MergeTree()\nPARTITION BY key\nORDER BY tuple()\nTTL dt + toIntervalMonth(1) RECOMPRESS CODEC(ZSTD(17)), dt + toIntervalYear(1) RECOMPRESS CODEC(LZ4HC(10))\nSETTINGS min_rows_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt64,\n `value` String\n)\nENGINE = MergeTree()\nPARTITION BY key\nORDER BY tuple()\nTTL dt + toIntervalMonth(1) RECOMPRESS CODEC(ZSTD(17)), dt + toIntervalYear(1) RECOMPRESS CODEC(LZ4HC(10))\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192 3000 1_1_1_0 LZ4 2_2_2_0 LZ4 @@ -6,7 +6,7 @@ CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt6 1_1_1_1 LZ4 2_2_2_1 ZSTD(17) 3_3_3_1 LZ4HC(10) -CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt64,\n `value` String\n)\nENGINE = MergeTree()\nPARTITION BY key\nORDER BY tuple()\nTTL dt + toIntervalDay(1) RECOMPRESS CODEC(ZSTD(12))\nSETTINGS min_rows_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt64,\n `value` String\n)\nENGINE = MergeTree()\nPARTITION BY key\nORDER BY tuple()\nTTL dt + toIntervalDay(1) RECOMPRESS CODEC(ZSTD(12))\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192 1_1_1_1_4 LZ4 2_2_2_1_4 ZSTD(17) 3_3_3_1_4 LZ4HC(10) diff --git a/tests/queries/0_stateless/01465_ttl_recompression.sql b/tests/queries/0_stateless/01465_ttl_recompression.sql index 0683f971d5f..92f20ddd495 100644 --- a/tests/queries/0_stateless/01465_ttl_recompression.sql +++ b/tests/queries/0_stateless/01465_ttl_recompression.sql @@ -10,7 +10,7 @@ CREATE TABLE recompression_table ORDER BY tuple() PARTITION BY key TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(17)), dt + INTERVAL 1 YEAR RECOMPRESS CODEC(LZ4HC(10)) -SETTINGS min_rows_for_wide_part = 0; +SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0; SHOW CREATE TABLE recompression_table; From f8932a7a6bbf10c80ceb05ea25f9a7502e1cb961 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 7 Sep 2020 16:40:03 +0300 Subject: [PATCH 097/298] Build and tests config --- tests/ci/build_config.json | 692 +++++++++++++++++++++++++++++-------- 1 file changed, 540 insertions(+), 152 deletions(-) diff --git a/tests/ci/build_config.json b/tests/ci/build_config.json index e4b9c1d6b75..02c96b085da 100644 --- a/tests/ci/build_config.json +++ b/tests/ci/build_config.json @@ -1,153 +1,541 @@ -[ - { - "compiler": "gcc-9", - "build-type": "", - "sanitizer": "", - "package-type": "deb", - "bundled": "bundled", - "splitted": "unsplitted", - "alien_pkgs": true, - "tidy": "disable", - "with_coverage": false - }, - { - "compiler": "gcc-9", - "build-type": "", - "sanitizer": "", - "package-type": "performance", - "bundled": "bundled", - "splitted": "unsplitted", - "tidy": "disable", - "with_coverage": false - }, - { - "compiler": "gcc-9", - "build-type": "", - "sanitizer": "", - "package-type": "binary", - "bundled": "bundled", - "splitted": "unsplitted", - "tidy": "disable", - "with_coverage": false - }, - { - "compiler": "clang-10", - "build-type": "", - "sanitizer": "address", - "package-type": "deb", - "bundled": "bundled", - "splitted": "unsplitted", - "tidy": "disable", - "with_coverage": false - }, - { - "compiler": "clang-10", - "build-type": "", - "sanitizer": "undefined", - "package-type": "deb", - "bundled": "bundled", - "splitted": "unsplitted", - "tidy": "disable", - "with_coverage": false - }, - { - "compiler": "clang-10", - "build-type": "", - "sanitizer": "thread", - "package-type": "deb", - "bundled": "bundled", - "splitted": "unsplitted", - "tidy": "disable", - "with_coverage": false - }, - { - "compiler": "clang-10", - "build-type": "", - "sanitizer": "memory", - "package-type": "deb", - "bundled": "bundled", - "splitted": "unsplitted", - "tidy": "disable", - "with_coverage": false - }, - { - "compiler": "clang-10", - "build-type": "", - "sanitizer": "", - "package-type": "deb", - "bundled": "bundled", - "splitted": "unsplitted", - "tidy": "disable", - "with_coverage": false - }, - { - "compiler": "clang-10", - "build-type": "debug", - "sanitizer": "", - "package-type": "deb", - "bundled": "bundled", - "splitted": "unsplitted", - "tidy": "disable", - "with_coverage": false - }, - { - "compiler": "gcc-9", - "build-type": "", - "sanitizer": "", - "package-type": "deb", - "bundled": "unbundled", - "splitted": "unsplitted", - "tidy": "disable", - "with_coverage": false - }, - { - "compiler": "clang-10", - "build-type": "", - "sanitizer": "", - "package-type": "binary", - "bundled": "bundled", - "splitted": "unsplitted", - "tidy": "disable", - "with_coverage": false - }, - { - "compiler": "clang-10", - "build-type": "", - "sanitizer": "", - "package-type": "binary", - "bundled": "bundled", - "splitted": "splitted", - "tidy": "disable", - "with_coverage": false - }, - { - "compiler": "clang-10-darwin", - "build-type": "", - "sanitizer": "", - "package-type": "binary", - "bundled": "bundled", - "splitted": "unsplitted", - "tidy": "disable", - "with_coverage": false - }, - { - "compiler": "clang-10-aarch64", - "build-type": "", - "sanitizer": "", - "package-type": "binary", - "bundled": "bundled", - "splitted": "unsplitted", - "tidy": "disable", - "with_coverage": false - }, - { - "compiler": "clang-10-freebsd", - "build-type": "", - "sanitizer": "", - "package-type": "binary", - "bundled": "bundled", - "splitted": "unsplitted", - "tidy": "disable", - "with_coverage": false +{ + "build_config": [ + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "alien_pkgs": true, + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "performance", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "address", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "undefined", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "thread", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "memory", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "debug", + "sanitizer": "", + "package-type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "gcc-9", + "build-type": "", + "sanitizer": "", + "package-type": "deb", + "bundled": "unbundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "splitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10-darwin", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10-aarch64", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + }, + { + "compiler": "clang-10-freebsd", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false + } + ], + "tests_config": { + "Functional stateful tests (address)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (thread)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (memory)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (ubsan)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "undefined", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (debug)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "debug", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (release)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateful tests (release, DatabaseAtomic)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (address)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (thread)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (memory)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (ubsan)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "undefined", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (debug)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "debug", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (release)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (unbundled)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "unbundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (release, polymorphic parts enabled)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Functional stateless tests (release, DatabaseAtomic)": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (address)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (thread)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (undefined)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "undefined", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Stress test (memory)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Integration tests (asan)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Integration tests (thread)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Integration tests (release)": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Compatibility check": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Split build smoke test": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "binary", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "splitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Testflows check": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Unit tests release gcc": { + "required_build_properties": { + "compiler": "gcc-9", + "package_type": "binary", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Unit tests release clang": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "binary", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Unit tests ASAN": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "binary", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Unit tests MSAN": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "binary", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Unit tests TSAN": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "binary", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "Unit tests UBSAN": { + "required_build_properties": { + "compiler": "clang-10", + "package_type": "binary", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + } } -] +} From df8dde109c698a40f5f281831cfa738eae2533ec Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 7 Sep 2020 16:40:16 +0300 Subject: [PATCH 098/298] Remove tests config --- tests/ci/tests_config.json | 242 ------------------------------------- 1 file changed, 242 deletions(-) delete mode 100644 tests/ci/tests_config.json diff --git a/tests/ci/tests_config.json b/tests/ci/tests_config.json deleted file mode 100644 index 481de51d08b..00000000000 --- a/tests/ci/tests_config.json +++ /dev/null @@ -1,242 +0,0 @@ -{ - "Functional stateful tests (address)": { - "required_build_properties": { - "compiler": "clang-10", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "address", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateful tests (thread)": { - "required_build_properties": { - "compiler": "clang-10", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "thread", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateful tests (memory)": { - "required_build_properties": { - "compiler": "clang-10", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "memory", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateful tests (ubsan)": { - "required_build_properties": { - "compiler": "clang-10", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "undefined", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateful tests (debug)": { - "required_build_properties": { - "compiler": "clang-10", - "package_type": "deb", - "build_type": "debug", - "sanitizer": "none", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateful tests (release)": { - "required_build_properties": { - "compiler": "gcc-9", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "none", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateful tests (release, DatabaseAtomic)": { - "required_build_properties": { - "compiler": "gcc-9", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "none", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateless tests (address)": { - "required_build_properties": { - "compiler": "clang-10", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "address", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateless tests (thread)": { - "required_build_properties": { - "compiler": "clang-10", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "thread", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateless tests (memory)": { - "required_build_properties": { - "compiler": "clang-10", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "memory", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateless tests (ubsan)": { - "required_build_properties": { - "compiler": "clang-10", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "undefined", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateless tests (debug)": { - "required_build_properties": { - "compiler": "clang-10", - "package_type": "deb", - "build_type": "debug", - "sanitizer": "none", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateless tests (release)": { - "required_build_properties": { - "compiler": "gcc-9", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "none", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateless tests (unbundled)": { - "required_build_properties": { - "compiler": "gcc-9", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "none", - "bundled": "unbundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateless tests (release, polymorphic parts enabled)": { - "required_build_properties": { - "compiler": "gcc-9", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "none", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Functional stateless tests (release, DatabaseAtomic)": { - "required_build_properties": { - "compiler": "gcc-9", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "none", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Stress test (address)": { - "required_build_properties": { - "compiler": "clang-10", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "address", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Stress test (thread)": { - "required_build_properties": { - "compiler": "clang-10", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "thread", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Stress test (undefined)": { - "required_build_properties": { - "compiler": "clang-10", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "undefined", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - }, - "Stress test (memory)": { - "required_build_properties": { - "compiler": "clang-10", - "package_type": "deb", - "build_type": "relwithdebuginfo", - "sanitizer": "memory", - "bundled": "bundled", - "splitted": "unsplitted", - "clang-tidy": "disable", - "with_coverage": false - } - } -} From 54e6257070b4560f3ab69813514b7c98d7a7a917 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 7 Sep 2020 17:23:20 +0300 Subject: [PATCH 099/298] Better name --- tests/ci/{build_config.json => ci_config.json} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/ci/{build_config.json => ci_config.json} (100%) diff --git a/tests/ci/build_config.json b/tests/ci/ci_config.json similarity index 100% rename from tests/ci/build_config.json rename to tests/ci/ci_config.json From 7b8f8acd381480b584a1b32cc295830d5b6d09a4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 7 Sep 2020 17:47:14 +0300 Subject: [PATCH 100/298] Bump all versions to gcc-10 --- tests/ci/ci_config.json | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index 02c96b085da..dbb7be0e438 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -1,7 +1,7 @@ { "build_config": [ { - "compiler": "gcc-9", + "compiler": "gcc-10", "build-type": "", "sanitizer": "", "package-type": "deb", @@ -12,7 +12,7 @@ "with_coverage": false }, { - "compiler": "gcc-9", + "compiler": "gcc-10", "build-type": "", "sanitizer": "", "package-type": "performance", @@ -22,7 +22,7 @@ "with_coverage": false }, { - "compiler": "gcc-9", + "compiler": "gcc-10", "build-type": "", "sanitizer": "", "package-type": "binary", @@ -92,7 +92,7 @@ "with_coverage": false }, { - "compiler": "gcc-9", + "compiler": "gcc-10", "build-type": "", "sanitizer": "", "package-type": "deb", @@ -215,7 +215,7 @@ }, "Functional stateful tests (release)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -227,7 +227,7 @@ }, "Functional stateful tests (release, DatabaseAtomic)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -299,7 +299,7 @@ }, "Functional stateless tests (release)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -311,7 +311,7 @@ }, "Functional stateless tests (unbundled)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -323,7 +323,7 @@ }, "Functional stateless tests (release, polymorphic parts enabled)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -335,7 +335,7 @@ }, "Functional stateless tests (release, DatabaseAtomic)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -431,7 +431,7 @@ }, "Compatibility check": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -455,7 +455,7 @@ }, "Testflows check": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -467,7 +467,7 @@ }, "Unit tests release gcc": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "binary", "build_type": "relwithdebuginfo", "sanitizer": "none", From 661d9bdb4c1489d6a9c5c8f0ae6d06bb5480a2b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 19:03:36 +0300 Subject: [PATCH 101/298] Skip test --- tests/queries/skip_list.json | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index adfc5f0e582..0aa98499d42 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -18,7 +18,8 @@ "00152_insert_different_granularity", "00151_replace_partition_with_different_granularity", "00157_cache_dictionary", - "01193_metadata_loading" + "01193_metadata_loading", + "01474_executable_dictionary" /// informational stderr from sanitizer at start ], "address-sanitizer": [ "00281", From 2c04b0a8e67d1aeefdbb523ac4f8cd321b83a347 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 7 Sep 2020 19:07:34 +0300 Subject: [PATCH 102/298] comment added --- .../Formats/Impl/JSONCompactEachRowRowInputFormat.h | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index 593f297108c..6845b2974ab 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -12,7 +12,12 @@ namespace DB class ReadBuffer; -/** A stream for reading data in JSONCompactEachRow- formats +/** A stream for reading data in a bunch of formats: + * - JSONCompactEachRow + * - JSONCompactEachRowWithNamesAndTypes + * - JSONCompactStringsEachRow + * - JSONCompactStringsEachRowWithNamesAndTypes + * */ class JSONCompactEachRowRowInputFormat : public IRowInputFormat { @@ -54,7 +59,9 @@ private: /// This is for the correct exceptions in skipping unknown fields. std::vector names_of_columns; + /// For *WithNamesAndTypes formats. bool with_names; + /// For JSONCompactString* formats. bool yield_strings; }; From 105f704efe163e702c5d23bd9b164c8a28df7657 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 7 Sep 2020 19:20:00 +0300 Subject: [PATCH 103/298] Back to gcc-9 --- tests/ci/ci_config.json | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index dbb7be0e438..02c96b085da 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -1,7 +1,7 @@ { "build_config": [ { - "compiler": "gcc-10", + "compiler": "gcc-9", "build-type": "", "sanitizer": "", "package-type": "deb", @@ -12,7 +12,7 @@ "with_coverage": false }, { - "compiler": "gcc-10", + "compiler": "gcc-9", "build-type": "", "sanitizer": "", "package-type": "performance", @@ -22,7 +22,7 @@ "with_coverage": false }, { - "compiler": "gcc-10", + "compiler": "gcc-9", "build-type": "", "sanitizer": "", "package-type": "binary", @@ -92,7 +92,7 @@ "with_coverage": false }, { - "compiler": "gcc-10", + "compiler": "gcc-9", "build-type": "", "sanitizer": "", "package-type": "deb", @@ -215,7 +215,7 @@ }, "Functional stateful tests (release)": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "gcc-9", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -227,7 +227,7 @@ }, "Functional stateful tests (release, DatabaseAtomic)": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "gcc-9", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -299,7 +299,7 @@ }, "Functional stateless tests (release)": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "gcc-9", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -311,7 +311,7 @@ }, "Functional stateless tests (unbundled)": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "gcc-9", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -323,7 +323,7 @@ }, "Functional stateless tests (release, polymorphic parts enabled)": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "gcc-9", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -335,7 +335,7 @@ }, "Functional stateless tests (release, DatabaseAtomic)": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "gcc-9", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -431,7 +431,7 @@ }, "Compatibility check": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "gcc-9", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -455,7 +455,7 @@ }, "Testflows check": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "gcc-9", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -467,7 +467,7 @@ }, "Unit tests release gcc": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "gcc-9", "package_type": "binary", "build_type": "relwithdebuginfo", "sanitizer": "none", From a56d42de67496404fb507d05c2d399012fd479ce Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 7 Sep 2020 20:55:06 +0300 Subject: [PATCH 104/298] fix arcadia --- src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h | 2 -- src/Processors/ya.make | 3 --- 2 files changed, 5 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index 6845b2974ab..4077eb6e008 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -1,7 +1,5 @@ #pragma once -#pragma once - #include #include #include diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 27893674859..30de38fedbd 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -31,9 +31,6 @@ SRCS( Formats/Impl/JSONEachRowRowOutputFormat.cpp Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp Formats/Impl/JSONRowOutputFormat.cpp - Formats/Impl/JSONStringsEachRowRowInputFormat.cpp - Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp - Formats/Impl/JSONStringsRowOutputFormat.cpp Formats/Impl/MarkdownRowOutputFormat.cpp Formats/Impl/MsgPackRowInputFormat.cpp Formats/Impl/MsgPackRowOutputFormat.cpp From b3eafc1106819099afc980f8033c9e430a564cec Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Mon, 7 Sep 2020 21:00:37 +0300 Subject: [PATCH 105/298] hide symbols in nameless namespace --- .../AggregateFunctionArray.cpp | 5 + .../AggregateFunctionDistinct.cpp | 6 +- .../AggregateFunctionForEach.cpp | 5 + .../AggregateFunctionMerge.cpp | 5 + .../AggregateFunctionNull.cpp | 5 + .../AggregateFunctionOrFill.cpp | 4 + .../AggregateFunctionResample.cpp | 5 + .../AggregateFunctionState.cpp | 5 + src/Functions/CRC.cpp | 5 + src/Functions/abs.cpp | 4 + src/Functions/acos.cpp | 4 + src/Functions/addressToLine.cpp | 5 + src/Functions/addressToSymbol.cpp | 5 + src/Functions/appendTrailingCharIfAbsent.cpp | 4 + src/Functions/asin.cpp | 4 + src/Functions/assumeNotNull.cpp | 3 + src/Functions/atan.cpp | 4 + src/Functions/bar.cpp | 4 + src/Functions/bitAnd.cpp | 5 + src/Functions/bitBoolMaskAnd.cpp | 72 ++++++------ src/Functions/bitBoolMaskOr.cpp | 72 ++++++------ src/Functions/bitCount.cpp | 5 + src/Functions/bitNot.cpp | 5 + src/Functions/bitOr.cpp | 5 + src/Functions/bitRotateLeft.cpp | 5 + src/Functions/bitRotateRight.cpp | 5 + src/Functions/bitShiftLeft.cpp | 5 + src/Functions/bitShiftRight.cpp | 5 + src/Functions/bitSwapLastTwo.cpp | 108 +++++++++--------- src/Functions/bitTest.cpp | 5 + src/Functions/bitTestAll.cpp | 4 + src/Functions/bitTestAny.cpp | 4 + src/Functions/bitWrapperFunc.cpp | 80 +++++++------ src/Functions/bitXor.cpp | 5 + src/Functions/blockNumber.cpp | 3 + src/Functions/blockSerializedSize.cpp | 3 + src/Functions/blockSize.cpp | 3 + src/Functions/buildId.cpp | 3 + src/Functions/caseWithExpression.cpp | 5 + src/Functions/cbrt.cpp | 4 + src/Functions/coalesce.cpp | 3 + src/Functions/concat.cpp | 3 + src/Functions/convertCharset.cpp | 3 + src/Functions/cos.cpp | 4 + src/Functions/countDigits.cpp | 4 + src/Functions/currentDatabase.cpp | 3 + src/Functions/currentUser.cpp | 3 + src/Functions/dateDiff.cpp | 5 + src/Functions/date_trunc.cpp | 3 + src/Functions/defaultValueOfArgumentType.cpp | 3 + src/Functions/defaultValueOfTypeName.cpp | 4 + src/Functions/demange.cpp | 5 + src/Functions/divide.cpp | 5 + src/Functions/dumpColumnStructure.cpp | 3 + src/Functions/e.cpp | 4 + src/Functions/empty.cpp | 4 + src/Functions/erf.cpp | 4 + src/Functions/erfc.cpp | 4 + src/Functions/evalMLMethod.cpp | 17 +-- src/Functions/exp.cpp | 4 + src/Functions/exp10.cpp | 5 +- src/Functions/exp2.cpp | 4 + src/Functions/extract.cpp | 4 + src/Functions/extractGroups.cpp | 5 +- src/Functions/filesystem.cpp | 3 + src/Functions/finalizeAggregation.cpp | 4 +- src/Functions/formatDateTime.cpp | 7 +- src/Functions/formatRow.cpp | 4 + src/Functions/formatString.cpp | 5 + src/Functions/fuzzBits.cpp | 3 +- src/Functions/gcd.cpp | 6 +- src/Functions/geoToH3.cpp | 4 + src/Functions/geohashDecode.cpp | 4 +- src/Functions/geohashEncode.cpp | 4 + src/Functions/geohashesInBox.cpp | 5 + src/Functions/getMacro.cpp | 5 +- src/Functions/getScalar.cpp | 5 +- src/Functions/getSetting.cpp | 5 +- src/Functions/getSizeOfEnumType.cpp | 4 +- src/Functions/globalVariable.cpp | 4 +- src/Functions/greatest.cpp | 4 + src/Functions/h3EdgeAngle.cpp | 4 + src/Functions/h3EdgeLengthM.cpp | 5 +- src/Functions/h3GetBaseCell.cpp | 5 + src/Functions/h3GetResolution.cpp | 5 + src/Functions/h3HexAreaM2.cpp | 5 +- src/Functions/h3IndexesAreNeighbors.cpp | 5 + src/Functions/h3IsValid.cpp | 5 + src/Functions/h3ToChildren.cpp | 5 +- src/Functions/h3ToParent.cpp | 5 +- src/Functions/h3ToString.cpp | 5 + src/Functions/h3kRing.cpp | 4 + src/Functions/hasColumnInTable.cpp | 4 +- src/Functions/hasThreadFuzzer.cpp | 3 + src/Functions/hasToken.cpp | 4 + src/Functions/hasTokenCaseInsensitive.cpp | 4 + src/Functions/hostName.cpp | 3 + src/Functions/identity.cpp | 3 + src/Functions/if.cpp | 5 +- src/Functions/ifNotFinite.cpp | 3 + src/Functions/ifNull.cpp | 3 + src/Functions/ignore.cpp | 3 + src/Functions/ilike.cpp | 11 +- src/Functions/in.cpp | 8 +- src/Functions/initializeAggregation.cpp | 4 +- src/Functions/intDiv.cpp | 5 + src/Functions/intDivOrZero.cpp | 4 + src/Functions/intExp10.cpp | 6 +- src/Functions/intExp2.cpp | 5 + src/Functions/isConstant.cpp | 3 + src/Functions/isDecimalOverflow.cpp | 5 +- src/Functions/isFinite.cpp | 3 + src/Functions/isInfinite.cpp | 3 + src/Functions/isNaN.cpp | 3 + src/Functions/isNotNull.cpp | 4 + src/Functions/isNull.cpp | 3 + src/Functions/isZeroOrNull.cpp | 5 +- src/Functions/jumpConsistentHash.cpp | 5 +- src/Functions/lcm.cpp | 6 +- src/Functions/least.cpp | 4 + src/Functions/lengthUTF8.cpp | 5 +- src/Functions/lgamma.cpp | 4 + src/Functions/like.cpp | 11 +- src/Functions/log.cpp | 8 +- src/Functions/log10.cpp | 4 + src/Functions/log2.cpp | 4 + src/Functions/lowCardinalityIndices.cpp | 5 +- src/Functions/lowCardinalityKeys.cpp | 4 +- src/Functions/lower.cpp | 4 + src/Functions/lowerUTF8.cpp | 4 + src/Functions/match.cpp | 4 + src/Functions/materialize.cpp | 3 + src/Functions/minus.cpp | 4 + src/Functions/modulo.cpp | 6 +- src/Functions/moduloOrZero.cpp | 4 + src/Functions/multiFuzzyMatchAllIndices.cpp | 4 + src/Functions/multiFuzzyMatchAny.cpp | 4 + src/Functions/multiFuzzyMatchAnyIndex.cpp | 4 + src/Functions/multiIf.cpp | 6 +- src/Functions/multiMatchAllIndices.cpp | 4 + src/Functions/multiMatchAny.cpp | 4 + src/Functions/multiMatchAnyIndex.cpp | 4 + src/Functions/multiSearchAllPositions.cpp | 4 + ...multiSearchAllPositionsCaseInsensitive.cpp | 4 + ...iSearchAllPositionsCaseInsensitiveUTF8.cpp | 4 + src/Functions/multiSearchAllPositionsUTF8.cpp | 4 + src/Functions/multiSearchAny.cpp | 4 + .../multiSearchAnyCaseInsensitive.cpp | 4 + .../multiSearchAnyCaseInsensitiveUTF8.cpp | 4 + src/Functions/multiSearchAnyUTF8.cpp | 4 + src/Functions/multiSearchFirstIndex.cpp | 4 + .../multiSearchFirstIndexCaseInsensitive.cpp | 4 + ...ltiSearchFirstIndexCaseInsensitiveUTF8.cpp | 4 + src/Functions/multiSearchFirstIndexUTF8.cpp | 4 + src/Functions/multiSearchFirstPosition.cpp | 4 + ...ultiSearchFirstPositionCaseInsensitive.cpp | 4 + ...SearchFirstPositionCaseInsensitiveUTF8.cpp | 4 + .../multiSearchFirstPositionUTF8.cpp | 4 + src/Functions/multiply.cpp | 4 + src/Functions/negate.cpp | 4 + src/Functions/neighbor.cpp | 5 + src/Functions/normalizeQuery.cpp | 1 - src/Functions/normalizedQueryHash.cpp | 1 - src/Functions/notEmpty.cpp | 4 + src/Functions/notILike.cpp | 10 +- src/Functions/notLike.cpp | 5 +- src/Functions/now.cpp | 7 +- src/Functions/now64.cpp | 6 +- src/Functions/nullIf.cpp | 3 + src/Functions/pi.cpp | 4 + src/Functions/plus.cpp | 4 + src/Functions/pointInEllipses.cpp | 5 +- src/Functions/pointInPolygon.cpp | 4 +- src/Functions/position.cpp | 4 + src/Functions/positionCaseInsensitive.cpp | 4 + src/Functions/positionCaseInsensitiveUTF8.cpp | 4 + src/Functions/positionUTF8.cpp | 4 + src/Functions/pow.cpp | 4 + src/Functions/rand.cpp | 4 + src/Functions/rand64.cpp | 4 + src/Functions/randConstant.cpp | 6 +- src/Functions/randomFixedString.cpp | 4 + src/Functions/randomPrintableASCII.cpp | 4 + src/Functions/randomString.cpp | 4 + src/Functions/randomStringUTF8.cpp | 4 + src/Functions/regexpQuoteMeta.cpp | 8 +- src/Functions/reinterpretAsFixedString.cpp | 5 +- src/Functions/reinterpretAsString.cpp | 7 +- src/Functions/reinterpretStringAs.cpp | 7 +- src/Functions/repeat.cpp | 4 + src/Functions/replaceAll.cpp | 4 + src/Functions/replaceOne.cpp | 4 + src/Functions/replaceRegexpAll.cpp | 4 + src/Functions/replaceRegexpOne.cpp | 4 + src/Functions/replicate.cpp | 5 +- src/Functions/reverse.cpp | 4 +- src/Functions/reverseUTF8.cpp | 5 +- src/Functions/roundAge.cpp | 4 + src/Functions/roundDuration.cpp | 4 + src/Functions/roundToExp2.cpp | 6 +- src/Functions/rowNumberInAllBlocks.cpp | 3 + src/Functions/rowNumberInBlock.cpp | 4 + src/Functions/runningAccumulate.cpp | 4 +- src/Functions/sigmoid.cpp | 4 + src/Functions/sin.cpp | 4 + src/Functions/sqrt.cpp | 4 + src/Functions/stringToH3.cpp | 4 + src/Functions/substring.cpp | 4 + src/Functions/sumburConsistentHash.cpp | 4 + src/Functions/tan.cpp | 4 + src/Functions/tanh.cpp | 9 +- src/Functions/tgamma.cpp | 4 + src/Functions/throwIf.cpp | 4 +- src/Functions/timeSlots.cpp | 6 +- src/Functions/timezone.cpp | 4 +- src/Functions/toColumnTypeName.cpp | 3 + src/Functions/toLowCardinality.cpp | 3 + src/Functions/toNullable.cpp | 3 + src/Functions/toStartOfInterval.cpp | 2 +- src/Functions/toTimeZone.cpp | 6 +- src/Functions/toTypeName.cpp | 3 + src/Functions/toUnixTimestamp64Micro.cpp | 4 + src/Functions/toUnixTimestamp64Milli.cpp | 4 + src/Functions/toUnixTimestamp64Nano.cpp | 4 + src/Functions/toValidUTF8.cpp | 5 + src/Functions/today.cpp | 4 + src/Functions/transform.cpp | 5 +- src/Functions/trim.cpp | 6 +- src/Functions/tuple.cpp | 5 +- src/Functions/tupleElement.cpp | 4 +- src/Functions/upper.cpp | 4 + src/Functions/upperUTF8.cpp | 4 + 232 files changed, 1095 insertions(+), 250 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionArray.cpp b/src/AggregateFunctions/AggregateFunctionArray.cpp index 7fe4f1f448b..d0f17da5aa4 100644 --- a/src/AggregateFunctions/AggregateFunctionArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionArray.cpp @@ -12,6 +12,9 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + class AggregateFunctionCombinatorArray final : public IAggregateFunctionCombinator { public: @@ -45,6 +48,8 @@ public: } }; +} + void registerAggregateFunctionCombinatorArray(AggregateFunctionCombinatorFactory & factory) { factory.registerCombinator(std::make_shared()); diff --git a/src/AggregateFunctions/AggregateFunctionDistinct.cpp b/src/AggregateFunctions/AggregateFunctionDistinct.cpp index 4d89e8fb199..8ad37f49797 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinct.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinct.cpp @@ -6,12 +6,14 @@ namespace DB { - namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ + class AggregateFunctionCombinatorDistinct final : public IAggregateFunctionCombinator { public: @@ -56,6 +58,8 @@ public: } }; +} + void registerAggregateFunctionCombinatorDistinct(AggregateFunctionCombinatorFactory & factory) { factory.registerCombinator(std::make_shared()); diff --git a/src/AggregateFunctions/AggregateFunctionForEach.cpp b/src/AggregateFunctions/AggregateFunctionForEach.cpp index 693bc6839fa..6e0365fc04b 100644 --- a/src/AggregateFunctions/AggregateFunctionForEach.cpp +++ b/src/AggregateFunctions/AggregateFunctionForEach.cpp @@ -12,6 +12,9 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + class AggregateFunctionCombinatorForEach final : public IAggregateFunctionCombinator { public: @@ -42,6 +45,8 @@ public: } }; +} + void registerAggregateFunctionCombinatorForEach(AggregateFunctionCombinatorFactory & factory) { factory.registerCombinator(std::make_shared()); diff --git a/src/AggregateFunctions/AggregateFunctionMerge.cpp b/src/AggregateFunctions/AggregateFunctionMerge.cpp index 2ce3f0e11f6..17157d21bd1 100644 --- a/src/AggregateFunctions/AggregateFunctionMerge.cpp +++ b/src/AggregateFunctions/AggregateFunctionMerge.cpp @@ -13,6 +13,9 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ + class AggregateFunctionCombinatorMerge final : public IAggregateFunctionCombinator { public: @@ -55,6 +58,8 @@ public: } }; +} + void registerAggregateFunctionCombinatorMerge(AggregateFunctionCombinatorFactory & factory) { factory.registerCombinator(std::make_shared()); diff --git a/src/AggregateFunctions/AggregateFunctionNull.cpp b/src/AggregateFunctions/AggregateFunctionNull.cpp index c88d1e7f24c..f584ae1f34c 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.cpp +++ b/src/AggregateFunctions/AggregateFunctionNull.cpp @@ -15,6 +15,9 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + class AggregateFunctionCombinatorNull final : public IAggregateFunctionCombinator { public: @@ -119,6 +122,8 @@ public: } }; +} + void registerAggregateFunctionCombinatorNull(AggregateFunctionCombinatorFactory & factory) { factory.registerCombinator(std::make_shared()); diff --git a/src/AggregateFunctions/AggregateFunctionOrFill.cpp b/src/AggregateFunctions/AggregateFunctionOrFill.cpp index ce8fc8d9ca5..af107e26ca9 100644 --- a/src/AggregateFunctions/AggregateFunctionOrFill.cpp +++ b/src/AggregateFunctions/AggregateFunctionOrFill.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ template class AggregateFunctionCombinatorOrFill final : public IAggregateFunctionCombinator @@ -32,6 +34,8 @@ public: } }; +} + void registerAggregateFunctionCombinatorOrFill(AggregateFunctionCombinatorFactory & factory) { factory.registerCombinator(std::make_shared>()); diff --git a/src/AggregateFunctions/AggregateFunctionResample.cpp b/src/AggregateFunctions/AggregateFunctionResample.cpp index 389c9048918..b81fb442f27 100644 --- a/src/AggregateFunctions/AggregateFunctionResample.cpp +++ b/src/AggregateFunctions/AggregateFunctionResample.cpp @@ -13,6 +13,9 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ + class AggregateFunctionCombinatorResample final : public IAggregateFunctionCombinator { public: @@ -93,6 +96,8 @@ public: } }; +} + void registerAggregateFunctionCombinatorResample(AggregateFunctionCombinatorFactory & factory) { factory.registerCombinator(std::make_shared()); diff --git a/src/AggregateFunctions/AggregateFunctionState.cpp b/src/AggregateFunctions/AggregateFunctionState.cpp index 9d1c677c0ff..348d8ba44dd 100644 --- a/src/AggregateFunctions/AggregateFunctionState.cpp +++ b/src/AggregateFunctions/AggregateFunctionState.cpp @@ -13,6 +13,9 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace +{ + class AggregateFunctionCombinatorState final : public IAggregateFunctionCombinator { public: @@ -33,6 +36,8 @@ public: } }; +} + void registerAggregateFunctionCombinatorState(AggregateFunctionCombinatorFactory & factory) { factory.registerCombinator(std::make_shared()); diff --git a/src/Functions/CRC.cpp b/src/Functions/CRC.cpp index 96edf9a0d8e..6083e5ef16f 100644 --- a/src/Functions/CRC.cpp +++ b/src/Functions/CRC.cpp @@ -72,6 +72,9 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + template struct CRCFunctionWrapper { @@ -127,6 +130,8 @@ using FunctionCRC32IEEE = FunctionCRC; // Uses CRC-64-ECMA polynomial using FunctionCRC64ECMA = FunctionCRC; +} + template void registerFunctionCRCImpl(FunctionFactory & factory) { diff --git a/src/Functions/abs.cpp b/src/Functions/abs.cpp index f0c530e0e8f..deb69d40035 100644 --- a/src/Functions/abs.cpp +++ b/src/Functions/abs.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ template struct AbsImpl @@ -34,6 +36,8 @@ struct AbsImpl struct NameAbs { static constexpr auto name = "abs"; }; using FunctionAbs = FunctionUnaryArithmetic; +} + template <> struct FunctionUnaryArithmeticMonotonicity { static bool has() { return true; } diff --git a/src/Functions/acos.cpp b/src/Functions/acos.cpp index 61e213acabf..62e68b5c17b 100644 --- a/src/Functions/acos.cpp +++ b/src/Functions/acos.cpp @@ -4,10 +4,14 @@ namespace DB { +namespace +{ struct AcosName { static constexpr auto name = "acos"; }; using FunctionAcos = FunctionMathUnary>; +} + void registerFunctionAcos(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/addressToLine.cpp b/src/Functions/addressToLine.cpp index b5a6fcfb30e..432761e8d28 100644 --- a/src/Functions/addressToLine.cpp +++ b/src/Functions/addressToLine.cpp @@ -29,6 +29,9 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ + class FunctionAddressToLine : public IFunction { public: @@ -144,6 +147,8 @@ private: } }; +} + void registerFunctionAddressToLine(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/addressToSymbol.cpp b/src/Functions/addressToSymbol.cpp index 077b4f9a80b..d2df064bf35 100644 --- a/src/Functions/addressToSymbol.cpp +++ b/src/Functions/addressToSymbol.cpp @@ -21,6 +21,9 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ + class FunctionAddressToSymbol : public IFunction { public: @@ -86,6 +89,8 @@ public: } }; +} + void registerFunctionAddressToSymbol(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/appendTrailingCharIfAbsent.cpp b/src/Functions/appendTrailingCharIfAbsent.cpp index b69edb718a4..67a3cbabe6d 100644 --- a/src/Functions/appendTrailingCharIfAbsent.cpp +++ b/src/Functions/appendTrailingCharIfAbsent.cpp @@ -17,6 +17,8 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace +{ class FunctionAppendTrailingCharIfAbsent : public IFunction { @@ -109,6 +111,8 @@ private: } }; +} + void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/asin.cpp b/src/Functions/asin.cpp index cccd3fc05d4..92391fdef70 100644 --- a/src/Functions/asin.cpp +++ b/src/Functions/asin.cpp @@ -4,10 +4,14 @@ namespace DB { +namespace +{ struct AsinName { static constexpr auto name = "asin"; }; using FunctionAsin = FunctionMathUnary>; +} + void registerFunctionAsin(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/assumeNotNull.cpp b/src/Functions/assumeNotNull.cpp index e2b543d1be8..331e6a62341 100644 --- a/src/Functions/assumeNotNull.cpp +++ b/src/Functions/assumeNotNull.cpp @@ -7,6 +7,8 @@ namespace DB { +namespace +{ /// Implements the function assumeNotNull which takes 1 argument and works as follows: /// - if the argument is a nullable column, return its embedded column; @@ -49,6 +51,7 @@ public: } }; +} void registerFunctionAssumeNotNull(FunctionFactory & factory) { diff --git a/src/Functions/atan.cpp b/src/Functions/atan.cpp index 00e871b9a84..be0af8a9108 100644 --- a/src/Functions/atan.cpp +++ b/src/Functions/atan.cpp @@ -4,10 +4,14 @@ namespace DB { +namespace +{ struct AtanName { static constexpr auto name = "atan"; }; using FunctionAtan = FunctionMathUnary>; +} + void registerFunctionAtan(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/bar.cpp b/src/Functions/bar.cpp index a80cddfb5e3..748c33025a4 100644 --- a/src/Functions/bar.cpp +++ b/src/Functions/bar.cpp @@ -19,6 +19,9 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + /** bar(x, min, max, width) - draws a strip from the number of characters proportional to (x - min) and equal to width for x == max. * Returns a string with nice Unicode-art bar with resolution of 1/8 part of symbol. */ @@ -160,6 +163,7 @@ private: } }; +} void registerFunctionBar(FunctionFactory & factory) { diff --git a/src/Functions/bitAnd.cpp b/src/Functions/bitAnd.cpp index c6e2b0a6c88..89c2758bc6a 100644 --- a/src/Functions/bitAnd.cpp +++ b/src/Functions/bitAnd.cpp @@ -9,6 +9,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace +{ + template struct BitAndImpl { @@ -36,6 +39,8 @@ struct BitAndImpl struct NameBitAnd { static constexpr auto name = "bitAnd"; }; using FunctionBitAnd = FunctionBinaryArithmetic; +} + void registerFunctionBitAnd(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/bitBoolMaskAnd.cpp b/src/Functions/bitBoolMaskAnd.cpp index 44dadad64c0..dd46fa8b1b1 100644 --- a/src/Functions/bitBoolMaskAnd.cpp +++ b/src/Functions/bitBoolMaskAnd.cpp @@ -5,44 +5,50 @@ namespace DB { - namespace ErrorCodes +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +/// Working with UInt8: last bit = can be true, previous = can be false (Like src/Storages/MergeTree/BoolMask.h). +/// This function provides "AND" operation for BoolMasks. +/// Returns: "can be true" = A."can be true" AND B."can be true" +/// "can be false" = A."can be false" OR B."can be false" +template +struct BitBoolMaskAndImpl +{ + using ResultType = UInt8; + static const constexpr bool allow_fixed_string = false; + + template + static inline Result apply([[maybe_unused]] A left, [[maybe_unused]] B right) { - extern const int BAD_ARGUMENTS; + // Should be a logical error, but this function is callable from SQL. + // Need to investigate this. + if constexpr (!std::is_same_v || !std::is_same_v) + throw DB::Exception("It's a bug! Only UInt8 type is supported by __bitBoolMaskAnd.", ErrorCodes::BAD_ARGUMENTS); + + auto left_bits = littleBits(left); + auto right_bits = littleBits(right); + return static_cast((left_bits & right_bits & 1) | ((((left_bits >> 1) | (right_bits >> 1)) & 1) << 1)); } - /// Working with UInt8: last bit = can be true, previous = can be false (Like src/Storages/MergeTree/BoolMask.h). - /// This function provides "AND" operation for BoolMasks. - /// Returns: "can be true" = A."can be true" AND B."can be true" - /// "can be false" = A."can be false" OR B."can be false" - template - struct BitBoolMaskAndImpl - { - using ResultType = UInt8; - static const constexpr bool allow_fixed_string = false; - - template - static inline Result apply([[maybe_unused]] A left, [[maybe_unused]] B right) - { - // Should be a logical error, but this function is callable from SQL. - // Need to investigate this. - if constexpr (!std::is_same_v || !std::is_same_v) - throw DB::Exception("It's a bug! Only UInt8 type is supported by __bitBoolMaskAnd.", ErrorCodes::BAD_ARGUMENTS); - - auto left_bits = littleBits(left); - auto right_bits = littleBits(right); - return static_cast((left_bits & right_bits & 1) | ((((left_bits >> 1) | (right_bits >> 1)) & 1) << 1)); - } - #if USE_EMBEDDED_COMPILER - static constexpr bool compilable = false; + static constexpr bool compilable = false; #endif - }; +}; - struct NameBitBoolMaskAnd { static constexpr auto name = "__bitBoolMaskAnd"; }; - using FunctionBitBoolMaskAnd = FunctionBinaryArithmetic; +struct NameBitBoolMaskAnd { static constexpr auto name = "__bitBoolMaskAnd"; }; +using FunctionBitBoolMaskAnd = FunctionBinaryArithmetic; + +} + +void registerFunctionBitBoolMaskAnd(FunctionFactory & factory) +{ + factory.registerFunction(); +} - void registerFunctionBitBoolMaskAnd(FunctionFactory & factory) - { - factory.registerFunction(); - } } diff --git a/src/Functions/bitBoolMaskOr.cpp b/src/Functions/bitBoolMaskOr.cpp index cfce7f27829..e86c7dcda8e 100644 --- a/src/Functions/bitBoolMaskOr.cpp +++ b/src/Functions/bitBoolMaskOr.cpp @@ -5,44 +5,50 @@ namespace DB { - namespace ErrorCodes +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +/// Working with UInt8: last bit = can be true, previous = can be false (Like src/Storages/MergeTree/BoolMask.h). +/// This function provides "OR" operation for BoolMasks. +/// Returns: "can be true" = A."can be true" OR B."can be true" +/// "can be false" = A."can be false" AND B."can be false" +template +struct BitBoolMaskOrImpl +{ + using ResultType = UInt8; + static const constexpr bool allow_fixed_string = false; + + template + static inline Result apply([[maybe_unused]] A left, [[maybe_unused]] B right) { - extern const int BAD_ARGUMENTS; + if constexpr (!std::is_same_v || !std::is_same_v) + // Should be a logical error, but this function is callable from SQL. + // Need to investigate this. + throw DB::Exception("It's a bug! Only UInt8 type is supported by __bitBoolMaskOr.", ErrorCodes::BAD_ARGUMENTS); + + auto left_bits = littleBits(left); + auto right_bits = littleBits(right); + return static_cast(((left_bits | right_bits) & 1) | ((((left_bits >> 1) & (right_bits >> 1)) & 1) << 1)); } - /// Working with UInt8: last bit = can be true, previous = can be false (Like src/Storages/MergeTree/BoolMask.h). - /// This function provides "OR" operation for BoolMasks. - /// Returns: "can be true" = A."can be true" OR B."can be true" - /// "can be false" = A."can be false" AND B."can be false" - template - struct BitBoolMaskOrImpl - { - using ResultType = UInt8; - static const constexpr bool allow_fixed_string = false; - - template - static inline Result apply([[maybe_unused]] A left, [[maybe_unused]] B right) - { - if constexpr (!std::is_same_v || !std::is_same_v) - // Should be a logical error, but this function is callable from SQL. - // Need to investigate this. - throw DB::Exception("It's a bug! Only UInt8 type is supported by __bitBoolMaskOr.", ErrorCodes::BAD_ARGUMENTS); - - auto left_bits = littleBits(left); - auto right_bits = littleBits(right); - return static_cast(((left_bits | right_bits) & 1) | ((((left_bits >> 1) & (right_bits >> 1)) & 1) << 1)); - } - #if USE_EMBEDDED_COMPILER - static constexpr bool compilable = false; + static constexpr bool compilable = false; #endif - }; +}; - struct NameBitBoolMaskOr { static constexpr auto name = "__bitBoolMaskOr"; }; - using FunctionBitBoolMaskOr = FunctionBinaryArithmetic; +struct NameBitBoolMaskOr { static constexpr auto name = "__bitBoolMaskOr"; }; +using FunctionBitBoolMaskOr = FunctionBinaryArithmetic; + +} + +void registerFunctionBitBoolMaskOr(FunctionFactory & factory) +{ + factory.registerFunction(); +} - void registerFunctionBitBoolMaskOr(FunctionFactory & factory) - { - factory.registerFunction(); - } } diff --git a/src/Functions/bitCount.cpp b/src/Functions/bitCount.cpp index 73df2c680da..0b7af6eca7f 100644 --- a/src/Functions/bitCount.cpp +++ b/src/Functions/bitCount.cpp @@ -6,6 +6,9 @@ namespace DB { +namespace +{ + template struct BitCountImpl { @@ -37,6 +40,8 @@ struct BitCountImpl struct NameBitCount { static constexpr auto name = "bitCount"; }; using FunctionBitCount = FunctionUnaryArithmetic; +} + /// The function has no ranges of monotonicity. template <> struct FunctionUnaryArithmeticMonotonicity { diff --git a/src/Functions/bitNot.cpp b/src/Functions/bitNot.cpp index 0a08b3aaaeb..050d8dc3e3d 100644 --- a/src/Functions/bitNot.cpp +++ b/src/Functions/bitNot.cpp @@ -10,6 +10,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace +{ + template struct BitNotImpl { @@ -36,6 +39,8 @@ struct BitNotImpl struct NameBitNot { static constexpr auto name = "bitNot"; }; using FunctionBitNot = FunctionUnaryArithmetic; +} + template <> struct FunctionUnaryArithmeticMonotonicity { static bool has() { return false; } diff --git a/src/Functions/bitOr.cpp b/src/Functions/bitOr.cpp index b8e77128216..0f339b328d8 100644 --- a/src/Functions/bitOr.cpp +++ b/src/Functions/bitOr.cpp @@ -8,6 +8,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace +{ + template struct BitOrImpl { @@ -35,6 +38,8 @@ struct BitOrImpl struct NameBitOr { static constexpr auto name = "bitOr"; }; using FunctionBitOr = FunctionBinaryArithmetic; +} + void registerFunctionBitOr(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/bitRotateLeft.cpp b/src/Functions/bitRotateLeft.cpp index a6975468c1e..5d52494eb7d 100644 --- a/src/Functions/bitRotateLeft.cpp +++ b/src/Functions/bitRotateLeft.cpp @@ -9,6 +9,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace +{ + template struct BitRotateLeftImpl { @@ -42,6 +45,8 @@ struct BitRotateLeftImpl struct NameBitRotateLeft { static constexpr auto name = "bitRotateLeft"; }; using FunctionBitRotateLeft = FunctionBinaryArithmetic; +} + void registerFunctionBitRotateLeft(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/bitRotateRight.cpp b/src/Functions/bitRotateRight.cpp index 71d7385bbdf..7cda0b4890b 100644 --- a/src/Functions/bitRotateRight.cpp +++ b/src/Functions/bitRotateRight.cpp @@ -9,6 +9,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace +{ + template struct BitRotateRightImpl { @@ -41,6 +44,8 @@ struct BitRotateRightImpl struct NameBitRotateRight { static constexpr auto name = "bitRotateRight"; }; using FunctionBitRotateRight = FunctionBinaryArithmetic; +} + void registerFunctionBitRotateRight(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/bitShiftLeft.cpp b/src/Functions/bitShiftLeft.cpp index d42082d7778..59d236ac6af 100644 --- a/src/Functions/bitShiftLeft.cpp +++ b/src/Functions/bitShiftLeft.cpp @@ -9,6 +9,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace +{ + template struct BitShiftLeftImpl { @@ -41,6 +44,8 @@ struct BitShiftLeftImpl struct NameBitShiftLeft { static constexpr auto name = "bitShiftLeft"; }; using FunctionBitShiftLeft = FunctionBinaryArithmetic; +} + void registerFunctionBitShiftLeft(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/bitShiftRight.cpp b/src/Functions/bitShiftRight.cpp index 249a86d6961..fe7def0b56b 100644 --- a/src/Functions/bitShiftRight.cpp +++ b/src/Functions/bitShiftRight.cpp @@ -9,6 +9,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace +{ + template struct BitShiftRightImpl { @@ -41,6 +44,8 @@ struct BitShiftRightImpl struct NameBitShiftRight { static constexpr auto name = "bitShiftRight"; }; using FunctionBitShiftRight = FunctionBinaryArithmetic; +} + void registerFunctionBitShiftRight(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/bitSwapLastTwo.cpp b/src/Functions/bitSwapLastTwo.cpp index 07c4db4ebe7..4103511085d 100644 --- a/src/Functions/bitSwapLastTwo.cpp +++ b/src/Functions/bitSwapLastTwo.cpp @@ -4,60 +4,66 @@ namespace DB { - namespace ErrorCodes +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +/// Working with UInt8: last bit = can be true, previous = can be false (Like src/Storages/MergeTree/BoolMask.h). +/// This function provides "NOT" operation for BoolMasks by swapping last two bits ("can be true" <-> "can be false"). +template +struct BitSwapLastTwoImpl +{ + using ResultType = UInt8; + static constexpr const bool allow_fixed_string = false; + + static inline ResultType NO_SANITIZE_UNDEFINED apply([[maybe_unused]] A a) { - extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; + if constexpr (!std::is_same_v) + // Should be a logical error, but this function is callable from SQL. + // Need to investigate this. + throw DB::Exception("It's a bug! Only UInt8 type is supported by __bitSwapLastTwo.", ErrorCodes::BAD_ARGUMENTS); + + auto little_bits = littleBits(a); + return static_cast(((little_bits & 1) << 1) | ((little_bits >> 1) & 1)); } - /// Working with UInt8: last bit = can be true, previous = can be false (Like src/Storages/MergeTree/BoolMask.h). - /// This function provides "NOT" operation for BoolMasks by swapping last two bits ("can be true" <-> "can be false"). - template - struct BitSwapLastTwoImpl - { - using ResultType = UInt8; - static constexpr const bool allow_fixed_string = false; - - static inline ResultType NO_SANITIZE_UNDEFINED apply([[maybe_unused]] A a) - { - if constexpr (!std::is_same_v) - // Should be a logical error, but this function is callable from SQL. - // Need to investigate this. - throw DB::Exception("It's a bug! Only UInt8 type is supported by __bitSwapLastTwo.", ErrorCodes::BAD_ARGUMENTS); - - auto little_bits = littleBits(a); - return static_cast(((little_bits & 1) << 1) | ((little_bits >> 1) & 1)); - } - #if USE_EMBEDDED_COMPILER - static constexpr bool compilable = true; +static constexpr bool compilable = true; + +static inline llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * arg, bool) +{ + if (!arg->getType()->isIntegerTy()) + throw Exception("__bitSwapLastTwo expected an integral type", ErrorCodes::LOGICAL_ERROR); + return b.CreateOr( + b.CreateShl(b.CreateAnd(arg, 1), 1), + b.CreateAnd(b.CreateLShr(arg, 1), 1) + ); +} +#endif +}; + +struct NameBitSwapLastTwo { static constexpr auto name = "__bitSwapLastTwo"; }; +using FunctionBitSwapLastTwo = FunctionUnaryArithmetic; + +} + +template <> struct FunctionUnaryArithmeticMonotonicity +{ + static bool has() { return false; } + static IFunction::Monotonicity get(const Field &, const Field &) + { + return {}; + } +}; + +void registerFunctionBitSwapLastTwo(FunctionFactory & factory) +{ + factory.registerFunction(); +} - static inline llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * arg, bool) - { - if (!arg->getType()->isIntegerTy()) - throw Exception("__bitSwapLastTwo expected an integral type", ErrorCodes::LOGICAL_ERROR); - return b.CreateOr( - b.CreateShl(b.CreateAnd(arg, 1), 1), - b.CreateAnd(b.CreateLShr(arg, 1), 1) - ); - } -#endif - }; - - struct NameBitSwapLastTwo { static constexpr auto name = "__bitSwapLastTwo"; }; - using FunctionBitSwapLastTwo = FunctionUnaryArithmetic; - - template <> struct FunctionUnaryArithmeticMonotonicity - { - static bool has() { return false; } - static IFunction::Monotonicity get(const Field &, const Field &) - { - return {}; - } - }; - - void registerFunctionBitSwapLastTwo(FunctionFactory & factory) - { - factory.registerFunction(); - } } diff --git a/src/Functions/bitTest.cpp b/src/Functions/bitTest.cpp index f34e300d675..9c9f16d87c4 100644 --- a/src/Functions/bitTest.cpp +++ b/src/Functions/bitTest.cpp @@ -10,6 +10,9 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +namespace +{ + template struct BitTestImpl { @@ -33,6 +36,8 @@ struct BitTestImpl struct NameBitTest { static constexpr auto name = "bitTest"; }; using FunctionBitTest = FunctionBinaryArithmetic; +} + void registerFunctionBitTest(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/bitTestAll.cpp b/src/Functions/bitTestAll.cpp index a359ea0b699..901fd600106 100644 --- a/src/Functions/bitTestAll.cpp +++ b/src/Functions/bitTestAll.cpp @@ -3,6 +3,8 @@ namespace DB { +namespace +{ struct BitTestAllImpl { @@ -13,6 +15,8 @@ struct BitTestAllImpl struct NameBitTestAll { static constexpr auto name = "bitTestAll"; }; using FunctionBitTestAll = FunctionBitTestMany; +} + void registerFunctionBitTestAll(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/bitTestAny.cpp b/src/Functions/bitTestAny.cpp index b6601dc55fd..6874e74e2be 100644 --- a/src/Functions/bitTestAny.cpp +++ b/src/Functions/bitTestAny.cpp @@ -3,6 +3,8 @@ namespace DB { +namespace +{ struct BitTestAnyImpl { @@ -13,6 +15,8 @@ struct BitTestAnyImpl struct NameBitTestAny { static constexpr auto name = "bitTestAny"; }; using FunctionBitTestAny = FunctionBitTestMany; +} + void registerFunctionBitTestAny(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/bitWrapperFunc.cpp b/src/Functions/bitWrapperFunc.cpp index 140163d0926..8395865d554 100644 --- a/src/Functions/bitWrapperFunc.cpp +++ b/src/Functions/bitWrapperFunc.cpp @@ -4,49 +4,53 @@ namespace DB { - namespace ErrorCodes +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +/// Working with UInt8: last bit = can be true, previous = can be false (Like src/Storages/MergeTree/BoolMask.h). +/// This function wraps bool atomic functions +/// and transforms their boolean return value to the BoolMask ("can be false" and "can be true" bits). +template +struct BitWrapperFuncImpl +{ + using ResultType = UInt8; + static constexpr const bool allow_fixed_string = false; + + static inline ResultType NO_SANITIZE_UNDEFINED apply(A a [[maybe_unused]]) { - extern const int BAD_ARGUMENTS; + // Should be a logical error, but this function is callable from SQL. + // Need to investigate this. + if constexpr (!is_integer_v) + throw DB::Exception("It's a bug! Only integer types are supported by __bitWrapperFunc.", ErrorCodes::BAD_ARGUMENTS); + return a == 0 ? static_cast(0b10) : static_cast(0b1); } - /// Working with UInt8: last bit = can be true, previous = can be false (Like src/Storages/MergeTree/BoolMask.h). - /// This function wraps bool atomic functions - /// and transforms their boolean return value to the BoolMask ("can be false" and "can be true" bits). - template - struct BitWrapperFuncImpl - { - using ResultType = UInt8; - static constexpr const bool allow_fixed_string = false; - - static inline ResultType NO_SANITIZE_UNDEFINED apply(A a [[maybe_unused]]) - { - // Should be a logical error, but this function is callable from SQL. - // Need to investigate this. - if constexpr (!is_integer_v) - throw DB::Exception("It's a bug! Only integer types are supported by __bitWrapperFunc.", ErrorCodes::BAD_ARGUMENTS); - return a == 0 ? static_cast(0b10) : static_cast(0b1); - } - #if USE_EMBEDDED_COMPILER - static constexpr bool compilable = false; + static constexpr bool compilable = false; #endif - }; +}; - struct NameBitWrapperFunc { static constexpr auto name = "__bitWrapperFunc"; }; - using FunctionBitWrapperFunc = FunctionUnaryArithmetic; - - template <> struct FunctionUnaryArithmeticMonotonicity - { - static bool has() { return false; } - static IFunction::Monotonicity get(const Field &, const Field &) - { - return {}; - } - }; - - void registerFunctionBitWrapperFunc(FunctionFactory & factory) - { - factory.registerFunction(); - } +struct NameBitWrapperFunc { static constexpr auto name = "__bitWrapperFunc"; }; +using FunctionBitWrapperFunc = FunctionUnaryArithmetic; } + +template <> struct FunctionUnaryArithmeticMonotonicity +{ + static bool has() { return false; } + static IFunction::Monotonicity get(const Field &, const Field &) + { + return {}; + } +}; + +void registerFunctionBitWrapperFunc(FunctionFactory & factory) +{ + factory.registerFunction(); +} +} diff --git a/src/Functions/bitXor.cpp b/src/Functions/bitXor.cpp index 188cad84f81..3d323fde8bb 100644 --- a/src/Functions/bitXor.cpp +++ b/src/Functions/bitXor.cpp @@ -8,6 +8,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace +{ + template struct BitXorImpl { @@ -35,6 +38,8 @@ struct BitXorImpl struct NameBitXor { static constexpr auto name = "bitXor"; }; using FunctionBitXor = FunctionBinaryArithmetic; +} + void registerFunctionBitXor(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/blockNumber.cpp b/src/Functions/blockNumber.cpp index e83850803f6..74d33b71fcf 100644 --- a/src/Functions/blockNumber.cpp +++ b/src/Functions/blockNumber.cpp @@ -7,6 +7,8 @@ namespace DB { +namespace +{ /** Incremental block number among calls of this function. */ class FunctionBlockNumber : public IFunction @@ -56,6 +58,7 @@ public: } }; +} void registerFunctionBlockNumber(FunctionFactory & factory) { diff --git a/src/Functions/blockSerializedSize.cpp b/src/Functions/blockSerializedSize.cpp index 47a221dc391..b14accf19da 100644 --- a/src/Functions/blockSerializedSize.cpp +++ b/src/Functions/blockSerializedSize.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ /// Returns size on disk for *block* (without taking into account compression). class FunctionBlockSerializedSize : public IFunction @@ -60,6 +62,7 @@ public: } }; +} void registerFunctionBlockSerializedSize(FunctionFactory & factory) { diff --git a/src/Functions/blockSize.cpp b/src/Functions/blockSize.cpp index 8fe304a17f9..65977bfded6 100644 --- a/src/Functions/blockSize.cpp +++ b/src/Functions/blockSize.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ /** blockSize() - get the block size in number of rows. */ @@ -47,6 +49,7 @@ public: } }; +} void registerFunctionBlockSize(FunctionFactory & factory) { diff --git a/src/Functions/buildId.cpp b/src/Functions/buildId.cpp index bf3640bf68f..117e1346f7b 100644 --- a/src/Functions/buildId.cpp +++ b/src/Functions/buildId.cpp @@ -9,6 +9,8 @@ namespace DB { +namespace +{ /** buildId() - returns the compiler build id of the running binary. */ @@ -42,6 +44,7 @@ public: } }; +} void registerFunctionBuildId(FunctionFactory & factory) { diff --git a/src/Functions/caseWithExpression.cpp b/src/Functions/caseWithExpression.cpp index eae1d7271c2..699db5cb090 100644 --- a/src/Functions/caseWithExpression.cpp +++ b/src/Functions/caseWithExpression.cpp @@ -12,6 +12,9 @@ namespace ErrorCodes extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; } +namespace +{ + /// Implements the CASE construction when it is /// provided an expression. Users should not call this function. class FunctionCaseWithExpression : public IFunction @@ -112,6 +115,8 @@ private: const Context & context; }; +} + void registerFunctionCaseWithExpression(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/cbrt.cpp b/src/Functions/cbrt.cpp index f12ae0a6504..94c8627c320 100644 --- a/src/Functions/cbrt.cpp +++ b/src/Functions/cbrt.cpp @@ -3,10 +3,14 @@ namespace DB { +namespace +{ struct CbrtName { static constexpr auto name = "cbrt"; }; using FunctionCbrt = FunctionMathUnary>; +} + void registerFunctionCbrt(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/coalesce.cpp b/src/Functions/coalesce.cpp index 2f755b8fd8d..05f70f7783b 100644 --- a/src/Functions/coalesce.cpp +++ b/src/Functions/coalesce.cpp @@ -12,6 +12,8 @@ namespace DB { +namespace +{ /// Implements the function coalesce which takes a set of arguments and /// returns the value of the leftmost non-null argument. If no such value is @@ -175,6 +177,7 @@ private: const Context & context; }; +} void registerFunctionCoalesce(FunctionFactory & factory) { diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index 12ab2d208a7..3c5c7d29e3f 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -25,6 +25,8 @@ namespace ErrorCodes using namespace GatherUtils; +namespace +{ template class ConcatImpl : public IFunction @@ -225,6 +227,7 @@ private: const Context & context; }; +} void registerFunctionsConcat(FunctionFactory & factory) { diff --git a/src/Functions/convertCharset.cpp b/src/Functions/convertCharset.cpp index 0bc2594bf37..9006dfd206a 100644 --- a/src/Functions/convertCharset.cpp +++ b/src/Functions/convertCharset.cpp @@ -30,6 +30,8 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } +namespace +{ /** convertCharset(s, from, to) * @@ -210,6 +212,7 @@ public: } }; +} void registerFunctionConvertCharset(FunctionFactory & factory) { diff --git a/src/Functions/cos.cpp b/src/Functions/cos.cpp index a047ea2c252..e18524dd56b 100644 --- a/src/Functions/cos.cpp +++ b/src/Functions/cos.cpp @@ -3,10 +3,14 @@ namespace DB { +namespace +{ struct CosName { static constexpr auto name = "cos"; }; using FunctionCos = FunctionMathUnary>; +} + void registerFunctionCos(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/countDigits.cpp b/src/Functions/countDigits.cpp index 9f60bdf9689..5130f0c463e 100644 --- a/src/Functions/countDigits.cpp +++ b/src/Functions/countDigits.cpp @@ -16,6 +16,9 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } +namespace +{ + /// Returns number of decimal digits you need to represent the value. /// For Decimal values takes in account their scales: calculates result over underlying int type which is (value * scale). /// countDigits(42) = 2, countDigits(42.000) = 5, countDigits(0.04200) = 4. @@ -136,6 +139,7 @@ private: } }; +} void registerFunctionCountDigits(FunctionFactory & factory) { diff --git a/src/Functions/currentDatabase.cpp b/src/Functions/currentDatabase.cpp index b6933051ac0..5be25c63ae6 100644 --- a/src/Functions/currentDatabase.cpp +++ b/src/Functions/currentDatabase.cpp @@ -7,6 +7,8 @@ namespace DB { +namespace +{ class FunctionCurrentDatabase : public IFunction { @@ -45,6 +47,7 @@ public: } }; +} void registerFunctionCurrentDatabase(FunctionFactory & factory) { diff --git a/src/Functions/currentUser.cpp b/src/Functions/currentUser.cpp index f51b92e68b8..ce4cd4a3f83 100644 --- a/src/Functions/currentUser.cpp +++ b/src/Functions/currentUser.cpp @@ -7,6 +7,8 @@ namespace DB { +namespace +{ class FunctionCurrentUser : public IFunction { @@ -45,6 +47,7 @@ public: } }; +} void registerFunctionCurrentUser(FunctionFactory & factory) { diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index a0263d80a5c..fea3b2c46ac 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -28,6 +28,9 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace +{ + /** dateDiff('unit', t1, t2, [timezone]) * t1 and t2 can be Date or DateTime * @@ -212,6 +215,8 @@ private: } }; +} + void registerFunctionDateDiff(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp index af92b59f941..f14241e1b86 100644 --- a/src/Functions/date_trunc.cpp +++ b/src/Functions/date_trunc.cpp @@ -17,6 +17,8 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace +{ class FunctionDateTrunc : public IFunction { @@ -159,6 +161,7 @@ private: mutable IntervalKind::Kind datepart_kind = IntervalKind::Kind::Second; }; +} void registerFunctionDateTrunc(FunctionFactory & factory) { diff --git a/src/Functions/defaultValueOfArgumentType.cpp b/src/Functions/defaultValueOfArgumentType.cpp index f9c906361f9..7bcb92b67f4 100644 --- a/src/Functions/defaultValueOfArgumentType.cpp +++ b/src/Functions/defaultValueOfArgumentType.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ /// Returns global default value for type of passed argument (example: 0 for numeric types, '' for String). class FunctionDefaultValueOfArgumentType : public IFunction @@ -47,6 +49,7 @@ public: } }; +} void registerFunctionDefaultValueOfArgumentType(FunctionFactory & factory) { diff --git a/src/Functions/defaultValueOfTypeName.cpp b/src/Functions/defaultValueOfTypeName.cpp index 40527b21935..8baea9b4501 100644 --- a/src/Functions/defaultValueOfTypeName.cpp +++ b/src/Functions/defaultValueOfTypeName.cpp @@ -13,6 +13,9 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } +namespace +{ + /// Returns global default value for type name (example: 0 for numeric types, '' for String). class FunctionDefaultValueOfTypeName : public IFunction { @@ -53,6 +56,7 @@ public: } }; +} void registerFunctionDefaultValueOfTypeName(FunctionFactory & factory) { diff --git a/src/Functions/demange.cpp b/src/Functions/demange.cpp index db525c4d0d1..24d792b6f4d 100644 --- a/src/Functions/demange.cpp +++ b/src/Functions/demange.cpp @@ -18,6 +18,9 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ + class FunctionDemangle : public IFunction { public: @@ -86,6 +89,8 @@ public: } }; +} + void registerFunctionDemangle(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/divide.cpp b/src/Functions/divide.cpp index cfc535320ed..178dd627b01 100644 --- a/src/Functions/divide.cpp +++ b/src/Functions/divide.cpp @@ -9,6 +9,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace +{ + template struct DivideFloatingImpl { @@ -40,6 +43,8 @@ struct DivideFloatingImpl struct NameDivide { static constexpr auto name = "divide"; }; using FunctionDivide = FunctionBinaryArithmetic; +} + void registerFunctionDivide(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/dumpColumnStructure.cpp b/src/Functions/dumpColumnStructure.cpp index 7f3d476ba4e..c7f54a72190 100644 --- a/src/Functions/dumpColumnStructure.cpp +++ b/src/Functions/dumpColumnStructure.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ /// Dump the structure of type and column. class FunctionDumpColumnStructure : public IFunction @@ -46,6 +48,7 @@ public: } }; +} void registerFunctionDumpColumnStructure(FunctionFactory & factory) { diff --git a/src/Functions/e.cpp b/src/Functions/e.cpp index cb4cd66b44c..c43bb7d572a 100644 --- a/src/Functions/e.cpp +++ b/src/Functions/e.cpp @@ -3,6 +3,8 @@ namespace DB { +namespace +{ struct EImpl { @@ -12,6 +14,8 @@ struct EImpl using FunctionE = FunctionMathConstFloat64; +} + void registerFunctionE(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/empty.cpp b/src/Functions/empty.cpp index dfc13253b48..552fce85de0 100644 --- a/src/Functions/empty.cpp +++ b/src/Functions/empty.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameEmpty { @@ -13,6 +15,8 @@ struct NameEmpty }; using FunctionEmpty = FunctionStringOrArrayToT, NameEmpty, UInt8>; +} + void registerFunctionEmpty(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/erf.cpp b/src/Functions/erf.cpp index 8ce6732213f..9c7ca637736 100644 --- a/src/Functions/erf.cpp +++ b/src/Functions/erf.cpp @@ -3,10 +3,14 @@ namespace DB { +namespace +{ struct ErfName { static constexpr auto name = "erf"; }; using FunctionErf = FunctionMathUnary>; +} + void registerFunctionErf(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/erfc.cpp b/src/Functions/erfc.cpp index cd7a36b6865..8e0b462ff8e 100644 --- a/src/Functions/erfc.cpp +++ b/src/Functions/erfc.cpp @@ -3,10 +3,14 @@ namespace DB { +namespace +{ struct ErfcName { static constexpr auto name = "erfc"; }; using FunctionErfc = FunctionMathUnary>; +} + void registerFunctionErfc(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/evalMLMethod.cpp b/src/Functions/evalMLMethod.cpp index f4c8ecf1c2b..b0912b8aac6 100644 --- a/src/Functions/evalMLMethod.cpp +++ b/src/Functions/evalMLMethod.cpp @@ -11,14 +11,15 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} - namespace ErrorCodes - { - extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - } - +namespace +{ /** finalizeAggregation(agg_state) - get the result from the aggregation state. * Takes state of aggregate function. Returns result of aggregation (finalized state). @@ -83,6 +84,8 @@ public: const Context & context; }; +} + void registerFunctionEvalMLMethod(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/exp.cpp b/src/Functions/exp.cpp index 550b3b9d3ab..c6eb3335f25 100644 --- a/src/Functions/exp.cpp +++ b/src/Functions/exp.cpp @@ -3,6 +3,8 @@ namespace DB { +namespace +{ struct ExpName { static constexpr auto name = "exp"; }; @@ -30,6 +32,8 @@ using FunctionExp = FunctionMathUnary; using FunctionExp = FunctionMathUnary>; #endif +} + void registerFunctionExp(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/exp10.cpp b/src/Functions/exp10.cpp index 230e2282ca6..4be7e15562e 100644 --- a/src/Functions/exp10.cpp +++ b/src/Functions/exp10.cpp @@ -4,11 +4,14 @@ namespace DB { +namespace +{ struct Exp10Name { static constexpr auto name = "exp10"; }; - using FunctionExp10 = FunctionMathUnary>; +} + void registerFunctionExp10(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/exp2.cpp b/src/Functions/exp2.cpp index e33d07b489f..8825b849346 100644 --- a/src/Functions/exp2.cpp +++ b/src/Functions/exp2.cpp @@ -3,10 +3,14 @@ namespace DB { +namespace +{ struct Exp2Name { static constexpr auto name = "exp2"; }; using FunctionExp2 = FunctionMathUnary>; +} + void registerFunctionExp2(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/extract.cpp b/src/Functions/extract.cpp index 695ba037a02..0296602d205 100644 --- a/src/Functions/extract.cpp +++ b/src/Functions/extract.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct ExtractImpl { @@ -61,6 +63,8 @@ struct NameExtract using FunctionExtract = FunctionsStringSearchToString; +} + void registerFunctionExtract(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/extractGroups.cpp b/src/Functions/extractGroups.cpp index 2146f8d72b9..61055b1652d 100644 --- a/src/Functions/extractGroups.cpp +++ b/src/Functions/extractGroups.cpp @@ -13,12 +13,13 @@ namespace DB { - namespace ErrorCodes { extern const int BAD_ARGUMENTS; } +namespace +{ /** Match all groups of given input string with given re, return array of arrays of matches. * @@ -103,6 +104,8 @@ public: } }; +} + void registerFunctionExtractGroups(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/filesystem.cpp b/src/Functions/filesystem.cpp index 25f4ada78fc..2f95f9b6e6d 100644 --- a/src/Functions/filesystem.cpp +++ b/src/Functions/filesystem.cpp @@ -7,6 +7,8 @@ namespace DB { +namespace +{ struct FilesystemAvailable { @@ -57,6 +59,7 @@ private: std::filesystem::space_info spaceinfo; }; +} void registerFunctionFilesystem(FunctionFactory & factory) { diff --git a/src/Functions/finalizeAggregation.cpp b/src/Functions/finalizeAggregation.cpp index 425b4e2079b..51afb4729dc 100644 --- a/src/Functions/finalizeAggregation.cpp +++ b/src/Functions/finalizeAggregation.cpp @@ -8,13 +8,14 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ /** finalizeAggregation(agg_state) - get the result from the aggregation state. * Takes state of aggregate function. Returns result of aggregation (finalized state). @@ -73,6 +74,7 @@ public: } }; +} void registerFunctionFinalizeAggregation(FunctionFactory & factory) { diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index c1140a21a9f..4a7d0b18fba 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -23,7 +23,6 @@ namespace DB { - namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; @@ -35,7 +34,7 @@ namespace ErrorCodes namespace { -// in private namespace to avoid GCC 9 error: "explicit specialization in non-namespace scope" + template struct ActionValueTypeMap {}; template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; @@ -50,7 +49,7 @@ template <> struct ActionValueTypeMap { using ActionValueTyp // TODO(vnemkov): once there is support for Int64 in LUT, make that Int64. // TODO(vnemkov): to add sub-second format instruction, make that DateTime64 and do some math in Action. template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; -} + /** formatDateTime(time, 'pattern') * Performs formatting of time, according to provided pattern. @@ -714,6 +713,8 @@ struct NameFromUnixTime using FunctionFormatDateTime = FunctionFormatDateTimeImpl; using FunctionFROM_UNIXTIME = FunctionFormatDateTimeImpl; +} + void registerFunctionFormatDateTime(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/formatRow.cpp b/src/Functions/formatRow.cpp index 9bd60f2fdc0..a8fce0144e2 100644 --- a/src/Functions/formatRow.cpp +++ b/src/Functions/formatRow.cpp @@ -21,6 +21,8 @@ namespace ErrorCodes extern const int UNKNOWN_FORMAT; } +namespace +{ /** formatRow(, x, y, ...) is a function that allows you to use RowOutputFormat over * several columns to generate a string per row, such as CSV, TSV, JSONEachRow, etc. @@ -112,6 +114,8 @@ private: const Context & context; }; +} + void registerFunctionFormatRow(FunctionFactory & factory) { factory.registerFunction>(); diff --git a/src/Functions/formatString.cpp b/src/Functions/formatString.cpp index 4c0ca01d207..cd727adcabc 100644 --- a/src/Functions/formatString.cpp +++ b/src/Functions/formatString.cpp @@ -22,6 +22,9 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ + template class FormatFunction : public IFunction { @@ -130,6 +133,8 @@ struct NameFormat }; using FunctionFormat = FormatFunction; +} + void registerFunctionFormat(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/fuzzBits.cpp b/src/Functions/fuzzBits.cpp index c5ef448e7f0..b379db3e8db 100644 --- a/src/Functions/fuzzBits.cpp +++ b/src/Functions/fuzzBits.cpp @@ -46,7 +46,6 @@ namespace ptr_out[i] = ptr_in[i] ^ mask; } } -} class FunctionFuzzBits : public IFunction @@ -143,6 +142,8 @@ public: } }; +} + void registerFunctionFuzzBits(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/gcd.cpp b/src/Functions/gcd.cpp index b5d1ed6e92c..244b25b194d 100644 --- a/src/Functions/gcd.cpp +++ b/src/Functions/gcd.cpp @@ -5,12 +5,14 @@ namespace DB { - namespace ErrorCodes { extern const int NOT_IMPLEMENTED; } +namespace +{ + template struct GCDImpl { @@ -40,6 +42,8 @@ struct GCDImpl struct NameGCD { static constexpr auto name = "gcd"; }; using FunctionGCD = FunctionBinaryArithmetic; +} + void registerFunctionGCD(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/geoToH3.cpp b/src/Functions/geoToH3.cpp index 924cdf68cb9..8bcec7b73c5 100644 --- a/src/Functions/geoToH3.cpp +++ b/src/Functions/geoToH3.cpp @@ -17,6 +17,9 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + /// Implements the function geoToH3 which takes 3 arguments (latitude, longitude and h3 resolution) /// and returns h3 index of this point class FunctionGeoToH3 : public IFunction @@ -83,6 +86,7 @@ public: } }; +} void registerFunctionGeoToH3(FunctionFactory & factory) { diff --git a/src/Functions/geohashDecode.cpp b/src/Functions/geohashDecode.cpp index f8e57d0feb1..bdfab59b91e 100644 --- a/src/Functions/geohashDecode.cpp +++ b/src/Functions/geohashDecode.cpp @@ -14,12 +14,13 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_COLUMN; } +namespace +{ // geohashDecode(string) => (lon float64, lat float64) class FunctionGeohashDecode : public IFunction @@ -89,6 +90,7 @@ public: } }; +} void registerFunctionGeohashDecode(FunctionFactory & factory) { diff --git a/src/Functions/geohashEncode.cpp b/src/Functions/geohashEncode.cpp index 99e7dd05e37..30448a5430a 100644 --- a/src/Functions/geohashEncode.cpp +++ b/src/Functions/geohashEncode.cpp @@ -20,6 +20,9 @@ namespace ErrorCodes extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } +namespace +{ + // geohashEncode(lon float32/64, lat float32/64, length UInt8) => string class FunctionGeohashEncode : public IFunction { @@ -127,6 +130,7 @@ public: } }; +} void registerFunctionGeohashEncode(FunctionFactory & factory) { diff --git a/src/Functions/geohashesInBox.cpp b/src/Functions/geohashesInBox.cpp index 7fc0b00fb57..b70f0cf02b3 100644 --- a/src/Functions/geohashesInBox.cpp +++ b/src/Functions/geohashesInBox.cpp @@ -22,6 +22,9 @@ extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int TOO_LARGE_ARRAY_SIZE; } +namespace +{ + class FunctionGeohashesInBox : public IFunction { public: @@ -172,6 +175,8 @@ public: } }; +} + void registerFunctionGeohashesInBox(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/getMacro.cpp b/src/Functions/getMacro.cpp index 06de09ea17c..e900678ae20 100644 --- a/src/Functions/getMacro.cpp +++ b/src/Functions/getMacro.cpp @@ -10,13 +10,15 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; } +namespace +{ + /** Get the value of macro from configuration file. * For example, it may be used as a sophisticated replacement for the function 'hostName' if servers have complicated hostnames * but you still need to distinguish them by some convenient names. @@ -75,6 +77,7 @@ public: } }; +} void registerFunctionGetMacro(FunctionFactory & factory) { diff --git a/src/Functions/getScalar.cpp b/src/Functions/getScalar.cpp index a9cf538000d..a989daf83fb 100644 --- a/src/Functions/getScalar.cpp +++ b/src/Functions/getScalar.cpp @@ -10,12 +10,14 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + /** Get scalar value of sub queries from query context via IAST::Hash. */ class FunctionGetScalar : public IFunction @@ -59,6 +61,7 @@ private: const Context & context; }; +} void registerFunctionGetScalar(FunctionFactory & factory) { diff --git a/src/Functions/getSetting.cpp b/src/Functions/getSetting.cpp index 7421aca81f3..c883931fbe0 100644 --- a/src/Functions/getSetting.cpp +++ b/src/Functions/getSetting.cpp @@ -9,13 +9,15 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; } +namespace +{ + /// Get the value of a setting. class FunctionGetSetting : public IFunction { @@ -58,6 +60,7 @@ private: const Context & context; }; +} void registerFunctionGetSetting(FunctionFactory & factory) { diff --git a/src/Functions/getSizeOfEnumType.cpp b/src/Functions/getSizeOfEnumType.cpp index 6db9a6c6837..54f8b12e819 100644 --- a/src/Functions/getSizeOfEnumType.cpp +++ b/src/Functions/getSizeOfEnumType.cpp @@ -7,12 +7,13 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ /// Returns number of fields in Enum data type of passed value. class FunctionGetSizeOfEnumType : public IFunction @@ -64,6 +65,7 @@ public: } }; +} void registerFunctionGetSizeOfEnumType(FunctionFactory & factory) { diff --git a/src/Functions/globalVariable.cpp b/src/Functions/globalVariable.cpp index 381651c30b9..4652ec39d23 100644 --- a/src/Functions/globalVariable.cpp +++ b/src/Functions/globalVariable.cpp @@ -13,12 +13,13 @@ namespace DB { - namespace ErrorCodes { extern const int BAD_ARGUMENTS; } +namespace +{ /** globalVariable('name') - takes constant string argument and returns the value of global variable with that name. * It is intended for compatibility with MySQL. @@ -81,6 +82,7 @@ private: {"max_allowed_packet", {std::make_shared(), 67108864}}, {"version", {std::make_shared(), "5.7.30"}}}; }; +} void registerFunctionGlobalVariable(FunctionFactory & factory) { diff --git a/src/Functions/greatest.cpp b/src/Functions/greatest.cpp index 39021935111..da1a372b0b2 100644 --- a/src/Functions/greatest.cpp +++ b/src/Functions/greatest.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ template struct GreatestBaseImpl @@ -58,6 +60,8 @@ using GreatestImpl = std::conditional_t; +} + void registerFunctionGreatest(FunctionFactory & factory) { factory.registerFunction>(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/h3EdgeAngle.cpp b/src/Functions/h3EdgeAngle.cpp index 4472c698d21..f1fbd1b9d72 100644 --- a/src/Functions/h3EdgeAngle.cpp +++ b/src/Functions/h3EdgeAngle.cpp @@ -18,6 +18,9 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; } +namespace +{ + class FunctionH3EdgeAngle : public IFunction { public: @@ -66,6 +69,7 @@ public: } }; +} void registerFunctionH3EdgeAngle(FunctionFactory & factory) { diff --git a/src/Functions/h3EdgeLengthM.cpp b/src/Functions/h3EdgeLengthM.cpp index cba0ed831c7..d4c9916e4b5 100644 --- a/src/Functions/h3EdgeLengthM.cpp +++ b/src/Functions/h3EdgeLengthM.cpp @@ -12,13 +12,15 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ARGUMENT_OUT_OF_BOUND; } +namespace +{ + // Average metric edge length of H3 hexagon. The edge length `e` for given resolution `res` can // be used for converting metric search radius `radius` to hexagon search ring size `k` that is // used by `H3kRing` function. For small enough search area simple flat approximation can be used, @@ -71,6 +73,7 @@ public: } }; +} void registerFunctionH3EdgeLengthM(FunctionFactory & factory) { diff --git a/src/Functions/h3GetBaseCell.cpp b/src/Functions/h3GetBaseCell.cpp index 15cd5d21c50..d8d3a85504b 100644 --- a/src/Functions/h3GetBaseCell.cpp +++ b/src/Functions/h3GetBaseCell.cpp @@ -14,6 +14,10 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; } + +namespace +{ + class FunctionH3GetBaseCell : public IFunction { public: @@ -58,6 +62,7 @@ public: } }; +} void registerFunctionH3GetBaseCell(FunctionFactory & factory) { diff --git a/src/Functions/h3GetResolution.cpp b/src/Functions/h3GetResolution.cpp index 52d2d987b5e..7692b3d0a98 100644 --- a/src/Functions/h3GetResolution.cpp +++ b/src/Functions/h3GetResolution.cpp @@ -14,6 +14,10 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; } + +namespace +{ + class FunctionH3GetResolution : public IFunction { public: @@ -58,6 +62,7 @@ public: } }; +} void registerFunctionH3GetResolution(FunctionFactory & factory) { diff --git a/src/Functions/h3HexAreaM2.cpp b/src/Functions/h3HexAreaM2.cpp index a2b76560a1d..7b12cc0201b 100644 --- a/src/Functions/h3HexAreaM2.cpp +++ b/src/Functions/h3HexAreaM2.cpp @@ -12,13 +12,15 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ARGUMENT_OUT_OF_BOUND; } +namespace +{ + class FunctionH3HexAreaM2 : public IFunction { public: @@ -66,6 +68,7 @@ public: } }; +} void registerFunctionH3HexAreaM2(FunctionFactory & factory) { diff --git a/src/Functions/h3IndexesAreNeighbors.cpp b/src/Functions/h3IndexesAreNeighbors.cpp index 2022ebd63d3..e1f69e30ecf 100644 --- a/src/Functions/h3IndexesAreNeighbors.cpp +++ b/src/Functions/h3IndexesAreNeighbors.cpp @@ -14,6 +14,10 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; } + +namespace +{ + class FunctionH3IndexesAreNeighbors : public IFunction { public: @@ -66,6 +70,7 @@ public: } }; +} void registerFunctionH3IndexesAreNeighbors(FunctionFactory & factory) { diff --git a/src/Functions/h3IsValid.cpp b/src/Functions/h3IsValid.cpp index a338ae0f131..60719718d75 100644 --- a/src/Functions/h3IsValid.cpp +++ b/src/Functions/h3IsValid.cpp @@ -14,6 +14,10 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; } + +namespace +{ + class FunctionH3IsValid : public IFunction { public: @@ -58,6 +62,7 @@ public: } }; +} void registerFunctionH3IsValid(FunctionFactory & factory) { diff --git a/src/Functions/h3ToChildren.cpp b/src/Functions/h3ToChildren.cpp index d9e402231f7..70add2c0f4b 100644 --- a/src/Functions/h3ToChildren.cpp +++ b/src/Functions/h3ToChildren.cpp @@ -17,7 +17,6 @@ static constexpr size_t MAX_ARRAY_SIZE = 1 << 30; namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; @@ -25,6 +24,9 @@ namespace ErrorCodes extern const int TOO_LARGE_ARRAY_SIZE; } +namespace +{ + class FunctionH3ToChildren : public IFunction { public: @@ -101,6 +103,7 @@ public: } }; +} void registerFunctionH3ToChildren(FunctionFactory & factory) { diff --git a/src/Functions/h3ToParent.cpp b/src/Functions/h3ToParent.cpp index 2f6a9f3264d..f5b4e106cba 100644 --- a/src/Functions/h3ToParent.cpp +++ b/src/Functions/h3ToParent.cpp @@ -12,13 +12,15 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ARGUMENT_OUT_OF_BOUND; } +namespace +{ + class FunctionH3ToParent : public IFunction { public: @@ -75,6 +77,7 @@ public: } }; +} void registerFunctionH3ToParent(FunctionFactory & factory) { diff --git a/src/Functions/h3ToString.cpp b/src/Functions/h3ToString.cpp index 31adbe56eca..5355adaff2a 100644 --- a/src/Functions/h3ToString.cpp +++ b/src/Functions/h3ToString.cpp @@ -14,6 +14,10 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; } + +namespace +{ + class FunctionH3ToString : public IFunction { public: @@ -74,6 +78,7 @@ public: } }; +} void registerFunctionH3ToString(FunctionFactory & factory) { diff --git a/src/Functions/h3kRing.cpp b/src/Functions/h3kRing.cpp index 9702edf7079..a1ea1e217f8 100644 --- a/src/Functions/h3kRing.cpp +++ b/src/Functions/h3kRing.cpp @@ -21,6 +21,9 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; } +namespace +{ + class FunctionH3KRing : public IFunction { public: @@ -97,6 +100,7 @@ public: } }; +} void registerFunctionH3KRing(FunctionFactory & factory) { diff --git a/src/Functions/hasColumnInTable.cpp b/src/Functions/hasColumnInTable.cpp index 258cbbac006..c69d1025740 100644 --- a/src/Functions/hasColumnInTable.cpp +++ b/src/Functions/hasColumnInTable.cpp @@ -12,7 +12,6 @@ namespace DB { - namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; @@ -20,6 +19,8 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; } +namespace +{ /** Usage: * hasColumnInTable(['hostname'[, 'username'[, 'password']],] 'database', 'table', 'column') @@ -140,6 +141,7 @@ void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers & block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, Field(has_column)); } +} void registerFunctionHasColumnInTable(FunctionFactory & factory) { diff --git a/src/Functions/hasThreadFuzzer.cpp b/src/Functions/hasThreadFuzzer.cpp index 0b0db50a37c..f16a4f34de3 100644 --- a/src/Functions/hasThreadFuzzer.cpp +++ b/src/Functions/hasThreadFuzzer.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ /** Returns whether Thread Fuzzer is effective. * It can be used in tests to prevent too long runs. @@ -40,6 +42,7 @@ public: } }; +} void registerFunctionHasThreadFuzzer(FunctionFactory & factory) { diff --git a/src/Functions/hasToken.cpp b/src/Functions/hasToken.cpp index ee04484ad54..f20edffbdd8 100644 --- a/src/Functions/hasToken.cpp +++ b/src/Functions/hasToken.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameHasToken { @@ -14,6 +16,8 @@ struct NameHasToken using FunctionHasToken = FunctionsStringSearch, NameHasToken>; +} + void registerFunctionHasToken(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/hasTokenCaseInsensitive.cpp b/src/Functions/hasTokenCaseInsensitive.cpp index c58df05d239..28f5b9e80c1 100644 --- a/src/Functions/hasTokenCaseInsensitive.cpp +++ b/src/Functions/hasTokenCaseInsensitive.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameHasTokenCaseInsensitive { @@ -15,6 +17,8 @@ struct NameHasTokenCaseInsensitive using FunctionHasTokenCaseInsensitive = FunctionsStringSearch, NameHasTokenCaseInsensitive>; +} + void registerFunctionHasTokenCaseInsensitive(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/hostName.cpp b/src/Functions/hostName.cpp index 3b463b0bab5..faa1c8a944f 100644 --- a/src/Functions/hostName.cpp +++ b/src/Functions/hostName.cpp @@ -7,6 +7,8 @@ namespace DB { +namespace +{ /// Get the host name. Is is constant on single server, but is not constant in distributed queries. class FunctionHostName : public IFunction @@ -50,6 +52,7 @@ public: } }; +} void registerFunctionHostName(FunctionFactory & factory) { diff --git a/src/Functions/identity.cpp b/src/Functions/identity.cpp index 5308c3d944f..86c56c4911f 100644 --- a/src/Functions/identity.cpp +++ b/src/Functions/identity.cpp @@ -4,6 +4,8 @@ namespace DB { +namespace +{ class FunctionIdentity : public IFunction { @@ -35,6 +37,7 @@ public: } }; +} void registerFunctionIdentity(FunctionFactory & factory) { diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 8bb04abe834..20848bede32 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -28,7 +28,6 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_COLUMN; @@ -36,6 +35,8 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +namespace +{ using namespace GatherUtils; @@ -1050,6 +1051,8 @@ public: } }; +} + void registerFunctionIf(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/ifNotFinite.cpp b/src/Functions/ifNotFinite.cpp index d806a974926..b9e04d733ae 100644 --- a/src/Functions/ifNotFinite.cpp +++ b/src/Functions/ifNotFinite.cpp @@ -8,6 +8,8 @@ namespace DB { +namespace +{ /// ifNotFinite(x, y) is equivalent to isFinite(x) ? x : y. class FunctionIfNotFinite : public IFunction @@ -63,6 +65,7 @@ private: const Context & context; }; +} void registerFunctionIfNotFinite(FunctionFactory & factory) { diff --git a/src/Functions/ifNull.cpp b/src/Functions/ifNull.cpp index e76378ef4e5..3d2b5b7210a 100644 --- a/src/Functions/ifNull.cpp +++ b/src/Functions/ifNull.cpp @@ -10,6 +10,8 @@ namespace DB { +namespace +{ /// Implements the function ifNull which takes 2 arguments and returns /// the value of the 1st argument if it is not null. Otherwise it returns @@ -92,6 +94,7 @@ private: const Context & context; }; +} void registerFunctionIfNull(FunctionFactory & factory) { diff --git a/src/Functions/ignore.cpp b/src/Functions/ignore.cpp index fe41d860291..61e42fc4eb7 100644 --- a/src/Functions/ignore.cpp +++ b/src/Functions/ignore.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ /** ignore(...) is a function that takes any arguments, and always returns 0. */ @@ -49,6 +51,7 @@ public: } }; +} void registerFunctionIgnore(FunctionFactory & factory) { diff --git a/src/Functions/ilike.cpp b/src/Functions/ilike.cpp index a39a907eff2..fc3e38daeba 100644 --- a/src/Functions/ilike.cpp +++ b/src/Functions/ilike.cpp @@ -4,21 +4,22 @@ namespace DB { +namespace +{ struct NameILike { static constexpr auto name = "ilike"; }; -namespace -{ - using ILikeImpl = MatchImpl; -} - +using ILikeImpl = MatchImpl; using FunctionILike = FunctionsStringSearch; +} + void registerFunctionILike(FunctionFactory & factory) { factory.registerFunction(); } + } diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index e4d503a2a93..a5eb2241f1b 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -12,12 +12,14 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_COLUMN; } +namespace +{ + /** in(x, set) - function for evaluating the IN * notIn(x, set) - and NOT IN. */ @@ -128,7 +130,7 @@ public: }; template -static void registerFunctionsInImpl(FunctionFactory & factory) +void registerFunctionsInImpl(FunctionFactory & factory) { factory.registerFunction>(); factory.registerFunction>(); @@ -140,6 +142,8 @@ static void registerFunctionsInImpl(FunctionFactory & factory) factory.registerFunction>(); } +} + void registerFunctionsIn(FunctionFactory & factory) { registerFunctionsInImpl(factory); diff --git a/src/Functions/initializeAggregation.cpp b/src/Functions/initializeAggregation.cpp index 85ee3e07969..8709b02d8e0 100644 --- a/src/Functions/initializeAggregation.cpp +++ b/src/Functions/initializeAggregation.cpp @@ -14,7 +14,6 @@ namespace DB { - namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; @@ -23,6 +22,8 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace +{ class FunctionInitializeAggregation : public IFunction { @@ -152,6 +153,7 @@ void FunctionInitializeAggregation::executeImpl(Block & block, const ColumnNumbe block.getByPosition(result).column = std::move(result_holder); } +} void registerFunctionInitializeAggregation(FunctionFactory & factory) { diff --git a/src/Functions/intDiv.cpp b/src/Functions/intDiv.cpp index 7e34f106147..a08525813b1 100644 --- a/src/Functions/intDiv.cpp +++ b/src/Functions/intDiv.cpp @@ -15,6 +15,9 @@ namespace ErrorCodes extern const int ILLEGAL_DIVISION; } +namespace +{ + /// Optimizations for integer division by a constant. template @@ -83,6 +86,8 @@ struct DivideIntegralByConstantImpl * Can be expanded to all possible combinations, but more code is needed. */ +} + template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; diff --git a/src/Functions/intDivOrZero.cpp b/src/Functions/intDivOrZero.cpp index 64b6994d438..cae901518c0 100644 --- a/src/Functions/intDivOrZero.cpp +++ b/src/Functions/intDivOrZero.cpp @@ -4,6 +4,8 @@ namespace DB { +namespace +{ template struct DivideIntegralOrZeroImpl @@ -28,6 +30,8 @@ struct DivideIntegralOrZeroImpl struct NameIntDivOrZero { static constexpr auto name = "intDivOrZero"; }; using FunctionIntDivOrZero = FunctionBinaryArithmetic; +} + void registerFunctionIntDivOrZero(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/intExp10.cpp b/src/Functions/intExp10.cpp index 863a0822765..b1964701ad7 100644 --- a/src/Functions/intExp10.cpp +++ b/src/Functions/intExp10.cpp @@ -5,12 +5,14 @@ namespace DB { - namespace ErrorCodes { extern const int NOT_IMPLEMENTED; } +namespace +{ + template struct IntExp10Impl { @@ -34,6 +36,8 @@ struct NameIntExp10 { static constexpr auto name = "intExp10"; }; /// Assumed to be injective for the purpose of query optimization, but in fact it is not injective because of possible overflow. using FunctionIntExp10 = FunctionUnaryArithmetic; +} + template <> struct FunctionUnaryArithmeticMonotonicity { static bool has() { return true; } diff --git a/src/Functions/intExp2.cpp b/src/Functions/intExp2.cpp index 453c5798f10..c87a6e31852 100644 --- a/src/Functions/intExp2.cpp +++ b/src/Functions/intExp2.cpp @@ -11,6 +11,9 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +namespace +{ + template struct IntExp2Impl { @@ -41,6 +44,8 @@ struct IntExp2Impl struct NameIntExp2 { static constexpr auto name = "intExp2"; }; using FunctionIntExp2 = FunctionUnaryArithmetic; +} + template <> struct FunctionUnaryArithmeticMonotonicity { static bool has() { return true; } diff --git a/src/Functions/isConstant.cpp b/src/Functions/isConstant.cpp index 88dd983f47b..fc3f78cd058 100644 --- a/src/Functions/isConstant.cpp +++ b/src/Functions/isConstant.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ /// Returns 1 if and only if the argument is constant expression. /// This function exists for development, debugging and demonstration purposes. @@ -42,6 +44,7 @@ public: } }; +} void registerFunctionIsConstant(FunctionFactory & factory) { diff --git a/src/Functions/isDecimalOverflow.cpp b/src/Functions/isDecimalOverflow.cpp index dbd57101a64..dc1b0d6300a 100644 --- a/src/Functions/isDecimalOverflow.cpp +++ b/src/Functions/isDecimalOverflow.cpp @@ -11,7 +11,6 @@ namespace DB { - namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; @@ -19,6 +18,9 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } +namespace +{ + /// Returns 1 if and Decimal value has more digits then it's Precision allow, 0 otherwise. /// Precision could be set as second argument or omitted. If ommited function uses Decimal presicion of the first argument. class FunctionIsDecimalOverflow : public IFunction @@ -142,6 +144,7 @@ private: } }; +} void registerFunctionIsDecimalOverflow(FunctionFactory & factory) { diff --git a/src/Functions/isFinite.cpp b/src/Functions/isFinite.cpp index e898ddc8304..72cdc4d3a4f 100644 --- a/src/Functions/isFinite.cpp +++ b/src/Functions/isFinite.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct IsFiniteImpl { @@ -35,6 +37,7 @@ struct IsFiniteImpl using FunctionIsFinite = FunctionNumericPredicate; +} void registerFunctionIsFinite(FunctionFactory & factory) { diff --git a/src/Functions/isInfinite.cpp b/src/Functions/isInfinite.cpp index 17208373cb3..12abb8eab6a 100644 --- a/src/Functions/isInfinite.cpp +++ b/src/Functions/isInfinite.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct IsInfiniteImpl { @@ -31,6 +33,7 @@ struct IsInfiniteImpl using FunctionIsInfinite = FunctionNumericPredicate; +} void registerFunctionIsInfinite(FunctionFactory & factory) { diff --git a/src/Functions/isNaN.cpp b/src/Functions/isNaN.cpp index 5146f7cfc0d..2e35e8cfbb5 100644 --- a/src/Functions/isNaN.cpp +++ b/src/Functions/isNaN.cpp @@ -4,6 +4,8 @@ namespace DB { +namespace +{ struct IsNaNImpl { @@ -18,6 +20,7 @@ struct IsNaNImpl using FunctionIsNaN = FunctionNumericPredicate; +} void registerFunctionIsNaN(FunctionFactory & factory) { diff --git a/src/Functions/isNotNull.cpp b/src/Functions/isNotNull.cpp index b5d856f31ff..12543e2a751 100644 --- a/src/Functions/isNotNull.cpp +++ b/src/Functions/isNotNull.cpp @@ -9,6 +9,8 @@ namespace DB { +namespace +{ /// Implements the function isNotNull which returns true if a value /// is not null, false otherwise. @@ -60,6 +62,8 @@ public: } }; +} + void registerFunctionIsNotNull(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/isNull.cpp b/src/Functions/isNull.cpp index 1451f8d8f4f..91d98f1fcd7 100644 --- a/src/Functions/isNull.cpp +++ b/src/Functions/isNull.cpp @@ -8,6 +8,8 @@ namespace DB { +namespace +{ /// Implements the function isNull which returns true if a value /// is null, false otherwise. @@ -53,6 +55,7 @@ public: } }; +} void registerFunctionIsNull(FunctionFactory & factory) { diff --git a/src/Functions/isZeroOrNull.cpp b/src/Functions/isZeroOrNull.cpp index 02d97181016..00b84c5713d 100644 --- a/src/Functions/isZeroOrNull.cpp +++ b/src/Functions/isZeroOrNull.cpp @@ -9,13 +9,15 @@ namespace DB { - namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; } +namespace +{ + /// Returns 1 if argument is zero or NULL. /// It can be used to negate filter in WHERE condition. /// "WHERE isZeroOrNull(expr)" will return exactly the same rows that "WHERE expr" will filter out. @@ -107,6 +109,7 @@ private: } }; +} void registerFunctionIsZeroOrNull(FunctionFactory & factory) { diff --git a/src/Functions/jumpConsistentHash.cpp b/src/Functions/jumpConsistentHash.cpp index b1a3109c066..32535cfcd6b 100644 --- a/src/Functions/jumpConsistentHash.cpp +++ b/src/Functions/jumpConsistentHash.cpp @@ -4,6 +4,8 @@ namespace DB { +namespace +{ /// Code from https://arxiv.org/pdf/1406.2294.pdf static inline int32_t JumpConsistentHash(uint64_t key, int32_t num_buckets) @@ -35,10 +37,11 @@ struct JumpConsistentHashImpl using FunctionJumpConsistentHash = FunctionConsistentHashImpl; +} + void registerFunctionJumpConsistentHash(FunctionFactory & factory) { factory.registerFunction(); } } - diff --git a/src/Functions/lcm.cpp b/src/Functions/lcm.cpp index ceca495ddce..06e8d7d89f4 100644 --- a/src/Functions/lcm.cpp +++ b/src/Functions/lcm.cpp @@ -27,12 +27,14 @@ constexpr T abs(T value) noexcept namespace DB { - namespace ErrorCodes { extern const int NOT_IMPLEMENTED; } +namespace +{ + template struct LCMImpl { @@ -78,6 +80,8 @@ struct LCMImpl struct NameLCM { static constexpr auto name = "lcm"; }; using FunctionLCM = FunctionBinaryArithmetic; +} + void registerFunctionLCM(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/least.cpp b/src/Functions/least.cpp index e84d9bd9e24..75e3e7b2a14 100644 --- a/src/Functions/least.cpp +++ b/src/Functions/least.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ template struct LeastBaseImpl @@ -57,6 +59,8 @@ using LeastImpl = std::conditional_t; +} + void registerFunctionLeast(FunctionFactory & factory) { factory.registerFunction>(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/lengthUTF8.cpp b/src/Functions/lengthUTF8.cpp index 7c71533acd9..c067fd4db3c 100644 --- a/src/Functions/lengthUTF8.cpp +++ b/src/Functions/lengthUTF8.cpp @@ -6,12 +6,13 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ /** If the string is UTF-8 encoded text, it returns the length of the text in code points. * (not in characters: the length of the text "ё" can be either 1 or 2, depending on the normalization) @@ -60,6 +61,8 @@ struct NameLengthUTF8 }; using FunctionLengthUTF8 = FunctionStringOrArrayToT; +} + void registerFunctionLengthUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/lgamma.cpp b/src/Functions/lgamma.cpp index 05f62431ac9..c631758b2cd 100644 --- a/src/Functions/lgamma.cpp +++ b/src/Functions/lgamma.cpp @@ -11,6 +11,8 @@ extern "C" namespace DB { +namespace +{ /// Use wrapper and use lgamma_r version because std::lgamma is not threadsafe. static Float64 lgamma_wrapper(Float64 arg) @@ -22,6 +24,8 @@ static Float64 lgamma_wrapper(Float64 arg) struct LGammaName { static constexpr auto name = "lgamma"; }; using FunctionLGamma = FunctionMathUnary>; +} + void registerFunctionLGamma(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/like.cpp b/src/Functions/like.cpp index f334cef7917..37d9f006187 100644 --- a/src/Functions/like.cpp +++ b/src/Functions/like.cpp @@ -5,21 +5,22 @@ namespace DB { +namespace +{ struct NameLike { static constexpr auto name = "like"; }; -namespace -{ - using LikeImpl = MatchImpl; -} - +using LikeImpl = MatchImpl; using FunctionLike = FunctionsStringSearch; +} + void registerFunctionLike(FunctionFactory & factory) { factory.registerFunction(); } + } diff --git a/src/Functions/log.cpp b/src/Functions/log.cpp index c12300d7be4..791c73d13aa 100644 --- a/src/Functions/log.cpp +++ b/src/Functions/log.cpp @@ -4,12 +4,13 @@ namespace DB { +namespace +{ + struct LogName { static constexpr auto name = "log"; }; #if USE_FASTOPS -namespace -{ struct Impl { static constexpr auto name = LogName::name; @@ -22,7 +23,6 @@ namespace NFastOps::Log(src, size, dst); } }; -} using FunctionLog = FunctionMathUnary; @@ -30,6 +30,8 @@ using FunctionLog = FunctionMathUnary; using FunctionLog = FunctionMathUnary>; #endif +} + void registerFunctionLog(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/log10.cpp b/src/Functions/log10.cpp index 53301a313df..2e0bd484ed3 100644 --- a/src/Functions/log10.cpp +++ b/src/Functions/log10.cpp @@ -3,10 +3,14 @@ namespace DB { +namespace +{ struct Log10Name { static constexpr auto name = "log10"; }; using FunctionLog10 = FunctionMathUnary>; +} + void registerFunctionLog10(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/log2.cpp b/src/Functions/log2.cpp index 903c9176622..6ca770eafee 100644 --- a/src/Functions/log2.cpp +++ b/src/Functions/log2.cpp @@ -3,10 +3,14 @@ namespace DB { +namespace +{ struct Log2Name { static constexpr auto name = "log2"; }; using FunctionLog2 = FunctionMathUnary>; +} + void registerFunctionLog2(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/lowCardinalityIndices.cpp b/src/Functions/lowCardinalityIndices.cpp index 702d90f261c..e79397158a4 100644 --- a/src/Functions/lowCardinalityIndices.cpp +++ b/src/Functions/lowCardinalityIndices.cpp @@ -9,12 +9,14 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + class FunctionLowCardinalityIndices: public IFunction { public: @@ -54,6 +56,7 @@ public: } }; +} void registerFunctionLowCardinalityIndices(FunctionFactory & factory) { diff --git a/src/Functions/lowCardinalityKeys.cpp b/src/Functions/lowCardinalityKeys.cpp index 34c66a59340..9f9b1348f22 100644 --- a/src/Functions/lowCardinalityKeys.cpp +++ b/src/Functions/lowCardinalityKeys.cpp @@ -7,12 +7,13 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ class FunctionLowCardinalityKeys: public IFunction { @@ -48,6 +49,7 @@ public: } }; +} void registerFunctionLowCardinalityKeys(FunctionFactory & factory) { diff --git a/src/Functions/lower.cpp b/src/Functions/lower.cpp index b3c939968cb..0b19ae03f86 100644 --- a/src/Functions/lower.cpp +++ b/src/Functions/lower.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameLower { @@ -13,6 +15,8 @@ struct NameLower }; using FunctionLower = FunctionStringToString, NameLower>; +} + void registerFunctionLower(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/lowerUTF8.cpp b/src/Functions/lowerUTF8.cpp index fed031b524c..c8ff9636209 100644 --- a/src/Functions/lowerUTF8.cpp +++ b/src/Functions/lowerUTF8.cpp @@ -7,6 +7,8 @@ namespace DB { +namespace +{ struct NameLowerUTF8 { @@ -15,6 +17,8 @@ struct NameLowerUTF8 using FunctionLowerUTF8 = FunctionStringToString>, NameLowerUTF8>; +} + void registerFunctionLowerUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/match.cpp b/src/Functions/match.cpp index c1d69712220..3460d54c6b6 100644 --- a/src/Functions/match.cpp +++ b/src/Functions/match.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NameMatch { @@ -13,6 +15,8 @@ struct NameMatch using FunctionMatch = FunctionsStringSearch, NameMatch>; +} + void registerFunctionMatch(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/materialize.cpp b/src/Functions/materialize.cpp index 847e74c9a2a..56de111abda 100644 --- a/src/Functions/materialize.cpp +++ b/src/Functions/materialize.cpp @@ -4,6 +4,8 @@ namespace DB { +namespace +{ /** materialize(x) - materialize the constant */ @@ -43,6 +45,7 @@ public: } }; +} void registerFunctionMaterialize(FunctionFactory & factory) { diff --git a/src/Functions/minus.cpp b/src/Functions/minus.cpp index cacde3936d9..fa0a0d4da30 100644 --- a/src/Functions/minus.cpp +++ b/src/Functions/minus.cpp @@ -4,6 +4,8 @@ namespace DB { +namespace +{ template struct MinusImpl @@ -46,6 +48,8 @@ struct MinusImpl struct NameMinus { static constexpr auto name = "minus"; }; using FunctionMinus = FunctionBinaryArithmetic; +} + void registerFunctionMinus(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/modulo.cpp b/src/Functions/modulo.cpp index 68cdceed6ea..a8ad15c3971 100644 --- a/src/Functions/modulo.cpp +++ b/src/Functions/modulo.cpp @@ -10,12 +10,14 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_DIVISION; } +namespace +{ + /// Optimizations for integer modulo by a constant. template @@ -71,6 +73,8 @@ struct ModuloByConstantImpl } }; +} + /** Specializations are specified for dividing numbers of the type UInt64 and UInt32 by the numbers of the same sign. * Can be expanded to all possible combinations, but more code is needed. */ diff --git a/src/Functions/moduloOrZero.cpp b/src/Functions/moduloOrZero.cpp index 457c67c9e93..1392b0294bb 100644 --- a/src/Functions/moduloOrZero.cpp +++ b/src/Functions/moduloOrZero.cpp @@ -4,6 +4,8 @@ namespace DB { +namespace +{ template struct ModuloOrZeroImpl @@ -36,6 +38,8 @@ struct ModuloOrZeroImpl struct NameModuloOrZero { static constexpr auto name = "moduloOrZero"; }; using FunctionModuloOrZero = FunctionBinaryArithmetic; +} + void registerFunctionModuloOrZero(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiFuzzyMatchAllIndices.cpp b/src/Functions/multiFuzzyMatchAllIndices.cpp index 5a0206d5713..8b104e9ed2d 100644 --- a/src/Functions/multiFuzzyMatchAllIndices.cpp +++ b/src/Functions/multiFuzzyMatchAllIndices.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NameMultiFuzzyMatchAllIndices { @@ -16,6 +18,8 @@ using FunctionMultiFuzzyMatchAllIndices = FunctionsMultiStringFuzzySearch< NameMultiFuzzyMatchAllIndices, std::numeric_limits::max()>; +} + void registerFunctionMultiFuzzyMatchAllIndices(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiFuzzyMatchAny.cpp b/src/Functions/multiFuzzyMatchAny.cpp index e4b439358b7..4d0c3470d91 100644 --- a/src/Functions/multiFuzzyMatchAny.cpp +++ b/src/Functions/multiFuzzyMatchAny.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NameMultiFuzzyMatchAny { @@ -16,6 +18,8 @@ using FunctionMultiFuzzyMatchAny = FunctionsMultiStringFuzzySearch< NameMultiFuzzyMatchAny, std::numeric_limits::max()>; +} + void registerFunctionMultiFuzzyMatchAny(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiFuzzyMatchAnyIndex.cpp b/src/Functions/multiFuzzyMatchAnyIndex.cpp index 13a78ea058a..1680f413154 100644 --- a/src/Functions/multiFuzzyMatchAnyIndex.cpp +++ b/src/Functions/multiFuzzyMatchAnyIndex.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NameMultiFuzzyMatchAnyIndex { @@ -16,6 +18,8 @@ using FunctionMultiFuzzyMatchAnyIndex = FunctionsMultiStringFuzzySearch< NameMultiFuzzyMatchAnyIndex, std::numeric_limits::max()>; +} + void registerFunctionMultiFuzzyMatchAnyIndex(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index bdb0e01d7b2..977d4bde1a2 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -12,13 +12,15 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ + /// Function multiIf, which generalizes the function if. /// /// Syntax: multiIf(cond_1, then_1, ..., cond_N, then_N, else) @@ -225,6 +227,8 @@ public: } }; +} + void registerFunctionMultiIf(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiMatchAllIndices.cpp b/src/Functions/multiMatchAllIndices.cpp index e3f3b995b5e..171fa6baf74 100644 --- a/src/Functions/multiMatchAllIndices.cpp +++ b/src/Functions/multiMatchAllIndices.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NameMultiMatchAllIndices { @@ -16,6 +18,8 @@ using FunctionMultiMatchAllIndices = FunctionsMultiStringSearch< NameMultiMatchAllIndices, std::numeric_limits::max()>; +} + void registerFunctionMultiMatchAllIndices(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiMatchAny.cpp b/src/Functions/multiMatchAny.cpp index ba89e9f9fcd..146c27e250c 100644 --- a/src/Functions/multiMatchAny.cpp +++ b/src/Functions/multiMatchAny.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NameMultiMatchAny { @@ -16,6 +18,8 @@ using FunctionMultiMatchAny = FunctionsMultiStringSearch< NameMultiMatchAny, std::numeric_limits::max()>; +} + void registerFunctionMultiMatchAny(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiMatchAnyIndex.cpp b/src/Functions/multiMatchAnyIndex.cpp index 667149ef8fd..c43cd061187 100644 --- a/src/Functions/multiMatchAnyIndex.cpp +++ b/src/Functions/multiMatchAnyIndex.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NameMultiMatchAnyIndex { @@ -16,6 +18,8 @@ using FunctionMultiMatchAnyIndex = FunctionsMultiStringSearch< NameMultiMatchAnyIndex, std::numeric_limits::max()>; +} + void registerFunctionMultiMatchAnyIndex(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchAllPositions.cpp b/src/Functions/multiSearchAllPositions.cpp index c7aeb4d6245..5d9b3f5e2fd 100644 --- a/src/Functions/multiSearchAllPositions.cpp +++ b/src/Functions/multiSearchAllPositions.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchAllPositions { @@ -15,6 +17,8 @@ struct NameMultiSearchAllPositions using FunctionMultiSearchAllPositions = FunctionsMultiStringPosition, NameMultiSearchAllPositions>; +} + void registerFunctionMultiSearchAllPositions(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp b/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp index 4abcf7c8405..9f93284a769 100644 --- a/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp +++ b/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchAllPositionsCaseInsensitive { @@ -15,6 +17,8 @@ struct NameMultiSearchAllPositionsCaseInsensitive using FunctionMultiSearchAllPositionsCaseInsensitive = FunctionsMultiStringPosition, NameMultiSearchAllPositionsCaseInsensitive>; +} + void registerFunctionMultiSearchAllPositionsCaseInsensitive(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp b/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp index d9dbc1a7c8c..8864a00a8d3 100644 --- a/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp +++ b/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchAllPositionsCaseInsensitiveUTF8 { @@ -16,6 +18,8 @@ using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 = FunctionsMultiStringP MultiSearchAllPositionsImpl, NameMultiSearchAllPositionsCaseInsensitiveUTF8>; +} + void registerFunctionMultiSearchAllPositionsCaseInsensitiveUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchAllPositionsUTF8.cpp b/src/Functions/multiSearchAllPositionsUTF8.cpp index 8f39c0eade9..3922a859c3a 100644 --- a/src/Functions/multiSearchAllPositionsUTF8.cpp +++ b/src/Functions/multiSearchAllPositionsUTF8.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchAllPositionsUTF8 { @@ -15,6 +17,8 @@ struct NameMultiSearchAllPositionsUTF8 using FunctionMultiSearchAllPositionsUTF8 = FunctionsMultiStringPosition, NameMultiSearchAllPositionsUTF8>; +} + void registerFunctionMultiSearchAllPositionsUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchAny.cpp b/src/Functions/multiSearchAny.cpp index 144dbdbfdc4..5cd688ac65d 100644 --- a/src/Functions/multiSearchAny.cpp +++ b/src/Functions/multiSearchAny.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchAny { @@ -14,6 +16,8 @@ struct NameMultiSearchAny using FunctionMultiSearch = FunctionsMultiStringSearch, NameMultiSearchAny>; +} + void registerFunctionMultiSearchAny(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchAnyCaseInsensitive.cpp b/src/Functions/multiSearchAnyCaseInsensitive.cpp index 8b33a61013b..2358ce64bf8 100644 --- a/src/Functions/multiSearchAnyCaseInsensitive.cpp +++ b/src/Functions/multiSearchAnyCaseInsensitive.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchAnyCaseInsensitive { @@ -14,6 +16,8 @@ struct NameMultiSearchAnyCaseInsensitive using FunctionMultiSearchCaseInsensitive = FunctionsMultiStringSearch, NameMultiSearchAnyCaseInsensitive>; +} + void registerFunctionMultiSearchAnyCaseInsensitive(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp b/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp index 49a8b95a0e0..f84762d2bb4 100644 --- a/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp +++ b/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchAnyCaseInsensitiveUTF8 { @@ -15,6 +17,8 @@ struct NameMultiSearchAnyCaseInsensitiveUTF8 using FunctionMultiSearchCaseInsensitiveUTF8 = FunctionsMultiStringSearch, NameMultiSearchAnyCaseInsensitiveUTF8>; +} + void registerFunctionMultiSearchAnyCaseInsensitiveUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchAnyUTF8.cpp b/src/Functions/multiSearchAnyUTF8.cpp index 55f2e449833..b2c8342ba7f 100644 --- a/src/Functions/multiSearchAnyUTF8.cpp +++ b/src/Functions/multiSearchAnyUTF8.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchAnyUTF8 { @@ -13,6 +15,8 @@ struct NameMultiSearchAnyUTF8 }; using FunctionMultiSearchUTF8 = FunctionsMultiStringSearch, NameMultiSearchAnyUTF8>; +} + void registerFunctionMultiSearchAnyUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchFirstIndex.cpp b/src/Functions/multiSearchFirstIndex.cpp index 65a25004964..fcbeb552ae1 100644 --- a/src/Functions/multiSearchFirstIndex.cpp +++ b/src/Functions/multiSearchFirstIndex.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchFirstIndex { @@ -15,6 +17,8 @@ struct NameMultiSearchFirstIndex using FunctionMultiSearchFirstIndex = FunctionsMultiStringSearch, NameMultiSearchFirstIndex>; +} + void registerFunctionMultiSearchFirstIndex(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp b/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp index bf643f0cf29..87483734cf9 100644 --- a/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp +++ b/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchFirstIndexCaseInsensitive { @@ -15,6 +17,8 @@ struct NameMultiSearchFirstIndexCaseInsensitive using FunctionMultiSearchFirstIndexCaseInsensitive = FunctionsMultiStringSearch, NameMultiSearchFirstIndexCaseInsensitive>; +} + void registerFunctionMultiSearchFirstIndexCaseInsensitive(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp b/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp index 005152388ab..69e14adb32a 100644 --- a/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp +++ b/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchFirstIndexCaseInsensitiveUTF8 { @@ -15,6 +17,8 @@ struct NameMultiSearchFirstIndexCaseInsensitiveUTF8 using FunctionMultiSearchFirstIndexCaseInsensitiveUTF8 = FunctionsMultiStringSearch, NameMultiSearchFirstIndexCaseInsensitiveUTF8>; +} + void registerFunctionMultiSearchFirstIndexCaseInsensitiveUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchFirstIndexUTF8.cpp b/src/Functions/multiSearchFirstIndexUTF8.cpp index 1158fda2a63..699281dad4b 100644 --- a/src/Functions/multiSearchFirstIndexUTF8.cpp +++ b/src/Functions/multiSearchFirstIndexUTF8.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchFirstIndexUTF8 { @@ -15,6 +17,8 @@ struct NameMultiSearchFirstIndexUTF8 using FunctionMultiSearchFirstIndexUTF8 = FunctionsMultiStringSearch, NameMultiSearchFirstIndexUTF8>; +} + void registerFunctionMultiSearchFirstIndexUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchFirstPosition.cpp b/src/Functions/multiSearchFirstPosition.cpp index 06ac396250e..003345afde6 100644 --- a/src/Functions/multiSearchFirstPosition.cpp +++ b/src/Functions/multiSearchFirstPosition.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchFirstPosition { @@ -15,6 +17,8 @@ struct NameMultiSearchFirstPosition using FunctionMultiSearchFirstPosition = FunctionsMultiStringSearch, NameMultiSearchFirstPosition>; +} + void registerFunctionMultiSearchFirstPosition(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp b/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp index 1d028ad4513..7aa1ef991f3 100644 --- a/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp +++ b/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchFirstPositionCaseInsensitive { @@ -15,6 +17,8 @@ struct NameMultiSearchFirstPositionCaseInsensitive using FunctionMultiSearchFirstPositionCaseInsensitive = FunctionsMultiStringSearch, NameMultiSearchFirstPositionCaseInsensitive>; +} + void registerFunctionMultiSearchFirstPositionCaseInsensitive(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp b/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp index 0b355ddc446..d20ce6c2de3 100644 --- a/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp +++ b/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchFirstPositionCaseInsensitiveUTF8 { @@ -16,6 +18,8 @@ using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiString MultiSearchFirstPositionImpl, NameMultiSearchFirstPositionCaseInsensitiveUTF8>; +} + void registerFunctionMultiSearchFirstPositionCaseInsensitiveUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiSearchFirstPositionUTF8.cpp b/src/Functions/multiSearchFirstPositionUTF8.cpp index 26e16d17e8d..c0739808f7b 100644 --- a/src/Functions/multiSearchFirstPositionUTF8.cpp +++ b/src/Functions/multiSearchFirstPositionUTF8.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameMultiSearchFirstPositionUTF8 { @@ -15,6 +17,8 @@ struct NameMultiSearchFirstPositionUTF8 using FunctionMultiSearchFirstPositionUTF8 = FunctionsMultiStringSearch, NameMultiSearchFirstPositionUTF8>; +} + void registerFunctionMultiSearchFirstPositionUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/multiply.cpp b/src/Functions/multiply.cpp index 7018da843b6..dbb0ee97cde 100644 --- a/src/Functions/multiply.cpp +++ b/src/Functions/multiply.cpp @@ -4,6 +4,8 @@ namespace DB { +namespace +{ template struct MultiplyImpl @@ -46,6 +48,8 @@ struct MultiplyImpl struct NameMultiply { static constexpr auto name = "multiply"; }; using FunctionMultiply = FunctionBinaryArithmetic; +} + void registerFunctionMultiply(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/negate.cpp b/src/Functions/negate.cpp index 39ca434ea89..fff7c7355f4 100644 --- a/src/Functions/negate.cpp +++ b/src/Functions/negate.cpp @@ -4,6 +4,8 @@ namespace DB { +namespace +{ template struct NegateImpl @@ -29,6 +31,8 @@ struct NegateImpl struct NameNegate { static constexpr auto name = "negate"; }; using FunctionNegate = FunctionUnaryArithmetic; +} + template <> struct FunctionUnaryArithmeticMonotonicity { static bool has() { return true; } diff --git a/src/Functions/neighbor.cpp b/src/Functions/neighbor.cpp index 1f0f945b828..144d20a0c66 100644 --- a/src/Functions/neighbor.cpp +++ b/src/Functions/neighbor.cpp @@ -15,6 +15,9 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; } +namespace +{ + // Implements function, giving value for column within range of given // Example: // | c1 | @@ -194,6 +197,8 @@ public: } }; +} + void registerFunctionNeighbor(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/normalizeQuery.cpp b/src/Functions/normalizeQuery.cpp index 2cbefef2cba..320c2fde643 100644 --- a/src/Functions/normalizeQuery.cpp +++ b/src/Functions/normalizeQuery.cpp @@ -9,7 +9,6 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_COLUMN; diff --git a/src/Functions/normalizedQueryHash.cpp b/src/Functions/normalizedQueryHash.cpp index 21309c2960e..4c222db0f2c 100644 --- a/src/Functions/normalizedQueryHash.cpp +++ b/src/Functions/normalizedQueryHash.cpp @@ -15,7 +15,6 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_COLUMN; diff --git a/src/Functions/notEmpty.cpp b/src/Functions/notEmpty.cpp index d8057501848..6285e59652c 100644 --- a/src/Functions/notEmpty.cpp +++ b/src/Functions/notEmpty.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameNotEmpty { @@ -13,6 +15,8 @@ struct NameNotEmpty }; using FunctionNotEmpty = FunctionStringOrArrayToT, NameNotEmpty, UInt8>; +} + void registerFunctionNotEmpty(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/notILike.cpp b/src/Functions/notILike.cpp index 0f95564b09e..1fc0ab65ea8 100644 --- a/src/Functions/notILike.cpp +++ b/src/Functions/notILike.cpp @@ -4,19 +4,19 @@ namespace DB { +namespace +{ struct NameNotILike { static constexpr auto name = "notILike"; }; -namespace -{ - using NotILikeImpl = MatchImpl; -} - +using NotILikeImpl = MatchImpl; using FunctionNotILike = FunctionsStringSearch; +} + void registerFunctionNotILike(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/notLike.cpp b/src/Functions/notLike.cpp index 775fc4f3a66..54e2c7481f0 100644 --- a/src/Functions/notLike.cpp +++ b/src/Functions/notLike.cpp @@ -2,9 +2,10 @@ #include "FunctionFactory.h" #include "MatchImpl.h" - namespace DB { +namespace +{ struct NameNotLike { @@ -13,6 +14,8 @@ struct NameNotLike using FunctionNotLike = FunctionsStringSearch, NameNotLike>; +} + void registerFunctionNotLike(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/now.cpp b/src/Functions/now.cpp index f38fc4006bc..e71c3a77f11 100644 --- a/src/Functions/now.cpp +++ b/src/Functions/now.cpp @@ -9,10 +9,11 @@ namespace DB - { -/// Get the current time. (It is a constant, it is evaluated once for the entire query.) +namespace +{ +/// Get the current time. (It is a constant, it is evaluated once for the entire query.) class ExecutableFunctionNow : public IExecutableFunctionImpl { public: @@ -82,6 +83,8 @@ public: } }; +} + void registerFunctionNow(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/now64.cpp b/src/Functions/now64.cpp index 57d1ff8c19e..cba66ffb531 100644 --- a/src/Functions/now64.cpp +++ b/src/Functions/now64.cpp @@ -11,13 +11,15 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int CANNOT_CLOCK_GETTIME; } +namespace +{ + static Field nowSubsecond(UInt32 scale) { static constexpr Int32 fractional_scale = 9; @@ -89,6 +91,8 @@ public: } }; +} + void registerFunctionNow64(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/nullIf.cpp b/src/Functions/nullIf.cpp index cde8a2ed343..57e1265793e 100644 --- a/src/Functions/nullIf.cpp +++ b/src/Functions/nullIf.cpp @@ -9,6 +9,8 @@ namespace DB { +namespace +{ /// Implements the function nullIf which takes 2 arguments and returns /// NULL if both arguments have the same value. Otherwise it returns the @@ -74,6 +76,7 @@ public: } }; +} void registerFunctionNullIf(FunctionFactory & factory) { diff --git a/src/Functions/pi.cpp b/src/Functions/pi.cpp index 4621ed8ecc0..efa536c7314 100644 --- a/src/Functions/pi.cpp +++ b/src/Functions/pi.cpp @@ -3,6 +3,8 @@ namespace DB { +namespace +{ struct PiImpl { @@ -12,6 +14,8 @@ struct PiImpl using FunctionPi = FunctionMathConstFloat64; +} + void registerFunctionPi(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/plus.cpp b/src/Functions/plus.cpp index 1421bfcd4c6..8aee3141759 100644 --- a/src/Functions/plus.cpp +++ b/src/Functions/plus.cpp @@ -4,6 +4,8 @@ namespace DB { +namespace +{ template struct PlusImpl @@ -47,6 +49,8 @@ struct PlusImpl struct NamePlus { static constexpr auto name = "plus"; }; using FunctionPlus = FunctionBinaryArithmetic; +} + void registerFunctionPlus(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/pointInEllipses.cpp b/src/Functions/pointInEllipses.cpp index d3f57e5fecd..79ea43cf04d 100644 --- a/src/Functions/pointInEllipses.cpp +++ b/src/Functions/pointInEllipses.cpp @@ -11,7 +11,6 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; @@ -20,6 +19,9 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } +namespace +{ + /** * The function checks if a point is in one of ellipses in set. * The number of arguments must be 2 + 4*N where N is the number of ellipses. @@ -188,6 +190,7 @@ private: } }; +} void registerFunctionPointInEllipses(FunctionFactory & factory) { diff --git a/src/Functions/pointInPolygon.cpp b/src/Functions/pointInPolygon.cpp index ccff56b9a89..5a9ce284ba9 100644 --- a/src/Functions/pointInPolygon.cpp +++ b/src/Functions/pointInPolygon.cpp @@ -34,7 +34,6 @@ namespace ProfileEvents namespace DB { - namespace ErrorCodes { extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; @@ -43,6 +42,8 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } +namespace +{ using CoordinateType = Float64; using Point = boost::geometry::model::d2::point_xy; @@ -561,6 +562,7 @@ private: } }; +} void registerFunctionPointInPolygon(FunctionFactory & factory) { diff --git a/src/Functions/position.cpp b/src/Functions/position.cpp index df1405ae94a..5b8af16fef1 100644 --- a/src/Functions/position.cpp +++ b/src/Functions/position.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NamePosition { @@ -13,6 +15,8 @@ struct NamePosition using FunctionPosition = FunctionsStringSearch, NamePosition>; +} + void registerFunctionPosition(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/positionCaseInsensitive.cpp b/src/Functions/positionCaseInsensitive.cpp index 00721dda212..f72766a1875 100644 --- a/src/Functions/positionCaseInsensitive.cpp +++ b/src/Functions/positionCaseInsensitive.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NamePositionCaseInsensitive { @@ -13,6 +15,8 @@ struct NamePositionCaseInsensitive using FunctionPositionCaseInsensitive = FunctionsStringSearch, NamePositionCaseInsensitive>; +} + void registerFunctionPositionCaseInsensitive(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/positionCaseInsensitiveUTF8.cpp b/src/Functions/positionCaseInsensitiveUTF8.cpp index 196a5d67cd2..0af545cc6a3 100644 --- a/src/Functions/positionCaseInsensitiveUTF8.cpp +++ b/src/Functions/positionCaseInsensitiveUTF8.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NamePositionCaseInsensitiveUTF8 { @@ -14,6 +16,8 @@ struct NamePositionCaseInsensitiveUTF8 using FunctionPositionCaseInsensitiveUTF8 = FunctionsStringSearch, NamePositionCaseInsensitiveUTF8>; +} + void registerFunctionPositionCaseInsensitiveUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/positionUTF8.cpp b/src/Functions/positionUTF8.cpp index 944b55005f3..68b2f5c274e 100644 --- a/src/Functions/positionUTF8.cpp +++ b/src/Functions/positionUTF8.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NamePositionUTF8 { @@ -13,6 +15,8 @@ struct NamePositionUTF8 using FunctionPositionUTF8 = FunctionsStringSearch, NamePositionUTF8>; +} + void registerFunctionPositionUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/pow.cpp b/src/Functions/pow.cpp index db16bc993c7..7e60e0e878e 100644 --- a/src/Functions/pow.cpp +++ b/src/Functions/pow.cpp @@ -3,10 +3,14 @@ namespace DB { +namespace +{ struct PowName { static constexpr auto name = "pow"; }; using FunctionPow = FunctionMathBinaryFloat64>; +} + void registerFunctionPow(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/rand.cpp b/src/Functions/rand.cpp index e704b7824b5..660119a64cb 100644 --- a/src/Functions/rand.cpp +++ b/src/Functions/rand.cpp @@ -3,10 +3,14 @@ namespace DB { +namespace +{ struct NameRand { static constexpr auto name = "rand"; }; using FunctionRand = FunctionRandom; +} + void registerFunctionRand(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/rand64.cpp b/src/Functions/rand64.cpp index cc942aee28c..9377d3d40d0 100644 --- a/src/Functions/rand64.cpp +++ b/src/Functions/rand64.cpp @@ -3,10 +3,14 @@ namespace DB { +namespace +{ struct NameRand64 { static constexpr auto name = "rand64"; }; using FunctionRand64 = FunctionRandom; +} + void registerFunctionRand64(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/randConstant.cpp b/src/Functions/randConstant.cpp index ac410fc35c4..b2880833e8e 100644 --- a/src/Functions/randConstant.cpp +++ b/src/Functions/randConstant.cpp @@ -9,6 +9,9 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ + template class ExecutableFunctionRandomConstant : public IExecutableFunctionImpl { @@ -107,10 +110,11 @@ public: } }; - struct NameRandConstant { static constexpr auto name = "randConstant"; }; using FunctionBuilderRandConstant = RandomConstantOverloadResolver; +} + void registerFunctionRandConstant(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/randomFixedString.cpp b/src/Functions/randomFixedString.cpp index 2df0d86aae1..1627716d9bf 100644 --- a/src/Functions/randomFixedString.cpp +++ b/src/Functions/randomFixedString.cpp @@ -21,6 +21,8 @@ namespace ErrorCodes extern const int DECIMAL_OVERFLOW; } +namespace +{ /* Generate random fixed string with fully random bytes (including zero). */ template @@ -103,6 +105,8 @@ private: ImplementationSelector selector; }; +} + void registerFunctionRandomFixedString(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/randomPrintableASCII.cpp b/src/Functions/randomPrintableASCII.cpp index 3f426aaa35d..775c50a3026 100644 --- a/src/Functions/randomPrintableASCII.cpp +++ b/src/Functions/randomPrintableASCII.cpp @@ -17,6 +17,8 @@ namespace ErrorCodes extern const int TOO_LARGE_STRING_SIZE; } +namespace +{ /** Generate random string of specified length with printable ASCII characters, almost uniformly distributed. * First argument is length, other optional arguments are ignored and used to prevent common subexpression elimination to get different values. @@ -108,6 +110,8 @@ public: } }; +} + void registerFunctionRandomPrintableASCII(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/randomString.cpp b/src/Functions/randomString.cpp index 58b53e47e02..66f77602d96 100644 --- a/src/Functions/randomString.cpp +++ b/src/Functions/randomString.cpp @@ -19,6 +19,8 @@ namespace ErrorCodes extern const int TOO_LARGE_STRING_SIZE; } +namespace +{ /* Generate random string of specified length with fully random bytes (including zero). */ template @@ -123,6 +125,8 @@ private: ImplementationSelector selector; }; +} + void registerFunctionRandomString(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/randomStringUTF8.cpp b/src/Functions/randomStringUTF8.cpp index 433442d6a77..5c7478e446e 100644 --- a/src/Functions/randomStringUTF8.cpp +++ b/src/Functions/randomStringUTF8.cpp @@ -17,6 +17,8 @@ namespace ErrorCodes extern const int TOO_LARGE_STRING_SIZE; } +namespace +{ /* Generate string with a UTF-8 encoded text. * Take a single argument - length of result string in Unicode code points. @@ -141,6 +143,8 @@ public: } }; +} + void registerFunctionRandomStringUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/regexpQuoteMeta.cpp b/src/Functions/regexpQuoteMeta.cpp index e99ef5a9bd0..c508daed75b 100644 --- a/src/Functions/regexpQuoteMeta.cpp +++ b/src/Functions/regexpQuoteMeta.cpp @@ -7,13 +7,15 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + class FunctionRegexpQuoteMeta : public IFunction { public: @@ -104,11 +106,13 @@ public: block.getByPosition(result).column = std::move(dst_column); } - }; +} + void registerFunctionRegexpQuoteMeta(FunctionFactory & factory) { factory.registerFunction(); } + } diff --git a/src/Functions/reinterpretAsFixedString.cpp b/src/Functions/reinterpretAsFixedString.cpp index 73e1387308d..bec5fb4522e 100644 --- a/src/Functions/reinterpretAsFixedString.cpp +++ b/src/Functions/reinterpretAsFixedString.cpp @@ -8,13 +8,14 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; } +namespace +{ class FunctionReinterpretAsFixedString : public IFunction { @@ -84,6 +85,8 @@ public: } }; +} + void registerFunctionReinterpretAsFixedString(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/reinterpretAsString.cpp b/src/Functions/reinterpretAsString.cpp index c9c0c7eaaf6..2a368392f3f 100644 --- a/src/Functions/reinterpretAsString.cpp +++ b/src/Functions/reinterpretAsString.cpp @@ -8,13 +8,15 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + /** Function for transforming numbers and dates to strings that contain the same set of bytes in the machine representation. */ class FunctionReinterpretAsString : public IFunction { @@ -80,6 +82,7 @@ public: } }; +} void registerFunctionReinterpretAsString(FunctionFactory & factory) { @@ -87,5 +90,3 @@ void registerFunctionReinterpretAsString(FunctionFactory & factory) } } - - diff --git a/src/Functions/reinterpretStringAs.cpp b/src/Functions/reinterpretStringAs.cpp index bb290b33b6d..71528b7cb61 100644 --- a/src/Functions/reinterpretStringAs.cpp +++ b/src/Functions/reinterpretStringAs.cpp @@ -16,13 +16,15 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + template class FunctionReinterpretStringAs : public IFunction { @@ -132,6 +134,7 @@ using FunctionReinterpretAsFloat64 = FunctionReinterpretStringAs; using FunctionReinterpretAsDateTime = FunctionReinterpretStringAs; +} void registerFunctionsReinterpretStringAs(FunctionFactory & factory) { @@ -150,5 +153,3 @@ void registerFunctionsReinterpretStringAs(FunctionFactory & factory) } } - - diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index 18350dd6207..fd0cf238f50 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -17,6 +17,9 @@ namespace ErrorCodes extern const int TOO_LARGE_STRING_SIZE; } +namespace +{ + struct RepeatImpl { /// Safety threshold against DoS. @@ -218,6 +221,7 @@ public: } }; +} void registerFunctionRepeat(FunctionFactory & factory) { diff --git a/src/Functions/replaceAll.cpp b/src/Functions/replaceAll.cpp index 56779952d8e..cc29e57ea69 100644 --- a/src/Functions/replaceAll.cpp +++ b/src/Functions/replaceAll.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NameReplaceAll { @@ -13,6 +15,8 @@ struct NameReplaceAll using FunctionReplaceAll = FunctionStringReplace, NameReplaceAll>; +} + void registerFunctionReplaceAll(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/replaceOne.cpp b/src/Functions/replaceOne.cpp index 78f2236b873..d9bcbef0e2d 100644 --- a/src/Functions/replaceOne.cpp +++ b/src/Functions/replaceOne.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NameReplaceOne { @@ -13,6 +15,8 @@ struct NameReplaceOne using FunctionReplaceOne = FunctionStringReplace, NameReplaceOne>; +} + void registerFunctionReplaceOne(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/replaceRegexpAll.cpp b/src/Functions/replaceRegexpAll.cpp index 9bbc28b1cec..ad67efa82f4 100644 --- a/src/Functions/replaceRegexpAll.cpp +++ b/src/Functions/replaceRegexpAll.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NameReplaceRegexpAll { @@ -13,6 +15,8 @@ struct NameReplaceRegexpAll using FunctionReplaceRegexpAll = FunctionStringReplace, NameReplaceRegexpAll>; +} + void registerFunctionReplaceRegexpAll(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/replaceRegexpOne.cpp b/src/Functions/replaceRegexpOne.cpp index a3fcf34517b..9844f77ee26 100644 --- a/src/Functions/replaceRegexpOne.cpp +++ b/src/Functions/replaceRegexpOne.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NameReplaceRegexpOne { @@ -13,6 +15,8 @@ struct NameReplaceRegexpOne using FunctionReplaceRegexpOne = FunctionStringReplace, NameReplaceRegexpOne>; +} + void registerFunctionReplaceRegexpOne(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/replicate.cpp b/src/Functions/replicate.cpp index bdbc957c805..73246651003 100644 --- a/src/Functions/replicate.cpp +++ b/src/Functions/replicate.cpp @@ -7,13 +7,15 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; } +namespace +{ + /** Creates an array, multiplying the column (the first argument) by the number of elements in the array (the second argument). */ class FunctionReplicate : public IFunction @@ -64,6 +66,7 @@ public: } }; +} void registerFunctionReplicate(FunctionFactory & factory) { diff --git a/src/Functions/reverse.cpp b/src/Functions/reverse.cpp index 60d20d2aee8..ad12b6f7651 100644 --- a/src/Functions/reverse.cpp +++ b/src/Functions/reverse.cpp @@ -9,13 +9,14 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ /** Reverse the string as a sequence of bytes. */ @@ -143,6 +144,7 @@ private: const Context & context; }; +} void registerFunctionReverse(FunctionFactory & factory) { diff --git a/src/Functions/reverseUTF8.cpp b/src/Functions/reverseUTF8.cpp index 40f3214f880..b19808c091e 100644 --- a/src/Functions/reverseUTF8.cpp +++ b/src/Functions/reverseUTF8.cpp @@ -6,12 +6,13 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_COLUMN; } +namespace +{ /** Reverse the sequence of code points in a UTF-8 encoded string. * The result may not match the expected result, because modifying code points (for example, diacritics) may be applied to another symbols. @@ -73,6 +74,8 @@ struct NameReverseUTF8 }; using FunctionReverseUTF8 = FunctionStringToString; +} + void registerFunctionReverseUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/roundAge.cpp b/src/Functions/roundAge.cpp index 67705983385..08ac941c2df 100644 --- a/src/Functions/roundAge.cpp +++ b/src/Functions/roundAge.cpp @@ -3,6 +3,8 @@ namespace DB { +namespace +{ template struct RoundAgeImpl @@ -29,6 +31,8 @@ struct RoundAgeImpl struct NameRoundAge { static constexpr auto name = "roundAge"; }; using FunctionRoundAge = FunctionUnaryArithmetic; +} + template <> struct FunctionUnaryArithmeticMonotonicity : PositiveMonotonicity {}; void registerFunctionRoundAge(FunctionFactory & factory) diff --git a/src/Functions/roundDuration.cpp b/src/Functions/roundDuration.cpp index 3b617822d9d..dec3427a7e1 100644 --- a/src/Functions/roundDuration.cpp +++ b/src/Functions/roundDuration.cpp @@ -3,6 +3,8 @@ namespace DB { +namespace +{ template struct RoundDurationImpl @@ -38,6 +40,8 @@ struct RoundDurationImpl struct NameRoundDuration { static constexpr auto name = "roundDuration"; }; using FunctionRoundDuration = FunctionUnaryArithmetic; +} + template <> struct FunctionUnaryArithmeticMonotonicity : PositiveMonotonicity {}; void registerFunctionRoundDuration(FunctionFactory & factory) diff --git a/src/Functions/roundToExp2.cpp b/src/Functions/roundToExp2.cpp index c6b6f672c66..50a548e095e 100644 --- a/src/Functions/roundToExp2.cpp +++ b/src/Functions/roundToExp2.cpp @@ -5,12 +5,14 @@ namespace DB { - namespace ErrorCodes { extern const int NOT_IMPLEMENTED; } +namespace +{ + template inline std::enable_if_t && (sizeof(T) <= sizeof(UInt32)), T> roundDownToPowerOfTwo(T x) @@ -88,6 +90,8 @@ struct RoundToExp2Impl struct NameRoundToExp2 { static constexpr auto name = "roundToExp2"; }; using FunctionRoundToExp2 = FunctionUnaryArithmetic; +} + template <> struct FunctionUnaryArithmeticMonotonicity : PositiveMonotonicity {}; void registerFunctionRoundToExp2(FunctionFactory & factory) diff --git a/src/Functions/rowNumberInAllBlocks.cpp b/src/Functions/rowNumberInAllBlocks.cpp index 4334a6f341a..9fd78857bf7 100644 --- a/src/Functions/rowNumberInAllBlocks.cpp +++ b/src/Functions/rowNumberInAllBlocks.cpp @@ -7,6 +7,8 @@ namespace DB { +namespace +{ /** Incremental number of row within all blocks passed to this function. */ class FunctionRowNumberInAllBlocks : public IFunction @@ -69,6 +71,7 @@ public: } }; +} void registerFunctionRowNumberInAllBlocks(FunctionFactory & factory) { diff --git a/src/Functions/rowNumberInBlock.cpp b/src/Functions/rowNumberInBlock.cpp index a86711afb56..c7df817fd28 100644 --- a/src/Functions/rowNumberInBlock.cpp +++ b/src/Functions/rowNumberInBlock.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ class FunctionRowNumberInBlock : public IFunction { @@ -56,6 +58,8 @@ public: } }; +} + void registerFunctionRowNumberInBlock(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/runningAccumulate.cpp b/src/Functions/runningAccumulate.cpp index 655663f6e9e..c442ab80348 100644 --- a/src/Functions/runningAccumulate.cpp +++ b/src/Functions/runningAccumulate.cpp @@ -10,7 +10,6 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_COLUMN; @@ -18,6 +17,8 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ /** runningAccumulate(agg_state) - takes the states of the aggregate function and returns a column with values, * are the result of the accumulation of these states for a set of block lines, from the first to the current line. @@ -133,6 +134,7 @@ public: } }; +} void registerFunctionRunningAccumulate(FunctionFactory & factory) { diff --git a/src/Functions/sigmoid.cpp b/src/Functions/sigmoid.cpp index e878a48046d..c4e00db6d5c 100644 --- a/src/Functions/sigmoid.cpp +++ b/src/Functions/sigmoid.cpp @@ -3,6 +3,8 @@ namespace DB { +namespace +{ struct SigmoidName { static constexpr auto name = "sigmoid"; }; @@ -37,6 +39,8 @@ using FunctionSigmoid = FunctionMathUnary(); diff --git a/src/Functions/sin.cpp b/src/Functions/sin.cpp index cd66b19c553..6fd5d189767 100644 --- a/src/Functions/sin.cpp +++ b/src/Functions/sin.cpp @@ -3,10 +3,14 @@ namespace DB { +namespace +{ struct SinName { static constexpr auto name = "sin"; }; using FunctionSin = FunctionMathUnary>; +} + void registerFunctionSin(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/sqrt.cpp b/src/Functions/sqrt.cpp index db861cde32e..725da874a51 100644 --- a/src/Functions/sqrt.cpp +++ b/src/Functions/sqrt.cpp @@ -3,10 +3,14 @@ namespace DB { +namespace +{ struct SqrtName { static constexpr auto name = "sqrt"; }; using FunctionSqrt = FunctionMathUnary>; +} + void registerFunctionSqrt(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/stringToH3.cpp b/src/Functions/stringToH3.cpp index d0d22f3927b..d6b17ce3f5d 100644 --- a/src/Functions/stringToH3.cpp +++ b/src/Functions/stringToH3.cpp @@ -18,6 +18,9 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + using namespace GatherUtils; class FunctionStringToH3 : public IFunction @@ -90,6 +93,7 @@ private: } }; +} void registerFunctionStringToH3(FunctionFactory & factory) { diff --git a/src/Functions/substring.cpp b/src/Functions/substring.cpp index bf8ba1d6b73..f043e08b139 100644 --- a/src/Functions/substring.cpp +++ b/src/Functions/substring.cpp @@ -26,6 +26,8 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ /// If 'is_utf8' - measure offset and length in code points instead of bytes. /// UTF8 variant is not available for FixedString arguments. @@ -176,6 +178,8 @@ public: } }; +} + void registerFunctionSubstring(FunctionFactory & factory) { factory.registerFunction>(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/sumburConsistentHash.cpp b/src/Functions/sumburConsistentHash.cpp index 1fc26502355..88de93f65d9 100644 --- a/src/Functions/sumburConsistentHash.cpp +++ b/src/Functions/sumburConsistentHash.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct SumburConsistentHashImpl { @@ -24,6 +26,8 @@ struct SumburConsistentHashImpl using FunctionSumburConsistentHash = FunctionConsistentHashImpl; +} + void registerFunctionSumburConsistentHash(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/tan.cpp b/src/Functions/tan.cpp index e18d81f1e01..db63ab13984 100644 --- a/src/Functions/tan.cpp +++ b/src/Functions/tan.cpp @@ -3,10 +3,14 @@ namespace DB { +namespace +{ struct TanName { static constexpr auto name = "tan"; }; using FunctionTan = FunctionMathUnary>; +} + void registerFunctionTan(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/tanh.cpp b/src/Functions/tanh.cpp index 4fe3d616d25..6fc1d2f79e5 100644 --- a/src/Functions/tanh.cpp +++ b/src/Functions/tanh.cpp @@ -3,13 +3,13 @@ namespace DB { +namespace +{ struct TanhName { static constexpr auto name = "tanh"; }; #if USE_FASTOPS -namespace -{ struct Impl { static constexpr auto name = TanhName::name; @@ -22,13 +22,12 @@ namespace NFastOps::Tanh<>(src, size, dst); } }; -} using FunctionTanh = FunctionMathUnary; #else -static double tanh(double x) +double tanh(double x) { return 2 / (1.0 + exp(-2 * x)) - 1; } @@ -36,6 +35,8 @@ static double tanh(double x) using FunctionTanh = FunctionMathUnary>; #endif +} + void registerFunctionTanh(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/tgamma.cpp b/src/Functions/tgamma.cpp index e5e68963856..8ad00bc79ff 100644 --- a/src/Functions/tgamma.cpp +++ b/src/Functions/tgamma.cpp @@ -3,10 +3,14 @@ namespace DB { +namespace +{ struct TGammaName { static constexpr auto name = "tgamma"; }; using FunctionTGamma = FunctionMathUnary>; +} + void registerFunctionTGamma(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/throwIf.cpp b/src/Functions/throwIf.cpp index 8faac6425c8..46a384ebc51 100644 --- a/src/Functions/throwIf.cpp +++ b/src/Functions/throwIf.cpp @@ -10,7 +10,6 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_COLUMN; @@ -19,6 +18,8 @@ namespace ErrorCodes extern const int FUNCTION_THROW_IF_VALUE_IS_NON_ZERO; } +namespace +{ /// Throw an exception if the argument is non zero. class FunctionThrowIf : public IFunction @@ -109,6 +110,7 @@ public: } }; +} void registerFunctionThrowIf(FunctionFactory & factory) { diff --git a/src/Functions/timeSlots.cpp b/src/Functions/timeSlots.cpp index 89eb56d8e61..11804ac72d7 100644 --- a/src/Functions/timeSlots.cpp +++ b/src/Functions/timeSlots.cpp @@ -13,7 +13,6 @@ namespace DB { - namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; @@ -21,6 +20,9 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } +namespace +{ + /** timeSlots(StartTime, Duration) * - for the time interval beginning at `StartTime` and continuing `Duration` seconds, * returns an array of time points, consisting of rounding down to half an hour (default; or another value) of points from this interval. @@ -190,6 +192,8 @@ public: } }; +} + void registerFunctionTimeSlots(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/timezone.cpp b/src/Functions/timezone.cpp index 9c15e5ae4b8..f697144c0d8 100644 --- a/src/Functions/timezone.cpp +++ b/src/Functions/timezone.cpp @@ -7,7 +7,8 @@ namespace DB { - +namespace +{ /** Returns the server time zone. */ @@ -42,6 +43,7 @@ public: } }; +} void registerFunctionTimeZone(FunctionFactory & factory) { diff --git a/src/Functions/toColumnTypeName.cpp b/src/Functions/toColumnTypeName.cpp index 32092d8a471..498f5095461 100644 --- a/src/Functions/toColumnTypeName.cpp +++ b/src/Functions/toColumnTypeName.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ /// Returns name of IColumn instance. class FunctionToColumnTypeName : public IFunction @@ -46,6 +48,7 @@ public: } }; +} void registerFunctionToColumnTypeName(FunctionFactory & factory) { diff --git a/src/Functions/toLowCardinality.cpp b/src/Functions/toLowCardinality.cpp index 8f0dfd2e932..a9873205114 100644 --- a/src/Functions/toLowCardinality.cpp +++ b/src/Functions/toLowCardinality.cpp @@ -7,6 +7,8 @@ namespace DB { +namespace +{ class FunctionToLowCardinality: public IFunction { @@ -47,6 +49,7 @@ public: } }; +} void registerFunctionToLowCardinality(FunctionFactory & factory) { diff --git a/src/Functions/toNullable.cpp b/src/Functions/toNullable.cpp index 8be5f3ad243..8d0eb927d55 100644 --- a/src/Functions/toNullable.cpp +++ b/src/Functions/toNullable.cpp @@ -7,6 +7,8 @@ namespace DB { +namespace +{ /// If value is not Nullable or NULL, wraps it to Nullable. class FunctionToNullable : public IFunction @@ -39,6 +41,7 @@ public: } }; +} void registerFunctionToNullable(FunctionFactory & factory) { diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 6536a8d88a6..83fcec30338 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -143,7 +143,6 @@ namespace return time_zone.toStartOfSecondInterval(t, seconds); } }; -} class FunctionToStartOfInterval : public IFunction @@ -340,6 +339,7 @@ private: } }; +} void registerFunctionToStartOfInterval(FunctionFactory & factory) { diff --git a/src/Functions/toTimeZone.cpp b/src/Functions/toTimeZone.cpp index d75b202ed41..46d4c4698f7 100644 --- a/src/Functions/toTimeZone.cpp +++ b/src/Functions/toTimeZone.cpp @@ -11,13 +11,15 @@ namespace DB { - namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + /// Just changes time zone information for data type. The calculation is free. class FunctionToTimeZone : public IFunction { @@ -58,6 +60,8 @@ public: } }; +} + void registerFunctionToTimeZone(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/toTypeName.cpp b/src/Functions/toTypeName.cpp index a47307acbe9..99f39523114 100644 --- a/src/Functions/toTypeName.cpp +++ b/src/Functions/toTypeName.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ /** toTypeName(x) - get the type name * Returns name of IDataType instance (name of data type). @@ -85,6 +87,7 @@ public: ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; } }; +} void registerFunctionToTypeName(FunctionFactory & factory) { diff --git a/src/Functions/toUnixTimestamp64Micro.cpp b/src/Functions/toUnixTimestamp64Micro.cpp index 845647fc0dd..e1a9382e69a 100644 --- a/src/Functions/toUnixTimestamp64Micro.cpp +++ b/src/Functions/toUnixTimestamp64Micro.cpp @@ -3,6 +3,8 @@ namespace DB { +namespace +{ struct TransformToMicro { @@ -12,6 +14,8 @@ struct TransformToMicro using ResultDataType = DataTypeInt64; }; +} + void registerToUnixTimestamp64Micro(FunctionFactory & factory) { factory.registerFunction>(); diff --git a/src/Functions/toUnixTimestamp64Milli.cpp b/src/Functions/toUnixTimestamp64Milli.cpp index 88baf5c0b61..b49b1c9042b 100644 --- a/src/Functions/toUnixTimestamp64Milli.cpp +++ b/src/Functions/toUnixTimestamp64Milli.cpp @@ -3,6 +3,8 @@ namespace DB { +namespace +{ struct TransformToMilli { @@ -12,6 +14,8 @@ struct TransformToMilli using ResultDataType = DataTypeInt64; }; +} + void registerToUnixTimestamp64Milli(FunctionFactory & factory) { factory.registerFunction>(); diff --git a/src/Functions/toUnixTimestamp64Nano.cpp b/src/Functions/toUnixTimestamp64Nano.cpp index 5e90a9504cf..832d845434c 100644 --- a/src/Functions/toUnixTimestamp64Nano.cpp +++ b/src/Functions/toUnixTimestamp64Nano.cpp @@ -3,6 +3,8 @@ namespace DB { +namespace +{ struct TransformToNano { @@ -12,6 +14,8 @@ struct TransformToNano using ResultDataType = DataTypeInt64; }; +} + void registerToUnixTimestamp64Nano(FunctionFactory & factory) { factory.registerFunction>(); diff --git a/src/Functions/toValidUTF8.cpp b/src/Functions/toValidUTF8.cpp index c18ef8e7622..189556f48ea 100644 --- a/src/Functions/toValidUTF8.cpp +++ b/src/Functions/toValidUTF8.cpp @@ -20,6 +20,9 @@ namespace ErrorCodes extern const UInt8 length_of_utf8_sequence[256]; +namespace +{ + struct ToValidUTF8Impl { static void toValidUTF8One(const char * begin, const char * end, WriteBuffer & write_buffer) @@ -135,6 +138,8 @@ struct NameToValidUTF8 }; using FunctionToValidUTF8 = FunctionStringToString; +} + void registerFunctionToValidUTF8(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/today.cpp b/src/Functions/today.cpp index 9c3395b6ab1..5522246b0a6 100644 --- a/src/Functions/today.cpp +++ b/src/Functions/today.cpp @@ -10,6 +10,8 @@ namespace DB { +namespace +{ class ExecutableFunctionToday : public IExecutableFunctionImpl { @@ -79,6 +81,8 @@ public: } }; +} + void registerFunctionToday(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index 884b972c2b4..8eab33e7633 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -17,7 +17,6 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; @@ -26,6 +25,8 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } +namespace +{ /** transform(x, from_array, to_array[, default]) - convert x according to an explicitly passed match. */ @@ -846,6 +847,8 @@ private: } }; +} + void registerFunctionTransform(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/trim.cpp b/src/Functions/trim.cpp index eba49739074..fdaf25b815e 100644 --- a/src/Functions/trim.cpp +++ b/src/Functions/trim.cpp @@ -6,12 +6,14 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +namespace +{ + struct TrimModeLeft { static constexpr auto name = "trimLeft"; @@ -103,6 +105,8 @@ using FunctionTrimLeft = FunctionStringToString, using FunctionTrimRight = FunctionStringToString, TrimModeRight>; using FunctionTrimBoth = FunctionStringToString, TrimModeBoth>; +} + void registerFunctionTrim(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/tuple.cpp b/src/Functions/tuple.cpp index 9e55ba656a2..2546ec68193 100644 --- a/src/Functions/tuple.cpp +++ b/src/Functions/tuple.cpp @@ -7,12 +7,13 @@ namespace DB { - namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ /** tuple(x, y, ...) is a function that allows you to group several columns * tupleElement(tuple, n) is a function that allows you to retrieve a column from tuple. @@ -75,6 +76,8 @@ public: } }; +} + void registerFunctionTuple(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index 7264fb67bfb..896d9ed5c13 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -14,13 +14,14 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_INDEX; } +namespace +{ /** Extract element of tuple by constant index or name. The operation is essentially free. * Also the function looks through Arrays: you can get Array of tuple elements from Array of Tuples. @@ -139,6 +140,7 @@ private: } }; +} void registerFunctionTupleElement(FunctionFactory & factory) { diff --git a/src/Functions/upper.cpp b/src/Functions/upper.cpp index e96a5a312fb..515574e2a09 100644 --- a/src/Functions/upper.cpp +++ b/src/Functions/upper.cpp @@ -5,6 +5,8 @@ namespace DB { +namespace +{ struct NameUpper { @@ -12,6 +14,8 @@ struct NameUpper }; using FunctionUpper = FunctionStringToString, NameUpper>; +} + void registerFunctionUpper(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/upperUTF8.cpp b/src/Functions/upperUTF8.cpp index a6c7a4d41cd..1a85b133757 100644 --- a/src/Functions/upperUTF8.cpp +++ b/src/Functions/upperUTF8.cpp @@ -6,6 +6,8 @@ namespace DB { +namespace +{ struct NameUpperUTF8 { @@ -14,6 +16,8 @@ struct NameUpperUTF8 using FunctionUpperUTF8 = FunctionStringToString>, NameUpperUTF8>; +} + void registerFunctionUpperUTF8(FunctionFactory & factory) { factory.registerFunction(); From ee218c354ed5a10f7d77cce354385796d708f965 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 8 Sep 2020 02:18:07 +0300 Subject: [PATCH 106/298] fix segfault in combinator -Resample --- src/AggregateFunctions/AggregateFunctionResample.h | 14 +++++++++++++- .../0_stateless/01463_resample_overflow.reference | 0 .../0_stateless/01463_resample_overflow.sql | 1 + 3 files changed, 14 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01463_resample_overflow.reference create mode 100644 tests/queries/0_stateless/01463_resample_overflow.sql diff --git a/src/AggregateFunctions/AggregateFunctionResample.h b/src/AggregateFunctions/AggregateFunctionResample.h index 92fa8fbb2a5..c1528686785 100644 --- a/src/AggregateFunctions/AggregateFunctionResample.h +++ b/src/AggregateFunctions/AggregateFunctionResample.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -60,7 +61,18 @@ public: if (end < begin) total = 0; else - total = (end - begin + step - 1) / step; + { + Key dif; + size_t sum; + if (common::subOverflow(end, begin, dif) + || common::addOverflow(static_cast(dif), step, sum)) + { + throw Exception("Overflow in internal computations in function " + getName() + + ". Too large arguments", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } + + total = (sum - 1) / step; // total = (end - begin + step - 1) / step + } if (total > MAX_ELEMENTS) throw Exception("The range given in function " diff --git a/tests/queries/0_stateless/01463_resample_overflow.reference b/tests/queries/0_stateless/01463_resample_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01463_resample_overflow.sql b/tests/queries/0_stateless/01463_resample_overflow.sql new file mode 100644 index 00000000000..298f852ed14 --- /dev/null +++ b/tests/queries/0_stateless/01463_resample_overflow.sql @@ -0,0 +1 @@ +select groupArrayResample(-9223372036854775808, 9223372036854775807, 9223372036854775807)(number, toInt64(number)) FROM numbers(7); -- { serverError 69 } From 7373aaf465134a9e9875f346171da0627bf26d6f Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Tue, 8 Sep 2020 03:59:13 +0300 Subject: [PATCH 107/298] add IsOperation helper --- src/Functions/FunctionBinaryArithmetic.h | 69 ++++++++---------------- src/Functions/FunctionUnaryArithmetic.h | 6 +-- src/Functions/FunctionsComparison.h | 10 ++-- src/Functions/IsOperation.h | 62 +++++++++++++++++++++ src/Functions/abs.cpp | 4 -- src/Functions/divide.cpp | 5 -- src/Functions/greatest.cpp | 4 -- src/Functions/intDivOrZero.cpp | 4 -- src/Functions/jumpConsistentHash.cpp | 2 +- src/Functions/least.cpp | 4 -- src/Functions/minus.cpp | 4 -- src/Functions/multiply.cpp | 4 -- src/Functions/negate.cpp | 4 -- src/Functions/plus.cpp | 4 -- 14 files changed, 93 insertions(+), 93 deletions(-) create mode 100644 src/Functions/IsOperation.h diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 2a467451684..d899a95ddc6 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -22,6 +22,7 @@ #include #include "IFunctionImpl.h" #include "FunctionHelpers.h" +#include "IsOperation.h" #include "DivisionUtils.h" #include "castTypeToEither.h" #include "FunctionFactory.h" @@ -167,17 +168,6 @@ struct BinaryOperationImpl : BinaryOperationImplBase }; -template struct PlusImpl; -template struct MinusImpl; -template struct MultiplyImpl; -template struct DivideFloatingImpl; -template struct DivideIntegralImpl; -template struct DivideIntegralOrZeroImpl; -template struct LeastBaseImpl; -template struct GreatestBaseImpl; -template struct ModuloImpl; - - /// Binary operations for Decimals need scale args /// +|- scale one of args (which scale factor is not 1). ScaleR = oneof(Scale1, Scale2); /// * no agrs scale. ScaleR = Scale1 + Scale2; @@ -185,15 +175,15 @@ template struct ModuloImpl; template typename Operation, typename ResultType_, bool _check_overflow = true> struct DecimalBinaryOperation { - static constexpr bool is_plus_minus = std::is_same_v, PlusImpl> || - std::is_same_v, MinusImpl>; - static constexpr bool is_multiply = std::is_same_v, MultiplyImpl>; - static constexpr bool is_float_division = std::is_same_v, DivideFloatingImpl>; - static constexpr bool is_int_division = std::is_same_v, DivideIntegralImpl> || - std::is_same_v, DivideIntegralOrZeroImpl>; + static constexpr bool is_plus_minus = IsOperation::plus || + IsOperation::minus; + static constexpr bool is_multiply = IsOperation::multiply; + static constexpr bool is_float_division = IsOperation::div_floating; + static constexpr bool is_int_division = IsOperation::div_int || + IsOperation::div_int_or_zero; static constexpr bool is_division = is_float_division || is_int_division; - static constexpr bool is_compare = std::is_same_v, LeastBaseImpl> || - std::is_same_v, GreatestBaseImpl>; + static constexpr bool is_compare = IsOperation::least || + IsOperation::greatest; static constexpr bool is_plus_minus_compare = is_plus_minus || is_compare; static constexpr bool can_overflow = is_plus_minus || is_multiply; @@ -529,15 +519,7 @@ private: /// it's not correct for Decimal using Op = Operation; public: - static constexpr bool allow_decimal = - std::is_same_v, PlusImpl> || - std::is_same_v, MinusImpl> || - std::is_same_v, MultiplyImpl> || - std::is_same_v, DivideFloatingImpl> || - std::is_same_v, DivideIntegralImpl> || - std::is_same_v, DivideIntegralOrZeroImpl> || - std::is_same_v, LeastBaseImpl> || - std::is_same_v, GreatestBaseImpl>; + static constexpr bool allow_decimal = IsOperation::allow_decimal; /// Appropriate result type for binary operator on numeric types. "Date" can also mean /// DateTime, but if both operands are Dates, their type must be the same (e.g. Date - DateTime is invalid). @@ -556,21 +538,21 @@ public: DataTypeFromFieldType>, /// Date + Integral -> Date /// Integral + Date -> Date - Case>, Switch< + Case::plus, Switch< Case, LeftDataType>, Case, RightDataType>>>, /// Date - Date -> Int32 /// Date - Integral -> Date - Case>, Switch< + Case::minus, Switch< Case, DataTypeInt32>, Case && IsIntegral, LeftDataType>>>, /// least(Date, Date) -> Date /// greatest(Date, Date) -> Date - Case && (std::is_same_v> || std::is_same_v>), + Case && (IsOperation::least || IsOperation::greatest), LeftDataType>, /// Date % Int32 -> Int32 /// Date % Float -> Float64 - Case>, Switch< + Case::modulo, Switch< Case && IsIntegral, RightDataType>, Case && IsFloatingPoint, DataTypeFloat64>>>>; }; @@ -627,10 +609,9 @@ class FunctionBinaryArithmetic : public IFunction /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. /// We construct another function (example: addMonths) and call it. - static constexpr bool function_is_plus = std::is_same_v, PlusImpl>; - static constexpr bool function_is_minus = std::is_same_v, MinusImpl>; - - if (!function_is_plus && !function_is_minus) + static constexpr bool function_is_plus = IsOperation::plus; + static constexpr bool function_is_minus = IsOperation::minus; + if constexpr (!function_is_plus && !function_is_minus) return {}; const DataTypePtr & type_time = first_is_date_or_datetime ? type0 : type1; @@ -669,7 +650,7 @@ class FunctionBinaryArithmetic : public IFunction bool isAggregateMultiply(const DataTypePtr & type0, const DataTypePtr & type1) const { - if constexpr (!std::is_same_v, MultiplyImpl>) + if constexpr (!IsOperation::multiply) return false; WhichDataType which0(type0); @@ -681,7 +662,7 @@ class FunctionBinaryArithmetic : public IFunction bool isAggregateAddition(const DataTypePtr & type0, const DataTypePtr & type1) const { - if constexpr (!std::is_same_v, PlusImpl>) + if constexpr (!IsOperation::plus) return false; WhichDataType which0(type0); @@ -891,10 +872,8 @@ public: { if constexpr (IsDataTypeDecimal && IsDataTypeDecimal) { - constexpr bool is_multiply = std::is_same_v, MultiplyImpl>; - constexpr bool is_division = std::is_same_v, DivideFloatingImpl> || - std::is_same_v, DivideIntegralImpl> || - std::is_same_v, DivideIntegralOrZeroImpl>; + constexpr bool is_multiply = IsOperation::multiply; + constexpr bool is_division = IsOperation::division; ResultDataType result_type = decimalResultType(left, right, is_multiply, is_division); type_res = std::make_shared(result_type.getPrecision(), result_type.getScale()); @@ -1016,10 +995,8 @@ public: if constexpr (!std::is_same_v) { constexpr bool result_is_decimal = IsDataTypeDecimal || IsDataTypeDecimal; - constexpr bool is_multiply = std::is_same_v, MultiplyImpl>; - constexpr bool is_division = std::is_same_v, DivideFloatingImpl> || - std::is_same_v, DivideIntegralImpl> || - std::is_same_v, DivideIntegralOrZeroImpl>; + constexpr bool is_multiply = IsOperation::multiply; + constexpr bool is_division = IsOperation::division; using T0 = typename LeftDataType::FieldType; using T1 = typename RightDataType::FieldType; diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index 5b072e5848a..94f3fd291de 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #if !defined(ARCADIA_BUILD) @@ -71,9 +72,6 @@ struct FixedStringUnaryOperationImpl template struct FunctionUnaryArithmeticMonotonicity; -template struct AbsImpl; -template struct NegateImpl; - /// Used to indicate undefined operation struct InvalidType; @@ -81,7 +79,7 @@ struct InvalidType; template