remove more stringstreams

This commit is contained in:
Alexander Tokmakov 2020-11-10 21:22:26 +03:00
parent 55631e442b
commit b94cc5c4e5
93 changed files with 323 additions and 471 deletions

View File

@ -77,7 +77,7 @@ public:
private:
Connection * conn;
std::ostringstream query_buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
std::ostringstream query_buf;
void executeImpl();
};

View File

@ -1189,6 +1189,7 @@ private:
fprintf(stderr, "dump after fuzz:\n");
WriteBufferFromOStream cerr_buf(std::cerr, 4096);
fuzz_base->dumpTree(cerr_buf);
cerr_buf.next();
fmt::print(stderr, "IAST::clone() is broken for some AST node. This is a bug. The original AST ('dump before fuzz') and its cloned copy ('dump of cloned AST') refer to the same nodes, which must never happen. This means that their parent node doesn't implement clone() correctly.");
@ -1533,6 +1534,7 @@ private:
std::cout << std::endl;
WriteBufferFromOStream res_buf(std::cout, 4096);
formatAST(*res, res_buf);
res_buf.next();
std::cout << std::endl << std::endl;
}

View File

@ -86,7 +86,7 @@ Suggest::Suggest()
void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeouts, size_t suggestion_limit)
{
std::stringstream query;
std::stringstream query; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM ("
"SELECT name FROM system.functions"
" UNION ALL "

View File

@ -93,7 +93,7 @@ private:
void parse(const String & hint)
{
std::stringstream ss;
std::stringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
ss << hint;
String item;

View File

@ -162,7 +162,7 @@ void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts,
if (!missing_partitions.empty())
{
std::stringstream ss;
WriteBufferFromOwnString ss;
for (const String & missing_partition : missing_partitions)
ss << " " << missing_partition;

View File

@ -13,7 +13,7 @@ using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data)
{
std::stringstream ss(xml_data);
std::stringstream ss(xml_data); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
Poco::XML::InputSource input_source{ss};
return {new Poco::Util::XMLConfiguration{&input_source}};
}

View File

@ -394,12 +394,8 @@ inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain()
inline String DB::TaskShard::getDescription() const
{
std::stringstream ss;
ss << "N" << numberInCluster()
<< " (having a replica " << getHostNameExample()
<< ", pull table " + getQuotedTable(task_table.table_pull)
<< " of cluster " + task_table.cluster_pull_name << ")";
return ss.str();
return fmt::format("N{} (having a replica {}, pull table {} of cluster {}",
numberInCluster(), getHostNameExample(), getQuotedTable(task_table.table_pull), task_table.cluster_pull_name);
}
inline String DB::TaskShard::getHostNameExample() const

View File

@ -422,7 +422,7 @@ static const char * minimal_default_user_xml =
static ConfigurationPtr getConfigurationFromXMLString(const char * xml_data)
{
std::stringstream ss{std::string{xml_data}};
std::stringstream ss{std::string{xml_data}}; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
Poco::XML::InputSource input_source{ss};
return {new Poco::Util::XMLConfiguration{&input_source}};
}

View File

@ -191,10 +191,10 @@ int Server::run()
if (config().hasOption("help"))
{
Poco::Util::HelpFormatter help_formatter(Server::options());
std::stringstream header;
header << commandName() << " [OPTION] [-- [ARG]...]\n";
header << "positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010";
help_formatter.setHeader(header.str());
auto header_str = fmt::format("{} [OPTION] [-- [ARG]...]\n"
"positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010",
commandName());
help_formatter.setHeader(header_str);
help_formatter.format(std::cout);
return 0;
}

View File

@ -1,7 +1,5 @@
#pragma once
#include <iostream>
#include <sstream>
#include <unordered_set>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnArray.h>

View File

@ -1,10 +1,8 @@
#pragma once
#include <bitset>
#include <iostream>
#include <map>
#include <queue>
#include <sstream>
#include <unordered_set>
#include <utility>
#include <Columns/ColumnArray.h>

View File

@ -1,7 +1,5 @@
#pragma once
#include <iostream>
#include <sstream>
#include <unordered_set>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDateTime.h>

View File

@ -3,7 +3,6 @@
#include <limits>
#include <algorithm>
#include <climits>
#include <sstream>
#include <common/types.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadHelpers.h>

View File

@ -3,7 +3,6 @@
#include <limits>
#include <algorithm>
#include <climits>
#include <sstream>
#include <AggregateFunctions/ReservoirSampler.h>
#include <common/types.h>
#include <Common/HashTable/Hash.h>

View File

@ -71,7 +71,7 @@ void checkColumn(
std::unordered_map<UInt32, T> map;
size_t num_collisions = 0;
std::stringstream collisions_str;
std::stringstream collisions_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
collisions_str.exceptions(std::ios::failbit);
for (size_t i = 0; i < eq_class.size(); ++i)

View File

@ -5,7 +5,6 @@
#include <cstdlib>
#include <cstring>
#include <algorithm>
#include <sstream>
#include <functional>
#include <filesystem>
#include <Poco/DOM/Text.h>
@ -17,6 +16,8 @@
#include <Common/StringUtils/StringUtils.h>
#include <Common/Exception.h>
#include <common/getResource.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#define PREPROCESSED_SUFFIX "-preprocessed"
@ -537,8 +538,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
if (has_zk_includes)
*has_zk_includes = !contributing_zk_paths.empty();
std::stringstream comment;
comment.exceptions(std::ios::failbit);
WriteBufferFromOwnString comment;
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:";

View File

@ -245,8 +245,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);
WriteBufferFromOwnString stream;
try
{
@ -365,8 +364,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);
WriteBufferFromOwnString stream;
try
{

View File

@ -133,17 +133,12 @@ void MemoryTracker::alloc(Int64 size)
BlockerInThread untrack_lock;
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;
message << ": fault injected. Would use " << formatReadableSizeWithBinarySuffix(will_be)
<< " (attempt to allocate chunk of " << size << " bytes)"
<< ", maximum: " << formatReadableSizeWithBinarySuffix(current_hard_limit);
const auto * description = description_ptr.load(std::memory_order_relaxed);
amount.fetch_sub(size, std::memory_order_relaxed);
throw DB::Exception(message.str(), DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED);
throw DB::Exception(DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED,
"Memory tracker {}: fault injected. Would use {} (attempt to allocate chunk of {} bytes), maximum: {}",
description ? description : "", formatReadableSizeWithBinarySuffix(will_be),
size, formatReadableSizeWithBinarySuffix(current_hard_limit));
}
if (unlikely(current_profiler_limit && will_be > current_profiler_limit))
@ -166,17 +161,12 @@ void MemoryTracker::alloc(Int64 size)
BlockerInThread untrack_lock;
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;
message << " exceeded: would use " << formatReadableSizeWithBinarySuffix(will_be)
<< " (attempt to allocate chunk of " << size << " bytes)"
<< ", maximum: " << formatReadableSizeWithBinarySuffix(current_hard_limit);
const auto * description = description_ptr.load(std::memory_order_relaxed);
amount.fetch_sub(size, std::memory_order_relaxed);
throw DB::Exception(message.str(), DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED);
throw DB::Exception(DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED,
"Memory limit {} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}",
description ? description : "", formatReadableSizeWithBinarySuffix(will_be),
size, formatReadableSizeWithBinarySuffix(current_hard_limit));
}
updatePeak(will_be);

View File

@ -8,6 +8,7 @@
#include <common/logger_useful.h>
#include <common/errnoToString.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <unistd.h>
#include <csignal>
@ -73,8 +74,7 @@ ShellCommand::~ShellCommand()
void ShellCommand::logCommand(const char * filename, char * const argv[])
{
std::stringstream args;
args.exceptions(std::ios::failbit);
WriteBufferFromOwnString args;
for (int i = 0; argv != nullptr && argv[i] != nullptr; ++i)
{
if (i > 0)

View File

@ -23,7 +23,7 @@
std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext_t & context)
{
std::stringstream error;
std::stringstream error; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
error.exceptions(std::ios::failbit);
switch (sig)
{
@ -319,7 +319,7 @@ static void toStringEveryLineImpl(
const DB::SymbolIndex & symbol_index = DB::SymbolIndex::instance();
std::unordered_map<std::string, DB::Dwarf> dwarfs;
std::stringstream out;
std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
out.exceptions(std::ios::failbit);
for (size_t i = offset; i < size; ++i)
@ -359,7 +359,7 @@ static void toStringEveryLineImpl(
out.str({});
}
#else
std::stringstream out;
std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
out.exceptions(std::ios::failbit);
for (size_t i = offset; i < size; ++i)
@ -375,7 +375,7 @@ static void toStringEveryLineImpl(
static std::string toStringImpl(const StackTrace::FramePointers & frame_pointers, size_t offset, size_t size)
{
std::stringstream out;
std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
out.exceptions(std::ios::failbit);
toStringEveryLineImpl(frame_pointers, offset, size, [&](const std::string & str) { out << str << '\n'; });
return out.str();

View File

@ -153,7 +153,7 @@ std::pair<bool, std::string> StudentTTest::compareAndReport(size_t confidence_le
double mean_confidence_interval = table_value * t_statistic;
std::stringstream ss;
std::stringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
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.

View File

@ -1,5 +1,3 @@
#include <sstream>
#include <Common/Exception.h>
#include <Common/ThreadProfileEvents.h>
#include <Common/QueryProfiler.h>
@ -79,12 +77,10 @@ void ThreadStatus::assertState(const std::initializer_list<int> & permitted_stat
return;
}
std::stringstream ss;
ss.exceptions(std::ios::failbit);
ss << "Unexpected thread state " << getCurrentState();
if (description)
ss << ": " << description;
throw Exception(ss.str(), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}: {}", getCurrentState(), description);
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}", getCurrentState());
}
void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue,

View File

@ -1,7 +1,6 @@
#pragma once
#include <tuple>
#include <sstream>
#include <iomanip>
#include <city.h>

View File

@ -1,6 +1,5 @@
#pragma once
#include <sstream>
#include <IO/ReadHelpers.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Interpreters/Context.h>
@ -307,9 +306,6 @@ struct ODBCBridgeMixin
std::vector<std::string> cmd_args;
path.setFileName("clickhouse-odbc-bridge");
std::stringstream command;
command.exceptions(std::ios::failbit);
#if !CLICKHOUSE_SPLIT_BINARY
cmd_args.push_back("odbc-bridge");
#endif

View File

@ -218,7 +218,7 @@ std::pair<ResponsePtr, Undo> TestKeeperCreateRequest::process(TestKeeper::Contai
auto seq_num = it->second.seq_num;
++it->second.seq_num;
std::stringstream seq_num_str;
std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
seq_num_str.exceptions(std::ios::failbit);
seq_num_str << std::setw(10) << std::setfill('0') << seq_num;

View File

@ -3,7 +3,6 @@
#include <ext/scope_guard.h>
#include <pthread.h>
#include <cstdint>
#include <sstream>
#if defined(__FreeBSD__)
# include <pthread_np.h>
@ -80,13 +79,8 @@ __attribute__((__weak__)) void checkStackSize()
/// It's safe to assume that overflow in multiplying by two cannot occur.
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
<< ", stack size: " << stack_size
<< ", maximum stack size: " << max_stack_size;
throw Exception(message.str(), ErrorCodes::TOO_DEEP_RECURSION);
throw Exception(ErrorCodes::TOO_DEEP_RECURSION,
"Stack size too large. Stack address: {}, frame address: {}, stack size: {}, maximum stack size: {}",
stack_address, frame_address, stack_size, max_stack_size);
}
}

View File

@ -1,6 +1,4 @@
#include <cmath>
#include <sstream>
#include <iomanip>
#include <Common/formatReadable.h>
#include <IO/DoubleConverter.h>

View File

@ -12,6 +12,7 @@
#include <IO/BufferWithOwnMemory.h>
#include <Compression/CompressionInfo.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
namespace ProfileEvents
@ -42,7 +43,7 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c
if (expected_checksum == calculated_checksum)
return;
std::stringstream message;
WriteBufferFromOwnString message;
/// TODO mess up of endianness in error message.
message << "Checksum doesn't match: corrupted data."
@ -50,7 +51,16 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c
+ ". Actual: " + getHexUIntLowercase(calculated_checksum.first) + getHexUIntLowercase(calculated_checksum.second)
+ ". Size of compressed block: " + toString(size);
const char * message_hardware_failure = "This is most likely due to hardware failure. If you receive broken data over network and the error does not repeat every time, this can be caused by bad RAM on network interface controller or bad controller itself or bad RAM on network switches or bad CPU on network switches (look at the logs on related network switches; note that TCP checksums don't help) or bad RAM on host (look at dmesg or kern.log for enormous amount of EDAC errors, ECC-related reports, Machine Check Exceptions, mcelog; note that ECC memory can fail if the number of errors is huge) or bad CPU on host. If you read data from disk, this can be caused by disk bit rott. This exception protects ClickHouse from data corruption due to hardware failures.";
const char * message_hardware_failure = "This is most likely due to hardware failure. "
"If you receive broken data over network and the error does not repeat every time, "
"this can be caused by bad RAM on network interface controller or bad controller itself "
"or bad RAM on network switches or bad CPU on network switches "
"(look at the logs on related network switches; note that TCP checksums don't help) "
"or bad RAM on host (look at dmesg or kern.log for enormous amount of EDAC errors, "
"ECC-related reports, Machine Check Exceptions, mcelog; note that ECC memory can fail "
"if the number of errors is huge) or bad CPU on host. If you read data from disk, "
"this can be caused by disk bit rott. This exception protects ClickHouse "
"from data corruption due to hardware failures.";
auto flip_bit = [](char * buf, size_t pos)
{

View File

@ -51,10 +51,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);
throw DB::Exception(0, "Failed!, read: {}, expected: {}", x, i);
}
}
stopwatch.stop();

View File

@ -1,5 +1,4 @@
#include <Core/MySQL/IMySQLReadPacket.h>
#include <sstream>
#include <IO/MySQLPacketPayloadReadBuffer.h>
#include <IO/LimitReadBuffer.h>
@ -21,10 +20,9 @@ void IMySQLReadPacket::readPayload(ReadBuffer & in, uint8_t & sequence_id)
readPayloadImpl(payload);
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);
throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT,
"Packet payload is not fully read. Stopped after {} bytes, while {} bytes are in buffer.",
payload.count(), payload.available());
}
}

View File

@ -1,10 +1,14 @@
#include <Core/MySQL/IMySQLWritePacket.h>
#include <IO/MySQLPacketPayloadWriteBuffer.h>
#include <sstream>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace MySQLProtocol
{
@ -15,10 +19,8 @@ void IMySQLWritePacket::writePayload(WriteBuffer & buffer, uint8_t & sequence_id
buf.next();
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);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incomplete payload. Written {} bytes, expected {} bytes.",
getPayloadSize() - buf.remainingPayloadSize(), getPayloadSize());
}
}

View File

@ -4,6 +4,7 @@
#include <IO/ReadBufferFromString.h>
#include <IO/MySQLBinlogEventReadBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/Operators.h>
#include <common/DateLUT.h>
#include <Common/FieldVisitors.h>
#include <Core/MySQL/PacketsGeneric.h>
@ -35,15 +36,15 @@ namespace MySQLReplication
payload.readStrict(reinterpret_cast<char *>(&flags), 2);
}
void EventHeader::dump(std::ostream & out) const
void EventHeader::dump(WriteBuffer & out) const
{
out << "\n=== " << to_string(this->type) << " ===" << std::endl;
out << "Timestamp: " << this->timestamp << std::endl;
out << "Event Type: " << this->type << std::endl;
out << "Server ID: " << this->server_id << std::endl;
out << "Event Size: " << this->event_size << std::endl;
out << "Log Pos: " << this->log_pos << std::endl;
out << "Flags: " << this->flags << std::endl;
out << "\n=== " << to_string(this->type) << " ===" << '\n';
out << "Timestamp: " << this->timestamp << '\n';
out << "Event Type: " << to_string(this->type) << '\n';
out << "Server ID: " << this->server_id << '\n';
out << "Event Size: " << this->event_size << '\n';
out << "Log Pos: " << this->log_pos << '\n';
out << "Flags: " << this->flags << '\n';
}
/// https://dev.mysql.com/doc/internals/en/format-description-event.html
@ -60,13 +61,13 @@ namespace MySQLReplication
readStringUntilEOF(event_type_header_length, payload);
}
void FormatDescriptionEvent::dump(std::ostream & out) const
void FormatDescriptionEvent::dump(WriteBuffer & out) const
{
header.dump(out);
out << "Binlog Version: " << this->binlog_version << std::endl;
out << "Server Version: " << this->server_version << std::endl;
out << "Create Timestamp: " << this->create_timestamp << std::endl;
out << "Event Header Len: " << std::to_string(this->event_header_length) << std::endl;
out << "Binlog Version: " << this->binlog_version << '\n';
out << "Server Version: " << this->server_version << '\n';
out << "Create Timestamp: " << this->create_timestamp << '\n';
out << "Event Header Len: " << std::to_string(this->event_header_length) << '\n';
}
/// https://dev.mysql.com/doc/internals/en/rotate-event.html
@ -76,11 +77,11 @@ namespace MySQLReplication
readStringUntilEOF(next_binlog, payload);
}
void RotateEvent::dump(std::ostream & out) const
void RotateEvent::dump(WriteBuffer & out) const
{
header.dump(out);
out << "Position: " << this->position << std::endl;
out << "Next Binlog: " << this->next_binlog << std::endl;
out << "Position: " << this->position << '\n';
out << "Next Binlog: " << this->next_binlog << '\n';
}
/// https://dev.mysql.com/doc/internals/en/query-event.html
@ -116,24 +117,24 @@ namespace MySQLReplication
}
}
void QueryEvent::dump(std::ostream & out) const
void QueryEvent::dump(WriteBuffer & out) const
{
header.dump(out);
out << "Thread ID: " << this->thread_id << std::endl;
out << "Execution Time: " << this->exec_time << std::endl;
out << "Schema Len: " << std::to_string(this->schema_len) << std::endl;
out << "Error Code: " << this->error_code << std::endl;
out << "Status Len: " << this->status_len << std::endl;
out << "Schema: " << this->schema << std::endl;
out << "Query: " << this->query << std::endl;
out << "Thread ID: " << this->thread_id << '\n';
out << "Execution Time: " << this->exec_time << '\n';
out << "Schema Len: " << std::to_string(this->schema_len) << '\n';
out << "Error Code: " << this->error_code << '\n';
out << "Status Len: " << this->status_len << '\n';
out << "Schema: " << this->schema << '\n';
out << "Query: " << this->query << '\n';
}
void XIDEvent::parseImpl(ReadBuffer & payload) { payload.readStrict(reinterpret_cast<char *>(&xid), 8); }
void XIDEvent::dump(std::ostream & out) const
void XIDEvent::dump(WriteBuffer & out) const
{
header.dump(out);
out << "XID: " << this->xid << std::endl;
out << "XID: " << this->xid << '\n';
}
void TableMapEvent::parseImpl(ReadBuffer & payload)
@ -238,21 +239,23 @@ namespace MySQLReplication
}
}
void TableMapEvent::dump(std::ostream & out) const
void TableMapEvent::dump(WriteBuffer & out) const
{
header.dump(out);
out << "Table ID: " << this->table_id << std::endl;
out << "Flags: " << this->flags << std::endl;
out << "Schema Len: " << std::to_string(this->schema_len) << std::endl;
out << "Schema: " << this->schema << std::endl;
out << "Table Len: " << std::to_string(this->table_len) << std::endl;
out << "Table: " << this->table << std::endl;
out << "Column Count: " << this->column_count << std::endl;
out << "Table ID: " << this->table_id << '\n';
out << "Flags: " << this->flags << '\n';
out << "Schema Len: " << std::to_string(this->schema_len) << '\n';
out << "Schema: " << this->schema << '\n';
out << "Table Len: " << std::to_string(this->table_len) << '\n';
out << "Table: " << this->table << '\n';
out << "Column Count: " << this->column_count << '\n';
for (auto i = 0U; i < column_count; i++)
{
out << "Column Type [" << i << "]: " << std::to_string(column_type[i]) << ", Meta: " << column_meta[i] << std::endl;
out << "Column Type [" << i << "]: " << std::to_string(column_type[i]) << ", Meta: " << column_meta[i] << '\n';
}
out << "Null Bitmap: " << this->null_bitmap << std::endl;
String bitmap_str;
boost::to_string(this->null_bitmap, bitmap_str);
out << "Null Bitmap: " << bitmap_str << '\n';
}
void RowsEvent::parseImpl(ReadBuffer & payload)
@ -631,16 +634,16 @@ namespace MySQLReplication
rows.push_back(row);
}
void RowsEvent::dump(std::ostream & out) const
void RowsEvent::dump(WriteBuffer & out) const
{
FieldVisitorToString to_string;
header.dump(out);
out << "Schema: " << this->schema << std::endl;
out << "Table: " << this->table << std::endl;
out << "Schema: " << this->schema << '\n';
out << "Table: " << this->table << '\n';
for (auto i = 0U; i < rows.size(); i++)
{
out << "Row[" << i << "]: " << applyVisitor(to_string, rows[i]) << std::endl;
out << "Row[" << i << "]: " << applyVisitor(to_string, rows[i]) << '\n';
}
}
@ -663,22 +666,22 @@ namespace MySQLReplication
payload.ignoreAll();
}
void GTIDEvent::dump(std::ostream & out) const
void GTIDEvent::dump(WriteBuffer & out) const
{
WriteBufferFromOwnString ws;
writeUUIDText(gtid.uuid, ws);
auto gtid_next = ws.str() + ":" + std::to_string(gtid.seq_no);
header.dump(out);
out << "GTID Next: " << gtid_next << std::endl;
out << "GTID Next: " << gtid_next << '\n';
}
void DryRunEvent::parseImpl(ReadBuffer & payload) { payload.ignoreAll(); }
void DryRunEvent::dump(std::ostream & out) const
void DryRunEvent::dump(WriteBuffer & out) const
{
header.dump(out);
out << "[DryRun Event]" << std::endl;
out << "[DryRun Event]" << '\n';
}
/// Update binlog name/position/gtid based on the event type.
@ -716,12 +719,12 @@ namespace MySQLReplication
gtid_sets.parse(gtid_sets_);
}
void Position::dump(std::ostream & out) const
void Position::dump(WriteBuffer & out) const
{
out << "\n=== Binlog Position ===" << std::endl;
out << "Binlog: " << this->binlog_name << std::endl;
out << "Position: " << this->binlog_pos << std::endl;
out << "GTIDSets: " << this->gtid_sets.toString() << std::endl;
out << "\n=== Binlog Position ===" << '\n';
out << "Binlog: " << this->binlog_name << '\n';
out << "Position: " << this->binlog_pos << '\n';
out << "GTIDSets: " << this->gtid_sets.toString() << '\n';
}
void MySQLFlavor::readPayloadImpl(ReadBuffer & payload)

View File

@ -309,7 +309,7 @@ namespace MySQLReplication
UInt16 flags;
EventHeader() : timestamp(0), server_id(0), event_size(0), log_pos(0), flags(0) { }
void dump(std::ostream & out) const;
void dump(WriteBuffer & out) const;
void parse(ReadBuffer & payload);
};
@ -321,7 +321,7 @@ namespace MySQLReplication
EventBase(EventHeader && header_) : header(std::move(header_)) {}
virtual ~EventBase() = default;
virtual void dump(std::ostream & out) const = 0;
virtual void dump(WriteBuffer & out) const = 0;
virtual void parseEvent(ReadBuffer & payload) { parseImpl(payload); }
virtual MySQLEventType type() const { return MYSQL_UNHANDLED_EVENT; }
@ -344,7 +344,7 @@ namespace MySQLReplication
UInt8 event_header_length;
String event_type_header_length;
void dump(std::ostream & out) const override;
void dump(WriteBuffer & out) const override;
void parseImpl(ReadBuffer & payload) override;
private:
@ -358,7 +358,7 @@ namespace MySQLReplication
String next_binlog;
RotateEvent(EventHeader && header_) : EventBase(std::move(header_)), position(0) {}
void dump(std::ostream & out) const override;
void dump(WriteBuffer & out) const override;
protected:
void parseImpl(ReadBuffer & payload) override;
@ -389,7 +389,7 @@ namespace MySQLReplication
{
}
void dump(std::ostream & out) const override;
void dump(WriteBuffer & out) const override;
MySQLEventType type() const override { return MYSQL_QUERY_EVENT; }
protected:
@ -404,7 +404,7 @@ namespace MySQLReplication
protected:
UInt64 xid;
void dump(std::ostream & out) const override;
void dump(WriteBuffer & out) const override;
void parseImpl(ReadBuffer & payload) override;
};
@ -423,7 +423,7 @@ namespace MySQLReplication
Bitmap null_bitmap;
TableMapEvent(EventHeader && header_) : EventBase(std::move(header_)), table_id(0), flags(0), schema_len(0), table_len(0), column_count(0) {}
void dump(std::ostream & out) const override;
void dump(WriteBuffer & out) const override;
protected:
void parseImpl(ReadBuffer & payload) override;
@ -445,7 +445,7 @@ namespace MySQLReplication
table = table_map->table;
}
void dump(std::ostream & out) const override;
void dump(WriteBuffer & out) const override;
protected:
UInt64 table_id;
@ -489,7 +489,7 @@ namespace MySQLReplication
GTID gtid;
GTIDEvent(EventHeader && header_) : EventBase(std::move(header_)), commit_flag(0) {}
void dump(std::ostream & out) const override;
void dump(WriteBuffer & out) const override;
protected:
void parseImpl(ReadBuffer & payload) override;
@ -499,7 +499,7 @@ namespace MySQLReplication
{
public:
DryRunEvent(EventHeader && header_) : EventBase(std::move(header_)) {}
void dump(std::ostream & out) const override;
void dump(WriteBuffer & out) const override;
protected:
void parseImpl(ReadBuffer & payload) override;
@ -515,7 +515,7 @@ namespace MySQLReplication
Position() : binlog_pos(0) { }
void update(BinlogEventPtr event);
void update(UInt64 binlog_pos_, const String & binlog_name_, const String & gtid_sets_);
void dump(std::ostream & out) const;
void dump(WriteBuffer & out) const;
};
class IFlavor : public MySQLProtocol::IMySQLReadPacket

View File

@ -60,10 +60,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();
return fmt::format("{}:{}:dir {}nulls ", column_name, column_number, direction, nulls_direction);
}
};

View File

@ -7,6 +7,7 @@
#include <Core/MySQL/PacketsProtocolText.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteBufferFromOStream.h>
#include <boost/program_options.hpp>
@ -329,6 +330,8 @@ int main(int argc, char ** argv)
slave.connect();
slave.startBinlogDumpGTID(slave_id, replicate_db, gtid_sets);
WriteBufferFromOStream cerr(std::cerr);
/// Read one binlog event on by one.
while (true)
{
@ -337,40 +340,40 @@ int main(int argc, char ** argv)
{
case MYSQL_QUERY_EVENT: {
auto binlog_event = std::static_pointer_cast<QueryEvent>(event);
binlog_event->dump(std::cerr);
binlog_event->dump(cerr);
Position pos = slave.getPosition();
pos.dump(std::cerr);
pos.dump(cerr);
break;
}
case MYSQL_WRITE_ROWS_EVENT: {
auto binlog_event = std::static_pointer_cast<WriteRowsEvent>(event);
binlog_event->dump(std::cerr);
binlog_event->dump(cerr);
Position pos = slave.getPosition();
pos.dump(std::cerr);
pos.dump(cerr);
break;
}
case MYSQL_UPDATE_ROWS_EVENT: {
auto binlog_event = std::static_pointer_cast<UpdateRowsEvent>(event);
binlog_event->dump(std::cerr);
binlog_event->dump(cerr);
Position pos = slave.getPosition();
pos.dump(std::cerr);
pos.dump(cerr);
break;
}
case MYSQL_DELETE_ROWS_EVENT: {
auto binlog_event = std::static_pointer_cast<DeleteRowsEvent>(event);
binlog_event->dump(std::cerr);
binlog_event->dump(cerr);
Position pos = slave.getPosition();
pos.dump(std::cerr);
pos.dump(cerr);
break;
}
default:
if (event->header.type != MySQLReplication::EventType::HEARTBEAT_EVENT)
{
event->dump(std::cerr);
event->dump(cerr);
}
break;
}

View File

@ -59,15 +59,10 @@ void CheckConstraintsBlockOutputStream::write(const Block & block)
/// Is violated.
if (!value)
{
std::stringstream exception_message;
exception_message.exceptions(std::ios::failbit);
exception_message << "Constraint " << backQuote(constraint_ptr->name)
<< " for table " << table_id.getNameForLogs()
<< " is violated, because it is a constant expression returning 0."
<< " It is most likely an error in table definition.";
throw Exception{exception_message.str(), ErrorCodes::VIOLATED_CONSTRAINT};
throw Exception(ErrorCodes::VIOLATED_CONSTRAINT,
"Constraint {} for table {} is violated, because it is a constant expression returning 0. "
"It is most likely an error in table definition.",
backQuote(constraint_ptr->name), table_id.getNameForLogs());
}
}
else
@ -87,28 +82,27 @@ 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()
<< " is violated at row " << (rows_written + row_idx + 1)
<< ". Expression: (" << serializeAST(*(constraint_ptr->expr), true) << ")"
<< ". Column values";
bool first = true;
String column_values_msg;
constexpr size_t approx_bytes_for_col = 32;
column_values_msg.reserve(approx_bytes_for_col * related_columns.size());
for (const auto & name : related_columns)
{
const IColumn & column = *block.getByName(name).column;
assert(row_idx < column.size());
exception_message << (first ? ": " : ", ")
<< backQuoteIfNeed(name) << " = " << applyVisitor(FieldVisitorToString(), column[row_idx]);
if (!first)
column_values_msg.append(", ");
column_values_msg.append(backQuoteIfNeed(name));
column_values_msg.append(" = ");
column_values_msg.append(applyVisitor(FieldVisitorToString(), column[row_idx]));
first = false;
}
throw Exception{exception_message.str(), ErrorCodes::VIOLATED_CONSTRAINT};
throw Exception(ErrorCodes::VIOLATED_CONSTRAINT,
"Constraint {} for table {} is violated at row {}. Expression: ({}). Column values: {}",
backQuote(constraint_ptr->name), table_id.getNameForLogs(), rows_written + row_idx + 1,
serializeAST(*(constraint_ptr->expr), true), column_values_msg);
}
}
}

View File

@ -4,7 +4,8 @@
#include <Interpreters/ProcessList.h>
#include <Access/EnabledQuota.h>
#include <Common/CurrentThread.h>
#include <common/sleep.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
namespace ProfileEvents
{
@ -359,8 +360,7 @@ Block IBlockInputStream::getExtremes()
String IBlockInputStream::getTreeID() const
{
std::stringstream s;
s.exceptions(std::ios::failbit);
WriteBufferFromOwnString s;
s << getName();
if (!children.empty())
@ -399,13 +399,13 @@ size_t IBlockInputStream::checkDepthImpl(size_t max_depth, size_t level) const
}
void IBlockInputStream::dumpTree(std::ostream & ostr, size_t indent, size_t multiplier) const
void IBlockInputStream::dumpTree(WriteBuffer & ostr, size_t indent, size_t multiplier) const
{
ostr << String(indent, ' ') << getName();
if (multiplier > 1)
ostr << " × " << multiplier;
//ostr << ": " << getHeader().dumpStructure();
ostr << std::endl;
ostr << '\n';
++indent;
/// If the subtree is repeated several times, then we output it once with the multiplier.

View File

@ -95,7 +95,7 @@ public:
virtual void readSuffix();
/// Must be called before `read()` and `readPrefix()`.
void dumpTree(std::ostream & ostr, size_t indent = 0, size_t multiplier = 1) const;
void dumpTree(WriteBuffer & ostr, size_t indent = 0, size_t multiplier = 1) const;
/** Check the depth of the pipeline.
* If max_depth is specified and the `depth` is greater - throw an exception.

View File

@ -1,6 +1,4 @@
#include <queue>
#include <iomanip>
#include <sstream>
#include <common/logger_useful.h>

View File

@ -1,4 +1,3 @@
#include <sstream>
#include <string>
#include <vector>

View File

@ -32,8 +32,7 @@ static const std::vector<String> supported_functions{"any", "anyLast", "min",
String DataTypeCustomSimpleAggregateFunction::getName() const
{
std::stringstream stream;
stream.exceptions(std::ios::failbit);
WriteBufferFromOwnString stream;
stream << "SimpleAggregateFunction(" << function->getName();
if (!parameters.empty())

View File

@ -29,10 +29,7 @@ namespace ErrorCodes
template <typename T>
std::string DataTypeDecimal<T>::doGetName() const
{
std::stringstream ss;
ss.exceptions(std::ios::failbit);
ss << "Decimal(" << this->precision << ", " << this->scale << ")";
return ss.str();
return fmt::format("Decimal({}, {})", this->precision, this->scale);
}

View File

@ -127,8 +127,7 @@ static String checkVariableAndGetVersion(const mysqlxx::Pool::Entry & connection
}
bool first = true;
std::stringstream error_message;
error_message.exceptions(std::ios::failbit);
WriteBufferFromOwnString error_message;
error_message << "Illegal MySQL variables, the MaterializeMySQL engine requires ";
for (const auto & [variable_name, variable_error_message] : variables_error_message)
{
@ -239,8 +238,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);
WriteBufferFromOwnString insert_columns_str;
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();
@ -331,10 +329,9 @@ std::optional<MaterializeMetadata> MaterializeMySQLSyncThread::prepareSynchroniz
const auto & position_message = [&]()
{
std::stringstream ss;
ss.exceptions(std::ios::failbit);
position.dump(ss);
return ss.str();
WriteBufferFromOwnString buf;
position.dump(buf);
return buf.str();
};
LOG_INFO(log, "MySQL dump database position: \n {}", position_message());
}
@ -374,10 +371,9 @@ 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();
WriteBufferFromOwnString buf;
client.getPosition().dump(buf);
return buf.str();
};
LOG_INFO(log, "MySQL executed position: \n {}", position_message());
}
@ -632,10 +628,9 @@ 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();
WriteBufferFromOwnString buf;
receive_event->dump(buf);
return buf.str();
};
LOG_DEBUG(log, "Skip MySQL event: \n {}", dump_event_message());

View File

@ -3,7 +3,6 @@
#if !defined(ARCADIA_BUILD) && USE_STATS
#include <math.h>
#include <sstream>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>

View File

@ -239,7 +239,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;
std::stringstream error_message; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
error_message.exceptions(std::ios::failbit);
error_message << "Received error from remote server " << request.getURI() << ". HTTP status code: " << status << " "
<< response.getReason() << ", body: " << istr.rdbuf();

View File

@ -1,5 +1,4 @@
#include <IO/MySQLPacketPayloadReadBuffer.h>
#include <sstream>
namespace DB
{

View File

@ -7,6 +7,8 @@
#include <utility>
#include <IO/HTTPCommon.h>
#include <IO/S3/PocoHTTPResponseStream.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Common/Stopwatch.h>
#include <aws/core/http/HttpRequest.h>
#include <aws/core/http/HttpResponse.h>
@ -247,8 +249,7 @@ void PocoHTTPClient::makeRequestInternal(
response->SetResponseCode(static_cast<Aws::Http::HttpResponseCode>(status_code));
response->SetContentType(poco_response.getContentType());
std::stringstream headers_ss;
headers_ss.exceptions(std::ios::failbit);
WriteBufferFromOwnString headers_ss;
for (const auto & [header_name, header_value] : poco_response)
{
response->AddHeader(header_name, header_value);

View File

@ -76,7 +76,7 @@ std::string dumpContents(const T& container,
const size_t cols_in_row = 8)
{
std::stringstream sstr;
std::stringstream sstr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
sstr.exceptions(std::ios::failbit);
dumpBuffer(std::begin(container), std::end(container), &sstr, col_sep, row_sep, cols_in_row);

View File

@ -22,7 +22,7 @@ static void test(size_t data_size)
for (size_t read_buffer_block_size : block_sizes)
{
std::cout << "block size " << read_buffer_block_size << std::endl;
std::stringstream io;
std::stringstream io; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
io.exceptions(std::ios::failbit);
DB::WriteBufferFromOStream out_impl(io);
DB::HashingWriteBuffer out(out_impl);

View File

@ -20,7 +20,7 @@ try
{
using namespace DB;
std::stringstream s;
std::stringstream s; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
s.exceptions(std::ios::failbit);
{

View File

@ -16,7 +16,7 @@ int main(int, char **)
DB::String c = "вася пе\tтя";
DB::String d = "'xyz\\";
std::stringstream s;
std::stringstream s; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
s.exceptions(std::ios::failbit);
{

View File

@ -1965,21 +1965,19 @@ 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"
<< "1. Size (" << size_str << ") is greater than max_[table/partition]_size_to_drop (" << max_size_to_drop_str << ")\n"
<< "2. File '" << force_file.path() << "' intended to force DROP "
<< (force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist") << "\n";
ostr << "How to fix this:\n"
<< "1. Either increase (or set to zero) max_[table/partition]_size_to_drop in server config\n"
<< "2. Either create forcing file " << force_file.path() << " and make sure that ClickHouse has write permission for it.\n"
<< "Example:\nsudo touch '" << force_file.path() << "' && sudo chmod 666 '" << force_file.path() << "'";
throw Exception(ostr.str(), ErrorCodes::TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT);
throw Exception(ErrorCodes::TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT,
"Table or Partition in {}.{} was not dropped.\nReason:\n"
"1. Size ({}) is greater than max_[table/partition]_size_to_drop ({})\n"
"2. File '{}' intended to force DROP {}\n"
"How to fix this:\n"
"1. Either increase (or set to zero) max_[table/partition]_size_to_drop in server config\n",
"2. Either create forcing file {} and make sure that ClickHouse has write permission for it.\n"
"Example:\nsudo touch '{}' && sudo chmod 666 '{}'",
backQuoteIfNeed(database), backQuoteIfNeed(table),
size_str, max_size_to_drop_str,
force_file.path(), force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist",
force_file.path(),
force_file.path(), force_file.path());
}

View File

@ -13,6 +13,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNullable.h>
#include <Functions/IFunction.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <optional>
#include <Columns/ColumnSet.h>
@ -453,8 +454,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const
std::string ExpressionAction::toString() const
{
std::stringstream ss;
ss.exceptions(std::ios::failbit);
WriteBufferFromOwnString ss;
switch (type)
{
case ADD_COLUMN:
@ -550,8 +550,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);
WriteBufferFromOwnString list_of_non_const_columns;
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;
@ -922,8 +921,7 @@ void ExpressionActions::finalize(const Names & output_columns)
std::string ExpressionActions::dumpActions() const
{
std::stringstream ss;
ss.exceptions(std::ios::failbit);
WriteBufferFromOwnString ss;
ss << "input:\n";
for (const auto & input_column : input_columns)
@ -1344,8 +1342,7 @@ void ExpressionActionsChain::finalize()
std::string ExpressionActionsChain::dumpChain() const
{
std::stringstream ss;
ss.exceptions(std::ios::failbit);
WriteBufferFromOwnString ss;
for (size_t i = 0; i < steps.size(); ++i)
{

View File

@ -11,10 +11,8 @@
#include <Parsers/queryToString.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <IO/WriteBufferFromOStream.h>
#include <Storages/StorageView.h>
#include <sstream>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/printPipeline.h>

View File

@ -26,7 +26,6 @@
#include <Core/Defines.h>
#include <ext/range.h>
#include <boost/range/algorithm/sort.hpp>
#include <sstream>
namespace DB

View File

@ -1,5 +1,3 @@
#include <sstream>
#include <Storages/IStorage.h>
#include <Parsers/TablePropertiesQueriesASTs.h>
#include <Parsers/formatAST.h>

View File

@ -6,8 +6,7 @@
#include <Interpreters/executeQuery.h>
#include <Interpreters/InterpreterShowTablesQuery.h>
#include <Common/typeid_cast.h>
#include <iomanip>
#include <sstream>
#include <IO/Operators.h>
namespace DB
@ -32,8 +31,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
/// SHOW DATABASES
if (query.databases)
{
std::stringstream rewritten_query;
rewritten_query.exceptions(std::ios::failbit);
WriteBufferFromOwnString rewritten_query;
rewritten_query << "SELECT name FROM system.databases";
if (!query.like.empty())
@ -42,7 +40,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
<< " WHERE name "
<< (query.not_like ? "NOT " : "")
<< (query.case_insensitive_like ? "ILIKE " : "LIKE ")
<< std::quoted(query.like, '\'');
<< DB::quote << query.like;
}
if (query.limit_length)
@ -54,8 +52,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
/// SHOW CLUSTER/CLUSTERS
if (query.clusters)
{
std::stringstream rewritten_query;
rewritten_query.exceptions(std::ios::failbit);
WriteBufferFromOwnString rewritten_query;
rewritten_query << "SELECT DISTINCT cluster FROM system.clusters";
if (!query.like.empty())
@ -64,7 +61,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
<< " WHERE cluster "
<< (query.not_like ? "NOT " : "")
<< (query.case_insensitive_like ? "ILIKE " : "LIKE ")
<< std::quoted(query.like, '\'');
<< DB::quote << query.like;
}
if (query.limit_length)
@ -74,11 +71,10 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
}
else if (query.cluster)
{
std::stringstream rewritten_query;
rewritten_query.exceptions(std::ios::failbit);
WriteBufferFromOwnString rewritten_query;
rewritten_query << "SELECT * FROM system.clusters";
rewritten_query << " WHERE cluster = " << std::quoted(query.cluster_str, '\'');
rewritten_query << " WHERE cluster = " << DB::quote << query.cluster_str;
return rewritten_query.str();
}
@ -89,8 +85,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
String database = context.resolveDatabase(query.from);
DatabaseCatalog::instance().assertDatabaseExists(database);
std::stringstream rewritten_query;
rewritten_query.exceptions(std::ios::failbit);
WriteBufferFromOwnString rewritten_query;
rewritten_query << "SELECT name FROM system.";
if (query.dictionaries)
@ -107,14 +102,14 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
rewritten_query << "is_temporary";
}
else
rewritten_query << "database = " << std::quoted(database, '\'');
rewritten_query << "database = " << DB::quote << database;
if (!query.like.empty())
rewritten_query
<< " AND name "
<< (query.not_like ? "NOT " : "")
<< (query.case_insensitive_like ? "ILIKE " : "LIKE ")
<< std::quoted(query.like, '\'');
<< DB::quote << query.like;
else if (query.where_expression)
rewritten_query << " AND (" << query.where_expression << ")";

View File

@ -1,6 +1,3 @@
#include <ostream>
#include <sstream>
#include <Common/typeid_cast.h>
#include <Interpreters/QueryAliasesVisitor.h>
#include <Parsers/ASTTablesInSelectQuery.h>

View File

@ -342,11 +342,9 @@ 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);
throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH,
"Number of columns in section IN doesn't match. {} at left, {} at right.",
num_key_columns, data_types.size());
}
}

View File

@ -552,8 +552,7 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select
if (!unknown_required_source_columns.empty())
{
std::stringstream ss;
ss.exceptions(std::ios::failbit);
WriteBufferFromOwnString ss;
ss << "Missing columns:";
for (const auto & name : unknown_required_source_columns)
ss << " '" << name << "'";

View File

@ -778,11 +778,9 @@ static std::tuple<ASTPtr, BlockIO> 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());
WriteBufferFromOwnString msg_buf;
res.in->dumpTree(msg_buf);
LOG_DEBUG(&Poco::Logger::get("executeQuery"), "Query pipeline:\n{}", msg_buf.str());
}
}
}

View File

@ -5,7 +5,6 @@
#include <Common/quoteString.h>
#include <ext/range.h>
#include <boost/range/algorithm/transform.hpp>
#include <sstream>
#include <IO/Operators.h>

View File

@ -1,6 +1,5 @@
#include <Parsers/ASTIdentifier.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/StorageID.h>

View File

@ -1,5 +1,4 @@
#include <Parsers/ASTWithAlias.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>

View File

@ -1,6 +1,5 @@
#pragma once
#include <sstream>
#include <common/logger_useful.h>
#include <Poco/Util/Application.h>
#include <IO/Operators.h>

View File

@ -1,4 +1,3 @@
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>

View File

@ -8,9 +8,7 @@
#include <IO/WriteBufferFromString.h>
#include <algorithm>
#include <ostream>
#include <set>
#include <sstream>
class SipHash;

View File

@ -1,6 +1,5 @@
#include <Parsers/formatAST.h>
#include <sstream>
namespace DB
{

View File

@ -16,16 +16,16 @@ void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one
String serializeAST(const IAST & ast, bool one_line = true);
//inline std::ostream & operator<<(std::ostream & os, const IAST & ast)
//{
// formatAST(ast, os, false, true);
// return os;
//}
//
//inline std::ostream & operator<<(std::ostream & os, const ASTPtr & ast)
//{
// formatAST(*ast, os, false, true);
// return os;
//}
inline WriteBuffer & operator<<(WriteBuffer & buf, const IAST & ast)
{
formatAST(ast, buf, false, true);
return buf;
}
inline WriteBuffer & operator<<(WriteBuffer & buf, const ASTPtr & ast)
{
formatAST(*ast, buf, false, true);
return buf;
}
}

View File

@ -2,8 +2,6 @@
#include <Interpreters/ProcessList.h>
#include <Formats/FormatFactory.h>
#include <Interpreters/Context.h>
#include <iomanip>
#include <sstream>
namespace DB
{

View File

@ -5,6 +5,7 @@
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Common/PODArray.h>
#include <Common/UTF8Helpers.h>
@ -163,15 +164,10 @@ void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind)
ascii_grid_symbols;
/// Create separators
std::stringstream top_separator;
std::stringstream middle_names_separator;
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);
WriteBufferFromOwnString top_separator;
WriteBufferFromOwnString middle_names_separator;
WriteBufferFromOwnString middle_values_separator;
WriteBufferFromOwnString bottom_separator;
top_separator << grid_symbols.bold_left_top_corner;
middle_names_separator << grid_symbols.bold_left_separator;

View File

@ -1,6 +1,7 @@
#include <Common/PODArray.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <Formats/FormatFactory.h>
#include <Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h>
@ -132,22 +133,17 @@ void PrettyCompactBlockOutputFormat::writeBottom(const Widths & max_widths)
const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ?
utf8_grid_symbols :
ascii_grid_symbols;
/// Create delimiters
std::stringstream bottom_separator;
bottom_separator.exceptions(std::ios::failbit);
bottom_separator << grid_symbols.left_bottom_corner;
/// Write delimiters
out << grid_symbols.left_bottom_corner;
for (size_t i = 0; i < max_widths.size(); ++i)
{
if (i != 0)
bottom_separator << grid_symbols.bottom_separator;
out << grid_symbols.bottom_separator;
for (size_t j = 0; j < max_widths[i] + 2; ++j)
bottom_separator << grid_symbols.dash;
out << grid_symbols.dash;
}
bottom_separator << grid_symbols.right_bottom_corner << "\n";
writeString(bottom_separator.str(), out);
out << grid_symbols.right_bottom_corner << "\n";
}
void PrettyCompactBlockOutputFormat::writeRow(

View File

@ -4,6 +4,7 @@
#include <Common/FieldVisitors.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <common/logger_useful.h>
@ -41,8 +42,7 @@ void CollapsingSortedAlgorithm::reportIncorrectData()
if (!log)
return;
std::stringstream s;
s.exceptions(std::ios::failbit);
WriteBufferFromOwnString s;
auto & sort_columns = *last_row.sort_columns;
for (size_t i = 0, size = sort_columns.size(); i < size; ++i)
{

View File

@ -36,8 +36,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request
const MergeTreeSettings & settings = context.getReplicatedMergeTreeSettings();
bool ok = true;
std::stringstream message;
message.exceptions(std::ios::failbit);
WriteBufferFromOwnString message;
auto databases = DatabaseCatalog::instance().getDatabases();
@ -83,7 +82,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request
}
if (verbose)
response.send() << message.rdbuf();
response.send() << message.str();
else
{
const char * data = "Ok.\n";

View File

@ -436,13 +436,9 @@ bool TCPHandler::readDataNext(const size_t & poll_interval, const int & receive_
double elapsed = watch.elapsedSeconds();
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<size_t>(elapsed) << " seconds,";
ss << " timeout is " << receive_timeout << " seconds.";
throw Exception(ss.str(), ErrorCodes::SOCKET_TIMEOUT);
throw Exception(ErrorCodes::SOCKET_TIMEOUT,
"Timeout exceeded while receiving data from client. Waited for {} seconds, timeout is {} seconds.",
static_cast<size_t>(elapsed), receive_timeout);
}
}

View File

@ -147,8 +147,7 @@ void DistributedBlockOutputStream::writeAsync(const Block & block)
std::string DistributedBlockOutputStream::getCurrentStateDescription()
{
std::stringstream buffer;
buffer.exceptions(std::ios::failbit);
WriteBufferFromOwnString buffer;
const auto & addresses = cluster->getShardsAddresses();
buffer << "Insertion status:\n";

View File

@ -246,10 +246,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();
return fmt::format("{}-{}-{}-{}", VERSION_NAME, getFQDNOrHostName(), table_id_.database_name, table_id_.table_name);
}
@ -400,10 +397,7 @@ ConsumerBufferPtr StorageKafka::createReadBuffer(const size_t consumer_number)
conf.set("group.id", group);
if (num_consumers > 1)
{
std::stringstream ss;
ss.exceptions(std::ios::failbit);
ss << client_id << "-" << consumer_number;
conf.set("client.id", ss.str());
conf.set("client.id", fmt::format("{}-{}", client_id, consumer_number));
}
else
{

View File

@ -1,6 +1,5 @@
#pragma once
#include <sstream>
#include <optional>
#include <Interpreters/Set.h>

View File

@ -442,12 +442,9 @@ 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 << ")";
throw Exception(error_msg.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH);
throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH,
"Different number of files: {} compressed (expected {}) and {} uncompressed ones (expected {})",
rhs.num_compressed_files, num_compressed_files, rhs.num_uncompressed_files, num_uncompressed_files);
}
Strings errors;

View File

@ -228,14 +228,8 @@ 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];
exception_message << ") is not used and setting 'force_primary_key' is set.";
throw Exception(exception_message.str(), ErrorCodes::INDEX_NOT_USED);
throw Exception(ErrorCodes::INDEX_NOT_USED, "Primary key ({}) is not used and setting 'force_primary_key' is set.",
boost::algorithm::join(primary_key_columns, ", "));
}
std::optional<KeyCondition> minmax_idx_condition;

View File

@ -941,8 +941,7 @@ size_t ReplicatedMergeTreeQueue::getConflictsCountForRange(
if (out_description)
{
std::stringstream ss;
ss.exceptions(std::ios::failbit);
WriteBufferFromOwnString ss;
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)

View File

@ -186,15 +186,10 @@ 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;
return fmt::format("{}_{}", table_id.database_name, table_id.table_name);
else
ss << name << "_" << table_id.database_name << "_" << table_id.table_name;
return ss.str();
return fmt::format("{}_{}_{}", name, table_id.database_name, table_id.table_name);
}

View File

@ -9,7 +9,7 @@
#include <Common/quoteString.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include <Processors/Pipe.h>
#include <sstream>
#include <IO/Operators.h>
namespace DB
@ -32,12 +32,9 @@ namespace
{
if (names_and_types_set.find(column) == names_and_types_set.end())
{
std::string message = "Not found column ";
message += column.name + " " + column.type->getName();
message += " in dictionary " + backQuote(dictionary_name) + ". ";
message += "There are only columns ";
message += StorageDictionary::generateNamesAndTypesDescription(dictionary_names_and_types);
throw Exception(message, ErrorCodes::THERE_IS_NO_COLUMN);
throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Not found column {} {} in dictionary {}. There are only columns {}",
column.name, column.type->getName(), backQuote(dictionary_name),
StorageDictionary::generateNamesAndTypesDescription(dictionary_names_and_types));
}
}
}
@ -81,8 +78,7 @@ NamesAndTypesList StorageDictionary::getNamesAndTypes(const DictionaryStructure
String StorageDictionary::generateNamesAndTypesDescription(const NamesAndTypesList & list)
{
std::stringstream ss;
ss.exceptions(std::ios::failbit);
WriteBufferFromOwnString ss;
bool first = true;
for (const auto & name_and_type : list)
{

View File

@ -4,6 +4,7 @@
#include <sparsehash/dense_hash_set>
#include <Common/quoteString.h>
#include <Core/ColumnWithTypeAndName.h>
#include <IO/Operators.h>
namespace DB
@ -427,8 +428,7 @@ namespace
String listOfColumns(const NamesAndTypesList & available_columns)
{
std::stringstream ss;
ss.exceptions(std::ios::failbit);
WriteBufferFromOwnString ss;
for (auto it = available_columns.begin(); it != available_columns.end(); ++it)
{
if (it != available_columns.begin())

View File

@ -1013,17 +1013,13 @@ 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;
else
message << " by some reason.";
LOG_INFO(log, message.str());
constexpr const char * message = "Cannot OPTIMIZE table: {}";
if (disable_reason.empty())
disable_reason = "unknown reason";
LOG_INFO(log, message, disable_reason);
if (context.getSettingsRef().optimize_throw_if_noop)
throw Exception(message.str(), ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, message, disable_reason);
return false;
}
}
@ -1036,17 +1032,13 @@ 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;
else
message << " by some reason.";
LOG_INFO(log, message.str());
constexpr const char * message = "Cannot OPTIMIZE table: {}";
if (disable_reason.empty())
disable_reason = "unknown reason";
LOG_INFO(log, message, disable_reason);
if (context.getSettingsRef().optimize_throw_if_noop)
throw Exception(message.str(), ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, message, disable_reason);
return false;
}
}

View File

@ -460,16 +460,12 @@ 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())
throw Exception(ErrorCodes::UNFINISHED, "Mutation is not finished, it will be done asynchronously");
if (!inactive_replicas.empty())
exception_message << " some replicas are inactive right now: " << boost::algorithm::join(inactive_replicas, ", ");
exception_message << ". Mutation will be done asynchronously";
throw Exception(exception_message.str(), ErrorCodes::UNFINISHED);
throw Exception(ErrorCodes::UNFINISHED,
"Mutation is not finished because some replicas are inactive right now: {}. Mutation will be done asynchronously",
boost::algorithm::join(inactive_replicas, ", "));
}
}
@ -1018,13 +1014,6 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
for (const String & name : parts_to_fetch)
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), "
<< parts_to_fetch.size() << " missing parts (with " << parts_to_fetch_blocks << " blocks).";
/** We can automatically synchronize data,
* if the ratio of the total number of errors to the total number of parts (minimum - on the local filesystem or in ZK)
* is no more than some threshold (for example 50%).
@ -1041,20 +1030,26 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
const auto storage_settings_ptr = getSettings();
bool insane = unexpected_parts_rows > total_rows_on_filesystem * storage_settings_ptr->replicated_max_ratio_of_wrong_parts;
constexpr const char * sanity_report_fmt = "The local set of parts of table {} doesn't look like the set of parts in ZooKeeper: "
"{} rows of {} total rows in filesystem are suspicious. "
"There are {} unexpected parts with {} rows ({} of them is not just-written with {} rows), "
"{} missing parts (with {} blocks).";
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)
<< " total rows in filesystem are suspicious.";
throw Exception(why.str() + " " + sanity_report.str(), ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS);
throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS, sanity_report_fmt, getStorageID().getNameForLogs(),
formatReadableQuantity(unexpected_parts_rows), formatReadableQuantity(total_rows_on_filesystem),
unexpected_parts.size(), unexpected_parts_rows, unexpected_parts_nonnew, unexpected_parts_nonnew_rows,
parts_to_fetch.size(), parts_to_fetch_blocks);
}
if (unexpected_parts_nonnew_rows > 0)
LOG_WARNING(log, sanity_report.str());
{
LOG_WARNING(log, sanity_report_fmt, getStorageID().getNameForLogs(),
formatReadableQuantity(unexpected_parts_rows), formatReadableQuantity(total_rows_on_filesystem),
unexpected_parts.size(), unexpected_parts_rows, unexpected_parts_nonnew, unexpected_parts_nonnew_rows,
parts_to_fetch.size(), parts_to_fetch_blocks);
}
/// Add to the queue jobs to pick up the missing parts from other replicas and remove from ZK the information that we have them.
std::vector<std::future<Coordination::ExistsResponse>> exists_futures;
@ -1343,15 +1338,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry)
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];
LOG_TRACE(log, "Executing log entry to merge parts {} to {}", source_parts_msg.str(), entry.new_part_name);
}
LOG_TRACE(log, "Executing log entry to merge parts {} to {}", boost::algorithm::join(entry.source_parts, ", "), entry.new_part_name);
const auto storage_settings_ptr = getSettings();
@ -3813,13 +3800,11 @@ 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;
LOG_INFO(log, message.str());
return handle_noop(message.str());
constexpr const char * message_fmt = "Cannot select parts for optimization: {}";
if (disable_reason.empty())
disable_reason = "unknown reason";
LOG_INFO(log, message_fmt, disable_reason);
return handle_noop(fmt::format(message_fmt, disable_reason));
}
ReplicatedMergeTreeLogEntryData merge_entry;

View File

@ -35,8 +35,6 @@
#include <Processors/Sources/SourceWithProgress.h>
#include <Processors/Pipe.h>
#include <sstream>
namespace DB
{

View File

@ -1,4 +1,3 @@
#include <sstream>
#include <Common/typeid_cast.h>
#include <Columns/ColumnConst.h>
#include <DataTypes/DataTypesNumber.h>

View File

@ -241,13 +241,9 @@ TableFunctionRemote::TableFunctionRemote(const std::string & name_, bool secure_
: name{name_}, secure{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"
<< ": <addresses pattern or cluster name>, <name of remote database>, <name of remote table>"
<< (is_cluster_function ? "" : ", [username, [password]].");
help_message = ss.str();
help_message = fmt::format("Table function '{}' requires from 2 to {} parameters: "
"<addresses pattern or cluster name>, <name of remote database>, <name of remote table>{}",
name, is_cluster_function ? 3 : 5, is_cluster_function ? "" : ", [username, [password]].");
}

View File

@ -9,6 +9,7 @@
#include <IO/LimitReadBuffer.h>
#include <IO/MySQLBinlogEventReadBuffer.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromOStream.h>
#include <Core/MySQL/MySQLReplication.h>
static DB::MySQLReplication::BinlogEventPtr parseSingleEventBody(
@ -126,18 +127,20 @@ static int checkBinLogFile(const std::string & bin_path, bool exist_checksum)
}
catch (...)
{
std::cerr << "Unable to parse MySQL binlog event. Code: " << DB::getCurrentExceptionCode() << ", Exception message: "
<< DB::getCurrentExceptionMessage(false) << std::endl << ", Previous event: " << std::endl;
last_event->dump(std::cerr);
std::cerr << std::endl << ", Event header: " << std::endl;
last_header->dump(std::cerr);
std::cerr << std::endl;
DB::WriteBufferFromOStream cerr(std::cerr);
cerr << "Unable to parse MySQL binlog event. Code: " << DB::getCurrentExceptionCode() << ", Exception message: "
<< DB::getCurrentExceptionMessage(false) << '\n' << ", Previous event: " << '\n';
last_event->dump(cerr);
cerr << '\n' << ", Event header: " << '\n';
last_header->dump(cerr);
cerr << '\n';
return DB::getCurrentExceptionCode();
}
std::cout << "Check passed. " << std::endl << "No exception was thrown." << std::endl << "The last binlog event: " << std::endl;
last_event->dump(std::cout);
std::cout << std::endl;
DB::WriteBufferFromOStream cout(std::cout);
cout << "Check passed. " << '\n' << "No exception was thrown." << '\n' << "The last binlog event: " << '\n';
last_event->dump(cout);
cout << '\n';
return 0;
}

View File

@ -107,4 +107,4 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P ' $' | grep -P '.' && echo "^ Trailing whitespaces."
# Forbid stringstream because it's easy to use them incorrectly and hard to debug possible issues
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep 'std::ostringstream\|std::istringstream' | grep -v "STYLE_CHECK_ALLOW_STD_STRING_STREAM" && echo "Use WriteBufferFromString or ReadBufferFromString instead of std::ostringstream or std::istringstream"
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep 'std::[io]\?stringstream' | grep -v "STYLE_CHECK_ALLOW_STD_STRING_STREAM" && echo "Use WriteBufferFromOwnString or ReadBufferFromString instead of std::stringstream"

View File

@ -9,7 +9,7 @@
#include "00825_protobuf_format_syntax2.pb.h"
void writeInsertDataQueryForInputTest(std::stringstream & delimited_messages, const std::string & table_name, const std::string & format_schema, std::ostream & out)
void writeInsertDataQueryForInputTest(std::stringstream & delimited_messages, const std::string & table_name, const std::string & format_schema, std::ostream & out) // STYLE_CHECK_ALLOW_STD_STRING_STREAM
{
out << "echo -ne '";
std::string bytes = delimited_messages.str();
@ -27,7 +27,7 @@ void writeInsertDataQueryForInputTest(std::stringstream & delimited_messages, co
void writeInsertDataQueriesForInputTest(std::ostream & out)
{
std::stringstream ss;
std::stringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
{
Person person;
person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49");

View File

@ -80,7 +80,7 @@ int main(int argc, char ** argv)
try
{
std::stringstream ss(line);
std::stringstream ss(line); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
std::string cmd;
ss >> cmd;