mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
remove more stringstreams
This commit is contained in:
parent
55631e442b
commit
b94cc5c4e5
@ -77,7 +77,7 @@ public:
|
||||
|
||||
private:
|
||||
Connection * conn;
|
||||
std::ostringstream query_buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
std::ostringstream query_buf;
|
||||
|
||||
void executeImpl();
|
||||
};
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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 "
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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}};
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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}};
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <iostream>
|
||||
#include <sstream>
|
||||
#include <unordered_set>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
|
@ -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>
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <iostream>
|
||||
#include <sstream>
|
||||
#include <unordered_set>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <limits>
|
||||
#include <algorithm>
|
||||
#include <climits>
|
||||
#include <sstream>
|
||||
#include <common/types.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
@ -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>
|
||||
|
@ -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)
|
||||
|
@ -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:";
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
|
@ -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();
|
||||
|
@ -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.
|
||||
|
@ -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,
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <tuple>
|
||||
#include <sstream>
|
||||
#include <iomanip>
|
||||
#include <city.h>
|
||||
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -1,6 +1,4 @@
|
||||
#include <cmath>
|
||||
#include <sstream>
|
||||
#include <iomanip>
|
||||
|
||||
#include <Common/formatReadable.h>
|
||||
#include <IO/DoubleConverter.h>
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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();
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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.
|
||||
|
@ -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.
|
||||
|
@ -1,6 +1,4 @@
|
||||
#include <queue>
|
||||
#include <iomanip>
|
||||
#include <sstream>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <sstream>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
|
@ -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())
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
@ -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());
|
||||
|
@ -3,7 +3,6 @@
|
||||
#if !defined(ARCADIA_BUILD) && USE_STATS
|
||||
|
||||
#include <math.h>
|
||||
#include <sstream>
|
||||
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
|
@ -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();
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <IO/MySQLPacketPayloadReadBuffer.h>
|
||||
#include <sstream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
||||
{
|
||||
|
@ -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);
|
||||
|
||||
{
|
||||
|
@ -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());
|
||||
}
|
||||
|
||||
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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>
|
||||
|
||||
|
@ -26,7 +26,6 @@
|
||||
#include <Core/Defines.h>
|
||||
#include <ext/range.h>
|
||||
#include <boost/range/algorithm/sort.hpp>
|
||||
#include <sstream>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,3 @@
|
||||
#include <sstream>
|
||||
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Parsers/TablePropertiesQueriesASTs.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
|
@ -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 << ")";
|
||||
|
||||
|
@ -1,6 +1,3 @@
|
||||
#include <ostream>
|
||||
#include <sstream>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Interpreters/QueryAliasesVisitor.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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 << "'";
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <ext/range.h>
|
||||
#include <boost/range/algorithm/transform.hpp>
|
||||
#include <sstream>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <Parsers/ASTWithAlias.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <sstream>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <IO/Operators.h>
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
@ -8,9 +8,7 @@
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <ostream>
|
||||
#include <set>
|
||||
#include <sstream>
|
||||
|
||||
|
||||
class SipHash;
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <Parsers/formatAST.h>
|
||||
|
||||
#include <sstream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,8 +2,6 @@
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <iomanip>
|
||||
#include <sstream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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(
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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";
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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";
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <sstream>
|
||||
#include <optional>
|
||||
|
||||
#include <Interpreters/Set.h>
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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())
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -35,8 +35,6 @@
|
||||
#include <Processors/Sources/SourceWithProgress.h>
|
||||
#include <Processors/Pipe.h>
|
||||
|
||||
#include <sstream>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <sstream>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
@ -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]].");
|
||||
}
|
||||
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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"
|
||||
|
@ -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");
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user