From fd84d16387e507763a9dfa661397c385d7ab6931 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 7 Nov 2020 03:14:53 +0300 Subject: [PATCH] Fix "server failed to start" error --- src/Access/DiskAccessStorage.cpp | 2 + src/Access/LDAPAccessStorage.cpp | 1 + src/Access/UsersConfigAccessStorage.cpp | 1 + .../AggregateFunctionGroupArray.h | 4 + src/AggregateFunctions/ReservoirSampler.h | 2 + src/Client/MultiplexedConnections.cpp | 1 + src/Columns/tests/gtest_weak_hash_32.cpp | 11 +- src/Common/Config/ConfigProcessor.cpp | 1 + src/Common/Exception.cpp | 2 + src/Common/HTMLForm.h | 2 + src/Common/MemoryTracker.cpp | 2 + src/Common/ShellCommand.cpp | 1 + src/Common/StackTrace.cpp | 4 + src/Common/StudentTTest.cpp | 2 + src/Common/ThreadProfileEvents.cpp | 1 + src/Common/ThreadStatus.cpp | 1 + src/Common/UInt128.h | 1 + src/Common/XDBCBridgeHelper.h | 1 + src/Common/ZooKeeper/TestKeeper.cpp | 1 + src/Common/checkStackSize.cpp | 1 + src/Common/parseGlobs.cpp | 4 + .../gtest_getMultipleValuesFromConfig.cpp | 2 + .../tests/gtest_sensitive_data_masker.cpp | 8 + src/Compression/tests/compressed_buffer.cpp | 1 + src/Core/MySQL/IMySQLReadPacket.cpp | 1 + src/Core/MySQL/IMySQLWritePacket.cpp | 1 + src/Core/SortDescription.h | 1 + .../CheckConstraintsBlockOutputStream.cpp | 2 + src/DataStreams/IBlockInputStream.cpp | 1 + .../DataTypeCustomSimpleAggregateFunction.cpp | 1 + src/DataTypes/DataTypesDecimal.cpp | 1 + .../tests/gtest_data_type_get_common_type.cpp | 1 + src/Databases/DatabaseOnDisk.cpp | 2 + .../MySQL/MaterializeMySQLSyncThread.cpp | 5 + src/Dictionaries/DictionaryStructure.cpp | 1 + .../tests/gtest_dictionary_configuration.cpp | 1 + src/Functions/abtesting.cpp | 42 ++- src/IO/HTTPCommon.cpp | 1 + src/IO/MySQLPacketPayloadReadBuffer.cpp | 14 +- src/IO/ReadWriteBufferFromHTTP.h | 5 +- src/IO/S3/PocoHTTPClient.cpp | 1 + src/IO/tests/gtest_bit_io.cpp | 1 + src/IO/tests/hashing_read_buffer.cpp | 1 + src/IO/tests/limit_read_buffer2.cpp | 1 + src/IO/tests/write_buffer.cpp | 1 + .../ClusterProxy/SelectStreamFactory.cpp | 1 + src/Interpreters/Context.cpp | 1 + src/Interpreters/ExpressionActions.cpp | 4 + src/Interpreters/InterpreterCreateQuery.cpp | 2 + src/Interpreters/InterpreterExplainQuery.cpp | 1 + .../InterpreterShowAccessQuery.cpp | 1 + ...InterpreterShowCreateAccessEntityQuery.cpp | 2 + .../InterpreterShowCreateQuery.cpp | 1 + .../InterpreterShowGrantsQuery.cpp | 2 + .../InterpreterShowTablesQuery.cpp | 4 + src/Interpreters/QueryAliasesVisitor.cpp | 1 + src/Interpreters/Set.cpp | 1 + src/Interpreters/TreeRewriter.cpp | 1 + src/Interpreters/executeQuery.cpp | 1 + src/Interpreters/tests/CMakeLists.txt | 3 - src/Interpreters/tests/users.cpp | 282 ------------------ src/Parsers/ASTCreateRowPolicyQuery.cpp | 1 + src/Parsers/DumpASTNode.h | 1 + src/Parsers/IAST.cpp | 1 + src/Parsers/IAST.h | 1 + src/Parsers/formatAST.cpp | 1 + src/Parsers/queryToString.cpp | 1 + src/Parsers/tests/gtest_dictionary_parser.cpp | 1 + .../Formats/Impl/AvroRowInputFormat.cpp | 1 + .../Formats/Impl/PrettyBlockOutputFormat.cpp | 5 + .../Impl/PrettyCompactBlockOutputFormat.cpp | 1 + .../Algorithms/CollapsingSortedAlgorithm.cpp | 1 + src/Server/ReplicasStatusHandler.cpp | 1 + src/Server/TCPHandler.cpp | 1 + .../DistributedBlockOutputStream.cpp | 1 + src/Storages/Kafka/StorageKafka.cpp | 2 + src/Storages/MergeTree/KeyCondition.cpp | 2 + src/Storages/MergeTree/MergeTreeData.cpp | 3 + .../MergeTree/MergeTreeDataPartChecksum.cpp | 1 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 1 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 + .../ReplicatedMergeTreeTableMetadata.cpp | 1 + src/Storages/MutationCommands.cpp | 1 + src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 1 + src/Storages/StorageDictionary.cpp | 1 + src/Storages/StorageDistributed.cpp | 2 + src/Storages/StorageInMemoryMetadata.cpp | 1 + src/Storages/StorageMergeTree.cpp | 3 + src/Storages/StorageReplicatedMergeTree.cpp | 5 + src/Storages/StorageS3.cpp | 1 + src/Storages/System/StorageSystemUsers.cpp | 1 + src/Storages/tests/gtest_storage_log.cpp | 1 + .../transformQueryForExternalDatabase.cpp | 1 + src/TableFunctions/TableFunctionRemote.cpp | 1 + 94 files changed, 173 insertions(+), 326 deletions(-) delete mode 100644 src/Interpreters/tests/users.cpp diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 0c7425327ad..abf4ff12d5a 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -198,6 +198,7 @@ namespace /// Serialize the list of ATTACH queries to a string. std::stringstream ss; + ss.exceptions(std::ios::failbit); for (const ASTPtr & query : queries) ss << *query << ";\n"; String file_contents = std::move(ss).str(); @@ -353,6 +354,7 @@ String DiskAccessStorage::getStorageParamsJSON() const if (readonly) json.set("readonly", readonly.load()); std::ostringstream oss; + oss.exceptions(std::ios::failbit); Poco::JSON::Stringifier::stringify(json, oss); return oss.str(); } diff --git a/src/Access/LDAPAccessStorage.cpp b/src/Access/LDAPAccessStorage.cpp index b20ef3a39a9..c9f00f2f4ab 100644 --- a/src/Access/LDAPAccessStorage.cpp +++ b/src/Access/LDAPAccessStorage.cpp @@ -151,6 +151,7 @@ String LDAPAccessStorage::getStorageParamsJSON() const params_json.set("roles", default_role_names); std::ostringstream oss; + oss.exceptions(std::ios::failbit); Poco::JSON::Stringifier::stringify(params_json, oss); return oss.str(); diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index ce10ebf0bcc..ba3c2d5b8e5 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -461,6 +461,7 @@ String UsersConfigAccessStorage::getStorageParamsJSON() const if (!path.empty()) json.set("path", path); std::ostringstream oss; + oss.exceptions(std::ios::failbit); Poco::JSON::Stringifier::stringify(json, oss); return oss.str(); } diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.h b/src/AggregateFunctions/AggregateFunctionGroupArray.h index 02b9003eb96..ba9b235de07 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -245,6 +245,7 @@ public: { DB::writeIntBinary(this->data(place).total_values, buf); std::ostringstream rng_stream; + rng_stream.exceptions(std::ios::failbit); rng_stream << this->data(place).rng; DB::writeStringBinary(rng_stream.str(), buf); } @@ -275,6 +276,7 @@ public: std::string rng_string; DB::readStringBinary(rng_string, buf); std::istringstream rng_stream(rng_string); + rng_stream.exceptions(std::ios::failbit); rng_stream >> this->data(place).rng; } @@ -564,6 +566,7 @@ public: { DB::writeIntBinary(data(place).total_values, buf); std::ostringstream rng_stream; + rng_stream.exceptions(std::ios::failbit); rng_stream << data(place).rng; DB::writeStringBinary(rng_stream.str(), buf); } @@ -598,6 +601,7 @@ public: std::string rng_string; DB::readStringBinary(rng_string, buf); std::istringstream rng_stream(rng_string); + rng_stream.exceptions(std::ios::failbit); rng_stream >> data(place).rng; } diff --git a/src/AggregateFunctions/ReservoirSampler.h b/src/AggregateFunctions/ReservoirSampler.h index bbb8172b209..f82b0b856a0 100644 --- a/src/AggregateFunctions/ReservoirSampler.h +++ b/src/AggregateFunctions/ReservoirSampler.h @@ -191,6 +191,7 @@ public: std::string rng_string; DB::readStringBinary(rng_string, buf); std::istringstream rng_stream(rng_string); + rng_stream.exceptions(std::ios::failbit); rng_stream >> rng; for (size_t i = 0; i < samples.size(); ++i) @@ -205,6 +206,7 @@ public: DB::writeIntBinary(total_values, buf); std::ostringstream rng_stream; + rng_stream.exceptions(std::ios::failbit); rng_stream << rng; DB::writeStringBinary(rng_stream.str(), buf); diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 282aaabd119..b6cb55fa979 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -223,6 +223,7 @@ std::string MultiplexedConnections::dumpAddressesUnlocked() const { bool is_first = true; std::ostringstream os; + os.exceptions(std::ios::failbit); for (const ReplicaState & state : replica_states) { const Connection * connection = state.connection; diff --git a/src/Columns/tests/gtest_weak_hash_32.cpp b/src/Columns/tests/gtest_weak_hash_32.cpp index ea4c0db33e9..a7fd60a3b9c 100644 --- a/src/Columns/tests/gtest_weak_hash_32.cpp +++ b/src/Columns/tests/gtest_weak_hash_32.cpp @@ -71,7 +71,8 @@ void checkColumn( std::unordered_map map; size_t num_collisions = 0; - std::stringstream collitions_str; + std::stringstream collisions_str; + collisions_str.exceptions(std::ios::failbit); for (size_t i = 0; i < eq_class.size(); ++i) { @@ -86,14 +87,14 @@ void checkColumn( if (num_collisions <= max_collisions_to_print) { - collitions_str << "Collision:\n"; - collitions_str << print_for_row(it->second) << '\n'; - collitions_str << print_for_row(i) << std::endl; + collisions_str << "Collision:\n"; + collisions_str << print_for_row(it->second) << '\n'; + collisions_str << print_for_row(i) << std::endl; } if (num_collisions > allowed_collisions) { - std::cerr << collitions_str.rdbuf(); + std::cerr << collisions_str.rdbuf(); break; } } diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index d2213a01498..8a6093c47c9 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -538,6 +538,7 @@ XMLDocumentPtr ConfigProcessor::processConfig( *has_zk_includes = !contributing_zk_paths.empty(); std::stringstream comment; + comment.exceptions(std::ios::failbit); comment << " This file was generated automatically.\n"; comment << " Do not edit it: it is likely to be discarded and generated again before it's read next time.\n"; comment << " Files used to generate this file:"; diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 0214fa7b065..820e3857bfc 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -246,6 +246,7 @@ static std::string getExtraExceptionInfo(const std::exception & e) std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded_stacktrace /*= false*/, bool with_extra_info /*= true*/) { std::stringstream stream; + stream.exceptions(std::ios::failbit); try { @@ -365,6 +366,7 @@ void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::str std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace) { std::stringstream stream; + stream.exceptions(std::ios::failbit); try { diff --git a/src/Common/HTMLForm.h b/src/Common/HTMLForm.h index 2490d613160..f9b5cc0c520 100644 --- a/src/Common/HTMLForm.h +++ b/src/Common/HTMLForm.h @@ -17,12 +17,14 @@ struct HTMLForm : public Poco::Net::HTMLForm { Poco::URI uri(request.getURI()); std::istringstream istr(uri.getRawQuery()); + istr.exceptions(std::ios::failbit); readUrl(istr); } HTMLForm(const Poco::URI & uri) { std::istringstream istr(uri.getRawQuery()); + istr.exceptions(std::ios::failbit); readUrl(istr); } diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 380fcb1b2b6..5257f95898a 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -134,6 +134,7 @@ void MemoryTracker::alloc(Int64 size) ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); std::stringstream message; + message.exceptions(std::ios::failbit); message << "Memory tracker"; if (const auto * description = description_ptr.load(std::memory_order_relaxed)) message << " " << description; @@ -166,6 +167,7 @@ void MemoryTracker::alloc(Int64 size) ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); std::stringstream message; + message.exceptions(std::ios::failbit); message << "Memory limit"; if (const auto * description = description_ptr.load(std::memory_order_relaxed)) message << " " << description; diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 992419adb6d..db0928ea605 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -74,6 +74,7 @@ ShellCommand::~ShellCommand() void ShellCommand::logCommand(const char * filename, char * const argv[]) { std::stringstream args; + args.exceptions(std::ios::failbit); for (int i = 0; argv != nullptr && argv[i] != nullptr; ++i) { if (i > 0) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 677af269ca0..7e9474ac3a7 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -24,6 +24,7 @@ std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext_t & context) { std::stringstream error; + error.exceptions(std::ios::failbit); switch (sig) { case SIGSEGV: @@ -319,6 +320,7 @@ static void toStringEveryLineImpl( std::unordered_map dwarfs; std::stringstream out; + out.exceptions(std::ios::failbit); for (size_t i = offset; i < size; ++i) { @@ -358,6 +360,7 @@ static void toStringEveryLineImpl( } #else std::stringstream out; + out.exceptions(std::ios::failbit); for (size_t i = offset; i < size; ++i) { @@ -373,6 +376,7 @@ static void toStringEveryLineImpl( static std::string toStringImpl(const StackTrace::FramePointers & frame_pointers, size_t offset, size_t size) { std::stringstream out; + out.exceptions(std::ios::failbit); toStringEveryLineImpl(frame_pointers, offset, size, [&](const std::string & str) { out << str << '\n'; }); return out.str(); } diff --git a/src/Common/StudentTTest.cpp b/src/Common/StudentTTest.cpp index 170f06c2877..fe605092acc 100644 --- a/src/Common/StudentTTest.cpp +++ b/src/Common/StudentTTest.cpp @@ -154,6 +154,8 @@ std::pair StudentTTest::compareAndReport(size_t confidence_le double mean_confidence_interval = table_value * t_statistic; std::stringstream ss; + ss.exceptions(std::ios::failbit); + if (mean_difference > mean_confidence_interval && (mean_difference - mean_confidence_interval > 0.0001)) /// difference must be more than 0.0001, to take into account connection latency. { ss << "Difference at " << confidence_level[confidence_level_index] << "% confidence : "; diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index ec9f47aebb1..cb519c9b928 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -415,6 +415,7 @@ std::vector PerfEventsCounters::eventIndicesFromString(const std::string } std::istringstream iss(events_list); + iss.exceptions(std::ios::failbit); std::string event_name; while (std::getline(iss, event_name, ',')) { diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index bac0559fc6b..f5ad28f57af 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -80,6 +80,7 @@ void ThreadStatus::assertState(const std::initializer_list & permitted_stat } std::stringstream ss; + ss.exceptions(std::ios::failbit); ss << "Unexpected thread state " << getCurrentState(); if (description) ss << ": " << description; diff --git a/src/Common/UInt128.h b/src/Common/UInt128.h index 3944d8073c2..7b6f8e7c7be 100644 --- a/src/Common/UInt128.h +++ b/src/Common/UInt128.h @@ -49,6 +49,7 @@ struct UInt128 String toHexString() const { std::ostringstream os; + os.exceptions(std::ios::failbit); os << std::setw(16) << std::setfill('0') << std::hex << high << low; return String(os.str()); } diff --git a/src/Common/XDBCBridgeHelper.h b/src/Common/XDBCBridgeHelper.h index a425cd36b11..c794d2fe3cd 100644 --- a/src/Common/XDBCBridgeHelper.h +++ b/src/Common/XDBCBridgeHelper.h @@ -308,6 +308,7 @@ struct ODBCBridgeMixin path.setFileName("clickhouse-odbc-bridge"); std::stringstream command; + command.exceptions(std::ios::failbit); #if !CLICKHOUSE_SPLIT_BINARY cmd_args.push_back("odbc-bridge"); diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 4f7beadef5f..f7db8a85f96 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -219,6 +219,7 @@ std::pair TestKeeperCreateRequest::process(TestKeeper::Contai ++it->second.seq_num; std::stringstream seq_num_str; + seq_num_str.exceptions(std::ios::failbit); seq_num_str << std::setw(10) << std::setfill('0') << seq_num; path_created += seq_num_str.str(); diff --git a/src/Common/checkStackSize.cpp b/src/Common/checkStackSize.cpp index 10e93a8356c..bdc117eccac 100644 --- a/src/Common/checkStackSize.cpp +++ b/src/Common/checkStackSize.cpp @@ -81,6 +81,7 @@ __attribute__((__weak__)) void checkStackSize() if (stack_size * 2 > max_stack_size) { std::stringstream message; + message.exceptions(std::ios::failbit); message << "Stack size too large" << ". Stack address: " << stack_address << ", frame address: " << frame_address diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index c07664c4662..19a9e9d50b6 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -20,6 +20,7 @@ namespace DB std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_globs) { std::ostringstream oss_for_escaping; + oss_for_escaping.exceptions(std::ios::failbit); /// Escaping only characters that not used in glob syntax for (const auto & letter : initial_str_with_globs) { @@ -33,6 +34,7 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob re2::StringPiece input(escaped_with_globs); re2::StringPiece matched; std::ostringstream oss_for_replacing; + oss_for_replacing.exceptions(std::ios::failbit); size_t current_index = 0; while (RE2::FindAndConsume(&input, enum_or_range, &matched)) { @@ -45,6 +47,7 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob size_t range_end = 0; char point; std::istringstream iss_range(buffer); + iss_range.exceptions(std::ios::failbit); iss_range >> range_begin >> point >> point >> range_end; assert(!iss_range.fail()); bool leading_zeros = buffer[0] == '0'; @@ -71,6 +74,7 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob oss_for_replacing << escaped_with_globs.substr(current_index); std::string almost_res = oss_for_replacing.str(); std::ostringstream oss_final_processing; + oss_final_processing.exceptions(std::ios::failbit); for (const auto & letter : almost_res) { if ((letter == '?') || (letter == '*')) diff --git a/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp b/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp index 4756043acbf..743091e7276 100644 --- a/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp +++ b/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp @@ -19,6 +19,8 @@ TEST(Common, getMultipleValuesFromConfig) )END"); + xml_isteam.exceptions(std::ios::failbit); + Poco::AutoPtr config = new Poco::Util::XMLConfiguration(xml_isteam); std::vector answer = getMultipleValuesFromConfig(*config, "first_level", "second_level"); std::vector right_answer = {"0", "1", "2", "3"}; diff --git a/src/Common/tests/gtest_sensitive_data_masker.cpp b/src/Common/tests/gtest_sensitive_data_masker.cpp index d5133b2ef95..43d7b9499a6 100644 --- a/src/Common/tests/gtest_sensitive_data_masker.cpp +++ b/src/Common/tests/gtest_sensitive_data_masker.cpp @@ -135,6 +135,8 @@ TEST(Common, SensitiveDataMasker) )END"); + xml_isteam.exceptions(std::ios::failbit); + Poco::AutoPtr xml_config = new Poco::Util::XMLConfiguration(xml_isteam); DB::SensitiveDataMasker masker_xml_based(*xml_config, "query_masking_rules"); std::string top_secret = "The e-mail of IVAN PETROV is kotik1902@sdsdf.test, and the password is qwerty123"; @@ -165,6 +167,8 @@ TEST(Common, SensitiveDataMasker) )END"); + + xml_isteam_bad.exceptions(std::ios::failbit); Poco::AutoPtr xml_config = new Poco::Util::XMLConfiguration(xml_isteam_bad); DB::SensitiveDataMasker masker_xml_based_exception_check(*xml_config, "query_masking_rules"); @@ -187,6 +191,8 @@ TEST(Common, SensitiveDataMasker) )END"); + xml_isteam_bad.exceptions(std::ios::failbit); + Poco::AutoPtr xml_config = new Poco::Util::XMLConfiguration(xml_isteam_bad); DB::SensitiveDataMasker masker_xml_based_exception_check(*xml_config, "query_masking_rules"); @@ -209,6 +215,8 @@ TEST(Common, SensitiveDataMasker) )END"); + xml_isteam_bad.exceptions(std::ios::failbit); + Poco::AutoPtr xml_config = new Poco::Util::XMLConfiguration(xml_isteam_bad); DB::SensitiveDataMasker masker_xml_based_exception_check(*xml_config, "query_masking_rules"); diff --git a/src/Compression/tests/compressed_buffer.cpp b/src/Compression/tests/compressed_buffer.cpp index 89bfe0d0cce..c018fc95995 100644 --- a/src/Compression/tests/compressed_buffer.cpp +++ b/src/Compression/tests/compressed_buffer.cpp @@ -52,6 +52,7 @@ int main(int, char **) if (x != i) { std::stringstream s; + s.exceptions(std::ios::failbit); s << "Failed!, read: " << x << ", expected: " << i; throw DB::Exception(s.str(), 0); } diff --git a/src/Core/MySQL/IMySQLReadPacket.cpp b/src/Core/MySQL/IMySQLReadPacket.cpp index 5f6bbc7bceb..676f3986ba4 100644 --- a/src/Core/MySQL/IMySQLReadPacket.cpp +++ b/src/Core/MySQL/IMySQLReadPacket.cpp @@ -22,6 +22,7 @@ void IMySQLReadPacket::readPayload(ReadBuffer & in, uint8_t & sequence_id) if (!payload.eof()) { std::stringstream tmp; + tmp.exceptions(std::ios::failbit); tmp << "Packet payload is not fully read. Stopped after " << payload.count() << " bytes, while " << payload.available() << " bytes are in buffer."; throw Exception(tmp.str(), ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT); } diff --git a/src/Core/MySQL/IMySQLWritePacket.cpp b/src/Core/MySQL/IMySQLWritePacket.cpp index f5bc339b079..3e97800177c 100644 --- a/src/Core/MySQL/IMySQLWritePacket.cpp +++ b/src/Core/MySQL/IMySQLWritePacket.cpp @@ -16,6 +16,7 @@ void IMySQLWritePacket::writePayload(WriteBuffer & buffer, uint8_t & sequence_id if (buf.remainingPayloadSize()) { std::stringstream ss; + ss.exceptions(std::ios::failbit); ss << "Incomplete payload. Written " << getPayloadSize() - buf.remainingPayloadSize() << " bytes, expected " << getPayloadSize() << " bytes."; throw Exception(ss.str(), 0); } diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index bd3b7bc45ff..98229bb73d7 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -61,6 +61,7 @@ struct SortColumnDescription std::string dump() const { std::stringstream ss; + ss.exceptions(std::ios::failbit); ss << column_name << ":" << column_number << ":dir " << direction << "nulls " << nulls_direction; return ss.str(); } diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 7a67074dbdf..d47d82689de 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -60,6 +60,7 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) if (!value) { std::stringstream exception_message; + exception_message.exceptions(std::ios::failbit); exception_message << "Constraint " << backQuote(constraint_ptr->name) << " for table " << table_id.getNameForLogs() @@ -87,6 +88,7 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) Names related_columns = constraint_expr->getRequiredColumns(); std::stringstream exception_message; + exception_message.exceptions(std::ios::failbit); exception_message << "Constraint " << backQuote(constraint_ptr->name) << " for table " << table_id.getNameForLogs() diff --git a/src/DataStreams/IBlockInputStream.cpp b/src/DataStreams/IBlockInputStream.cpp index e954225fdf9..23ba9ff2970 100644 --- a/src/DataStreams/IBlockInputStream.cpp +++ b/src/DataStreams/IBlockInputStream.cpp @@ -360,6 +360,7 @@ Block IBlockInputStream::getExtremes() String IBlockInputStream::getTreeID() const { std::stringstream s; + s.exceptions(std::ios::failbit); s << getName(); if (!children.empty()) diff --git a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp index 2583f4f2753..3fe17a4bbfc 100644 --- a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp +++ b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp @@ -33,6 +33,7 @@ static const std::vector supported_functions{"any", "anyLast", "min", String DataTypeCustomSimpleAggregateFunction::getName() const { std::stringstream stream; + stream.exceptions(std::ios::failbit); stream << "SimpleAggregateFunction(" << function->getName(); if (!parameters.empty()) diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp index 5aefd39fb16..bd4329f6f58 100644 --- a/src/DataTypes/DataTypesDecimal.cpp +++ b/src/DataTypes/DataTypesDecimal.cpp @@ -30,6 +30,7 @@ template std::string DataTypeDecimal::doGetName() const { std::stringstream ss; + ss.exceptions(std::ios::failbit); ss << "Decimal(" << this->precision << ", " << this->scale << ")"; return ss.str(); } diff --git a/src/DataTypes/tests/gtest_data_type_get_common_type.cpp b/src/DataTypes/tests/gtest_data_type_get_common_type.cpp index 8212555e8bc..5256d49ba6d 100644 --- a/src/DataTypes/tests/gtest_data_type_get_common_type.cpp +++ b/src/DataTypes/tests/gtest_data_type_get_common_type.cpp @@ -27,6 +27,7 @@ static auto typeFromString(const std::string & str) static auto typesFromString(const std::string & str) { std::istringstream data_types_stream(str); + data_types_stream.exceptions(std::ios::failbit); DataTypes data_types; std::string data_type; while (data_types_stream >> data_type) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 4d7fcd4e248..83e70a25f87 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -95,6 +95,7 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) if (!create) { std::ostringstream query_stream; + query_stream.exceptions(std::ios::failbit); formatAST(*query, query_stream, true); throw Exception("Query '" + query_stream.str() + "' is not CREATE query", ErrorCodes::LOGICAL_ERROR); } @@ -121,6 +122,7 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) create->table = TABLE_WITH_UUID_NAME_PLACEHOLDER; std::ostringstream statement_stream; + statement_stream.exceptions(std::ios::failbit); formatAST(*create, statement_stream, false); statement_stream << '\n'; return statement_stream.str(); diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index e26f5c2fd52..d86462a41bc 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -128,6 +128,7 @@ static String checkVariableAndGetVersion(const mysqlxx::Pool::Entry & connection bool first = true; std::stringstream error_message; + error_message.exceptions(std::ios::failbit); error_message << "Illegal MySQL variables, the MaterializeMySQL engine requires "; for (const auto & [variable_name, variable_error_message] : variables_error_message) { @@ -239,6 +240,7 @@ static inline BlockOutputStreamPtr getTableOutput(const String & database_name, const StoragePtr & storage = DatabaseCatalog::instance().getTable(StorageID(database_name, table_name), query_context); std::stringstream insert_columns_str; + insert_columns_str.exceptions(std::ios::failbit); const StorageInMemoryMetadata & storage_metadata = storage->getInMemoryMetadata(); const ColumnsDescription & storage_columns = storage_metadata.getColumns(); const NamesAndTypesList & insert_columns_names = insert_materialized ? storage_columns.getAllPhysical() : storage_columns.getOrdinary(); @@ -330,6 +332,7 @@ std::optional MaterializeMySQLSyncThread::prepareSynchroniz const auto & position_message = [&]() { std::stringstream ss; + ss.exceptions(std::ios::failbit); position.dump(ss); return ss.str(); }; @@ -372,6 +375,7 @@ void MaterializeMySQLSyncThread::flushBuffersData(Buffers & buffers, Materialize const auto & position_message = [&]() { std::stringstream ss; + ss.exceptions(std::ios::failbit); client.getPosition().dump(ss); return ss.str(); }; @@ -643,6 +647,7 @@ void MaterializeMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPtr const auto & dump_event_message = [&]() { std::stringstream ss; + ss.exceptions(std::ios::failbit); receive_event->dump(ss); return ss.str(); }; diff --git a/src/Dictionaries/DictionaryStructure.cpp b/src/Dictionaries/DictionaryStructure.cpp index fca56442e6f..4c7cc5b4118 100644 --- a/src/Dictionaries/DictionaryStructure.cpp +++ b/src/Dictionaries/DictionaryStructure.cpp @@ -231,6 +231,7 @@ std::string DictionaryStructure::getKeyDescription() const return "UInt64"; std::ostringstream out; + out.exceptions(std::ios::failbit); out << '('; diff --git a/src/Dictionaries/tests/gtest_dictionary_configuration.cpp b/src/Dictionaries/tests/gtest_dictionary_configuration.cpp index 453ce2b81f0..62422124bd8 100644 --- a/src/Dictionaries/tests/gtest_dictionary_configuration.cpp +++ b/src/Dictionaries/tests/gtest_dictionary_configuration.cpp @@ -19,6 +19,7 @@ static std::string configurationToString(const DictionaryConfigurationPtr & conf { const Poco::Util::XMLConfiguration * xml_config = dynamic_cast(config.get()); std::ostringstream oss; + oss.exceptions(std::ios::failbit); xml_config->save(oss); return oss.str(); } diff --git a/src/Functions/abtesting.cpp b/src/Functions/abtesting.cpp index c57119d1c34..25c0abbdded 100644 --- a/src/Functions/abtesting.cpp +++ b/src/Functions/abtesting.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #define STATS_ENABLE_STDVEC_WRAPPERS #include @@ -139,31 +139,29 @@ Variants bayesian_ab_test(String distribution, PODArray & xs, PODArray< String convertToJson(const PODArray & variant_names, const Variants & variants) { FormatSettings settings; - std::stringstream s; + WriteBufferOwnString buf; + + writeCString("{\"data\":[", buf); + for (size_t i = 0; i < variants.size(); ++i) { - WriteBufferFromOStream buf(s); - - writeCString("{\"data\":[", buf); - for (size_t i = 0; i < variants.size(); ++i) - { - writeCString("{\"variant_name\":", buf); - writeJSONString(variant_names[i], buf, settings); - writeCString(",\"x\":", buf); - writeText(variants[i].x, buf); - writeCString(",\"y\":", buf); - writeText(variants[i].y, buf); - writeCString(",\"beats_control\":", buf); - writeText(variants[i].beats_control, buf); - writeCString(",\"to_be_best\":", buf); - writeText(variants[i].best, buf); - writeCString("}", buf); - if (i != variant_names.size() -1) writeCString(",", buf); - } - writeCString("]}", buf); + writeCString("{\"variant_name\":", buf); + writeJSONString(variant_names[i], buf, settings); + writeCString(",\"x\":", buf); + writeText(variants[i].x, buf); + writeCString(",\"y\":", buf); + writeText(variants[i].y, buf); + writeCString(",\"beats_control\":", buf); + writeText(variants[i].beats_control, buf); + writeCString(",\"to_be_best\":", buf); + writeText(variants[i].best, buf); + writeCString("}", buf); + if (i != variant_names.size() -1) + writeCString(",", buf); } + writeCString("]}", buf); - return s.str(); + return buf.str(); } class FunctionBayesAB : public IFunction diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index bda615edcd5..04fec145775 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -240,6 +240,7 @@ void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPR if (!(status == Poco::Net::HTTPResponse::HTTP_OK || (isRedirect(status) && allow_redirects))) { std::stringstream error_message; + error_message.exceptions(std::ios::failbit); error_message << "Received error from remote server " << request.getURI() << ". HTTP status code: " << status << " " << response.getReason() << ", body: " << istr.rdbuf(); diff --git a/src/IO/MySQLPacketPayloadReadBuffer.cpp b/src/IO/MySQLPacketPayloadReadBuffer.cpp index 16b1cd5de19..f6f899e0ac7 100644 --- a/src/IO/MySQLPacketPayloadReadBuffer.cpp +++ b/src/IO/MySQLPacketPayloadReadBuffer.cpp @@ -27,20 +27,14 @@ bool MySQLPacketPayloadReadBuffer::nextImpl() in.readStrict(reinterpret_cast(&payload_length), 3); if (payload_length > MAX_PACKET_LENGTH) - { - std::ostringstream tmp; - tmp << "Received packet with payload larger than max_packet_size: " << payload_length; - throw Exception(tmp.str(), ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT); - } + throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT, + "Received packet with payload larger than max_packet_size: {}", payload_length); size_t packet_sequence_id = 0; in.read(reinterpret_cast(packet_sequence_id)); if (packet_sequence_id != sequence_id) - { - std::ostringstream tmp; - tmp << "Received packet with wrong sequence-id: " << packet_sequence_id << ". Expected: " << static_cast(sequence_id) << '.'; - throw Exception(tmp.str(), ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT); - } + throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT, + "Received packet with wrong sequence-id: {}. Expected: {}.", packet_sequence_id, static_cast(sequence_id)); sequence_id++; if (payload_length == 0) diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index ee6fcc58ab0..267800d8900 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -72,10 +72,7 @@ public: } else { - std::stringstream error_message; - error_message << "Too many redirects while trying to access " << initial_uri.toString(); - - throw Exception(error_message.str(), ErrorCodes::TOO_MANY_REDIRECTS); + throw Exception(ErrorCodes::TOO_MANY_REDIRECTS, "Too many redirects while trying to access {}", initial_uri.toString()); } } diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index a649b76730b..49ccb6dc1b3 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -248,6 +248,7 @@ void PocoHTTPClient::makeRequestInternal( response->SetContentType(poco_response.getContentType()); std::stringstream headers_ss; + headers_ss.exceptions(std::ios::failbit); for (const auto & [header_name, header_value] : poco_response) { response->AddHeader(header_name, header_value); diff --git a/src/IO/tests/gtest_bit_io.cpp b/src/IO/tests/gtest_bit_io.cpp index f75abf92f30..dce146eaad7 100644 --- a/src/IO/tests/gtest_bit_io.cpp +++ b/src/IO/tests/gtest_bit_io.cpp @@ -77,6 +77,7 @@ std::string dumpContents(const T& container, { std::stringstream sstr; + sstr.exceptions(std::ios::failbit); dumpBuffer(std::begin(container), std::end(container), &sstr, col_sep, row_sep, cols_in_row); return sstr.str(); diff --git a/src/IO/tests/hashing_read_buffer.cpp b/src/IO/tests/hashing_read_buffer.cpp index dbec6b2374b..a1140160c32 100644 --- a/src/IO/tests/hashing_read_buffer.cpp +++ b/src/IO/tests/hashing_read_buffer.cpp @@ -23,6 +23,7 @@ static void test(size_t data_size) { std::cout << "block size " << read_buffer_block_size << std::endl; std::stringstream io; + io.exceptions(std::ios::failbit); DB::WriteBufferFromOStream out_impl(io); DB::HashingWriteBuffer out(out_impl); out.write(data, data_size); diff --git a/src/IO/tests/limit_read_buffer2.cpp b/src/IO/tests/limit_read_buffer2.cpp index 826fb048a0c..416eae0966b 100644 --- a/src/IO/tests/limit_read_buffer2.cpp +++ b/src/IO/tests/limit_read_buffer2.cpp @@ -21,6 +21,7 @@ try using namespace DB; std::stringstream s; + s.exceptions(std::ios::failbit); { std::string src = "1"; diff --git a/src/IO/tests/write_buffer.cpp b/src/IO/tests/write_buffer.cpp index 14beb6d0539..c0e9150d372 100644 --- a/src/IO/tests/write_buffer.cpp +++ b/src/IO/tests/write_buffer.cpp @@ -17,6 +17,7 @@ int main(int, char **) DB::String d = "'xyz\\"; std::stringstream s; + s.exceptions(std::ios::failbit); { DB::WriteBufferFromOStream out(s); diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 7d99cbd1d43..9e64695d1a0 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -107,6 +107,7 @@ String formattedAST(const ASTPtr & ast) if (!ast) return {}; std::stringstream ss; + ss.exceptions(std::ios::failbit); formatAST(*ast, ss, false, true); return ss.str(); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index befb097faf7..fb6e218b33b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1962,6 +1962,7 @@ void Context::checkCanBeDropped(const String & database, const String & table, c String size_str = formatReadableSizeWithDecimalSuffix(size); String max_size_to_drop_str = formatReadableSizeWithDecimalSuffix(max_size_to_drop); std::stringstream ostr; + ostr.exceptions(std::ios::failbit); ostr << "Table or Partition in " << backQuoteIfNeed(database) << "." << backQuoteIfNeed(table) << " was not dropped.\n" << "Reason:\n" diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 762ad6ae575..75872553ec3 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -454,6 +454,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const std::string ExpressionAction::toString() const { std::stringstream ss; + ss.exceptions(std::ios::failbit); switch (type) { case ADD_COLUMN: @@ -550,6 +551,7 @@ void ExpressionActions::checkLimits(Block & block) const if (non_const_columns > settings.max_temporary_non_const_columns) { std::stringstream list_of_non_const_columns; + list_of_non_const_columns.exceptions(std::ios::failbit); for (size_t i = 0, size = block.columns(); i < size; ++i) if (block.safeGetByPosition(i).column && !isColumnConst(*block.safeGetByPosition(i).column)) list_of_non_const_columns << "\n" << block.safeGetByPosition(i).name; @@ -921,6 +923,7 @@ void ExpressionActions::finalize(const Names & output_columns) std::string ExpressionActions::dumpActions() const { std::stringstream ss; + ss.exceptions(std::ios::failbit); ss << "input:\n"; for (const auto & input_column : input_columns) @@ -1342,6 +1345,7 @@ void ExpressionActionsChain::finalize() std::string ExpressionActionsChain::dumpChain() const { std::stringstream ss; + ss.exceptions(std::ios::failbit); for (size_t i = 0; i < steps.size(); ++i) { diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6a8bdbea1ec..286d5269a64 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -136,6 +136,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) { /// Currently, there are no database engines, that support any arguments. std::stringstream ostr; + ostr.exceptions(std::ios::failbit); formatAST(*create.storage, ostr, false, false); throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE); } @@ -182,6 +183,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) create.if_not_exists = false; std::ostringstream statement_stream; + statement_stream.exceptions(std::ios::failbit); formatAST(create, statement_stream, false); statement_stream << '\n'; String statement = statement_stream.str(); diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index a0a63dfed08..ed791f0d592 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -223,6 +223,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() MutableColumns res_columns = sample_block.cloneEmptyColumns(); std::stringstream ss; + ss.exceptions(std::ios::failbit); if (ast.getKind() == ASTExplainQuery::ParsedAST) { diff --git a/src/Interpreters/InterpreterShowAccessQuery.cpp b/src/Interpreters/InterpreterShowAccessQuery.cpp index c9541b4f5bf..5f28c49c0bc 100644 --- a/src/Interpreters/InterpreterShowAccessQuery.cpp +++ b/src/Interpreters/InterpreterShowAccessQuery.cpp @@ -35,6 +35,7 @@ BlockInputStreamPtr InterpreterShowAccessQuery::executeImpl() const /// Build the result column. MutableColumnPtr column = ColumnString::create(); std::stringstream ss; + ss.exceptions(std::ios::failbit); for (const auto & query : queries) { ss.str(""); diff --git a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp index 8d5f27e116d..749a5811e13 100644 --- a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp @@ -239,6 +239,7 @@ BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl() /// Build the result column. MutableColumnPtr column = ColumnString::create(); std::stringstream create_query_ss; + create_query_ss.exceptions(std::ios::failbit); for (const auto & create_query : create_queries) { formatAST(*create_query, create_query_ss, false, true); @@ -248,6 +249,7 @@ BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl() /// Prepare description of the result column. std::stringstream desc_ss; + desc_ss.exceptions(std::ios::failbit); const auto & show_query = query_ptr->as(); formatAST(show_query, desc_ss, false, true); String desc = desc_ss.str(); diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index b14baaafbb9..8861914a68a 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -79,6 +79,7 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() } std::stringstream stream; + stream.exceptions(std::ios::failbit); formatAST(*create_query, stream, false, false); String res = stream.str(); diff --git a/src/Interpreters/InterpreterShowGrantsQuery.cpp b/src/Interpreters/InterpreterShowGrantsQuery.cpp index 45e065dcfd9..7de51b6a7ee 100644 --- a/src/Interpreters/InterpreterShowGrantsQuery.cpp +++ b/src/Interpreters/InterpreterShowGrantsQuery.cpp @@ -119,6 +119,7 @@ BlockInputStreamPtr InterpreterShowGrantsQuery::executeImpl() /// Build the result column. MutableColumnPtr column = ColumnString::create(); std::stringstream grant_ss; + grant_ss.exceptions(std::ios::failbit); for (const auto & grant_query : grant_queries) { grant_ss.str(""); @@ -128,6 +129,7 @@ BlockInputStreamPtr InterpreterShowGrantsQuery::executeImpl() /// Prepare description of the result column. std::stringstream desc_ss; + desc_ss.exceptions(std::ios::failbit); const auto & show_query = query_ptr->as(); formatAST(show_query, desc_ss, false, true); String desc = desc_ss.str(); diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index cb5db386f5a..8e67cecdd5e 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -33,6 +33,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery() if (query.databases) { std::stringstream rewritten_query; + rewritten_query.exceptions(std::ios::failbit); rewritten_query << "SELECT name FROM system.databases"; if (!query.like.empty()) @@ -54,6 +55,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery() if (query.clusters) { std::stringstream rewritten_query; + rewritten_query.exceptions(std::ios::failbit); rewritten_query << "SELECT DISTINCT cluster FROM system.clusters"; if (!query.like.empty()) @@ -73,6 +75,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery() else if (query.cluster) { std::stringstream rewritten_query; + rewritten_query.exceptions(std::ios::failbit); rewritten_query << "SELECT * FROM system.clusters"; rewritten_query << " WHERE cluster = " << std::quoted(query.cluster_str, '\''); @@ -87,6 +90,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery() DatabaseCatalog::instance().assertDatabaseExists(database); std::stringstream rewritten_query; + rewritten_query.exceptions(std::ios::failbit); rewritten_query << "SELECT name FROM system."; if (query.dictionaries) diff --git a/src/Interpreters/QueryAliasesVisitor.cpp b/src/Interpreters/QueryAliasesVisitor.cpp index 6eae5594810..9de1d04990d 100644 --- a/src/Interpreters/QueryAliasesVisitor.cpp +++ b/src/Interpreters/QueryAliasesVisitor.cpp @@ -21,6 +21,7 @@ namespace ErrorCodes static String wrongAliasMessage(const ASTPtr & ast, const ASTPtr & prev_ast, const String & alias) { std::stringstream message; + message.exceptions(std::ios::failbit); message << "Different expressions with the same alias " << backQuoteIfNeed(alias) << ":" << std::endl; formatAST(*ast, message, false, true); message << std::endl << "and" << std::endl; diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 13606e1d54c..907cbaee243 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -343,6 +343,7 @@ void Set::checkColumnsNumber(size_t num_key_columns) const if (data_types.size() != num_key_columns) { std::stringstream message; + message.exceptions(std::ios::failbit); message << "Number of columns in section IN doesn't match. " << num_key_columns << " at left, " << data_types.size() << " at right."; throw Exception(message.str(), ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index c8691c25f1b..8d3cb123955 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -553,6 +553,7 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (!unknown_required_source_columns.empty()) { std::stringstream ss; + ss.exceptions(std::ios::failbit); ss << "Missing columns:"; for (const auto & name : unknown_required_source_columns) ss << " '" << name << "'"; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 8faccf7bc7b..cdb3d9b7d7b 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -779,6 +779,7 @@ static std::tuple executeQueryImpl( if (!internal && res.in) { std::stringstream log_str; + log_str.exceptions(std::ios::failbit); log_str << "Query pipeline:\n"; res.in->dumpTree(log_str); LOG_DEBUG(&Poco::Logger::get("executeQuery"), log_str.str()); diff --git a/src/Interpreters/tests/CMakeLists.txt b/src/Interpreters/tests/CMakeLists.txt index 4ab7da014e4..20aa73166fb 100644 --- a/src/Interpreters/tests/CMakeLists.txt +++ b/src/Interpreters/tests/CMakeLists.txt @@ -37,9 +37,6 @@ add_executable (in_join_subqueries_preprocessor in_join_subqueries_preprocessor. target_link_libraries (in_join_subqueries_preprocessor PRIVATE clickhouse_aggregate_functions dbms clickhouse_parsers) add_check(in_join_subqueries_preprocessor) -add_executable (users users.cpp) -target_link_libraries (users PRIVATE clickhouse_aggregate_functions dbms clickhouse_common_config) - if (OS_LINUX) add_executable (internal_iotop internal_iotop.cpp) target_link_libraries (internal_iotop PRIVATE dbms) diff --git a/src/Interpreters/tests/users.cpp b/src/Interpreters/tests/users.cpp deleted file mode 100644 index acd0cfd0519..00000000000 --- a/src/Interpreters/tests/users.cpp +++ /dev/null @@ -1,282 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace -{ - -namespace fs = std::filesystem; - -struct TestEntry -{ - std::string user_name; - std::string database_name; - bool is_allowed; -}; - -using TestEntries = std::vector; - -struct TestDescriptor -{ - const char * config_content; - TestEntries entries; -}; - -using TestSet = std::vector; - -/// Tests description. - -TestSet test_set = -{ - { - "" - " " - " " - " " - " defaultdefault" - " " - " default" - " test" - " " - " " - " " - " defaultdefault" - " " - " " - " " - "", - - { - { "default", "default", true }, - { "default", "test", true }, - { "default", "stats", false }, - { "web", "default", true }, - { "web", "test", true }, - { "web", "stats", true }, - { "analytics", "default", false }, - { "analytics", "test", false }, - { "analytics", "stats", false } - } - }, - - { - "" - " " - " " - " " - " defaultdefault" - " " - " default" - " " - " " - " " - " defaultdefault" - " " - " " - " " - "", - - { - { "default", "default", true }, - { "default", "test", false }, - { "default", "stats", false }, - { "web", "default", true }, - { "web", "test", true }, - { "web", "stats", true }, - { "analytics", "default", false }, - { "analytics", "test", false }, - { "analytics", "stats", false } - } - }, - - { - "" - " " - " " - " " - " defaultdefault" - " " - " " - " " - " " - " defaultdefault" - " " - " " - " " - "", - - { - { "default", "default", true }, - { "default", "test", true }, - { "default", "stats", true }, - { "web", "default", true }, - { "web", "test", true }, - { "web", "stats", true }, - { "analytics", "default", false }, - { "analytics", "test", false }, - { "analytics", "stats", false } - } - }, - - { - "" - " " - " " - " " - " defaultdefault" - " " - " default" - " " - " " - " " - " defaultdefault" - " " - " test" - " " - " " - " " - " " - "", - - { - { "default", "default", true }, - { "default", "test", false }, - { "default", "stats", false }, - { "web", "default", false }, - { "web", "test", true }, - { "web", "stats", false }, - { "analytics", "default", false }, - { "analytics", "test", false }, - { "analytics", "stats", false } - } - } -}; - -std::string createTmpPath(const std::string & filename) -{ - char pattern[] = "/tmp/fileXXXXXX"; - char * dir = mkdtemp(pattern); - if (dir == nullptr) - throw std::runtime_error("Could not create directory"); - - return std::string(dir) + "/" + filename; -} - -void createFile(const std::string & filename, const char * data) -{ - std::ofstream ofs(filename.c_str()); - if (!ofs.is_open()) - throw std::runtime_error("Could not open file " + filename); - ofs << data; -} - -void runOneTest(const TestDescriptor & test_descriptor) -{ - const auto path_name = createTmpPath("users.xml"); - createFile(path_name, test_descriptor.config_content); - - DB::ConfigurationPtr config; - - try - { - config = DB::ConfigProcessor(path_name).loadConfig().configuration; - } - catch (const Poco::Exception & ex) - { - std::ostringstream os; - os << "Error: " << ex.what() << ": " << ex.displayText(); - throw std::runtime_error(os.str()); - } - - DB::AccessControlManager acl_manager; - - try - { - acl_manager.setUsersConfig(*config); - } - catch (const Poco::Exception & ex) - { - std::ostringstream os; - os << "Error: " << ex.what() << ": " << ex.displayText(); - throw std::runtime_error(os.str()); - } - - for (const auto & entry : test_descriptor.entries) - { - bool res; - - try - { - res = acl_manager.read(entry.user_name)->access.isGranted(DB::AccessType::ALL, entry.database_name); - } - catch (const Poco::Exception &) - { - res = false; - } - - if (res != entry.is_allowed) - { - auto to_string = [](bool access){ return (access ? "'granted'" : "'denied'"); }; - std::ostringstream os; - os << "(user=" << entry.user_name << ", database=" << entry.database_name << "): "; - os << "Expected " << to_string(entry.is_allowed) << " but got " << to_string(res); - throw std::runtime_error(os.str()); - } - } - - fs::remove_all(fs::path(path_name).parent_path().string()); -} - -auto runTestSet() -{ - size_t test_num = 1; - size_t failure_count = 0; - - for (const auto & test_descriptor : test_set) - { - try - { - runOneTest(test_descriptor); - std::cout << "Test " << test_num << " passed\n"; - } - catch (const std::runtime_error & ex) - { - std::cerr << "Test " << test_num << " failed with reason: " << ex.what() << "\n"; - ++failure_count; - } - catch (...) - { - std::cerr << "Test " << test_num << " failed with unknown reason\n"; - ++failure_count; - } - - ++test_num; - } - - return std::make_tuple(test_set.size(), failure_count); -} - -} - -int main() -{ - size_t test_count; - size_t failure_count; - - std::tie(test_count, failure_count) = runTestSet(); - - std::cout << (test_count - failure_count) << " test(s) passed out of " << test_count << "\n"; - - return (failure_count == 0) ? 0 : EXIT_FAILURE; -} diff --git a/src/Parsers/ASTCreateRowPolicyQuery.cpp b/src/Parsers/ASTCreateRowPolicyQuery.cpp index 640b030b6cf..6224b534851 100644 --- a/src/Parsers/ASTCreateRowPolicyQuery.cpp +++ b/src/Parsers/ASTCreateRowPolicyQuery.cpp @@ -63,6 +63,7 @@ namespace { std::vector> conditions_as_strings; std::stringstream temp_sstream; + temp_sstream.exceptions(std::ios::failbit); IAST::FormatSettings temp_settings(temp_sstream, settings); for (const auto & [condition_type, condition] : conditions) { diff --git a/src/Parsers/DumpASTNode.h b/src/Parsers/DumpASTNode.h index 430e70de8da..01447850c74 100644 --- a/src/Parsers/DumpASTNode.h +++ b/src/Parsers/DumpASTNode.h @@ -95,6 +95,7 @@ public: DebugASTLog() : log(nullptr) { + ss.exceptions(std::ios::failbit); if constexpr (_enable) log = &Poco::Logger::get("AST"); } diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 8ee4154541b..d716a796b77 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -90,6 +90,7 @@ size_t IAST::checkDepthImpl(size_t max_depth, size_t level) const std::string IAST::formatForErrorMessage() const { std::stringstream ss; + ss.exceptions(std::ios::failbit); format(FormatSettings(ss, true /* one line */)); return ss.str(); } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index c88c80021d6..cc9e593d7cb 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -243,6 +243,7 @@ template std::string IAST::formatForErrorMessage(const AstArray & array) { std::stringstream ss; + ss.exceptions(std::ios::failbit); for (size_t i = 0; i < array.size(); ++i) { if (i > 0) diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index fca5130cb89..e19dc715d51 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -16,6 +16,7 @@ void formatAST(const IAST & ast, std::ostream & s, bool hilite, bool one_line) String serializeAST(const IAST & ast, bool one_line) { std::stringstream ss; + ss.exceptions(std::ios::failbit); formatAST(ast, ss, false, one_line); return ss.str(); } diff --git a/src/Parsers/queryToString.cpp b/src/Parsers/queryToString.cpp index d214468c2a9..44ea721485f 100644 --- a/src/Parsers/queryToString.cpp +++ b/src/Parsers/queryToString.cpp @@ -12,6 +12,7 @@ namespace DB String queryToString(const IAST & query) { std::ostringstream out; + out.exceptions(std::ios::failbit); formatAST(query, out, false, true); return out.str(); } diff --git a/src/Parsers/tests/gtest_dictionary_parser.cpp b/src/Parsers/tests/gtest_dictionary_parser.cpp index c2bde5fa8f1..c418759aa21 100644 --- a/src/Parsers/tests/gtest_dictionary_parser.cpp +++ b/src/Parsers/tests/gtest_dictionary_parser.cpp @@ -18,6 +18,7 @@ using namespace DB; static String astToString(IAST * ast) { std::ostringstream oss; + oss.exceptions(std::ios::failbit); dumpAST(*ast, oss); return oss.str(); } diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index cf7a020ee0b..8a416ade740 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -161,6 +161,7 @@ static void insertNumber(IColumn & column, WhichDataType type, T value) static std::string nodeToJson(avro::NodePtr root_node) { std::ostringstream ss; + ss.exceptions(std::ios::failbit); root_node->printJson(ss, 0); return ss.str(); } diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 628a90beefb..96a458eb49f 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -168,6 +168,11 @@ void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) std::stringstream middle_values_separator; std::stringstream bottom_separator; + top_separator.exceptions(std::ios::failbit); + middle_names_separator.exceptions(std::ios::failbit); + middle_values_separator.exceptions(std::ios::failbit); + bottom_separator.exceptions(std::ios::failbit); + top_separator << grid_symbols.bold_left_top_corner; middle_names_separator << grid_symbols.bold_left_separator; middle_values_separator << grid_symbols.left_separator; diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index b81ba84c732..9320b159836 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -134,6 +134,7 @@ void PrettyCompactBlockOutputFormat::writeBottom(const Widths & max_widths) ascii_grid_symbols; /// Create delimiters std::stringstream bottom_separator; + bottom_separator.exceptions(std::ios::failbit); bottom_separator << grid_symbols.left_bottom_corner; for (size_t i = 0; i < max_widths.size(); ++i) diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp index 697ac9496b5..e7a7200ac34 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp @@ -42,6 +42,7 @@ void CollapsingSortedAlgorithm::reportIncorrectData() return; std::stringstream s; + s.exceptions(std::ios::failbit); auto & sort_columns = *last_row.sort_columns; for (size_t i = 0, size = sort_columns.size(); i < size; ++i) { diff --git a/src/Server/ReplicasStatusHandler.cpp b/src/Server/ReplicasStatusHandler.cpp index 1aa5c10afd7..bf7a3b8ab52 100644 --- a/src/Server/ReplicasStatusHandler.cpp +++ b/src/Server/ReplicasStatusHandler.cpp @@ -37,6 +37,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request bool ok = true; std::stringstream message; + message.exceptions(std::ios::failbit); auto databases = DatabaseCatalog::instance().getDatabases(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 1b0cbc69b29..4dceb0aa905 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -437,6 +437,7 @@ bool TCPHandler::readDataNext(const size_t & poll_interval, const int & receive_ if (elapsed > receive_timeout) { std::stringstream ss; + ss.exceptions(std::ios::failbit); ss << "Timeout exceeded while receiving data from client."; ss << " Waited for " << static_cast(elapsed) << " seconds,"; ss << " timeout is " << receive_timeout << " seconds."; diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index f08cdf76cbf..c9f4ffe8b6a 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -148,6 +148,7 @@ void DistributedBlockOutputStream::writeAsync(const Block & block) std::string DistributedBlockOutputStream::getCurrentStateDescription() { std::stringstream buffer; + buffer.exceptions(std::ios::failbit); const auto & addresses = cluster->getShardsAddresses(); buffer << "Insertion status:\n"; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index a63a4309775..22b4026340c 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -247,6 +247,7 @@ Names StorageKafka::parseTopics(String topic_list) String StorageKafka::getDefaultClientId(const StorageID & table_id_) { std::stringstream ss; + ss.exceptions(std::ios::failbit); ss << VERSION_NAME << "-" << getFQDNOrHostName() << "-" << table_id_.database_name << "-" << table_id_.table_name; return ss.str(); } @@ -400,6 +401,7 @@ ConsumerBufferPtr StorageKafka::createReadBuffer(const size_t consumer_number) if (num_consumers > 1) { std::stringstream ss; + ss.exceptions(std::ios::failbit); ss << client_id << "-" << consumer_number; conf.set("client.id", ss.str()); } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 7cb872f174a..3ccb2f8e2a6 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -33,6 +33,7 @@ namespace ErrorCodes String Range::toString() const { std::stringstream str; + str.exceptions(std::ios::failbit); if (!left_bounded) str << "(-inf, "; @@ -1443,6 +1444,7 @@ String KeyCondition::RPNElement::toString() const }; std::ostringstream ss; + ss.exceptions(std::ios::failbit); switch (function) { case FUNCTION_AND: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1a7062766b0..7884dc7beaa 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3206,6 +3206,7 @@ void MergeTreeData::Transaction::rollbackPartsToTemporaryState() if (!isEmpty()) { std::stringstream ss; + ss.exceptions(std::ios::failbit); ss << " Rollbacking parts state to temporary and removing from working set:"; for (const auto & part : precommitted_parts) ss << " " << part->relative_path; @@ -3224,6 +3225,7 @@ void MergeTreeData::Transaction::rollback() if (!isEmpty()) { std::stringstream ss; + ss.exceptions(std::ios::failbit); ss << " Removing parts:"; for (const auto & part : precommitted_parts) ss << " " << part->relative_path; @@ -3759,6 +3761,7 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S || settings.min_rows_for_compact_part != 0 || settings.min_bytes_for_compact_part != 0)) { std::ostringstream message; + message.exceptions(std::ios::failbit); message << "Table can't create parts with adaptive granularity, but settings" << " min_rows_for_wide_part = " << settings.min_rows_for_wide_part << ", min_bytes_for_wide_part = " << settings.min_bytes_for_wide_part diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index c75970f6cc1..58bdbcdcdcd 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -443,6 +443,7 @@ void MinimalisticDataPartChecksums::checkEqualImpl(const MinimalisticDataPartChe if (num_compressed_files != rhs.num_compressed_files || num_uncompressed_files != rhs.num_uncompressed_files) { std::stringstream error_msg; + error_msg.exceptions(std::ios::failbit); error_msg << "Different number of files: " << rhs.num_compressed_files << " compressed (expected " << num_compressed_files << ")" << " and " << rhs.num_uncompressed_files << " uncompressed ones (expected " << num_uncompressed_files << ")"; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 7eb21503fe6..e34096b1309 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -228,6 +228,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue()) { std::stringstream exception_message; + exception_message.exceptions(std::ios::failbit); exception_message << "Primary key ("; for (size_t i = 0, size = primary_key_columns.size(); i < size; ++i) exception_message << (i == 0 ? "" : ", ") << primary_key_columns[i]; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 4aa186473a9..597ff5e8fee 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -942,6 +942,7 @@ size_t ReplicatedMergeTreeQueue::getConflictsCountForRange( if (out_description) { std::stringstream ss; + ss.exceptions(std::ios::failbit); ss << "Can't execute command for range " << range.getPartName() << " (entry " << entry.znode_name << "). "; ss << "There are " << conflicts.size() << " currently executing entries blocking it: "; for (const auto & conflict : conflicts) @@ -1693,6 +1694,7 @@ std::vector ReplicatedMergeTreeQueue::getMutationsStatu for (const MutationCommand & command : entry.commands) { std::stringstream ss; + ss.exceptions(std::ios::failbit); formatAST(*command.ast, ss, false, true); result.push_back(MergeTreeMutationStatus { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 378b1b284a3..48f05b50675 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -19,6 +19,7 @@ static String formattedAST(const ASTPtr & ast) if (!ast) return ""; std::stringstream ss; + ss.exceptions(std::ios::failbit); formatAST(*ast, ss, false, true); return ss.str(); } diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index ba998dd5951..53c9b50cb9d 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -127,6 +127,7 @@ std::shared_ptr MutationCommands::ast() const void MutationCommands::writeText(WriteBuffer & out) const { std::stringstream commands_ss; + commands_ss.exceptions(std::ios::failbit); formatAST(*ast(), commands_ss, /* hilite = */ false, /* one_line = */ true); out << escape << commands_ss.str(); } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 9735c4d7fd3..55cecf6f202 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -187,6 +187,7 @@ AMQP::ExchangeType StorageRabbitMQ::defineExchangeType(String exchange_type_) String StorageRabbitMQ::getTableBasedName(String name, const StorageID & table_id) { std::stringstream ss; + ss.exceptions(std::ios::failbit); if (name.empty()) ss << table_id.database_name << "_" << table_id.table_name; diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index e859baa702e..a81c80cf466 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -82,6 +82,7 @@ NamesAndTypesList StorageDictionary::getNamesAndTypes(const DictionaryStructure String StorageDictionary::generateNamesAndTypesDescription(const NamesAndTypesList & list) { std::stringstream ss; + ss.exceptions(std::ios::failbit); bool first = true; for (const auto & name_and_type : list) { diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 4409be73e52..a6f8cde37c9 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -176,6 +176,7 @@ UInt64 getMaximumFileNumber(const std::string & dir_path) std::string makeFormattedListOfShards(const ClusterPtr & cluster) { std::ostringstream os; + os.exceptions(std::ios::failbit); bool head = true; os << "["; @@ -749,6 +750,7 @@ ClusterPtr StorageDistributed::getOptimizedCluster(const Context & context, cons if (force) { std::stringstream exception_message; + exception_message.exceptions(std::ios::failbit); if (!has_sharding_key) exception_message << "No sharding key"; else if (!sharding_key_is_usable) diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index f410fa34f59..5cc435f91fa 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -428,6 +428,7 @@ namespace String listOfColumns(const NamesAndTypesList & available_columns) { std::stringstream ss; + ss.exceptions(std::ios::failbit); for (auto it = available_columns.begin(); it != available_columns.end(); ++it) { if (it != available_columns.begin()) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 11e8859e76c..b8b40356c52 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -555,6 +555,7 @@ std::vector StorageMergeTree::getMutationsStatus() cons for (const MutationCommand & command : entry.commands) { std::stringstream ss; + ss.exceptions(std::ios::failbit); formatAST(*command.ast, ss, false, true); result.push_back(MergeTreeMutationStatus { @@ -1030,6 +1031,7 @@ bool StorageMergeTree::optimize( if (!merge(true, partition_id, true, deduplicate, &disable_reason)) { std::stringstream message; + message.exceptions(std::ios::failbit); message << "Cannot OPTIMIZE table"; if (!disable_reason.empty()) message << ": " << disable_reason; @@ -1052,6 +1054,7 @@ bool StorageMergeTree::optimize( if (!merge(true, partition_id, final, deduplicate, &disable_reason)) { std::stringstream message; + message.exceptions(std::ios::failbit); message << "Cannot OPTIMIZE table"; if (!disable_reason.empty()) message << ": " << disable_reason; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7412031c595..9065bfd2d1d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -460,6 +460,7 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( if (!inactive_replicas.empty()) { std::stringstream exception_message; + exception_message.exceptions(std::ios::failbit); exception_message << "Mutation is not finished because"; if (!inactive_replicas.empty()) @@ -1017,6 +1018,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) parts_to_fetch_blocks += get_blocks_count_in_data_part(name); std::stringstream sanity_report; + sanity_report.exceptions(std::ios::failbit); sanity_report << "There are " << unexpected_parts.size() << " unexpected parts with " << unexpected_parts_rows << " rows (" << unexpected_parts_nonnew << " of them is not just-written with " << unexpected_parts_rows << " rows), " @@ -1041,6 +1043,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) if (insane && !skip_sanity_checks) { std::stringstream why; + why.exceptions(std::ios::failbit); why << "The local set of parts of table " << getStorageID().getNameForLogs() << " doesn't look like the set of parts " << "in ZooKeeper: " << formatReadableQuantity(unexpected_parts_rows) << " rows of " << formatReadableQuantity(total_rows_on_filesystem) @@ -1342,6 +1345,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) // Log source part names just in case { std::stringstream source_parts_msg; + source_parts_msg.exceptions(std::ios::failbit); for (auto i : ext::range(0, entry.source_parts.size())) source_parts_msg << (i != 0 ? ", " : "") << entry.source_parts[i]; @@ -3824,6 +3828,7 @@ bool StorageReplicatedMergeTree::optimize( if (!selected) { std::stringstream message; + message.exceptions(std::ios::failbit); message << "Cannot select parts for optimization"; if (!disable_reason.empty()) message << ": " << disable_reason; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index e4228c0d4ec..67a81045f2e 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -255,6 +255,7 @@ Strings listFilesWithRegexpMatching(Aws::S3::S3Client & client, const S3::URI & if (!outcome.IsSuccess()) { std::ostringstream message; + message.exceptions(std::ios::failbit); message << "Could not list objects in bucket " << quoteString(request.GetBucket()) << " with prefix " << quoteString(request.GetPrefix()); diff --git a/src/Storages/System/StorageSystemUsers.cpp b/src/Storages/System/StorageSystemUsers.cpp index 9f0b81263e3..70c67683b25 100644 --- a/src/Storages/System/StorageSystemUsers.cpp +++ b/src/Storages/System/StorageSystemUsers.cpp @@ -97,6 +97,7 @@ void StorageSystemUsers::fillData(MutableColumns & res_columns, const Context & auth_params_json.set("server", authentication.getServerName()); std::ostringstream oss; + oss.exceptions(std::ios::failbit); Poco::JSON::Stringifier::stringify(auth_params_json, oss); const auto str = oss.str(); diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 8de14b53471..e29cfabf328 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -128,6 +128,7 @@ std::string readData(DB::StoragePtr & table, const DB::Context & context) } std::ostringstream ss; + ss.exceptions(std::ios::failbit); WriteBufferFromOStream out_buf(ss); BlockOutputStreamPtr output = FormatFactory::instance().getOutput("Values", out_buf, sample, context); diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 2556cd10648..3148ab1112a 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -221,6 +221,7 @@ String transformQueryForExternalDatabase( dropAliases(select_ptr); std::stringstream out; + out.exceptions(std::ios::failbit); IAST::FormatSettings settings(out, true); settings.identifier_quoting_style = identifier_quoting_style; settings.always_quote_identifiers = identifier_quoting_style != IdentifierQuotingStyle::None; diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 2e34e82ce36..22a07a4d284 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -243,6 +243,7 @@ TableFunctionRemote::TableFunctionRemote(const std::string & name_, bool secure_ is_cluster_function = (name == "cluster" || name == "clusterAllReplicas"); std::stringstream ss; + ss.exceptions(std::ios::failbit); ss << "Table function '" << name + "' requires from 2 to " << (is_cluster_function ? 3 : 5) << " parameters" << ": , , " << (is_cluster_function ? "" : ", [username, [password]].");