dbms: improvement [#CONV-2944].

This commit is contained in:
Alexey Milovidov 2013-06-21 20:34:19 +00:00
parent bd1f289b7b
commit fe0dd1d0b2
60 changed files with 179 additions and 167 deletions

View File

@ -16,7 +16,7 @@ public:
void setArguments(const DataTypes & arguments)
{
if (arguments.size() != 0)
throw Exception("Passed " + Poco::NumberFormatter::format(arguments.size()) + " arguments to nullary aggregate function " + this->getName(),
throw Exception("Passed " + toString(arguments.size()) + " arguments to nullary aggregate function " + this->getName(),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}

View File

@ -15,7 +15,7 @@ public:
void setArguments(const DataTypes & arguments)
{
if (arguments.size() != 1)
throw Exception("Passed " + Poco::NumberFormatter::format(arguments.size()) + " arguments to unary aggregate function " + this->getName(),
throw Exception("Passed " + toString(arguments.size()) + " arguments to unary aggregate function " + this->getName(),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
setArgument(arguments[0]);
}

View File

@ -148,9 +148,9 @@ public:
{
if (start + length > this->data.size())
throw Exception("Parameters start = "
+ Poco::NumberFormatter::format(start) + ", length = "
+ Poco::NumberFormatter::format(length) + " are out of bound in IColumnVector<T>::cut() method"
" (data.size() = " + Poco::NumberFormatter::format(this->data.size()) + ").",
+ toString(start) + ", length = "
+ toString(length) + " are out of bound in IColumnVector<T>::cut() method"
" (data.size() = " + toString(this->data.size()) + ").",
ErrorCodes::PARAMETER_OUT_OF_BOUND);
Self * res = new Self(length);

View File

@ -4,11 +4,11 @@
#include <DB/Core/ErrorCodes.h>
#include <DB/Core/Exception.h>
#include <DB/IO/WriteHelpers.h>
#include <Yandex/Common.h>
#include <Poco/String.h>
#include <Poco/NumberFormatter.h>
#include <boost/noncopyable.hpp>
@ -42,7 +42,7 @@ public:
UCollationResult compare_result = ucol_strcollIter(collator, &iter1, &iter2, &status);
if (status != U_ZERO_ERROR)
throw DB::Exception("ICU collation comparison failed with error code: " + Poco::NumberFormatter::format(status),
throw DB::Exception("ICU collation comparison failed with error code: " + DB::toString(status),
DB::ErrorCodes::COLLATION_COMPARISON_FAILED);
/** Значения enum UCollationResult совпадают с нужными нам:

View File

@ -5,8 +5,6 @@
#include <boost/static_assert.hpp>
#include <Poco/NumberFormatter.h>
#include <mysqlxx/Date.h>
#include <mysqlxx/DateTime.h>
#include <mysqlxx/Manip.h>

View File

@ -104,10 +104,10 @@ protected:
{
if (overflow_mode == Limits::THROW)
throw Exception("DISTINCT-Set size limit exceeded."
" Rows: " + Poco::NumberFormatter::format(set.size()) +
", limit: " + Poco::NumberFormatter::format(max_rows) +
". Bytes: " + Poco::NumberFormatter::format(set.getBufferSizeInBytes()) +
", limit: " + Poco::NumberFormatter::format(max_bytes) + ".",
" Rows: " + toString(set.size()) +
", limit: " + toString(max_rows) +
". Bytes: " + toString(set.getBufferSizeInBytes()) +
", limit: " + toString(max_bytes) + ".",
ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
if (overflow_mode == Limits::BREAK)

View File

@ -3,7 +3,6 @@
#include <ostream>
#include <Poco/SharedPtr.h>
#include <Poco/NumberFormatter.h>
#include <DB/DataTypes/IDataType.h>
@ -24,7 +23,7 @@ public:
std::string getName() const
{
return "FixedString(" + Poco::NumberFormatter::format(n) + ")";
return "FixedString(" + toString(n) + ")";
}
DataTypePtr clone() const

View File

@ -1,7 +1,5 @@
#pragma once
#include <Poco/NumberFormatter.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Functions/IFunction.h>
#include <DB/Functions/NumberTraits.h>
@ -376,7 +374,7 @@ public:
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
DataTypePtr type_res;
@ -477,7 +475,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
DataTypePtr result;

View File

@ -206,7 +206,7 @@ public:
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const DataTypeArray * array_type = dynamic_cast<const DataTypeArray *>(&*arguments[0]);
@ -443,7 +443,7 @@ public:
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const DataTypeArray * array_type = dynamic_cast<const DataTypeArray *>(&*arguments[0]);

View File

@ -48,7 +48,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeUInt32 *>(&*arguments[0]))
@ -150,7 +150,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]))
@ -238,7 +238,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]) &&
@ -433,7 +433,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]))
@ -538,7 +538,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeUInt8 *>(&*arguments[0]) &&

View File

@ -1,7 +1,5 @@
#pragma once
#include <Poco/NumberFormatter.h>
#include <DB/Columns/ColumnsNumber.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Columns/ColumnString.h>
@ -1314,7 +1312,7 @@ public:
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!( (arguments[0]->isNumeric() && arguments[1]->isNumeric())

View File

@ -294,7 +294,7 @@ public:
{
if (arguments.size() != 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 3.",
+ toString(arguments.size()) + ", should be 3.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeUInt8 *>(&*arguments[0]))

View File

@ -276,7 +276,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return new ToDataType;

View File

@ -192,7 +192,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return new ToDataType;
@ -229,7 +229,7 @@ public:
{
if (arguments.size() != 0)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 0.",
+ toString(arguments.size()) + ", should be 0.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return new DataTypeDateTime;
@ -259,7 +259,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeDateTime *>(&*arguments[0]))
@ -391,7 +391,7 @@ public:
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeDateTime *>(&*arguments[0]))

View File

@ -140,7 +140,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (arguments[0]->getName() != TypeName<T>::get())
@ -206,7 +206,7 @@ public:
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (arguments[0]->getName() != TypeName<T>::get())
@ -315,7 +315,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (arguments[0]->getName() != TypeName<T>::get())
@ -440,7 +440,7 @@ public:
{
if (arguments.size() != 1 && arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1 or 2.",
+ toString(arguments.size()) + ", should be 1 or 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (arguments[0]->getName() != TypeName<UInt32>::get())

View File

@ -29,7 +29,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType * type = &*arguments[0];

View File

@ -85,7 +85,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]))
@ -172,7 +172,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!arguments[0]->isNumeric())

View File

@ -301,7 +301,7 @@ public:
{
if (arguments.size() < 1)
throw Exception("Function " + getName() + " needs at least one argument; passed "
+ Poco::NumberFormatter::format(arguments.size()) + ".",
+ toString(arguments.size()) + ".",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (arguments.size() == 1)
@ -313,7 +313,7 @@ public:
{
const DataTypeArray * array_type = dynamic_cast<const DataTypeArray *>(&*arguments[i + 1]);
if (!array_type)
throw Exception("Argument " + Poco::NumberFormatter::format(i + 2) + " of function " + getName() + " must be array. Found "
throw Exception("Argument " + toString(i + 2) + " of function " + getName() + " must be array. Found "
+ arguments[i + 1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
nested_types[i] = array_type->getNestedType();
}
@ -321,7 +321,7 @@ public:
const DataTypeExpression * expression_type = dynamic_cast<const DataTypeExpression *>(&*arguments[0]);
if (!expression_type || expression_type->getArgumentTypes().size() != nested_types.size())
throw Exception("First argument for this overload of " + getName() + " must be an expression with "
+ Poco::NumberFormatter::format(nested_types.size()) + " arguments. Found "
+ toString(nested_types.size()) + " arguments. Found "
+ arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
arguments[0] = new DataTypeExpression(nested_types);
@ -334,8 +334,8 @@ public:
size_t min_args = Impl::needExpression() ? 2 : 1;
if (arguments.size() < min_args)
throw Exception("Function " + getName() + " needs at least "
+ Poco::NumberFormatter::format(min_args) + " argument; passed "
+ Poco::NumberFormatter::format(arguments.size()) + ".",
+ toString(min_args) + " argument; passed "
+ toString(arguments.size()) + ".",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (arguments.size() == 1)

View File

@ -1,7 +1,5 @@
#pragma once
#include <Poco/NumberFormatter.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Functions/IFunction.h>
@ -237,7 +235,7 @@ public:
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!(arguments[0]->isNumeric() && arguments[1]->isNumeric()))
@ -314,7 +312,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!arguments[0]->isNumeric())

View File

@ -186,7 +186,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return new DataTypeUInt64;
@ -332,7 +332,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return new DataTypeString;
@ -360,7 +360,7 @@ public:
{
if (!arguments.empty())
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 0.",
+ toString(arguments.size()) + ", should be 0.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return new DataTypeUInt64;
@ -389,7 +389,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeFloat64 *>(&*arguments[0]) &&
@ -455,7 +455,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return arguments[0];
@ -492,7 +492,7 @@ public:
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function '" + getName() + "' doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return new DataTypeUInt8;
@ -687,7 +687,7 @@ class FunctionReplicate : public IFunction
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const DataTypeArray * array_type = dynamic_cast<const DataTypeArray *>(&*arguments[1]);

View File

@ -102,7 +102,7 @@ public:
{
if (arguments.size() > 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 0 or 1.",
+ toString(arguments.size()) + ", should be 0 or 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return new typename DataTypeFromFieldType<typename Impl::ReturnType>::Type;

View File

@ -35,7 +35,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType * type = &*arguments[0];
@ -130,7 +130,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType * type = &*arguments[0];

View File

@ -1,6 +1,5 @@
#pragma once
#include <Poco/NumberFormatter.h>
#include <Poco/UTF8Encoding.h>
#include <Poco/Unicode.h>
@ -791,7 +790,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]) && !dynamic_cast<const DataTypeFixedString *>(&*arguments[0])
@ -884,7 +883,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]) && !dynamic_cast<const DataTypeFixedString *>(&*arguments[0]))
@ -942,7 +941,7 @@ public:
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]) && !dynamic_cast<const DataTypeFixedString *>(&*arguments[0]))
@ -1074,7 +1073,7 @@ public:
{
if (arguments.size() != 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 3.",
+ toString(arguments.size()) + ", should be 3.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]) && !dynamic_cast<const DataTypeFixedString *>(&*arguments[0]))

View File

@ -48,7 +48,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]))
@ -103,7 +103,7 @@ public:
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]))

View File

@ -372,7 +372,7 @@ public:
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]))
@ -438,7 +438,7 @@ public:
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]))

View File

@ -418,7 +418,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]))
@ -481,7 +481,7 @@ public:
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]))

View File

@ -1,6 +1,5 @@
#pragma once
#include <Poco/NumberFormatter.h>
#include <Poco/UTF8Encoding.h>
#include <Poco/Unicode.h>
#include <Poco/NumberParser.h>

View File

@ -1,9 +1,7 @@
#pragma once
#include <Poco/NumberFormatter.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteHelpers.h>
#include <iostream>
@ -53,8 +51,8 @@ protected:
if (query_id != assert_query_id)
throw Exception("Received data for wrong query id (expected "
+ Poco::NumberFormatter::format(assert_query_id) + ", got "
+ Poco::NumberFormatter::format(query_id) + ")", ErrorCodes::RECEIVED_DATA_FOR_WRONG_QUERY_ID);
+ toString(assert_query_id) + ", got "
+ toString(query_id) + ")", ErrorCodes::RECEIVED_DATA_FOR_WRONG_QUERY_ID);
/// Флаг конца.
readIntBinary(all_read, in);

View File

@ -3,12 +3,11 @@
#include <unistd.h>
#include <errno.h>
#include <Poco/NumberFormatter.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/BufferWithOwnMemory.h>
@ -49,7 +48,7 @@ protected:
/// Имя или описание файла
virtual std::string getFileName()
{
return "(fd = " + Poco::NumberFormatter::format(fd) + ")";
return "(fd = " + toString(fd) + ")";
}
public:

View File

@ -570,4 +570,21 @@ static inline const char * tryReadIntText(T & x, const char * pos, const char *
return pos;
}
/// Простые для использования методы чтения чего-либо из строки в текстовом виде.
template <typename T>
inline T parse(const char * data, size_t size)
{
T res;
ReadBuffer buf(data, size, 0);
readText(res, buf);
return res;
}
template <typename T>
inline T parse(const String & s)
{
return parse<T>(s.data(), s.size());
}
}

View File

@ -3,12 +3,11 @@
#include <unistd.h>
#include <errno.h>
#include <Poco/NumberFormatter.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/BufferWithOwnMemory.h>
@ -43,7 +42,7 @@ protected:
/// Имя или описание файла
virtual std::string getFileName()
{
return "(fd = " + Poco::NumberFormatter::format(fd) + ")";
return "(fd = " + toString(fd) + ")";
}
public:

View File

@ -17,6 +17,7 @@
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/WriteIntText.h>
#include <DB/IO/VarInt.h>
#include <DB/IO/WriteBufferFromString.h>
#define WRITE_HELPERS_DEFAULT_FLOAT_PRECISION 6U
@ -524,4 +525,16 @@ inline void writeDoubleQuoted(const mysqlxx::DateTime & x, WriteBuffer & buf)
void writeException(const Exception & e, WriteBuffer & buf);
/// Простой для использования метод преобразования чего-либо в строку в текстовом виде.
template <typename T>
inline String toString(const T & x)
{
String res;
{
WriteBufferFromString buf(res);
writeText(x, buf);
}
return res;
}
}

View File

@ -4,13 +4,13 @@
#include <sstream>
#include <Poco/SharedPtr.h>
#include <Poco/NumberFormatter.h>
#include <Yandex/Common.h>
#include <DB/Core/Types.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/Parsers/StringRange.h>
#include <iostream>
@ -97,7 +97,7 @@ public:
size_t res = 0;
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
if (max_depth == 0 || (res = (*it)->checkDepth(max_depth - 1)) > max_depth - 1)
throw Exception("AST is too deep. Maximum: " + Poco::NumberFormatter::format(max_depth), ErrorCodes::TOO_DEEP_AST);
throw Exception("AST is too deep. Maximum: " + toString(max_depth), ErrorCodes::TOO_DEEP_AST);
return res + 1;
}
@ -111,7 +111,7 @@ public:
res += (*it)->checkSize(max_size);
if (res > max_size)
throw Exception("AST is too big. Maximum: " + Poco::NumberFormatter::format(max_size), ErrorCodes::TOO_BIG_AST);
throw Exception("AST is too big. Maximum: " + toString(max_size), ErrorCodes::TOO_BIG_AST);
return res;
}

View File

@ -281,8 +281,8 @@ private:
/// Для массивов используются отдельные потоки для размеров.
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
{
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level);
String escaped_size_name = escaped_column_name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level);
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
String escaped_size_name = escaped_column_name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
streams.insert(std::make_pair(size_name, new Stream(
path + escaped_size_name,
@ -303,7 +303,7 @@ private:
{
type_arr->deserializeOffsets(
column,
streams[name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level)]->compressed,
streams[name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level)]->compressed,
max_rows_to_read);
if (column.size())

View File

@ -182,7 +182,7 @@ private:
/// Для массивов требуется сначала сериализовать размеры, а потом значения.
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
{
String size_name = escaped_column_name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level);
String size_name = escaped_column_name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
WriteBufferFromFile plain(path + size_name + ".bin", DBMS_DEFAULT_BUFFER_SIZE, flags);
WriteBufferFromFile marks(path + size_name + ".mrk", 4096, flags);

View File

@ -122,8 +122,8 @@ private:
/// Для массивов используются отдельные потоки для размеров.
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
{
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level);
String escaped_size_name = escaped_column_name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level);
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
String escaped_size_name = escaped_column_name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
column_streams[size_name] = new ColumnStream(
part_tmp_path + escaped_size_name + ".bin",
@ -146,7 +146,7 @@ private:
/// Для массивов требуется сначала сериализовать размеры, а потом значения.
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
{
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level);
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
ColumnStream & stream = *column_streams[size_name];

View File

@ -287,7 +287,7 @@ Connection::Packet Connection::receivePacket()
/// Закроем соединение, чтобы не было рассинхронизации.
disconnect();
throw Exception("Unknown packet "
+ Poco::NumberFormatter::format(res.type)
+ toString(res.type)
+ " from server " + getServerAddress(), ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
}
}

View File

@ -2,11 +2,10 @@
#include <iomanip>
#include <map>
#include <Poco/NumberFormatter.h>
#include <DB/Core/Field.h>
#include <DB/Interpreters/HashMap.h>
#include <DB/Common/AutoArray.h>
#include <DB/IO/WriteHelpers.h>
#include <statdaemons/Stopwatch.h>
@ -19,7 +18,7 @@ int main(int argc, char ** argv)
DB::AutoArray<T> arr(n);
for (size_t i = 0; i < arr.size(); ++i)
arr[i] = "Hello, world! " + Poco::NumberFormatter::format(i);
arr[i] = "Hello, world! " + DB::toString(i);
for (size_t i = 0; i < arr.size(); ++i)
std::cerr << arr[i] << std::endl;
@ -33,7 +32,7 @@ int main(int argc, char ** argv)
DB::AutoArray<T> arr(n, DB::DontInitElemsTag());
for (size_t i = 0; i < arr.size(); ++i)
new (arr.place(i)) std::string("Hello, world! " + Poco::NumberFormatter::format(i));
new (arr.place(i)) std::string("Hello, world! " + DB::toString(i));
for (size_t i = 0; i < arr.size(); ++i)
std::cerr << arr[i] << std::endl;
@ -49,7 +48,7 @@ int main(int argc, char ** argv)
arr.resize(n);
for (size_t i = 0; i < arr.size(); ++i)
arr[i] = "Hello, world! " + Poco::NumberFormatter::format(i);
arr[i] = "Hello, world! " + DB::toString(i);
for (size_t i = 0; i < arr.size(); ++i)
std::cerr << arr[i] << std::endl;
@ -78,9 +77,9 @@ int main(int argc, char ** argv)
{
Arr key(n);
for (size_t j = 0; j < n; ++j)
key[j] = Poco::NumberFormatter::format(rand());
key[j] = DB::toString(rand());
map[key] = "Hello, world! " + Poco::NumberFormatter::format(i);
map[key] = "Hello, world! " + DB::toString(i);
}
for (Map::const_iterator it = map.begin(); it != map.end(); ++it)
@ -122,7 +121,7 @@ int main(int argc, char ** argv)
{
Arr key(n);
for (size_t j = 0; j < n; ++j)
key[j] = Poco::NumberFormatter::format(rand());
key[j] = DB::toString(rand());
vec.push_back(key);
}
@ -221,8 +220,8 @@ int main(int argc, char ** argv)
for (size_t i = 0; i < n; ++i)
{
arr1[i] = "Hello, world! " + Poco::NumberFormatter::format(i);
arr2[i] = "Goodbye, world! " + Poco::NumberFormatter::format(i);
arr1[i] = "Hello, world! " + DB::toString(i);
arr2[i] = "Goodbye, world! " + DB::toString(i);
}
arr2 = arr1;

View File

@ -46,7 +46,7 @@ void Block::insert(size_t position, const ColumnWithNameAndType & elem)
{
if (position > index_by_position.size())
throw Exception("Position out of bound in Block::insert(), max position = "
+ Poco::NumberFormatter::format(index_by_position.size()), ErrorCodes::POSITION_OUT_OF_BOUND);
+ toString(index_by_position.size()), ErrorCodes::POSITION_OUT_OF_BOUND);
if (position == index_by_position.size())
{
@ -79,7 +79,7 @@ void Block::erase(size_t position)
{
if (position >= index_by_position.size())
throw Exception("Position out of bound in Block::erase(), max position = "
+ Poco::NumberFormatter::format(index_by_position.size()), ErrorCodes::POSITION_OUT_OF_BOUND);
+ toString(index_by_position.size()), ErrorCodes::POSITION_OUT_OF_BOUND);
Container_t::iterator it = index_by_position[position];
index_by_name.erase(index_by_name.find(it->name));
@ -105,9 +105,9 @@ void Block::erase(const String & name)
ColumnWithNameAndType & Block::getByPosition(size_t position)
{
if (position >= index_by_position.size())
throw Exception("Position " + Poco::NumberFormatter::format(position)
throw Exception("Position " + toString(position)
+ " is out of bound in Block::getByPosition(), max position = "
+ Poco::NumberFormatter::format(index_by_position.size() - 1)
+ toString(index_by_position.size() - 1)
+ ", there are columns: " + dumpNames(), ErrorCodes::POSITION_OUT_OF_BOUND);
return *index_by_position[position];
@ -117,9 +117,9 @@ ColumnWithNameAndType & Block::getByPosition(size_t position)
const ColumnWithNameAndType & Block::getByPosition(size_t position) const
{
if (position >= index_by_position.size())
throw Exception("Position " + Poco::NumberFormatter::format(position)
throw Exception("Position " + toString(position)
+ " is out of bound in Block::getByPosition(), max position = "
+ Poco::NumberFormatter::format(index_by_position.size() - 1)
+ toString(index_by_position.size() - 1)
+ ", there are columns: " + dumpNames(), ErrorCodes::POSITION_OUT_OF_BOUND);
return *index_by_position[position];
@ -174,8 +174,8 @@ size_t Block::rows() const
if (res != 0 && size != res)
throw Exception("Sizes of columns doesn't match: "
+ data.begin()->name + ": " + Poco::NumberFormatter::format(res)
+ ", " + it->name + ": " + Poco::NumberFormatter::format(size)
+ data.begin()->name + ": " + toString(res)
+ ", " + it->name + ": " + toString(size)
, ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
res = size;

View File

@ -1,7 +1,7 @@
#include <errno.h>
#include <string.h>
#include <Poco/NumberFormatter.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/Core/Exception.h>
@ -48,7 +48,7 @@ void Exception::rethrow() const
void throwFromErrno(const std::string & s, int code, int e)
{
char buf[128];
throw Exception(s + ", errno: " + Poco::NumberFormatter::format(e) + ", strerror: " + std::string(strerror_r(e, buf, sizeof(buf))), code);
throw Exception(s + ", errno: " + toString(e) + ", strerror: " + std::string(strerror_r(e, buf, sizeof(buf))), code);
}
}

View File

@ -3,8 +3,6 @@
#include <sstream>
#include <Poco/NumberParser.h>
#include <Poco/NumberFormatter.h>
#include <DB/Core/Field.h>
#include <statdaemons/Stopwatch.h>

View File

@ -48,7 +48,7 @@ Block BlockInputStreamFromRowInputStream::readImpl()
}
catch (const DB::Exception & e)
{
throw DB::Exception(e.message() + " (at row " + Poco::NumberFormatter::format(total_rows + 1) + ")", e, e.code());
throw DB::Exception(e.message() + " (at row " + toString(total_rows + 1) + ")", e, e.code());
}
return res;

View File

@ -126,7 +126,7 @@ void CollapsingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPt
++count_negative;
}
else
throw Exception("Incorrect data: Sign = " + Poco::NumberFormatter::format(sign) + " (must be 1 or -1).",
throw Exception("Incorrect data: Sign = " + toString(sign) + " (must be 1 or -1).",
ErrorCodes::INCORRECT_DATA);
if (!current->isLast())

View File

@ -40,7 +40,7 @@ size_t IBlockInputStream::checkDepthImpl(size_t max_depth, size_t level) const
return 0;
if (level > max_depth)
throw Exception("Query pipeline is too deep. Maximum: " + Poco::NumberFormatter::format(max_depth), ErrorCodes::TOO_DEEP_PIPELINE);
throw Exception("Query pipeline is too deep. Maximum: " + toString(max_depth), ErrorCodes::TOO_DEEP_PIPELINE);
size_t res = 0;
for (BlockInputStreams::const_iterator it = children.begin(); it != children.end(); ++it)

View File

@ -193,8 +193,8 @@ Block IProfilingBlockInputStream::read()
|| (limits.max_bytes_to_read && info.bytes > limits.max_bytes_to_read))
{
if (limits.read_overflow_mode == Limits::THROW)
throw Exception("Limit for rows to read exceeded: read " + Poco::NumberFormatter::format(info.rows)
+ " rows, maximum: " + Poco::NumberFormatter::format(limits.max_rows_to_read),
throw Exception("Limit for rows to read exceeded: read " + toString(info.rows)
+ " rows, maximum: " + toString(limits.max_rows_to_read),
ErrorCodes::TOO_MUCH_ROWS);
if (limits.read_overflow_mode == Limits::BREAK)
@ -207,8 +207,8 @@ Block IProfilingBlockInputStream::read()
&& info.total_stopwatch.elapsed() > static_cast<UInt64>(limits.max_execution_time.totalMicroseconds()) * 1000)
{
if (limits.timeout_overflow_mode == Limits::THROW)
throw Exception("Timeout exceeded: elapsed " + Poco::NumberFormatter::format(info.total_stopwatch.elapsedSeconds())
+ " seconds, maximum: " + Poco::NumberFormatter::format(limits.max_execution_time.totalMicroseconds() / 1000000.0),
throw Exception("Timeout exceeded: elapsed " + toString(info.total_stopwatch.elapsedSeconds())
+ " seconds, maximum: " + toString(limits.max_execution_time.totalMicroseconds() / 1000000.0),
ErrorCodes::TIMEOUT_EXCEEDED);
if (limits.timeout_overflow_mode == Limits::BREAK)
@ -221,8 +221,8 @@ Block IProfilingBlockInputStream::read()
&& info.total_stopwatch.elapsed() > static_cast<UInt64>(limits.timeout_before_checking_execution_speed.totalMicroseconds()) * 1000
&& info.rows / info.total_stopwatch.elapsedSeconds() < limits.min_execution_speed)
{
throw Exception("Query is executing too slow: " + Poco::NumberFormatter::format(info.rows / info.total_stopwatch.elapsedSeconds())
+ " rows/sec., minimum: " + Poco::NumberFormatter::format(limits.min_execution_speed),
throw Exception("Query is executing too slow: " + toString(info.rows / info.total_stopwatch.elapsedSeconds())
+ " rows/sec., minimum: " + toString(limits.min_execution_speed),
ErrorCodes::TOO_SLOW);
}

View File

@ -9,6 +9,7 @@
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/WriteIntText.h>
#include <DB/IO/WriteBufferFromVector.h>
#include <DB/IO/WriteBufferFromString.h>
#include <DB/IO/CompressedWriteBuffer.h>
@ -1141,12 +1142,12 @@ int main(int argc, char ** argv)
{
/* std::string s(' ', 20);
DB::WriteBufferFromString wb(s);
::writeIntText(Poco::NumberParser::parseUnsigned64(std::string(argv[1])), wb);
DB::Faster::writeIntText(Poco::NumberParser::parseUnsigned64(std::string(argv[1])), wb);
std::cerr << s << std::endl;*/
try
{
typedef Int64 T;
typedef UInt8 T;
size_t n = atoi(argv[1]);
std::vector<T> data(n);
@ -1156,7 +1157,7 @@ int main(int argc, char ** argv)
Stopwatch watch;
for (size_t i = 0; i < n; ++i)
data[i] = lrand48() ^ (lrand48() << 24) ^ (lrand48() << 48);
data[i] = lrand48();// / lrand48();// ^ (lrand48() << 24) ^ (lrand48() << 48);
watch.stop();
std::cerr << std::fixed << std::setprecision(2)
@ -1178,7 +1179,9 @@ int main(int argc, char ** argv)
for (size_t i = 0; i < n; ++i)
{
writeIntTextTable(data[i], wb);
//writeIntTextTable(data[i], wb);
DB::writeIntText(data[i], wb);
//DB::writeIntText(data[i], wb);
DB::writeChar('\t', wb);
}

View File

@ -462,8 +462,8 @@ void Aggregator::execute(BlockInputStreamPtr stream, AggregatedDataVariants & re
if (!no_more_keys && max_rows_to_group_by && result.size() > max_rows_to_group_by)
{
if (group_by_overflow_mode == Limits::THROW)
throw Exception("Limit for rows to GROUP BY exceeded: has " + Poco::NumberFormatter::format(result.size())
+ " rows, maximum: " + Poco::NumberFormatter::format(max_rows_to_group_by),
throw Exception("Limit for rows to GROUP BY exceeded: has " + toString(result.size())
+ " rows, maximum: " + toString(max_rows_to_group_by),
ErrorCodes::TOO_MUCH_ROWS);
else if (group_by_overflow_mode == Limits::BREAK)
break;

View File

@ -222,7 +222,7 @@ ASTPtr Context::getCreateQuery(const String & database_name, const String & tabl
/// Распарсенный запрос должен заканчиваться на конец входных данных или на точку с запятой.
if (!parse_res || (pos != end && *pos != ';'))
throw DB::Exception("Syntax error while parsing query from file " + metadata_path + ": failed at position "
+ Poco::NumberFormatter::format(pos - begin) + ": "
+ toString(pos - begin) + ": "
+ std::string(pos, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, end - pos))
+ ", expected " + (parse_res ? "end of query" : expected) + ".",
DB::ErrorCodes::SYNTAX_ERROR);

View File

@ -320,7 +320,7 @@ void ExpressionActions::checkLimits(Block & block) const
const Limits & limits = settings.limits;
if (limits.max_temporary_columns && block.columns() > limits.max_temporary_columns)
throw Exception("Too many temporary columns: " + block.dumpNames()
+ ". Maximum: " + Poco::NumberFormatter::format(limits.max_temporary_columns),
+ ". Maximum: " + toString(limits.max_temporary_columns),
ErrorCodes::TOO_MUCH_TEMPORARY_COLUMNS);
size_t non_const_columns = 0;
@ -336,7 +336,7 @@ void ExpressionActions::checkLimits(Block & block) const
list_of_non_const_columns << (i == 0 ? "" : ", ") << block.getByPosition(i).name;
throw Exception("Too many temporary non-const columns: " + list_of_non_const_columns.str()
+ ". Maximum: " + Poco::NumberFormatter::format(limits.max_temporary_non_const_columns),
+ ". Maximum: " + toString(limits.max_temporary_non_const_columns),
ErrorCodes::TOO_MUCH_TEMPORARY_NON_CONST_COLUMNS);
}
}

View File

@ -476,9 +476,9 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block)
static std::string getUniqueName(const Block & block, const std::string & prefix)
{
int i = 1;
while (block.has(prefix + Poco::NumberFormatter::format(i)))
while (block.has(prefix + toString(i)))
++i;
return prefix + Poco::NumberFormatter::format(i);
return prefix + toString(i);
}

View File

@ -78,7 +78,7 @@ void InterpreterRenameQuery::execute()
/// Распарсенный запрос должен заканчиваться на конец входных данных или на точку с запятой.
if (!parse_res || (pos != end && *pos != ';'))
throw Exception("Syntax error in file " + from_metadata_path + ": failed at position "
+ Poco::NumberFormatter::format(pos - create_query.data()) + ": "
+ toString(pos - create_query.data()) + ": "
+ std::string(pos, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, end - pos))
+ ", expected " + (parse_res ? "end of query" : expected) + ".",
ErrorCodes::SYNTAX_ERROR);

View File

@ -34,7 +34,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(ASTPtr query_ptr_, const Context
log(&Logger::get("InterpreterSelectQuery"))
{
if (settings.limits.max_subquery_depth && subquery_depth > settings.limits.max_subquery_depth)
throw Exception("Too deep subqueries. Maximum: " + Poco::NumberFormatter::format(settings.limits.max_subquery_depth),
throw Exception("Too deep subqueries. Maximum: " + toString(settings.limits.max_subquery_depth),
ErrorCodes::TOO_DEEP_SUBQUERIES);
context.setColumns(!query.table || !dynamic_cast<ASTSelectQuery *>(&*query.table)
@ -358,8 +358,8 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(BlockInpu
/// Ограничение на количество столбцов для чтения.
if (settings.limits.max_columns_to_read && required_columns.size() > settings.limits.max_columns_to_read)
throw Exception("Limit for number of columns to read exceeded. "
"Requested: " + Poco::NumberFormatter::format(required_columns.size())
+ ", maximum: " + Poco::NumberFormatter::format(settings.limits.max_columns_to_read),
"Requested: " + toString(required_columns.size())
+ ", maximum: " + toString(settings.limits.max_columns_to_read),
ErrorCodes::TOO_MUCH_COLUMNS);
size_t limit_length = 0;

View File

@ -213,10 +213,10 @@ void Set::create(BlockInputStreamPtr stream)
{
if (overflow_mode == Limits::THROW)
throw Exception("IN-Set size exceeded."
" Rows: " + Poco::NumberFormatter::format(getTotalRowCount()) +
", limit: " + Poco::NumberFormatter::format(max_rows) +
". Bytes: " + Poco::NumberFormatter::format(getTotalByteCount()) +
", limit: " + Poco::NumberFormatter::format(max_bytes) + ".",
" Rows: " + toString(getTotalRowCount()) +
", limit: " + toString(max_rows) +
". Bytes: " + toString(getTotalByteCount()) +
", limit: " + toString(max_bytes) + ".",
ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
if (overflow_mode == Limits::BREAK)
@ -267,7 +267,7 @@ void Set::create(DataTypes & types, ASTPtr node)
ColumnWithNameAndType col;
col.type = data_types[i];
col.column = data_types[i]->createColumn();
col.name = "_" + Poco::NumberFormatter::format(i);
col.name = "_" + toString(i);
block.insert(col);
}
@ -353,7 +353,7 @@ void Set::execute(Block & block, const ColumnNumbers & arguments, size_t result,
key_columns[i] = block.getByPosition(arguments[i]).column;
if (data_types[i]->getName() != block.getByPosition(arguments[i]).type->getName())
throw Exception("Types of column " + Poco::NumberFormatter::format(i + 1) + " in section IN don't match: " + data_types[i]->getName() + " on the right, " + block.getByPosition(arguments[i]).type->getName() + " on the left.", ErrorCodes::TYPE_MISMATCH);
throw Exception("Types of column " + toString(i + 1) + " in section IN don't match: " + data_types[i]->getName() + " on the right, " + block.getByPosition(arguments[i]).type->getName() + " on the left.", ErrorCodes::TYPE_MISMATCH);
}
executeOrdinary(key_columns, vec_res, negative);

View File

@ -61,7 +61,7 @@ void executeQuery(
/// Распарсенный запрос должен заканчиваться на конец входных данных или на точку с запятой.
if (!parse_res || (pos != end && *pos != ';'))
throw Exception("Syntax error: failed at position "
+ Poco::NumberFormatter::format(pos - begin) + ": "
+ toString(pos - begin) + ": "
+ std::string(pos, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, end - pos))
+ ", expected " + (parse_res ? "end of query" : expected) + ".",
ErrorCodes::SYNTAX_ERROR);
@ -97,7 +97,7 @@ BlockIO executeQuery(
/// Распарсенный запрос должен заканчиваться на конец входных данных или на точку с запятой.
if (!parse_res || (pos != end && *pos != ';'))
throw Exception("Syntax error: failed at position "
+ Poco::NumberFormatter::format(pos - begin) + ": "
+ toString(pos - begin) + ": "
+ std::string(pos, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, end - pos))
+ ", expected " + (parse_res ? "end of query" : expected) + ".",
ErrorCodes::SYNTAX_ERROR);

View File

@ -34,7 +34,7 @@ static void executeCreateQuery(const String & query, Context & context, const St
/// Распарсенный запрос должен заканчиваться на конец входных данных или на точку с запятой.
if (!parse_res || (pos != end && *pos != ';'))
throw DB::Exception("Syntax error while executing query from file " + file_name + ": failed at position "
+ Poco::NumberFormatter::format(pos - begin) + ": "
+ toString(pos - begin) + ": "
+ std::string(pos, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, end - pos))
+ ", expected " + (parse_res ? "end of query" : expected) + ".",
DB::ErrorCodes::SYNTAX_ERROR);

View File

@ -2,8 +2,6 @@
#include <boost/variant/static_visitor.hpp>
#include <Poco/NumberFormatter.h>
#include <mysqlxx/Manip.h>
#include <DB/IO/WriteBufferFromOStream.h>

View File

@ -1,7 +1,6 @@
#include "OLAPQueryConverter.h"
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/WriteBufferFromString.h>
#include <Poco/NumberFormatter.h>
namespace DB
@ -149,7 +148,7 @@ void QueryConverter::OLAPServerQueryToClickhouse(const QueryParseResult & query,
/// Ограничение на количество выводимых строк.
if (query.limit != 0)
out_query += " LIMIT " + Poco::NumberFormatter::format(query.limit);
out_query += " LIMIT " + toString(query.limit);
}
std::string QueryConverter::convertAttributeFormatted(const std::string & attribute, unsigned parameter)
@ -257,7 +256,7 @@ std::string QueryConverter::convertConstant(const std::string & attribute, const
{
if (!attribute_metadatas.count(attribute))
throw Exception("Unknown attribute " + attribute, ErrorCodes::UNKNOWN_IDENTIFIER);
return Poco::NumberFormatter::format(attribute_metadatas[attribute]->parse(value));
return toString(attribute_metadatas[attribute]->parse(value));
}
std::string QueryConverter::convertCondition(const std::string & attribute, unsigned parameter, const std::string & name, const std::string & rhs)
@ -327,7 +326,7 @@ std::string QueryConverter::convertDateRange(time_t date_first, time_t date_last
std::string QueryConverter::convertCounterID(Yandex::CounterID_t CounterID)
{
return "CounterID == " + Poco::NumberFormatter::format(CounterID);
return "CounterID == " + toString(CounterID);
}
std::string QueryConverter::getTableName(Yandex::CounterID_t CounterID)

View File

@ -356,7 +356,7 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks)
if (!parse_res || pos != end)
throw DB::Exception("Syntax error while parsing create query made by ChunkMerger."
" The query is \"" + create_query + "\"."
+ " Failed at position " + Poco::NumberFormatter::format(pos - begin) + ": "
+ " Failed at position " + toString(pos - begin) + ": "
+ std::string(pos, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, end - pos))
+ ", expected " + (parse_res ? "end of query" : expected) + ".",
DB::ErrorCodes::LOGICAL_ERROR);

View File

@ -85,7 +85,7 @@ void LogBlockInputStream::addStream(const String & name, const IDataType & type,
/// Для массивов используются отдельные потоки для размеров.
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
{
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level);
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
streams.insert(std::make_pair(size_name, new Stream(
storage.files[size_name].data_file.path(),
mark_number
@ -110,7 +110,7 @@ void LogBlockInputStream::readData(const String & name, const IDataType & type,
{
type_arr->deserializeOffsets(
column,
streams[name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level)]->compressed,
streams[name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level)]->compressed,
max_rows_to_read);
if (column.size())
@ -155,7 +155,7 @@ void LogBlockOutputStream::addStream(const String & name, const IDataType & type
/// Для массивов используются отдельные потоки для размеров.
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
{
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level);
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
streams.insert(std::make_pair(size_name, new Stream(
storage.files[size_name].data_file.path())));
@ -172,7 +172,7 @@ void LogBlockOutputStream::writeData(const String & name, const IDataType & type
/// Для массивов требуется сначала сериализовать размеры, а потом значения.
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
{
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level);
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
Mark mark;
mark.rows = (storage.files[size_name].marks.empty() ? 0 : storage.files[size_name].marks.back().rows) + column.size();
@ -254,7 +254,7 @@ void StorageLog::addFile(const String & column_name, const IDataType & type, siz
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
{
String size_column_suffix = ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level);
String size_column_suffix = ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
ColumnData & column_data = files.insert(std::make_pair(column_name + size_column_suffix, ColumnData())).first->second;
column_data.column_index = column_names.size();

View File

@ -78,7 +78,7 @@ void TinyLogBlockInputStream::addStream(const String & name, const IDataType & t
/// Для массивов используются отдельные потоки для размеров.
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
{
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level);
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
streams.insert(std::make_pair(size_name, new Stream(storage.files[size_name].data_file.path())));
addStream(name, *type_arr->getNestedType(), level + 1);
@ -95,7 +95,7 @@ void TinyLogBlockInputStream::readData(const String & name, const IDataType & ty
{
type_arr->deserializeOffsets(
column,
streams[name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level)]->compressed,
streams[name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level)]->compressed,
limit);
if (column.size())
@ -126,7 +126,7 @@ void TinyLogBlockOutputStream::addStream(const String & name, const IDataType &
/// Для массивов используются отдельные потоки для размеров.
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
{
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level);
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
streams.insert(std::make_pair(size_name, new Stream(storage.files[size_name].data_file.path())));
addStream(name, *type_arr->getNestedType(), level + 1);
@ -143,7 +143,7 @@ void TinyLogBlockOutputStream::writeData(const String & name, const IDataType &
{
type_arr->serializeOffsets(
column,
streams[name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level)]->compressed);
streams[name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level)]->compressed);
writeData(name, *type_arr->getNestedType(), dynamic_cast<const ColumnArray &>(column).getData(), level + 1);
}
@ -196,7 +196,7 @@ void StorageTinyLog::addFile(const String & column_name, const IDataType & type,
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
{
String size_column_suffix = ARRAY_SIZES_COLUMN_NAME_SUFFIX + Poco::NumberFormatter::format(level);
String size_column_suffix = ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
ColumnData column_data;
files.insert(std::make_pair(column_name + size_column_suffix, column_data));