From f8f67a788e4c8dc41b59d6f22631172fb4a431df Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 25 Jun 2020 19:55:45 +0300 Subject: [PATCH 001/341] 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/341] 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/341] 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 230938d3a3082fbf241c9d873571231a69a5f450 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 11 Jul 2020 15:12:42 +0800 Subject: [PATCH 004/341] 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 005/341] 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 006/341] 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 007/341] 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 008/341] 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 009/341] 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 25140b9bd5b6421b84ef8586827cc49b9d015e7b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Sep 2020 04:39:36 +0300 Subject: [PATCH 010/341] 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 011/341] 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 012/341] 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 013/341] 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 014/341] 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 015/341] 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 016/341] 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 3cadc9033ae63d7faa851b1707b3c6f9ce1a36aa Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Sep 2020 18:26:49 +0300 Subject: [PATCH 017/341] 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 018/341] 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 019/341] 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 020/341] 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 23b9677879a2a0618b35032439650ec08e760c57 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 4 Sep 2020 08:46:58 +0300 Subject: [PATCH 021/341] 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 022/341] 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 7b95e56e8c902578f8fcebc5d9edeccce1eb35ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 03:09:40 +0300 Subject: [PATCH 023/341] 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 024/341] 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 025/341] 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 026/341] 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 027/341] 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 028/341] 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 029/341] 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 030/341] 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 031/341] 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 032/341] 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 033/341] 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 034/341] 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 035/341] 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 036/341] 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 037/341] 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 038/341] 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 039/341] 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 040/341] 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 041/341] 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 042/341] 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 043/341] 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 044/341] 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 045/341] 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 046/341] 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 047/341] 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 048/341] 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 3942cc615f03ecb8e5b9e7437fdc5c57613c245d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 7 Sep 2020 10:09:42 +0300 Subject: [PATCH 049/341] 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 4c3c1cdaf3e4064a5d65a40dea5383e522e8f2ee Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 7 Sep 2020 15:24:27 +0800 Subject: [PATCH 050/341] 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 99ebab706cf70a286a5a6b5b2ac6070085f1ebf0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 12:02:22 +0300 Subject: [PATCH 051/341] 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 661d9bdb4c1489d6a9c5c8f0ae6d06bb5480a2b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 19:03:36 +0300 Subject: [PATCH 052/341] 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 053/341] 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 a56d42de67496404fb507d05c2d399012fd479ce Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 7 Sep 2020 20:55:06 +0300 Subject: [PATCH 054/341] 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 0c00b992d5741f7d9750567afd3cbda92b019273 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 7 Sep 2020 16:01:49 -0700 Subject: [PATCH 055/341] StorageReplicatedMergeTree - cleanup data dir after Zk exceptions It's possible for `getZooKeeper()` to timeout if zookeeper host(s) can't be reached. In such cases `Poco::Exception` is thrown after a connection timeout - refer to `src/Common/ZooKeeper/ZooKeeperImpl.cpp:866` for more info. Side effect of this is that the CreateQuery gets interrupted and it exits. But the data Directories for the tables being created aren't cleaned up. This unclean state will hinder table creation on any retries and will complain that the Directory for table already exists. To achieve a clean state on failed table creations, catch this error if the exception is of type Poco::Exception and call `dropIfEmpty()` method, then proceed throwing the exception. Without this, the Directory for the tables need to be manually deleted before retrying the CreateQuery. --- src/Storages/StorageReplicatedMergeTree.cpp | 25 ++++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6058632d220..64eb1c8d4cd 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -216,7 +216,30 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mutationsFinalizingTask)", [this] { mutationsFinalizingTask(); }); if (global_context.hasZooKeeper()) - current_zookeeper = global_context.getZooKeeper(); + { + /// It's possible for getZooKeeper() to timeout if zookeeper host(s) can't + /// be reached. In such cases Poco::Exception is thrown after a connection + /// timeout - refer to src/Common/ZooKeeper/ZooKeeperImpl.cpp:866 for more info. + /// + /// Side effect of this is that the CreateQuery gets interrupted and it exits. + /// But the data Directories for the tables being created aren't cleaned up. + /// This unclean state will hinder table creation on any retries and will + /// complain that the Directory for table already exists. + /// + /// To acheive a clean state on failed table creations, catch this error if + /// the excaption is of type Poco::Exception and call dropIfEmpty() method, + /// then proceed throwing the exception. Without this, the Directory for the + /// tables need to be manually deleted before retrying the CreateQuery. + try + { + current_zookeeper = global_context.getZooKeeper(); + } + catch (Poco::Exception & e) + { + dropIfEmpty(); + throw e; + } + } bool skip_sanity_checks = false; From 806334a642a87c09b77a2b69c765ea10ecbcaad7 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 7 Sep 2020 22:27:49 -0700 Subject: [PATCH 056/341] StorageReplicatedMergeTree - add integration tests that test dir cleanup This adds a integration test that tests if table directory is cleaned up after a ZooKeeper connection failure for ReplicatedMergeTree tables. --- .../__init__.py | 0 .../configs/remote_servers.xml | 14 ++++++ .../test_replicated_zk_conn_failure/test.py | 45 +++++++++++++++++++ 3 files changed, 59 insertions(+) create mode 100644 tests/integration/test_replicated_zk_conn_failure/__init__.py create mode 100644 tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml create mode 100644 tests/integration/test_replicated_zk_conn_failure/test.py diff --git a/tests/integration/test_replicated_zk_conn_failure/__init__.py b/tests/integration/test_replicated_zk_conn_failure/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml b/tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml new file mode 100644 index 00000000000..538aa72d386 --- /dev/null +++ b/tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml @@ -0,0 +1,14 @@ + + + + + true + + shard_0 + node1 + 9000 + + + + + diff --git a/tests/integration/test_replicated_zk_conn_failure/test.py b/tests/integration/test_replicated_zk_conn_failure/test.py new file mode 100644 index 00000000000..3f106bd2981 --- /dev/null +++ b/tests/integration/test_replicated_zk_conn_failure/test.py @@ -0,0 +1,45 @@ +import time + +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager + + +# This tests if the data directory for a table is cleaned up if there is a Zookeeper +# connection exception during a CreateQuery operation involving ReplicatedMergeTree tables. +# Test flow is as follows: +# 1. Configure cluster with ZooKeeper and create a database. +# 2. Drop all connections to ZooKeeper. +# 3. Try creating the table and there would be a Poco:Exception. +# 4. Try creating the table again and there should not be any error +# that indicates that the Directory for table already exists. + + +def test_replicated_zk_conn_failure(): + cluster = ClickHouseCluster(__file__) + node1 = cluster.add_instance('node1', main_configs=["configs/remote_servers.xml"], with_zookeeper=True) + try: + cluster.start() + node1.query("CREATE DATABASE replica;") + query_create = '''CREATE TABLE replica.test + ( + id Int64, + event_time DateTime + ) + Engine=ReplicatedMergeTree('/clickhouse/tables/replica/test', 'node1') + PARTITION BY toYYYYMMDD(event_time) + ORDER BY id;'''.format(replica=node1.name) + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node1) + time.sleep(5) + error = node1.query_and_get_error(query_create) + # Assert that there was net exception. + assert "Poco::Exception. Code: 1000" in error + # Assert that the exception was due to ZooKeeper connectivity. + assert "All connection tries failed while connecting to ZooKeeper" in error + # retry table creation + error = node1.query_and_get_error(query_create) + # Should not expect any errors related to directory already existing + # and those should have been already cleaned up during the previous retry. + assert "Directory for table data data/replica/test/ already exists" not in error + finally: + cluster.shutdown() From 78eac658b0d3250c10e177f38a88cc32ef102b3a Mon Sep 17 00:00:00 2001 From: myrrc Date: Tue, 8 Sep 2020 22:51:44 +0300 Subject: [PATCH 057/341] adding correct LC nested nullability checker --- src/Columns/ColumnLowCardinality.h | 3 ++- src/Functions/array/arrayIndex.h | 8 +++++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index e3b879d6dd5..00f58a133cf 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -170,7 +170,8 @@ public: size_t sizeOfValueIfFixed() const override { return getDictionary().sizeOfValueIfFixed(); } bool isNumeric() const override { return getDictionary().isNumeric(); } bool lowCardinality() const override { return true; } - bool isNullable() const override { return isColumnNullable(*dictionary.getColumnUniquePtr()); } + + bool nestedIsNullable() const { return isColumnNullable(*dictionary.getColumnUnique().getNestedColumn()); } const IColumnUnique & getDictionary() const { return dictionary.getColumnUnique(); } const ColumnPtr & getDictionaryPtr() const { return dictionary.getColumnUniquePtr(); } diff --git a/src/Functions/array/arrayIndex.h b/src/Functions/array/arrayIndex.h index b12372d28ce..f96eb09c861 100644 --- a/src/Functions/array/arrayIndex.h +++ b/src/Functions/array/arrayIndex.h @@ -1,5 +1,3 @@ -#include -#include #include #include #include @@ -741,6 +739,10 @@ private: if (!col_lc) return false; +// assert(checkAndGetColumn(col_lc->getDictionaryPtr().get())); +// assert(col_lc->isNullable()); +// assert(isColumnNullable(*col_lc->getDictionaryPtr().get())); + const auto [null_map_data, null_map_item] = getNullMaps(block, arguments); const IColumn& col_arg = *block.getByPosition(arguments[1]).column.get(); @@ -799,7 +801,7 @@ private: block.getByPosition(result).column = std::move(col_result); return true; } - else if (col_lc->getDictionaryPtr()->isNullable()) // LC(Nullable(T)) and U + else if (col_lc->nestedIsNullable()) // LC(Nullable(T)) and U { const ColumnPtr left_casted = col_lc->convertToFullColumnIfLowCardinality(); // Nullable(T) const ColumnNullable& left_nullable = *checkAndGetColumn(left_casted.get()); From 4f1321daef3af1a4eac14d7c0d33f925e6bb5557 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 8 Sep 2020 23:05:27 +0300 Subject: [PATCH 058/341] Make 00443_merge_tree_uniform_read_distribution_0.reference real file (was symlink) --- .../00443_merge_tree_uniform_read_distribution_0.reference | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) mode change 120000 => 100644 tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.reference diff --git a/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.reference b/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.reference deleted file mode 120000 index 4d2577d02a3..00000000000 --- a/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.reference +++ /dev/null @@ -1 +0,0 @@ -00443_optimize_final_vertical_merge.reference \ No newline at end of file diff --git a/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.reference b/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.reference new file mode 100644 index 00000000000..bb6e92ae8e7 --- /dev/null +++ b/tests/queries/0_stateless/00443_merge_tree_uniform_read_distribution_0.reference @@ -0,0 +1,3 @@ +1500000 1500000 1500000 1500000 1500000 1500000 +[['def']] [['','']] +0 From 014c7c02bdc454a4b53c44d1525c9932906d07ca Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 8 Sep 2020 23:34:10 +0300 Subject: [PATCH 059/341] Fix some trailing whitespaces in query format The following statements still has the trailing whitespace: - WITH - SELECT - SELECT DISTINCT - ARRAY JOIN - GROUP BY - ORDER BY - LIMIT BY --- src/Parsers/ASTExpressionList.cpp | 9 +++++++++ src/Parsers/ASTFunction.cpp | 1 + src/Parsers/ASTSelectQuery.cpp | 12 +++++++----- src/Parsers/ASTTablesInSelectQuery.cpp | 5 ++++- src/Parsers/IAST.h | 1 + 5 files changed, 22 insertions(+), 6 deletions(-) diff --git a/src/Parsers/ASTExpressionList.cpp b/src/Parsers/ASTExpressionList.cpp index abab1e895cf..de38e1fd7ea 100644 --- a/src/Parsers/ASTExpressionList.cpp +++ b/src/Parsers/ASTExpressionList.cpp @@ -13,6 +13,9 @@ ASTPtr ASTExpressionList::clone() const void ASTExpressionList::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { + if (frame.expression_list_prepend_whitespace) + settings.ostr << ' '; + for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { if (it != children.begin()) @@ -30,6 +33,12 @@ void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, For { std::string indent_str = "\n" + std::string(4 * (frame.indent + 1), ' '); + if (frame.expression_list_prepend_whitespace) + { + if (!(children.size() > 1 || frame.expression_list_always_start_on_new_line)) + settings.ostr << ' '; + } + ++frame.indent; for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 07429c8104f..ebef4261d01 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -114,6 +114,7 @@ static bool highlightStringLiteralWithMetacharacters(const ASTPtr & node, const void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { + frame.expression_list_prepend_whitespace = false; FormatStateStacked nested_need_parens = frame; FormatStateStacked nested_dont_need_parens = frame; nested_need_parens.need_parens = true; diff --git a/src/Parsers/ASTSelectQuery.cpp b/src/Parsers/ASTSelectQuery.cpp index fdc7bd47e4d..499761c4634 100644 --- a/src/Parsers/ASTSelectQuery.cpp +++ b/src/Parsers/ASTSelectQuery.cpp @@ -72,18 +72,20 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F { frame.current_select = this; frame.need_parens = false; + frame.expression_list_prepend_whitespace = true; + std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' '); if (with()) { - s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "WITH " << (s.hilite ? hilite_none : ""); + s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "WITH" << (s.hilite ? hilite_none : ""); s.one_line ? with()->formatImpl(s, state, frame) : with()->as().formatImplMultiline(s, state, frame); s.ostr << s.nl_or_ws; } - s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "SELECT " << (distinct ? "DISTINCT " : "") << (s.hilite ? hilite_none : ""); + s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "SELECT" << (distinct ? " DISTINCT" : "") << (s.hilite ? hilite_none : ""); s.one_line ? select()->formatImpl(s, state, frame) @@ -109,7 +111,7 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F if (groupBy()) { - s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "GROUP BY " << (s.hilite ? hilite_none : ""); + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "GROUP BY" << (s.hilite ? hilite_none : ""); s.one_line ? groupBy()->formatImpl(s, state, frame) : groupBy()->as().formatImplMultiline(s, state, frame); @@ -132,7 +134,7 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F if (orderBy()) { - s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "ORDER BY " << (s.hilite ? hilite_none : ""); + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "ORDER BY" << (s.hilite ? hilite_none : ""); s.one_line ? orderBy()->formatImpl(s, state, frame) : orderBy()->as().formatImplMultiline(s, state, frame); @@ -147,7 +149,7 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F s.ostr << ", "; } limitByLength()->formatImpl(s, state, frame); - s.ostr << (s.hilite ? hilite_keyword : "") << " BY " << (s.hilite ? hilite_none : ""); + s.ostr << (s.hilite ? hilite_keyword : "") << " BY" << (s.hilite ? hilite_none : ""); s.one_line ? limitBy()->formatImpl(s, state, frame) : limitBy()->as().formatImplMultiline(s, state, frame); diff --git a/src/Parsers/ASTTablesInSelectQuery.cpp b/src/Parsers/ASTTablesInSelectQuery.cpp index 0fd93bbd04d..eb3446ca1c4 100644 --- a/src/Parsers/ASTTablesInSelectQuery.cpp +++ b/src/Parsers/ASTTablesInSelectQuery.cpp @@ -210,6 +210,7 @@ void ASTTableJoin::formatImplBeforeTable(const FormatSettings & settings, Format void ASTTableJoin::formatImplAfterTable(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { frame.need_parens = false; + frame.expression_list_prepend_whitespace = false; if (using_expression_list) { @@ -236,8 +237,10 @@ void ASTTableJoin::formatImpl(const FormatSettings & settings, FormatState & sta void ASTArrayJoin::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { + frame.expression_list_prepend_whitespace = true; + settings.ostr << (settings.hilite ? hilite_keyword : "") - << (kind == Kind::Left ? "LEFT " : "") << "ARRAY JOIN " << (settings.hilite ? hilite_none : ""); + << (kind == Kind::Left ? "LEFT " : "") << "ARRAY JOIN" << (settings.hilite ? hilite_none : ""); settings.one_line ? expression_list->formatImpl(settings, state, frame) diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index c0c286ac0d2..cf6a7efc102 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -203,6 +203,7 @@ public: UInt8 indent = 0; bool need_parens = false; bool expression_list_always_start_on_new_line = false; /// Line feed and indent before expression list even if it's of single element. + bool expression_list_prepend_whitespace = false; /// Prepend whitespace (if it is required) const IAST * current_select = nullptr; }; From 03247707737af0f76cacdd6b8b4544e1967f7ed9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 8 Sep 2020 23:05:47 +0300 Subject: [PATCH 060/341] Remove all trailing whitespaces in tests sed -r -i \ -e 's/SELECT $/SELECT/' \ -e 's/SELECT DISTINCT $/SELECT DISTINCT/' \ -e 's/WITH $/WITH/' \ # zero matches, new test will be added -e 's/ARRAY JOIN $/ARRAY JOIN/' \ -e 's/GROUP BY $/GROUP BY/' \ -e 's/ORDER BY $/ORDER BY/' \ -e 's/LIMIT ([0-9]+) BY $/LIMIT \1 BY/' \ # zero matches, new test will be added tests/queries/*/*.sql \ tests/queries/*/*.reference (With except for tests/queries/0_stateless/00751_default_databasename_for_view.reference) --- .../00239_type_conversion_in_in.sql | 2 +- .../0_stateless/00250_tuple_comparison.sql | 2 +- .../00344_row_number_in_all_blocks.sql | 2 +- .../0_stateless/00436_convert_charset.sql | 4 +- .../00541_to_start_of_fifteen_minutes.sql | 4 +- .../00552_logical_functions_uint8_as_bool.sql | 2 +- ...ll_subquery_aggregation_column_removal.sql | 72 +++++----- .../00597_push_down_predicate.reference | 128 +++++++++--------- .../queries/0_stateless/00618_nullable_in.sql | 4 +- .../00712_prewhere_with_alias_bug_2.sql | 2 +- .../00740_optimize_predicate_expression.sql | 2 +- .../00743_limit_by_not_found_column.sql | 2 +- ...51_default_databasename_for_view.reference | 2 +- tests/queries/0_stateless/00759_kodieg.sql | 2 +- .../00826_cross_to_inner_join.reference | 16 +-- .../00849_multiple_comma_join.reference | 40 +++--- .../00849_multiple_comma_join_2.reference | 30 ++-- .../0_stateless/00908_analyze_query.reference | 2 +- .../0_stateless/00941_to_custom_week.sql | 2 +- ...0957_format_with_clashed_aliases.reference | 2 +- ...58_format_of_tuple_array_element.reference | 2 +- ...hecksums_in_system_parts_columns_table.sql | 2 +- .../01056_predicate_optimizer_bugs.reference | 26 ++-- ...76_predicate_optimizer_with_view.reference | 4 +- .../01083_cross_to_inner_with_like.reference | 6 +- .../0_stateless/01117_chain_finalize_bug.sql | 2 +- .../01250_fixed_string_comparison.sql | 2 +- ...ithmetic_operations_in_aggr_func.reference | 84 ++++++------ .../01272_totals_and_filter_bug.sql | 6 +- .../01278_format_multiple_queries.reference | 4 +- .../01300_group_by_other_keys.reference | 18 +-- ...01300_group_by_other_keys_having.reference | 8 +- ...egate_functions_of_group_by_keys.reference | 28 ++-- ...monotonous_functions_in_order_by.reference | 12 +- .../01322_any_input_optimize.reference | 4 +- ..._redundant_functions_in_order_by.reference | 30 ++-- .../01355_defaultValueOfArgumentType_bug.sql | 2 +- .../01372_wrong_order_by_removal.reference | 6 +- .../01379_with_fill_several_columns.sql | 8 +- .../01390_remove_injective_in_uniq.reference | 28 ++-- ...dicate_when_contains_with_clause.reference | 4 +- .../0_stateless/01418_index_analysis_bug.sql | 4 +- ..._duplicate_distinct_optimization.reference | 28 ++-- .../01470_columns_transformers.reference | 22 +-- .../1_stateful/00063_loyalty_joins.sql | 20 +-- 45 files changed, 341 insertions(+), 341 deletions(-) diff --git a/tests/queries/0_stateless/00239_type_conversion_in_in.sql b/tests/queries/0_stateless/00239_type_conversion_in_in.sql index 6e76a31ac56..5589d91ce74 100644 --- a/tests/queries/0_stateless/00239_type_conversion_in_in.sql +++ b/tests/queries/0_stateless/00239_type_conversion_in_in.sql @@ -2,7 +2,7 @@ select 1 as x, x = 1 or x = 2 or x = 3 or x = -1; select 1 as x, x = 1.0 or x = 2 or x = 3 or x = -1; select 1 as x, x = 1.5 or x = 2 or x = 3 or x = -1; -SELECT +SELECT 1 IN (1, -1, 2.0, 2.5), 1.0 IN (1, -1, 2.0, 2.5), 1 IN (1.0, -1, 2.0, 2.5), diff --git a/tests/queries/0_stateless/00250_tuple_comparison.sql b/tests/queries/0_stateless/00250_tuple_comparison.sql index 2ee29ef5863..03a4d23a271 100644 --- a/tests/queries/0_stateless/00250_tuple_comparison.sql +++ b/tests/queries/0_stateless/00250_tuple_comparison.sql @@ -1,4 +1,4 @@ -SELECT +SELECT (1, 'Hello', 23) = (1, 'Hello', 23), (1, 'Hello', 23) != (1, 'Hello', 23), (1, 'Hello', 23) < (1, 'Hello', 23), diff --git a/tests/queries/0_stateless/00344_row_number_in_all_blocks.sql b/tests/queries/0_stateless/00344_row_number_in_all_blocks.sql index 865d0084ce8..43727f2a14e 100644 --- a/tests/queries/0_stateless/00344_row_number_in_all_blocks.sql +++ b/tests/queries/0_stateless/00344_row_number_in_all_blocks.sql @@ -1,5 +1,5 @@ SET max_block_size = 1000; -SELECT +SELECT groupUniqArray(blockSize()), uniqExact(rowNumberInAllBlocks()), min(rowNumberInAllBlocks()), diff --git a/tests/queries/0_stateless/00436_convert_charset.sql b/tests/queries/0_stateless/00436_convert_charset.sql index cba91fe67f4..1b7baf22450 100644 --- a/tests/queries/0_stateless/00436_convert_charset.sql +++ b/tests/queries/0_stateless/00436_convert_charset.sql @@ -1,4 +1,4 @@ -SELECT +SELECT 'абвгдеёжзийклмнопрстуфхцчшщъыьэюяАБВГДЕЁЖЗИЙКЛМНОПРСТУФХЦЧШЩЪЫЬЭЮЯ' AS orig, hex(convertCharset(orig, 'utf-8', 'cp1251') AS cp1251) AS cp1251_hex, hex(convertCharset(orig, 'utf-8', 'utf-7')) AS utf7_hex, @@ -13,7 +13,7 @@ SELECT convertCharset(broken3, 'utf-8', 'koi8-r') AS restored3 FORMAT Vertical; -SELECT +SELECT materialize('абвгдеёжзийклмнопрстуфхцчшщъыьэюяАБВГДЕЁЖЗИЙКЛМНОПРСТУФХЦЧШЩЪЫЬЭЮЯ') AS orig, hex(convertCharset(orig, 'utf-8', 'cp1251') AS cp1251) AS cp1251_hex, hex(convertCharset(orig, 'utf-8', 'utf-7')) AS utf7_hex, diff --git a/tests/queries/0_stateless/00541_to_start_of_fifteen_minutes.sql b/tests/queries/0_stateless/00541_to_start_of_fifteen_minutes.sql index 29056eea3d6..0c20670fad2 100644 --- a/tests/queries/0_stateless/00541_to_start_of_fifteen_minutes.sql +++ b/tests/queries/0_stateless/00541_to_start_of_fifteen_minutes.sql @@ -1,7 +1,7 @@ -SELECT +SELECT DISTINCT result FROM ( - SELECT + SELECT toStartOfFifteenMinutes(toDateTime('2017-12-25 00:00:00') + number * 60) AS result FROM system.numbers LIMIT 120 diff --git a/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.sql b/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.sql index feee33add1c..f62a02288ed 100644 --- a/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.sql +++ b/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.sql @@ -10,7 +10,7 @@ SELECT 1 OR 2 OR 4 ; -SELECT +SELECT toUInt8(bitAnd(number, 4)) AS a, toUInt8(bitAnd(number, 2)) AS b, toUInt8(bitAnd(number, 1)) AS c, diff --git a/tests/queries/0_stateless/00585_union_all_subquery_aggregation_column_removal.sql b/tests/queries/0_stateless/00585_union_all_subquery_aggregation_column_removal.sql index 47e6582134a..bf5d2251470 100644 --- a/tests/queries/0_stateless/00585_union_all_subquery_aggregation_column_removal.sql +++ b/tests/queries/0_stateless/00585_union_all_subquery_aggregation_column_removal.sql @@ -8,19 +8,19 @@ INSERT INTO clicks VALUES ('facebook.com'), ('yandex.ru'), ('google.com'); INSERT INTO transactions VALUES ('facebook.com'), ('yandex.ru'), ('baidu.com'); -SELECT +SELECT sum(total_count) AS total, domain FROM ( - SELECT + SELECT COUNT(*) AS total_count, toUInt64(0) AS facebookHits, domain FROM transactions GROUP BY domain UNION ALL - SELECT + SELECT COUNT(*) AS total_count, SUM(if(domain = 'facebook.com', 1, 0)) AS facebookHits, domain @@ -33,19 +33,19 @@ LIMIT 10 FORMAT JSONEachRow; -SELECT +SELECT sum(total_count) AS total, domain FROM ( - SELECT + SELECT COUNT(*) AS total_count, SUM(if(domain = 'facebook.com', 1, 0)) AS facebookHits, domain FROM clicks GROUP BY domain UNION ALL - SELECT + SELECT COUNT(*) AS total_count, toUInt64(0) AS facebookHits, domain @@ -60,19 +60,19 @@ FORMAT JSONEachRow; SELECT DISTINCT * FROM ( -SELECT +SELECT sum(total_count) AS total, domain FROM ( - SELECT + SELECT COUNT(*) AS total_count, toUInt64(0) AS facebookHits, domain FROM transactions GROUP BY domain UNION ALL - SELECT + SELECT COUNT(*) AS total_count, SUM(if(domain = 'facebook.com', 1, 0)) AS facebookHits, domain @@ -85,19 +85,19 @@ LIMIT 10 UNION ALL -SELECT +SELECT sum(total_count) AS total, domain FROM ( - SELECT + SELECT COUNT(*) AS total_count, SUM(if(domain = 'facebook.com', 1, 0)) AS facebookHits, domain FROM clicks GROUP BY domain UNION ALL - SELECT + SELECT COUNT(*) AS total_count, toUInt64(0) AS facebookHits, domain @@ -112,20 +112,20 @@ LIMIT 10 SELECT DISTINCT total, domain FROM ( -SELECT +SELECT sum(total_count) AS total, sum(facebookHits) AS facebook, domain FROM ( - SELECT + SELECT COUNT(*) AS total_count, toUInt64(0) AS facebookHits, domain FROM transactions GROUP BY domain UNION ALL - SELECT + SELECT COUNT(*) AS total_count, SUM(if(domain = 'facebook.com', 1, 0)) AS facebookHits, domain @@ -138,20 +138,20 @@ LIMIT 10 UNION ALL -SELECT +SELECT sum(total_count) AS total, max(facebookHits) AS facebook, domain FROM ( - SELECT + SELECT COUNT(*) AS total_count, SUM(if(domain = 'facebook.com', 1, 0)) AS facebookHits, domain FROM clicks GROUP BY domain UNION ALL - SELECT + SELECT COUNT(*) AS total_count, toUInt64(0) AS facebookHits, domain @@ -167,19 +167,19 @@ ORDER BY domain, total; SELECT * FROM ( -SELECT +SELECT sum(total_count) AS total, domain FROM ( - SELECT + SELECT COUNT(*) AS total_count, toUInt64(0) AS facebookHits, domain FROM transactions GROUP BY domain UNION ALL - SELECT + SELECT COUNT(*) AS total_count, SUM(if(domain = 'facebook.com', 1, 0)) AS facebookHits, domain @@ -192,19 +192,19 @@ LIMIT 10 ) js1 ALL FULL OUTER JOIN ( -SELECT +SELECT sum(total_count) AS total, domain FROM ( - SELECT + SELECT COUNT(*) AS total_count, SUM(if(domain = 'facebook.com', 1, 0)) AS facebookHits, domain FROM clicks GROUP BY domain UNION ALL - SELECT + SELECT COUNT(*) AS total_count, toUInt64(0) AS facebookHits, domain @@ -221,19 +221,19 @@ ORDER BY total, domain; SELECT total FROM ( -SELECT +SELECT sum(total_count) AS total, domain FROM ( - SELECT + SELECT COUNT(*) AS total_count, toUInt64(0) AS facebookHits, domain FROM transactions GROUP BY domain UNION ALL - SELECT + SELECT COUNT(*) AS total_count, SUM(if(domain = 'facebook.com', 1, 0)) AS facebookHits, domain @@ -246,19 +246,19 @@ LIMIT 10 ) js1 ALL FULL OUTER JOIN ( -SELECT +SELECT sum(total_count) AS total, domain FROM ( - SELECT + SELECT COUNT(*) AS total_count, SUM(if(domain = 'facebook.com', 1, 0)) AS facebookHits, domain FROM clicks GROUP BY domain UNION ALL - SELECT + SELECT COUNT(*) AS total_count, toUInt64(0) AS facebookHits, domain @@ -275,19 +275,19 @@ ORDER BY total, domain; SELECT domain FROM ( -SELECT +SELECT sum(total_count) AS total, domain FROM ( - SELECT + SELECT COUNT(*) AS total_count, toUInt64(0) AS facebookHits, domain FROM transactions GROUP BY domain UNION ALL - SELECT + SELECT COUNT(*) AS total_count, SUM(if(domain = 'facebook.com', 1, 0)) AS facebookHits, domain @@ -300,19 +300,19 @@ LIMIT 10 ) js1 ALL FULL OUTER JOIN ( -SELECT +SELECT sum(total_count) AS total, domain FROM ( - SELECT + SELECT COUNT(*) AS total_count, SUM(if(domain = 'facebook.com', 1, 0)) AS facebookHits, domain FROM clicks GROUP BY domain UNION ALL - SELECT + SELECT COUNT(*) AS total_count, toUInt64(0) AS facebookHits, domain diff --git a/tests/queries/0_stateless/00597_push_down_predicate.reference b/tests/queries/0_stateless/00597_push_down_predicate.reference index 1798c727088..83f783138a0 100644 --- a/tests/queries/0_stateless/00597_push_down_predicate.reference +++ b/tests/queries/0_stateless/00597_push_down_predicate.reference @@ -7,18 +7,18 @@ SELECT count() FROM ( - SELECT + SELECT [number] AS a, [number * 2] AS b FROM system.numbers LIMIT 1 ) AS t -ARRAY JOIN +ARRAY JOIN a, b WHERE NOT ignore(a + b) 1 -SELECT +SELECT a, b FROM @@ -27,17 +27,17 @@ FROM ) ANY LEFT JOIN ( - SELECT + SELECT 1 AS a, 1 AS b ) USING (a) WHERE b = 0 -SELECT +SELECT a, b FROM ( - SELECT + SELECT 1 AS a, 1 AS b ) @@ -46,7 +46,7 @@ ANY RIGHT JOIN SELECT 1 AS a ) USING (a) WHERE b = 0 -SELECT +SELECT a, b FROM @@ -55,17 +55,17 @@ FROM ) ANY FULL OUTER JOIN ( - SELECT + SELECT 1 AS a, 1 AS b ) USING (a) WHERE b = 0 -SELECT +SELECT a, b FROM ( - SELECT + SELECT 1 AS a, 1 AS b ) @@ -107,22 +107,22 @@ FROM ) WHERE id = 1 1 -SELECT +SELECT id, subquery FROM ( - SELECT + SELECT 1 AS id, CAST(1, \'UInt8\') AS subquery ) 1 1 -SELECT +SELECT a, b FROM ( - SELECT + SELECT toUInt64(sum(id) AS b) AS a, b FROM test_00597 @@ -130,20 +130,20 @@ FROM ) WHERE a = 3 3 3 -SELECT +SELECT date, id, name, value FROM ( - SELECT + SELECT date, name, value, min(id) AS id FROM test_00597 - GROUP BY + GROUP BY date, name, value @@ -151,12 +151,12 @@ FROM ) WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT +SELECT a, b FROM ( - SELECT + SELECT toUInt64(sum(id) AS b) AS a, b FROM test_00597 AS table_alias @@ -164,14 +164,14 @@ FROM ) AS outer_table_alias WHERE b = 3 3 3 -SELECT +SELECT date, id, name, value FROM ( - SELECT + SELECT date, id, name, @@ -181,21 +181,21 @@ FROM ) WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT +SELECT date, id, name, value FROM ( - SELECT + SELECT date, id, name, value FROM ( - SELECT + SELECT date, id, name, @@ -207,21 +207,21 @@ FROM ) WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT +SELECT date, id, name, value FROM ( - SELECT + SELECT date, id, name, value FROM ( - SELECT + SELECT date, id, name, @@ -233,14 +233,14 @@ FROM ) WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT +SELECT date, id, name, value FROM ( - SELECT + SELECT date, id, name, @@ -250,21 +250,21 @@ FROM ) WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT +SELECT date, id, name, value FROM ( - SELECT + SELECT date, id, name, value FROM ( - SELECT + SELECT date, id, name, @@ -276,14 +276,14 @@ FROM ) WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT +SELECT date, id, name, value FROM ( - SELECT + SELECT date, id, name, @@ -293,21 +293,21 @@ FROM ) AS b WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT +SELECT date, id, name, value FROM ( - SELECT + SELECT date, id, name, value FROM ( - SELECT + SELECT date, id, name, @@ -319,32 +319,32 @@ FROM ) AS b WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT +SELECT id, date, value FROM ( - SELECT + SELECT id, date, min(value) AS value FROM test_00597 WHERE id = 1 - GROUP BY + GROUP BY id, date ) WHERE id = 1 1 2000-01-01 1 -SELECT +SELECT date, id, name, value FROM ( - SELECT + SELECT date, id, name, @@ -352,7 +352,7 @@ FROM FROM test_00597 WHERE id = 1 UNION ALL - SELECT + SELECT date, id, name, @@ -363,7 +363,7 @@ FROM WHERE id = 1 2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 -SELECT +SELECT date, id, name, @@ -373,7 +373,7 @@ SELECT value FROM ( - SELECT + SELECT date, id, name, @@ -388,7 +388,7 @@ ANY LEFT JOIN ) USING (id) WHERE id = 1 2000-01-01 1 test string 1 1 2000-01-01 test string 1 1 -SELECT +SELECT id, date, name, @@ -399,7 +399,7 @@ FROM ) ANY LEFT JOIN ( - SELECT + SELECT date, id, name, @@ -416,14 +416,14 @@ FROM ANY LEFT JOIN test_00597 AS b USING (id) WHERE value = 1 1 -SELECT +SELECT date, id, name, value FROM ( - SELECT + SELECT date, id, name, @@ -433,7 +433,7 @@ FROM value FROM ( - SELECT + SELECT date, id, name, @@ -450,7 +450,7 @@ FROM ) WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT +SELECT date, id, name, @@ -460,7 +460,7 @@ SELECT b.value FROM ( - SELECT + SELECT date, id, name, @@ -469,7 +469,7 @@ FROM ) ANY LEFT JOIN ( - SELECT + SELECT date, id, name, @@ -478,14 +478,14 @@ ANY LEFT JOIN ) AS b USING (id) WHERE b.id = 1 2000-01-01 1 test string 1 1 2000-01-01 test string 1 1 -SELECT +SELECT id, date, name, value FROM ( - SELECT + SELECT toInt8(1) AS id, toDate(\'2000-01-01\') AS date FROM system.numbers @@ -493,7 +493,7 @@ FROM ) ANY LEFT JOIN ( - SELECT + SELECT date, id, name, @@ -502,7 +502,7 @@ ANY LEFT JOIN ) AS b USING (date, id) WHERE b.date = toDate(\'2000-01-01\') 1 2000-01-01 test string 1 1 -SELECT +SELECT date, id, name, @@ -513,7 +513,7 @@ SELECT `b.value` FROM ( - SELECT + SELECT date, id, name, @@ -524,7 +524,7 @@ FROM b.value FROM ( - SELECT + SELECT date, id, name, @@ -534,7 +534,7 @@ FROM ) AS a ANY LEFT JOIN ( - SELECT + SELECT date, id, name, @@ -545,7 +545,7 @@ FROM ) WHERE id = 1 2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 -SELECT +SELECT date, id, name, @@ -555,7 +555,7 @@ SELECT r.value FROM ( - SELECT + SELECT date, id, name, @@ -564,14 +564,14 @@ FROM ) SEMI LEFT JOIN ( - SELECT + SELECT date, id, name, value FROM ( - SELECT + SELECT date, id, name, diff --git a/tests/queries/0_stateless/00618_nullable_in.sql b/tests/queries/0_stateless/00618_nullable_in.sql index 8e8c26d225d..72e166dc0f5 100644 --- a/tests/queries/0_stateless/00618_nullable_in.sql +++ b/tests/queries/0_stateless/00618_nullable_in.sql @@ -1,13 +1,13 @@ SELECT sum(toNullable('a') IN 'a'); SELECT countIf(number, toNullable('a') IN ('a', 'b')) FROM numbers(100); -SELECT +SELECT uniqExact(x) AS u, uniqExactIf(x, name = 'a') AS ue, uniqExactIf(x, name IN ('a', 'b')) AS ui FROM ( - SELECT + SELECT toNullable('a') AS name, arrayJoin(range(10)) AS x ) diff --git a/tests/queries/0_stateless/00712_prewhere_with_alias_bug_2.sql b/tests/queries/0_stateless/00712_prewhere_with_alias_bug_2.sql index 2c5c494fc1f..97d5e33633a 100644 --- a/tests/queries/0_stateless/00712_prewhere_with_alias_bug_2.sql +++ b/tests/queries/0_stateless/00712_prewhere_with_alias_bug_2.sql @@ -4,7 +4,7 @@ CREATE TABLE table (a UInt32, date Date, b UInt64, c UInt64, str String, d Int SELECT alias2 AS alias3 FROM table -ARRAY JOIN +ARRAY JOIN arr_alias AS alias2, arrayEnumerateUniq(arr_alias) AS _uniq_Event WHERE (date = toDate('2010-10-10')) AND (a IN (2, 3)) AND (str NOT IN ('z', 'x')) AND (d != -1) diff --git a/tests/queries/0_stateless/00740_optimize_predicate_expression.sql b/tests/queries/0_stateless/00740_optimize_predicate_expression.sql index b016ab49ddd..65b06635808 100644 --- a/tests/queries/0_stateless/00740_optimize_predicate_expression.sql +++ b/tests/queries/0_stateless/00740_optimize_predicate_expression.sql @@ -5,7 +5,7 @@ SELECT * FROM (SELECT perf_1.z AS z_1 FROM perf AS perf_1); SELECT sum(mul)/sqrt(sum(sqr_dif_1) * sum(sqr_dif_2)) AS z_r FROM( -SELECT +SELECT (SELECT avg(z_1) AS z_1_avg, avg(z_2) AS z_2_avg FROM ( diff --git a/tests/queries/0_stateless/00743_limit_by_not_found_column.sql b/tests/queries/0_stateless/00743_limit_by_not_found_column.sql index 46c6bcb99b2..d20b3b0209e 100644 --- a/tests/queries/0_stateless/00743_limit_by_not_found_column.sql +++ b/tests/queries/0_stateless/00743_limit_by_not_found_column.sql @@ -24,7 +24,7 @@ CREATE TEMPORARY TABLE Accounts (AccountID UInt64, Currency String); SELECT AccountID FROM ( - SELECT + SELECT AccountID, Currency FROM Accounts diff --git a/tests/queries/0_stateless/00751_default_databasename_for_view.reference b/tests/queries/0_stateless/00751_default_databasename_for_view.reference index 4814cc77b37..76d5cee02e2 100644 --- a/tests/queries/0_stateless/00751_default_databasename_for_view.reference +++ b/tests/queries/0_stateless/00751_default_databasename_for_view.reference @@ -7,7 +7,7 @@ CREATE MATERIALIZED VIEW test_00751.t_mv_00751 ENGINE = MergeTree ORDER BY date SETTINGS index_granularity = 8192 AS -SELECT +SELECT date, platform, app diff --git a/tests/queries/0_stateless/00759_kodieg.sql b/tests/queries/0_stateless/00759_kodieg.sql index 2037f210dea..9cbe2a0cd7f 100644 --- a/tests/queries/0_stateless/00759_kodieg.sql +++ b/tests/queries/0_stateless/00759_kodieg.sql @@ -1,4 +1,4 @@ -SELECT +SELECT [1, 2, 3, 1, 3] AS a, indexOf(arrayReverse(arraySlice(a, 1, -1)), 3) AS offset_from_right, arraySlice(a, multiIf(offset_from_right = 0, 1, (length(a) - offset_from_right) + 1)); diff --git a/tests/queries/0_stateless/00826_cross_to_inner_join.reference b/tests/queries/0_stateless/00826_cross_to_inner_join.reference index 6e5cbdcab4e..e7c8d6b1ea9 100644 --- a/tests/queries/0_stateless/00826_cross_to_inner_join.reference +++ b/tests/queries/0_stateless/00826_cross_to_inner_join.reference @@ -35,7 +35,7 @@ comma nullable 1 1 1 1 2 2 1 2 cross -SELECT +SELECT a, b, t2_00826.a, @@ -44,7 +44,7 @@ FROM t1_00826 ALL INNER JOIN t2_00826 ON a = t2_00826.a WHERE a = t2_00826.a cross nullable -SELECT +SELECT a, b, t2_00826.a, @@ -53,7 +53,7 @@ FROM t1_00826 ALL INNER JOIN t2_00826 ON a = t2_00826.a WHERE a = t2_00826.a cross nullable vs not nullable -SELECT +SELECT a, b, t2_00826.a, @@ -62,7 +62,7 @@ FROM t1_00826 ALL INNER JOIN t2_00826 ON a = t2_00826.b WHERE a = t2_00826.b cross self -SELECT +SELECT a, b, y.a, @@ -71,7 +71,7 @@ FROM t1_00826 AS x ALL INNER JOIN t1_00826 AS y ON (a = y.a) AND (b = y.b) WHERE (a = y.a) AND (b = y.b) cross one table expr -SELECT +SELECT a, b, t2_00826.a, @@ -80,7 +80,7 @@ FROM t1_00826 CROSS JOIN t2_00826 WHERE a = b cross multiple ands -SELECT +SELECT a, b, t2_00826.a, @@ -89,7 +89,7 @@ FROM t1_00826 ALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b) WHERE (a = t2_00826.a) AND (b = t2_00826.b) cross and inside and -SELECT +SELECT a, b, t2_00826.a, @@ -98,7 +98,7 @@ FROM t1_00826 ALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (a = t2_00826.a) AND (a = t2_00826.a) AND (b = t2_00826.b) WHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b))) cross split conjunction -SELECT +SELECT a, b, t2_00826.a, diff --git a/tests/queries/0_stateless/00849_multiple_comma_join.reference b/tests/queries/0_stateless/00849_multiple_comma_join.reference index 5a5a90cbdf2..f4db2238dd1 100644 --- a/tests/queries/0_stateless/00849_multiple_comma_join.reference +++ b/tests/queries/0_stateless/00849_multiple_comma_join.reference @@ -12,7 +12,7 @@ WHERE b = t2_00849.b SELECT `--t1_00849.a` AS `t1_00849.a` FROM ( - SELECT + SELECT a AS `--t1_00849.a`, b, t2_00849.a AS `--t2_00849.a`, @@ -25,7 +25,7 @@ WHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a) SELECT `--t1_00849.a` AS `t1_00849.a` FROM ( - SELECT + SELECT a AS `--t1_00849.a`, b AS `--t1_00849.b`, t2_00849.a, @@ -38,7 +38,7 @@ WHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b) SELECT `--t1_00849.a` AS `t1_00849.a` FROM ( - SELECT + SELECT `--t1_00849.a`, b, `--t2_00849.a`, @@ -47,7 +47,7 @@ FROM t3_00849.b FROM ( - SELECT + SELECT a AS `--t1_00849.a`, b, t2_00849.a AS `--t2_00849.a`, @@ -62,7 +62,7 @@ WHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AN SELECT `--t1_00849.a` AS `t1_00849.a` FROM ( - SELECT + SELECT `--t1_00849.a`, `--t1_00849.b`, `t2_00849.a`, @@ -71,7 +71,7 @@ FROM b AS `--t3_00849.b` FROM ( - SELECT + SELECT a AS `--t1_00849.a`, b AS `--t1_00849.b`, t2_00849.a, @@ -86,7 +86,7 @@ WHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AN SELECT `--t1_00849.a` AS `t1_00849.a` FROM ( - SELECT + SELECT `--t1_00849.a`, b, `--t2_00849.a`, @@ -95,7 +95,7 @@ FROM t3_00849.b FROM ( - SELECT + SELECT a AS `--t1_00849.a`, b, t2_00849.a AS `--t2_00849.a`, @@ -110,7 +110,7 @@ WHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AN SELECT `--t1_00849.a` AS `t1_00849.a` FROM ( - SELECT + SELECT `--t1_00849.a`, b, `--t2_00849.a`, @@ -119,7 +119,7 @@ FROM t3_00849.b FROM ( - SELECT + SELECT a AS `--t1_00849.a`, b, t2_00849.a AS `--t2_00849.a`, @@ -134,7 +134,7 @@ WHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AN SELECT `--t1_00849.a` AS `t1_00849.a` FROM ( - SELECT + SELECT `--t1_00849.a`, b, `--t2_00849.a`, @@ -143,7 +143,7 @@ FROM t3_00849.b FROM ( - SELECT + SELECT a AS `--t1_00849.a`, b, t2_00849.a AS `--t2_00849.a`, @@ -158,7 +158,7 @@ WHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`) SELECT `--t1_00849.a` AS `t1_00849.a` FROM ( - SELECT + SELECT `--t1_00849.a`, b, `--t2_00849.a`, @@ -167,7 +167,7 @@ FROM t3_00849.b FROM ( - SELECT + SELECT a AS `--t1_00849.a`, b, t2_00849.a AS `--t2_00849.a`, @@ -182,7 +182,7 @@ WHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AN SELECT `--t1_00849.a` AS `t1_00849.a` FROM ( - SELECT + SELECT `--t1_00849.a`, b, `t2_00849.a`, @@ -191,7 +191,7 @@ FROM t3_00849.b FROM ( - SELECT + SELECT a AS `--t1_00849.a`, b, t2_00849.a, @@ -205,7 +205,7 @@ CROSS JOIN t4_00849 SELECT `--t1_00849.a` AS `t1_00849.a` FROM ( - SELECT + SELECT `--t1_00849.a`, b, `t2_00849.a`, @@ -214,7 +214,7 @@ FROM t3_00849.b FROM ( - SELECT + SELECT a AS `--t1_00849.a`, b, t2_00849.a, @@ -228,7 +228,7 @@ CROSS JOIN t4_00849 SELECT `--t1_00849.a` AS `t1_00849.a` FROM ( - SELECT + SELECT a AS `--t1_00849.a`, b, t2_00849.a, @@ -240,7 +240,7 @@ CROSS JOIN t3_00849 SELECT `--t1_00849.a` AS `t1_00849.a` FROM ( - SELECT + SELECT a AS `--t1_00849.a`, b, t2_00849.a AS `--t2_00849.a`, diff --git a/tests/queries/0_stateless/00849_multiple_comma_join_2.reference b/tests/queries/0_stateless/00849_multiple_comma_join_2.reference index e08d6ff1192..fc39ef13935 100644 --- a/tests/queries/0_stateless/00849_multiple_comma_join_2.reference +++ b/tests/queries/0_stateless/00849_multiple_comma_join_2.reference @@ -12,7 +12,7 @@ WHERE b = t2.b SELECT `--t1.a` AS `t1.a` FROM ( - SELECT + SELECT a AS `--t1.a`, t2.a AS `--t2.a` FROM t1 @@ -23,7 +23,7 @@ WHERE (`--t1.a` = `--t2.a`) AND (`--t1.a` = a) SELECT `--t1.a` AS `t1.a` FROM ( - SELECT + SELECT b AS `--t1.b`, a AS `--t1.a`, t2.b AS `--t2.b` @@ -35,13 +35,13 @@ WHERE (`--t1.b` = `--t2.b`) AND (`--t1.b` = b) SELECT `--t1.a` AS `t1.a` FROM ( - SELECT + SELECT `--t1.a`, `--t2.a`, a AS `--t3.a` FROM ( - SELECT + SELECT a AS `--t1.a`, t2.a AS `--t2.a` FROM t1 @@ -54,14 +54,14 @@ WHERE (`--t1.a` = `--t2.a`) AND (`--t1.a` = `--t3.a`) AND (`--t1.a` = a) SELECT `--t1.a` AS `t1.a` FROM ( - SELECT + SELECT `--t1.b`, `--t1.a`, `--t2.b`, b AS `--t3.b` FROM ( - SELECT + SELECT b AS `--t1.b`, a AS `--t1.a`, t2.b AS `--t2.b` @@ -75,13 +75,13 @@ WHERE (`--t1.b` = `--t2.b`) AND (`--t1.b` = `--t3.b`) AND (`--t1.b` = b) SELECT `--t1.a` AS `t1.a` FROM ( - SELECT + SELECT `--t1.a`, `--t2.a`, a AS `--t3.a` FROM ( - SELECT + SELECT a AS `--t1.a`, t2.a AS `--t2.a` FROM t1 @@ -94,13 +94,13 @@ WHERE (`--t2.a` = `--t1.a`) AND (`--t2.a` = `--t3.a`) AND (`--t2.a` = a) SELECT `--t1.a` AS `t1.a` FROM ( - SELECT + SELECT `--t1.a`, `--t2.a`, a AS `--t3.a` FROM ( - SELECT + SELECT a AS `--t1.a`, t2.a AS `--t2.a` FROM t1 @@ -113,13 +113,13 @@ WHERE (`--t3.a` = `--t1.a`) AND (`--t3.a` = `--t2.a`) AND (`--t3.a` = a) SELECT `--t1.a` AS `t1.a` FROM ( - SELECT + SELECT `--t1.a`, `--t2.a`, a AS `--t3.a` FROM ( - SELECT + SELECT a AS `--t1.a`, t2.a AS `--t2.a` FROM t1 @@ -132,13 +132,13 @@ WHERE (a = `--t1.a`) AND (a = `--t2.a`) AND (a = `--t3.a`) SELECT `--t1.a` AS `t1.a` FROM ( - SELECT + SELECT `--t1.a`, `--t2.a`, a AS `--t3.a` FROM ( - SELECT + SELECT a AS `--t1.a`, t2.a AS `--t2.a` FROM t1 @@ -185,7 +185,7 @@ CROSS JOIN t3 SELECT `--t1.a` AS `t1.a` FROM ( - SELECT + SELECT a AS `--t1.a`, t2.a AS `--t2.a` FROM t1 diff --git a/tests/queries/0_stateless/00908_analyze_query.reference b/tests/queries/0_stateless/00908_analyze_query.reference index 0305f528b25..ab9237531f7 100644 --- a/tests/queries/0_stateless/00908_analyze_query.reference +++ b/tests/queries/0_stateless/00908_analyze_query.reference @@ -1,4 +1,4 @@ -SELECT +SELECT a, b FROM a diff --git a/tests/queries/0_stateless/00941_to_custom_week.sql b/tests/queries/0_stateless/00941_to_custom_week.sql index a6ff40a6d3f..c7d52e7438b 100644 --- a/tests/queries/0_stateless/00941_to_custom_week.sql +++ b/tests/queries/0_stateless/00941_to_custom_week.sql @@ -25,7 +25,7 @@ SELECT toWeek(toDate('2001-01-01'),0) AS w0, toWeek(toDate('2001-01-01'),1) AS w SELECT toYearWeek(toDate('2000-12-31'),0), toYearWeek(toDate('2000-12-31'),1), toYearWeek(toDate('2000-12-31'),2), toYearWeek(toDate('2000-12-31'),3), toYearWeek(toDate('2000-12-31'),4), toYearWeek(toDate('2000-12-31'),5), toYearWeek(toDate('2000-12-31'),6), toYearWeek(toDate('2000-12-31'),7); -- week mode 8,9 -SELECT +SELECT toDate('2016-12-21') + number AS d, toWeek(d, 8) AS week8, toWeek(d, 9) AS week9, diff --git a/tests/queries/0_stateless/00957_format_with_clashed_aliases.reference b/tests/queries/0_stateless/00957_format_with_clashed_aliases.reference index d1c8033b363..d6e53c8b48b 100644 --- a/tests/queries/0_stateless/00957_format_with_clashed_aliases.reference +++ b/tests/queries/0_stateless/00957_format_with_clashed_aliases.reference @@ -1,4 +1,4 @@ -SELECT +SELECT 1 AS x, x.y FROM diff --git a/tests/queries/0_stateless/00958_format_of_tuple_array_element.reference b/tests/queries/0_stateless/00958_format_of_tuple_array_element.reference index eaea02ba40b..a2953fa0264 100644 --- a/tests/queries/0_stateless/00958_format_of_tuple_array_element.reference +++ b/tests/queries/0_stateless/00958_format_of_tuple_array_element.reference @@ -1,4 +1,4 @@ -SELECT +SELECT (x.1)[1], (((x[1]).1)[1]).1, (NOT x)[1], diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql index b9eed1e8183..3b99ebf8b22 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql @@ -4,7 +4,7 @@ CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789); -SELECT +SELECT name, table, hash_of_all_files, diff --git a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index fd9d96bdf5f..4227af86be7 100644 --- a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -1,11 +1,11 @@ -SELECT +SELECT k, v, d, i FROM ( - SELECT + SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, @@ -21,26 +21,26 @@ a 2 1 0 a 3 1 0 b 13 2 0 b 15 2 0 -SELECT +SELECT co, co2, co3, num FROM ( - SELECT + SELECT co, co2, co3, count() AS num FROM ( - SELECT + SELECT 1 AS co, 2 AS co2, 3 AS co3 ) - GROUP BY + GROUP BY co, co2, co3 @@ -84,7 +84,7 @@ FROM ) WHERE ccc > 1 2 -SELECT +SELECT ts, id, id_b, @@ -93,7 +93,7 @@ SELECT id_c FROM ( - SELECT + SELECT ts, id, id_b @@ -102,7 +102,7 @@ FROM ) AS a ALL LEFT JOIN B AS b ON b.id = id_b WHERE ts <= toDateTime(\'1970-01-01 03:00:00\') -SELECT +SELECT ts AS `--a.ts`, id AS `--a.id`, id_b AS `--a.id_b`, @@ -111,7 +111,7 @@ SELECT id_c AS `--b.id_c` FROM ( - SELECT + SELECT ts, id, id_b @@ -137,19 +137,19 @@ FROM ) WHERE arrayMap(x -> (x + 1), [dummy]) = [1] 0 -SELECT +SELECT id, value, value_1 FROM ( - SELECT + SELECT 1 AS id, 2 AS value ) ALL INNER JOIN ( - SELECT + SELECT 1 AS id, 3 AS value_1 ) USING (id) diff --git a/tests/queries/0_stateless/01076_predicate_optimizer_with_view.reference b/tests/queries/0_stateless/01076_predicate_optimizer_with_view.reference index d6426f679c5..dfab41b5e4c 100644 --- a/tests/queries/0_stateless/01076_predicate_optimizer_with_view.reference +++ b/tests/queries/0_stateless/01076_predicate_optimizer_with_view.reference @@ -1,4 +1,4 @@ -SELECT +SELECT date, id, name, @@ -10,7 +10,7 @@ FROM HAVING id = 1 ) AS test_view WHERE id = 1 -SELECT +SELECT date, id, name, diff --git a/tests/queries/0_stateless/01083_cross_to_inner_with_like.reference b/tests/queries/0_stateless/01083_cross_to_inner_with_like.reference index 5491e82c7d3..42bbeb05ecb 100644 --- a/tests/queries/0_stateless/01083_cross_to_inner_with_like.reference +++ b/tests/queries/0_stateless/01083_cross_to_inner_with_like.reference @@ -1,18 +1,18 @@ -SELECT +SELECT k, r.k, name FROM n ALL INNER JOIN r ON k = r.k WHERE (k = r.k) AND (name = \'A\') -SELECT +SELECT k, r.k, name FROM n ALL INNER JOIN r ON k = r.k WHERE (k = r.k) AND (name LIKE \'A%\') -SELECT +SELECT k, r.k, name diff --git a/tests/queries/0_stateless/01117_chain_finalize_bug.sql b/tests/queries/0_stateless/01117_chain_finalize_bug.sql index f79f82b8d4d..273b742d0bd 100644 --- a/tests/queries/0_stateless/01117_chain_finalize_bug.sql +++ b/tests/queries/0_stateless/01117_chain_finalize_bug.sql @@ -14,7 +14,7 @@ SET group_by_two_level_threshold = 2; SELECT count() FROM ( - SELECT + SELECT arrayJoin(arrayMap(i -> (i + 1), range(2))) AS index, number FROM numbers_mt(100000) diff --git a/tests/queries/0_stateless/01250_fixed_string_comparison.sql b/tests/queries/0_stateless/01250_fixed_string_comparison.sql index 8481b3572bb..d574fd082f0 100644 --- a/tests/queries/0_stateless/01250_fixed_string_comparison.sql +++ b/tests/queries/0_stateless/01250_fixed_string_comparison.sql @@ -1,5 +1,5 @@ WITH 'abb' AS b, 'abc' AS c, 'abd' AS d, toFixedString(b, 5) AS bf, toFixedString(c, 5) AS cf, toFixedString(d, 5) AS df -SELECT +SELECT b = b, b > b, b < b, b = c, b > c, b < c, b = d, b > d, b < d, diff --git a/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func.reference b/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func.reference index eeba2646046..669221005f4 100644 --- a/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func.reference +++ b/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func.reference @@ -1,4 +1,4 @@ -SELECT +SELECT sum(n + 1), sum(1 + n), sum(n - 1), @@ -8,7 +8,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT sum(n) * 2, 2 * sum(n), sum(n) / 2, @@ -18,7 +18,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT min(n) + 1, 1 + min(n), min(n) - 1, @@ -28,7 +28,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT min(n) * 2, 2 * min(n), min(n) / 2, @@ -38,7 +38,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT max(n) + 1, 1 + max(n), max(n) - 1, @@ -48,7 +48,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT max(n) * 2, 2 * max(n), max(n) / 2, @@ -58,7 +58,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT sum(n + -1), sum(-1 + n), sum(n - -1), @@ -68,7 +68,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT sum(n) * -2, -2 * sum(n), sum(n) / -2, @@ -78,7 +78,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT min(n) + -1, -1 + min(n), min(n) - -1, @@ -88,7 +88,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT max(n) * -2, -2 * max(n), max(n) / -2, @@ -98,7 +98,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT max(n) + -1, -1 + max(n), max(n) - -1, @@ -108,7 +108,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT min(n) * -2, -2 * min(n), min(n) / -2, @@ -118,7 +118,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT sum(abs(2) + 1), sum(abs(2) + n), sum(n - abs(2)), @@ -128,7 +128,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT sum(abs(2)) * 2, sum(abs(2) * n), sum(n / abs(2)), @@ -138,7 +138,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT min(abs(2)) + 1, min(abs(2) + n), min(n - abs(2)), @@ -148,7 +148,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT min(abs(2)) * 2, min(abs(2) * n), min(n / abs(2)), @@ -158,7 +158,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT max(abs(2)) + 1, max(abs(2) + n), max(n - abs(2)), @@ -168,7 +168,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT max(abs(2)) * 2, max(abs(2) * n), max(n / abs(2)), @@ -178,7 +178,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT sum(abs(n) + 1), sum(abs(n) + n), sum(n - abs(n)), @@ -188,7 +188,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT sum(abs(n)) * 2, sum(abs(n) * n), sum(n / abs(n)), @@ -198,7 +198,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT min(abs(n)) + 1, min(abs(n) + n), min(n - abs(n)), @@ -208,7 +208,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT min(abs(n)) * 2, min(abs(n) * n), min(n / abs(n)), @@ -218,7 +218,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT max(abs(n)) + 1, max(abs(n) + n), max(n - abs(n)), @@ -228,7 +228,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT max(abs(n)) * 2, max(abs(n) * n), max(n / abs(n)), @@ -238,7 +238,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT sum((n * n) + 1), sum(1 + (n * n)), sum((n * n) - 1), @@ -248,7 +248,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT sum(n * n) * 2, sum((2 * n) * n), sum(n * n) / 2, @@ -258,7 +258,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT min(n * n) + 1, 1 + min(n * n), min(n * n) - 1, @@ -268,7 +268,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT min(n * n) * 2, min((2 * n) * n), min(n * n) / 2, @@ -278,7 +278,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT max(n * n) + 1, 1 + max(n * n), max(n * n) - 1, @@ -288,7 +288,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT max(n * n) * 2, max((2 * n) * n), max(n * n) / 2, @@ -298,7 +298,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT sum((1 + n) + 1), sum((1 + 1) + n), sum((1 + n) - 1), @@ -308,7 +308,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT sum(1 + (n * 2)), sum(1 + (2 * n)), sum(1 + (n / 2)), @@ -318,7 +318,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT (1 + min(n)) + 1, min((1 + 1) + n), (1 + min(n)) - 1, @@ -328,7 +328,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT 1 + min(n * 2), 1 + min(2 * n), 1 + min(n / 2), @@ -338,7 +338,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT (1 + max(n)) + 1, max((1 + 1) + n), (1 + max(n)) - 1, @@ -348,7 +348,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT 1 + max(n * 2), 1 + max(2 * n), 1 + max(n / 2), @@ -358,7 +358,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT sum((n + -1) + -1), sum((-1 + n) + -1), sum((n - -1) + -1), @@ -368,7 +368,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT (sum(n) * -2) * -1, (-2 * sum(n)) * -1, (sum(n) / -2) / -1, @@ -378,7 +378,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT (min(n) + -1) + -1, (-1 + min(n)) + -1, (min(n) - -1) + -1, @@ -388,7 +388,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT (min(n) * -2) * -1, (-2 * min(n)) * -1, (min(n) / -2) / -1, @@ -398,7 +398,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT (max(n) + -1) + -1, (-1 + max(n)) + -1, (max(n) - -1) + -1, @@ -408,7 +408,7 @@ FROM SELECT number AS n FROM numbers(10) ) -SELECT +SELECT (max(n) * -2) * -1, (-2 * max(n)) * -1, (max(n) / -2) / -1, diff --git a/tests/queries/0_stateless/01272_totals_and_filter_bug.sql b/tests/queries/0_stateless/01272_totals_and_filter_bug.sql index a6082ae9f11..de751eb73bd 100644 --- a/tests/queries/0_stateless/01272_totals_and_filter_bug.sql +++ b/tests/queries/0_stateless/01272_totals_and_filter_bug.sql @@ -8,13 +8,13 @@ CREATE TABLE bar (server_date Date, dimension_1 String, metric_2 UInt32) ENGINE INSERT INTO foo VALUES ('2020-01-01', 'test1', 10), ('2020-01-01', 'test2', 20); INSERT INTO bar VALUES ('2020-01-01', 'test2', 30), ('2020-01-01', 'test3', 40); -SELECT +SELECT dimension_1, sum_metric_1, sum_metric_2 FROM ( - SELECT + SELECT dimension_1, sum(metric_1) AS sum_metric_1 FROM foo @@ -23,7 +23,7 @@ FROM ) AS subquery_1 ALL FULL OUTER JOIN ( - SELECT + SELECT dimension_1, sum(metric_2) AS sum_metric_2 FROM bar diff --git a/tests/queries/0_stateless/01278_format_multiple_queries.reference b/tests/queries/0_stateless/01278_format_multiple_queries.reference index b12e3b30f0c..001b10b0990 100644 --- a/tests/queries/0_stateless/01278_format_multiple_queries.reference +++ b/tests/queries/0_stateless/01278_format_multiple_queries.reference @@ -1,4 +1,4 @@ -SELECT +SELECT a, b AS x FROM table AS t @@ -6,7 +6,7 @@ INNER JOIN table2 AS t2 ON t.id = t2.t_id WHERE 1 = 1 ; -SELECT +SELECT a, b AS x, if(x = 0, a, b) diff --git a/tests/queries/0_stateless/01300_group_by_other_keys.reference b/tests/queries/0_stateless/01300_group_by_other_keys.reference index bd2372dca93..1db34d884a3 100644 --- a/tests/queries/0_stateless/01300_group_by_other_keys.reference +++ b/tests/queries/0_stateless/01300_group_by_other_keys.reference @@ -24,7 +24,7 @@ 3465735.9028 SELECT max(log(2) * number) AS k FROM numbers(10000000) -GROUP BY +GROUP BY number % 2, number % 3, ((number % 2) + (number % 3)) % 2 @@ -35,19 +35,19 @@ GROUP BY number % 5 ORDER BY k ASC SELECT avg(log(2) * number) AS k FROM numbers(10000000) -GROUP BY +GROUP BY (number % 2) * (number % 3), number % 3 ORDER BY k ASC SELECT avg(log(2) * number) AS k FROM numbers(10000000) -GROUP BY +GROUP BY number % 3, number % 2 ORDER BY k ASC SELECT avg(log(2) * number) AS k FROM numbers(10000000) -GROUP BY +GROUP BY (number % 2) % 3, number % 2 ORDER BY k ASC @@ -77,33 +77,33 @@ ORDER BY k ASC 3465735.9028 SELECT max(log(2) * number) AS k FROM numbers(10000000) -GROUP BY +GROUP BY number % 2, number % 3, ((number % 2) + (number % 3)) % 2 ORDER BY k ASC SELECT avg(log(2) * number) AS k FROM numbers(10000000) -GROUP BY +GROUP BY number % 5, (number % 5) * (number % 5) ORDER BY k ASC SELECT avg(log(2) * number) AS k FROM numbers(10000000) -GROUP BY +GROUP BY (number % 2) * (number % 3), number % 3 ORDER BY k ASC SELECT avg(log(2) * number) AS k FROM numbers(10000000) -GROUP BY +GROUP BY (number % 2) * (number % 3), number % 3, number % 2 ORDER BY k ASC SELECT avg(log(2) * number) AS k FROM numbers(10000000) -GROUP BY +GROUP BY (number % 2) % 3, number % 2 ORDER BY k ASC diff --git a/tests/queries/0_stateless/01300_group_by_other_keys_having.reference b/tests/queries/0_stateless/01300_group_by_other_keys_having.reference index 0bec0ebdf9b..a470c19a244 100644 --- a/tests/queries/0_stateless/01300_group_by_other_keys_having.reference +++ b/tests/queries/0_stateless/01300_group_by_other_keys_having.reference @@ -9,7 +9,7 @@ 4 SELECT avg(log(2) * number) AS k FROM numbers(10000000) -GROUP BY +GROUP BY number % 3, number % 2 HAVING avg(log(2) * number) > 3465735.3 @@ -35,7 +35,7 @@ ORDER BY k ASC 4 SELECT avg(log(2) * number) AS k FROM numbers(10000000) -GROUP BY +GROUP BY (number % 2) * (number % 3), number % 3, number % 2 @@ -44,14 +44,14 @@ ORDER BY k ASC SELECT avg(log(2) * number) AS k FROM numbers(10000000) WHERE ((number % 5) * (number % 5)) < 5 -GROUP BY +GROUP BY number % 5, (number % 5) * (number % 5) ORDER BY k ASC SELECT (number % 5) * (number % 5) AS k FROM numbers(10000000) WHERE ((number % 5) * (number % 5)) < 5 -GROUP BY +GROUP BY number % 5, (number % 5) * (number % 5) ORDER BY k ASC diff --git a/tests/queries/0_stateless/01321_aggregate_functions_of_group_by_keys.reference b/tests/queries/0_stateless/01321_aggregate_functions_of_group_by_keys.reference index 875a6753f84..92d6e5c37e6 100644 --- a/tests/queries/0_stateless/01321_aggregate_functions_of_group_by_keys.reference +++ b/tests/queries/0_stateless/01321_aggregate_functions_of_group_by_keys.reference @@ -47,29 +47,29 @@ 24 0 0 -SELECT +SELECT number % 2 AS a, number % 3 AS b FROM numbers(10000000) -GROUP BY +GROUP BY number % 2, number % 3 -ORDER BY +ORDER BY min(number % 2) AS a ASC, max(number % 3) AS b ASC -SELECT +SELECT number % 2 AS a, number % 3 AS b FROM numbers(10000000) -GROUP BY +GROUP BY number % 2, number % 3 -ORDER BY +ORDER BY any(number % 2) AS a ASC, anyLast(number % 3) AS b ASC SELECT (number % 5) * (number % 7) AS a FROM numbers(10000000) -GROUP BY +GROUP BY number % 7, number % 5 ORDER BY max((number % 5) * (number % 7)) AS a ASC @@ -128,29 +128,29 @@ FROM 20 24 0 -SELECT +SELECT min(number % 2) AS a, max(number % 3) AS b FROM numbers(10000000) -GROUP BY +GROUP BY number % 2, number % 3 -ORDER BY +ORDER BY a ASC, b ASC -SELECT +SELECT any(number % 2) AS a, anyLast(number % 3) AS b FROM numbers(10000000) -GROUP BY +GROUP BY number % 2, number % 3 -ORDER BY +ORDER BY a ASC, b ASC SELECT max((number % 5) * (number % 7)) AS a FROM numbers(10000000) -GROUP BY +GROUP BY number % 7, number % 5 ORDER BY a ASC diff --git a/tests/queries/0_stateless/01321_monotonous_functions_in_order_by.reference b/tests/queries/0_stateless/01321_monotonous_functions_in_order_by.reference index e8e7d754ed9..ffa91586f35 100644 --- a/tests/queries/0_stateless/01321_monotonous_functions_in_order_by.reference +++ b/tests/queries/0_stateless/01321_monotonous_functions_in_order_by.reference @@ -54,12 +54,12 @@ FROM numbers(3) ORDER BY exp(number) ASC SELECT roundToExp2(number) AS x FROM numbers(3) -ORDER BY +ORDER BY number ASC, number ASC SELECT number AS x FROM numbers(3) -ORDER BY +ORDER BY number ASC, number ASC SELECT number @@ -79,7 +79,7 @@ FROM numbers(3) ORDER BY exp(number) DESC SELECT roundToExp2(number) AS x FROM numbers(3) -ORDER BY +ORDER BY number DESC, number DESC 0 @@ -138,12 +138,12 @@ FROM numbers(3) ORDER BY exp(number) ASC SELECT roundToExp2(number) AS x FROM numbers(3) -ORDER BY +ORDER BY x ASC, toFloat32(x) ASC SELECT number AS x FROM numbers(3) -ORDER BY +ORDER BY toFloat32(x) AS k ASC, toFloat64(k) ASC SELECT number @@ -163,6 +163,6 @@ FROM numbers(3) ORDER BY exp(number) DESC SELECT roundToExp2(number) AS x FROM numbers(3) -ORDER BY +ORDER BY x DESC, toFloat32(x) DESC diff --git a/tests/queries/0_stateless/01322_any_input_optimize.reference b/tests/queries/0_stateless/01322_any_input_optimize.reference index c02c9fbeae4..f88f2f5937c 100644 --- a/tests/queries/0_stateless/01322_any_input_optimize.reference +++ b/tests/queries/0_stateless/01322_any_input_optimize.reference @@ -8,7 +8,7 @@ WITH any(number) * 3 AS x SELECT x FROM numbers(1, 2) 3 -SELECT +SELECT anyLast(number) * 3 AS x, x FROM numbers(1, 2) @@ -23,7 +23,7 @@ WITH any(number * 3) AS x SELECT x FROM numbers(1, 2) 3 -SELECT +SELECT anyLast(number * 3) AS x, x FROM numbers(1, 2) diff --git a/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference b/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference index 8751f269c4a..fd52438e9b0 100644 --- a/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference +++ b/tests/queries/0_stateless/01323_redundant_functions_in_order_by.reference @@ -34,11 +34,11 @@ FROM ( SELECT number AS x FROM numbers(3) - ORDER BY + ORDER BY exp(x) ASC, x ASC ) -SELECT +SELECT key, a, b, @@ -49,17 +49,17 @@ FROM FROM numbers(4) ) AS s ALL FULL OUTER JOIN test AS t USING (key) -ORDER BY +ORDER BY key ASC, t.key ASC -SELECT +SELECT key, a FROM test -ORDER BY +ORDER BY key ASC, a ASC -SELECT +SELECT key, a FROM test @@ -86,7 +86,7 @@ FROM ( SELECT number AS x FROM numbers(3) - ORDER BY + ORDER BY x ASC, exp(x) ASC ) @@ -95,7 +95,7 @@ FROM ( SELECT number AS x FROM numbers(3) - ORDER BY + ORDER BY x ASC, exp(exp(x)) ASC ) @@ -104,11 +104,11 @@ FROM ( SELECT number AS x FROM numbers(3) - ORDER BY + ORDER BY exp(x) ASC, x ASC ) -SELECT +SELECT key, a, b, @@ -119,21 +119,21 @@ FROM FROM numbers(4) ) AS s ALL FULL OUTER JOIN test AS t USING (key) -ORDER BY +ORDER BY key ASC, t.key ASC -SELECT +SELECT key, a FROM test -ORDER BY +ORDER BY key ASC, a ASC, exp(key + a) ASC -SELECT +SELECT key, a FROM test -ORDER BY +ORDER BY key ASC, exp(key + a) ASC diff --git a/tests/queries/0_stateless/01355_defaultValueOfArgumentType_bug.sql b/tests/queries/0_stateless/01355_defaultValueOfArgumentType_bug.sql index e3168eb09a0..2313cb686a4 100644 --- a/tests/queries/0_stateless/01355_defaultValueOfArgumentType_bug.sql +++ b/tests/queries/0_stateless/01355_defaultValueOfArgumentType_bug.sql @@ -1,4 +1,4 @@ -SELECT +SELECT materialize(toLowCardinality('')) AS lc, toTypeName(lc) WHERE lc = defaultValueOfArgumentType(lc) diff --git a/tests/queries/0_stateless/01372_wrong_order_by_removal.reference b/tests/queries/0_stateless/01372_wrong_order_by_removal.reference index 8ed303c04aa..f1f1bcef6e5 100644 --- a/tests/queries/0_stateless/01372_wrong_order_by_removal.reference +++ b/tests/queries/0_stateless/01372_wrong_order_by_removal.reference @@ -1,14 +1,14 @@ -SELECT +SELECT k, groupArrayMovingSum(v) FROM ( - SELECT + SELECT k, dt, v FROM moving_sum_num - ORDER BY + ORDER BY k ASC, dt ASC ) diff --git a/tests/queries/0_stateless/01379_with_fill_several_columns.sql b/tests/queries/0_stateless/01379_with_fill_several_columns.sql index 5d1cb4e6828..f98431b61b9 100644 --- a/tests/queries/0_stateless/01379_with_fill_several_columns.sql +++ b/tests/queries/0_stateless/01379_with_fill_several_columns.sql @@ -1,21 +1,21 @@ -SELECT +SELECT toDate((number * 10) * 86400) AS d1, toDate(number * 86400) AS d2, 'original' AS source FROM numbers(10) WHERE (number % 3) = 1 -ORDER BY +ORDER BY d2 WITH FILL, d1 WITH FILL STEP 5; SELECT '==============='; -SELECT +SELECT toDate((number * 10) * 86400) AS d1, toDate(number * 86400) AS d2, 'original' AS source FROM numbers(10) WHERE (number % 3) = 1 -ORDER BY +ORDER BY d1 WITH FILL STEP 5, d2 WITH FILL; \ No newline at end of file diff --git a/tests/queries/0_stateless/01390_remove_injective_in_uniq.reference b/tests/queries/0_stateless/01390_remove_injective_in_uniq.reference index 8fe2933b95e..94e1dbc5da7 100644 --- a/tests/queries/0_stateless/01390_remove_injective_in_uniq.reference +++ b/tests/queries/0_stateless/01390_remove_injective_in_uniq.reference @@ -1,4 +1,4 @@ -SELECT +SELECT uniq(x), uniqExact(x), uniqHLL12(x), @@ -9,7 +9,7 @@ FROM SELECT number % 2 AS x FROM numbers(10) ) -SELECT +SELECT uniq(x + y), uniqExact(x + y), uniqHLL12(x + y), @@ -17,12 +17,12 @@ SELECT uniqCombined64(x + y) FROM ( - SELECT + SELECT number % 2 AS x, number % 3 AS y FROM numbers(10) ) -SELECT +SELECT uniq(x), uniqExact(x), uniqHLL12(x), @@ -33,7 +33,7 @@ FROM SELECT number % 2 AS x FROM numbers(10) ) -SELECT +SELECT uniq(x), uniqExact(x), uniqHLL12(x), @@ -44,7 +44,7 @@ FROM SELECT number % 2 AS x FROM numbers(10) ) -SELECT +SELECT uniq(x), uniqExact(x), uniqHLL12(x), @@ -55,7 +55,7 @@ FROM SELECT number % 2 AS x FROM numbers(10) ) -SELECT +SELECT uniq(x), uniqExact(x), uniqHLL12(x), @@ -74,7 +74,7 @@ FROM ) SELECT uniq(concatAssumeInjective(\'x\', \'y\')) FROM numbers(10) -SELECT +SELECT uniq(x), uniqExact(x), uniqHLL12(x), @@ -85,7 +85,7 @@ FROM SELECT number % 2 AS x FROM numbers(10) ) -SELECT +SELECT uniq(x + y), uniqExact(x + y), uniqHLL12(x + y), @@ -93,12 +93,12 @@ SELECT uniqCombined64(x + y) FROM ( - SELECT + SELECT number % 2 AS x, number % 3 AS y FROM numbers(10) ) -SELECT +SELECT uniq(-x), uniqExact(-x), uniqHLL12(-x), @@ -109,7 +109,7 @@ FROM SELECT number % 2 AS x FROM numbers(10) ) -SELECT +SELECT uniq(bitNot(x)), uniqExact(bitNot(x)), uniqHLL12(bitNot(x)), @@ -120,7 +120,7 @@ FROM SELECT number % 2 AS x FROM numbers(10) ) -SELECT +SELECT uniq(bitNot(-x)), uniqExact(bitNot(-x)), uniqHLL12(bitNot(-x)), @@ -131,7 +131,7 @@ FROM SELECT number % 2 AS x FROM numbers(10) ) -SELECT +SELECT uniq(-bitNot(-x)), uniqExact(-bitNot(-x)), uniqHLL12(-bitNot(-x)), diff --git a/tests/queries/0_stateless/01414_push_predicate_when_contains_with_clause.reference b/tests/queries/0_stateless/01414_push_predicate_when_contains_with_clause.reference index a2ee0336191..13c8fe551c7 100644 --- a/tests/queries/0_stateless/01414_push_predicate_when_contains_with_clause.reference +++ b/tests/queries/0_stateless/01414_push_predicate_when_contains_with_clause.reference @@ -1,12 +1,12 @@ 999 1998 999 1998 -SELECT +SELECT number, square_number FROM ( WITH number * 2 AS square_number - SELECT + SELECT number, square_number FROM numbers_indexed diff --git a/tests/queries/0_stateless/01418_index_analysis_bug.sql b/tests/queries/0_stateless/01418_index_analysis_bug.sql index c5033ac7d96..aae76b63b7d 100644 --- a/tests/queries/0_stateless/01418_index_analysis_bug.sql +++ b/tests/queries/0_stateless/01418_index_analysis_bug.sql @@ -9,13 +9,13 @@ ENGINE = MergeTree() PARTITION BY toYYYYMM(eventday) ORDER BY (eventday, user_id); -INSERT INTO mytable_local SELECT +INSERT INTO mytable_local SELECT toDateTime('2020-06-01 00:00:00') + toIntervalMinute(number) AS created, toDate(created) AS eventday, if((number % 100) > 50, 742522, number % 32141) AS user_id FROM numbers(100000); -SELECT +SELECT eventday, count(*) FROM mytable_local diff --git a/tests/queries/0_stateless/01455_duplicate_distinct_optimization.reference b/tests/queries/0_stateless/01455_duplicate_distinct_optimization.reference index c5a06ca0cd3..2c54899f9f5 100644 --- a/tests/queries/0_stateless/01455_duplicate_distinct_optimization.reference +++ b/tests/queries/0_stateless/01455_duplicate_distinct_optimization.reference @@ -9,7 +9,7 @@ FROM SELECT DISTINCT number * 2 FROM ( - SELECT DISTINCT + SELECT DISTINCT number * 2, number FROM numbers(1) @@ -20,12 +20,12 @@ FROM SELECT DISTINCT number * 2 AS number FROM numbers(1) ) -SELECT +SELECT b, a FROM ( - SELECT DISTINCT + SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100) @@ -33,7 +33,7 @@ FROM SELECT DISTINCT a FROM ( - SELECT DISTINCT + SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100) @@ -44,7 +44,7 @@ FROM SELECT DISTINCT a FROM ( - SELECT DISTINCT + SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100) @@ -53,45 +53,45 @@ FROM SELECT DISTINCT a FROM ( - SELECT + SELECT a, b FROM ( - SELECT DISTINCT + SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100) ) ) -SELECT +SELECT a, b FROM ( - SELECT + SELECT b, a FROM ( - SELECT DISTINCT + SELECT DISTINCT number AS a, number AS b FROM numbers(1) ) ) -SELECT +SELECT a, b FROM ( - SELECT + SELECT b, a, a + b FROM ( - SELECT DISTINCT + SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100) @@ -103,7 +103,7 @@ FROM SELECT a FROM ( - SELECT DISTINCT + SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100) diff --git a/tests/queries/0_stateless/01470_columns_transformers.reference b/tests/queries/0_stateless/01470_columns_transformers.reference index 595d99b917f..c0f02e51ccf 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.reference +++ b/tests/queries/0_stateless/01470_columns_transformers.reference @@ -9,54 +9,54 @@ 222 18 347 111 11 173.5 1970-04-11 1970-01-11 1970-11-21 -SELECT +SELECT sum(i), sum(j), sum(k) FROM columns_transformers -SELECT +SELECT avg(i), avg(j), avg(k) FROM columns_transformers -SELECT +SELECT toDate(any(i)), toDate(any(j)), toDate(any(k)) FROM columns_transformers AS a -SELECT +SELECT length(toString(j)), length(toString(k)) FROM columns_transformers -SELECT +SELECT sum(j), sum(k) FROM columns_transformers -SELECT +SELECT avg(i), avg(k) FROM columns_transformers -SELECT +SELECT toDate(any(i)), toDate(any(j)), toDate(any(k)) FROM columns_transformers AS a -SELECT +SELECT sum(i + 1 AS i), sum(j), sum(k) FROM columns_transformers -SELECT +SELECT avg(i + 1 AS i), avg(j + 2 AS j), avg(k) FROM columns_transformers -SELECT +SELECT toDate(any(i)), toDate(any(j)), toDate(any(k)) FROM columns_transformers AS a -SELECT +SELECT (i + 1) + 1 AS i, j, k diff --git a/tests/queries/1_stateful/00063_loyalty_joins.sql b/tests/queries/1_stateful/00063_loyalty_joins.sql index b2491346673..7713c65838c 100644 --- a/tests/queries/1_stateful/00063_loyalty_joins.sql +++ b/tests/queries/1_stateful/00063_loyalty_joins.sql @@ -1,12 +1,12 @@ SET any_join_distinct_right_table_keys = 1; SET joined_subquery_requires_alias = 0; -SELECT +SELECT loyalty, count() FROM test.hits ANY LEFT JOIN ( - SELECT + SELECT UserID, sum(SearchEngineID = 2) AS yandex, sum(SearchEngineID = 3) AS google, @@ -20,7 +20,7 @@ GROUP BY loyalty ORDER BY loyalty ASC; -SELECT +SELECT loyalty, count() FROM @@ -29,7 +29,7 @@ FROM FROM test.hits ) ANY LEFT JOIN ( - SELECT + SELECT UserID, sum(SearchEngineID = 2) AS yandex, sum(SearchEngineID = 3) AS google, @@ -43,12 +43,12 @@ GROUP BY loyalty ORDER BY loyalty ASC; -SELECT +SELECT loyalty, count() FROM ( - SELECT + SELECT loyalty, UserID FROM @@ -57,7 +57,7 @@ FROM FROM test.hits ) ANY LEFT JOIN ( - SELECT + SELECT UserID, sum(SearchEngineID = 2) AS yandex, sum(SearchEngineID = 3) AS google, @@ -72,18 +72,18 @@ GROUP BY loyalty ORDER BY loyalty ASC; -SELECT +SELECT loyalty, count() AS c, bar(log(c + 1) * 1000, 0, log(3000000) * 1000, 80) FROM test.hits ANY INNER JOIN ( - SELECT + SELECT UserID, toInt8(if(yandex > google, yandex / (yandex + google), -google / (yandex + google)) * 10) AS loyalty FROM ( - SELECT + SELECT UserID, sum(SearchEngineID = 2) AS yandex, sum(SearchEngineID = 3) AS google From 11ba7049c0ad5be7aee7d6e96a73b375c8ab3b99 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 8 Sep 2020 23:33:08 +0300 Subject: [PATCH 061/341] Cover WITH format --- tests/queries/0_stateless/01471_with_format.reference | 6 ++++++ tests/queries/0_stateless/01471_with_format.sql | 2 ++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/01471_with_format.reference create mode 100644 tests/queries/0_stateless/01471_with_format.sql diff --git a/tests/queries/0_stateless/01471_with_format.reference b/tests/queries/0_stateless/01471_with_format.reference new file mode 100644 index 00000000000..806ec312bb4 --- /dev/null +++ b/tests/queries/0_stateless/01471_with_format.reference @@ -0,0 +1,6 @@ +WITH 1 +SELECT 1 +WITH + 1, + 2 +SELECT 1 diff --git a/tests/queries/0_stateless/01471_with_format.sql b/tests/queries/0_stateless/01471_with_format.sql new file mode 100644 index 00000000000..60f6fe4135a --- /dev/null +++ b/tests/queries/0_stateless/01471_with_format.sql @@ -0,0 +1,2 @@ +EXPLAIN SYNTAX WITH 1 SELECT 1; +EXPLAIN SYNTAX WITH 1, 2 SELECT 1; From 68c441e07b0072d7ffcd71491dcee219b2d90b62 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 8 Sep 2020 23:54:46 +0300 Subject: [PATCH 062/341] Add LIMIT BY format test --- .../0_stateless/01471_limit_by_format.reference | 10 ++++++++++ tests/queries/0_stateless/01471_limit_by_format.sql | 2 ++ 2 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/01471_limit_by_format.reference create mode 100644 tests/queries/0_stateless/01471_limit_by_format.sql diff --git a/tests/queries/0_stateless/01471_limit_by_format.reference b/tests/queries/0_stateless/01471_limit_by_format.reference new file mode 100644 index 00000000000..aeab30435c0 --- /dev/null +++ b/tests/queries/0_stateless/01471_limit_by_format.reference @@ -0,0 +1,10 @@ +SELECT dummy +FROM system.one +LIMIT 1 BY dummy +LIMIT 1 +SELECT dummy +FROM system.one +LIMIT 1 BY + 0 + dummy, + 0 - dummy +LIMIT 1 diff --git a/tests/queries/0_stateless/01471_limit_by_format.sql b/tests/queries/0_stateless/01471_limit_by_format.sql new file mode 100644 index 00000000000..a58099a223a --- /dev/null +++ b/tests/queries/0_stateless/01471_limit_by_format.sql @@ -0,0 +1,2 @@ +EXPLAIN SYNTAX SELECT * FROM system.one LIMIT 1 BY * LIMIT 1; +EXPLAIN SYNTAX SELECT * FROM system.one LIMIT 1 BY 0+dummy, 0-dummy LIMIT 1; From bee629c971d8f5add8fe4f205aa30f8f4e66375f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 8 Sep 2020 02:08:42 +0300 Subject: [PATCH 063/341] Use join() instead of detach() for the lists_writing_thread in DiskAccessStorage. --- src/Access/DiskAccessStorage.cpp | 47 ++++++++++++++------------------ src/Access/DiskAccessStorage.h | 5 ++-- 2 files changed, 22 insertions(+), 30 deletions(-) diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index fc80859885d..6162e4aacc2 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -426,33 +426,41 @@ bool DiskAccessStorage::writeLists() void DiskAccessStorage::scheduleWriteLists(EntityType type) { if (failed_to_write_lists) - return; + return; /// We don't try to write list files after the first fail. + /// The next restart of the server will invoke rebuilding of the list files. - bool already_scheduled = !types_of_lists_to_write.empty(); types_of_lists_to_write.insert(type); - if (already_scheduled) - return; + if (lists_writing_thread_is_waiting) + return; /// If the lists' writing thread is still waiting we can update `types_of_lists_to_write` easily, + /// without restarting that thread. + + if (lists_writing_thread.joinable()) + lists_writing_thread.join(); /// Create the 'need_rebuild_lists.mark' file. /// This file will be used later to find out if writing lists is successful or not. std::ofstream{getNeedRebuildListsMarkFilePath(directory_path)}; - startListsWritingThread(); + lists_writing_thread = ThreadFromGlobalPool{&DiskAccessStorage::listsWritingThreadFunc, this}; + lists_writing_thread_is_waiting = true; } -void DiskAccessStorage::startListsWritingThread() +void DiskAccessStorage::listsWritingThreadFunc() { - if (lists_writing_thread.joinable()) + std::unique_lock lock{mutex}; + { - if (!lists_writing_thread_exited) - return; - lists_writing_thread.detach(); + /// It's better not to write the lists files too often, that's why we need + /// the following timeout. + const auto timeout = std::chrono::minutes(1); + SCOPE_EXIT({ lists_writing_thread_is_waiting = false; }); + if (lists_writing_thread_should_exit.wait_for(lock, timeout) != std::cv_status::timeout) + return; /// The destructor requires us to exit. } - lists_writing_thread_exited = false; - lists_writing_thread = ThreadFromGlobalPool{&DiskAccessStorage::listsWritingThreadFunc, this}; + writeLists(); } @@ -466,21 +474,6 @@ void DiskAccessStorage::stopListsWritingThread() } -void DiskAccessStorage::listsWritingThreadFunc() -{ - std::unique_lock lock{mutex}; - SCOPE_EXIT({ lists_writing_thread_exited = true; }); - - /// It's better not to write the lists files too often, that's why we need - /// the following timeout. - const auto timeout = std::chrono::minutes(1); - if (lists_writing_thread_should_exit.wait_for(lock, timeout) != std::cv_status::timeout) - return; /// The destructor requires us to exit. - - writeLists(); -} - - /// Reads and parses all the ".sql" files from a specified directory /// and then saves the files "users.list", "roles.list", etc. to the same directory. bool DiskAccessStorage::rebuildLists() diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index 11eb1c3b1ad..ed2dc8b1242 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -42,9 +42,8 @@ private: void scheduleWriteLists(EntityType type); bool rebuildLists(); - void startListsWritingThread(); - void stopListsWritingThread(); void listsWritingThreadFunc(); + void stopListsWritingThread(); void insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, Notifications & notifications); void removeNoLock(const UUID & id, Notifications & notifications); @@ -74,7 +73,7 @@ private: bool failed_to_write_lists = false; /// Whether writing of the list files has been failed since the recent restart of the server. ThreadFromGlobalPool lists_writing_thread; /// List files are written in a separate thread. std::condition_variable lists_writing_thread_should_exit; /// Signals `lists_writing_thread` to exit. - std::atomic lists_writing_thread_exited = false; + bool lists_writing_thread_is_waiting = false; mutable std::list handlers_by_type[static_cast(EntityType::MAX)]; mutable std::mutex mutex; }; From cce970e40cdf1eba81a1d34c6e692ec883d544e2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 8 Sep 2020 02:09:03 +0300 Subject: [PATCH 064/341] Use join() instead of detach() for loading threads in ExternalLoader. --- src/Interpreters/ExternalLoader.cpp | 29 +++++++++++++++++++++++------ 1 file changed, 23 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index e8df205760a..dcef36de175 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -893,6 +893,8 @@ private: cancelLoading(info); } + putBackFinishedThreadsToPool(); + /// All loadings have unique loading IDs. size_t loading_id = next_id_counter++; info.loading_id = loading_id; @@ -914,6 +916,21 @@ private: } } + void putBackFinishedThreadsToPool() + { + for (auto loading_id : recently_finished_loadings) + { + auto it = loading_threads.find(loading_id); + if (it != loading_threads.end()) + { + auto thread = std::move(it->second); + loading_threads.erase(it); + thread.join(); /// It's very likely that `thread` has already finished. + } + } + recently_finished_loadings.clear(); + } + static void cancelLoading(Info & info) { if (!info.isLoading()) @@ -1095,12 +1112,11 @@ private: } min_id_to_finish_loading_dependencies.erase(std::this_thread::get_id()); - auto it = loading_threads.find(loading_id); - if (it != loading_threads.end()) - { - it->second.detach(); - loading_threads.erase(it); - } + /// Add `loading_id` to the list of recently finished loadings. + /// This list is used to later put the threads which finished loading back to the thread pool. + /// (We can't put the loading thread back to the thread pool immediately here because at this point + /// the loading thread is about to finish but it's not finished yet right now.) + recently_finished_loadings.push_back(loading_id); } /// Calculate next update time for loaded_object. Can be called without mutex locking, @@ -1158,6 +1174,7 @@ private: bool always_load_everything = false; std::atomic enable_async_loading = false; std::unordered_map loading_threads; + std::vector recently_finished_loadings; std::unordered_map min_id_to_finish_loading_dependencies; size_t next_id_counter = 1; /// should always be > 0 mutable pcg64 rnd_engine{randomSeed()}; From 0c2ecb53d13f49437ec47d336fbf68ae9e20c2f7 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 9 Sep 2020 09:35:05 +0300 Subject: [PATCH 065/341] Update arrayIndex.h --- src/Functions/array/arrayIndex.h | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Functions/array/arrayIndex.h b/src/Functions/array/arrayIndex.h index f96eb09c861..d4695ec0af5 100644 --- a/src/Functions/array/arrayIndex.h +++ b/src/Functions/array/arrayIndex.h @@ -739,10 +739,6 @@ private: if (!col_lc) return false; -// assert(checkAndGetColumn(col_lc->getDictionaryPtr().get())); -// assert(col_lc->isNullable()); -// assert(isColumnNullable(*col_lc->getDictionaryPtr().get())); - const auto [null_map_data, null_map_item] = getNullMaps(block, arguments); const IColumn& col_arg = *block.getByPosition(arguments[1]).column.get(); From 48bf65d63de2cacab0742f79fcfbab499dae384e Mon Sep 17 00:00:00 2001 From: bharatnc Date: Tue, 8 Sep 2020 23:45:04 -0700 Subject: [PATCH 066/341] StorageReplicatedMergeTree - improve integration test --- .../configs/remote_servers.xml | 14 -------------- .../test_replicated_zk_conn_failure/test.py | 13 +++++++++---- 2 files changed, 9 insertions(+), 18 deletions(-) delete mode 100644 tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml diff --git a/tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml b/tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml deleted file mode 100644 index 538aa72d386..00000000000 --- a/tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml +++ /dev/null @@ -1,14 +0,0 @@ - - - - - true - - shard_0 - node1 - 9000 - - - - - diff --git a/tests/integration/test_replicated_zk_conn_failure/test.py b/tests/integration/test_replicated_zk_conn_failure/test.py index 3f106bd2981..a860716ee82 100644 --- a/tests/integration/test_replicated_zk_conn_failure/test.py +++ b/tests/integration/test_replicated_zk_conn_failure/test.py @@ -12,11 +12,11 @@ from helpers.network import PartitionManager # 3. Try creating the table and there would be a Poco:Exception. # 4. Try creating the table again and there should not be any error # that indicates that the Directory for table already exists. - - +# 5. Final step is to restore ZooKeeper connection and verify that +# the table creation and queries work. def test_replicated_zk_conn_failure(): cluster = ClickHouseCluster(__file__) - node1 = cluster.add_instance('node1', main_configs=["configs/remote_servers.xml"], with_zookeeper=True) + node1 = cluster.add_instance('node1', with_zookeeper=True) try: cluster.start() node1.query("CREATE DATABASE replica;") @@ -27,7 +27,7 @@ def test_replicated_zk_conn_failure(): ) Engine=ReplicatedMergeTree('/clickhouse/tables/replica/test', 'node1') PARTITION BY toYYYYMMDD(event_time) - ORDER BY id;'''.format(replica=node1.name) + ORDER BY id;''' with PartitionManager() as pm: pm.drop_instance_zk_connections(node1) time.sleep(5) @@ -41,5 +41,10 @@ def test_replicated_zk_conn_failure(): # Should not expect any errors related to directory already existing # and those should have been already cleaned up during the previous retry. assert "Directory for table data data/replica/test/ already exists" not in error + # restore ZooKeeper connections. + pm.restore_instance_zk_connections(node1) + # retry create query and query the table created. + node1.query(query_create) + assert "0\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') finally: cluster.shutdown() From c34eaf5de3380e8b12f0f6e8b578bb13744660bf Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 10:08:38 +0300 Subject: [PATCH 067/341] Update ci_config and llvm --- contrib/llvm | 2 +- tests/ci/ci_config.json | 26 +++++++++++++------------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/contrib/llvm b/contrib/llvm index 3d6c7e91676..8f24d507c1c 160000 --- a/contrib/llvm +++ b/contrib/llvm @@ -1 +1 @@ -Subproject commit 3d6c7e916760b395908f28a1c885c8334d4fa98b +Subproject commit 8f24d507c1cfeec66d27f48fe74518fd278e2d25 diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index 44e9df49216..adb736a8df3 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", @@ -227,7 +227,7 @@ }, "Functional stateful tests (release)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -239,7 +239,7 @@ }, "Functional stateful tests (release, DatabaseAtomic)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -311,7 +311,7 @@ }, "Functional stateless tests (release)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -323,7 +323,7 @@ }, "Functional stateless tests (unbundled)": { "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, polymorphic parts enabled)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -347,7 +347,7 @@ }, "Functional stateless tests (release, DatabaseAtomic)": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -443,7 +443,7 @@ }, "Compatibility check": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -467,7 +467,7 @@ }, "Testflows check": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -479,7 +479,7 @@ }, "Unit tests release gcc": { "required_build_properties": { - "compiler": "gcc-9", + "compiler": "gcc-10", "package_type": "binary", "build_type": "relwithdebuginfo", "sanitizer": "none", From b40998ca007afbe702768ede9bf5776274347040 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 9 Sep 2020 15:41:38 +0800 Subject: [PATCH 068/341] Treat query as function argument. --- src/Interpreters/QueryNormalizer.cpp | 2 +- src/Parsers/ASTFunction.cpp | 13 +++++++++++-- src/Parsers/ASTFunction.h | 4 +++- src/Parsers/ExpressionElementParsers.cpp | 6 ++++-- src/TableFunctions/TableFunctionView.cpp | 19 ++++++++----------- 5 files changed, 27 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 59233218a50..07d4888b555 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -152,7 +152,7 @@ void QueryNormalizer::visitChildren(const ASTPtr & node, Data & data) { if (const auto * func_node = node->as()) { - if (func_node->query) + if (func_node->tryGetQueryArgument()) { if (func_node->name != "view") throw Exception("Query argument can only be used in the `view` TableFunction", ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 07429c8104f..bbd910ae875 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -48,7 +48,6 @@ ASTPtr ASTFunction::clone() const auto res = std::make_shared(*this); res->children.clear(); - if (query) { res->query = query->clone(); res->children.push_back(res->query); } if (arguments) { res->arguments = arguments->clone(); res->children.push_back(res->arguments); } if (parameters) { res->parameters = parameters->clone(); res->children.push_back(res->parameters); } @@ -112,6 +111,16 @@ static bool highlightStringLiteralWithMetacharacters(const ASTPtr & node, const } +ASTSelectWithUnionQuery * ASTFunction::tryGetQueryArgument() const +{ + if (arguments && arguments->children.size() == 1) + { + return arguments->children[0]->as(); + } + return nullptr; +} + + void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { FormatStateStacked nested_need_parens = frame; @@ -119,7 +128,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format nested_need_parens.need_parens = true; nested_dont_need_parens.need_parens = false; - if (query) + if (auto * query = tryGetQueryArgument()) { std::string nl_or_nothing = settings.one_line ? "" : "\n"; std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' '); diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index b94614426d8..3b87ab68282 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -13,7 +14,6 @@ class ASTFunction : public ASTWithAlias { public: String name; - ASTPtr query; // It's possible for a function to accept a query as its only argument. ASTPtr arguments; /// parameters - for parametric aggregate function. Example: quantile(0.9)(x) - what in first parens are 'parameters'. ASTPtr parameters; @@ -26,6 +26,8 @@ public: void updateTreeHashImpl(SipHash & hash_state) const override; + ASTSelectWithUnionQuery * tryGetQueryArgument() const; + protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 985507071be..64e3a0363d1 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -260,8 +260,10 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ++pos; auto function_node = std::make_shared(); tryGetIdentifierNameInto(identifier, function_node->name); - function_node->query = query; - function_node->children.push_back(function_node->query); + auto expr_list_with_single_query = std::make_shared(); + expr_list_with_single_query->children.push_back(query); + function_node->arguments = expr_list_with_single_query; + function_node->children.push_back(function_node->arguments); node = function_node; return true; } diff --git a/src/TableFunctions/TableFunctionView.cpp b/src/TableFunctions/TableFunctionView.cpp index 6166fa56f47..8d3f7b06fa3 100644 --- a/src/TableFunctions/TableFunctionView.cpp +++ b/src/TableFunctions/TableFunctionView.cpp @@ -20,18 +20,15 @@ StoragePtr TableFunctionView::executeImpl(const ASTPtr & ast_function, const Con { if (const auto * function = ast_function->as()) { - if (function->query) + if (auto * select = function->tryGetQueryArgument()) { - if (auto * select = function->query->as()) - { - auto sample = InterpreterSelectWithUnionQuery::getSampleBlock(function->query, context); - auto columns = ColumnsDescription(sample.getNamesAndTypesList()); - ASTCreateQuery create; - create.select = select; - auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns); - res->startup(); - return res; - } + auto sample = InterpreterSelectWithUnionQuery::getSampleBlock(function->arguments->children[0] /* ASTPtr */, context); + auto columns = ColumnsDescription(sample.getNamesAndTypesList()); + ASTCreateQuery create; + create.select = select; + auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns); + res->startup(); + return res; } } throw Exception("Table function '" + getName() + "' requires a query argument.", ErrorCodes::BAD_ARGUMENTS); From d8fce448a29eecff6e1dc77299f63c3e75f0fbbc Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 9 Sep 2020 14:20:14 +0800 Subject: [PATCH 069/341] Implement null_as_default for JSONStrings formats --- src/DataTypes/DataTypeNullable.cpp | 12 ++++++++++-- src/DataTypes/DataTypeNullable.h | 2 ++ .../Impl/JSONCompactEachRowRowInputFormat.cpp | 7 ++++--- .../Formats/Impl/JSONEachRowRowInputFormat.cpp | 7 ++++--- .../01016_input_null_as_default.reference | 5 +++++ .../0_stateless/01016_input_null_as_default.sh | 8 ++++++++ 6 files changed, 33 insertions(+), 8 deletions(-) diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 3318196b951..6e452b2759b 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -318,13 +318,20 @@ ReturnType DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer void DataTypeNullable::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - safeDeserialize(column, *nested_data_type, + deserializeWholeText(column, istr, settings, nested_data_type); +} + +template +ReturnType DataTypeNullable::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, + const DataTypePtr & nested_data_type) +{ + return safeDeserialize(column, *nested_data_type, [&istr] { return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr) || checkStringByFirstCharacterAndAssertTheRest("ᴺᵁᴸᴸ", istr); }, - [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsWholeText(nested, istr, settings); }); + [&nested_data_type, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsWholeText(nested, istr, settings); }); } @@ -551,6 +558,7 @@ DataTypePtr removeNullable(const DataTypePtr & type) } +template bool DataTypeNullable::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const DataTypePtr & nested); template bool DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const DataTypePtr & nested); template bool DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &, const DataTypePtr & nested); template bool DataTypeNullable::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const DataTypePtr & nested); diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index 22d403da6c4..587eecdf32e 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -103,6 +103,8 @@ public: /// If ReturnType is bool, check for NULL and deserialize value into non-nullable column (and return true) or insert default value of nested type (and return false) /// If ReturnType is void, deserialize Nullable(T) template + static ReturnType deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const DataTypePtr & nested); + template static ReturnType deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const DataTypePtr & nested); template static ReturnType deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &, const DataTypePtr & nested); diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index eb697ce5318..49c8d29ca2f 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -205,14 +205,15 @@ void JSONCompactEachRowRowInputFormat::readField(size_t index, MutableColumns & 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); + if (format_settings.null_as_default && !type->isNullable()) + read_columns[index] = DataTypeNullable::deserializeWholeText(*columns[index], buf, format_settings, type); + else + type->deserializeAsWholeText(*columns[index], buf, format_settings); } else { diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 9ba82fbb009..ab775a3e7aa 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -146,14 +146,15 @@ void JSONEachRowRowInputFormat::readField(size_t index, MutableColumns & columns 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); + if (format_settings.null_as_default && !type->isNullable()) + read_columns[index] = DataTypeNullable::deserializeWholeText(*columns[index], buf, format_settings, type); + else + type->deserializeAsWholeText(*columns[index], buf, format_settings); } else { diff --git a/tests/queries/0_stateless/01016_input_null_as_default.reference b/tests/queries/0_stateless/01016_input_null_as_default.reference index ba9657bf16e..d7010f42d4e 100644 --- a/tests/queries/0_stateless/01016_input_null_as_default.reference +++ b/tests/queries/0_stateless/01016_input_null_as_default.reference @@ -18,6 +18,11 @@ JSONEachRow 1 world 3 2019-07-23 [1,2,3] ('tuple',3.14) 2 Hello 123 2019-06-19 [] ('test',2.71828) 3 Hello 42 2019-06-19 [1,2,3] ('default',0.75) +JSONStringsEachRow +0 1 42 2019-07-22 [10,20,30] ('default',0) +1 world 3 2019-07-23 [1,2,3] ('tuple',3.14) +2 Hello 123 2019-06-19 [] ('test',2.71828) +3 Hello 42 2019-06-19 [1,2,3] ('default',0.75) Template (Quoted) 0 1 42 2019-07-22 [10,20,30] ('default',0) 1 world 3 2019-07-23 [1,2,3] ('tuple',3.14) diff --git a/tests/queries/0_stateless/01016_input_null_as_default.sh b/tests/queries/0_stateless/01016_input_null_as_default.sh index a40287eaba8..f31e6591e97 100755 --- a/tests/queries/0_stateless/01016_input_null_as_default.sh +++ b/tests/queries/0_stateless/01016_input_null_as_default.sh @@ -38,6 +38,14 @@ echo '{"i": null, "s": "1", "n": null, "d": "2019-07-22", "a": [10, 20, 30], "t" $CLICKHOUSE_CLIENT --query="SELECT * FROM null_as_default ORDER BY i"; $CLICKHOUSE_CLIENT --query="TRUNCATE TABLE null_as_default"; +echo 'JSONStringsEachRow' +echo '{"i": "null", "s": "1", "n": "ᴺᵁᴸᴸ", "d": "2019-07-22", "a": "[10, 20, 30]", "t": "NULL"} +{"i": "1", "s": "world", "n": "3", "d": "2019-07-23", "a": "null", "t": "('\''tuple'\'', 3.14)"} +{"i": "2", "s": "null", "n": "123", "d": "null", "a": "[]", "t": "('\''test'\'', 2.71828)"} +{"i": "3", "s": "null", "n": "null", "d": "null", "a": "null", "t": "null"}' | $CLICKHOUSE_CLIENT --input_format_null_as_default=1 --query="INSERT INTO null_as_default FORMAT JSONStringsEachRow"; +$CLICKHOUSE_CLIENT --query="SELECT * FROM null_as_default ORDER BY i"; +$CLICKHOUSE_CLIENT --query="TRUNCATE TABLE null_as_default"; + echo 'Template (Quoted)' echo 'NULL, '\''1'\'', null, '\''2019-07-22'\'', [10, 20, 30], NuLl 1, '\''world'\'', 3, '\''2019-07-23'\'', NULL, ('\''tuple'\'', 3.14) From 4ba8f8960bd4e86a57dafba6a0aa1574b66d97db Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 12:53:24 +0300 Subject: [PATCH 070/341] Increase frame-larger-than --- cmake/warnings.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 2f78dc34079..aec3e46ffa6 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -23,7 +23,7 @@ option (WEVERYTHING "Enables -Weverything option with some exceptions. This is i # Control maximum size of stack frames. It can be important if the code is run in fibers with small stack size. # Only in release build because debug has too large stack frames. if ((NOT CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") AND (NOT SANITIZE)) - add_warning(frame-larger-than=16384) + add_warning(frame-larger-than=32768) endif () if (COMPILER_CLANG) From 2ea59cb0c2fc9ea25cb6029f910952903e1d0bbd Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 9 Sep 2020 17:54:41 +0800 Subject: [PATCH 071/341] Fix tests --- .../0_stateless/01446_json_strings_each_row.reference | 4 ++-- .../01448_json_compact_strings_each_row.reference | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01446_json_strings_each_row.reference b/tests/queries/0_stateless/01446_json_strings_each_row.reference index 84d41095b77..812026534ea 100644 --- a/tests/queries/0_stateless/01446_json_strings_each_row.reference +++ b/tests/queries/0_stateless/01446_json_strings_each_row.reference @@ -16,7 +16,7 @@ {"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"} +{"v1":"first","v2":"1","v3":"2","v4":"8"} +{"v1":"second","v2":"2","v3":"32","v4":"6"} 7 {"v1":"16","n.id":"[15,16,17]","n.name":"['first','second','third']"} diff --git a/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference b/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference index 0b05f050b29..fb1a066f272 100644 --- a/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference +++ b/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference @@ -24,16 +24,16 @@ ["first", "1", "2", "0"] ["second", "2", "0", "6"] 6 -["first", "1", "2", "0"] -["second", "2", "0", "6"] +["first", "1", "2", "8"] +["second", "2", "32", "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"] +["first", "1", "2", "8"] +["second", "2", "32", "6"] 10 ["first", "1", "16", "8"] ["second", "2", "32", "8"] From f528cd9f97b4f7c54a6c22406f09983d055ce642 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 13:01:12 +0300 Subject: [PATCH 072/341] Forward compiler version to unbundled build --- docker/packager/packager | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index 5874bedd17a..909f20acd6d 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -93,7 +93,7 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ cxx = cc.replace('gcc', 'g++').replace('clang', 'clang++') - if image_type == "deb": + if image_type == "deb" or image_type == "unbundled": result.append("DEB_CC={}".format(cc)) result.append("DEB_CXX={}".format(cxx)) elif image_type == "binary": From ca6b634eb0466361da6f3526a6611ab0ccd8bfc1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 13:51:01 +0300 Subject: [PATCH 073/341] Install gcc-10 from proposed repo --- docker/packager/binary/Dockerfile | 13 +++++++++++-- docker/packager/deb/Dockerfile | 12 ++++++++++-- 2 files changed, 21 insertions(+), 4 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 45c35c2e0f3..b911b59a41d 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -32,8 +32,6 @@ RUN apt-get update \ curl \ gcc-9 \ g++-9 \ - gcc-10 \ - g++-10 \ llvm-${LLVM_VERSION} \ clang-${LLVM_VERSION} \ lld-${LLVM_VERSION} \ @@ -93,5 +91,16 @@ RUN wget -nv "https://developer.arm.com/-/media/Files/downloads/gnu-a/8.3-2019.0 # Download toolchain for FreeBSD 11.3 RUN wget -nv https://clickhouse-datasets.s3.yandex.net/toolchains/toolchains/freebsd-11.3-toolchain.tar.xz +# NOTE: For some reason we have outdated version of gcc-10 in ubuntu 20.04 stable. +# Current workaround is to use latest version proposed repo. Remove as soon as +# gcc-10.2 appear in stable repo. +RUN echo 'deb http://archive.ubuntu.com/ubuntu/ focal-proposed restricted main multiverse universe' > /etc/apt/sources.list.d/proposed-repositories.list + +RUN apt-get update \ + && apt-get install gcc-10 g++10 --yes + +RUN rm /etc/apt/sources.list.d/proposed-repositories.list + + COPY build.sh / CMD ["/bin/bash", "/build.sh"] diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index 87f4582f8e2..30334504c55 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -42,8 +42,6 @@ RUN export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ # Libraries from OS are only needed to test the "unbundled" build (this is not used in production). RUN apt-get update \ && apt-get install \ - gcc-10 \ - g++-10 \ gcc-9 \ g++-9 \ clang-11 \ @@ -75,6 +73,16 @@ RUN apt-get update \ pigz \ --yes --no-install-recommends +# NOTE: For some reason we have outdated version of gcc-10 in ubuntu 20.04 stable. +# Current workaround is to use latest version proposed repo. Remove as soon as +# gcc-10.2 appear in stable repo. +RUN echo 'deb http://archive.ubuntu.com/ubuntu/ focal-proposed restricted main multiverse universe' > /etc/apt/sources.list.d/proposed-repositories.list + +RUN apt-get update \ + && apt-get install gcc-10 g++10 --yes --no-install-recommends + +RUN rm /etc/apt/sources.list.d/proposed-repositories.list + # This symlink required by gcc to find lld compiler RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld From 150d8d4e79b0fff23cde361ed460e71c8729a4c5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 14:11:59 +0300 Subject: [PATCH 074/341] Better recursive copy in integration tests --- tests/integration/helpers/cluster.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6b8cdcf7989..44a22d3fe2e 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1,6 +1,5 @@ import base64 import cassandra.cluster -import distutils.dir_util import docker import errno import httplib @@ -19,6 +18,7 @@ import socket import subprocess import time import urllib +import traceback import xml.dom.minidom from dicttoxml import dicttoxml from kazoo.client import KazooClient @@ -667,6 +667,7 @@ class ClickHouseCluster: except BaseException, e: print "Failed to start cluster: " print str(e) + print traceback.print_exc() raise def shutdown(self, kill=True): @@ -1164,10 +1165,10 @@ class ClickHouseInstance: db_dir = p.abspath(p.join(self.path, 'database')) print "Setup database dir {}".format(db_dir) - os.mkdir(db_dir) if self.clickhouse_path_dir is not None: print "Database files taken from {}".format(self.clickhouse_path_dir) - distutils.dir_util.copy_tree(self.clickhouse_path_dir, db_dir) + shutil.copytree(self.clickhouse_path_dir, db_dir) + print "Database copied from {} to {}".format(self.clickhouse_path_dir, db_dir) logs_dir = p.abspath(p.join(self.path, 'logs')) print "Setup logs dir {}".format(logs_dir) @@ -1228,7 +1229,6 @@ class ClickHouseInstance: binary_volume = "- " + self.server_bin_path + ":/usr/share/clickhouse_fresh" odbc_bridge_volume = "- " + self.odbc_bridge_bin_path + ":/usr/share/clickhouse-odbc-bridge_fresh" - with open(self.docker_compose_path, 'w') as docker_compose: docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format( image=self.image, @@ -1251,8 +1251,8 @@ class ClickHouseInstance: app_net=app_net, ipv4_address=ipv4_address, ipv6_address=ipv6_address, - net_aliases = net_aliases, - net_alias1 = net_alias1, + net_aliases=net_aliases, + net_alias1=net_alias1, )) def destroy_dir(self): From 98f19a5d50b7c1a1017628f63813e6bbabb6a2e5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 14:29:53 +0300 Subject: [PATCH 075/341] Better permissions --- .../clickhouse_path/format_schemas/rabbitmq.proto | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto diff --git a/tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto b/tests/integration/test_storage_rabbitmq/clickhouse_path/format_schemas/rabbitmq.proto old mode 100644 new mode 100755 From 97616f2982b6c334ac9894fc302686982e1f0213 Mon Sep 17 00:00:00 2001 From: Simon Podlipsky Date: Wed, 9 Sep 2020 13:33:34 +0200 Subject: [PATCH 076/341] Mention db requirement in dictionary functions --- docs/en/sql-reference/functions/ext-dict-functions.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/sql-reference/functions/ext-dict-functions.md b/docs/en/sql-reference/functions/ext-dict-functions.md index 49b1c2dda2c..e0ecdd74fad 100644 --- a/docs/en/sql-reference/functions/ext-dict-functions.md +++ b/docs/en/sql-reference/functions/ext-dict-functions.md @@ -3,6 +3,9 @@ toc_priority: 58 toc_title: External Dictionaries --- +!!! attention "Attention" + `dict_name` parameter must be fully qualified for dictionaries created with DDL queries. Eg. `.`. + # Functions for Working with External Dictionaries {#ext_dict_functions} For information on connecting and configuring external dictionaries, see [External dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). From c535d752438c9616dab8fac79bf8594acb44665a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 14:47:34 +0300 Subject: [PATCH 077/341] Add update --- docker/packager/binary/Dockerfile | 2 +- docker/packager/deb/Dockerfile | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index b911b59a41d..893e9191b1e 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -99,7 +99,7 @@ RUN echo 'deb http://archive.ubuntu.com/ubuntu/ focal-proposed restricted main m RUN apt-get update \ && apt-get install gcc-10 g++10 --yes -RUN rm /etc/apt/sources.list.d/proposed-repositories.list +RUN rm /etc/apt/sources.list.d/proposed-repositories.list && apt-get update COPY build.sh / diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index 30334504c55..4b7c2ae53a4 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -81,7 +81,7 @@ RUN echo 'deb http://archive.ubuntu.com/ubuntu/ focal-proposed restricted main m RUN apt-get update \ && apt-get install gcc-10 g++10 --yes --no-install-recommends -RUN rm /etc/apt/sources.list.d/proposed-repositories.list +RUN rm /etc/apt/sources.list.d/proposed-repositories.list && apt-get update # This symlink required by gcc to find lld compiler RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld From a8f5d9e70fef1202285eab1911f7aa794e0828f6 Mon Sep 17 00:00:00 2001 From: myrrc Date: Wed, 9 Sep 2020 15:52:44 +0300 Subject: [PATCH 078/341] added the test and comment --- src/Columns/ColumnLowCardinality.h | 4 ++++ .../0_stateless/01414_low_cardinality_nullable.reference | 1 + tests/queries/0_stateless/01414_low_cardinality_nullable.sql | 2 ++ 3 files changed, 7 insertions(+) diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 00f58a133cf..0aeda4567fd 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -171,6 +171,10 @@ public: bool isNumeric() const override { return getDictionary().isNumeric(); } bool lowCardinality() const override { return true; } + /** + * Checks if the dictionary column is Nullable(T). + * So LC(Nullable(T)) would return true, LC(U) -- false. + */ bool nestedIsNullable() const { return isColumnNullable(*dictionary.getColumnUnique().getNestedColumn()); } const IColumnUnique & getDictionary() const { return dictionary.getColumnUnique(); } diff --git a/tests/queries/0_stateless/01414_low_cardinality_nullable.reference b/tests/queries/0_stateless/01414_low_cardinality_nullable.reference index bf7b6cf4f76..51825f5cb76 100644 --- a/tests/queries/0_stateless/01414_low_cardinality_nullable.reference +++ b/tests/queries/0_stateless/01414_low_cardinality_nullable.reference @@ -80,3 +80,4 @@ 1 1 1 +2 diff --git a/tests/queries/0_stateless/01414_low_cardinality_nullable.sql b/tests/queries/0_stateless/01414_low_cardinality_nullable.sql index 6e311f9d519..9a554ead776 100644 --- a/tests/queries/0_stateless/01414_low_cardinality_nullable.sql +++ b/tests/queries/0_stateless/01414_low_cardinality_nullable.sql @@ -217,4 +217,6 @@ SELECT count() FROM lc_nullable WHERE has(date_time, toDateTime('1970-01-01 03:0 SELECT count() FROM lc_nullable WHERE has(str, '100'); SELECT count() FROM lc_nullable WHERE has(fixed_string, toFixedString('100', 5)); +SELECT count() FROM lc_nullable WHERE has(date, toDate(has(u64, 1), '1970-01\002')); + DROP TABLE IF EXISTS lc_nullable; From 956138635de536560d0843025720d7ce7b947cf3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 15:59:26 +0300 Subject: [PATCH 079/341] Fix compiler name --- docker/packager/binary/Dockerfile | 2 +- docker/packager/deb/Dockerfile | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 893e9191b1e..03bb3b5aefa 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -97,7 +97,7 @@ RUN wget -nv https://clickhouse-datasets.s3.yandex.net/toolchains/toolchains/fre RUN echo 'deb http://archive.ubuntu.com/ubuntu/ focal-proposed restricted main multiverse universe' > /etc/apt/sources.list.d/proposed-repositories.list RUN apt-get update \ - && apt-get install gcc-10 g++10 --yes + && apt-get install gcc-10 g++-10 --yes RUN rm /etc/apt/sources.list.d/proposed-repositories.list && apt-get update diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index 4b7c2ae53a4..a3c87f13fe4 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -79,7 +79,7 @@ RUN apt-get update \ RUN echo 'deb http://archive.ubuntu.com/ubuntu/ focal-proposed restricted main multiverse universe' > /etc/apt/sources.list.d/proposed-repositories.list RUN apt-get update \ - && apt-get install gcc-10 g++10 --yes --no-install-recommends + && apt-get install gcc-10 g++-10 --yes --no-install-recommends RUN rm /etc/apt/sources.list.d/proposed-repositories.list && apt-get update From b68782d285e5ea76f7318b55bf41cf337dfa71fc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 9 Sep 2020 16:32:50 +0300 Subject: [PATCH 080/341] enable more tests with Atomic database --- docker/test/stress/stress | 2 +- programs/client/Client.cpp | 28 +++++++- src/Interpreters/DatabaseCatalog.cpp | 5 +- src/Interpreters/InterpreterCreateQuery.cpp | 1 + .../MergeTree/MergeTreeWriteAheadLog.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 16 +++-- src/Storages/System/StorageSystemTables.cpp | 6 ++ .../queries/0_stateless/00116_storage_set.sql | 2 +- .../00180_attach_materialized_view.sql | 2 +- ...per_deduplication_and_unexpected_parts.sql | 2 +- .../00281_compile_sizeof_packed.re | 0 .../0_stateless/00311_array_primary_key.sql | 2 +- .../00423_storage_log_single_thread.sql | 6 +- .../00816_long_concurrent_alter_column.sh | 27 +++++--- .../01190_full_attach_syntax.reference | 13 ++++ .../0_stateless/01190_full_attach_syntax.sql | 66 +++++++++++++++++++ .../01305_replica_create_drop_zookeeper.sh | 20 ++++-- .../00065_loyalty_with_storage_join.sql | 2 +- tests/queries/skip_list.json | 33 +--------- 19 files changed, 172 insertions(+), 62 deletions(-) delete mode 100644 tests/queries/0_stateless/00281_compile_sizeof_packed.re create mode 100644 tests/queries/0_stateless/01190_full_attach_syntax.reference create mode 100644 tests/queries/0_stateless/01190_full_attach_syntax.sql diff --git a/docker/test/stress/stress b/docker/test/stress/stress index e8675da1546..60db5ec465c 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -28,7 +28,7 @@ def get_options(i): options = "" if 0 < i: options += " --order=random" - if i == 1: + if i % 2 == 1: options += " --atomic-db-engine" return options diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index c9701950dc5..83e4062b1f3 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -919,7 +919,33 @@ private: while (begin < end) { const char * pos = begin; - ASTPtr orig_ast = parseQuery(pos, end, true); + + ASTPtr orig_ast; + try + { + orig_ast = parseQuery(pos, end, true); + } + catch (Exception & e) + { + if (!test_mode) + throw; + + /// Try find test hint for syntax error + const char * end_of_line = find_first_symbols<'\n'>(begin, end); + TestHint hint(true, String(begin, end_of_line - begin)); + if (hint.serverError()) /// Syntax errors are considered as client errors + throw; + if (hint.clientError() != e.code()) + { + if (hint.clientError()) + e.addMessage("\nExpected clinet error: " + std::to_string(hint.clientError())); + throw; + } + + /// It's expected syntax error, skip the line + begin = end_of_line; + continue; + } if (!orig_ast) { diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 6153f6b52fb..049341918b9 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -657,7 +657,10 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr /// Table was removed from database. Enqueue removal of its data from disk. time_t drop_time; if (table) + { drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + table->is_dropped = true; + } else { /// Try load table from metadata to drop it correctly (e.g. remove metadata from zk or remove data from all volumes) @@ -674,6 +677,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr try { table = createTableFromAST(*create, table_id.getDatabaseName(), data_path, *global_context, false).second; + table->is_dropped = true; } catch (...) { @@ -763,7 +767,6 @@ void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) const if (table.table) { table.table->drop(); - table.table->is_dropped = true; } /// Even if table is not loaded, try remove its data from disk. diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 06973ab029b..d7230940bb2 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -673,6 +673,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) create.attach_short_syntax = true; create.if_not_exists = if_not_exists; } + /// TODO maybe assert table structure if create.attach_short_syntax is false? if (!create.temporary && create.database.empty()) create.database = current_database; diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 53ef72f3208..3fa3a7e3e40 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -66,6 +66,7 @@ void MergeTreeWriteAheadLog::dropPart(const String & part_name) writeIntBinary(static_cast(0), *out); writeIntBinary(static_cast(ActionType::DROP_PART), *out); writeStringBinary(part_name, *out); + out->next(); } void MergeTreeWriteAheadLog::rotate(const std::lock_guard &) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6058632d220..6458fe127da 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4260,9 +4260,13 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( * To do this, check its node `log_pointer` - the maximum number of the element taken from `log` + 1. */ - const auto & check_replica_become_inactive = [this, &replica]() + bool waiting_itself = replica == replica_name; + + const auto & stop_waiting = [&]() { - return !getZooKeeper()->exists(zookeeper_path + "/replicas/" + replica + "/is_active"); + bool stop_waiting_itself = waiting_itself && is_dropped; + bool stop_waiting_non_active = !wait_for_non_active && !getZooKeeper()->exists(zookeeper_path + "/replicas/" + replica + "/is_active"); + return stop_waiting_itself || stop_waiting_non_active; }; constexpr auto event_wait_timeout_ms = 1000; @@ -4277,7 +4281,7 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( LOG_DEBUG(log, "Waiting for {} to pull {} to queue", replica, log_node_name); /// Let's wait until entry gets into the replica queue. - while (wait_for_non_active || !check_replica_become_inactive()) + while (!stop_waiting()) { zkutil::EventPtr event = std::make_shared(); @@ -4325,7 +4329,7 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( LOG_DEBUG(log, "Waiting for {} to pull {} to queue", replica, log_node_name); /// Let's wait until the entry gets into the replica queue. - while (wait_for_non_active || !check_replica_become_inactive()) + while (!stop_waiting()) { zkutil::EventPtr event = std::make_shared(); @@ -4378,10 +4382,8 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( /// Third - wait until the entry disappears from the replica queue or replica become inactive. String path_to_wait_on = zookeeper_path + "/replicas/" + replica + "/queue/" + queue_entry_to_wait_for; - if (wait_for_non_active) - return getZooKeeper()->waitForDisappear(path_to_wait_on); - return getZooKeeper()->waitForDisappear(path_to_wait_on, check_replica_become_inactive); + return getZooKeeper()->waitForDisappear(path_to_wait_on, stop_waiting); } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 5b7dad836e9..0ad961ad7d8 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -344,6 +344,12 @@ protected: { ASTPtr ast = database->tryGetCreateTableQuery(table_name, context); + if (ast && !context.getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil) + { + auto & create = ast->as(); + create.uuid = UUIDHelpers::Nil; + } + if (columns_mask[src_index++]) res_columns[res_index++]->insert(ast ? queryToString(ast) : ""); diff --git a/tests/queries/0_stateless/00116_storage_set.sql b/tests/queries/0_stateless/00116_storage_set.sql index aa93a0620d0..0eeed7e859a 100644 --- a/tests/queries/0_stateless/00116_storage_set.sql +++ b/tests/queries/0_stateless/00116_storage_set.sql @@ -19,7 +19,7 @@ INSERT INTO set2 VALUES ('abc'), ('World'); SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2; DETACH TABLE set2; -ATTACH TABLE set2 (x String) ENGINE = Set; +ATTACH TABLE set2; SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2; diff --git a/tests/queries/0_stateless/00180_attach_materialized_view.sql b/tests/queries/0_stateless/00180_attach_materialized_view.sql index 089e4926bcf..d674c0bd277 100644 --- a/tests/queries/0_stateless/00180_attach_materialized_view.sql +++ b/tests/queries/0_stateless/00180_attach_materialized_view.sql @@ -6,7 +6,7 @@ CREATE TABLE t_00180 (x UInt8) ENGINE = Null; CREATE MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180; DETACH TABLE mv_00180; -ATTACH MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180; +ATTACH TABLE mv_00180; DROP TABLE t_00180; DROP TABLE mv_00180; diff --git a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql index 623218af167..c14ce53d4a3 100644 --- a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql +++ b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql @@ -21,7 +21,7 @@ INSERT INTO deduplication (x) VALUES (1); SELECT * FROM deduplication; DETACH TABLE deduplication; -ATTACH TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00226/deduplication', 'r1', d, x, 1); +ATTACH TABLE deduplication; SELECT * FROM deduplication; diff --git a/tests/queries/0_stateless/00281_compile_sizeof_packed.re b/tests/queries/0_stateless/00281_compile_sizeof_packed.re deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/00311_array_primary_key.sql b/tests/queries/0_stateless/00311_array_primary_key.sql index 0ea368609da..0e066c64f89 100644 --- a/tests/queries/0_stateless/00311_array_primary_key.sql +++ b/tests/queries/0_stateless/00311_array_primary_key.sql @@ -11,7 +11,7 @@ INSERT INTO array_pk VALUES ([5, 6], 'ghi', 6); SELECT * FROM array_pk ORDER BY n; DETACH TABLE array_pk; -ATTACH TABLE array_pk (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1); +ATTACH TABLE array_pk; SELECT * FROM array_pk ORDER BY n; diff --git a/tests/queries/0_stateless/00423_storage_log_single_thread.sql b/tests/queries/0_stateless/00423_storage_log_single_thread.sql index 7d5e14c9ee5..8eff9323564 100644 --- a/tests/queries/0_stateless/00423_storage_log_single_thread.sql +++ b/tests/queries/0_stateless/00423_storage_log_single_thread.sql @@ -5,7 +5,7 @@ SELECT * FROM log LIMIT 1; SELECT * FROM log; DETACH TABLE log; -ATTACH TABLE log (s String) ENGINE = Log; +ATTACH TABLE log; SELECT * FROM log; SELECT * FROM log LIMIT 1; @@ -15,13 +15,13 @@ INSERT INTO log VALUES ('Hello'), ('World'); SELECT * FROM log LIMIT 1; DETACH TABLE log; -ATTACH TABLE log (s String) ENGINE = Log; +ATTACH TABLE log; SELECT * FROM log LIMIT 1; SELECT * FROM log; DETACH TABLE log; -ATTACH TABLE log (s String) ENGINE = Log; +ATTACH TABLE log; SELECT * FROM log; SELECT * FROM log LIMIT 1; diff --git a/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh b/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh index 93421e003f6..8fdd6654bae 100755 --- a/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh +++ b/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh @@ -11,34 +11,34 @@ echo "CREATE TABLE concurrent_alter_column (ts DATETIME) ENGINE = MergeTree PART function thread1() { while true; do - for i in {1..500}; do echo "ALTER TABLE concurrent_alter_column ADD COLUMN c$i DOUBLE;"; done | ${CLICKHOUSE_CLIENT} -n --query_id=alter1 + for i in {1..500}; do echo "ALTER TABLE concurrent_alter_column ADD COLUMN c$i DOUBLE;"; done | ${CLICKHOUSE_CLIENT} -n --query_id=alter_00816_1 done } function thread2() { while true; do - echo "ALTER TABLE concurrent_alter_column ADD COLUMN d DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter2; + echo "ALTER TABLE concurrent_alter_column ADD COLUMN d DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_2; sleep "$(echo 0.0$RANDOM)"; - echo "ALTER TABLE concurrent_alter_column DROP COLUMN d" | ${CLICKHOUSE_CLIENT} --query_id=alter2; + echo "ALTER TABLE concurrent_alter_column DROP COLUMN d" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_2; done } function thread3() { while true; do - echo "ALTER TABLE concurrent_alter_column ADD COLUMN e DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter3; + echo "ALTER TABLE concurrent_alter_column ADD COLUMN e DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_3; sleep "$(echo 0.0$RANDOM)"; - echo "ALTER TABLE concurrent_alter_column DROP COLUMN e" | ${CLICKHOUSE_CLIENT} --query_id=alter3; + echo "ALTER TABLE concurrent_alter_column DROP COLUMN e" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_3; done } function thread4() { while true; do - echo "ALTER TABLE concurrent_alter_column ADD COLUMN f DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter4; + echo "ALTER TABLE concurrent_alter_column ADD COLUMN f DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_4; sleep "$(echo 0.0$RANDOM)"; - echo "ALTER TABLE concurrent_alter_column DROP COLUMN f" | ${CLICKHOUSE_CLIENT} --query_id=alter4; + echo "ALTER TABLE concurrent_alter_column DROP COLUMN f" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_4; done } @@ -57,9 +57,18 @@ timeout $TIMEOUT bash -c thread4 2> /dev/null & wait -echo "DROP TABLE concurrent_alter_column" | ${CLICKHOUSE_CLIENT} +echo "DROP TABLE concurrent_alter_column NO DELAY" | ${CLICKHOUSE_CLIENT} # NO DELAY has effect only for Atomic database + +db_engine=`$CLICKHOUSE_CLIENT -q "SELECT engine FROM system.databases WHERE name=currentDatabase()"` +if [[ $db_engine == "Atomic" ]]; then + # DROP is non-blocking, so wait for alters + while true; do + $CLICKHOUSE_CLIENT -q "SELECT c = 0 FROM (SELECT count() as c FROM system.processes WHERE query_id LIKE 'alter_00816_%')" | grep 1 > /dev/null && break; + sleep 1; + done +fi # Check for deadlocks -echo "SELECT * FROM system.processes WHERE query_id LIKE 'alter%'" | ${CLICKHOUSE_CLIENT} +echo "SELECT * FROM system.processes WHERE query_id LIKE 'alter_00816_%'" | ${CLICKHOUSE_CLIENT} echo 'did not crash' diff --git a/tests/queries/0_stateless/01190_full_attach_syntax.reference b/tests/queries/0_stateless/01190_full_attach_syntax.reference new file mode 100644 index 00000000000..619861849c8 --- /dev/null +++ b/tests/queries/0_stateless/01190_full_attach_syntax.reference @@ -0,0 +1,13 @@ +CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) +CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) +CREATE TABLE default.log\n(\n `s` String\n)\nENGINE = Log +CREATE TABLE default.log\n(\n `s` String\n)\nENGINE = Log() +test +CREATE TABLE default.mt\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date MATERIALIZED \'2000-01-01\'\n)\nENGINE = MergeTree(d, (key, s, n), 1) +[1,2] Hello 2 +CREATE TABLE default.mt\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date\n)\nENGINE = MergeTree(d, (key, s, n), 1) +CREATE MATERIALIZED VIEW default.mv\n(\n `s` String\n)\nENGINE = Null AS\nSELECT *\nFROM default.log +CREATE MATERIALIZED VIEW default.mv\n(\n `s` String\n)\nENGINE = Null AS\nSELECT *\nFROM default.log +CREATE MATERIALIZED VIEW default.mv\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date\n)\nENGINE = Null AS\nSELECT *\nFROM default.mt +CREATE LIVE VIEW default.lv\n(\n `1` UInt8\n) AS\nSELECT 1 +CREATE LIVE VIEW default.lv\n(\n `1` UInt8\n) AS\nSELECT 1 diff --git a/tests/queries/0_stateless/01190_full_attach_syntax.sql b/tests/queries/0_stateless/01190_full_attach_syntax.sql new file mode 100644 index 00000000000..3a91eccc8cd --- /dev/null +++ b/tests/queries/0_stateless/01190_full_attach_syntax.sql @@ -0,0 +1,66 @@ +DROP DATABASE IF EXISTS test_01190; +CREATE DATABASE test_01190; + +CREATE TABLE test_01190.table_for_dict (key UInt64, col UInt8) ENGINE = Memory; + +CREATE DICTIONARY test_01190.dict (key UInt64 DEFAULT 0, col UInt8 DEFAULT 1) PRIMARY KEY key SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'test_01190')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); + +SHOW CREATE DICTIONARY test_01190.dict; + +DETACH DICTIONARY test_01190.dict; +ATTACH TABLE test_01190.dict; -- { serverError 80 } +-- Full ATTACH syntax is not allowed for dictionaries +ATTACH DICTIONARY test_01190.dict (key UInt64 DEFAULT 0, col UInt8 DEFAULT 42) PRIMARY KEY key SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'test_01190')) LIFETIME(MIN 1 MAX 100) LAYOUT(FLAT()); -- { clientError 62 } +ATTACH DICTIONARY test_01190.dict; +SHOW CREATE DICTIONARY test_01190.dict; + +DROP DATABASE test_01190; + + +DROP TABLE IF EXISTS log; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS mv; +DROP TABLE IF EXISTS lv; + +CREATE TABLE log ENGINE = Log AS SELECT 'test' AS s; +SHOW CREATE log; +DETACH TABLE log; +ATTACH DICTIONARY log; -- { serverError 487 } +ATTACH TABLE log (s String) ENGINE = Log(); +SHOW CREATE log; +SELECT * FROM log; + +DROP TABLE IF EXISTS mt; +CREATE TABLE mt (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1); +INSERT INTO mt VALUES ([1, 2], 'Hello', 2); +DETACH TABLE mt; +ATTACH TABLE mt (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree ORDER BY (key, s, n) PARTITION BY toYYYYMM(d); -- { serverError 342 } +ATTACH TABLE mt (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1); +SHOW CREATE mt; +SELECT * FROM mt; +DETACH TABLE mt; +ATTACH TABLE mt (key Array(UInt8), s String, n UInt64, d Date) ENGINE = MergeTree(d, (key, s, n), 1); -- It works (with Ordinary database), but probably it shouldn't +SHOW CREATE mt; + +CREATE MATERIALIZED VIEW mv ENGINE = Null AS SELECT * FROM log; +SHOW CREATE mv; +DETACH VIEW mv; +ATTACH MATERIALIZED VIEW mv ENGINE = Null AS SELECT * FROM log; +SHOW CREATE mv; +DETACH VIEW mv; +ATTACH MATERIALIZED VIEW mv ENGINE = Null AS SELECT * FROM mt; -- It works (with Ordinary database), but probably it shouldn't +SHOW CREATE mv; + +SET allow_experimental_live_view = 1; +CREATE LIVE VIEW lv AS SELECT 1; +SHOW CREATE lv; +DETACH VIEW lv; +ATTACH LIVE VIEW lv AS SELECT 1; +SHOW CREATE lv; + +DROP TABLE log; +DROP TABLE mt; +DROP TABLE mv; +DROP TABLE lv; + + diff --git a/tests/queries/0_stateless/01305_replica_create_drop_zookeeper.sh b/tests/queries/0_stateless/01305_replica_create_drop_zookeeper.sh index 0a47c6df46c..1313830d589 100755 --- a/tests/queries/0_stateless/01305_replica_create_drop_zookeeper.sh +++ b/tests/queries/0_stateless/01305_replica_create_drop_zookeeper.sh @@ -7,11 +7,21 @@ set -e function thread() { - while true; do - $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS test_table_$1; - CREATE TABLE test_table_$1 (a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01305/alter_table', 'r_$1') ORDER BY tuple();" 2>&1 | - grep -vP '(^$)|(^Received exception from server)|(^\d+\. )|because the last replica of the table was dropped right now|is already started to be removing by another replica right now|is already finished removing by another replica right now|Removing leftovers from table|Another replica was suddenly created|was successfully removed from ZooKeeper|was created by another server at the same moment|was suddenly removed|some other replicas were created at the same time' - done + db_engine=`$CLICKHOUSE_CLIENT -q "SELECT engine FROM system.databases WHERE name=currentDatabase()"` + if [[ $db_engine == "Atomic" ]]; then + # Ignore "Replica already exists" exception + while true; do + $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS test_table_$1 NO DELAY; + CREATE TABLE test_table_$1 (a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01305/alter_table', 'r_$1') ORDER BY tuple();" 2>&1 | + grep -vP '(^$)|(^Received exception from server)|(^\d+\. )|because the last replica of the table was dropped right now|is already started to be removing by another replica right now|is already finished removing by another replica right now|Removing leftovers from table|Another replica was suddenly created|was successfully removed from ZooKeeper|was created by another server at the same moment|was suddenly removed|some other replicas were created at the same time|already exists' + done + else + while true; do + $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS test_table_$1; + CREATE TABLE test_table_$1 (a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01305/alter_table', 'r_$1') ORDER BY tuple();" 2>&1 | + grep -vP '(^$)|(^Received exception from server)|(^\d+\. )|because the last replica of the table was dropped right now|is already started to be removing by another replica right now|is already finished removing by another replica right now|Removing leftovers from table|Another replica was suddenly created|was successfully removed from ZooKeeper|was created by another server at the same moment|was suddenly removed|some other replicas were created at the same time' + done + fi } diff --git a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql index 15a2a75cf58..515a2410583 100644 --- a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql +++ b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql @@ -22,7 +22,7 @@ GROUP BY loyalty ORDER BY loyalty ASC; DETACH TABLE join; -ATTACH TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(SEMI, LEFT, UserID); +ATTACH TABLE join; SELECT loyalty, diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index adfc5f0e582..efd622402b2 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -3,10 +3,8 @@ */ { "thread-sanitizer": [ - "00281", "00877", "00985", - "avx2", "query_profiler", "memory_profiler", /// 01083 and 00505 and 00505 are critical and temproray disabled @@ -21,9 +19,7 @@ "01193_metadata_loading" ], "address-sanitizer": [ - "00281", "00877", - "avx2", "query_profiler", "memory_profiler", "odbc_roundtrip", @@ -31,9 +27,7 @@ "01193_metadata_loading" ], "ub-sanitizer": [ - "00281", "capnproto", - "avx2", "query_profiler", "memory_profiler", "01103_check_cpu_instructions_at_startup", @@ -41,9 +35,7 @@ "01193_metadata_loading" ], "memory-sanitizer": [ - "00281", "capnproto", - "avx2", "query_profiler", "memory_profiler", "01103_check_cpu_instructions_at_startup", @@ -53,8 +45,6 @@ "01193_metadata_loading" ], "debug-build": [ - "00281", - "avx2", "query_profiler", "memory_profiler", "00899_long_attach", @@ -70,12 +60,10 @@ ], "unbundled-build": [ "00429", - "00428", "00877", "pocopatch", "parquet", "xxhash", - "avx2", "_h3", "query_profiler", "memory_profiler", @@ -98,33 +86,19 @@ "01455_time_zones" ], "release-build": [ - "avx2" ], "database-atomic": [ - "00065_loyalty_with_storage_join", - "avx", /// Inner tables of materialized views have different names "00738_lock_for_inner_table", - "00699_materialized_view_mutations", "00609_mv_index_in_in", "00510_materizlized_view_and_deduplication_zookeeper", - /// Create queries contain UUID + /// Different database engine "00604_show_create_database", - "00080_show_tables_and_system_tables", - "01272_suspicious_codecs", /// UUID must be specified in ATTACH TABLE - "01249_bad_arguments_for_bloom_filter", - "00423_storage_log_single_thread", - "00311_array_primary_key", - "00226_zookeeper_deduplication_and_unexpected_parts", - "00180_attach_materialized_view", - "00116_storage_set", + "01190_full_attach_syntax", /// Assumes blocking DROP - "00816_long_concurrent_alter_column", - "00992_system_parts_race_condition_zookeeper", /// FIXME "01320_create_sync_race_condition", - "01305_replica_create_drop_zookeeper", - "01130_in_memory_parts_partitons", + /// Internal distionary name is different "01225_show_create_table_from_dictionary", "01224_no_superfluous_dict_reload" ], @@ -132,7 +106,6 @@ /// These tests fail with compact parts, because they /// check some implementation defined things /// like checksums, computed granularity, ProfileEvents, etc. - "avx", "01045_order_by_pk_special_storages", "01042_check_query_and_last_granule_size", "00961_checksums_in_system_parts_columns_table", From 83ec93dec868a1e17950f6298b5ec1ce0d5352db Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 9 Sep 2020 16:46:59 +0300 Subject: [PATCH 081/341] Fuzzer: reset default database before reconnect --- programs/client/Client.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index c9701950dc5..99598c70397 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -866,6 +866,8 @@ private: // will exit. The ping() would be the best match here, but it's // private, probably for a good reason that the protocol doesn't allow // pings at any possible moment. + // Don't forget to reset the default database which might have changed. + connection->setDefaultDatabase(""); connection->forceConnected(connection_parameters.timeouts); if (text.size() > 4 * 1024) @@ -1103,7 +1105,9 @@ private: { last_exception_received_from_server = std::make_unique(getCurrentExceptionMessage(true), getCurrentExceptionCode()); received_exception_from_server = true; - std::cerr << "Error on processing query: " << ast_to_process->formatForErrorMessage() << std::endl << last_exception_received_from_server->message(); + fmt::print(stderr, "Error on processing query '{}': {}\n", + ast_to_process->formatForErrorMessage(), + last_exception_received_from_server->message()); } if (!connection->isConnected()) From 50dee3f4493d7ffb2c75d195cc39862f8f8d8a86 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 17:43:17 +0300 Subject: [PATCH 082/341] Remove false-positive warning --- src/Storages/MergeTree/MergeTreePartition.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 4a846f63b7c..8ef3e458871 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -29,6 +29,9 @@ String MergeTreePartition::getID(const MergeTreeData & storage) const return getID(storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block); } +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wstringop-overflow" + /// NOTE: This ID is used to create part names which are then persisted in ZK and as directory names on the file system. /// So if you want to change this method, be sure to guarantee compatibility with existing table data. String MergeTreePartition::getID(const Block & partition_key_sample) const @@ -87,6 +90,8 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const return result; } +#pragma GCC diagnostic pop + void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); From 673244876ea15008b54bf93e9d535a0a65e68696 Mon Sep 17 00:00:00 2001 From: Mikhail Cheshkov Date: Wed, 9 Sep 2020 17:57:15 +0300 Subject: [PATCH 083/341] Use global ICU ADDINCL for Arcadia build --- src/Columns/ya.make | 2 -- src/Functions/ya.make | 1 - src/Functions/ya.make.in | 1 - 3 files changed, 4 deletions(-) diff --git a/src/Columns/ya.make b/src/Columns/ya.make index 78c0e1b992d..910c479c2a9 100644 --- a/src/Columns/ya.make +++ b/src/Columns/ya.make @@ -2,8 +2,6 @@ LIBRARY() ADDINCL( - contrib/libs/icu/common - contrib/libs/icu/i18n contrib/libs/pdqsort ) diff --git a/src/Functions/ya.make b/src/Functions/ya.make index b9a7b5b64ea..f48b4d607ed 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -10,7 +10,6 @@ ADDINCL( contrib/libs/farmhash contrib/libs/h3/h3lib/include contrib/libs/hyperscan/src - contrib/libs/icu/common contrib/libs/libdivide contrib/libs/rapidjson/include contrib/libs/xxhash diff --git a/src/Functions/ya.make.in b/src/Functions/ya.make.in index a5a54d94c6c..2a66aa5553e 100644 --- a/src/Functions/ya.make.in +++ b/src/Functions/ya.make.in @@ -9,7 +9,6 @@ ADDINCL( contrib/libs/farmhash contrib/libs/h3/h3lib/include contrib/libs/hyperscan/src - contrib/libs/icu/common contrib/libs/libdivide contrib/libs/rapidjson/include contrib/libs/xxhash From 7f4106687cb14491246f218654ed8a0a3b751b29 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 9 Sep 2020 19:23:31 +0300 Subject: [PATCH 084/341] fix --- tests/queries/0_stateless/01114_database_atomic.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01114_database_atomic.reference b/tests/queries/0_stateless/01114_database_atomic.reference index 7980819f9af..a79784230a6 100644 --- a/tests/queries/0_stateless/01114_database_atomic.reference +++ b/tests/queries/0_stateless/01114_database_atomic.reference @@ -7,7 +7,7 @@ test_01114_3 Ordinary test_01114_3 test_01114_3 1 20 100 CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -mt 00001114-0000-4000-8000-000000000002 CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\' (`n` UInt64) ENGINE = MergeTree() PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192 +mt 00001114-0000-4000-8000-000000000002 CREATE TABLE test_01114_2.mt (`n` UInt64) ENGINE = MergeTree() PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192 20 CREATE TABLE test_01114_1.mt UUID \'00001114-0000-4000-8000-000000000001\'\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 From d1d3af7501a35629d48b046387dfb95e0731f657 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Wed, 9 Sep 2020 00:22:24 -0700 Subject: [PATCH 085/341] StorageReplicatedMergeTree - more improvements to integration test --- .../__init__.py | 0 .../test.py | 24 +++++++------------ 2 files changed, 9 insertions(+), 15 deletions(-) rename tests/integration/{test_replicated_zk_conn_failure => test_cleanup_dir_after_bad_zk_conn}/__init__.py (100%) rename tests/integration/{test_replicated_zk_conn_failure => test_cleanup_dir_after_bad_zk_conn}/test.py (63%) diff --git a/tests/integration/test_replicated_zk_conn_failure/__init__.py b/tests/integration/test_cleanup_dir_after_bad_zk_conn/__init__.py similarity index 100% rename from tests/integration/test_replicated_zk_conn_failure/__init__.py rename to tests/integration/test_cleanup_dir_after_bad_zk_conn/__init__.py diff --git a/tests/integration/test_replicated_zk_conn_failure/test.py b/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py similarity index 63% rename from tests/integration/test_replicated_zk_conn_failure/test.py rename to tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py index a860716ee82..ef31e8487be 100644 --- a/tests/integration/test_replicated_zk_conn_failure/test.py +++ b/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py @@ -9,12 +9,12 @@ from helpers.network import PartitionManager # Test flow is as follows: # 1. Configure cluster with ZooKeeper and create a database. # 2. Drop all connections to ZooKeeper. -# 3. Try creating the table and there would be a Poco:Exception. +# 3. Try creating the table and there will be a Poco:Exception. # 4. Try creating the table again and there should not be any error -# that indicates that the Directory for table already exists. +# that indicates that the directory for table already exists. # 5. Final step is to restore ZooKeeper connection and verify that -# the table creation and queries work. -def test_replicated_zk_conn_failure(): +# the table creation works. +def test_cleanup_dir_after_bad_zk_conn(): cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', with_zookeeper=True) try: @@ -30,21 +30,15 @@ def test_replicated_zk_conn_failure(): ORDER BY id;''' with PartitionManager() as pm: pm.drop_instance_zk_connections(node1) - time.sleep(5) + time.sleep(3) error = node1.query_and_get_error(query_create) - # Assert that there was net exception. - assert "Poco::Exception. Code: 1000" in error - # Assert that the exception was due to ZooKeeper connectivity. - assert "All connection tries failed while connecting to ZooKeeper" in error - # retry table creation + assert "Poco::Exception. Code: 1000" and \ + "All connection tries failed while connecting to ZooKeeper" in error error = node1.query_and_get_error(query_create) - # Should not expect any errors related to directory already existing - # and those should have been already cleaned up during the previous retry. assert "Directory for table data data/replica/test/ already exists" not in error - # restore ZooKeeper connections. pm.restore_instance_zk_connections(node1) - # retry create query and query the table created. node1.query(query_create) - assert "0\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') + node1.query('''INSERT INTO replica.test VALUES (1, now())''') + assert "1\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') finally: cluster.shutdown() From 62428845a0fdcaaa19ecc5fd33f3ecd849104cf5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 19:47:06 +0300 Subject: [PATCH 086/341] Bug in mutation --- src/Columns/ColumnVector.h | 7 ++++--- .../0_stateless/01475_mutation_with_if.reference | 1 + .../0_stateless/01475_mutation_with_if.sql | 16 ++++++++++++++++ 3 files changed, 21 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01475_mutation_with_if.reference create mode 100644 tests/queries/0_stateless/01475_mutation_with_if.sql diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 1090de556a0..55ab67d6214 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -130,7 +131,7 @@ public: void insertFrom(const IColumn & src, size_t n) override { - data.push_back(static_cast(src).getData()[n]); + data.push_back(assert_cast(src).getData()[n]); } void insertData(const char * pos, size_t) override @@ -205,14 +206,14 @@ public: /// This method implemented in header because it could be possibly devirtualized. int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override { - return CompareHelper::compare(data[n], static_cast(rhs_).data[m], nan_direction_hint); + return CompareHelper::compare(data[n], assert_cast(rhs_).data[m], nan_direction_hint); } void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override { - return this->template doCompareColumn(static_cast(rhs), rhs_row_num, row_indexes, + return this->template doCompareColumn(assert_cast(rhs), rhs_row_num, row_indexes, compare_results, direction, nan_direction_hint); } diff --git a/tests/queries/0_stateless/01475_mutation_with_if.reference b/tests/queries/0_stateless/01475_mutation_with_if.reference new file mode 100644 index 00000000000..2874a18147f --- /dev/null +++ b/tests/queries/0_stateless/01475_mutation_with_if.reference @@ -0,0 +1 @@ +1 150 diff --git a/tests/queries/0_stateless/01475_mutation_with_if.sql b/tests/queries/0_stateless/01475_mutation_with_if.sql new file mode 100644 index 00000000000..6f0ef8924be --- /dev/null +++ b/tests/queries/0_stateless/01475_mutation_with_if.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS mutation_table; +CREATE TABLE mutation_table ( + id int, + price Nullable(Int32) +) +ENGINE = MergeTree() +PARTITION BY id +ORDER BY id; + +INSERT INTO mutation_table (id, price) VALUES (1, 100); + +ALTER TABLE mutation_table UPDATE price = 150 WHERE id = 1 SETTINGS mutations_sync = 2; + +SELECT * FROM mutation_table; + +DROP TABLE IF EXISTS mutation_table; From 9fa04cf48b1f2aa9288dbd026e37b7aae9e8a6f3 Mon Sep 17 00:00:00 2001 From: Peng Jian Date: Thu, 10 Sep 2020 00:59:38 +0800 Subject: [PATCH 087/341] Add QueryMemoryLimitExceeded event --- src/Common/MemoryTracker.cpp | 7 +++++++ src/Common/ProfileEvents.cpp | 1 + 2 files changed, 8 insertions(+) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 9d073cf8dd8..5d51fc9f301 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -22,6 +23,10 @@ namespace DB } } +namespace ProfileEvents +{ + extern const Event QueryMemoryLimitExceeded; +} static constexpr size_t log_peak_memory_usage_every = 1ULL << 30; @@ -104,6 +109,7 @@ void MemoryTracker::alloc(Int64 size) /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc auto untrack_lock = blocker.cancel(); // NOLINT + ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); std::stringstream message; message << "Memory tracker"; if (const auto * description = description_ptr.load(std::memory_order_relaxed)) @@ -136,6 +142,7 @@ void MemoryTracker::alloc(Int64 size) /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc auto no_track = blocker.cancel(); // NOLINT + ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); std::stringstream message; message << "Memory limit"; if (const auto * description = description_ptr.load(std::memory_order_relaxed)) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 475e073d253..c9ff9642361 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -233,6 +233,7 @@ M(S3WriteRequestsErrors, "Number of non-throttling errors in POST, DELETE, PUT and PATCH requests to S3 storage.") \ M(S3WriteRequestsThrottling, "Number of 429 and 503 errors in POST, DELETE, PUT and PATCH requests to S3 storage.") \ M(S3WriteRequestsRedirects, "Number of redirects in POST, DELETE, PUT and PATCH requests to S3 storage.") \ + M(QueryMemoryLimitExceeded, "Number of times of memory limit exceeded for query.") \ namespace ProfileEvents From dee1fefeb472bebdfa4e15eb1b7ec7e428069f8a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 9 Sep 2020 20:39:49 +0300 Subject: [PATCH 088/341] add more tests --- src/Storages/StorageReplicatedMergeTree.cpp | 5 +- .../test.py | 74 ++++++++++++------- 2 files changed, 50 insertions(+), 29 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a1027a8be53..00dcc7aeb08 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -612,7 +612,10 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr return true; } - throw Exception("Cannot create table, because it is created concurrently every time or because of logical error", ErrorCodes::LOGICAL_ERROR); + /// Do not use LOGICAL_ERROR code, because it may happen if user has specified wrong zookeeper_path + throw Exception("Cannot create table, because it is created concurrently every time " + "or because of wrong zookeeper_path " + "or because of logical error", ErrorCodes::REPLICA_IS_ALREADY_EXIST); } void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metadata_snapshot) diff --git a/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py b/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py index ef31e8487be..4cb243160a0 100644 --- a/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py +++ b/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py @@ -1,9 +1,21 @@ import time +import pytest from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', with_zookeeper=True) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + # This tests if the data directory for a table is cleaned up if there is a Zookeeper # connection exception during a CreateQuery operation involving ReplicatedMergeTree tables. # Test flow is as follows: @@ -14,31 +26,37 @@ from helpers.network import PartitionManager # that indicates that the directory for table already exists. # 5. Final step is to restore ZooKeeper connection and verify that # the table creation works. -def test_cleanup_dir_after_bad_zk_conn(): - cluster = ClickHouseCluster(__file__) - node1 = cluster.add_instance('node1', with_zookeeper=True) - try: - cluster.start() - node1.query("CREATE DATABASE replica;") - query_create = '''CREATE TABLE replica.test - ( - id Int64, - event_time DateTime - ) - Engine=ReplicatedMergeTree('/clickhouse/tables/replica/test', 'node1') - PARTITION BY toYYYYMMDD(event_time) - ORDER BY id;''' - with PartitionManager() as pm: - pm.drop_instance_zk_connections(node1) - time.sleep(3) - error = node1.query_and_get_error(query_create) - assert "Poco::Exception. Code: 1000" and \ - "All connection tries failed while connecting to ZooKeeper" in error - error = node1.query_and_get_error(query_create) - assert "Directory for table data data/replica/test/ already exists" not in error - pm.restore_instance_zk_connections(node1) - node1.query(query_create) - node1.query('''INSERT INTO replica.test VALUES (1, now())''') - assert "1\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') - finally: - cluster.shutdown() +def test_cleanup_dir_after_bad_zk_conn(start_cluster): + node1.query("CREATE DATABASE replica;") + query_create = '''CREATE TABLE replica.test + ( + id Int64, + event_time DateTime + ) + Engine=ReplicatedMergeTree('/clickhouse/tables/replica/test', 'node1') + PARTITION BY toYYYYMMDD(event_time) + ORDER BY id;''' + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node1) + time.sleep(3) + error = node1.query_and_get_error(query_create) + assert "Poco::Exception. Code: 1000" and \ + "All connection tries failed while connecting to ZooKeeper" in error + error = node1.query_and_get_error(query_create) + assert "Directory for table data data/replica/test/ already exists" not in error + node1.query(query_create) + node1.query('''INSERT INTO replica.test VALUES (1, now())''') + assert "1\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') + +def test_cleanup_dir_after_wrong_replica_name(start_cluster): + node1.query("CREATE TABLE test2_r1 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test2/', 'r1') ORDER BY n") + error = node1.query_and_get_error("CREATE TABLE test2_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test2/', 'r1') ORDER BY n") + assert "already exists" in error + node1.query("CREATE TABLE test_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test2/', 'r2') ORDER BY n") + + +def test_cleanup_dir_after_wrong_zk_path(start_cluster): + node1.query("CREATE TABLE test3_r1 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test3/', 'r1') ORDER BY n") + error = node1.query_and_get_error("CREATE TABLE test3_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/', 'r2') ORDER BY n") + assert "Cannot create" in error + node1.query("CREATE TABLE test3_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test3/', 'r2') ORDER BY n") From 7304bad56fff15882c82a556a4bd8b197c65092b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Sep 2020 20:51:05 +0300 Subject: [PATCH 089/341] Support for nested multiline comments --- src/Parsers/Lexer.cpp | 20 +++++++++++++++---- .../01491_nested_multiline_comments.reference | 3 +++ .../01491_nested_multiline_comments.sql | 3 +++ 3 files changed, 22 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/01491_nested_multiline_comments.reference create mode 100644 tests/queries/0_stateless/01491_nested_multiline_comments.sql diff --git a/src/Parsers/Lexer.cpp b/src/Parsers/Lexer.cpp index baf547a0a1a..ffa8250a3f3 100644 --- a/src/Parsers/Lexer.cpp +++ b/src/Parsers/Lexer.cpp @@ -253,15 +253,27 @@ Token Lexer::nextTokenImpl() else { ++pos; + + /// Nested multiline comments are supported according to the SQL standard. + size_t nesting_level = 1; + while (pos + 2 <= end) { - /// This means that nested multiline comments are not supported. - if (pos[0] == '*' && pos[1] == '/') + if (pos[0] == '/' && pos[1] == '*') { pos += 2; - return Token(TokenType::Comment, token_begin, pos); + ++nesting_level; } - ++pos; + else if (pos[0] == '*' && pos[1] == '/') + { + pos += 2; + --nesting_level; + + if (nesting_level == 0) + return Token(TokenType::Comment, token_begin, pos); + } + else + ++pos; } return Token(TokenType::ErrorMultilineCommentIsNotClosed, token_begin, end); } diff --git a/tests/queries/0_stateless/01491_nested_multiline_comments.reference b/tests/queries/0_stateless/01491_nested_multiline_comments.reference new file mode 100644 index 00000000000..e8183f05f5d --- /dev/null +++ b/tests/queries/0_stateless/01491_nested_multiline_comments.reference @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/tests/queries/0_stateless/01491_nested_multiline_comments.sql b/tests/queries/0_stateless/01491_nested_multiline_comments.sql new file mode 100644 index 00000000000..4c6f7634701 --- /dev/null +++ b/tests/queries/0_stateless/01491_nested_multiline_comments.sql @@ -0,0 +1,3 @@ +SELECT /*/**/*/ 1; +SELECT /*a/*b*/c*/ 1; +SELECT /*ab/*cd*/ef*/ 1; From 453914b6b860dc558719269ab9af35828376fac7 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 9 Sep 2020 21:02:53 +0300 Subject: [PATCH 090/341] Update ProfileEvents.cpp --- src/Common/ProfileEvents.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index c9ff9642361..486cb7e1a6e 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -233,7 +233,7 @@ M(S3WriteRequestsErrors, "Number of non-throttling errors in POST, DELETE, PUT and PATCH requests to S3 storage.") \ M(S3WriteRequestsThrottling, "Number of 429 and 503 errors in POST, DELETE, PUT and PATCH requests to S3 storage.") \ M(S3WriteRequestsRedirects, "Number of redirects in POST, DELETE, PUT and PATCH requests to S3 storage.") \ - M(QueryMemoryLimitExceeded, "Number of times of memory limit exceeded for query.") \ + M(QueryMemoryLimitExceeded, "Number of times when memory limit exceeded for query.") \ namespace ProfileEvents From 11ffbda7cc48082ec0c3a44dc34946cf03a9dc4a Mon Sep 17 00:00:00 2001 From: myrrc Date: Wed, 9 Sep 2020 21:17:01 +0300 Subject: [PATCH 091/341] added the debugger info option --- src/Functions/CMakeLists.txt | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 78caabb6941..08eefec84d4 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -53,8 +53,18 @@ endif() target_include_directories(clickhouse_functions SYSTEM PRIVATE ${SPARSEHASH_INCLUDE_DIR}) -# Won't generate debug info for files with heavy template instantiation to achieve faster linking and lower size. -target_compile_options(clickhouse_functions PRIVATE "-g0") +option(STRIP_DEBUG_SYMBOLS_FUNCTIONS + "Do not generate debugger info for ClickHouse functions. + Provides faster linking and lower binary size. + Tradeoff is the inability to debug some source files with e.g. gdb + (empty stack frames and no local variables)." OFF) + +if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) + message(WARNING "Not generating debugger info for ClickHouse functions") + target_compile_options(clickhouse_functions PRIVATE "-g0") +else() + message(STATUS "Generating debugger info for ClickHouse functions") +endif() if (USE_ICU) target_link_libraries (clickhouse_functions PRIVATE ${ICU_LIBRARIES}) From 2a9ab482792cdadf0d4e2365c3d11494a3e38230 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 8 Sep 2020 02:08:17 +0300 Subject: [PATCH 092/341] Use join() instead of detach() for the no_users_thread in StorageLiveView. --- src/Interpreters/Context.cpp | 8 + src/Interpreters/Context.h | 8 +- src/Interpreters/InterpreterDropQuery.h | 1 + .../LiveView/LiveViewBlockInputStream.h | 15 +- .../LiveView/LiveViewEventsBlockInputStream.h | 14 +- src/Storages/LiveView/StorageLiveView.cpp | 144 +---------------- src/Storages/LiveView/StorageLiveView.h | 23 ++- .../LiveView/TemporaryLiveViewCleaner.cpp | 148 ++++++++++++++++++ .../LiveView/TemporaryLiveViewCleaner.h | 51 ++++++ src/Storages/ya.make | 1 + 10 files changed, 233 insertions(+), 180 deletions(-) create mode 100644 src/Storages/LiveView/TemporaryLiveViewCleaner.cpp create mode 100644 src/Storages/LiveView/TemporaryLiveViewCleaner.h diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 70cf41a679c..3c4c095cc26 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -423,6 +424,7 @@ struct ContextShared if (system_logs) system_logs->shutdown(); + TemporaryLiveViewCleaner::shutdown(); DatabaseCatalog::shutdown(); /// Preemptive destruction is important, because these objects may have a refcount to ContextShared (cyclic reference). @@ -479,6 +481,12 @@ Context Context::createGlobal(ContextShared * shared) return res; } +void Context::initGlobal() +{ + DatabaseCatalog::init(this); + TemporaryLiveViewCleaner::init(*this); +} + SharedContextHolder Context::createShared() { return SharedContextHolder(std::make_unique()); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c8d13baa9ae..743c92d56b5 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -445,11 +445,7 @@ public: void makeQueryContext() { query_context = this; } void makeSessionContext() { session_context = this; } - void makeGlobalContext() - { - global_context = this; - DatabaseCatalog::init(this); - } + void makeGlobalContext() { initGlobal(); global_context = this; } const Settings & getSettingsRef() const { return settings; } @@ -622,6 +618,8 @@ public: private: std::unique_lock getLock() const; + void initGlobal(); + /// Compute and set actual user settings, client_info.current_user should be set void calculateAccessRights(); diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index 80bd6c6531a..b54736b5c21 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -10,6 +10,7 @@ namespace DB { class Context; using DatabaseAndTable = std::pair; +class AccessRightsElements; /** Allow to either drop table with all its data (DROP), * or remove information about table (just forget) from server (DETACH), diff --git a/src/Storages/LiveView/LiveViewBlockInputStream.h b/src/Storages/LiveView/LiveViewBlockInputStream.h index 7cab2cb41ed..737e76754c5 100644 --- a/src/Storages/LiveView/LiveViewBlockInputStream.h +++ b/src/Storages/LiveView/LiveViewBlockInputStream.h @@ -16,27 +16,17 @@ class LiveViewBlockInputStream : public IBlockInputStream using NonBlockingResult = std::pair; public: - ~LiveViewBlockInputStream() override - { - /// Start storage no users thread - /// if we are the last active user - if (!storage->is_dropped && blocks_ptr.use_count() < 3) - storage->startNoUsersThread(temporary_live_view_timeout_sec); - } - LiveViewBlockInputStream(std::shared_ptr storage_, std::shared_ptr blocks_ptr_, std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, const bool has_limit_, const UInt64 limit_, - const UInt64 heartbeat_interval_sec_, - const UInt64 temporary_live_view_timeout_sec_) + const UInt64 heartbeat_interval_sec_) : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), active_ptr(std::move(active_ptr_)), has_limit(has_limit_), limit(limit_), - heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000), - temporary_live_view_timeout_sec(temporary_live_view_timeout_sec_) + heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000) { /// grab active pointer active = active_ptr.lock(); @@ -205,7 +195,6 @@ private: Int64 num_updates = -1; bool end_of_blocks = false; UInt64 heartbeat_interval_usec; - UInt64 temporary_live_view_timeout_sec; UInt64 last_event_timestamp_usec = 0; }; diff --git a/src/Storages/LiveView/LiveViewEventsBlockInputStream.h b/src/Storages/LiveView/LiveViewEventsBlockInputStream.h index ac5e7e3d6fd..4060b17c1ed 100644 --- a/src/Storages/LiveView/LiveViewEventsBlockInputStream.h +++ b/src/Storages/LiveView/LiveViewEventsBlockInputStream.h @@ -34,13 +34,6 @@ class LiveViewEventsBlockInputStream : public IBlockInputStream using NonBlockingResult = std::pair; public: - ~LiveViewEventsBlockInputStream() override - { - /// Start storage no users thread - /// if we are the last active user - if (!storage->is_dropped && blocks_ptr.use_count() < 3) - storage->startNoUsersThread(temporary_live_view_timeout_sec); - } /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update /// and LIMIT 0 just returns data without waiting for any updates LiveViewEventsBlockInputStream(std::shared_ptr storage_, @@ -48,14 +41,12 @@ public: std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, const bool has_limit_, const UInt64 limit_, - const UInt64 heartbeat_interval_sec_, - const UInt64 temporary_live_view_timeout_sec_) + const UInt64 heartbeat_interval_sec_) : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), active_ptr(std::move(active_ptr_)), has_limit(has_limit_), limit(limit_), - heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000), - temporary_live_view_timeout_sec(temporary_live_view_timeout_sec_) + heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000) { /// grab active pointer active = active_ptr.lock(); @@ -236,7 +227,6 @@ private: Int64 num_updates = -1; bool end_of_blocks = false; UInt64 heartbeat_interval_usec; - UInt64 temporary_live_view_timeout_sec; UInt64 last_event_timestamp_usec = 0; Poco::Timestamp timestamp; }; diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 54ac5bcc791..b16c02eec6b 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -12,10 +12,8 @@ limitations under the License. */ #include #include #include -#include #include #include -#include #include #include #include @@ -31,6 +29,7 @@ limitations under the License. */ #include #include #include +#include #include #include @@ -276,7 +275,7 @@ StorageLiveView::StorageLiveView( if (query.live_view_timeout) { is_temporary = true; - temporary_live_view_timeout = *query.live_view_timeout; + temporary_live_view_timeout = std::chrono::seconds{*query.live_view_timeout}; } blocks_ptr = std::make_shared(); @@ -384,128 +383,21 @@ void StorageLiveView::checkTableCanBeDropped() const } } -void StorageLiveView::noUsersThread(std::shared_ptr storage, const UInt64 & timeout) -{ - bool drop_table = false; - - if (storage->shutdown_called) - return; - - auto table_id = storage->getStorageID(); - { - while (true) - { - std::unique_lock lock(storage->no_users_thread_wakeup_mutex); - if (!storage->no_users_thread_condition.wait_for(lock, std::chrono::seconds(timeout), [&] { return storage->no_users_thread_wakeup; })) - { - storage->no_users_thread_wakeup = false; - if (storage->shutdown_called) - return; - if (storage->hasUsers()) - return; - if (!DatabaseCatalog::instance().getDependencies(table_id).empty()) - continue; - drop_table = true; - } - break; - } - } - - if (drop_table) - { - if (DatabaseCatalog::instance().tryGetTable(table_id, storage->global_context)) - { - try - { - /// We create and execute `drop` query for this table - auto drop_query = std::make_shared(); - drop_query->database = table_id.database_name; - drop_query->table = table_id.table_name; - drop_query->kind = ASTDropQuery::Kind::Drop; - ASTPtr ast_drop_query = drop_query; - InterpreterDropQuery drop_interpreter(ast_drop_query, storage->global_context); - drop_interpreter.execute(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - } -} - -void StorageLiveView::startNoUsersThread(const UInt64 & timeout) -{ - bool expected = false; - if (!start_no_users_thread_called.compare_exchange_strong(expected, true)) - return; - - if (is_temporary) - { - std::lock_guard no_users_thread_lock(no_users_thread_mutex); - - if (shutdown_called) - return; - - if (no_users_thread.joinable()) - { - { - std::lock_guard lock(no_users_thread_wakeup_mutex); - no_users_thread_wakeup = true; - no_users_thread_condition.notify_one(); - } - no_users_thread.join(); - } - { - std::lock_guard lock(no_users_thread_wakeup_mutex); - no_users_thread_wakeup = false; - } - if (!is_dropped) - no_users_thread = std::thread(&StorageLiveView::noUsersThread, - std::static_pointer_cast(shared_from_this()), timeout); - } - - start_no_users_thread_called = false; -} - void StorageLiveView::startup() { - startNoUsersThread(temporary_live_view_timeout); + if (is_temporary) + TemporaryLiveViewCleaner::instance().addView(std::static_pointer_cast(shared_from_this())); } void StorageLiveView::shutdown() { + shutdown_called = true; DatabaseCatalog::instance().removeDependency(select_table_id, getStorageID()); - bool expected = false; - if (!shutdown_called.compare_exchange_strong(expected, true)) - return; - - /// WATCH queries should be stopped after setting shutdown_called to true. - /// Otherwise livelock is possible for LiveView table in Atomic database: - /// WATCH query will wait for table to be dropped and DatabaseCatalog will wait for queries to finish - - { - std::lock_guard no_users_thread_lock(no_users_thread_mutex); - if (no_users_thread.joinable()) - { - { - std::lock_guard lock(no_users_thread_wakeup_mutex); - no_users_thread_wakeup = true; - no_users_thread_condition.notify_one(); - } - } - } } StorageLiveView::~StorageLiveView() { shutdown(); - - { - std::lock_guard lock(no_users_thread_mutex); - if (no_users_thread.joinable()) - no_users_thread.detach(); - } } void StorageLiveView::drop() @@ -572,18 +464,7 @@ BlockInputStreams StorageLiveView::watch( auto reader = std::make_shared( std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, - context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), - temporary_live_view_timeout); - - { - std::lock_guard no_users_thread_lock(no_users_thread_mutex); - if (no_users_thread.joinable()) - { - std::lock_guard lock(no_users_thread_wakeup_mutex); - no_users_thread_wakeup = true; - no_users_thread_condition.notify_one(); - } - } + context.getSettingsRef().live_view_heartbeat_interval.totalSeconds()); { std::lock_guard lock(mutex); @@ -603,18 +484,7 @@ BlockInputStreams StorageLiveView::watch( auto reader = std::make_shared( std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, - context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), - temporary_live_view_timeout); - - { - std::lock_guard no_users_thread_lock(no_users_thread_mutex); - if (no_users_thread.joinable()) - { - std::lock_guard lock(no_users_thread_wakeup_mutex); - no_users_thread_wakeup = true; - no_users_thread_condition.notify_one(); - } - } + context.getSettingsRef().live_view_heartbeat_interval.totalSeconds()); { std::lock_guard lock(mutex); diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 43afd169a92..32e18ef6092 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -38,6 +38,10 @@ using ASTPtr = std::shared_ptr; using BlocksMetadataPtr = std::shared_ptr; using MergeableBlocksPtr = std::shared_ptr; +class Pipe; +using Pipes = std::vector; + + class StorageLiveView final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; @@ -70,7 +74,9 @@ public: NamesAndTypesList getVirtuals() const override; - bool isTemporary() { return is_temporary; } + bool isTemporary() const { return is_temporary; } + std::chrono::seconds getTimeout() const { return temporary_live_view_timeout; } + /// Check if we have any readers /// must be called with mutex locked @@ -85,11 +91,7 @@ public: { return active_ptr.use_count() > 1; } - /// No users thread mutex, predicate and wake up condition - void startNoUsersThread(const UInt64 & timeout); - std::mutex no_users_thread_wakeup_mutex; - bool no_users_thread_wakeup = false; - std::condition_variable no_users_thread_condition; + /// Get blocks hash /// must be called with mutex locked String getBlocksHashKey() @@ -175,6 +177,8 @@ private: std::unique_ptr live_view_context; bool is_temporary = false; + std::chrono::seconds temporary_live_view_timeout; + /// Mutex to protect access to sample block and inner_blocks_query mutable std::mutex sample_block_lock; mutable Block sample_block; @@ -193,14 +197,7 @@ private: std::shared_ptr blocks_metadata_ptr; MergeableBlocksPtr mergeable_blocks; - /// Background thread for temporary tables - /// which drops this table if there are no users - static void noUsersThread(std::shared_ptr storage, const UInt64 & timeout); - std::mutex no_users_thread_mutex; - std::thread no_users_thread; std::atomic shutdown_called = false; - std::atomic start_no_users_thread_called = false; - UInt64 temporary_live_view_timeout; StorageLiveView( const StorageID & table_id_, diff --git a/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp b/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp new file mode 100644 index 00000000000..0f7c1039d72 --- /dev/null +++ b/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp @@ -0,0 +1,148 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + + +namespace +{ + void executeDropQuery(const StorageID & storage_id, Context & context) + { + if (!DatabaseCatalog::instance().isTableExist(storage_id, context)) + return; + try + { + /// We create and execute `drop` query for this table + auto drop_query = std::make_shared(); + drop_query->database = storage_id.database_name; + drop_query->table = storage_id.table_name; + drop_query->kind = ASTDropQuery::Kind::Drop; + ASTPtr ast_drop_query = drop_query; + InterpreterDropQuery drop_interpreter(ast_drop_query, context); + drop_interpreter.execute(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } +} + + +std::unique_ptr TemporaryLiveViewCleaner::the_instance; + + +void TemporaryLiveViewCleaner::init(Context & global_context_) +{ + if (the_instance) + throw Exception("TemporaryLiveViewCleaner already initialized", ErrorCodes::LOGICAL_ERROR); + the_instance.reset(new TemporaryLiveViewCleaner(global_context_)); +} + + +void TemporaryLiveViewCleaner::shutdown() +{ + the_instance.reset(); +} + + +TemporaryLiveViewCleaner::TemporaryLiveViewCleaner(Context & global_context_) + : global_context(global_context_) +{ +} + + +TemporaryLiveViewCleaner::~TemporaryLiveViewCleaner() +{ + stopBackgroundThread(); +} + + +void TemporaryLiveViewCleaner::addView(const std::shared_ptr & view) +{ + if (!view->isTemporary()) + return; + + auto current_time = std::chrono::system_clock::now(); + auto time_of_next_check = current_time + view->getTimeout(); + + std::lock_guard lock{mutex}; + + /// Keep the vector `views` sorted by time of next check. + StorageAndTimeOfCheck storage_and_time_of_check{view, time_of_next_check}; + views.insert(std::upper_bound(views.begin(), views.end(), storage_and_time_of_check), storage_and_time_of_check); + + if (!background_thread.joinable()) + background_thread = ThreadFromGlobalPool{&TemporaryLiveViewCleaner::backgroundThreadFunc, this}; + + background_thread_wake_up.notify_one(); +} + + +void TemporaryLiveViewCleaner::backgroundThreadFunc() +{ + std::unique_lock lock{mutex}; + while (!background_thread_should_exit && !views.empty()) + { + background_thread_wake_up.wait_until(lock, views.front().time_of_check); + if (background_thread_should_exit) + return; + + auto current_time = std::chrono::system_clock::now(); + std::vector storages_to_drop; + + auto it = views.begin(); + while (it != views.end()) + { + std::shared_ptr storage = it->storage.lock(); + auto & time_of_check = it->time_of_check; + if (!storage) + { + /// Storage has been already removed. + it = views.erase(it); + continue; + } + + ++it; + + if (current_time < time_of_check) + break; /// It's not the time to check it yet. + + time_of_check = current_time + storage->getTimeout(); + + auto storage_id = storage->getStorageID(); + if (storage->hasUsers() || !DatabaseCatalog::instance().getDependencies(storage_id).empty()) + continue; + + storages_to_drop.emplace_back(storage_id); + } + + lock.unlock(); + for (const auto & storage_id : storages_to_drop) + executeDropQuery(storage_id, global_context); + lock.lock(); + } +} + + +void TemporaryLiveViewCleaner::stopBackgroundThread() +{ + std::lock_guard lock{mutex}; + if (background_thread.joinable()) + { + background_thread_should_exit = true; + background_thread_wake_up.notify_one(); + background_thread.join(); + } +} + +} diff --git a/src/Storages/LiveView/TemporaryLiveViewCleaner.h b/src/Storages/LiveView/TemporaryLiveViewCleaner.h new file mode 100644 index 00000000000..57c12bd1c07 --- /dev/null +++ b/src/Storages/LiveView/TemporaryLiveViewCleaner.h @@ -0,0 +1,51 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class StorageLiveView; +struct StorageID; + +/// This class removes temporary live views in the background thread when it's possible. +/// There should only a single instance of this class. +class TemporaryLiveViewCleaner +{ +public: + static TemporaryLiveViewCleaner & instance() { return *the_instance; } + + /// Drops a specified live view after a while if it's temporary. + void addView(const std::shared_ptr & view); + + /// Should be called once. + static void init(Context & global_context_); + static void shutdown(); + +private: + friend std::unique_ptr::deleter_type; + + TemporaryLiveViewCleaner(Context & global_context_); + ~TemporaryLiveViewCleaner(); + + void backgroundThreadFunc(); + void stopBackgroundThread(); + + struct StorageAndTimeOfCheck + { + std::weak_ptr storage; + std::chrono::system_clock::time_point time_of_check; + bool operator <(const StorageAndTimeOfCheck & other) const { return time_of_check < other.time_of_check; } + }; + + static std::unique_ptr the_instance; + Context & global_context; + std::mutex mutex; + std::vector views; + ThreadFromGlobalPool background_thread; + std::atomic background_thread_should_exit = false; + std::condition_variable background_thread_wake_up; +}; + +} diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 1ddb8c77072..fed961ed2bb 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -20,6 +20,7 @@ SRCS( IStorage.cpp KeyDescription.cpp LiveView/StorageLiveView.cpp + LiveView/TemporaryLiveViewCleaner.cpp MergeTree/ActiveDataPartSet.cpp MergeTree/AllMergeSelector.cpp MergeTree/BackgroundProcessingPool.cpp From b8a2c1d2a29517c2bd0e8f791ce31c474f30f7d5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 22:45:37 +0300 Subject: [PATCH 093/341] Push pragma only for new gcc --- src/Storages/MergeTree/MergeTreePartition.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 8ef3e458871..2802b842f54 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -29,8 +29,10 @@ String MergeTreePartition::getID(const MergeTreeData & storage) const return getID(storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block); } -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wstringop-overflow" +#if defined (__GNUC__) && __GNUC__ >= 10 + #pragma GCC diagnostic push + #pragma GCC diagnostic ignored "-Wstringop-overflow" +#endif /// NOTE: This ID is used to create part names which are then persisted in ZK and as directory names on the file system. /// So if you want to change this method, be sure to guarantee compatibility with existing table data. @@ -90,7 +92,9 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const return result; } -#pragma GCC diagnostic pop +#if defined (__GNUC__) && __GNUC__ >= 10 + #pragma GCC diagnostic pop +#endif void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const { From 7080d56470a9bfed57dde63ef2e40c9832599c72 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 9 Sep 2020 22:59:34 +0300 Subject: [PATCH 094/341] fix missed database name when altering mv --- src/Storages/SelectQueryDescription.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/SelectQueryDescription.cpp b/src/Storages/SelectQueryDescription.cpp index bb8295df6f3..0935a5be5ca 100644 --- a/src/Storages/SelectQueryDescription.cpp +++ b/src/Storages/SelectQueryDescription.cpp @@ -105,14 +105,14 @@ SelectQueryDescription SelectQueryDescription::getSelectQueryFromASTForMatView(c if (new_select.list_of_selects->children.size() != 1) throw Exception("UNION is not supported for MATERIALIZED VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW); - SelectQueryDescription result; - - result.inner_query = new_select.list_of_selects->children.at(0)->clone(); - - auto & select_query = result.inner_query->as(); + auto & new_inner_query = new_select.list_of_selects->children.at(0); + auto & select_query = new_inner_query->as(); checkAllowedQueries(select_query); + + SelectQueryDescription result; result.select_table_id = extractDependentTableFromSelectQuery(select_query, context); - result.select_query = select->clone(); + result.select_query = new_select.clone(); + result.inner_query = new_inner_query->clone(); return result; } From 27258c8e70213cf57e1bbf36176cda961d56e12f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 9 Sep 2020 23:47:42 +0300 Subject: [PATCH 095/341] utils/list-licenses/list-licenses.sh: ignore more files - *.rtf They can have NULL byte, and StorageSystemLicenses.sh will warn: ./StorageSystemLicenses.sh: line 11: warning: command substitution: ignored null byte in input Found with: find contrib/ -type f -and '(' -iname 'LICENSE*' -or -iname 'COPYING*' -or -iname 'COPYRIGHT*' ')' -and -not -iname '*.html' | xargs grep -Pa '\x00' - *.h - *.cpp - *.htm And after verified with: $ find contrib/ -type f -and '(' -iname 'LICENSE*' -or -iname 'COPYING*' -or -iname 'COPYRIGHT*' ')' -and -not '(' -iname '*.html' -or -iname '*.htm' -or -iname '*.rtf' -or -name '*.cpp' -or -name '*.h' -or -iname '*.json' ')' | xargs file -b | sort -u ASCII text ASCII text, with CR line terminators ASCII text, with very long lines empty UTF-8 Unicode text --- utils/list-licenses/list-licenses.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/list-licenses/list-licenses.sh b/utils/list-licenses/list-licenses.sh index 987179e26a8..8eee3f97253 100755 --- a/utils/list-licenses/list-licenses.sh +++ b/utils/list-licenses/list-licenses.sh @@ -7,7 +7,7 @@ ls -1 -d ${LIBS_PATH}/*/ | grep -F -v -- '-cmake' | while read LIB; do LIB_NAME=$(basename $LIB) LIB_LICENSE=$( - LC_ALL=C find "$LIB" -type f -and '(' -iname 'LICENSE*' -or -iname 'COPYING*' -or -iname 'COPYRIGHT*' ')' -and -not -iname '*.html' -printf "%d\t%p\n" | + LC_ALL=C find "$LIB" -type f -and '(' -iname 'LICENSE*' -or -iname 'COPYING*' -or -iname 'COPYRIGHT*' ')' -and -not '(' -iname '*.html' -or -iname '*.htm' -or -iname '*.rtf' -or -name '*.cpp' -or -name '*.h' -or -iname '*.json' ')' -printf "%d\t%p\n" | awk ' BEGIN { IGNORECASE=1; min_depth = 0 } /LICENSE/ { if (!min_depth || $1 <= min_depth) { min_depth = $1; license = $2 } } From 0f4fdcbf389909ed2e642263b0d6a65a3580d8e0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 10 Sep 2020 02:05:41 +0300 Subject: [PATCH 096/341] Pass -fsanitize-blacklist for TSAN only under clang (gcc does not support this) And no such check for -fsnaitize=memory, since gcc does not support it anyway. --- cmake/sanitize.cmake | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 32443ed78c3..7c7e9c388a0 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -36,7 +36,15 @@ if (SANITIZE) endif () elseif (SANITIZE STREQUAL "thread") - set (TSAN_FLAGS "-fsanitize=thread -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/tsan_suppressions.txt") + set (TSAN_FLAGS "-fsanitize=thread") + if (COMPILER_CLANG) + set (TSAN_FLAGS "${TSAN_FLAGS} -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/tsan_suppressions.txt") + else() + message (WARNING "TSAN suppressions was not passed to the compiler (since the compiler is not clang)") + message (WARNING "Use the following command to pass them manually:") + message (WARNING " export TSAN_OPTIONS=\"$TSAN_OPTIONS suppressions=${CMAKE_SOURCE_DIR}/tests/tsan_suppressions.txt\"") + endif() + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${TSAN_FLAGS}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${TSAN_FLAGS}") From 0a3f7d0fccc8b960d59b415b02313673d1bdc698 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 19:49:47 -0700 Subject: [PATCH 097/341] AsynchronousMetricsLog - add improved tests This adds integration tests to test the asynchronous_metric_log table for event_time_microseconds field. Asynchronous metrics are updated once every 60s, so this setting needs to be overridden for having those metrics available in time so that the tests can run. So the `asynchronous_metric_update_period_s` setting is being overriden to be 2s. --- .../__init__.py | 0 .../asynchronous_metrics_update_period_s.xml | 3 ++ .../test.py | 32 +++++++++++++++++++ .../01473_event_time_microseconds.reference | 4 +-- .../01473_event_time_microseconds.sql | 12 ++++--- 5 files changed, 43 insertions(+), 8 deletions(-) create mode 100644 tests/integration/test_asynchronous_metric_log_table/__init__.py create mode 100644 tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml create mode 100644 tests/integration/test_asynchronous_metric_log_table/test.py diff --git a/tests/integration/test_asynchronous_metric_log_table/__init__.py b/tests/integration/test_asynchronous_metric_log_table/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml b/tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml new file mode 100644 index 00000000000..840c19f03a6 --- /dev/null +++ b/tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml @@ -0,0 +1,3 @@ + + 2 + diff --git a/tests/integration/test_asynchronous_metric_log_table/test.py b/tests/integration/test_asynchronous_metric_log_table/test.py new file mode 100644 index 00000000000..10face9abc6 --- /dev/null +++ b/tests/integration/test_asynchronous_metric_log_table/test.py @@ -0,0 +1,32 @@ +import time + +from helpers.cluster import ClickHouseCluster + +# Tests that the event_time_microseconds field in system.asynchronous_metric_log table gets populated. +# asynchronous metrics are updated once every 60s by default. To make the test run faster, the setting +# asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and +# are available for querying during the test. +def test_asynchronous_metric_log(): + cluster = ClickHouseCluster(__file__) + node1 = cluster.add_instance('node1', with_zookeeper=True, main_configs=['configs/asynchronous_metrics_update_period_s.xml']) + try: + cluster.start() + node1.query("SET log_queries = 1;") + node1.query("CREATE DATABASE replica;") + query_create = '''CREATE TABLE replica.test + ( + id Int64, + event_time DateTime + ) + Engine=MergeTree() + PARTITION BY toYYYYMMDD(event_time) + ORDER BY id;''' + time.sleep(2) + node1.query(query_create) + node1.query('''INSERT INTO replica.test VALUES (1, now())''') + node1.query("SYSTEM FLUSH LOGS;") + node1.query("SELECT * FROM system.asynchronous_metrics LIMIT 10") + assert "1\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') + assert "ok\n" in node1.query("SELECT If((select count(event_time_microseconds) from system.asynchronous_metric_log) > 0, 'ok', 'fail');") + finally: + cluster.shutdown() diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference index cac87f32a29..8aa31f9ab6a 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.reference +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -1,4 +1,2 @@ -01473_asynchronous_metric_log_event_start_time_milliseconds_test -ok -01473_metric_log_event_start_time_milliseconds_test +01473_metric_log_table_event_start_time_microseconds_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 6a13d6e1543..9bafd3bdbda 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -1,9 +1,11 @@ +-- This file contains tests for the event_time_microseconds field for various tables. +-- Note: Only event_time_microseconds for asynchronous_metric_log table is tested via +-- an integration test as those metrics take 60s by default to be updated. +-- Refer: tests/integration/test_asynchronous_metric_log_table. + 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'; +select '01473_metric_log_table_event_start_time_microseconds_test'; system flush logs; +SELECT sleep(3) Format Null; SELECT If((select count(event_time_microseconds) from system.metric_log) > 0, 'ok', 'fail'); -- success From 2ad63d2b4de47ca598712f9404a26ea53ba35e70 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Wed, 9 Sep 2020 17:56:21 -0700 Subject: [PATCH 098/341] AsynchronousMetricLog - refactor integration test --- .../test.py | 23 +++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_asynchronous_metric_log_table/test.py b/tests/integration/test_asynchronous_metric_log_table/test.py index 10face9abc6..9910a188e2c 100644 --- a/tests/integration/test_asynchronous_metric_log_table/test.py +++ b/tests/integration/test_asynchronous_metric_log_table/test.py @@ -1,14 +1,28 @@ import time +import pytest from helpers.cluster import ClickHouseCluster +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', with_zookeeper=True, + main_configs=['configs/asynchronous_metrics_update_period_s.xml']) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + # Tests that the event_time_microseconds field in system.asynchronous_metric_log table gets populated. # asynchronous metrics are updated once every 60s by default. To make the test run faster, the setting # asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and # are available for querying during the test. -def test_asynchronous_metric_log(): - cluster = ClickHouseCluster(__file__) - node1 = cluster.add_instance('node1', with_zookeeper=True, main_configs=['configs/asynchronous_metrics_update_period_s.xml']) +def test_event_time_microseconds_field(started_cluster): try: cluster.start() node1.query("SET log_queries = 1;") @@ -27,6 +41,7 @@ def test_asynchronous_metric_log(): node1.query("SYSTEM FLUSH LOGS;") node1.query("SELECT * FROM system.asynchronous_metrics LIMIT 10") assert "1\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') - assert "ok\n" in node1.query("SELECT If((select count(event_time_microseconds) from system.asynchronous_metric_log) > 0, 'ok', 'fail');") + assert "ok\n" in node1.query( + "SELECT If((select count(event_time_microseconds) from system.asynchronous_metric_log) > 0, 'ok', 'fail');") finally: cluster.shutdown() From 31b0bd481c687160e2217a1443b244fad2733a04 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Thu, 10 Sep 2020 00:17:20 -0700 Subject: [PATCH 099/341] Integration tests - more accurate tests --- .../test.py | 20 +++++++++++++++---- .../01473_event_time_microseconds.sql | 16 ++++++++++++++- 2 files changed, 31 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_asynchronous_metric_log_table/test.py b/tests/integration/test_asynchronous_metric_log_table/test.py index 9910a188e2c..0eb72c20376 100644 --- a/tests/integration/test_asynchronous_metric_log_table/test.py +++ b/tests/integration/test_asynchronous_metric_log_table/test.py @@ -39,9 +39,21 @@ def test_event_time_microseconds_field(started_cluster): node1.query(query_create) node1.query('''INSERT INTO replica.test VALUES (1, now())''') node1.query("SYSTEM FLUSH LOGS;") - node1.query("SELECT * FROM system.asynchronous_metrics LIMIT 10") - assert "1\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') - assert "ok\n" in node1.query( - "SELECT If((select count(event_time_microseconds) from system.asynchronous_metric_log) > 0, 'ok', 'fail');") + #query assumes that the event_time field is accurate + equals_query = '''WITH ( + ( + SELECT event_time_microseconds + FROM system.asynchronous_metric_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.asynchronous_metric_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time) + SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail')''' + assert "ok\n" in node1.query(equals_query) finally: cluster.shutdown() diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index 9bafd3bdbda..dd79a735d8f 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -8,4 +8,18 @@ set log_queries = 1; select '01473_metric_log_table_event_start_time_microseconds_test'; system flush logs; SELECT sleep(3) Format Null; -SELECT If((select count(event_time_microseconds) from system.metric_log) > 0, 'ok', 'fail'); -- success +-- query assumes that the event_time field is accurate. +WITH ( + ( + SELECT event_time_microseconds + FROM system.metric_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.metric_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time) +SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail') From 7fb45461d174c3ae122c9d11ba5a4414fe0d3fd6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 10 Sep 2020 10:36:29 +0300 Subject: [PATCH 100/341] Try fix totals port for IAccumulatingTransform. --- src/Processors/IAccumulatingTransform.cpp | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Processors/IAccumulatingTransform.cpp b/src/Processors/IAccumulatingTransform.cpp index 2905d185df2..64bdbe2410f 100644 --- a/src/Processors/IAccumulatingTransform.cpp +++ b/src/Processors/IAccumulatingTransform.cpp @@ -49,13 +49,15 @@ IAccumulatingTransform::Status IAccumulatingTransform::prepare() return Status::Finished; } - /// Close input if flag was set manually. + if (input.isFinished()) + finished_input = true; + if (finished_input) + { + /// Close input if flag was set manually. input.close(); - /// Read from totals port if has it. - if (input.isFinished()) - { + /// Read from totals port if has it. if (inputs.size() > 1) { auto & totals_input = inputs.back(); @@ -69,12 +71,8 @@ IAccumulatingTransform::Status IAccumulatingTransform::prepare() totals_input.close(); } } - } - /// Generate output block. - if (input.isFinished()) - { - finished_input = true; + /// Generate output block. return Status::Ready; } From a64473313971bbd3d461d5c7b68165b39d4515e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Sep 2020 12:05:57 +0300 Subject: [PATCH 101/341] Attempt to make performance test more reliable --- programs/server/Server.cpp | 8 +- src/Common/remapExecutable.cpp | 213 +++++++++++++++++++++++++++++++++ src/Common/remapExecutable.h | 7 ++ 3 files changed, 227 insertions(+), 1 deletion(-) create mode 100644 src/Common/remapExecutable.cpp create mode 100644 src/Common/remapExecutable.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f24ba444203..8149623ffce 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -307,6 +308,11 @@ int Server::main(const std::vector & /*args*/) { if (config().getBool("mlock_executable", false)) { + LOG_DEBUG(log, "Will remap executable in memory."); + remapExecutable(); + LOG_DEBUG(log, "The code in memory has been successfully remapped."); + +/* if (hasLinuxCapability(CAP_IPC_LOCK)) { LOG_TRACE(log, "Will mlockall to prevent executable memory from being paged out. It may take a few seconds."); @@ -321,7 +327,7 @@ int Server::main(const std::vector & /*args*/) " It could happen due to incorrect ClickHouse package installation." " You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep {}'." " Note that it will not work on 'nosuid' mounted filesystems.", executable_path); - } + }*/ } } #endif diff --git a/src/Common/remapExecutable.cpp b/src/Common/remapExecutable.cpp new file mode 100644 index 00000000000..f7f353a83c6 --- /dev/null +++ b/src/Common/remapExecutable.cpp @@ -0,0 +1,213 @@ +#if defined(__linux__) && defined(__amd64__) && defined(__SSE2__) + +#include +#include +#include + +#include + +#include + +#include +#include +#include +#include +#include + +#include "remapExecutable.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int CANNOT_ALLOCATE_MEMORY; +} + + +namespace +{ + +uintptr_t readAddressHex(DB::ReadBuffer & in) +{ + uintptr_t res = 0; + while (!in.eof()) + { + if (isHexDigit(*in.position())) + { + res *= 16; + res += unhex(*in.position()); + ++in.position(); + } + else + break; + } + return res; +} + + +/** Find the address and size of the mapped memory region pointed by ptr. + */ +std::pair getMappedArea(void * ptr) +{ + using namespace DB; + + uintptr_t uintptr = reinterpret_cast(ptr); + ReadBufferFromFile in("/proc/self/maps"); + + while (!in.eof()) + { + uintptr_t begin = readAddressHex(in); + assertChar('-', in); + uintptr_t end = readAddressHex(in); + skipToNextLineOrEOF(in); + + if (begin <= uintptr && uintptr < end) + return {reinterpret_cast(begin), end - begin}; + } + + throw Exception("Cannot find mapped area for pointer", ErrorCodes::LOGICAL_ERROR); +} + + +__attribute__((__noinline__)) int64_t our_syscall(...) +{ + __asm__ __volatile__ (R"( + movq %%rdi,%%rax; + movq %%rsi,%%rdi; + movq %%rdx,%%rsi; + movq %%rcx,%%rdx; + movq %%r8,%%r10; + movq %%r9,%%r8; + movq 8(%%rsp),%%r9; + syscall; + ret + )" : : : "memory"); + return 0; +} + + +__attribute__((__always_inline__)) void our_memcpy(char * __restrict dst, const char * __restrict src, ssize_t n) +{ + while (n > 0) + { + _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), + _mm_loadu_si128(reinterpret_cast(src))); + + dst += 16; + src += 16; + n -= 16; + } +} + + +__attribute__((__noinline__)) void remapToHugeStep3(void * scratch, size_t size, size_t offset) +{ + /// The function should not use the stack, otherwise various optimizations, including "omit-frame-pointer" may break the code. + + /// Unmap the scratch area. + our_syscall(SYS_munmap, scratch, size); + + /** The return address of this function is pointing to scratch area (because it was called from there). + * But the scratch area no longer exists. We should correct the return address by subtracting the offset. + */ + __asm__ __volatile__("subq %0, 8(%%rsp)" : : "r"(offset) : "memory"); +} + + +__attribute__((__noinline__)) void remapToHugeStep2(void * begin, size_t size, void * scratch) +{ + /** Unmap old memory region with the code of our program. + * Our instruction pointer is located inside scratch area and this function can execute after old code is unmapped. + * But it cannot call any other functions because they are not available at usual addresses + * - that's why we have to use "our_syscall" and "our_memcpy" functions. + * (Relative addressing may continue to work but we should not assume that). + */ + + int64_t offset = reinterpret_cast(scratch) - reinterpret_cast(begin); + int64_t (*syscall_func)(...) = reinterpret_cast(reinterpret_cast(our_syscall) + offset); + + //char dot = '.'; + //syscall_func(SYS_write, 2, &dot, 1); + + int64_t munmap_res = syscall_func(SYS_munmap, begin, size); + if (munmap_res != 0) + return; + + //syscall_func(SYS_write, 2, &dot, 1); + + /// Map new anonymous memory region in place of old region with code. + + int64_t mmap_res = syscall_func(SYS_mmap, begin, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS | MAP_FIXED, -1, 0); + if (-1 == mmap_res) + syscall_func(SYS_exit, 1); + //syscall_func(SYS_write, 2, &dot, 1); + + /// As the memory region is anonymous, we can do madvise with MADV_HUGEPAGE. + + syscall_func(SYS_madvise, begin, size, MADV_HUGEPAGE); + //syscall_func(SYS_write, 2, &dot, 1); + + /// Copy the code from scratch area to the old memory location. + + our_memcpy(reinterpret_cast(begin), reinterpret_cast(scratch), size); + //syscall_func(SYS_write, 2, &dot, 1); + + /// Make the memory area with the code executable and non-writable. + + syscall_func(SYS_mprotect, begin, size, PROT_READ | PROT_EXEC); + //syscall_func(SYS_write, 2, &dot, 1); + + /** Step 3 function should unmap the scratch area. + * The currently executed code is located in the scratch area and cannot be removed here. + * We have to call another function and use its address from the original location (not in scratch area). + * To do it, we obtain it's pointer and call by pointer. + */ + + void(* volatile step3)(void*, size_t, size_t) = remapToHugeStep3; + step3(scratch, size, offset); +} + + +__attribute__((__noinline__)) void remapToHugeStep1(void * begin, size_t size) +{ + /// Allocate scratch area and copy the code there. + + void * scratch = mmap(nullptr, size, PROT_READ | PROT_WRITE | PROT_EXEC, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); + if (MAP_FAILED == scratch) + throwFromErrno(fmt::format("Cannot mmap {} bytes", size), ErrorCodes::CANNOT_ALLOCATE_MEMORY); + + memcpy(scratch, begin, size); + + /// Offset to the scratch area from previous location. + + int64_t offset = reinterpret_cast(scratch) - reinterpret_cast(begin); + + /// Jump to the next function inside the scratch area. + + reinterpret_cast(reinterpret_cast(remapToHugeStep2) + offset)(begin, size, scratch); +} + +} + + +void remapExecutable() +{ + auto [begin, size] = getMappedArea(reinterpret_cast(remapExecutable)); + remapToHugeStep1(begin, size); +} + +} + +#else + +namespace DB +{ + +void remapExecutable() {} + +} + +#endif diff --git a/src/Common/remapExecutable.h b/src/Common/remapExecutable.h new file mode 100644 index 00000000000..7acb61f13bd --- /dev/null +++ b/src/Common/remapExecutable.h @@ -0,0 +1,7 @@ +namespace DB +{ + +/// This function tries to reallocate the code of the running program in a more efficient way. +void remapExecutable(); + +} From 532d121100fc696fa2edb2d3dff863907850c218 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Sep 2020 12:14:31 +0300 Subject: [PATCH 102/341] Fix typo --- src/Common/remapExecutable.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/remapExecutable.cpp b/src/Common/remapExecutable.cpp index f7f353a83c6..ec8b1703b0a 100644 --- a/src/Common/remapExecutable.cpp +++ b/src/Common/remapExecutable.cpp @@ -163,7 +163,7 @@ __attribute__((__noinline__)) void remapToHugeStep2(void * begin, size_t size, v /** Step 3 function should unmap the scratch area. * The currently executed code is located in the scratch area and cannot be removed here. * We have to call another function and use its address from the original location (not in scratch area). - * To do it, we obtain it's pointer and call by pointer. + * To do it, we obtain its pointer and call by pointer. */ void(* volatile step3)(void*, size_t, size_t) = remapToHugeStep3; From 20b3823770f60feefcd73306141bc25875d87f9a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Sep 2020 13:21:13 +0300 Subject: [PATCH 103/341] do not remove directory on attach --- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++++--- .../test_cleanup_dir_after_bad_zk_conn/test.py | 13 ++++++++++++- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 00dcc7aeb08..908112c17ff 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -234,10 +234,11 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( { current_zookeeper = global_context.getZooKeeper(); } - catch (Poco::Exception & e) + catch (...) { - dropIfEmpty(); - throw e; + if (!attach) + dropIfEmpty(); + throw; } } @@ -262,7 +263,10 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( if (!current_zookeeper) { if (!attach) + { + dropIfEmpty(); throw Exception("Can't create replicated table without ZooKeeper", ErrorCodes::NO_ZOOKEEPER); + } /// Do not activate the replica. It will be readonly. LOG_ERROR(log, "No ZooKeeper: table will be in readonly mode."); diff --git a/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py b/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py index 4cb243160a0..ab2db469157 100644 --- a/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py +++ b/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py @@ -54,9 +54,20 @@ def test_cleanup_dir_after_wrong_replica_name(start_cluster): assert "already exists" in error node1.query("CREATE TABLE test_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test2/', 'r2') ORDER BY n") - def test_cleanup_dir_after_wrong_zk_path(start_cluster): node1.query("CREATE TABLE test3_r1 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test3/', 'r1') ORDER BY n") error = node1.query_and_get_error("CREATE TABLE test3_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/', 'r2') ORDER BY n") assert "Cannot create" in error node1.query("CREATE TABLE test3_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test3/', 'r2') ORDER BY n") + +def test_attach_without_zk(start_cluster): + node1.query("CREATE TABLE test4_r1 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test4/', 'r1') ORDER BY n") + node1.query("DETACH TABLE test4_r1") + with PartitionManager() as pm: + pm._add_rule({'probability': 0.5, 'source': node1.ip_address, 'destination_port': 2181, 'action': 'DROP'}) + try: + node1.query("ATTACH TABLE test4_r1") + except: + pass + node1.query("ATTACH TABLE IF NOT EXISTS test4_r1") + node1.query("SELECT * FROM test4_r1") From c309f55c207d5e864f1b05200e2a706627163bac Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 10 Sep 2020 14:02:52 +0300 Subject: [PATCH 104/341] updated setting and added default value --- src/Functions/CMakeLists.txt | 5 ++++- src/Functions/GatherUtils/CMakeLists.txt | 5 +++-- src/Functions/URL/CMakeLists.txt | 5 +++-- src/Functions/array/CMakeLists.txt | 5 +++-- 4 files changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 08eefec84d4..b19266726ae 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -57,7 +57,10 @@ option(STRIP_DEBUG_SYMBOLS_FUNCTIONS "Do not generate debugger info for ClickHouse functions. Provides faster linking and lower binary size. Tradeoff is the inability to debug some source files with e.g. gdb - (empty stack frames and no local variables)." OFF) + (empty stack frames and no local variables)." + CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" + OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" + OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL") if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) message(WARNING "Not generating debugger info for ClickHouse functions") diff --git a/src/Functions/GatherUtils/CMakeLists.txt b/src/Functions/GatherUtils/CMakeLists.txt index 921b06fb1c2..3f7f08621a1 100644 --- a/src/Functions/GatherUtils/CMakeLists.txt +++ b/src/Functions/GatherUtils/CMakeLists.txt @@ -3,5 +3,6 @@ add_headers_and_sources(clickhouse_functions_gatherutils .) add_library(clickhouse_functions_gatherutils ${clickhouse_functions_gatherutils_sources} ${clickhouse_functions_gatherutils_headers}) target_link_libraries(clickhouse_functions_gatherutils PRIVATE dbms) -# Won't generate debug info for files with heavy template instantiation to achieve faster linking and lower size. -target_compile_options(clickhouse_functions_gatherutils PRIVATE "-g0") +if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) + target_compile_options(clickhouse_functions_gatherutils PRIVATE "-g0") +endif() diff --git a/src/Functions/URL/CMakeLists.txt b/src/Functions/URL/CMakeLists.txt index 72fdc3174fc..0cf02fd524f 100644 --- a/src/Functions/URL/CMakeLists.txt +++ b/src/Functions/URL/CMakeLists.txt @@ -3,8 +3,9 @@ add_headers_and_sources(clickhouse_functions_url .) add_library(clickhouse_functions_url ${clickhouse_functions_url_sources} ${clickhouse_functions_url_headers}) target_link_libraries(clickhouse_functions_url PRIVATE dbms) -# Won't generate debug info for files with heavy template instantiation to achieve faster linking and lower size. -target_compile_options(clickhouse_functions_url PRIVATE "-g0") +if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) + target_compile_options(clickhouse_functions_url PRIVATE "-g0") +endif() # TODO: move Functions/Regexps.h to some lib and use here target_link_libraries(clickhouse_functions_url PRIVATE hyperscan) diff --git a/src/Functions/array/CMakeLists.txt b/src/Functions/array/CMakeLists.txt index 32ef0216caa..1457e671261 100644 --- a/src/Functions/array/CMakeLists.txt +++ b/src/Functions/array/CMakeLists.txt @@ -3,5 +3,6 @@ add_headers_and_sources(clickhouse_functions_array .) add_library(clickhouse_functions_array ${clickhouse_functions_array_sources} ${clickhouse_functions_array_headers}) target_link_libraries(clickhouse_functions_array PRIVATE dbms clickhouse_functions_gatherutils) -# Won't generate debug info for files with heavy template instantiation to achieve faster linking and lower size. -target_compile_options(clickhouse_functions_array PRIVATE "-g0") +if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) + target_compile_options(clickhouse_functions_array PRIVATE "-g0") +endif() From 598c81ed0ac1e127136b54d167e9209e252818be Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 10 Sep 2020 14:07:08 +0300 Subject: [PATCH 105/341] fix: default value --- src/Functions/CMakeLists.txt | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index b19266726ae..4f6952aad8f 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -53,14 +53,21 @@ endif() target_include_directories(clickhouse_functions SYSTEM PRIVATE ${SPARSEHASH_INCLUDE_DIR}) +if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" + OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" + OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL") + set (STRIP_DSF_DEFAULT ON) +else() + set (STRIP_DSF_DEFAULT OFF) +endif() + + option(STRIP_DEBUG_SYMBOLS_FUNCTIONS "Do not generate debugger info for ClickHouse functions. Provides faster linking and lower binary size. Tradeoff is the inability to debug some source files with e.g. gdb (empty stack frames and no local variables)." - CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" - OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" - OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL") + STRIP_DSF_DEFAULT) if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) message(WARNING "Not generating debugger info for ClickHouse functions") From 00b8d91eb859a2642a0583107498d4642c7d4348 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Thu, 10 Sep 2020 19:46:59 +0800 Subject: [PATCH 106/341] Fix mysql replication TableMapEvent optional metadata ignore --- src/Core/MySQL/IMySQLReadPacket.cpp | 29 ++++++++++--------- src/Core/MySQL/MySQLReplication.cpp | 6 +++- src/Core/tests/mysql_protocol.cpp | 1 + .../MySQL/MaterializeMySQLSyncThread.cpp | 2 ++ 4 files changed, 23 insertions(+), 15 deletions(-) diff --git a/src/Core/MySQL/IMySQLReadPacket.cpp b/src/Core/MySQL/IMySQLReadPacket.cpp index 8fc8855c8a4..5f6bbc7bceb 100644 --- a/src/Core/MySQL/IMySQLReadPacket.cpp +++ b/src/Core/MySQL/IMySQLReadPacket.cpp @@ -50,21 +50,22 @@ uint64_t readLengthEncodedNumber(ReadBuffer & buffer) uint64_t buf = 0; buffer.readStrict(c); auto cc = static_cast(c); - if (cc < 0xfc) + switch (cc) { - return cc; - } - else if (cc < 0xfd) - { - buffer.readStrict(reinterpret_cast(&buf), 2); - } - else if (cc < 0xfe) - { - buffer.readStrict(reinterpret_cast(&buf), 3); - } - else - { - buffer.readStrict(reinterpret_cast(&buf), 8); + /// NULL + case 0xfb: + break; + case 0xfc: + buffer.readStrict(reinterpret_cast(&buf), 2); + break; + case 0xfd: + buffer.readStrict(reinterpret_cast(&buf), 3); + break; + case 0xfe: + buffer.readStrict(reinterpret_cast(&buf), 8); + break; + default: + return cc; } return buf; } diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 42d077260f8..808e9d1a247 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -171,7 +171,9 @@ namespace MySQLReplication /// Ignore MySQL 8.0 optional metadata fields. /// https://mysqlhighavailability.com/more-metadata-is-written-into-binary-log/ - payload.ignore(payload.available() - CHECKSUM_CRC32_SIGNATURE_LENGTH); + auto remain = payload.available(); + if (remain > CHECKSUM_CRC32_SIGNATURE_LENGTH) + payload.ignore(remain); } /// Types that do not used in the binlog event: @@ -221,6 +223,7 @@ namespace MySQLReplication } case MYSQL_TYPE_NEWDECIMAL: case MYSQL_TYPE_STRING: { + /// Big-Endian auto b0 = UInt16(meta[pos] << 8); auto b1 = UInt8(meta[pos + 1]); column_meta.emplace_back(UInt16(b0 + b1)); @@ -231,6 +234,7 @@ namespace MySQLReplication case MYSQL_TYPE_BIT: case MYSQL_TYPE_VARCHAR: case MYSQL_TYPE_VAR_STRING: { + /// Little-Endian auto b0 = UInt8(meta[pos]); auto b1 = UInt16(meta[pos + 1] << 8); column_meta.emplace_back(UInt16(b0 + b1)); diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index acae8603c40..6cad095fc85 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -283,6 +283,7 @@ int main(int argc, char ** argv) } { + /// mysql_protocol --host=172.17.0.3 --user=root --password=123 --db=sbtest try { boost::program_options::options_description desc("Allowed options"); diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 851ea351876..465a7cb912a 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -195,6 +195,7 @@ void MaterializeMySQLSyncThread::synchronization(const String & mysql_version) } catch (...) { + client.disconnect(); tryLogCurrentException(log); getDatabase(database_name).setException(std::current_exception()); } @@ -206,6 +207,7 @@ void MaterializeMySQLSyncThread::stopSynchronization() { sync_quit = true; background_thread_pool->join(); + client.disconnect(); } } From e0081980230e636545a43a67238dc25b44f7777c Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 10 Sep 2020 15:44:49 +0300 Subject: [PATCH 107/341] fix: default value --- src/Functions/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 4f6952aad8f..0a99a034a33 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -67,7 +67,7 @@ option(STRIP_DEBUG_SYMBOLS_FUNCTIONS Provides faster linking and lower binary size. Tradeoff is the inability to debug some source files with e.g. gdb (empty stack frames and no local variables)." - STRIP_DSF_DEFAULT) + ${STRIP_DSF_DEFAULT}) if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) message(WARNING "Not generating debugger info for ClickHouse functions") From 65e9c678f8990756ac22ad2ad10bc08a9c0ec4b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Sep 2020 17:47:02 +0300 Subject: [PATCH 108/341] Disable under certain conditions --- src/Common/remapExecutable.cpp | 2 +- src/Common/ya.make | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/remapExecutable.cpp b/src/Common/remapExecutable.cpp index ec8b1703b0a..b41fece0c79 100644 --- a/src/Common/remapExecutable.cpp +++ b/src/Common/remapExecutable.cpp @@ -1,4 +1,4 @@ -#if defined(__linux__) && defined(__amd64__) && defined(__SSE2__) +#if defined(__linux__) && defined(__amd64__) && defined(__SSE2__) && !defined(SANITIZER) && defined(NDEBUG) #include #include diff --git a/src/Common/ya.make b/src/Common/ya.make index d9a7a2ce4de..72f1fa42756 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -74,6 +74,7 @@ SRCS( QueryProfiler.cpp quoteString.cpp randomSeed.cpp + remapExecutable.cpp RemoteHostFilter.cpp renameat2.cpp RWLock.cpp From f19f02028b7dca3844127a550c8703a8f40fc383 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Sep 2020 17:50:10 +0300 Subject: [PATCH 109/341] fix misleading exception message about uuid macro --- src/Common/Macros.cpp | 8 +++++++- src/Storages/MergeTree/registerStorageMergeTree.cpp | 9 +++++---- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index 7b5a896015b..a4981fa5be3 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -68,8 +68,14 @@ String Macros::expand(const String & s, res += database_name; else if (macro_name == "table" && !table_name.empty()) res += table_name; - else if (macro_name == "uuid" && uuid != UUIDHelpers::Nil) + else if (macro_name == "uuid") + { + if (uuid == UUIDHelpers::Nil) + throw Exception("Macro 'uuid' and empty arguments of ReplicatedMergeTree " + "are supported only for ON CLUSTER queries with Atomic database engine", + ErrorCodes::SYNTAX_ERROR); res += toString(uuid); + } else throw Exception("No macro '" + macro_name + "' in config while processing substitutions in '" + s + "' at '" diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index b0c422bd79f..d3af3942428 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -395,9 +395,10 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) { - bool has_arguments = arg_num + 2 <= arg_cnt && engine_args[arg_num]->as() && engine_args[arg_num + 1]->as(); + bool has_arguments = arg_num + 2 <= arg_cnt; + bool has_valid_arguments = has_arguments && engine_args[arg_num]->as() && engine_args[arg_num + 1]->as(); - if (has_arguments) + if (has_valid_arguments) { const auto * ast = engine_args[arg_num]->as(); if (ast && ast->value.getType() == Field::Types::String) @@ -420,7 +421,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) "No replica name in config" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::NO_REPLICA_NAME_GIVEN); ++arg_num; } - else if (is_extended_storage_def) + else if (is_extended_storage_def && !has_arguments) { /// Try use default values if arguments are not specified. /// It works for ON CLUSTER queries when database engine is Atomic and there are {shard} and {replica} in config. @@ -428,7 +429,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) replica_name = "{replica}"; /// TODO maybe use hostname if {replica} is not defined? } else - throw Exception("Expected zookeper_path and replica_name arguments", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Expected two string literal arguments: zookeper_path and replica_name", ErrorCodes::BAD_ARGUMENTS); /// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries bool is_on_cluster = args.local_context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; From ca2a33008b291bc5d1507b568ac31d588a6aa3d8 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 2 Sep 2020 19:42:24 +0300 Subject: [PATCH 110/341] faster --- docker/test/performance-comparison/eqmed.sql | 4 ++-- docker/test/performance-comparison/perf.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/performance-comparison/eqmed.sql b/docker/test/performance-comparison/eqmed.sql index f7f8d6ac40d..139f0758798 100644 --- a/docker/test/performance-comparison/eqmed.sql +++ b/docker/test/performance-comparison/eqmed.sql @@ -8,7 +8,7 @@ select from ( -- quantiles of randomization distributions - select quantileExactForEach(0.999)( + select quantileExactForEach(0.99)( arrayMap(x, y -> abs(x - y), metrics_by_label[1], metrics_by_label[2]) as d ) threshold ---- uncomment to see what the distribution is really like @@ -33,7 +33,7 @@ from -- strip the query away before the join -- it might be several kB long; (select metrics, run, version from table) no_query, -- duplicate input measurements into many virtual runs - numbers(1, 100000) nn + numbers(1, 10000) nn -- for each virtual run, randomly reorder measurements order by virtual_run, rand() ) virtual_runs diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index e1476d9aeb4..05e89c9e44c 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -20,7 +20,7 @@ parser = argparse.ArgumentParser(description='Run performance test.') parser.add_argument('file', metavar='FILE', type=argparse.FileType('r', encoding='utf-8'), nargs=1, help='test description file') parser.add_argument('--host', nargs='*', default=['localhost'], help="Server hostname(s). Corresponds to '--port' options.") parser.add_argument('--port', nargs='*', default=[9000], help="Server port(s). Corresponds to '--host' options.") -parser.add_argument('--runs', type=int, default=int(os.environ.get('CHPC_RUNS', 13)), help='Number of query runs per server. Defaults to CHPC_RUNS environment variable.') +parser.add_argument('--runs', type=int, default=int(os.environ.get('CHPC_RUNS', 7)), help='Number of query runs per server. Defaults to CHPC_RUNS environment variable.') parser.add_argument('--long', action='store_true', help='Do not skip the tests tagged as long.') parser.add_argument('--print-queries', action='store_true', help='Print test queries and exit.') parser.add_argument('--print-settings', action='store_true', help='Print test settings and exit.') From 26348ad0143f881c8d14e41e0c80d706614ab110 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 10 Sep 2020 18:48:39 +0300 Subject: [PATCH 111/341] fixup --- docker/test/performance-comparison/report.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 1003a6d0e1a..b3f8ef01138 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -372,7 +372,7 @@ if args.report == 'main': 'New, s', # 1 'Ratio of speedup (-) or slowdown (+)', # 2 'Relative difference (new − old) / old', # 3 - 'p < 0.001 threshold', # 4 + 'p < 0.01 threshold', # 4 # Failed # 5 'Test', # 6 '#', # 7 @@ -416,7 +416,7 @@ if args.report == 'main': 'Old, s', #0 'New, s', #1 'Relative difference (new - old)/old', #2 - 'p < 0.001 threshold', #3 + 'p < 0.01 threshold', #3 # Failed #4 'Test', #5 '#', #6 @@ -649,7 +649,7 @@ elif args.report == 'all-queries': 'New, s', #3 'Ratio of speedup (-) or slowdown (+)', #4 'Relative difference (new − old) / old', #5 - 'p < 0.001 threshold', #6 + 'p < 0.01 threshold', #6 'Test', #7 '#', #8 'Query', #9 From 397ebdb855c75be3df9f6a007a4e2cf3d0321bb5 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Thu, 10 Sep 2020 08:55:39 -0700 Subject: [PATCH 112/341] remove sleep from unit test --- .../configs/asynchronous_metrics_update_period_s.xml | 2 +- tests/queries/0_stateless/01473_event_time_microseconds.sql | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml b/tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml index 840c19f03a6..b658b0436b1 100644 --- a/tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml +++ b/tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml @@ -1,3 +1,3 @@ - 2 + 1 diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index dd79a735d8f..138df77ffec 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -7,7 +7,6 @@ set log_queries = 1; select '01473_metric_log_table_event_start_time_microseconds_test'; system flush logs; -SELECT sleep(3) Format Null; -- query assumes that the event_time field is accurate. WITH ( ( From 964c18241cabe7566f2e639190c143d8f9c7ca2b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 10 Sep 2020 19:04:53 +0300 Subject: [PATCH 113/341] Added test from #13829 --- tests/queries/0_stateless/01492_array_join_crash_13829.reference | 0 tests/queries/0_stateless/01492_array_join_crash_13829.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/01492_array_join_crash_13829.reference create mode 100644 tests/queries/0_stateless/01492_array_join_crash_13829.sql diff --git a/tests/queries/0_stateless/01492_array_join_crash_13829.reference b/tests/queries/0_stateless/01492_array_join_crash_13829.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01492_array_join_crash_13829.sql b/tests/queries/0_stateless/01492_array_join_crash_13829.sql new file mode 100644 index 00000000000..9e11c3b69d4 --- /dev/null +++ b/tests/queries/0_stateless/01492_array_join_crash_13829.sql @@ -0,0 +1 @@ +SELECT NULL = countEqual(materialize([arrayJoin([NULL, NULL, NULL]), NULL AS x, arrayJoin([255, 1025, NULL, NULL]), arrayJoin([2, 1048576, NULL, NULL])]), materialize(x)) format Null; From 3b3b7bd5e790ec6ef0dff61793c2790167ea3469 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Thu, 10 Sep 2020 09:09:48 -0700 Subject: [PATCH 114/341] minor changes to comment --- src/Storages/StorageReplicatedMergeTree.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 908112c17ff..feb2f95849c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -226,10 +226,10 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( /// This unclean state will hinder table creation on any retries and will /// complain that the Directory for table already exists. /// - /// To acheive a clean state on failed table creations, catch this error if - /// the excaption is of type Poco::Exception and call dropIfEmpty() method, - /// then proceed throwing the exception. Without this, the Directory for the - /// tables need to be manually deleted before retrying the CreateQuery. + /// To achieve a clean state on failed table creations, catch this error and + /// call dropIfEmpty() method only if the operation isn't ATTACH then proceed + /// throwing the exception. Without this, the Directory for the tables need + /// to be manually deleted before retrying the CreateQuery. try { current_zookeeper = global_context.getZooKeeper(); From 15bdb6048e610fb5787e581b7fa346bd7512e109 Mon Sep 17 00:00:00 2001 From: Artem Hnilov Date: Thu, 10 Sep 2020 16:25:07 +0000 Subject: [PATCH 115/341] Function formatReadableQuantity() added. --- src/Functions/FunctionsFormatting.cpp | 1 + src/Functions/FunctionsFormatting.h | 76 +++++++++++++++++++++++++++ 2 files changed, 77 insertions(+) diff --git a/src/Functions/FunctionsFormatting.cpp b/src/Functions/FunctionsFormatting.cpp index 7582e234622..1187749aa2d 100644 --- a/src/Functions/FunctionsFormatting.cpp +++ b/src/Functions/FunctionsFormatting.cpp @@ -9,6 +9,7 @@ void registerFunctionsFormatting(FunctionFactory & factory) { factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/FunctionsFormatting.h b/src/Functions/FunctionsFormatting.h index e11254e2775..da22babec38 100644 --- a/src/Functions/FunctionsFormatting.h +++ b/src/Functions/FunctionsFormatting.h @@ -202,4 +202,80 @@ private: } }; + +class FunctionFormatReadableQuantity : public IFunction +{ +public: + static constexpr auto name = "formatReadableQuantity"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 1; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const IDataType & type = *arguments[0]; + + if (!isNativeNumber(type)) + throw Exception("Cannot format " + type.getName() + " as quantity", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override + { + if (!(executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result))) + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + +private: + template + bool executeType(Block & block, const ColumnNumbers & arguments, size_t result) const + { + if (const ColumnVector * col_from = checkAndGetColumn>(block.getByPosition(arguments[0]).column.get())) + { + auto col_to = ColumnString::create(); + + const typename ColumnVector::Container & vec_from = col_from->getData(); + ColumnString::Chars & data_to = col_to->getChars(); + ColumnString::Offsets & offsets_to = col_to->getOffsets(); + size_t size = vec_from.size(); + data_to.resize(size * 2); + offsets_to.resize(size); + + WriteBufferFromVector buf_to(data_to); + + for (size_t i = 0; i < size; ++i) + { + formatReadableQuantity(static_cast(vec_from[i]), buf_to); + writeChar(0, buf_to); + offsets_to[i] = buf_to.count(); + } + + buf_to.finalize(); + block.getByPosition(result).column = std::move(col_to); + return true; + } + + return false; + } +}; + } From 6e0d2a427c4e1a3145d2ac33d6cc94040ffd95dc Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Thu, 10 Sep 2020 19:30:11 +0300 Subject: [PATCH 116/341] add another test --- .../01460_line_as_string_format.reference | 2 ++ .../01460_line_as_string_format.sh | 24 ++++++++++++++----- 2 files changed, 20 insertions(+), 6 deletions(-) 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 dec67eb2e0a..c795fba4309 100644 --- a/tests/queries/0_stateless/01460_line_as_string_format.reference +++ b/tests/queries/0_stateless/01460_line_as_string_format.reference @@ -4,3 +4,5 @@ "array" : [1, 2, 3], Finally implement this new feature. +42 ClickHouse +42 ClickHouse is a `fast` #open-source# (OLAP) database "management" :system: diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sh b/tests/queries/0_stateless/01460_line_as_string_format.sh index a985bc207a8..60a960d2bf8 100755 --- a/tests/queries/0_stateless/01460_line_as_string_format.sh +++ b/tests/queries/0_stateless/01460_line_as_string_format.sh @@ -3,17 +3,29 @@ 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"; +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS line_as_string1"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE line_as_string1(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"; +Finally implement this new feature.' | $CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string1 FORMAT LineAsString"; -$CLICKHOUSE_CLIENT --query="SELECT * FROM line_as_string"; -$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string" +$CLICKHOUSE_CLIENT --query="SELECT * FROM line_as_string1"; +$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string1" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS line_as_string2"; +$CLICKHOUSE_CLIENT --query="create table line_as_string2( + a UInt64 default 42, + b String materialized toString(a), + c String +) engine=MergeTree() order by tuple();"; + +$CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string2(c) values ('ClickHouse')"; + +echo 'ClickHouse is a `fast` #open-source# (OLAP) 'database' "management" :system:' | $CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string2(c) FORMAT LineAsString"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM line_as_string2 order by c"; +$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string2" From d0c259974d1486dc90ae29958a3f1dd196052961 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Thu, 10 Sep 2020 19:35:18 +0300 Subject: [PATCH 117/341] style check --- .../Formats/Impl/LineAsStringRowInputFormat.cpp | 15 ++++++--------- src/Processors/ya.make | 2 +- 2 files changed, 7 insertions(+), 10 deletions(-) diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index 27bc71d764d..befc635386f 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -8,8 +8,7 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; - extern const int INCORRECT_DATA; + extern const int INCORRECT_QUERY; } LineAsStringRowInputFormat::LineAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) : @@ -17,7 +16,7 @@ LineAsStringRowInputFormat::LineAsStringRowInputFormat(const Block & header_, Re { 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); + throw Exception("This input format is only suitable for tables with a single column of type String.", ErrorCodes::INCORRECT_QUERY); } } @@ -39,7 +38,8 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) { pos = find_first_symbols<'\n', '\\'>(buf.position(), buf.buffer().end()); buf.position() = pos; - if (buf.position() == buf.buffer().end()) { + if (buf.position() == buf.buffer().end()) + { over = true; break; } @@ -48,14 +48,11 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) newline = false; } else if (*buf.position() == '\\') - { + { ++buf.position(); if (!buf.eof()) - { - ++buf.position(); - } + ++buf.position(); } - } buf.makeContinuousMemoryFromCheckpointToPos(); diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 8fe4e651ade..c69d274a70e 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -25,7 +25,6 @@ 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 @@ -34,6 +33,7 @@ SRCS( Formats/Impl/JSONEachRowRowOutputFormat.cpp Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp Formats/Impl/JSONRowOutputFormat.cpp + Formats/Impl/LineAsStringRowInputFormat.cpp Formats/Impl/MarkdownRowOutputFormat.cpp Formats/Impl/MsgPackRowInputFormat.cpp Formats/Impl/MsgPackRowOutputFormat.cpp From 4e197cf069c18cafb55b5fa22499483d8f19666d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Sep 2020 19:41:46 +0300 Subject: [PATCH 118/341] Remove unused headers --- base/common/types.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/base/common/types.h b/base/common/types.h index 682fe94366c..a02398a3365 100644 --- a/base/common/types.h +++ b/base/common/types.h @@ -1,8 +1,6 @@ #pragma once -#include #include -#include #include #include From 45340c701dc517b29db5a1047c306f88ba891722 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 10 Sep 2020 19:49:57 +0300 Subject: [PATCH 119/341] changelog for 20.8 --- CHANGELOG.md | 148 ++++++++++++++++++++++ utils/simple-backport/backport.sh | 7 +- utils/simple-backport/format-changelog.py | 2 +- 3 files changed, 154 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 345ee2c6213..f3266520eb1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,151 @@ +## ClickHouse release 20.8 + +### ClickHouse release v20.8.2.3-stable, 2020-09-08 + +#### Backward Incompatible Change + +* Now `OPTIMIZE FINAL` query doesn't recalculate TTL for parts that were added before TTL was created. Use `ALTER TABLE ... MATERIALIZE TTL` once to calculate them, after that `OPTIMIZE FINAL` will evaluate TTL's properly. This behavior never worked for replicated tables. [#14220](https://github.com/ClickHouse/ClickHouse/pull/14220) ([alesapin](https://github.com/alesapin)). +* Extend `parallel_distributed_insert_select` setting, adding an option to run `INSERT` into local table. The setting changes type from `Bool` to `UInt64`, so the values `false` and `true` are no longer supported. If you have these values in server configuration, the server will not start. Please replace them with `0` and `1`, respectively. [#14060](https://github.com/ClickHouse/ClickHouse/pull/14060) ([Azat Khuzhin](https://github.com/azat)). +* Remove support for the `ODBCDriver` input/output format. This was a deprecated format once used for communication with the ClickHouse ODBC driver, now long superseded by the `ODBCDriver2` format. Resolves [#13629](https://github.com/ClickHouse/ClickHouse/issues/13629). [#13847](https://github.com/ClickHouse/ClickHouse/pull/13847) ([hexiaoting](https://github.com/hexiaoting)). + +#### New Feature + +* Add `countDigits(x)` function that count number of decimal digits in integer or decimal column. Add `isDecimalOverflow(d, [p])` function that checks if the value in Decimal column is out of its (or specified) precision. [#14151](https://github.com/ClickHouse/ClickHouse/pull/14151) ([Artem Zuikov](https://github.com/4ertus2)). +* Add setting `min_index_granularity_bytes` that protects against accidentally creating a table with very low `index_granularity_bytes` setting. [#14139](https://github.com/ClickHouse/ClickHouse/pull/14139) ([Bharat Nallan](https://github.com/bharatnc)). +* Add the ability to specify `Default` compression codec for columns that correspond to settings specified in `config.xml`. Implements: [#9074](https://github.com/ClickHouse/ClickHouse/issues/9074). [#14049](https://github.com/ClickHouse/ClickHouse/pull/14049) ([alesapin](https://github.com/alesapin)). +* Added `date_trunc` function that truncates a date/time value to a specified date/time part. [#13888](https://github.com/ClickHouse/ClickHouse/pull/13888) ([Vladimir Golovchenko](https://github.com/vladimir-golovchenko)). +* Add `time_zones` table. [#13880](https://github.com/ClickHouse/ClickHouse/pull/13880) ([Bharat Nallan](https://github.com/bharatnc)). +* Add function `defaultValueOfTypeName` that returns the default value for a given type. [#13877](https://github.com/ClickHouse/ClickHouse/pull/13877) ([hcz](https://github.com/hczhcz)). +* Add `quantileExactLow` and `quantileExactHigh` implementations with respective aliases for `medianExactLow` and `medianExactHigh`. [#13818](https://github.com/ClickHouse/ClickHouse/pull/13818) ([Bharat Nallan](https://github.com/bharatnc)). +* Add function `normalizeQuery` that replaces literals, sequences of literals and complex aliases with placeholders. Add function `normalizedQueryHash` that returns identical 64bit hash values for similar queries. It helps to analyze query log. This closes [#11271](https://github.com/ClickHouse/ClickHouse/issues/11271). [#13816](https://github.com/ClickHouse/ClickHouse/pull/13816) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add new optional section to the main config. [#13425](https://github.com/ClickHouse/ClickHouse/pull/13425) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add `ALTER SAMPLE BY` statement that allows to change table sample clause. [#13280](https://github.com/ClickHouse/ClickHouse/pull/13280) ([Amos Bird](https://github.com/amosbird)). +* Function `position` now supports optional `start_pos` argument. [#13237](https://github.com/ClickHouse/ClickHouse/pull/13237) ([vdimir](https://github.com/vdimir)). +* Add types `Int128`, `Int256`, `UInt256` and related functions for them. Extend Decimals with Decimal256 (precision up to 76 digits). New types are under the setting `allow_experimental_bigint_types`. [#13097](https://github.com/ClickHouse/ClickHouse/pull/13097) ([Artem Zuikov](https://github.com/4ertus2)). +* Support Kerberos authentication in Kafka, using `krb5` and `cyrus-sasl` libraries. [#12771](https://github.com/ClickHouse/ClickHouse/pull/12771) ([Ilya Golshtein](https://github.com/ilejn)). +* Support `MaterializeMySQL` database engine. Implements [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#10851](https://github.com/ClickHouse/ClickHouse/pull/10851) ([Winter Zhang](https://github.com/zhang2014)). + +#### Bug Fix + +* Check for array size overflow in `topK` aggregate function. Without this check the user may send a query with carefully crafter parameters that will lead to server crash. This closes [#14452](https://github.com/ClickHouse/ClickHouse/issues/14452). [#14467](https://github.com/ClickHouse/ClickHouse/pull/14467) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bug which leads to wrong merges assignment if table has partitions with a single part. [#14444](https://github.com/ClickHouse/ClickHouse/pull/14444) ([alesapin](https://github.com/alesapin)). +* Stop query execution if exception happened in `PipelineExecutor` itself. This could prevent rare possible query hung. Continuation of [#14334](https://github.com/ClickHouse/ClickHouse/issues/14334). [#14402](https://github.com/ClickHouse/ClickHouse/pull/14402) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Stop query execution if exception happened in `PipelineExecutor` itself. This could prevent rare possible query hung. [#14334](https://github.com/ClickHouse/ClickHouse/pull/14334) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix crash during `ALTER` query for table which was created `AS table_function`. Fixes [#14212](https://github.com/ClickHouse/ClickHouse/issues/14212). [#14326](https://github.com/ClickHouse/ClickHouse/pull/14326) ([alesapin](https://github.com/alesapin)). +* Fix exception during ALTER LIVE VIEW query with REFRESH command. [#14320](https://github.com/ClickHouse/ClickHouse/pull/14320) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix QueryPlan lifetime (for EXPLAIN PIPELINE graph=1) for queries with nested interpreter. [#14315](https://github.com/ClickHouse/ClickHouse/pull/14315) ([Azat Khuzhin](https://github.com/azat)). +* Fix segfault in `clickhouse-odbc-bridge` during schema fetch from some external sources. This PR fixes https://github.com/ClickHouse/ClickHouse/issues/13861. [#14267](https://github.com/ClickHouse/ClickHouse/pull/14267) ([Vitaly Baranov](https://github.com/vitlibar)). +* Disallows `CODEC` on `ALIAS` column type. Fixes [#13911](https://github.com/ClickHouse/ClickHouse/issues/13911). [#14263](https://github.com/ClickHouse/ClickHouse/pull/14263) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix handling of empty transactions in `MaterializeMySQL` database engine. This fixes [#14235](https://github.com/ClickHouse/ClickHouse/issues/14235). [#14253](https://github.com/ClickHouse/ClickHouse/pull/14253) ([BohuTANG](https://github.com/BohuTANG)). +* fixes [#14231](https://github.com/ClickHouse/ClickHouse/issues/14231) fix wrong lexer in MaterializeMySQL database engine dump stage. [#14232](https://github.com/ClickHouse/ClickHouse/pull/14232) ([Winter Zhang](https://github.com/zhang2014)). +* Fix crash in mark inclusion search introduced in https://github.com/ClickHouse/ClickHouse/pull/12277 . [#14225](https://github.com/ClickHouse/ClickHouse/pull/14225) ([Amos Bird](https://github.com/amosbird)). +* Fixed incorrect sorting order if LowCardinality column. This fixes [#13958](https://github.com/ClickHouse/ClickHouse/issues/13958). [#14223](https://github.com/ClickHouse/ClickHouse/pull/14223) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix creation of tables with named tuples. This fixes [#13027](https://github.com/ClickHouse/ClickHouse/issues/13027). [#14143](https://github.com/ClickHouse/ClickHouse/pull/14143) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix formatting of minimal negative decimal numbers. This fixes https://github.com/ClickHouse/ClickHouse/issues/14111. [#14119](https://github.com/ClickHouse/ClickHouse/pull/14119) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* When waiting for a dictionary update to complete, use the timeout specified by `query_wait_timeout_milliseconds` setting instead of a hard-coded value. [#14105](https://github.com/ClickHouse/ClickHouse/pull/14105) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix DistributedFilesToInsert metric (zeroed when it should not). [#14095](https://github.com/ClickHouse/ClickHouse/pull/14095) ([Azat Khuzhin](https://github.com/azat)). +* Fix pointInPolygon with const 2d array as polygon. [#14079](https://github.com/ClickHouse/ClickHouse/pull/14079) ([Alexey Ilyukhov](https://github.com/livace)). +* Fixed wrong mount point in extra info for `Poco::Exception: no space left on device`. [#14050](https://github.com/ClickHouse/ClickHouse/pull/14050) ([tavplubix](https://github.com/tavplubix)). +* Fix GRANT ALL statement when executed on a non-global level. [#13987](https://github.com/ClickHouse/ClickHouse/pull/13987) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix parser to reject create table as table function with engine. [#13940](https://github.com/ClickHouse/ClickHouse/pull/13940) ([hcz](https://github.com/hczhcz)). +* Fix wrong results in select queries with `DISTINCT` keyword in case `optimize_duplicate_order_by_and_distinct` setting is enabled. [#13925](https://github.com/ClickHouse/ClickHouse/pull/13925) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixed potential deadlock when renaming `Distributed` table. [#13922](https://github.com/ClickHouse/ClickHouse/pull/13922) ([tavplubix](https://github.com/tavplubix)). +* Fix incorrect sorting for `FixedString` columns. Fixes [#13182](https://github.com/ClickHouse/ClickHouse/issues/13182). [#13887](https://github.com/ClickHouse/ClickHouse/pull/13887) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix topK/topKWeighted merge (with non-default parameters). [#13817](https://github.com/ClickHouse/ClickHouse/pull/13817) ([Azat Khuzhin](https://github.com/azat)). +* Fix reading from MergeTree table with INDEX of type SET fails when comparing against NULL. This fixes [#13686](https://github.com/ClickHouse/ClickHouse/issues/13686). [#13793](https://github.com/ClickHouse/ClickHouse/pull/13793) ([Amos Bird](https://github.com/amosbird)). +* Fix arrayJoin() capturing in lambda (LOGICAL_ERROR). [#13792](https://github.com/ClickHouse/ClickHouse/pull/13792) ([Azat Khuzhin](https://github.com/azat)). +* Fix step overflow in range(). [#13790](https://github.com/ClickHouse/ClickHouse/pull/13790) ([Azat Khuzhin](https://github.com/azat)). +* Fixed `Directory not empty` error when concurrently executing `DROP DATABASE` and `CREATE TABLE`. [#13756](https://github.com/ClickHouse/ClickHouse/pull/13756) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add range check for h3KRing function. This fixes [#13633](https://github.com/ClickHouse/ClickHouse/issues/13633). [#13752](https://github.com/ClickHouse/ClickHouse/pull/13752) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix race condition between DETACH and background merges. Parts may revive after detach. This is continuation of [#8602](https://github.com/ClickHouse/ClickHouse/issues/8602) that did not fix the issue but introduced a test that started to fail in very rare cases, demonstrating the issue. [#13746](https://github.com/ClickHouse/ClickHouse/pull/13746) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix logging Settings.Names/Values when log_queries_min_type > QUERY_START. [#13737](https://github.com/ClickHouse/ClickHouse/pull/13737) ([Azat Khuzhin](https://github.com/azat)). +* Fixes /replicas_status endpoint response status code when verbose=1. [#13722](https://github.com/ClickHouse/ClickHouse/pull/13722) ([javi santana](https://github.com/javisantana)). +* Fix incorrect message in `clickhouse-server.init` while checking user and group. [#13711](https://github.com/ClickHouse/ClickHouse/pull/13711) ([ylchou](https://github.com/ylchou)). +* Fix visible data clobbering by progress bar in client in interactive mode. This fixes [#12562](https://github.com/ClickHouse/ClickHouse/issues/12562) and [#13369](https://github.com/ClickHouse/ClickHouse/issues/13369) and [#13584](https://github.com/ClickHouse/ClickHouse/issues/13584) and fixes [#12964](https://github.com/ClickHouse/ClickHouse/issues/12964). [#13691](https://github.com/ClickHouse/ClickHouse/pull/13691) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Do not optimize any(arrayJoin()) -> arrayJoin() under optimize_move_functions_out_of_any. [#13681](https://github.com/ClickHouse/ClickHouse/pull/13681) ([Azat Khuzhin](https://github.com/azat)). +* Fix crash in JOIN with StorageMerge and `set enable_optimize_predicate_expression=1`. [#13679](https://github.com/ClickHouse/ClickHouse/pull/13679) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix typo in error message about `The value of 'number_of_free_entries_in_pool_to_lower_max_size_of_merge' setting`. [#13678](https://github.com/ClickHouse/ClickHouse/pull/13678) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Concurrent `ALTER ... REPLACE/MOVE PARTITION ...` queries might cause deadlock. It's fixed. [#13626](https://github.com/ClickHouse/ClickHouse/pull/13626) ([tavplubix](https://github.com/tavplubix)). +* Fixed the behaviour when sometimes cache-dictionary returned default value instead of present value from source. [#13624](https://github.com/ClickHouse/ClickHouse/pull/13624) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix secondary indices corruption in compact parts. [#13538](https://github.com/ClickHouse/ClickHouse/pull/13538) ([Anton Popov](https://github.com/CurtizJ)). +* Fix premature `ON CLUSTER` timeouts for queries that must be executed on a single replica. Fixes [#6704](https://github.com/ClickHouse/ClickHouse/issues/6704), [#7228](https://github.com/ClickHouse/ClickHouse/issues/7228), [#13361](https://github.com/ClickHouse/ClickHouse/issues/13361), [#11884](https://github.com/ClickHouse/ClickHouse/issues/11884). [#13450](https://github.com/ClickHouse/ClickHouse/pull/13450) ([alesapin](https://github.com/alesapin)). +* Fix wrong code in function `netloc`. This fixes [#13335](https://github.com/ClickHouse/ClickHouse/issues/13335). [#13446](https://github.com/ClickHouse/ClickHouse/pull/13446) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix possible race in `StorageMemory`. https://clickhouse-test-reports.s3.yandex.net/0/9cac8a7244063d2092ad25d45502611e18d3749c/stress_test_(thread)/stderr.log Have no idea how to write a test. [#13416](https://github.com/ClickHouse/ClickHouse/pull/13416) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix missing or excessive headers in `TSV/CSVWithNames` formats. This fixes [#12504](https://github.com/ClickHouse/ClickHouse/issues/12504). [#13343](https://github.com/ClickHouse/ClickHouse/pull/13343) ([Azat Khuzhin](https://github.com/azat)). +* Fix parsing row policies from users.xml when names of databases or tables contain dots. This fixes https://github.com/ClickHouse/ClickHouse/issues/5779, https://github.com/ClickHouse/ClickHouse/issues/12527. [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix access to redis dictionary after connection was dropped once. It may happen with `cache` and `direct` dictionary layouts. [#13082](https://github.com/ClickHouse/ClickHouse/pull/13082) ([Anton Popov](https://github.com/CurtizJ)). +* Removed wrong auth access check when using ClickHouseDictionarySource to query remote tables. [#12756](https://github.com/ClickHouse/ClickHouse/pull/12756) ([sundyli](https://github.com/sundy-li)). +* subquery hash values are not enough to distinguish. https://github.com/ClickHouse/ClickHouse/issues/8333. [#8367](https://github.com/ClickHouse/ClickHouse/pull/8367) ([Amos Bird](https://github.com/amosbird)). + +#### Improvement + +* Now it's possible to `ALTER TABLE table_name FETCH PARTITION partition_expr FROM 'zk://:/path-in-zookeeper'`. It's useful for shipping data to new clusters. [#14155](https://github.com/ClickHouse/ClickHouse/pull/14155) ([Amos Bird](https://github.com/amosbird)). +* Slightly better performance of Memory table if it was constructed from a huge number of very small blocks (that's unlikely). Author of the idea: [Mark Papadakis](https://github.com/markpapadakis). Closes [#14043](https://github.com/ClickHouse/ClickHouse/issues/14043). [#14056](https://github.com/ClickHouse/ClickHouse/pull/14056) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Conditional aggregate functions (for example: `avgIf`, `sumIf`, `maxIf`) should return `NULL` when miss rows and use nullable arguments. [#13964](https://github.com/ClickHouse/ClickHouse/pull/13964) ([Winter Zhang](https://github.com/zhang2014)). +* Increase limit in -Resample combinator to 1M. [#13947](https://github.com/ClickHouse/ClickHouse/pull/13947) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Corrected an error in AvroConfluent format that caused the Kafka table engine to stop processing messages when an abnormally small, malformed, message was received. [#13941](https://github.com/ClickHouse/ClickHouse/pull/13941) ([Gervasio Varela](https://github.com/gervarela)). +* Fix wrong error for long queries. It was possible to get syntax error other than `Max query size exceeded` for correct query. [#13928](https://github.com/ClickHouse/ClickHouse/pull/13928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Better error message for null value of TabSeparatedRow format. [#13906](https://github.com/ClickHouse/ClickHouse/pull/13906) ([jiang tao](https://github.com/tomjiang1987)). +* Function `arrayCompact` will compare NaNs bitwise if the type of array elements is Float32/Float64. In previous versions NaNs were always not equal if the type of array elements is Float32/Float64 and were always equal if the type is more complex, like Nullable(Float64). This closes [#13857](https://github.com/ClickHouse/ClickHouse/issues/13857). [#13868](https://github.com/ClickHouse/ClickHouse/pull/13868) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix data race in `lgamma` function. This race was caught only in `tsan`, no side effects a really happened. [#13842](https://github.com/ClickHouse/ClickHouse/pull/13842) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* 1. Add [GTID-Based Replication](https://dev.mysql.com/doc/refman/5.7/en/replication-gtids-concepts.html), it works even when replication topology changes, and supported/prefered in MySQL 5.6/5.7/8.0 2. Add BIT/SET filed type supports 3. Fix up varchar type meta length bug. [#13820](https://github.com/ClickHouse/ClickHouse/pull/13820) ([BohuTANG](https://github.com/BohuTANG)). +* Avoid too slow queries when arrays are manipulated as fields. Throw exception instead. [#13753](https://github.com/ClickHouse/ClickHouse/pull/13753) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added Redis requirepass authorization. [#13688](https://github.com/ClickHouse/ClickHouse/pull/13688) ([Ivan Torgashov](https://github.com/it1804)). +* Add MergeTree Write-Ahead-Log(WAL) dump tool. [#13640](https://github.com/ClickHouse/ClickHouse/pull/13640) ([BohuTANG](https://github.com/BohuTANG)). +* In previous versions `lcm` function may produce assertion violation in debug build if called with specifically crafted arguments. This fixes [#13368](https://github.com/ClickHouse/ClickHouse/issues/13368). [#13510](https://github.com/ClickHouse/ClickHouse/pull/13510) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Provide monotonicity for `toDate/toDateTime` functions in more cases. Now the input arguments are saturated more naturally and provides better monotonicity. [#13497](https://github.com/ClickHouse/ClickHouse/pull/13497) ([Amos Bird](https://github.com/amosbird)). +* Support compound identifiers for custom settings. [#13496](https://github.com/ClickHouse/ClickHouse/pull/13496) ([Vitaly Baranov](https://github.com/vitlibar)). +* Move parts from DIskLocal to DiskS3 in parallel. [#13459](https://github.com/ClickHouse/ClickHouse/pull/13459) ([Pavel Kovalenko](https://github.com/Jokser)). +* Enable mixed granularity parts by default. [#13449](https://github.com/ClickHouse/ClickHouse/pull/13449) ([alesapin](https://github.com/alesapin)). +* Proper remote host checking in S3 redirects (security-related thing). [#13404](https://github.com/ClickHouse/ClickHouse/pull/13404) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Add QueryTimeMicroseconds, SelectQueryTimeMicroseconds and InsertQueryTimeMicroseconds to system.events. [#13336](https://github.com/ClickHouse/ClickHouse/pull/13336) ([ianton-ru](https://github.com/ianton-ru)). +* Fix assert when decimal has too large negative exponent. Fixes [#13188](https://github.com/ClickHouse/ClickHouse/issues/13188). [#13228](https://github.com/ClickHouse/ClickHouse/pull/13228) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added cache layer for DiskS3 (cache to local disk mark and index files). [#13076](https://github.com/ClickHouse/ClickHouse/pull/13076) ([Pavel Kovalenko](https://github.com/Jokser)). + +#### Performance Improvement + +* Slightly optimize very short queries with LowCardinality. [#14129](https://github.com/ClickHouse/ClickHouse/pull/14129) ([Anton Popov](https://github.com/CurtizJ)). +* Enable parallel INSERTs for table engines `Null`, `Memory`, `Distributed` and `Buffer`. [#14120](https://github.com/ClickHouse/ClickHouse/pull/14120) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fail fast if `max_rows_to_read` limit is exceeded on parts scan. The motivation behind this change is to skip ranges scan for all selected parts if it is clear that `max_rows_to_read` is already exceeded. The change is quite noticeable for queries over big number of parts. [#13677](https://github.com/ClickHouse/ClickHouse/pull/13677) ([Roman Khavronenko](https://github.com/hagen1778)). +* Slightly improve performance of aggregation by UInt8/UInt16 keys. [#13099](https://github.com/ClickHouse/ClickHouse/pull/13099) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Optimize `has()`, `indexOf()` and `countEqual()` functions for `Array(LowCardinality(T))` and constant right arguments. [#12550](https://github.com/ClickHouse/ClickHouse/pull/12550) ([myrrc](https://github.com/myrrc)). +* When performing trivial `INSERT SELECT` queries, automatically set `max_threads` to 1 or `max_insert_threads`, and set `max_block_size` to `min_insert_block_size_rows`. Related to [#5907](https://github.com/ClickHouse/ClickHouse/issues/5907). [#12195](https://github.com/ClickHouse/ClickHouse/pull/12195) ([flynn](https://github.com/ucasFL)). + +#### Build/Testing/Packaging Improvement + +* Actually there are no symlinks there, so `-type f` is enough ``` ~/workspace/ClickHouse/contrib/cctz/testdata/zoneinfo$ find . -type l -ls | wc -l 0 ``` Closes [#14209](https://github.com/ClickHouse/ClickHouse/issues/14209). [#14215](https://github.com/ClickHouse/ClickHouse/pull/14215) ([filimonov](https://github.com/filimonov)). +* Switch tests docker images to use test-base parent. [#14167](https://github.com/ClickHouse/ClickHouse/pull/14167) ([Ilya Yatsishin](https://github.com/qoega)). +* Add the ability to write js-style comments in skip_list.json. [#14159](https://github.com/ClickHouse/ClickHouse/pull/14159) ([alesapin](https://github.com/alesapin)). +* * Adding retry logic when bringing up docker-compose cluster * Increasing COMPOSE_HTTP_TIMEOUT. [#14112](https://github.com/ClickHouse/ClickHouse/pull/14112) ([vzakaznikov](https://github.com/vzakaznikov)). +* Enabled text-log in stress test to find more bugs. [#13855](https://github.com/ClickHouse/ClickHouse/pull/13855) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Testflows LDAP module: adding missing certificates and dhparam.pem for openldap4. [#13780](https://github.com/ClickHouse/ClickHouse/pull/13780) ([vzakaznikov](https://github.com/vzakaznikov)). +* ZooKeeper cannot work reliably in unit tests in CI infrastructure. Using unit tests for ZooKeeper interaction with real ZooKeeper is bad idea from the start (unit tests are not supposed to verify complex distributed systems). We already using integration tests for this purpose and they are better suited. [#13745](https://github.com/ClickHouse/ClickHouse/pull/13745) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added docker image for style check. Added style check that all docker and docker compose files are located in docker directory. [#13724](https://github.com/ClickHouse/ClickHouse/pull/13724) ([Ilya Yatsishin](https://github.com/qoega)). +* FIx cassandra build on Mac OS. [#13708](https://github.com/ClickHouse/ClickHouse/pull/13708) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix link error in shared build. [#13700](https://github.com/ClickHouse/ClickHouse/pull/13700) ([Amos Bird](https://github.com/amosbird)). +* Add a CMake option to fail configuration instead of auto-reconfiguration, enabled by default. [#13687](https://github.com/ClickHouse/ClickHouse/pull/13687) ([Konstantin](https://github.com/podshumok)). +* Updating LDAP user authentication suite to check that it works with RBAC. [#13656](https://github.com/ClickHouse/ClickHouse/pull/13656) ([vzakaznikov](https://github.com/vzakaznikov)). +* Expose version of embedded tzdata via TZDATA_VERSION in system.build_options. [#13648](https://github.com/ClickHouse/ClickHouse/pull/13648) ([filimonov](https://github.com/filimonov)). +* Removed `-DENABLE_CURL_CLIENT` for `contrib/aws`. [#13628](https://github.com/ClickHouse/ClickHouse/pull/13628) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Build ClickHouse with the most fresh tzdata from package repository. [#13623](https://github.com/ClickHouse/ClickHouse/pull/13623) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Increasing health-check timeouts for ClickHouse nodes and adding support to dump docker-compose logs if unhealthy containers found. [#13612](https://github.com/ClickHouse/ClickHouse/pull/13612) ([vzakaznikov](https://github.com/vzakaznikov)). +* Make sure https://github.com/ClickHouse/ClickHouse/issues/10977 is invalid. [#13539](https://github.com/ClickHouse/ClickHouse/pull/13539) ([Amos Bird](https://github.com/amosbird)). +* Enable Shellcheck in CI as a linter of .sh tests. This closes [#13168](https://github.com/ClickHouse/ClickHouse/issues/13168). [#13530](https://github.com/ClickHouse/ClickHouse/pull/13530) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix the remaining shellcheck notices. A preparation to enable Shellcheck. [#13529](https://github.com/ClickHouse/ClickHouse/pull/13529) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added `clickhouse install` script, that is useful if you only have a single binary. [#13528](https://github.com/ClickHouse/ClickHouse/pull/13528) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow to run `clickhouse` binary without configuration. [#13515](https://github.com/ClickHouse/ClickHouse/pull/13515) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Ensure that there is no copy-pasted GPL code. [#13514](https://github.com/ClickHouse/ClickHouse/pull/13514) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Enable check for typos in code with `codespell`. [#13513](https://github.com/ClickHouse/ClickHouse/pull/13513) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix typos in code with codespell. [#13511](https://github.com/ClickHouse/ClickHouse/pull/13511) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Skip PR's from robot-clickhouse. [#13489](https://github.com/ClickHouse/ClickHouse/pull/13489) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Move Dockerfiles from integration tests to `docker/test` directory. docker_compose files are available in `runner` docker container. Docker images are built in CI and not in integration tests. [#13448](https://github.com/ClickHouse/ClickHouse/pull/13448) ([Ilya Yatsishin](https://github.com/qoega)). + +#### Other + +* Create `system` database with `Atomic` engine by default. [#13680](https://github.com/ClickHouse/ClickHouse/pull/13680) ([tavplubix](https://github.com/tavplubix)). +* Fix readline so it dumps history to file now. [#13600](https://github.com/ClickHouse/ClickHouse/pull/13600) ([Amos Bird](https://github.com/amosbird)). + + ## ClickHouse release 20.7 ### ClickHouse release v20.7.2.30-stable, 2020-08-31 diff --git a/utils/simple-backport/backport.sh b/utils/simple-backport/backport.sh index 71920304d56..64f8e6004bf 100755 --- a/utils/simple-backport/backport.sh +++ b/utils/simple-backport/backport.sh @@ -4,7 +4,10 @@ set -e branch="$1" merge_base=$(git merge-base origin/master "origin/$branch") master_git_cmd=(git log "$merge_base..origin/master" --first-parent) -branch_git_cmd=(git log "$merge_base..origin/$branch" --first-parent) +# The history in back branches shouldn't be too crazy, and sometimes we have a PR +# that merges several backport commits there (3f2cba6824fddf31c30bde8c6f4f860572f4f580), +# so don't use --first-parent +branch_git_cmd=(git log "$merge_base..origin/$branch") # Make lists of PRs that were merged into each branch. Use first parent here, or else # we'll get weird things like seeing older master that was merged into a PR branch @@ -30,7 +33,7 @@ fi # Search for PR numbers in commit messages. First variant is normal merge, and second # variant is squashed. Next are some backport message variants. find_prs=(sed -n "s/^.*merg[eding]*.*#\([[:digit:]]\+\).*$/\1/Ip; - s/^.*(#\([[:digit:]]\+\))$/\1/p; + s/^.*#\([[:digit:]]\+\))$/\1/p; s/^.*back[- ]*port[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip; s/^.*cherry[- ]*pick[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip") diff --git a/utils/simple-backport/format-changelog.py b/utils/simple-backport/format-changelog.py index ccda88c6809..5dff4f1c5e8 100755 --- a/utils/simple-backport/format-changelog.py +++ b/utils/simple-backport/format-changelog.py @@ -93,7 +93,7 @@ for line in args.file: # Normalize category name for c in categories_preferred_order: - if fuzzywuzzy.fuzz.ratio(pr['category'], c) >= 90: + if fuzzywuzzy.fuzz.ratio(pr['category'].lower(), c.lower()) >= 90: pr['category'] = c break From 142026f4abc810426d4ebf195bb8316a83b6adff Mon Sep 17 00:00:00 2001 From: Artem Hnilov Date: Thu, 10 Sep 2020 16:54:52 +0000 Subject: [PATCH 120/341] 01492_format_readable_quantity test added --- .../01492_format_readable_quantity.reference | 50 +++++++++++++++++++ .../01492_format_readable_quantity.sql | 4 ++ 2 files changed, 54 insertions(+) create mode 100644 tests/queries/0_stateless/01492_format_readable_quantity.reference create mode 100644 tests/queries/0_stateless/01492_format_readable_quantity.sql diff --git a/tests/queries/0_stateless/01492_format_readable_quantity.reference b/tests/queries/0_stateless/01492_format_readable_quantity.reference new file mode 100644 index 00000000000..3736933b5ee --- /dev/null +++ b/tests/queries/0_stateless/01492_format_readable_quantity.reference @@ -0,0 +1,50 @@ +1.00 1.00 1.00 +2.72 2.00 2.00 +7.39 7.00 7.00 +20.09 20.00 20.00 +54.60 54.00 54.00 +148.41 148.00 148.00 +403.43 403.00 403.00 +1.10 thousand 1.10 thousand 1.10 thousand +2.98 thousand 2.98 thousand 2.98 thousand +8.10 thousand 8.10 thousand 8.10 thousand +22.03 thousand 22.03 thousand 22.03 thousand +59.87 thousand 59.87 thousand 59.87 thousand +162.75 thousand 162.75 thousand 162.75 thousand +442.41 thousand 442.41 thousand 442.41 thousand +1.20 million 1.20 million 1.20 million +3.27 million 3.27 million 3.27 million +8.89 million 8.89 million 8.89 million +24.15 million 24.15 million 24.15 million +65.66 million 65.66 million 65.66 million +178.48 million 178.48 million 178.48 million +485.17 million 485.17 million 485.17 million +1.32 billion 1.32 billion 1.32 billion +3.58 billion 3.58 billion -2.15 billion +9.74 billion 9.74 billion -2.15 billion +26.49 billion 26.49 billion -2.15 billion +72.00 billion 72.00 billion -2.15 billion +195.73 billion 195.73 billion -2.15 billion +532.05 billion 532.05 billion -2.15 billion +1.45 trillion 1.45 trillion -2.15 billion +3.93 trillion 3.93 trillion -2.15 billion +10.69 trillion 10.69 trillion -2.15 billion +29.05 trillion 29.05 trillion -2.15 billion +78.96 trillion 78.96 trillion -2.15 billion +214.64 trillion 214.64 trillion -2.15 billion +583.46 trillion 583.46 trillion -2.15 billion +1.59 quadrillion 1.59 quadrillion -2.15 billion +4.31 quadrillion 4.31 quadrillion -2.15 billion +11.72 quadrillion 11.72 quadrillion -2.15 billion +31.86 quadrillion 31.86 quadrillion -2.15 billion +86.59 quadrillion 86.59 quadrillion -2.15 billion +235.39 quadrillion 235.39 quadrillion -2.15 billion +639.84 quadrillion 639.84 quadrillion -2.15 billion +1739.27 quadrillion 1739.27 quadrillion -2.15 billion +4727.84 quadrillion 4727.84 quadrillion -2.15 billion +12851.60 quadrillion 12851.60 quadrillion -2.15 billion +34934.27 quadrillion 0.00 -2.15 billion +94961.19 quadrillion 0.00 -2.15 billion +258131.29 quadrillion 0.00 -2.15 billion +701673.59 quadrillion 0.00 -2.15 billion +1907346.57 quadrillion 0.00 -2.15 billion diff --git a/tests/queries/0_stateless/01492_format_readable_quantity.sql b/tests/queries/0_stateless/01492_format_readable_quantity.sql new file mode 100644 index 00000000000..c2ebcfc1a97 --- /dev/null +++ b/tests/queries/0_stateless/01492_format_readable_quantity.sql @@ -0,0 +1,4 @@ +WITH round(exp(number), 6) AS x, toUInt64(x) AS y, toInt32(x) AS z +SELECT formatReadableQuantity(x), formatReadableQuantity(y), formatReadableQuantity(z) +FROM system.numbers +LIMIT 50; From 8689797efc4d749f4cda139818b1a94caccfe628 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 10 Sep 2020 19:57:26 +0300 Subject: [PATCH 121/341] Run only some queries in perf tests, not all combos --- docker/test/performance-comparison/compare.sh | 17 ++++++++++++++++- docker/test/performance-comparison/perf.py | 12 ++++++++++-- 2 files changed, 26 insertions(+), 3 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 364e9994ab7..0b678024765 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -121,7 +121,7 @@ function run_tests then # Use the explicitly set path to directory with test files. test_prefix="$CHPC_TEST_PATH" - elif [ "$PR_TO_TEST" = "0" ] + elif [ "$PR_TO_TEST" == "0" ] then # When testing commits from master, use the older test files. This # allows the tests to pass even when we add new functions and tests for @@ -155,6 +155,20 @@ function run_tests test_files=$(ls "$test_prefix"/*.xml) fi + # For PRs, test only a subset of queries, and run them less times. + # If the corresponding environment variables are already set, keep + # those values. + if [ "$PR_TO_TEST" == "0" ] + then + CHPC_TEST_RUNS=${CHPC_RUNS:-7} + CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-15} + else + CHPC_TEST_RUNS=${CHPC_RUNS:-13} + CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-0} + fi + export CHPC_TEST_RUNS + export CHPC_MAX_QUERIES + # Determine which concurrent benchmarks to run. For now, the only test # we run as a concurrent benchmark is 'website'. Run it as benchmark if we # are also going to run it as a normal test. @@ -187,6 +201,7 @@ function run_tests # the grep is to filter out set -x output and keep only time output { \ time "$script_dir/perf.py" --host localhost localhost --port 9001 9002 \ + --runs "$CHPC_RUNS" --max-queries "$CHPC_MAX_QUERIES" \ -- "$test" > "$test_name-raw.tsv" 2> "$test_name-err.log" ; \ } 2>&1 >/dev/null | grep -v ^+ >> "wall-clock-times.tsv" \ || echo "Test $test_name failed with error code $?" >> "$test_name-err.log" diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index e1476d9aeb4..64314c129b5 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -7,6 +7,7 @@ import clickhouse_driver import xml.etree.ElementTree as et import argparse import pprint +import random import re import string import time @@ -20,7 +21,8 @@ parser = argparse.ArgumentParser(description='Run performance test.') parser.add_argument('file', metavar='FILE', type=argparse.FileType('r', encoding='utf-8'), nargs=1, help='test description file') parser.add_argument('--host', nargs='*', default=['localhost'], help="Server hostname(s). Corresponds to '--port' options.") parser.add_argument('--port', nargs='*', default=[9000], help="Server port(s). Corresponds to '--host' options.") -parser.add_argument('--runs', type=int, default=int(os.environ.get('CHPC_RUNS', 13)), help='Number of query runs per server. Defaults to CHPC_RUNS environment variable.') +parser.add_argument('--runs', type=int, default=1, help='Number of query runs per server.') +parser.add_argument('--max-queries', type=int, default=None, help='Test no more than this number of queries, chosen at random.') parser.add_argument('--long', action='store_true', help='Do not skip the tests tagged as long.') parser.add_argument('--print-queries', action='store_true', help='Print test queries and exit.') parser.add_argument('--print-settings', action='store_true', help='Print test settings and exit.') @@ -189,8 +191,14 @@ for conn_index, c in enumerate(connections): c.execute(q) print(f'fill\t{conn_index}\t{c.last_query.elapsed}\t{tsv_escape(q)}') +# Run the queries in randomized order, but preserve their indexes as specified +# in the test XML. To avoid using too much time, limit the number of queries +# we run per test. +queries_to_run = random.sample(range(0, len(test_queries)), args.max_queries or len(test_queries)) + # Run test queries. -for query_index, q in enumerate(test_queries): +for query_index in queries_to_run: + q = test_queries[query_index] query_prefix = f'{test_name}.query{query_index}' # We have some crazy long queries (about 100kB), so trim them to a sane From bbe379d58e96968ae24c7dee5fef9e872aa2f272 Mon Sep 17 00:00:00 2001 From: Artem Hnilov Date: Thu, 10 Sep 2020 17:06:32 +0000 Subject: [PATCH 122/341] 00534_filimonov.data was updated --- tests/queries/0_stateless/00534_filimonov.data | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/00534_filimonov.data b/tests/queries/0_stateless/00534_filimonov.data index b4c15b01ef4..393ff5b7155 100644 --- a/tests/queries/0_stateless/00534_filimonov.data +++ b/tests/queries/0_stateless/00534_filimonov.data @@ -174,6 +174,7 @@ SELECT sipHash64(NULL); SELECT protocol(NULL); SELECT toInt16OrZero(NULL); SELECT formatReadableSize(NULL); +SELECT formatReadableQuantity(NULL); SELECT concatAssumeInjective(NULL); SELECT toString(NULL); SELECT MACStringToNum(NULL); From 5675efbd47fde50524463a14758c672091264897 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Sep 2020 20:16:12 +0300 Subject: [PATCH 123/341] Fix build --- src/Common/remapExecutable.cpp | 30 ++++++++++++++---------------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/src/Common/remapExecutable.cpp b/src/Common/remapExecutable.cpp index b41fece0c79..6b86e8717a4 100644 --- a/src/Common/remapExecutable.cpp +++ b/src/Common/remapExecutable.cpp @@ -89,20 +89,6 @@ __attribute__((__noinline__)) int64_t our_syscall(...) } -__attribute__((__always_inline__)) void our_memcpy(char * __restrict dst, const char * __restrict src, ssize_t n) -{ - while (n > 0) - { - _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), - _mm_loadu_si128(reinterpret_cast(src))); - - dst += 16; - src += 16; - n -= 16; - } -} - - __attribute__((__noinline__)) void remapToHugeStep3(void * scratch, size_t size, size_t offset) { /// The function should not use the stack, otherwise various optimizations, including "omit-frame-pointer" may break the code. @@ -122,7 +108,7 @@ __attribute__((__noinline__)) void remapToHugeStep2(void * begin, size_t size, v /** Unmap old memory region with the code of our program. * Our instruction pointer is located inside scratch area and this function can execute after old code is unmapped. * But it cannot call any other functions because they are not available at usual addresses - * - that's why we have to use "our_syscall" and "our_memcpy" functions. + * - that's why we have to use "our_syscall" function and a substitution for memcpy. * (Relative addressing may continue to work but we should not assume that). */ @@ -152,7 +138,19 @@ __attribute__((__noinline__)) void remapToHugeStep2(void * begin, size_t size, v /// Copy the code from scratch area to the old memory location. - our_memcpy(reinterpret_cast(begin), reinterpret_cast(scratch), size); + { + __m128i * __restrict dst = reinterpret_cast<__m128i *>(begin); + const __m128i * __restrict src = reinterpret_cast(scratch); + const __m128i * __restrict src_end = reinterpret_cast(reinterpret_cast(scratch) + size); + while (src < src_end) + { + _mm_storeu_si128(dst, _mm_loadu_si128(src)); + + ++dst; + ++src; + } + } + //syscall_func(SYS_write, 2, &dot, 1); /// Make the memory area with the code executable and non-writable. From 294af54007b761fbab57f006b4e074f9f996944a Mon Sep 17 00:00:00 2001 From: Artem Hnilov Date: Thu, 10 Sep 2020 20:27:37 +0300 Subject: [PATCH 124/341] Update other-functions.md --- .../functions/other-functions.md | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 468e15e7d57..7b9dacf21cd 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -508,6 +508,29 @@ SELECT └────────────────┴────────────┘ ``` +## formatReadableQuantity(x) {#formatreadablequantityx} + +Принимает число. Возвращает округленное число с суффиксом (thousand, million, billion и т.д.) в виде строки. + +Облегчает визуальное восприятие больших чисел живым человеком. + +Пример: + +``` sql +SELECT + arrayJoin([1024, 1234 * 1000, (4567 * 1000) * 1000, 98765432101234]) AS number, + formatReadableQuantity(number) AS number_for_humans +``` + +``` text +┌─────────number─┬─number_for_humans─┐ +│ 1024 │ 1.02 thousand │ +│ 1234000 │ 1.23 million │ +│ 4567000000 │ 4.57 billion │ +│ 98765432101234 │ 98.77 trillion │ +└────────────────┴───────────────────┘ +``` + ## least(a, b) {#leasta-b} Возвращает наименьшее значение из a и b. From 726277adec1da5924da59883f17945a8e29e0195 Mon Sep 17 00:00:00 2001 From: Artem Hnilov Date: Thu, 10 Sep 2020 20:34:23 +0300 Subject: [PATCH 125/341] Update other-functions.md --- .../functions/other-functions.md | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 05247b6db7d..1c059e9f97b 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -515,6 +515,29 @@ SELECT └────────────────┴────────────┘ ``` +## formatReadableQuantity(x) {#formatreadablequantityx} + +Accepts the number. Returns a rounded number with a suffix (thousand, million, billion, etc.) as a string. + +It is useful for reading big numbers by human. + +Example: + +``` sql +SELECT + arrayJoin([1024, 1234 * 1000, (4567 * 1000) * 1000, 98765432101234]) AS number, + formatReadableQuantity(number) AS number_for_humans +``` + +``` text +┌─────────number─┬─number_for_humans─┐ +│ 1024 │ 1.02 thousand │ +│ 1234000 │ 1.23 million │ +│ 4567000000 │ 4.57 billion │ +│ 98765432101234 │ 98.77 trillion │ +└────────────────┴───────────────────┘ +``` + ## least(a, b) {#leasta-b} Returns the smallest value from a and b. From 54491b8608043871634973b997042864ca29bc6f Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 10 Sep 2020 22:23:23 +0300 Subject: [PATCH 126/341] Update README.md --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 7f6a102a2dd..f14f2e88886 100644 --- a/README.md +++ b/README.md @@ -17,5 +17,4 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events -* [ClickHouse Data Integration Virtual Meetup](https://www.eventbrite.com/e/clickhouse-september-virtual-meetup-data-integration-tickets-117421895049) on September 10, 2020. * [ClickHouse talk at Ya.Subbotnik (in Russian)](https://ya.cc/t/cIBI-3yECj5JF) on September 12, 2020. From f2a5216e97f1283b373720717f1d6f7ac79af86d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 11 Sep 2020 02:24:16 +0300 Subject: [PATCH 127/341] add waiting for fsync in WAL --- src/Common/FileSyncGuard.h | 2 +- src/Storages/MergeTree/MergeTreeSettings.h | 1 + src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp | 14 +++++++++----- src/Storages/MergeTree/MergeTreeWriteAheadLog.h | 5 +++-- utils/durability-test/durability-test.sh | 12 ++++++++++++ 5 files changed, 26 insertions(+), 8 deletions(-) diff --git a/src/Common/FileSyncGuard.h b/src/Common/FileSyncGuard.h index 5ec9b1d0c98..6451f6ebf36 100644 --- a/src/Common/FileSyncGuard.h +++ b/src/Common/FileSyncGuard.h @@ -17,7 +17,7 @@ public: 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)) {} + : disk(disk_), fd(disk_->open(path, O_RDWR)) {} ~FileSyncGuard() { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 3f8f44dc11e..8652a6ef691 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -47,6 +47,7 @@ struct Settings; 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) \ + M(Bool, in_memory_parts_insert_sync, false, "If true insert of part with in-memory format will wait for fsync of WAL", 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 5cfe9017248..bc6738a8321 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -33,6 +33,7 @@ MergeTreeWriteAheadLog::MergeTreeWriteAheadLog( std::lock_guard lock(write_mutex); out->sync(); sync_scheduled = false; + sync_cv.notify_all(); }); } @@ -50,7 +51,7 @@ void MergeTreeWriteAheadLog::init() void MergeTreeWriteAheadLog::addPart(const Block & block, const String & part_name) { - std::lock_guard lock(write_mutex); + std::unique_lock lock(write_mutex); auto part_info = MergeTreePartInfo::fromPartName(part_name, storage.format_version); min_block_number = std::min(min_block_number, part_info.min_block); @@ -70,7 +71,7 @@ void MergeTreeWriteAheadLog::addPart(const Block & block, const String & part_na void MergeTreeWriteAheadLog::dropPart(const String & part_name) { - std::lock_guard lock(write_mutex); + std::unique_lock lock(write_mutex); writeIntBinary(static_cast(0), *out); writeIntBinary(static_cast(ActionType::DROP_PART), *out); @@ -78,7 +79,7 @@ void MergeTreeWriteAheadLog::dropPart(const String & part_name) sync(lock); } -void MergeTreeWriteAheadLog::rotate(const std::lock_guard &) +void MergeTreeWriteAheadLog::rotate(const std::unique_lock &) { String new_name = String(WAL_FILE_NAME) + "_" + toString(min_block_number) + "_" @@ -90,7 +91,7 @@ void MergeTreeWriteAheadLog::rotate(const std::lock_guard &) MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const StorageMetadataPtr & metadata_snapshot) { - std::lock_guard lock(write_mutex); + std::unique_lock lock(write_mutex); MergeTreeData::MutableDataPartsVector parts; auto in = disk->readFile(path, DBMS_DEFAULT_BUFFER_SIZE); @@ -185,7 +186,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor return result; } -void MergeTreeWriteAheadLog::sync(const std::lock_guard &) +void MergeTreeWriteAheadLog::sync(std::unique_lock & lock) { 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; @@ -201,6 +202,9 @@ void MergeTreeWriteAheadLog::sync(const std::lock_guard &) sync_task->scheduleAfter(time_to_sync); sync_scheduled = true; } + + if (storage.getSettings()->in_memory_parts_insert_sync) + sync_cv.wait(lock, [this] { return !sync_scheduled; }); } std::optional diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 43abf3c04be..c5675eac696 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -44,8 +44,8 @@ public: private: void init(); - void rotate(const std::lock_guard & lock); - void sync(const std::lock_guard & lock); + void rotate(const std::unique_lock & lock); + void sync(std::unique_lock & lock); const MergeTreeData & storage; DiskPtr disk; @@ -60,6 +60,7 @@ private: BackgroundSchedulePool & pool; BackgroundSchedulePoolTaskHolder sync_task; + std::condition_variable sync_cv; size_t bytes_at_last_sync = 0; bool sync_scheduled = false; diff --git a/utils/durability-test/durability-test.sh b/utils/durability-test/durability-test.sh index c7f8936ec95..97c39473b69 100755 --- a/utils/durability-test/durability-test.sh +++ b/utils/durability-test/durability-test.sh @@ -1,5 +1,17 @@ #!/bin/bash +: ' +A simple test for durability. It starts up clickhouse server in qemu VM and runs +inserts via clickhouse benchmark tool. Then it kills VM in random moment and +checks whether table contains broken parts. With enabled fsync no broken parts +should be appeared. + +Usage: + +./install.sh +./durability-test.sh +' + URL=http://cloud-images.ubuntu.com/bionic/current IMAGE=bionic-server-cloudimg-amd64.img SSH_PORT=11022 From 2873777857351a939564e7fac0a285bf8222f19c Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Fri, 11 Sep 2020 12:54:53 +0800 Subject: [PATCH 128/341] Ignore unprocessed replication payload --- src/Core/MySQL/MySQLReplication.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 808e9d1a247..e7f113ba7af 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -171,9 +171,7 @@ namespace MySQLReplication /// Ignore MySQL 8.0 optional metadata fields. /// https://mysqlhighavailability.com/more-metadata-is-written-into-binary-log/ - auto remain = payload.available(); - if (remain > CHECKSUM_CRC32_SIGNATURE_LENGTH) - payload.ignore(remain); + payload.ignoreAll(); } /// Types that do not used in the binlog event: @@ -915,7 +913,7 @@ namespace MySQLReplication break; } } - payload.tryIgnore(CHECKSUM_CRC32_SIGNATURE_LENGTH); + payload.ignoreAll(); } } From da2bb4e0d3d2e642993f070b923401a9db470d81 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 11 Sep 2020 15:46:14 +0800 Subject: [PATCH 129/341] Fix missing clone in replace column transformer --- src/Parsers/ASTColumnsTransformers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index 2625a03830b..43d54f07ab8 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -110,7 +110,7 @@ void ASTColumnsReplaceTransformer::replaceChildren(ASTPtr & node, const ASTPtr & if (const auto * id = child->as()) { if (id->shortName() == name) - child = replacement; + child = replacement->clone(); } else replaceChildren(child, replacement, name); From 3b9ab3f1be330b5ae7ffd7c68fd629ad3ebc9f6b Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 11 Sep 2020 12:23:31 +0300 Subject: [PATCH 130/341] Fix if --- src/Functions/if.cpp | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 20848bede32..584bed3f8c5 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -604,7 +604,6 @@ private: const ColumnUInt8 * cond_col, Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) { /// Convert both columns to the common type (if needed). - const ColumnWithTypeAndName & arg1 = block.getByPosition(arguments[1]); const ColumnWithTypeAndName & arg2 = block.getByPosition(arguments[2]); @@ -765,10 +764,22 @@ private: return ColumnNullable::create(materialized, ColumnUInt8::create(column->size(), 0)); } - static ColumnPtr getNestedColumn(const ColumnPtr & column) + /// Return nested column recursively removing Nullable, examples: + /// Nullable(size = 1, Int32(size = 1), UInt8(size = 1)) -> Int32(size = 1) + /// Const(size = 0, Nullable(size = 1, Int32(size = 1), UInt8(size = 1))) -> + /// Const(size = 0, Int32(size = 1)) + static ColumnPtr recursiveGetNestedColumnWithoutNullable(const ColumnPtr & column) { if (const auto * nullable = checkAndGetColumn(*column)) + { + /// Nullable cannot contain Nullable return nullable->getNestedColumnPtr(); + } + else if (const auto * column_const = checkAndGetColumn(*column)) + { + /// Save Constant, but remove Nullable + return ColumnConst::create(recursiveGetNestedColumnWithoutNullable(column_const->getDataColumnPtr()), column->size()); + } return column; } @@ -826,12 +837,12 @@ private: { arg_cond, { - getNestedColumn(arg_then.column), + recursiveGetNestedColumnWithoutNullable(arg_then.column), removeNullable(arg_then.type), "" }, { - getNestedColumn(arg_else.column), + recursiveGetNestedColumnWithoutNullable(arg_else.column), removeNullable(arg_else.type), "" }, From 5de3d9c03298101876df60eed11ca2499676faf5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 11 Sep 2020 12:58:04 +0300 Subject: [PATCH 131/341] Update version_date.tsv after release 20.6.6.7 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index d97fdbfa080..3915000cec3 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,5 +1,6 @@ v20.8.2.3-stable 2020-09-08 v20.7.2.30-stable 2020-08-31 +v20.6.6.7-stable 2020-09-11 v20.6.5.8-stable 2020-09-03 v20.6.4.44-stable 2020-08-20 v20.6.3.28-stable 2020-08-07 From e25b1da29fa168b24464c83c1f661b363916afad Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 11 Sep 2020 13:53:26 +0300 Subject: [PATCH 132/341] Disable -Wstringop-overflow for gcc-10 --- cmake/warnings.cmake | 11 +++++++++-- src/Storages/MergeTree/MergeTreePartition.cpp | 8 -------- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index aec3e46ffa6..6b26b9b95a5 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -169,9 +169,16 @@ elseif (COMPILER_GCC) # Warn if vector operation is not implemented via SIMD capabilities of the architecture add_cxx_compile_options(-Wvector-operation-performance) - # XXX: gcc10 stuck with this option while compiling GatherUtils code - # (anyway there are builds with clang, that will warn) if (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL 10) + # XXX: gcc10 stuck with this option while compiling GatherUtils code + # (anyway there are builds with clang, that will warn) add_cxx_compile_options(-Wno-sequence-point) + # XXX: gcc10 false positive with this warning in MergeTreePartition.cpp + # inlined from 'void writeHexByteLowercase(UInt8, void*)' at ../src/Common/hex.h:39:11, + # inlined from 'DB::String DB::MergeTreePartition::getID(const DB::Block&) const' at ../src/Storages/MergeTree/MergeTreePartition.cpp:85:30: + # ../contrib/libc-headers/x86_64-linux-gnu/bits/string_fortified.h:34:33: error: writing 2 bytes into a region of size 0 [-Werror=stringop-overflow=] + # 34 | return __builtin___memcpy_chk (__dest, __src, __len, __bos0 (__dest)); + # For some reason (bug in gcc?) macro 'GCC diagnostic ignored "-Wstringop-overflow"' doesn't help. + add_cxx_compile_options(-Wno-stringop-overflow) endif() endif () diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 2802b842f54..880a3aa181d 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -29,11 +29,6 @@ String MergeTreePartition::getID(const MergeTreeData & storage) const return getID(storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block); } -#if defined (__GNUC__) && __GNUC__ >= 10 - #pragma GCC diagnostic push - #pragma GCC diagnostic ignored "-Wstringop-overflow" -#endif - /// NOTE: This ID is used to create part names which are then persisted in ZK and as directory names on the file system. /// So if you want to change this method, be sure to guarantee compatibility with existing table data. String MergeTreePartition::getID(const Block & partition_key_sample) const @@ -92,9 +87,6 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const return result; } -#if defined (__GNUC__) && __GNUC__ >= 10 - #pragma GCC diagnostic pop -#endif void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const { From c36192db233af7ce3f971a0cd950db4cfbb6175d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 11 Sep 2020 13:54:03 +0300 Subject: [PATCH 133/341] Remove diff --- src/Storages/MergeTree/MergeTreePartition.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 880a3aa181d..4a846f63b7c 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -87,7 +87,6 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const return result; } - void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); From ebb9de1376d50e834b61b48cc2f4695513244ad9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 11 Sep 2020 14:13:41 +0300 Subject: [PATCH 134/341] Supress strange warning --- src/Functions/negate.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Functions/negate.cpp b/src/Functions/negate.cpp index 39ca434ea89..3101513648b 100644 --- a/src/Functions/negate.cpp +++ b/src/Functions/negate.cpp @@ -13,7 +13,14 @@ struct NegateImpl static inline NO_SANITIZE_UNDEFINED ResultType apply(A a) { - return -static_cast(a); +#if defined (__GNUC__) && __GNUC__ >= 10 + #pragma GCC diagnostic push + #pragma GCC diagnostic ignored "-Wvector-operation-performance" +#endif + return -(static_cast(a)); +#if defined (__GNUC__) && __GNUC__ >= 10 + #pragma GCC diagnostic pop +#endif } #if USE_EMBEDDED_COMPILER From 5ce0c21bbe3c08a0f5169bced9dcea208857c88a Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 11 Sep 2020 14:24:42 +0300 Subject: [PATCH 135/341] Remove redundant change --- src/Functions/negate.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/negate.cpp b/src/Functions/negate.cpp index 3101513648b..de3995927d3 100644 --- a/src/Functions/negate.cpp +++ b/src/Functions/negate.cpp @@ -17,7 +17,7 @@ struct NegateImpl #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wvector-operation-performance" #endif - return -(static_cast(a)); + return -static_cast(a); #if defined (__GNUC__) && __GNUC__ >= 10 #pragma GCC diagnostic pop #endif From eb7e480d399788dae32dc83bd4b32dfdc6ec1aa3 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 11 Sep 2020 15:47:36 +0300 Subject: [PATCH 136/341] DOCSUP-2040: Translate PR to RU (#14551) * DOCSUP-2040: Update by PR#1127. * DOCSUP-2040: Update by PR#11088. * DOCSUP-2040: Update by PR#10923. * DOCSUP-2040: Minor fix. * Update docs/ru/sql-reference/functions/random-functions.md Co-authored-by: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> * Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> * Update translation. * Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: BayoNet * Update docs/ru/sql-reference/functions/random-functions.md Co-authored-by: BayoNet * Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: BayoNet * Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: BayoNet Co-authored-by: romanzhukov Co-authored-by: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Co-authored-by: BayoNet --- docs/ru/interfaces/formats.md | 20 +++- .../functions/random-functions.md | 46 ++++++++++ .../functions/type-conversion-functions.md | 91 +++++++++++++++++++ 3 files changed, 156 insertions(+), 1 deletion(-) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 054f75e8da8..04bca115974 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -28,6 +28,8 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT | [PrettySpace](#prettyspace) | ✗ | ✔ | | [Protobuf](#protobuf) | ✔ | ✔ | | [Parquet](#data-format-parquet) | ✔ | ✔ | +| [Arrow](#data-format-arrow) | ✔ | ✔ | +| [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ | | [ORC](#data-format-orc) | ✔ | ✗ | | [RowBinary](#rowbinary) | ✔ | ✔ | | [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ | @@ -947,6 +949,12 @@ ClickHouse пишет и читает сообщения `Protocol Buffers` в ## Avro {#data-format-avro} +[Apache Avro](https://avro.apache.org/) — это ориентированный на строки фреймворк для сериализации данных. Разработан в рамках проекта Apache Hadoop. + +В ClickHouse формат Avro поддерживает чтение и запись [файлов данных Avro](https://avro.apache.org/docs/current/spec.html#Object+Container+Files). + +[Логические типы Avro](https://avro.apache.org/docs/current/spec.html#Logical+Types) + ## AvroConfluent {#data-format-avro-confluent} Для формата `AvroConfluent` ClickHouse поддерживает декодирование сообщений `Avro` с одним объектом. Такие сообщения используются с [Kafka] (http://kafka.apache.org/) и реестром схем [Confluent](https://docs.confluent.io/current/schema-registry/index.html). @@ -996,7 +1004,7 @@ SELECT * FROM topic1_stream; ## Parquet {#data-format-parquet} -[Apache Parquet](http://parquet.apache.org/) — формат поколоночного хранения данных, который распространён в экосистеме Hadoop. Для формата `Parquet` ClickHouse поддерживает операции чтения и записи. +[Apache Parquet](https://parquet.apache.org/) — формат поколоночного хранения данных, который распространён в экосистеме Hadoop. Для формата `Parquet` ClickHouse поддерживает операции чтения и записи. ### Соответствие типов данных {#sootvetstvie-tipov-dannykh} @@ -1042,6 +1050,16 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Parquet" > {some_ Для обмена данными с экосистемой Hadoop можно использовать движки таблиц [HDFS](../engines/table-engines/integrations/hdfs.md). +## Arrow {data-format-arrow} + +[Apache Arrow](https://arrow.apache.org/) поставляется с двумя встроенными поколоночнами форматами хранения. ClickHouse поддерживает операции чтения и записи для этих форматов. + +`Arrow` — это Apache Arrow's "file mode" формат. Он предназначен для произвольного доступа в памяти. + +## ArrowStream {data-format-arrow-stream} + +`ArrowStream` — это Apache Arrow's "stream mode" формат. Он предназначен для обработки потоков в памяти. + ## ORC {#data-format-orc} [Apache ORC](https://orc.apache.org/) - это column-oriented формат данных, распространённый в экосистеме Hadoop. Вы можете только вставлять данные этого формата в ClickHouse. diff --git a/docs/ru/sql-reference/functions/random-functions.md b/docs/ru/sql-reference/functions/random-functions.md index b425505b69d..4aaaef5cb5d 100644 --- a/docs/ru/sql-reference/functions/random-functions.md +++ b/docs/ru/sql-reference/functions/random-functions.md @@ -55,4 +55,50 @@ FROM numbers(3) └────────────┴────────────┴──────────────┴────────────────┴─────────────────┴──────────────────────┘ ``` +# Случайные функции для работы со строками {#random-functions-for-working-with-strings} + +## randomString {#random-string} + +## randomFixedString {#random-fixed-string} + +## randomPrintableASCII {#random-printable-ascii} + +## randomStringUTF8 {#random-string-utf8} + +## fuzzBits {#fuzzbits} + +**Синтаксис** + +``` sql +fuzzBits([s], [prob]) +``` +Инвертирует каждый бит `s` с вероятностью `prob`. + +**Параметры** + +- `s` — `String` or `FixedString` +- `prob` — constant `Float32/64` + +**Возвращаемое значение** + +Измененная случайным образом строка с тем же типом, что и `s`. + +**Пример** + +Запрос: + +``` sql +SELECT fuzzBits(materialize('abacaba'), 0.1) +FROM numbers(3) +``` + +Результат: + +``` text +┌─fuzzBits(materialize('abacaba'), 0.1)─┐ +│ abaaaja │ +│ a*cjab+ │ +│ aeca2A │ +└───────────────────────────────────────┘ + [Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/random_functions/) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 41ded78055c..c7d74a9d881 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -513,4 +513,95 @@ SELECT parseDateTimeBestEffort('10 20:19') - [toDate](#todate) - [toDateTime](#todatetime) +## toUnixTimestamp64Milli +## toUnixTimestamp64Micro +## toUnixTimestamp64Nano + +Преобразует значение `DateTime64` в значение `Int64` с фиксированной точностью менее одной секунды. +Входное значение округляется соответствующим образом вверх или вниз в зависимости от его точности. Обратите внимание, что возвращаемое значение - это временная метка в UTC, а не в часовом поясе `DateTime64`. + +**Синтаксис** + +``` sql +toUnixTimestamp64Milli(value) +``` + +**Параметры** + +- `value` — значение `DateTime64` с любой точностью. + +**Возвращаемое значение** + +- Значение `value`, преобразованное в тип данных `Int64`. + +**Примеры** + +Запрос: + +``` sql +WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 +SELECT toUnixTimestamp64Milli(dt64) +``` + +Ответ: + +``` text +┌─toUnixTimestamp64Milli(dt64)─┐ +│ 1568650812345 │ +└──────────────────────────────┘ +``` + +Запрос: + +``` sql +WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 +SELECT toUnixTimestamp64Nano(dt64) +``` + +Ответ: + +``` text +┌─toUnixTimestamp64Nano(dt64)─┐ +│ 1568650812345678000 │ +└─────────────────────────────┘ +``` + +## fromUnixTimestamp64Milli +## fromUnixTimestamp64Micro +## fromUnixTimestamp64Nano + +Преобразует значение `Int64` в значение `DateTime64` с фиксированной точностью менее одной секунды и дополнительным часовым поясом. Входное значение округляется соответствующим образом вверх или вниз в зависимости от его точности. Обратите внимание, что входное значение обрабатывается как метка времени UTC, а не метка времени в заданном (или неявном) часовом поясе. + +**Синтаксис** + +``` sql +fromUnixTimestamp64Milli(value [, ti]) +``` + +**Параметры** + +- `value` — значение типы `Int64` с любой точностью. +- `timezone` — (не обязательный параметр) часовой пояс в формате `String` для возвращаемого результата. + +**Возвращаемое значение** + +- Значение `value`, преобразованное в тип данных `DateTime64`. + +**Пример** + +Запрос: + +``` sql +WITH CAST(1234567891011, 'Int64') AS i64 +SELECT fromUnixTimestamp64Milli(i64, 'UTC') +``` + +Ответ: + +``` text +┌─fromUnixTimestamp64Milli(i64, 'UTC')─┐ +│ 2009-02-13 23:31:31.011 │ +└──────────────────────────────────────┘ +``` + [Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/type_conversion_functions/) From a64331d79f04bb9321383269150fe8302289e9b2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 11 Sep 2020 16:09:26 +0300 Subject: [PATCH 137/341] fix syncing of WAL --- src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp | 7 +++++++ src/Storages/MergeTree/MergeTreeWriteAheadLog.h | 2 ++ 2 files changed, 9 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index bc6738a8321..35fadb999b4 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -37,6 +37,13 @@ MergeTreeWriteAheadLog::MergeTreeWriteAheadLog( }); } +MergeTreeWriteAheadLog::~MergeTreeWriteAheadLog() +{ + std::unique_lock lock(write_mutex); + if (sync_scheduled) + sync_cv.wait(lock, [this] { return !sync_scheduled; }); +} + void MergeTreeWriteAheadLog::init() { out = disk->writeFile(path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append); diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index c5675eac696..77c7c7e11e7 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -35,6 +35,8 @@ public: MergeTreeWriteAheadLog(MergeTreeData & storage_, const DiskPtr & disk_, const String & name = DEFAULT_WAL_FILE_NAME); + ~MergeTreeWriteAheadLog(); + void addPart(const Block & block, const String & part_name); void dropPart(const String & part_name); std::vector restore(const StorageMetadataPtr & metadata_snapshot); From 6dd764bcfe5b28d3ccb89ca950558bd82c911847 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 11 Sep 2020 18:12:08 +0300 Subject: [PATCH 138/341] FunctionBinaryArithmetic refactoring (#14712) --- src/Functions/FunctionBinaryArithmetic.h | 165 ++++++++++------------- src/Functions/divide.cpp | 1 - src/Functions/minus.cpp | 1 - src/Functions/multiply.cpp | 1 - src/Functions/plus.cpp | 2 +- 5 files changed, 72 insertions(+), 98 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 15b6ea6ca5d..ca0cc876035 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -561,6 +561,8 @@ public: template