mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
remove other stringstreams
This commit is contained in:
parent
62ff00ee8b
commit
5cdfcfb307
@ -77,7 +77,7 @@ public:
|
||||
|
||||
private:
|
||||
Connection * conn;
|
||||
std::ostringstream query_buf;
|
||||
std::ostringstream query_buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
|
||||
void executeImpl();
|
||||
};
|
||||
|
@ -1159,13 +1159,13 @@ private:
|
||||
ASTPtr ast_to_process;
|
||||
try
|
||||
{
|
||||
std::stringstream dump_before_fuzz;
|
||||
WriteBufferFromOwnString dump_before_fuzz;
|
||||
fuzz_base->dumpTree(dump_before_fuzz);
|
||||
auto base_before_fuzz = fuzz_base->formatForErrorMessage();
|
||||
|
||||
ast_to_process = fuzz_base->clone();
|
||||
|
||||
std::stringstream dump_of_cloned_ast;
|
||||
WriteBufferFromOwnString dump_of_cloned_ast;
|
||||
ast_to_process->dumpTree(dump_of_cloned_ast);
|
||||
|
||||
// Run the original query as well.
|
||||
@ -1187,7 +1187,8 @@ private:
|
||||
fprintf(stderr, "dump of cloned ast:\n%s\n",
|
||||
dump_of_cloned_ast.str().c_str());
|
||||
fprintf(stderr, "dump after fuzz:\n");
|
||||
fuzz_base->dumpTree(std::cerr);
|
||||
WriteBufferFromOStream cerr_buf(std::cerr, 4096);
|
||||
fuzz_base->dumpTree(cerr_buf);
|
||||
|
||||
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.");
|
||||
|
||||
|
@ -132,6 +132,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
|
||||
{
|
||||
WriteBufferFromOStream res_buf(std::cout, 4096);
|
||||
formatAST(*res, res_buf, hilite, oneline);
|
||||
res_buf.next();
|
||||
if (multiple)
|
||||
std::cout << "\n;\n";
|
||||
std::cout << std::endl;
|
||||
|
@ -355,7 +355,7 @@ String DiskAccessStorage::getStorageParamsJSON() const
|
||||
json.set("path", directory_path);
|
||||
if (readonly)
|
||||
json.set("readonly", readonly.load());
|
||||
std::ostringstream oss;
|
||||
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
oss.exceptions(std::ios::failbit);
|
||||
Poco::JSON::Stringifier::stringify(json, oss);
|
||||
return oss.str();
|
||||
|
@ -150,7 +150,7 @@ String LDAPAccessStorage::getStorageParamsJSON() const
|
||||
params_json.set("server", ldap_server);
|
||||
params_json.set("roles", default_role_names);
|
||||
|
||||
std::ostringstream oss;
|
||||
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
oss.exceptions(std::ios::failbit);
|
||||
Poco::JSON::Stringifier::stringify(params_json, oss);
|
||||
|
||||
|
@ -460,7 +460,7 @@ String UsersConfigAccessStorage::getStorageParamsJSON() const
|
||||
Poco::JSON::Object json;
|
||||
if (!path.empty())
|
||||
json.set("path", path);
|
||||
std::ostringstream oss;
|
||||
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
oss.exceptions(std::ios::failbit);
|
||||
Poco::JSON::Stringifier::stringify(json, oss);
|
||||
return oss.str();
|
||||
|
@ -16,13 +16,13 @@ struct HTMLForm : public Poco::Net::HTMLForm
|
||||
HTMLForm(const Poco::Net::HTTPRequest & request)
|
||||
{
|
||||
Poco::URI uri(request.getURI());
|
||||
std::istringstream istr(uri.getRawQuery());
|
||||
std::istringstream istr(uri.getRawQuery()); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
readUrl(istr);
|
||||
}
|
||||
|
||||
HTMLForm(const Poco::URI & uri)
|
||||
{
|
||||
std::istringstream istr(uri.getRawQuery());
|
||||
std::istringstream istr(uri.getRawQuery()); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
readUrl(istr);
|
||||
}
|
||||
|
||||
|
@ -413,7 +413,8 @@ std::vector<size_t> PerfEventsCounters::eventIndicesFromString(const std::string
|
||||
return result;
|
||||
}
|
||||
|
||||
std::istringstream iss(events_list);
|
||||
|
||||
std::istringstream iss(events_list); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
std::string event_name;
|
||||
while (std::getline(iss, event_name, ','))
|
||||
{
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <city.h>
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Common/hex.h>
|
||||
|
||||
#ifdef __SSE4_2__
|
||||
#include <nmmintrin.h>
|
||||
@ -48,10 +49,9 @@ struct UInt128
|
||||
|
||||
String toHexString() const
|
||||
{
|
||||
std::ostringstream os;
|
||||
os.exceptions(std::ios::failbit);
|
||||
os << std::setw(16) << std::setfill('0') << std::hex << high << low;
|
||||
return String(os.str());
|
||||
String res(2 * sizeof(UInt128), 0);
|
||||
writeHexUIntLowercase(*this, res.data());
|
||||
return res;
|
||||
}
|
||||
|
||||
bool inline operator== (const UInt128 rhs) const { return tuple() == rhs.tuple(); }
|
||||
|
@ -1,4 +1,7 @@
|
||||
#include <Common/parseGlobs.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <re2/re2.h>
|
||||
#include <re2/stringpiece.h>
|
||||
#include <algorithm>
|
||||
@ -18,21 +21,21 @@ namespace DB
|
||||
*/
|
||||
std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_globs)
|
||||
{
|
||||
std::ostringstream oss_for_escaping;
|
||||
oss_for_escaping.exceptions(std::ios::failbit);
|
||||
/// FIXME make it better
|
||||
WriteBufferFromOwnString buf_for_escaping;
|
||||
/// Escaping only characters that not used in glob syntax
|
||||
for (const auto & letter : initial_str_with_globs)
|
||||
{
|
||||
if ((letter == '[') || (letter == ']') || (letter == '|') || (letter == '+') || (letter == '-') || (letter == '(') || (letter == ')'))
|
||||
oss_for_escaping << '\\';
|
||||
oss_for_escaping << letter;
|
||||
if ((letter == '[') || (letter == ']') || (letter == '|') || (letter == '+') || (letter == '-') || (letter == '(') || (letter == ')') || (letter == '\\'))
|
||||
buf_for_escaping << '\\';
|
||||
buf_for_escaping << letter;
|
||||
}
|
||||
std::string escaped_with_globs = oss_for_escaping.str();
|
||||
std::string escaped_with_globs = buf_for_escaping.str();
|
||||
|
||||
static const re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+,[^{}*]*[^{}*,])})"); /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without {}*,
|
||||
re2::StringPiece input(escaped_with_globs);
|
||||
re2::StringPiece matched;
|
||||
std::ostringstream oss_for_replacing;
|
||||
std::ostringstream oss_for_replacing; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
oss_for_replacing.exceptions(std::ios::failbit);
|
||||
size_t current_index = 0;
|
||||
while (RE2::FindAndConsume(&input, enum_or_range, &matched))
|
||||
@ -45,9 +48,8 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob
|
||||
size_t range_begin = 0;
|
||||
size_t range_end = 0;
|
||||
char point;
|
||||
std::istringstream iss_range(buffer);
|
||||
iss_range.exceptions(std::ios::failbit);
|
||||
iss_range >> range_begin >> point >> point >> range_end;
|
||||
ReadBufferFromString buf_range(buffer);
|
||||
buf_range >> range_begin >> point >> point >> range_end;
|
||||
bool leading_zeros = buffer[0] == '0';
|
||||
size_t num_len = std::to_string(range_end).size();
|
||||
if (leading_zeros)
|
||||
@ -71,20 +73,19 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob
|
||||
}
|
||||
oss_for_replacing << escaped_with_globs.substr(current_index);
|
||||
std::string almost_res = oss_for_replacing.str();
|
||||
std::ostringstream oss_final_processing;
|
||||
oss_final_processing.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString buf_final_processing;
|
||||
for (const auto & letter : almost_res)
|
||||
{
|
||||
if ((letter == '?') || (letter == '*'))
|
||||
{
|
||||
oss_final_processing << "[^/]"; /// '?' is any symbol except '/'
|
||||
buf_final_processing << "[^/]"; /// '?' is any symbol except '/'
|
||||
if (letter == '?')
|
||||
continue;
|
||||
}
|
||||
if ((letter == '.') || (letter == '{') || (letter == '}'))
|
||||
oss_final_processing << '\\';
|
||||
oss_final_processing << letter;
|
||||
buf_final_processing << '\\';
|
||||
buf_final_processing << letter;
|
||||
}
|
||||
return oss_final_processing.str();
|
||||
return buf_final_processing.str();
|
||||
}
|
||||
}
|
||||
|
@ -9,7 +9,8 @@ using namespace DB;
|
||||
|
||||
TEST(Common, getMultipleValuesFromConfig)
|
||||
{
|
||||
std::istringstream xml_isteam(R"END(<?xml version="1.0"?>
|
||||
std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
xml_isteam(R"END(<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<first_level>
|
||||
<second_level>0</second_level>
|
||||
|
@ -102,7 +102,8 @@ TEST(Common, SensitiveDataMasker)
|
||||
EXPECT_EQ(maskerbad.wipeSensitiveData(x), 0);
|
||||
|
||||
{
|
||||
std::istringstream xml_isteam(R"END(<?xml version="1.0"?>
|
||||
std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
xml_isteam(R"END(<?xml version="1.0"?>
|
||||
<clickhouse>
|
||||
<query_masking_rules>
|
||||
<rule>
|
||||
@ -152,7 +153,8 @@ TEST(Common, SensitiveDataMasker)
|
||||
|
||||
try
|
||||
{
|
||||
std::istringstream xml_isteam_bad(R"END(<?xml version="1.0"?>
|
||||
std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
xml_isteam_bad(R"END(<?xml version="1.0"?>
|
||||
<clickhouse>
|
||||
<query_masking_rules>
|
||||
<rule>
|
||||
@ -181,7 +183,8 @@ TEST(Common, SensitiveDataMasker)
|
||||
|
||||
try
|
||||
{
|
||||
std::istringstream xml_isteam_bad(R"END(<?xml version="1.0"?>
|
||||
std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
xml_isteam_bad(R"END(<?xml version="1.0"?>
|
||||
<clickhouse>
|
||||
<query_masking_rules>
|
||||
<rule><name>test</name></rule>
|
||||
@ -203,7 +206,8 @@ TEST(Common, SensitiveDataMasker)
|
||||
|
||||
try
|
||||
{
|
||||
std::istringstream xml_isteam_bad(R"END(<?xml version="1.0"?>
|
||||
std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
xml_isteam_bad(R"END(<?xml version="1.0"?>
|
||||
<clickhouse>
|
||||
<query_masking_rules>
|
||||
<rule><name>test</name><regexp>())(</regexp></rule>
|
||||
|
@ -26,7 +26,7 @@ static auto typeFromString(const std::string & str)
|
||||
|
||||
static auto typesFromString(const std::string & str)
|
||||
{
|
||||
std::istringstream data_types_stream(str);
|
||||
std::istringstream data_types_stream(str); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
DataTypes data_types;
|
||||
std::string data_type;
|
||||
while (data_types_stream >> data_type)
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
|
||||
#include <numeric>
|
||||
@ -230,8 +231,7 @@ std::string DictionaryStructure::getKeyDescription() const
|
||||
if (id)
|
||||
return "UInt64";
|
||||
|
||||
std::ostringstream out;
|
||||
out.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString out;
|
||||
|
||||
out << '(';
|
||||
|
||||
|
@ -18,7 +18,7 @@ static bool registered = false;
|
||||
static std::string configurationToString(const DictionaryConfigurationPtr & config)
|
||||
{
|
||||
const Poco::Util::XMLConfiguration * xml_config = dynamic_cast<const Poco::Util::XMLConfiguration *>(config.get());
|
||||
std::ostringstream oss;
|
||||
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
oss.exceptions(std::ios::failbit);
|
||||
xml_config->save(oss);
|
||||
return oss.str();
|
||||
|
@ -89,4 +89,12 @@ void writeProbablyBackQuotedStringMySQL(const StringRef & s, WriteBuffer & buf)
|
||||
writeProbablyQuotedStringImpl(s, buf, [](const StringRef & s_, WriteBuffer & buf_) { return writeBackQuotedStringMySQL(s_, buf_); });
|
||||
}
|
||||
|
||||
void writePointerHex(const void * ptr, WriteBuffer & buf)
|
||||
{
|
||||
writeString("0x", buf);
|
||||
char hex_str[2 * sizeof(ptr)];
|
||||
writeHexUIntLowercase(reinterpret_cast<uintptr_t>(ptr), hex_str);
|
||||
buf.write(hex_str, 2 * sizeof(ptr));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1110,4 +1110,6 @@ struct PcgSerializer
|
||||
}
|
||||
};
|
||||
|
||||
void writePointerHex(const void * ptr, WriteBuffer & buf);
|
||||
|
||||
}
|
||||
|
@ -148,4 +148,14 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const
|
||||
}
|
||||
}
|
||||
|
||||
void IAST::dumpTree(WriteBuffer & ostr, size_t indent) const
|
||||
{
|
||||
String indent_str(indent, '-');
|
||||
ostr << indent_str << getID() << ", ";
|
||||
writePointerHex(this, ostr);
|
||||
writeChar('\n', ostr);
|
||||
for (const auto & child : children)
|
||||
child->dumpTree(ostr, indent + 1);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -78,13 +78,7 @@ public:
|
||||
void updateTreeHash(SipHash & hash_state) const;
|
||||
virtual void updateTreeHashImpl(SipHash & hash_state) const;
|
||||
|
||||
void dumpTree(std::ostream & ostr, size_t indent = 0) const
|
||||
{
|
||||
String indent_str(indent, '-');
|
||||
ostr << indent_str << getID() << ", " << this << std::endl;
|
||||
for (const auto & child : children)
|
||||
child->dumpTree(ostr, indent + 1);
|
||||
}
|
||||
void dumpTree(WriteBuffer & ostr, size_t indent = 0) const;
|
||||
|
||||
/** Check the depth of the tree.
|
||||
* If max_depth is specified and the depth is greater - throw an exception.
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Parsers/MySQL/ASTDeclareOption.h>
|
||||
#include <Parsers/MySQL/ASTDeclarePartitionOptions.h>
|
||||
#include <Parsers/MySQL/ASTCreateDefines.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
|
||||
using namespace DB;
|
||||
using namespace DB::MySQLParser;
|
||||
@ -37,6 +38,7 @@ TEST(CreateTableParser, SS)
|
||||
ParserCreateQuery p_create_query;
|
||||
String input = "CREATE TABLE `test_table_1` (`a` int DEFAULT NULL, `b` int DEFAULT NULL) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci";
|
||||
ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0);
|
||||
ast->dumpTree(std::cerr);
|
||||
WriteBufferFromOStream buf(std::cerr, 4096);
|
||||
ast->dumpTree(buf);
|
||||
|
||||
}
|
||||
|
@ -3,6 +3,8 @@
|
||||
#include <Parsers/IParser.h>
|
||||
#include <Parsers/Lexer.h>
|
||||
#include <Parsers/TokenIterator.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -23,9 +25,10 @@ std::ostream & operator<<(std::ostream & stream, const Expected & what)
|
||||
|
||||
std::ostream & operator<<(std::ostream & stream, const IAST & what)
|
||||
{
|
||||
stream << "IAST{";
|
||||
what.dumpTree(stream);
|
||||
stream << "}";
|
||||
WriteBufferFromOStream buf(stream, 4096);
|
||||
buf << "IAST{";
|
||||
what.dumpTree(buf);
|
||||
buf << "}";
|
||||
return stream;
|
||||
}
|
||||
|
||||
|
@ -52,7 +52,6 @@ SRCS(
|
||||
ASTShowAccessEntitiesQuery.cpp
|
||||
ASTShowCreateAccessEntityQuery.cpp
|
||||
ASTShowGrantsQuery.cpp
|
||||
ASTShowPrivilegesQuery.cpp
|
||||
ASTShowTablesQuery.cpp
|
||||
ASTSubquery.cpp
|
||||
ASTSystemQuery.cpp
|
||||
|
@ -160,7 +160,7 @@ static void insertNumber(IColumn & column, WhichDataType type, T value)
|
||||
|
||||
static std::string nodeToJson(avro::NodePtr root_node)
|
||||
{
|
||||
std::ostringstream ss;
|
||||
std::ostringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
ss.exceptions(std::ios::failbit);
|
||||
root_node->printJson(ss, 0);
|
||||
return ss.str();
|
||||
|
@ -17,6 +17,8 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <cassert>
|
||||
|
||||
@ -33,8 +35,7 @@ namespace ErrorCodes
|
||||
|
||||
String Range::toString() const
|
||||
{
|
||||
std::stringstream str;
|
||||
str.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString str;
|
||||
|
||||
if (!left_bounded)
|
||||
str << "(-inf, ";
|
||||
@ -1559,19 +1560,18 @@ bool KeyCondition::mayBeTrueAfter(
|
||||
|
||||
String KeyCondition::RPNElement::toString() const
|
||||
{
|
||||
auto print_wrapped_column = [this](std::ostringstream & ss)
|
||||
auto print_wrapped_column = [this](WriteBuffer & buf)
|
||||
{
|
||||
for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it)
|
||||
ss << (*it)->getName() << "(";
|
||||
buf << (*it)->getName() << "(";
|
||||
|
||||
ss << "column " << key_column;
|
||||
buf << "column " << key_column;
|
||||
|
||||
for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it)
|
||||
ss << ")";
|
||||
buf << ")";
|
||||
};
|
||||
|
||||
std::ostringstream ss;
|
||||
ss.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString buf;
|
||||
switch (function)
|
||||
{
|
||||
case FUNCTION_AND:
|
||||
@ -1585,24 +1585,24 @@ String KeyCondition::RPNElement::toString() const
|
||||
case FUNCTION_NOT_IN_SET:
|
||||
case FUNCTION_IN_SET:
|
||||
{
|
||||
ss << "(";
|
||||
print_wrapped_column(ss);
|
||||
ss << (function == FUNCTION_IN_SET ? " in " : " notIn ");
|
||||
buf << "(";
|
||||
print_wrapped_column(buf);
|
||||
buf << (function == FUNCTION_IN_SET ? " in " : " notIn ");
|
||||
if (!set_index)
|
||||
ss << "unknown size set";
|
||||
buf << "unknown size set";
|
||||
else
|
||||
ss << set_index->size() << "-element set";
|
||||
ss << ")";
|
||||
return ss.str();
|
||||
buf << set_index->size() << "-element set";
|
||||
buf << ")";
|
||||
return buf.str();
|
||||
}
|
||||
case FUNCTION_IN_RANGE:
|
||||
case FUNCTION_NOT_IN_RANGE:
|
||||
{
|
||||
ss << "(";
|
||||
print_wrapped_column(ss);
|
||||
ss << (function == FUNCTION_NOT_IN_RANGE ? " not" : "") << " in " << range.toString();
|
||||
ss << ")";
|
||||
return ss.str();
|
||||
buf << "(";
|
||||
print_wrapped_column(buf);
|
||||
buf << (function == FUNCTION_NOT_IN_RANGE ? " not" : "") << " in " << range.toString();
|
||||
buf << ")";
|
||||
return buf.str();
|
||||
}
|
||||
case ALWAYS_FALSE:
|
||||
return "false";
|
||||
|
@ -3205,13 +3205,12 @@ void MergeTreeData::Transaction::rollbackPartsToTemporaryState()
|
||||
{
|
||||
if (!isEmpty())
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss.exceptions(std::ios::failbit);
|
||||
ss << " Rollbacking parts state to temporary and removing from working set:";
|
||||
WriteBufferFromOwnString buf;
|
||||
buf << " Rollbacking parts state to temporary and removing from working set:";
|
||||
for (const auto & part : precommitted_parts)
|
||||
ss << " " << part->relative_path;
|
||||
ss << ".";
|
||||
LOG_DEBUG(data.log, "Undoing transaction.{}", ss.str());
|
||||
buf << " " << part->relative_path;
|
||||
buf << ".";
|
||||
LOG_DEBUG(data.log, "Undoing transaction.{}", buf.str());
|
||||
|
||||
data.removePartsFromWorkingSetImmediatelyAndSetTemporaryState(
|
||||
DataPartsVector(precommitted_parts.begin(), precommitted_parts.end()));
|
||||
@ -3224,13 +3223,12 @@ void MergeTreeData::Transaction::rollback()
|
||||
{
|
||||
if (!isEmpty())
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss.exceptions(std::ios::failbit);
|
||||
ss << " Removing parts:";
|
||||
WriteBufferFromOwnString buf;
|
||||
buf << " Removing parts:";
|
||||
for (const auto & part : precommitted_parts)
|
||||
ss << " " << part->relative_path;
|
||||
ss << ".";
|
||||
LOG_DEBUG(data.log, "Undoing transaction.{}", ss.str());
|
||||
buf << " " << part->relative_path;
|
||||
buf << ".";
|
||||
LOG_DEBUG(data.log, "Undoing transaction.{}", buf.str());
|
||||
|
||||
data.removePartsFromWorkingSet(
|
||||
DataPartsVector(precommitted_parts.begin(), precommitted_parts.end()),
|
||||
@ -3760,15 +3758,15 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S
|
||||
if (out_reason && (settings.min_rows_for_wide_part != 0 || settings.min_bytes_for_wide_part != 0
|
||||
|| settings.min_rows_for_compact_part != 0 || settings.min_bytes_for_compact_part != 0))
|
||||
{
|
||||
std::ostringstream message;
|
||||
message.exceptions(std::ios::failbit);
|
||||
message << "Table can't create parts with adaptive granularity, but settings"
|
||||
<< " min_rows_for_wide_part = " << settings.min_rows_for_wide_part
|
||||
<< ", min_bytes_for_wide_part = " << settings.min_bytes_for_wide_part
|
||||
<< ", min_rows_for_compact_part = " << settings.min_rows_for_compact_part
|
||||
<< ", min_bytes_for_compact_part = " << settings.min_bytes_for_compact_part
|
||||
<< ". Parts with non-adaptive granularity can be stored only in Wide (default) format.";
|
||||
*out_reason = message.str();
|
||||
*out_reason = fmt::format(
|
||||
"Table can't create parts with adaptive granularity, but settings"
|
||||
" min_rows_for_wide_part = {}"
|
||||
", min_bytes_for_wide_part = {}"
|
||||
", min_rows_for_compact_part = {}"
|
||||
", min_bytes_for_compact_part = {}"
|
||||
". Parts with non-adaptive granularity can be stored only in Wide (default) format.",
|
||||
settings.min_rows_for_wide_part, settings.min_bytes_for_wide_part,
|
||||
settings.min_rows_for_compact_part, settings.min_bytes_for_compact_part);
|
||||
}
|
||||
|
||||
return false;
|
||||
|
@ -49,6 +49,8 @@
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
|
||||
@ -175,19 +177,18 @@ UInt64 getMaximumFileNumber(const std::string & dir_path)
|
||||
|
||||
std::string makeFormattedListOfShards(const ClusterPtr & cluster)
|
||||
{
|
||||
std::ostringstream os;
|
||||
os.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString buf;
|
||||
|
||||
bool head = true;
|
||||
os << "[";
|
||||
buf << "[";
|
||||
for (const auto & shard_info : cluster->getShardsInfo())
|
||||
{
|
||||
(head ? os : os << ", ") << shard_info.shard_num;
|
||||
(head ? buf : buf << ", ") << shard_info.shard_num;
|
||||
head = false;
|
||||
}
|
||||
os << "]";
|
||||
buf << "]";
|
||||
|
||||
return os.str();
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
ExpressionActionsPtr buildShardingKeyExpression(const ASTPtr & sharding_key, const Context & context, const NamesAndTypesList & columns, bool project)
|
||||
@ -744,8 +745,7 @@ ClusterPtr StorageDistributed::getOptimizedCluster(const Context & context, cons
|
||||
UInt64 force = settings.force_optimize_skip_unused_shards;
|
||||
if (force)
|
||||
{
|
||||
std::stringstream exception_message;
|
||||
exception_message.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOwnString exception_message;
|
||||
if (!has_sharding_key)
|
||||
exception_message << "No sharding key";
|
||||
else if (!sharding_key_is_usable)
|
||||
|
@ -254,18 +254,14 @@ Strings listFilesWithRegexpMatching(Aws::S3::S3Client & client, const S3::URI &
|
||||
outcome = client.ListObjectsV2(request);
|
||||
if (!outcome.IsSuccess())
|
||||
{
|
||||
std::ostringstream message;
|
||||
message.exceptions(std::ios::failbit);
|
||||
message << "Could not list objects in bucket " << quoteString(request.GetBucket())
|
||||
<< " with prefix " << quoteString(request.GetPrefix());
|
||||
|
||||
if (page > 1)
|
||||
message << ", page " << std::to_string(page);
|
||||
throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, page {}, S3 exception: {}, message: {}",
|
||||
quoteString(request.GetBucket()), quoteString(request.GetPrefix()), page,
|
||||
backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage()));
|
||||
|
||||
message << ", S3 exception: " + backQuote(outcome.GetError().GetExceptionName())
|
||||
<< ", message: " + quoteString(outcome.GetError().GetMessage());
|
||||
|
||||
throw Exception(message.str(), ErrorCodes::S3_ERROR);
|
||||
throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}",
|
||||
quoteString(request.GetBucket()), quoteString(request.GetPrefix()),
|
||||
backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage()));
|
||||
}
|
||||
|
||||
for (const auto & row : outcome.GetResult().GetContents())
|
||||
|
@ -96,7 +96,7 @@ void StorageSystemUsers::fillData(MutableColumns & res_columns, const Context &
|
||||
|
||||
auth_params_json.set("server", authentication.getServerName());
|
||||
|
||||
std::ostringstream oss;
|
||||
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
oss.exceptions(std::ios::failbit);
|
||||
Poco::JSON::Stringifier::stringify(auth_params_json, oss);
|
||||
const auto str = oss.str();
|
||||
|
@ -133,16 +133,14 @@ std::string readData(DB::StoragePtr & table, const DB::Context & context)
|
||||
|
||||
tryRegisterFormats();
|
||||
|
||||
std::ostringstream ss;
|
||||
ss.exceptions(std::ios::failbit);
|
||||
WriteBufferFromOStream out_buf(ss);
|
||||
WriteBufferFromOwnString out_buf;
|
||||
BlockOutputStreamPtr output = FormatFactory::instance().getOutput("Values", out_buf, sample, context);
|
||||
|
||||
copyData(*in, *output);
|
||||
|
||||
output->flush();
|
||||
|
||||
return ss.str();
|
||||
return out_buf.str();
|
||||
}
|
||||
|
||||
TYPED_TEST(StorageLogTest, testReadWrite)
|
||||
|
@ -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' && echo "Use WriteBufferFromString or ReadBufferFromString instead of std::ostringstream or std::istringstream"
|
||||
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"
|
||||
|
Loading…
Reference in New Issue
Block a user