提交 7b3f533e 编写于 作者: A Alexey Milovidov

Using own Field class instead of boost::variant (experimental) [#CONV-2944].

上级 3e12dc2c
......@@ -45,7 +45,7 @@ public:
void addOne(const Field & value)
{
sum += boost::get<T>(value);
sum += get<T>(value);
++count;
}
......
......@@ -61,12 +61,12 @@ public:
if (params.size() != 1)
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
level = boost::apply_visitor(FieldVisitorConvertToNumber<Float64>(), params[0]);
level = apply_visitor(FieldVisitorConvertToNumber<Float64>(), params[0]);
}
void addOne(const Field & value)
{
sample.insert(boost::get<typename NearestFieldType<ArgumentFieldType>::Type>(value));
sample.insert(get<typename NearestFieldType<ArgumentFieldType>::Type>(value));
}
void merge(const IAggregateFunction & rhs)
......
......@@ -67,7 +67,7 @@ public:
void addOne(const Field & value)
{
sum += boost::get<T>(value);
sum += get<T>(value);
}
void merge(const IAggregateFunction & rhs)
......
......@@ -74,7 +74,7 @@ public:
void addOne(const Field & value)
{
set.insert(AggregateFunctionUniqTraits<T>::hash(boost::get<T>(value)));
set.insert(AggregateFunctionUniqTraits<T>::hash(get<T>(value)));
}
void merge(const IAggregateFunction & rhs)
......
......@@ -49,7 +49,7 @@ public:
void insert(const Field & x)
{
data.push_back(boost::get<const AggregateFunctionPtr &>(x));
data.push_back(get<const AggregateFunctionPtr &>(x));
}
int compareAt(size_t n, size_t m, const IColumn & rhs_) const
......
......@@ -95,7 +95,7 @@ public:
void insert(const Field & x)
{
const Array & array = boost::get<const Array &>(x);
const Array & array = get<const Array &>(x);
size_t size = array.size();
for (size_t i = 0; i < size; ++i)
data->insert(array[i]);
......
......@@ -58,7 +58,7 @@ public:
void insert(const Field & x)
{
const Array & array = boost::get<Array &>(x);
const Array & array = get<const Array &>(x);
if (n != array.size())
throw Exception("Size of array doesn't match size of FixedArray column",
ErrorCodes::SIZE_OF_ARRAY_DOESNT_MATCH_SIZE_OF_FIXEDARRAY_COLUMN);
......
......@@ -45,7 +45,7 @@ public:
void insert(const Field & x)
{
const String & s = boost::get<const String &>(x);
const String & s = get<const String &>(x);
if (s.size() > n)
throw Exception("Too large string '" + s + "' for FixedString column", ErrorCodes::TOO_LARGE_STRING_SIZE);
......
......@@ -47,7 +47,7 @@ public:
void insert(const Field & x)
{
const String & s = boost::get<const String &>(x);
const String & s = get<const String &>(x);
size_t old_size = char_data.size();
size_t size_to_append = s.size() + 1;
......
......@@ -57,7 +57,7 @@ public:
void insert(const Field & x)
{
const Array & arr = boost::get<const Array &>(x);
const Array & arr = get<const Array &>(x);
size_t size = columns.size();
if (arr.size() != size)
......
......@@ -15,7 +15,7 @@ namespace DB
*/
template <>
class FieldVisitorConvertToNumber<SharedPtr<IAggregateFunction> >
: public boost::static_visitor<SharedPtr<IAggregateFunction> >
: public StaticVisitor<SharedPtr<IAggregateFunction> >
{
public:
typedef SharedPtr<IAggregateFunction> T;
......@@ -116,7 +116,7 @@ public:
void insert(const Field & x)
{
data.push_back(boost::apply_visitor(FieldVisitorConvertToNumber<typename NearestFieldType<T>::Type>(), x));
data.push_back(apply_visitor(FieldVisitorConvertToNumber<typename NearestFieldType<T>::Type>(), x));
}
void insertFrom(const IColumn & src, size_t n)
......
......@@ -175,6 +175,8 @@ namespace ErrorCodes
TOO_MUCH_TEMPORARY_NON_CONST_COLUMNS,
TOO_DEEP_AST,
TOO_BIG_AST,
BAD_TYPE_OF_FIELD,
BAD_GET,
POCO_EXCEPTION = 1000,
STD_EXCEPTION,
......
此差异已折叠。
......@@ -20,7 +20,7 @@ public:
void serializeText(const Field & field, WriteBuffer & ostr) const
{
writeDateText(Yandex::DayNum_t(boost::get<UInt64>(field)), ostr);
writeDateText(Yandex::DayNum_t(get<UInt64>(field)), ostr);
}
void deserializeText(Field & field, ReadBuffer & istr) const
......
......@@ -20,7 +20,7 @@ public:
void serializeText(const Field & field, WriteBuffer & ostr) const
{
writeDateTimeText(boost::get<UInt64>(field), ostr);
writeDateTimeText(get<UInt64>(field), ostr);
}
void deserializeText(Field & field, ReadBuffer & istr) const
......
......@@ -10,7 +10,7 @@ namespace DB
{
/// Для заданного значения Field возвращает минимальный тип данных, позволяющий хранить значение этого типа.
class FieldToDataType : public boost::static_visitor<DataTypePtr>
class FieldToDataType : public StaticVisitor<DataTypePtr>
{
public:
DataTypePtr operator() (const Null & x) const
......@@ -51,7 +51,7 @@ public:
DataTypePtr operator() (const Array & x) const
{
return new DataTypeArray(boost::apply_visitor(FieldToDataType(), x.at(0)));
return new DataTypeArray(apply_visitor(FieldToDataType(), x.at(0)));
}
};
......
......@@ -23,7 +23,7 @@ public:
void serializeText(const Field & field, WriteBuffer & ostr) const
{
writeText(boost::get<typename NearestFieldType<FieldType>::Type>(field), ostr);
writeText(get<typename NearestFieldType<FieldType>::Type>(field), ostr);
}
void deserializeText(Field & field, ReadBuffer & istr) const
......
......@@ -28,7 +28,7 @@ public:
void serializeBinary(const Field & field, WriteBuffer & ostr) const
{
/// ColumnType::value_type - более узкий тип. Например, UInt8, когда тип Field - UInt64
typename ColumnType::value_type x = boost::get<typename NearestFieldType<FieldType>::Type>(field);
typename ColumnType::value_type x = get<typename NearestFieldType<FieldType>::Type>(field);
writeBinary(x, ostr);
}
......@@ -66,7 +66,7 @@ public:
ColumnPtr createConstColumn(size_t size, const Field & field) const
{
return new ColumnConst<FieldType>(size, boost::get<typename NearestFieldType<FieldType>::Type>(field));
return new ColumnConst<FieldType>(size, get<typename NearestFieldType<FieldType>::Type>(field));
}
};
......
......@@ -225,7 +225,7 @@ public:
if (!block.getByPosition(arguments[1]).column->isConst())
throw Exception("Second argument for function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN);
UInt64 index = boost::get<UInt64>((*block.getByPosition(arguments[1]).column)[0]);
UInt64 index = safeGet<UInt64>((*block.getByPosition(arguments[1]).column)[0]);
if (index == 0)
throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
......@@ -370,7 +370,7 @@ private:
ArrayIndexNumImpl<T, IndexConv>::vector(
col_nested->getData(),
col_array->getOffsets(),
boost::get<typename NearestFieldType<T>::Type>(value),
safeGet<typename NearestFieldType<T>::Type>(value),
col_res->getData());
return true;
......@@ -395,7 +395,7 @@ private:
dynamic_cast<const ColumnUInt8 &>(col_nested->getData()).getData(),
col_array->getOffsets(),
col_nested->getOffsets(),
boost::get<const String &>(value),
safeGet<const String &>(value),
col_res->getData());
return true;
......
......@@ -1100,8 +1100,8 @@ public:
FieldVisitorConvertToNumber<UInt64> converter;
Field start_field = (*block.getByPosition(arguments[1]).column)[0];
Field length_field = (*block.getByPosition(arguments[2]).column)[0];
UInt64 start = boost::apply_visitor(converter, start_field);
UInt64 length = boost::apply_visitor(converter, length_field);
UInt64 start = apply_visitor(converter, start_field);
UInt64 length = apply_visitor(converter, length_field);
if (start == 0)
throw Exception("Second argument of function substring must be greater than 0.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
......
......@@ -45,7 +45,7 @@ struct StringHash
/** Преобразование значения в 64 бита. Для чисел - однозначное, для строк - некриптографический хэш. */
class FieldVisitorToUInt64 : public boost::static_visitor<UInt64>
class FieldVisitorToUInt64 : public StaticVisitor<UInt64>
{
public:
FieldVisitorToUInt64() {}
......
......@@ -82,39 +82,39 @@ struct Limits
/// Установить настройку по имени.
bool trySet(const String & name, const Field & value)
{
if (name == "max_rows_to_read") max_rows_to_read = boost::get<UInt64>(value);
else if (name == "max_bytes_to_read") max_bytes_to_read = boost::get<UInt64>(value);
else if (name == "read_overflow_mode") read_overflow_mode = getOverflowMode(boost::get<const String &>(value));
if (name == "max_rows_to_read") max_rows_to_read = safeGet<UInt64>(value);
else if (name == "max_bytes_to_read") max_bytes_to_read = safeGet<UInt64>(value);
else if (name == "read_overflow_mode") read_overflow_mode = getOverflowMode(safeGet<const String &>(value));
else if (name == "max_rows_to_group_by") max_rows_to_group_by = boost::get<UInt64>(value);
else if (name == "group_by_overflow_mode") group_by_overflow_mode = getOverflowModeForGroupBy(boost::get<const String &>(value));
else if (name == "max_rows_to_group_by") max_rows_to_group_by = safeGet<UInt64>(value);
else if (name == "group_by_overflow_mode") group_by_overflow_mode = getOverflowModeForGroupBy(safeGet<const String &>(value));
else if (name == "max_rows_to_sort") max_rows_to_sort = boost::get<UInt64>(value);
else if (name == "max_bytes_to_sort") max_bytes_to_sort = boost::get<UInt64>(value);
else if (name == "sort_overflow_mode") sort_overflow_mode = getOverflowMode(boost::get<const String &>(value));
else if (name == "max_rows_to_sort") max_rows_to_sort = safeGet<UInt64>(value);
else if (name == "max_bytes_to_sort") max_bytes_to_sort = safeGet<UInt64>(value);
else if (name == "sort_overflow_mode") sort_overflow_mode = getOverflowMode(safeGet<const String &>(value));
else if (name == "max_result_rows") max_result_rows = boost::get<UInt64>(value);
else if (name == "max_result_bytes") max_result_bytes = boost::get<UInt64>(value);
else if (name == "result_overflow_mode") result_overflow_mode = getOverflowMode(boost::get<const String &>(value));
else if (name == "max_result_rows") max_result_rows = safeGet<UInt64>(value);
else if (name == "max_result_bytes") max_result_bytes = safeGet<UInt64>(value);
else if (name == "result_overflow_mode") result_overflow_mode = getOverflowMode(safeGet<const String &>(value));
else if (name == "max_execution_time") max_execution_time = Poco::Timespan(boost::get<UInt64>(value), 0);
else if (name == "timeout_overflow_mode") timeout_overflow_mode = getOverflowMode(boost::get<const String &>(value));
else if (name == "max_execution_time") max_execution_time = Poco::Timespan(safeGet<UInt64>(value), 0);
else if (name == "timeout_overflow_mode") timeout_overflow_mode = getOverflowMode(safeGet<const String &>(value));
else if (name == "min_execution_speed") min_execution_speed = boost::get<UInt64>(value);
else if (name == "min_execution_speed") min_execution_speed = safeGet<UInt64>(value);
else if (name == "timeout_before_checking_execution_speed")
timeout_before_checking_execution_speed = Poco::Timespan(boost::get<UInt64>(value), 0);
timeout_before_checking_execution_speed = Poco::Timespan(safeGet<UInt64>(value), 0);
else if (name == "max_columns_to_read") max_columns_to_read = boost::get<UInt64>(value);
else if (name == "max_temporary_columns") max_temporary_columns = boost::get<UInt64>(value);
else if (name == "max_columns_to_read") max_columns_to_read = safeGet<UInt64>(value);
else if (name == "max_temporary_columns") max_temporary_columns = safeGet<UInt64>(value);
else if (name == "max_temporary_non_const_columns")
max_temporary_non_const_columns = boost::get<UInt64>(value);
max_temporary_non_const_columns = safeGet<UInt64>(value);
else if (name == "max_subquery_depth") max_subquery_depth = boost::get<UInt64>(value);
else if (name == "max_pipeline_depth") max_pipeline_depth = boost::get<UInt64>(value);
else if (name == "max_ast_depth") max_ast_depth = boost::get<UInt64>(value);
else if (name == "max_ast_elements") max_ast_elements = boost::get<UInt64>(value);
else if (name == "max_subquery_depth") max_subquery_depth = safeGet<UInt64>(value);
else if (name == "max_pipeline_depth") max_pipeline_depth = safeGet<UInt64>(value);
else if (name == "max_ast_depth") max_ast_depth = safeGet<UInt64>(value);
else if (name == "max_ast_elements") max_ast_elements = safeGet<UInt64>(value);
else if (name == "readonly") readonly = boost::get<UInt64>(value);
else if (name == "readonly") readonly = safeGet<UInt64>(value);
else
return false;
......
......@@ -60,20 +60,20 @@ struct Settings
/// Установить настройку по имени.
void set(const String & name, const Field & value)
{
if (name == "max_block_size") max_block_size = boost::get<UInt64>(value);
else if (name == "max_threads") max_threads = boost::get<UInt64>(value);
else if (name == "max_query_size") max_query_size = boost::get<UInt64>(value);
else if (name == "asynchronous") asynchronous = boost::get<UInt64>(value);
else if (name == "interactive_delay") interactive_delay = boost::get<UInt64>(value);
else if (name == "connect_timeout") connect_timeout = Poco::Timespan(boost::get<UInt64>(value), 0);
else if (name == "receive_timeout") receive_timeout = Poco::Timespan(boost::get<UInt64>(value), 0);
else if (name == "send_timeout") send_timeout = Poco::Timespan(boost::get<UInt64>(value), 0);
else if (name == "poll_interval") poll_interval = boost::get<UInt64>(value);
if (name == "max_block_size") max_block_size = safeGet<UInt64>(value);
else if (name == "max_threads") max_threads = safeGet<UInt64>(value);
else if (name == "max_query_size") max_query_size = safeGet<UInt64>(value);
else if (name == "asynchronous") asynchronous = safeGet<UInt64>(value);
else if (name == "interactive_delay") interactive_delay = safeGet<UInt64>(value);
else if (name == "connect_timeout") connect_timeout = Poco::Timespan(safeGet<UInt64>(value), 0);
else if (name == "receive_timeout") receive_timeout = Poco::Timespan(safeGet<UInt64>(value), 0);
else if (name == "send_timeout") send_timeout = Poco::Timespan(safeGet<UInt64>(value), 0);
else if (name == "poll_interval") poll_interval = safeGet<UInt64>(value);
else if (name == "connect_timeout_with_failover_ms")
connect_timeout_with_failover_ms = Poco::Timespan(boost::get<UInt64>(value) * 1000);
else if (name == "max_distributed_connections") max_distributed_connections = boost::get<UInt64>(value);
else if (name == "distributed_connections_pool_size") distributed_connections_pool_size = boost::get<UInt64>(value);
else if (name == "connections_with_failover_max_tries") connections_with_failover_max_tries = boost::get<UInt64>(value);
connect_timeout_with_failover_ms = Poco::Timespan(safeGet<UInt64>(value) * 1000);
else if (name == "max_distributed_connections") max_distributed_connections = safeGet<UInt64>(value);
else if (name == "distributed_connections_pool_size") distributed_connections_pool_size = safeGet<UInt64>(value);
else if (name == "connections_with_failover_max_tries") connections_with_failover_max_tries = safeGet<UInt64>(value);
else if (!limits.trySet(name, value))
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
}
......
......@@ -22,12 +22,12 @@ public:
ASTLiteral() {}
ASTLiteral(StringRange range_, const Field & value_) : IAST(range_), value(value_) {}
String getColumnName() const { return boost::apply_visitor(FieldVisitorToString(), value); }
String getColumnName() const { return apply_visitor(FieldVisitorToString(), value); }
String getAlias() const { return alias.empty() ? getColumnName() : alias; }
/** Получить текст, который идентифицирует этот элемент. */
String getID() const { return "Literal_" + boost::apply_visitor(FieldVisitorDump(), value); }
String getID() const { return "Literal_" + apply_visitor(FieldVisitorDump(), value); }
ASTPtr clone() const { return new ASTLiteral(*this); }
};
......
......@@ -80,7 +80,7 @@ struct Range
bool rightThan(const Field & x)
{
return (left_bounded
? !(boost::apply_visitor(FieldVisitorGreater(), x, left) || (left_included && x == left))
? !((x > left) || (left_included && x == left))
: false);
}
......@@ -88,7 +88,7 @@ struct Range
bool leftThan(const Field & x)
{
return (right_bounded
? !(boost::apply_visitor(FieldVisitorLess(), x, right) || (right_included && x == right))
? !((x < right) || (right_included && x == right))
: false);
}
......@@ -97,16 +97,16 @@ struct Range
/// r левее меня.
if (r.right_bounded &&
left_bounded &&
(boost::apply_visitor(FieldVisitorLess(), r.right, left) ||
((r.right < left) ||
((!left_included || !r.right_included) &&
r.right == left)))
return false;
/// r правее меня.
if (r.left_bounded &&
right_bounded &&
(boost::apply_visitor(FieldVisitorGreater(), r.left, right) ||
((r.left > right) ||
((!right_included || !r.left_included) &&
r.left== right)))
r.left == right)))
return false;
return true;
}
......@@ -116,7 +116,7 @@ struct Range
/// r начинается левее меня.
if (left_bounded &&
(!r.left_bounded ||
boost::apply_visitor(FieldVisitorLess(), r.left, left) ||
(r.left < left) ||
(r.left_included &&
!left_included &&
r.left == left)))
......@@ -124,7 +124,7 @@ struct Range
/// r заканчивается правее меня.
if (right_bounded &&
(!r.right_bounded ||
boost::apply_visitor(FieldVisitorGreater(), r.right, right) ||
(r.right > right) ||
(r.right_included &&
!right_included &&
r.right == right)))
......@@ -139,12 +139,12 @@ struct Range
if (!left_bounded)
str << "(-inf, ";
else
str << (left_included ? '[' : '(') << boost::apply_visitor(FieldVisitorToString(), left) << ", ";
str << (left_included ? '[' : '(') << apply_visitor(FieldVisitorToString(), left) << ", ";
if (!right_bounded)
str << "+inf)";
else
str << boost::apply_visitor(FieldVisitorToString(), right) << (right_included ? ']' : ')');
str << apply_visitor(FieldVisitorToString(), right) << (right_included ? ']' : ')');
return str.str();
}
......
......@@ -34,7 +34,7 @@ template <> ColumnPtr ColumnConst<Array>::convertToFullColumn() const
size_t array_size = data.size();
ColumnPtr nested_column = data.empty()
? new ColumnUInt8 /// Для пустого массива непонятно, какого типа будет материализованный столбец. Пусть будет UInt8.
: boost::apply_visitor(FieldToDataType(), data.at(0))->createColumn();
: apply_visitor(FieldToDataType(), data.at(0))->createColumn();
ColumnArray * res = new ColumnArray(nested_column);
ColumnArray::Offsets_t & offsets = res->getOffsets();
......
#include <iostream>
#include <iomanip>
#include <sstream>
#include <Poco/NumberParser.h>
#include <Poco/NumberFormatter.h>
#include <DB/Core/Field.h>
#include <statdaemons/Stopwatch.h>
int main(int argc, char ** argv)
{
/* DB::FieldVisitorToString to_string;
DB::Field field = DB::UInt64(0);
std::cerr << DB::apply_visitor(to_string, field) << std::endl;
field = std::string("Hello, world!");
std::cerr << DB::apply_visitor(to_string, field) << std::endl;
field = DB::Null();
std::cerr << DB::apply_visitor(to_string, field) << std::endl;
field = DB::Field::AggregateFunctionPtr(NULL);
std::cerr << DB::apply_visitor(to_string, field) << std::endl;
DB::Field field2;
field2 = field;
std::cerr << DB::apply_visitor(to_string, field2) << std::endl;
DB::Field::Array array;
array.push_back(DB::UInt64(123));
array.push_back(DB::Int64(-123));
array.push_back(DB::String("Hello"));
array.push_back(DB::Field::AggregateFunctionPtr(NULL));
field = array;
std::cerr << DB::apply_visitor(to_string, field) << std::endl;
DB::get<DB::Field::Array &>(field).push_back(field);
std::cerr << DB::apply_visitor(to_string, field) << std::endl;
DB::FieldVisitorLess less;
std::cerr << (field < field2) << std::endl;
std::cerr << (field2 < field) << std::endl;
std::cerr << DB::apply_visitor(less, field, field2) << std::endl;
std::cerr << DB::apply_visitor(less, field2, field) << std::endl;*/
try
{
size_t n = argc == 2 ? Poco::NumberParser::parseUnsigned64(argv[1]) : 10000000;
Stopwatch watch;
{
DB::Field::Array array(n);
{
Stopwatch watch;
for (size_t i = 0; i < n; ++i)
array[i] = DB::String(i % 32, '!');
watch.stop();
std::cerr << std::fixed << std::setprecision(2)
<< "Set " << n << " fields (" << n * sizeof(array[0]) / 1000000.0 << " MB) in " << watch.elapsedSeconds() << " sec., "
<< n / watch.elapsedSeconds() << " elem/sec. (" << n * sizeof(array[0]) / watch.elapsedSeconds() / 1000000 << " MB/s.)"
<< std::endl;
}
{
Stopwatch watch;
size_t sum = 0;
for (size_t i = 0; i < n; ++i)
sum += DB::safeGet<const DB::String &>(array[i]).size();
watch.stop();
std::cerr << std::fixed << std::setprecision(2)
<< "Got " << n << " fields (" << n * sizeof(array[0]) / 1000000.0 << " MB) in " << watch.elapsedSeconds() << " sec., "
<< n / watch.elapsedSeconds() << " elem/sec. (" << n * sizeof(array[0]) / watch.elapsedSeconds() / 1000000 << " MB/s.)"
<< std::endl;
std::cerr << sum << std::endl;
}
watch.restart();
}
watch.stop();
std::cerr << std::fixed << std::setprecision(2)
<< "Destroyed " << n << " fields (" << n * sizeof(DB::Field::Array::value_type) / 1000000.0 << " MB) in " << watch.elapsedSeconds() << " sec., "
<< n / watch.elapsedSeconds() << " elem/sec. (" << n * sizeof(DB::Field::Array::value_type) / watch.elapsedSeconds() / 1000000 << " MB/s.)"
<< std::endl;
}
catch (const DB::Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
return 1;
}
return 0;
}
......@@ -16,7 +16,7 @@ void CollapsingSortedBlockInputStream::reportIncorrectData()
{
if (i != 0)
s << ", ";
s << boost::apply_visitor(FieldVisitorToString(), current_key[i]);
s << apply_visitor(FieldVisitorToString(), current_key[i]);
}
s << ").";
......@@ -85,7 +85,7 @@ Block CollapsingSortedBlockInputStream::readImpl()
SortCursor current = queue.top();
queue.pop();
Int8 sign = boost::get<Int64>((*current->all_columns[sign_column_number])[current->pos]);
Int8 sign = get<Int64>((*current->all_columns[sign_column_number])[current->pos]);
setPrimaryKey(next_key, current);
if (next_key != current_key)
......
......@@ -180,7 +180,7 @@ void PrettyBlockOutputStream::write(const Block & block_)
if (col.type->isNumeric())
{
size_t width = boost::get<UInt64>((*block.getByPosition(columns + j).column)[i]);
size_t width = get<UInt64>((*block.getByPosition(columns + j).column)[i]);
for (size_t k = 0; k < max_widths[j] - width; ++k)
writeChar(' ', ostr);
......@@ -190,7 +190,7 @@ void PrettyBlockOutputStream::write(const Block & block_)
{
col.type->serializeTextEscaped((*col.column)[i], ostr);
size_t width = boost::get<UInt64>((*block.getByPosition(columns + j).column)[i]);
size_t width = get<UInt64>((*block.getByPosition(columns + j).column)[i]);
for (size_t k = 0; k < max_widths[j] - width; ++k)
writeChar(' ', ostr);
}
......
......@@ -91,7 +91,7 @@ void PrettyCompactBlockOutputStream::write(const Block & block_)
if (col.type->isNumeric())
{
size_t width = boost::get<UInt64>((*block.getByPosition(columns + j).column)[i]);
size_t width = get<UInt64>((*block.getByPosition(columns + j).column)[i]);
for (size_t k = 0; k < max_widths[j] - width; ++k)
writeChar(' ', ostr);
......@@ -101,7 +101,7 @@ void PrettyCompactBlockOutputStream::write(const Block & block_)
{
col.type->serializeTextEscaped((*col.column)[i], ostr);
size_t width = boost::get<UInt64>((*block.getByPosition(columns + j).column)[i]);
size_t width = get<UInt64>((*block.getByPosition(columns + j).column)[i]);
for (size_t k = 0; k < max_widths[j] - width; ++k)
writeChar(' ', ostr);
}
......
......@@ -78,7 +78,7 @@ void PrettySpaceBlockOutputStream::write(const Block & block_)
if (col.type->isNumeric())
{
size_t width = boost::get<UInt64>((*block.getByPosition(columns + j).column)[i]);
size_t width = get<UInt64>((*block.getByPosition(columns + j).column)[i]);
for (ssize_t k = 0; k < std::max(0L, static_cast<ssize_t>(max_widths[j] - width)); ++k)
writeChar(' ', ostr);
......@@ -88,7 +88,7 @@ void PrettySpaceBlockOutputStream::write(const Block & block_)
{
col.type->serializeTextEscaped((*col.column)[i], ostr);
size_t width = boost::get<UInt64>((*block.getByPosition(columns + j).column)[i]);
size_t width = get<UInt64>((*block.getByPosition(columns + j).column)[i]);
for (ssize_t k = 0; k < std::max(0L, static_cast<ssize_t>(max_widths[j] - width)); ++k)
writeChar(' ', ostr);
}
......
......@@ -15,7 +15,7 @@ using Poco::SharedPtr;
void DataTypeAggregateFunction::serializeBinary(const Field & field, WriteBuffer & ostr) const
{
const AggregateFunctionPtr & value = boost::get<const AggregateFunctionPtr &>(field);
const AggregateFunctionPtr & value = get<const AggregateFunctionPtr &>(field);
value->serialize(ostr);
}
......@@ -95,7 +95,7 @@ ColumnPtr DataTypeAggregateFunction::createColumn() const
ColumnPtr DataTypeAggregateFunction::createConstColumn(size_t size, const Field & field) const
{
return new ColumnConst<AggregateFunctionPtr>(size, boost::get<AggregateFunctionPtr>(field));
return new ColumnConst<AggregateFunctionPtr>(size, get<AggregateFunctionPtr>(field));
}
......
......@@ -107,7 +107,7 @@ void DataTypeArray::deserializeOffsets(IColumn & column, ReadBuffer & istr, size
void DataTypeArray::serializeText(const Field & field, WriteBuffer & ostr) const
{
const Array & arr = boost::get<const Array &>(field);
const Array & arr = get<const Array &>(field);
writeChar('[', ostr);
for (size_t i = 0, size = arr.size(); i < size; ++i)
......@@ -179,7 +179,7 @@ ColumnPtr DataTypeArray::createColumn() const
ColumnPtr DataTypeArray::createConstColumn(size_t size, const Field & field) const
{
return new ColumnConst<Array>(size, boost::get<Array>(field));
return new ColumnConst<Array>(size, get<const Array &>(field));
}
}
......@@ -20,7 +20,7 @@ using Poco::SharedPtr;
void DataTypeFixedString::serializeBinary(const Field & field, WriteBuffer & ostr) const
{
const String & s = boost::get<String>(field);
const String & s = get<const String &>(field);
ostr.write(s.data(), std::min(s.size(), n));
if (s.size() < n)
for (size_t i = s.size(); i < n; ++i)
......@@ -31,7 +31,7 @@ void DataTypeFixedString::serializeBinary(const Field & field, WriteBuffer & ost
void DataTypeFixedString::deserializeBinary(Field & field, ReadBuffer & istr) const
{
field = String();
String & s = boost::get<String>(field);
String & s = get<String &>(field);
s.resize(n);
/// непереносимо, но (действительно) быстрее
istr.readStrict(const_cast<char*>(s.data()), n);
......@@ -71,7 +71,7 @@ void DataTypeFixedString::deserializeBinary(IColumn & column, ReadBuffer & istr,
void DataTypeFixedString::serializeText(const Field & field, WriteBuffer & ostr) const
{
writeString(boost::get<const String &>(field), ostr);
writeString(get<const String &>(field), ostr);
}
......@@ -85,7 +85,7 @@ void DataTypeFixedString::deserializeText(Field & field, ReadBuffer & istr) cons
void DataTypeFixedString::serializeTextEscaped(const Field & field, WriteBuffer & ostr) const
{
writeEscapedString(boost::get<const String &>(field), ostr);
writeEscapedString(get<const String &>(field), ostr);
}
......@@ -99,7 +99,7 @@ void DataTypeFixedString::deserializeTextEscaped(Field & field, ReadBuffer & ist
void DataTypeFixedString::serializeTextQuoted(const Field & field, WriteBuffer & ostr) const
{
writeQuotedString(boost::get<const String &>(field), ostr);
writeQuotedString(get<const String &>(field), ostr);
}
......@@ -119,7 +119,7 @@ ColumnPtr DataTypeFixedString::createColumn() const
ColumnPtr DataTypeFixedString::createConstColumn(size_t size, const Field & field) const
{
return new ColumnConst<String>(size, boost::get<String>(field));
return new ColumnConst<String>(size, get<const String &>(field));
}
}
......@@ -22,7 +22,7 @@ using Poco::SharedPtr;
void DataTypeString::serializeBinary(const Field & field, WriteBuffer & ostr) const
{
const String & s = boost::get<String>(field);
const String & s = get<const String &>(field);
writeVarUInt(s.size(), ostr);
writeString(s, ostr);
}
......@@ -33,7 +33,7 @@ void DataTypeString::deserializeBinary(Field & field, ReadBuffer & istr) const
UInt64 size;
readVarUInt(size, istr);
field = String();
String & s = boost::get<String>(field);
String & s = get<String &>(field);
s.resize(size);
/// непереносимо, но (действительно) быстрее
istr.readStrict(const_cast<char*>(s.data()), size);
......@@ -105,7 +105,7 @@ void DataTypeString::deserializeBinary(IColumn & column, ReadBuffer & istr, size
void DataTypeString::serializeText(const Field & field, WriteBuffer & ostr) const
{
writeString(boost::get<const String &>(field), ostr);
writeString(get<const String &>(field), ostr);
}
......@@ -119,7 +119,7 @@ void DataTypeString::deserializeText(Field & field, ReadBuffer & istr) const
void DataTypeString::serializeTextEscaped(const Field & field, WriteBuffer & ostr) const
{
writeEscapedString(boost::get<const String &>(field), ostr);
writeEscapedString(get<const String &>(field), ostr);
}
......@@ -133,7 +133,7 @@ void DataTypeString::deserializeTextEscaped(Field & field, ReadBuffer & istr) co
void DataTypeString::serializeTextQuoted(const Field & field, WriteBuffer & ostr) const
{
writeQuotedString(boost::get<const String &>(field), ostr);
writeQuotedString(get<const String &>(field), ostr);
}
......@@ -153,7 +153,7 @@ ColumnPtr DataTypeString::createColumn() const
ColumnPtr DataTypeString::createConstColumn(size_t size, const Field & field) const
{
return new ColumnConst<String>(size, boost::get<String>(field));
return new ColumnConst<String>(size, get<const String &>(field));
}
}
......@@ -58,8 +58,8 @@ int main(int argc, char ** argv)
std::cout << "Reading, elapsed: " << static_cast<double>(stopwatch.elapsed()) / 1000000 << std::endl;
std::cout << std::endl
<< boost::get<DB::String>((*column)[0]) << std::endl
<< boost::get<DB::String>((*column)[n - 1]) << std::endl;
<< DB::get<DB::String>((*column)[0]) << std::endl
<< DB::get<DB::String>((*column)[n - 1]) << std::endl;
}
}
catch (const DB::Exception & e)
......
......@@ -68,7 +68,7 @@ int main(int argc, char ** argv)
DB::Float64 x = 0;
for (size_t i = 0; i < n; ++i)
x += boost::get<DB::Float64>((*block.getByPosition(2).column)[i]);
x += DB::get<DB::Float64>((*block.getByPosition(2).column)[i]);
std::cout << x << std::endl;
}
......
......@@ -197,7 +197,7 @@ void Aggregator::execute(BlockInputStreamPtr stream, AggregatedDataVariants & re
{
/// Строим ключ
Field field = column[i];
UInt64 key = boost::apply_visitor(visitor, field);
UInt64 key = apply_visitor(visitor, field);
AggregatedDataWithUInt64Key::iterator it;
bool inserted;
......@@ -436,7 +436,7 @@ Block Aggregator::convertToBlock(AggregatedDataVariants & data_variants)
size_t i = 0;
for (AggregateFunctionsPlainPtrs::const_iterator jt = data.begin(); jt != data.end(); ++jt, ++i)
res.getByPosition(i).column->insert(*jt);
res.getByPosition(i).column->insert(AggregateFunctionPtr(*jt));
}
else if (data_variants.type == AggregatedDataVariants::KEY_64)
{
......@@ -456,7 +456,7 @@ Block Aggregator::convertToBlock(AggregatedDataVariants & data_variants)
size_t i = 1;
for (AggregateFunctionsPlainPtrs::const_iterator jt = it->second.begin(); jt != it->second.end(); ++jt, ++i)
res.getByPosition(i).column->insert(*jt);
res.getByPosition(i).column->insert(AggregateFunctionPtr(*jt));
}
}
else if (data_variants.type == AggregatedDataVariants::KEY_STRING)
......@@ -471,7 +471,7 @@ Block Aggregator::convertToBlock(AggregatedDataVariants & data_variants)
size_t i = 1;
for (AggregateFunctionsPlainPtrs::const_iterator jt = it->second.begin(); jt != it->second.end(); ++jt, ++i)
res.getByPosition(i).column->insert(*jt);
res.getByPosition(i).column->insert(AggregateFunctionPtr(*jt));
}
}
else if (data_variants.type == AggregatedDataVariants::HASHED)
......@@ -485,7 +485,7 @@ Block Aggregator::convertToBlock(AggregatedDataVariants & data_variants)
res.getByPosition(i).column->insert(*jt);
for (AggregateFunctionsPlainPtrs::const_iterator jt = it->second.second.begin(); jt != it->second.second.end(); ++jt, ++i)
res.getByPosition(i).column->insert(*jt);
res.getByPosition(i).column->insert(AggregateFunctionPtr(*jt));
}
}
else if (data_variants.type == AggregatedDataVariants::GENERIC)
......@@ -499,7 +499,7 @@ Block Aggregator::convertToBlock(AggregatedDataVariants & data_variants)
res.getByPosition(i).column->insert(*jt);
for (AggregateFunctionsPlainPtrs::const_iterator jt = it->second.begin(); jt != it->second.end(); ++jt, ++i)
res.getByPosition(i).column->insert(*jt);
res.getByPosition(i).column->insert(AggregateFunctionPtr(*jt));
}
}
else
......@@ -717,7 +717,7 @@ void Aggregator::merge(BlockInputStreamPtr stream, AggregatedDataVariants & resu
{
/// Строим ключ
Field field = column[i];
UInt64 key = boost::apply_visitor(visitor, field);
UInt64 key = apply_visitor(visitor, field);
AggregatedDataWithUInt64Key::iterator it;
bool inserted;
......
......@@ -136,7 +136,7 @@ void Expression::addSemantic(ASTPtr & ast)
if (arguments.size() != 2)
throw Exception("Function tupleElement requires exactly two arguments: tuple and element index.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
node->return_type = func_tuple_elem->getReturnType(argument_types, boost::get<UInt64>(dynamic_cast<ASTLiteral &>(*arguments[1]).value));
node->return_type = func_tuple_elem->getReturnType(argument_types, safeGet<UInt64>(dynamic_cast<ASTLiteral &>(*arguments[1]).value));
}
else
node->return_type = node->function->getReturnType(argument_types);
......@@ -168,7 +168,7 @@ void Expression::addSemantic(ASTPtr & ast)
}
else if (ASTLiteral * node = dynamic_cast<ASTLiteral *>(&*ast))
{
node->type = boost::apply_visitor(FieldToDataType(), node->value);
node->type = apply_visitor(FieldToDataType(), node->value);
}
}
......
......@@ -214,9 +214,9 @@ static void getLimitLengthAndOffset(ASTSelectQuery & query, size_t & length, siz
offset = 0;
if (query.limit_length)
{
length = boost::get<UInt64>(dynamic_cast<ASTLiteral &>(*query.limit_length).value);
length = safeGet<UInt64>(dynamic_cast<ASTLiteral &>(*query.limit_length).value);
if (query.limit_offset)
offset = boost::get<UInt64>(dynamic_cast<ASTLiteral &>(*query.limit_offset).value);
offset = safeGet<UInt64>(dynamic_cast<ASTLiteral &>(*query.limit_offset).value);
}
}
......
......@@ -95,7 +95,7 @@ void Set::create(BlockInputStreamPtr stream)
{
/// Строим ключ
Field field = column[i];
UInt64 key = boost::apply_visitor(visitor, field);
UInt64 key = apply_visitor(visitor, field);
res.insert(key);
}
......@@ -304,7 +304,7 @@ void Set::execute(Block & block, const ColumnNumbers & arguments, size_t result,
{
/// Строим ключ
Field field = column[i];
UInt64 key = boost::apply_visitor(visitor, field);
UInt64 key = apply_visitor(visitor, field);
vec_res[i] = negative ^ (set.end() != set.find(key));
}
}
......
......@@ -118,12 +118,12 @@ int main(int argc, char ** argv)
for (DB::AggregatedData::const_iterator it = aggregated_data.begin(); it != aggregated_data.end(); ++it)
{
for (DB::Row::const_iterator jt = it->first.begin(); jt != it->first.end(); ++jt)
std::cout << boost::apply_visitor(DB::FieldVisitorToString(), *jt) << '\t';
std::cout << apply_visitor(DB::FieldVisitorToString(), *jt) << '\t';
for (DB::AggregateFunctionsPlainPtrs::const_iterator jt = it->second.begin(); jt != it->second.end(); ++jt)
{
DB::Field result = (*jt)->getResult();
std::cout << boost::apply_visitor(DB::FieldVisitorToString(), result) << '\t';
std::cout << apply_visitor(DB::FieldVisitorToString(), result) << '\t';
}
std::cout << '\n';
......
......@@ -36,7 +36,7 @@ void dump(DB::IAST & ast, int level = 0)
}
else if (DB::ASTLiteral * node = dynamic_cast<DB::ASTLiteral *>(&ast))
{
std::cout << prefix << node << " Literal, " << boost::apply_visitor(DB::FieldVisitorToString(), node->value)
std::cout << prefix << node << " Literal, " << apply_visitor(DB::FieldVisitorToString(), node->value)
<< ", type = " << node->type->getName() << std::endl;
}
......
......@@ -74,7 +74,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String
if (database_ast)
query->from = dynamic_cast<ASTIdentifier &>(*database_ast).name;
if (like_ast)
query->like = boost::get<const String &>(dynamic_cast<ASTLiteral &>(*like_ast).value);
query->like = safeGet<const String &>(dynamic_cast<ASTLiteral &>(*like_ast).value);
return true;
}
......
......@@ -255,7 +255,7 @@ void formatAST(const ASTSetQuery & ast, std::ostream & s, size_t indent, bool
if (it != ast.changes.begin())
s << ", ";
s << it->name << " = " << boost::apply_visitor(FieldVisitorToString(), it->value);
s << it->name << " = " << apply_visitor(FieldVisitorToString(), it->value);
}
}
......@@ -373,7 +373,7 @@ void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent, bo
void formatAST(const ASTLiteral & ast, std::ostream & s, size_t indent, bool hilite, bool one_line)
{
s << boost::apply_visitor(FieldVisitorToString(), ast.value);
s << apply_visitor(FieldVisitorToString(), ast.value);
if (!ast.alias.empty())
writeAlias(ast.alias, s, hilite, one_line);
......
......@@ -62,7 +62,7 @@ StoragePtr StorageFactory::get(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
String source_database = dynamic_cast<ASTIdentifier &>(*args[0]).name;
String table_name_regexp = boost::get<const String &>(dynamic_cast<ASTLiteral &>(*args[1]).value);
String table_name_regexp = safeGet<const String &>(dynamic_cast<ASTLiteral &>(*args[1]).value);
return new StorageMerge(table_name, columns, source_database, table_name_regexp, context);
}
......@@ -182,7 +182,7 @@ StoragePtr StorageFactory::get(
String date_column_name = dynamic_cast<ASTIdentifier &>(*args[0]).name;
ASTPtr sampling_expression = arg_offset == 0 ? NULL : args[1];
UInt64 index_granularity = boost::get<UInt64>(dynamic_cast<ASTLiteral &>(*args[arg_offset + 2]).value);
UInt64 index_granularity = safeGet<UInt64>(dynamic_cast<ASTLiteral &>(*args[arg_offset + 2]).value);
ASTFunction & primary_expr_func = dynamic_cast<ASTFunction &>(*args[arg_offset + 1]);
if (primary_expr_func.name != "tuple")
......@@ -221,7 +221,7 @@ StoragePtr StorageFactory::get(
String date_column_name = dynamic_cast<ASTIdentifier &>(*args[0]).name;
ASTPtr sampling_expression = arg_offset == 0 ? NULL : args[1];
UInt64 index_granularity = boost::get<UInt64>(dynamic_cast<ASTLiteral &>(*args[arg_offset + 2]).value);
UInt64 index_granularity = safeGet<UInt64>(dynamic_cast<ASTLiteral &>(*args[arg_offset + 2]).value);
String sign_column_name = dynamic_cast<ASTIdentifier &>(*args[arg_offset + 3]).name;
ASTFunction & primary_expr_func = dynamic_cast<ASTFunction &>(*args[arg_offset + 1]);
......
......@@ -846,13 +846,13 @@ BlockInputStreams StorageMergeTree::read(
ASTSelectQuery & select = *dynamic_cast<ASTSelectQuery*>(&*query);
if (select.sample_size)
{
double size = boost::apply_visitor(FieldVisitorConvertToNumber<double>(),
double size = apply_visitor(FieldVisitorConvertToNumber<double>(),
dynamic_cast<ASTLiteral&>(*select.sample_size).value);
if (size < 0)
throw Exception("Negative sample size", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
if (size > 1)
{
size_t requested_count = boost::apply_visitor(FieldVisitorConvertToNumber<UInt64>(), dynamic_cast<ASTLiteral&>(*select.sample_size).value);
size_t requested_count = apply_visitor(FieldVisitorConvertToNumber<UInt64>(), dynamic_cast<ASTLiteral&>(*select.sample_size).value);
/// Узнаем, сколько строк мы бы прочли без семплирования.
LOG_DEBUG(log, "Preliminary index scan with condition: " << key_condition.toString());
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册