mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge
This commit is contained in:
commit
360fc4cf71
@ -10,6 +10,6 @@ else()
|
||||
endif()
|
||||
|
||||
macro (add_check target)
|
||||
add_test(test_${target} ${target})
|
||||
add_test(NAME test_${target} COMMAND ${target} WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR})
|
||||
add_dependencies(check ${target})
|
||||
endmacro (add_check)
|
@ -0,0 +1,489 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||
#include <DB/Parsers/CommonParsers.h>
|
||||
#include <statdaemons/ext/range.hpp>
|
||||
#include <boost/range/iterator_range_core.hpp>
|
||||
#include <DB/Parsers/ExpressionElementParsers.h>
|
||||
#include <DB/Parsers/ASTLiteral.h>
|
||||
#include <bitset>
|
||||
#include <stack>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// helper type for comparing `std::pair`s using solely the .first member
|
||||
template <template <typename> class Comparator>
|
||||
struct ComparePairFirst final
|
||||
{
|
||||
template <typename T1, typename T2>
|
||||
bool operator()(const std::pair<T1, T2> & lhs, const std::pair<T1, T2> & rhs) const
|
||||
{
|
||||
return Comparator<T1>{}(lhs.first, rhs.first);
|
||||
}
|
||||
};
|
||||
|
||||
struct AggregateFunctionSequenceMatchData final
|
||||
{
|
||||
static constexpr auto max_events = 32;
|
||||
|
||||
using Timestamp = std::uint32_t;
|
||||
using Events = std::bitset<max_events>;
|
||||
using TimestampEvents = std::pair<Timestamp, Events>;
|
||||
using Comparator = ComparePairFirst<std::less>;
|
||||
|
||||
bool sorted = true;
|
||||
std::vector<TimestampEvents> eventsList;
|
||||
|
||||
void add(const Timestamp timestamp, const Events & events)
|
||||
{
|
||||
/// store information exclusively for rows with at least one event
|
||||
if (events.any())
|
||||
{
|
||||
eventsList.emplace_back(timestamp, events);
|
||||
sorted = false;
|
||||
}
|
||||
}
|
||||
|
||||
void merge(const AggregateFunctionSequenceMatchData & other)
|
||||
{
|
||||
const auto size = eventsList.size();
|
||||
|
||||
eventsList.insert(std::end(eventsList), std::begin(other.eventsList), std::end(other.eventsList));
|
||||
|
||||
/// either sort whole container or do so partially merging ranges afterwards
|
||||
if (!sorted && !other.sorted)
|
||||
std::sort(std::begin(eventsList), std::end(eventsList), Comparator{});
|
||||
else
|
||||
{
|
||||
const auto begin = std::begin(eventsList);
|
||||
const auto middle = std::next(begin, size);
|
||||
const auto end = std::end(eventsList);
|
||||
|
||||
if (!sorted)
|
||||
std::sort(begin, middle, Comparator{});
|
||||
|
||||
if (!other.sorted)
|
||||
std::sort(middle, end, Comparator{});
|
||||
|
||||
std::inplace_merge(begin, middle, end, Comparator{});
|
||||
}
|
||||
|
||||
sorted = true;
|
||||
}
|
||||
|
||||
void sort()
|
||||
{
|
||||
if (!sorted)
|
||||
{
|
||||
std::sort(std::begin(eventsList), std::end(eventsList), Comparator{});
|
||||
sorted = true;
|
||||
}
|
||||
}
|
||||
|
||||
void serialize(WriteBuffer & buf) const
|
||||
{
|
||||
writeBinary(sorted, buf);
|
||||
writeBinary(eventsList.size(), buf);
|
||||
|
||||
for (const auto & events : eventsList)
|
||||
{
|
||||
writeBinary(events.first, buf);
|
||||
writeBinary(events.second.to_ulong(), buf);
|
||||
}
|
||||
}
|
||||
|
||||
void deserialize(ReadBuffer & buf)
|
||||
{
|
||||
readBinary(sorted, buf);
|
||||
|
||||
std::size_t size;
|
||||
readBinary(size, buf);
|
||||
|
||||
decltype(eventsList) eventsList;
|
||||
eventsList.reserve(size);
|
||||
|
||||
for (std::size_t i = 0; i < size; ++i)
|
||||
{
|
||||
std::uint32_t timestamp;
|
||||
readBinary(timestamp, buf);
|
||||
|
||||
unsigned long events;
|
||||
readBinary(events, buf);
|
||||
|
||||
eventsList.emplace_back(timestamp, Events{events});
|
||||
}
|
||||
|
||||
this->eventsList = std::move(eventsList);
|
||||
}
|
||||
};
|
||||
|
||||
class AggregateFunctionSequenceMatch final : public IAggregateFunctionHelper<AggregateFunctionSequenceMatchData>
|
||||
{
|
||||
public:
|
||||
static bool sufficientArgs(const std::size_t arg_count) { return arg_count >= 3; }
|
||||
|
||||
String getName() const override { return "sequenceMatch"; }
|
||||
|
||||
DataTypePtr getReturnType() const override { return new DataTypeUInt8; }
|
||||
|
||||
void setParameters(const Array & params) override
|
||||
{
|
||||
if (params.size() != 1)
|
||||
throw Exception{
|
||||
"Aggregate function " + getName() + " requires exactly one parameter.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
|
||||
};
|
||||
|
||||
pattern = params.front().safeGet<std::string>();
|
||||
}
|
||||
|
||||
void setArguments(const DataTypes & arguments) override
|
||||
{
|
||||
arg_count = arguments.size();
|
||||
|
||||
if (!sufficientArgs(arg_count))
|
||||
throw Exception{
|
||||
"Aggregate function " + getName() + " requires at least 3 arguments.",
|
||||
ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION
|
||||
};
|
||||
|
||||
if (arg_count - 1 > Data::max_events)
|
||||
throw Exception{
|
||||
"Aggregate function " + getName() + " supports up to " +
|
||||
std::to_string(Data::max_events) + " event arguments.",
|
||||
ErrorCodes::TOO_MUCH_ARGUMENTS_FOR_FUNCTION
|
||||
};
|
||||
|
||||
const auto time_arg = arguments.front().get();
|
||||
if (!typeid_cast<const DataTypeDateTime *>(time_arg))
|
||||
throw Exception{
|
||||
"Illegal type " + time_arg->getName() + " of first argument of aggregate function " +
|
||||
getName() + ", must be DateTime",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
|
||||
for (const auto i : ext::range(1, arg_count))
|
||||
{
|
||||
const auto cond_arg = arguments[i].get();
|
||||
if (!typeid_cast<const DataTypeUInt8 *>(cond_arg))
|
||||
throw Exception{
|
||||
"Illegal type " + cond_arg->getName() + " of argument " + toString(i + 1) +
|
||||
" of aggregate function " + getName() + ", must be UInt8",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
}
|
||||
|
||||
parsePattern();
|
||||
}
|
||||
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num) const override
|
||||
{
|
||||
const auto timestamp = static_cast<const ColumnUInt32 *>(columns[0])->getData()[row_num];
|
||||
|
||||
Data::Events events;
|
||||
for (const auto i : ext::range(1, arg_count))
|
||||
{
|
||||
const auto event = static_cast<const ColumnUInt8 *>(columns[i])->getData()[row_num];
|
||||
events.set(i - 1, event);
|
||||
}
|
||||
|
||||
data(place).add(timestamp, events);
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
|
||||
{
|
||||
data(place).merge(data(rhs));
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
||||
{
|
||||
data(place).serialize(buf);
|
||||
}
|
||||
|
||||
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
|
||||
{
|
||||
AggregateFunctionSequenceMatchData tmp;
|
||||
tmp.deserialize(buf);
|
||||
|
||||
data(place).merge(tmp);
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
||||
{
|
||||
const_cast<Data &>(data(place)).sort();
|
||||
static_cast<ColumnUInt8 &>(to).getData().push_back(match(place));
|
||||
}
|
||||
|
||||
private:
|
||||
enum class PatternActionType
|
||||
{
|
||||
SpecificEvent,
|
||||
AnyEvent,
|
||||
KleeneStar,
|
||||
TimeLessOrEqual,
|
||||
TimeLess,
|
||||
TimeGreaterOrEqual,
|
||||
TimeGreater
|
||||
};
|
||||
|
||||
static std::string to_string(const PatternActionType type)
|
||||
{
|
||||
static const std::map<PatternActionType, std::string> map{
|
||||
{ PatternActionType::SpecificEvent, "SpecificEvent" },
|
||||
{ PatternActionType::AnyEvent, "AnyEvent" },
|
||||
{ PatternActionType::KleeneStar, "KleeneStar" },
|
||||
{ PatternActionType::TimeLessOrEqual, "TimeLessOrEqual" },
|
||||
{ PatternActionType::TimeLess, "TimeLess", },
|
||||
{ PatternActionType::TimeGreaterOrEqual, "TimeGreaterOrEqual" },
|
||||
{ PatternActionType::TimeGreater, "TimeGreater" }
|
||||
};
|
||||
|
||||
return map.find(type)->second;
|
||||
}
|
||||
|
||||
struct PatternAction final
|
||||
{
|
||||
PatternActionType type;
|
||||
std::uint32_t extra;
|
||||
|
||||
PatternAction() = default;
|
||||
PatternAction(const PatternActionType type, const std::uint32_t extra = 0) : type{type}, extra{extra} {}
|
||||
};
|
||||
|
||||
using PatternActions = std::vector<PatternAction>;
|
||||
|
||||
|
||||
void parsePattern()
|
||||
{
|
||||
PatternActions actions{
|
||||
{ PatternActionType::KleeneStar }
|
||||
};
|
||||
|
||||
ParserString special_open_p("(?");
|
||||
ParserString special_close_p(")");
|
||||
ParserString t_p("t");
|
||||
ParserString less_or_equal_p("<=");
|
||||
ParserString less_p("<");
|
||||
ParserString greater_or_equal_p(">=");
|
||||
ParserString greater_p(">");
|
||||
ParserString dot_closure_p(".*");
|
||||
ParserString dot_p(".");
|
||||
ParserNumber number_p;
|
||||
|
||||
auto pos = pattern.data();
|
||||
const auto begin = pos;
|
||||
const auto end = pos + pattern.size();
|
||||
|
||||
ASTPtr node;
|
||||
decltype(pos) max_parsed_pos{};
|
||||
Expected expected;
|
||||
|
||||
const auto throw_exception = [&] (const std::string & msg) {
|
||||
throw Exception{
|
||||
msg + " '" + std::string(pos, end) + "' at position " + std::to_string(pos - begin),
|
||||
ErrorCodes::SYNTAX_ERROR
|
||||
};
|
||||
};
|
||||
|
||||
while (pos < end)
|
||||
{
|
||||
if (special_open_p.ignore(pos, end))
|
||||
{
|
||||
if (t_p.ignore(pos, end))
|
||||
{
|
||||
PatternActionType type;
|
||||
|
||||
if (less_or_equal_p.ignore(pos, end))
|
||||
type = PatternActionType::TimeLessOrEqual;
|
||||
else if (less_p.ignore(pos, end))
|
||||
type = PatternActionType::TimeLess;
|
||||
else if (greater_or_equal_p.ignore(pos, end))
|
||||
type = PatternActionType::TimeGreaterOrEqual;
|
||||
else if (greater_p.ignore(pos, end))
|
||||
type = PatternActionType::TimeGreater;
|
||||
else
|
||||
throw_exception("Unknown time condition");
|
||||
|
||||
if (!number_p.parse(pos, end, node, max_parsed_pos, expected))
|
||||
throw_exception("Could not parse number");
|
||||
|
||||
if (actions.back().type != PatternActionType::SpecificEvent &&
|
||||
actions.back().type != PatternActionType::AnyEvent &&
|
||||
actions.back().type != PatternActionType::KleeneStar)
|
||||
throw Exception{
|
||||
"Temporal condition should be preceeded by an event condition",
|
||||
ErrorCodes::BAD_ARGUMENTS
|
||||
};
|
||||
|
||||
actions.emplace_back(type, typeid_cast<const ASTLiteral &>(*node).value.safeGet<UInt64>());
|
||||
}
|
||||
else if (number_p.parse(pos, end, node, max_parsed_pos, expected))
|
||||
{
|
||||
const auto event_number = typeid_cast<const ASTLiteral &>(*node).value.safeGet<UInt64>();
|
||||
if (event_number > arg_count - 1)
|
||||
throw Exception{
|
||||
"Event number " + std::to_string(event_number) + " is out of range",
|
||||
ErrorCodes::BAD_ARGUMENTS
|
||||
};
|
||||
|
||||
actions.emplace_back(PatternActionType::SpecificEvent, event_number - 1);
|
||||
}
|
||||
else
|
||||
throw_exception("Unexpected special sequence");
|
||||
|
||||
if (!special_close_p.ignore(pos, end))
|
||||
throw_exception("Expected closing parenthesis, found");
|
||||
|
||||
}
|
||||
else if (dot_closure_p.ignore(pos, end))
|
||||
actions.emplace_back(PatternActionType::KleeneStar);
|
||||
else if (dot_p.ignore(pos, end))
|
||||
actions.emplace_back(PatternActionType::AnyEvent);
|
||||
else
|
||||
throw_exception("Could not parse pattern, unexpected starting symbol");
|
||||
}
|
||||
|
||||
this->actions = std::move(actions);
|
||||
}
|
||||
|
||||
bool match(const ConstAggregateDataPtr & place) const
|
||||
{
|
||||
const auto action_begin = std::begin(actions);
|
||||
const auto action_end = std::end(actions);
|
||||
auto action_it = action_begin;
|
||||
|
||||
const auto & data_ref = data(place);
|
||||
const auto events_begin = std::begin(data_ref.eventsList);
|
||||
const auto events_end = std::end(data_ref.eventsList);
|
||||
auto events_it = events_begin;
|
||||
|
||||
auto base_it = events_begin;
|
||||
|
||||
/// an iterator to action plus an iterator to row in events list plus timestamp at the start of sequence
|
||||
using backtrack_info = std::tuple<decltype(action_it), decltype(events_it), decltype(base_it)>;
|
||||
std::stack<backtrack_info> back_stack;
|
||||
|
||||
/// backtrack if possible
|
||||
const auto do_backtrack = [&] {
|
||||
while (!back_stack.empty())
|
||||
{
|
||||
auto & top = back_stack.top();
|
||||
|
||||
action_it = std::get<0>(top);
|
||||
events_it = std::next(std::get<1>(top));
|
||||
base_it = std::get<2>(top);
|
||||
|
||||
back_stack.pop();
|
||||
|
||||
if (events_it != events_end)
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
};
|
||||
|
||||
while (action_it != action_end && events_it != events_end)
|
||||
{
|
||||
// std::cout << "start_timestamp " << base_it->first << "; ";
|
||||
// std::cout << "elapsed " << (events_it->first - base_it->first) << "; ";
|
||||
// std::cout << "action " << (action_it - action_begin) << " { " << to_string(action_it->type) << ' ' << action_it->extra << " }; ";
|
||||
// std::cout << "symbol " << (events_it - events_begin) << " { " << events_it->first << ' ' << events_it->second.to_ulong() << " }" << std::endl;
|
||||
|
||||
if (action_it->type == PatternActionType::SpecificEvent)
|
||||
{
|
||||
if (events_it->second.test(action_it->extra))
|
||||
{
|
||||
/// move to the next action and events
|
||||
base_it = events_it;
|
||||
++action_it, ++events_it;
|
||||
}
|
||||
else if (!do_backtrack())
|
||||
/// backtracking failed, bail out
|
||||
break;
|
||||
}
|
||||
else if (action_it->type == PatternActionType::AnyEvent)
|
||||
{
|
||||
base_it = events_it;
|
||||
++action_it, ++events_it;
|
||||
}
|
||||
else if (action_it->type == PatternActionType::KleeneStar)
|
||||
{
|
||||
back_stack.emplace(action_it, events_it, base_it);
|
||||
base_it = events_it;
|
||||
++action_it;
|
||||
}
|
||||
else if (action_it->type == PatternActionType::TimeLessOrEqual)
|
||||
{
|
||||
if (events_it->first - base_it->first <= action_it->extra)
|
||||
{
|
||||
/// condition satisfied, move onto next action
|
||||
back_stack.emplace(action_it, events_it, base_it);
|
||||
base_it = events_it;
|
||||
++action_it;
|
||||
}
|
||||
else if (!do_backtrack())
|
||||
break;
|
||||
}
|
||||
else if (action_it->type == PatternActionType::TimeLess)
|
||||
{
|
||||
if (events_it->first - base_it->first < action_it->extra)
|
||||
{
|
||||
back_stack.emplace(action_it, events_it, base_it);
|
||||
base_it = events_it;
|
||||
++action_it;
|
||||
}
|
||||
else if (!do_backtrack())
|
||||
break;
|
||||
}
|
||||
else if (action_it->type == PatternActionType::TimeGreaterOrEqual)
|
||||
{
|
||||
if (events_it->first - base_it->first >= action_it->extra)
|
||||
{
|
||||
back_stack.emplace(action_it, events_it, base_it);
|
||||
base_it = events_it;
|
||||
++action_it;
|
||||
}
|
||||
else if (++events_it == events_end && !do_backtrack())
|
||||
break;
|
||||
}
|
||||
else if (action_it->type == PatternActionType::TimeGreater)
|
||||
{
|
||||
if (events_it->first - base_it->first > action_it->extra)
|
||||
{
|
||||
back_stack.emplace(action_it, events_it, base_it);
|
||||
base_it = events_it;
|
||||
++action_it;
|
||||
}
|
||||
else if (++events_it == events_end && !do_backtrack())
|
||||
break;
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
"Unknown PatternActionType",
|
||||
ErrorCodes::LOGICAL_ERROR
|
||||
};
|
||||
}
|
||||
|
||||
/// if there are some actions remaining
|
||||
if (action_it != action_end)
|
||||
{
|
||||
/// match multiple empty strings at end
|
||||
while (action_it->type == PatternActionType::KleeneStar ||
|
||||
action_it->type == PatternActionType::TimeLessOrEqual ||
|
||||
action_it->type == PatternActionType::TimeLess ||
|
||||
(action_it->type == PatternActionType::TimeGreaterOrEqual && action_it->extra == 0))
|
||||
++action_it;
|
||||
}
|
||||
|
||||
return action_it == action_end;
|
||||
}
|
||||
|
||||
std::string pattern;
|
||||
std::size_t arg_count;
|
||||
PatternActions actions;
|
||||
};
|
||||
|
||||
}
|
@ -0,0 +1,504 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
#include <DB/IO/ReadHelpers.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
|
||||
#include <DB/AggregateFunctions/IBinaryAggregateFunction.h>
|
||||
#include <DB/Columns/ColumnVector.h>
|
||||
|
||||
#include <cmath>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Эта функция возвращает true если оба значения велики и сравнимы.
|
||||
/// Она употребляется для вычисления среднего значения путём слияния двух источников.
|
||||
/// Ибо если размеры обоих источников велики и сравнимы, то надо применить особенную
|
||||
/// формулу гарантирующую больше стабильности.
|
||||
bool areComparable(UInt64 a, UInt64 b)
|
||||
{
|
||||
const Float64 sensitivity = 0.001;
|
||||
const UInt64 threshold = 10000;
|
||||
|
||||
if ((a == 0) || (b == 0))
|
||||
return false;
|
||||
|
||||
auto res = std::minmax(a, b);
|
||||
return (((1 - static_cast<Float64>(res.first) / res.second) < sensitivity) && (res.first > threshold));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/** Статистические аггрегатные функции:
|
||||
* varSamp - выборочная дисперсия
|
||||
* stddevSamp - среднее выборочное квадратичное отклонение
|
||||
* varPop - дисперсия
|
||||
* stddevPop - среднее квадратичное отклонение
|
||||
* covarSamp - выборочная ковариация
|
||||
* covarPop - ковариация
|
||||
* corr - корреляция
|
||||
*/
|
||||
|
||||
/** Параллельный и инкрементальный алгоритм для вычисления дисперсии.
|
||||
* Источник: "Updating formulae and a pairwise algorithm for computing sample variances"
|
||||
* (Chan et al., Stanford University, 12.1979)
|
||||
*/
|
||||
template<typename T, typename Op>
|
||||
class AggregateFunctionVarianceData
|
||||
{
|
||||
public:
|
||||
AggregateFunctionVarianceData() = default;
|
||||
|
||||
void update(const IColumn & column, size_t row_num)
|
||||
{
|
||||
T received = static_cast<const ColumnVector<T> &>(column).getData()[row_num];
|
||||
Float64 val = static_cast<Float64>(received);
|
||||
Float64 delta = val - mean;
|
||||
|
||||
++count;
|
||||
mean += delta / count;
|
||||
m2 += delta * (val - mean);
|
||||
}
|
||||
|
||||
void mergeWith(const AggregateFunctionVarianceData & source)
|
||||
{
|
||||
UInt64 total_count = count + source.count;
|
||||
if (total_count == 0)
|
||||
return;
|
||||
|
||||
Float64 factor = static_cast<Float64>(count * source.count) / total_count;
|
||||
Float64 delta = mean - source.mean;
|
||||
|
||||
if (areComparable(count, source.count))
|
||||
mean = (source.count * source.mean + count * mean) / total_count;
|
||||
else
|
||||
mean = source.mean + delta * (static_cast<Float64>(count) / total_count);
|
||||
|
||||
m2 += source.m2 + delta * delta * factor;
|
||||
count = total_count;
|
||||
}
|
||||
|
||||
void serialize(WriteBuffer & buf) const
|
||||
{
|
||||
writeVarUInt(count, buf);
|
||||
writeBinary(mean, buf);
|
||||
writeBinary(m2, buf);
|
||||
}
|
||||
|
||||
void deserialize(ReadBuffer & buf)
|
||||
{
|
||||
readVarUInt(count, buf);
|
||||
readBinary(mean, buf);
|
||||
readBinary(m2, buf);
|
||||
}
|
||||
|
||||
void publish(IColumn & to) const
|
||||
{
|
||||
static_cast<ColumnFloat64 &>(to).getData().push_back(Op::apply(m2, count));
|
||||
}
|
||||
|
||||
private:
|
||||
UInt64 count = 0;
|
||||
Float64 mean = 0.0;
|
||||
Float64 m2 = 0.0;
|
||||
};
|
||||
|
||||
/** Основной код для реализации функций varSamp, stddevSamp, varPop, stddevPop.
|
||||
*/
|
||||
template<typename T, typename Op>
|
||||
class AggregateFunctionVariance final
|
||||
: public IUnaryAggregateFunction<AggregateFunctionVarianceData<T, Op>,
|
||||
AggregateFunctionVariance<T, Op> >
|
||||
{
|
||||
public:
|
||||
String getName() const override { return Op::name; }
|
||||
|
||||
DataTypePtr getReturnType() const override
|
||||
{
|
||||
return new DataTypeFloat64;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
{
|
||||
if (!argument->behavesAsNumber())
|
||||
throw Exception("Illegal type " + argument->getName() + " of argument for aggregate function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
|
||||
{
|
||||
this->data(place).update(column, row_num);
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
|
||||
{
|
||||
this->data(place).mergeWith(this->data(rhs));
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
||||
{
|
||||
this->data(place).serialize(buf);
|
||||
}
|
||||
|
||||
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
|
||||
{
|
||||
AggregateFunctionVarianceData<T, Op> source;
|
||||
source.deserialize(buf);
|
||||
|
||||
this->data(place).mergeWith(source);
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
||||
{
|
||||
this->data(place).publish(to);
|
||||
}
|
||||
};
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/** Реализации функции varSamp.
|
||||
*/
|
||||
struct VarSampImpl
|
||||
{
|
||||
static constexpr auto name = "varSamp";
|
||||
|
||||
static inline Float64 apply(Float64 m2, UInt64 count)
|
||||
{
|
||||
if (count < 2)
|
||||
return std::numeric_limits<Float64>::infinity();
|
||||
else
|
||||
return m2 / (count - 1);
|
||||
}
|
||||
};
|
||||
|
||||
/** Реализация функции stddevSamp.
|
||||
*/
|
||||
struct StdDevSampImpl
|
||||
{
|
||||
static constexpr auto name = "stddevSamp";
|
||||
|
||||
static inline Float64 apply(Float64 m2, UInt64 count)
|
||||
{
|
||||
return sqrt(VarSampImpl::apply(m2, count));
|
||||
}
|
||||
};
|
||||
|
||||
/** Реализация функции varPop.
|
||||
*/
|
||||
struct VarPopImpl
|
||||
{
|
||||
static constexpr auto name = "varPop";
|
||||
|
||||
static inline Float64 apply(Float64 m2, UInt64 count)
|
||||
{
|
||||
if (count == 0)
|
||||
return std::numeric_limits<Float64>::infinity();
|
||||
else if (count == 1)
|
||||
return 0.0;
|
||||
else
|
||||
return m2 / count;
|
||||
}
|
||||
};
|
||||
|
||||
/** Реализация функции stddevPop.
|
||||
*/
|
||||
struct StdDevPopImpl
|
||||
{
|
||||
static constexpr auto name = "stddevPop";
|
||||
|
||||
static inline Float64 apply(Float64 m2, UInt64 count)
|
||||
{
|
||||
return sqrt(VarPopImpl::apply(m2, count));
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
/** Если флаг compute_marginal_moments установлен, этот класс предоставялет наследнику
|
||||
* CovarianceData поддержку маргинальных моментов для вычисления корреляции.
|
||||
*/
|
||||
template<bool compute_marginal_moments>
|
||||
class BaseCovarianceData
|
||||
{
|
||||
protected:
|
||||
void incrementMarginalMoments(Float64 left_incr, Float64 right_incr) {}
|
||||
void mergeWith(const BaseCovarianceData & source) {}
|
||||
void serialize(WriteBuffer & buf) const {}
|
||||
void deserialize(const ReadBuffer & buf) {}
|
||||
};
|
||||
|
||||
template<>
|
||||
class BaseCovarianceData<true>
|
||||
{
|
||||
protected:
|
||||
void incrementMarginalMoments(Float64 left_incr, Float64 right_incr)
|
||||
{
|
||||
left_m2 += left_incr;
|
||||
right_m2 += right_incr;
|
||||
}
|
||||
|
||||
void mergeWith(const BaseCovarianceData & source)
|
||||
{
|
||||
left_m2 += source.left_m2;
|
||||
right_m2 += source.right_m2;
|
||||
}
|
||||
|
||||
void serialize(WriteBuffer & buf) const
|
||||
{
|
||||
writeBinary(left_m2, buf);
|
||||
writeBinary(right_m2, buf);
|
||||
}
|
||||
|
||||
void deserialize(ReadBuffer & buf)
|
||||
{
|
||||
readBinary(left_m2, buf);
|
||||
readBinary(right_m2, buf);
|
||||
}
|
||||
|
||||
protected:
|
||||
Float64 left_m2 = 0.0;
|
||||
Float64 right_m2 = 0.0;
|
||||
};
|
||||
|
||||
/** Параллельный и инкрементальный алгоритм для вычисления ковариации.
|
||||
* Источник: "Numerically Stable, Single-Pass, Parallel Statistics Algorithms"
|
||||
* (J. Bennett et al., Sandia National Laboratories,
|
||||
* 2009 IEEE International Conference on Cluster Computing)
|
||||
*/
|
||||
template<typename T, typename U, typename Op, bool compute_marginal_moments>
|
||||
class CovarianceData : public BaseCovarianceData<compute_marginal_moments>
|
||||
{
|
||||
private:
|
||||
using Base = BaseCovarianceData<compute_marginal_moments>;
|
||||
|
||||
public:
|
||||
void update(const IColumn & column_left, const IColumn & column_right, size_t row_num)
|
||||
{
|
||||
T left_received = static_cast<const ColumnVector<T> &>(column_left).getData()[row_num];
|
||||
Float64 left_val = static_cast<Float64>(left_received);
|
||||
Float64 left_delta = left_val - left_mean;
|
||||
|
||||
U right_received = static_cast<const ColumnVector<U> &>(column_right).getData()[row_num];
|
||||
Float64 right_val = static_cast<Float64>(right_received);
|
||||
Float64 right_delta = right_val - right_mean;
|
||||
|
||||
Float64 old_right_mean = right_mean;
|
||||
|
||||
++count;
|
||||
|
||||
left_mean += left_delta / count;
|
||||
right_mean += right_delta / count;
|
||||
co_moment += (left_val - left_mean) * (right_val - old_right_mean);
|
||||
|
||||
/// Обновить маргинальные моменты, если они есть.
|
||||
if (compute_marginal_moments)
|
||||
{
|
||||
Float64 left_incr = left_delta * (left_val - left_mean);
|
||||
Float64 right_incr = right_delta * (right_val - right_mean);
|
||||
Base::incrementMarginalMoments(left_incr, right_incr);
|
||||
}
|
||||
}
|
||||
|
||||
void mergeWith(const CovarianceData & source)
|
||||
{
|
||||
UInt64 total_count = count + source.count;
|
||||
if (total_count == 0)
|
||||
return;
|
||||
|
||||
Float64 factor = static_cast<Float64>(count * source.count) / total_count;
|
||||
Float64 left_delta = left_mean - source.left_mean;
|
||||
Float64 right_delta = right_mean - source.right_mean;
|
||||
|
||||
if (areComparable(count, source.count))
|
||||
{
|
||||
left_mean = (source.count * source.left_mean + count * left_mean) / total_count;
|
||||
right_mean = (source.count * source.right_mean + count * right_mean) / total_count;
|
||||
}
|
||||
else
|
||||
{
|
||||
left_mean = source.left_mean + left_delta * (static_cast<Float64>(count) / total_count);
|
||||
right_mean = source.right_mean + right_delta * (static_cast<Float64>(count) / total_count);
|
||||
}
|
||||
|
||||
co_moment += source.co_moment + left_delta * right_delta * factor;
|
||||
count = total_count;
|
||||
|
||||
/// Обновить маргинальные моменты, если они есть.
|
||||
if (compute_marginal_moments)
|
||||
{
|
||||
Float64 left_incr = left_delta * left_delta * factor;
|
||||
Float64 right_incr = right_delta * right_delta * factor;
|
||||
Base::mergeWith(source);
|
||||
Base::incrementMarginalMoments(left_incr, right_incr);
|
||||
}
|
||||
}
|
||||
|
||||
void serialize(WriteBuffer & buf) const
|
||||
{
|
||||
writeVarUInt(count, buf);
|
||||
writeBinary(left_mean, buf);
|
||||
writeBinary(right_mean, buf);
|
||||
writeBinary(co_moment, buf);
|
||||
Base::serialize(buf);
|
||||
}
|
||||
|
||||
void deserialize(ReadBuffer & buf)
|
||||
{
|
||||
readVarUInt(count, buf);
|
||||
readBinary(left_mean, buf);
|
||||
readBinary(right_mean, buf);
|
||||
readBinary(co_moment, buf);
|
||||
Base::deserialize(buf);
|
||||
}
|
||||
|
||||
template<bool compute = compute_marginal_moments>
|
||||
void publish(IColumn & to, typename std::enable_if<compute>::type * = nullptr) const
|
||||
{
|
||||
static_cast<ColumnFloat64 &>(to).getData().push_back(Op::apply(co_moment, Base::left_m2, Base::right_m2, count));
|
||||
}
|
||||
|
||||
template<bool compute = compute_marginal_moments>
|
||||
void publish(IColumn & to, typename std::enable_if<!compute>::type * = nullptr) const
|
||||
{
|
||||
static_cast<ColumnFloat64 &>(to).getData().push_back(Op::apply(co_moment, count));
|
||||
}
|
||||
|
||||
private:
|
||||
UInt64 count = 0;
|
||||
Float64 left_mean = 0.0;
|
||||
Float64 right_mean = 0.0;
|
||||
Float64 co_moment = 0.0;
|
||||
};
|
||||
|
||||
template<typename T, typename U, typename Op, bool compute_marginal_moments = false>
|
||||
class AggregateFunctionCovariance final
|
||||
: public IBinaryAggregateFunction<
|
||||
CovarianceData<T, U, Op, compute_marginal_moments>,
|
||||
AggregateFunctionCovariance<T, U, Op, compute_marginal_moments> >
|
||||
{
|
||||
public:
|
||||
String getName() const override { return Op::name; }
|
||||
|
||||
DataTypePtr getReturnType() const override
|
||||
{
|
||||
return new DataTypeFloat64;
|
||||
}
|
||||
|
||||
void setArgumentsImpl(const DataTypes & arguments)
|
||||
{
|
||||
if (!arguments[0]->behavesAsNumber())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument to function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[1]->behavesAsNumber())
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument to function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
void addOne(AggregateDataPtr place, const IColumn & column_left, const IColumn & column_right, size_t row_num) const
|
||||
{
|
||||
this->data(place).update(column_left, column_right, row_num);
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
|
||||
{
|
||||
this->data(place).mergeWith(this->data(rhs));
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
||||
{
|
||||
this->data(place).serialize(buf);
|
||||
}
|
||||
|
||||
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
|
||||
{
|
||||
CovarianceData<T, U, Op, compute_marginal_moments> source;
|
||||
source.deserialize(buf);
|
||||
this->data(place).mergeWith(source);
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
||||
{
|
||||
this->data(place).publish(to);
|
||||
}
|
||||
};
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/** Реализация функции covarSamp.
|
||||
*/
|
||||
struct CovarSampImpl
|
||||
{
|
||||
static constexpr auto name = "covarSamp";
|
||||
|
||||
static inline Float64 apply(Float64 co_moment, UInt64 count)
|
||||
{
|
||||
if (count < 2)
|
||||
return std::numeric_limits<Float64>::infinity();
|
||||
else
|
||||
return co_moment / (count - 1);
|
||||
}
|
||||
};
|
||||
|
||||
/** Реализация функции covarPop.
|
||||
*/
|
||||
struct CovarPopImpl
|
||||
{
|
||||
static constexpr auto name = "covarPop";
|
||||
|
||||
static inline Float64 apply(Float64 co_moment, UInt64 count)
|
||||
{
|
||||
if (count == 0)
|
||||
return std::numeric_limits<Float64>::infinity();
|
||||
else if (count == 1)
|
||||
return 0.0;
|
||||
else
|
||||
return co_moment / count;
|
||||
}
|
||||
};
|
||||
|
||||
/** Реализация функции corr.
|
||||
*/
|
||||
struct CorrImpl
|
||||
{
|
||||
static constexpr auto name = "corr";
|
||||
|
||||
static inline Float64 apply(Float64 co_moment, Float64 left_m2, Float64 right_m2, UInt64 count)
|
||||
{
|
||||
if (count < 2)
|
||||
return std::numeric_limits<Float64>::infinity();
|
||||
else
|
||||
return co_moment / sqrt(left_m2 * right_m2);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
template<typename T>
|
||||
using AggregateFunctionVarSamp = AggregateFunctionVariance<T, VarSampImpl>;
|
||||
|
||||
template<typename T>
|
||||
using AggregateFunctionStdDevSamp = AggregateFunctionVariance<T, StdDevSampImpl>;
|
||||
|
||||
template<typename T>
|
||||
using AggregateFunctionVarPop = AggregateFunctionVariance<T, VarPopImpl>;
|
||||
|
||||
template<typename T>
|
||||
using AggregateFunctionStdDevPop = AggregateFunctionVariance<T, StdDevPopImpl>;
|
||||
|
||||
template<typename T, typename U>
|
||||
using AggregateFunctionCovarSamp = AggregateFunctionCovariance<T, U, CovarSampImpl>;
|
||||
|
||||
template<typename T, typename U>
|
||||
using AggregateFunctionCovarPop = AggregateFunctionCovariance<T, U, CovarPopImpl>;
|
||||
|
||||
template<typename T, typename U>
|
||||
using AggregateFunctionCorr = AggregateFunctionCovariance<T, U, CorrImpl, true>;
|
||||
|
||||
}
|
@ -41,14 +41,14 @@ public:
|
||||
bool isNumeric() const override { return IsNumber<T>::value; }
|
||||
bool isFixed() const override { return IsNumber<T>::value; }
|
||||
size_t sizeOfField() const override { return sizeof(T); }
|
||||
ColumnPtr cloneResized(size_t s_) const override { return new ColumnConst(s_, data); }
|
||||
ColumnPtr cloneResized(size_t s_) const override { return new ColumnConst(s_, data, data_type); }
|
||||
size_t size() const override { return s; }
|
||||
Field operator[](size_t n) const override { return FieldType(data); }
|
||||
void get(size_t n, Field & res) const override { res = FieldType(data); }
|
||||
|
||||
ColumnPtr cut(size_t start, size_t length) const override
|
||||
{
|
||||
return new ColumnConst<T>(length, data, data_type);
|
||||
return new ColumnConst(length, data, data_type);
|
||||
}
|
||||
|
||||
void insert(const Field & x) override
|
||||
@ -79,7 +79,7 @@ public:
|
||||
if (s != filt.size())
|
||||
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
return new ColumnConst<T>(countBytesInFilter(filt), data, data_type);
|
||||
return new ColumnConst(countBytesInFilter(filt), data, data_type);
|
||||
}
|
||||
|
||||
ColumnPtr replicate(const Offsets_t & offsets) const override
|
||||
@ -88,7 +88,7 @@ public:
|
||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
size_t replicated_size = 0 == s ? 0 : offsets.back();
|
||||
return new ColumnConst<T>(replicated_size, data, data_type);
|
||||
return new ColumnConst(replicated_size, data, data_type);
|
||||
}
|
||||
|
||||
size_t byteSize() const override { return sizeof(data) + sizeof(s); }
|
||||
@ -103,7 +103,7 @@ public:
|
||||
if (perm.size() < limit)
|
||||
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
return new ColumnConst<T>(limit, data, data_type);
|
||||
return new ColumnConst(limit, data, data_type);
|
||||
}
|
||||
|
||||
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <unordered_map>
|
||||
#include <list>
|
||||
#include <memory>
|
||||
#include <chrono>
|
||||
#include <Poco/ScopedLock.h>
|
||||
#include <Poco/Mutex.h>
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
@ -20,22 +21,27 @@ struct TrivialWeightFunction
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Кеш, вытесняющий долго не использовавшиеся записи. thread-safe.
|
||||
/** Кеш, вытесняющий долго не использовавшиеся и устаревшие записи. thread-safe.
|
||||
* WeightFunction - тип, оператор () которого принимает Mapped и возвращает "вес" (примерный размер) этого значения.
|
||||
* Кеш начинает выбрасывать значения, когда их суммарный вес превышает max_size.
|
||||
* Кеш начинает выбрасывать значения, когда их суммарный вес превышает max_size и срок годности этих значений истёк.
|
||||
* После вставки значения его вес не должен меняться.
|
||||
*/
|
||||
template <typename TKey, typename TMapped, typename HashFunction = std::hash<TMapped>, typename WeightFunction = TrivialWeightFunction<TMapped> >
|
||||
class LRUCache
|
||||
{
|
||||
public:
|
||||
typedef TKey Key;
|
||||
typedef TMapped Mapped;
|
||||
typedef std::shared_ptr<Mapped> MappedPtr;
|
||||
using Key = TKey;
|
||||
using Mapped = TMapped;
|
||||
using MappedPtr = std::shared_ptr<Mapped>;
|
||||
using Delay = std::chrono::seconds;
|
||||
|
||||
LRUCache(size_t max_size_)
|
||||
: max_size(std::max(1ul, max_size_)) {}
|
||||
private:
|
||||
using Clock = std::chrono::steady_clock;
|
||||
using Timestamp = Clock::time_point;
|
||||
|
||||
public:
|
||||
LRUCache(size_t max_size_, const Delay & expiration_delay_ = Delay::zero())
|
||||
: max_size(std::max(1ul, max_size_)), expiration_delay(expiration_delay_) {}
|
||||
|
||||
MappedPtr get(const Key & key)
|
||||
{
|
||||
@ -50,6 +56,7 @@ public:
|
||||
|
||||
++hits;
|
||||
Cell & cell = it->second;
|
||||
updateCellTimestamp(cell);
|
||||
|
||||
/// Переместим ключ в конец очереди. Итератор остается валидным.
|
||||
queue.splice(queue.end(), queue, cell.queue_iterator);
|
||||
@ -81,8 +88,9 @@ public:
|
||||
cell.value = mapped;
|
||||
cell.size = cell.value ? weight_function(*cell.value) : 0;
|
||||
current_size += cell.size;
|
||||
updateCellTimestamp(cell);
|
||||
|
||||
removeOverflow();
|
||||
removeOverflow(cell.timestamp);
|
||||
}
|
||||
|
||||
void getStats(size_t & out_hits, size_t & out_misses) const
|
||||
@ -120,17 +128,26 @@ protected:
|
||||
/// Суммарный вес выброшенных из кеша элементов.
|
||||
/// Обнуляется каждый раз, когда информация добавляется в Profile events
|
||||
private:
|
||||
typedef std::list<Key> LRUQueue;
|
||||
typedef typename LRUQueue::iterator LRUQueueIterator;
|
||||
using LRUQueue = std::list<Key>;
|
||||
using LRUQueueIterator = typename LRUQueue::iterator;
|
||||
|
||||
struct Cell
|
||||
{
|
||||
public:
|
||||
bool expired(const Timestamp & last_timestamp, const Delay & expiration_delay) const
|
||||
{
|
||||
return (expiration_delay == Delay::zero()) ||
|
||||
((last_timestamp > timestamp) && ((last_timestamp - timestamp) > expiration_delay));
|
||||
}
|
||||
|
||||
public:
|
||||
MappedPtr value;
|
||||
size_t size;
|
||||
LRUQueueIterator queue_iterator;
|
||||
Timestamp timestamp;
|
||||
};
|
||||
|
||||
typedef std::unordered_map<Key, Cell, HashFunction> Cells;
|
||||
using Cells = std::unordered_map<Key, Cell, HashFunction>;
|
||||
|
||||
LRUQueue queue;
|
||||
Cells cells;
|
||||
@ -138,6 +155,7 @@ private:
|
||||
/// Суммарный вес значений.
|
||||
size_t current_size = 0;
|
||||
const size_t max_size;
|
||||
const Delay expiration_delay;
|
||||
|
||||
mutable Poco::FastMutex mutex;
|
||||
size_t hits = 0;
|
||||
@ -145,15 +163,31 @@ private:
|
||||
|
||||
WeightFunction weight_function;
|
||||
|
||||
void removeOverflow()
|
||||
void updateCellTimestamp(Cell & cell)
|
||||
{
|
||||
if (expiration_delay != Delay::zero())
|
||||
cell.timestamp = Clock::now();
|
||||
}
|
||||
|
||||
void removeOverflow(const Timestamp & last_timestamp)
|
||||
{
|
||||
size_t queue_size = cells.size();
|
||||
while (current_size > max_size && queue_size > 1)
|
||||
while ((current_size > max_size) && (queue_size > 1))
|
||||
{
|
||||
const Key & key = queue.front();
|
||||
|
||||
auto it = cells.find(key);
|
||||
current_size -= it->second.size;
|
||||
current_weight_lost += it->second.size;
|
||||
if (it == cells.end())
|
||||
throw Exception("LRUCache became inconsistent. There must be a bug in it. Clearing it for now.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
const auto & cell = it->second;
|
||||
if (!cell.expired(last_timestamp, expiration_delay))
|
||||
break;
|
||||
|
||||
current_size -= cell.size;
|
||||
current_weight_lost += cell.size;
|
||||
|
||||
cells.erase(it);
|
||||
queue.pop_front();
|
||||
--queue_size;
|
||||
|
@ -66,6 +66,9 @@
|
||||
M(DistributedConnectionFailTry) \
|
||||
M(DistributedConnectionFailAtAll) \
|
||||
\
|
||||
M(CompileAttempt) \
|
||||
M(CompileSuccess) \
|
||||
\
|
||||
M(END)
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -9,285 +9,285 @@ namespace ErrorCodes
|
||||
enum ErrorCodes
|
||||
{
|
||||
UNSUPPORTED_METHOD = 1,
|
||||
UNSUPPORTED_PARAMETER,
|
||||
UNEXPECTED_END_OF_FILE,
|
||||
EXPECTED_END_OF_FILE,
|
||||
CANNOT_READ_DATA_FROM_READ_BUFFER,
|
||||
CANNOT_PARSE_TEXT,
|
||||
INCORRECT_NUMBER_OF_COLUMNS,
|
||||
THERE_IS_NO_COLUMN,
|
||||
SIZES_OF_COLUMNS_DOESNT_MATCH,
|
||||
NOT_FOUND_COLUMN_IN_BLOCK,
|
||||
POSITION_OUT_OF_BOUND,
|
||||
PARAMETER_OUT_OF_BOUND,
|
||||
SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH,
|
||||
EMPTY_TUPLE,
|
||||
DUPLICATE_COLUMN,
|
||||
NO_SUCH_COLUMN_IN_TABLE,
|
||||
DELIMITER_IN_STRING_LITERAL_DOESNT_MATCH,
|
||||
CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN,
|
||||
SIZE_OF_ARRAY_DOESNT_MATCH_SIZE_OF_FIXEDARRAY_COLUMN,
|
||||
NUMBER_OF_COLUMNS_DOESNT_MATCH,
|
||||
CANNOT_READ_ALL_DATA_FROM_TAB_SEPARATED_INPUT,
|
||||
CANNOT_PARSE_ALL_VALUE_FROM_TAB_SEPARATED_INPUT,
|
||||
CANNOT_READ_FROM_ISTREAM,
|
||||
CANNOT_WRITE_TO_OSTREAM,
|
||||
CANNOT_PARSE_ESCAPE_SEQUENCE,
|
||||
CANNOT_PARSE_QUOTED_STRING,
|
||||
CANNOT_PARSE_INPUT_ASSERTION_FAILED,
|
||||
CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER,
|
||||
CANNOT_PRINT_INTEGER,
|
||||
CANNOT_READ_SIZE_OF_COMPRESSED_CHUNK,
|
||||
CANNOT_READ_COMPRESSED_CHUNK,
|
||||
ATTEMPT_TO_READ_AFTER_EOF,
|
||||
CANNOT_READ_ALL_DATA,
|
||||
TOO_MUCH_ARGUMENTS_FOR_FUNCTION,
|
||||
TOO_LESS_ARGUMENTS_FOR_FUNCTION,
|
||||
BAD_ARGUMENTS,
|
||||
UNKNOWN_ELEMENT_IN_AST,
|
||||
CANNOT_PARSE_DATE,
|
||||
TOO_LARGE_SIZE_COMPRESSED,
|
||||
CHECKSUM_DOESNT_MATCH,
|
||||
CANNOT_PARSE_DATETIME,
|
||||
NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
ILLEGAL_COLUMN,
|
||||
ILLEGAL_NUMBER_OF_RESULT_COLUMNS,
|
||||
UNKNOWN_FUNCTION,
|
||||
UNKNOWN_IDENTIFIER,
|
||||
NOT_IMPLEMENTED,
|
||||
LOGICAL_ERROR,
|
||||
UNKNOWN_TYPE,
|
||||
EMPTY_LIST_OF_COLUMNS_QUERIED,
|
||||
COLUMN_QUERIED_MORE_THAN_ONCE,
|
||||
TYPE_MISMATCH,
|
||||
STORAGE_DOESNT_ALLOW_PARAMETERS,
|
||||
STORAGE_REQUIRES_PARAMETER,
|
||||
UNKNOWN_STORAGE,
|
||||
TABLE_ALREADY_EXISTS,
|
||||
TABLE_METADATA_ALREADY_EXISTS,
|
||||
ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER,
|
||||
UNKNOWN_TABLE,
|
||||
ONLY_FILTER_COLUMN_IN_BLOCK,
|
||||
SYNTAX_ERROR,
|
||||
UNKNOWN_AGGREGATE_FUNCTION,
|
||||
CANNOT_READ_AGGREGATE_FUNCTION_FROM_TEXT,
|
||||
CANNOT_WRITE_AGGREGATE_FUNCTION_AS_TEXT,
|
||||
NOT_A_COLUMN,
|
||||
ILLEGAL_KEY_OF_AGGREGATION,
|
||||
CANNOT_GET_SIZE_OF_FIELD,
|
||||
ARGUMENT_OUT_OF_BOUND,
|
||||
CANNOT_CONVERT_TYPE,
|
||||
CANNOT_WRITE_AFTER_END_OF_BUFFER,
|
||||
CANNOT_PARSE_NUMBER,
|
||||
UNKNOWN_FORMAT,
|
||||
CANNOT_READ_FROM_FILE_DESCRIPTOR,
|
||||
CANNOT_WRITE_TO_FILE_DESCRIPTOR,
|
||||
CANNOT_OPEN_FILE,
|
||||
CANNOT_CLOSE_FILE,
|
||||
UNKNOWN_TYPE_OF_QUERY,
|
||||
INCORRECT_FILE_NAME,
|
||||
INCORRECT_QUERY,
|
||||
UNKNOWN_DATABASE,
|
||||
DATABASE_ALREADY_EXISTS,
|
||||
DIRECTORY_DOESNT_EXIST,
|
||||
DIRECTORY_ALREADY_EXISTS,
|
||||
FORMAT_IS_NOT_SUITABLE_FOR_INPUT,
|
||||
RECEIVED_ERROR_FROM_REMOTE_IO_SERVER,
|
||||
CANNOT_SEEK_THROUGH_FILE,
|
||||
CANNOT_TRUNCATE_FILE,
|
||||
UNKNOWN_COMPRESSION_METHOD,
|
||||
EMPTY_LIST_OF_COLUMNS_PASSED,
|
||||
SIZES_OF_MARKS_FILES_ARE_INCONSISTENT,
|
||||
EMPTY_DATA_PASSED,
|
||||
UNKNOWN_AGGREGATED_DATA_VARIANT,
|
||||
CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS,
|
||||
CANNOT_READ_FROM_SOCKET,
|
||||
CANNOT_WRITE_TO_SOCKET,
|
||||
CANNOT_READ_ALL_DATA_FROM_CHUNKED_INPUT,
|
||||
CANNOT_WRITE_TO_EMPTY_BLOCK_OUTPUT_STREAM,
|
||||
UNKNOWN_PACKET_FROM_CLIENT,
|
||||
UNKNOWN_PACKET_FROM_SERVER,
|
||||
UNEXPECTED_PACKET_FROM_CLIENT,
|
||||
UNEXPECTED_PACKET_FROM_SERVER,
|
||||
RECEIVED_DATA_FOR_WRONG_QUERY_ID,
|
||||
TOO_SMALL_BUFFER_SIZE,
|
||||
CANNOT_READ_HISTORY,
|
||||
CANNOT_APPEND_HISTORY,
|
||||
FILE_DOESNT_EXIST,
|
||||
NO_DATA_TO_INSERT,
|
||||
CANNOT_BLOCK_SIGNAL,
|
||||
CANNOT_UNBLOCK_SIGNAL,
|
||||
CANNOT_MANIPULATE_SIGSET,
|
||||
CANNOT_WAIT_FOR_SIGNAL,
|
||||
THERE_IS_NO_SESSION,
|
||||
CANNOT_CLOCK_GETTIME,
|
||||
UNKNOWN_SETTING,
|
||||
THERE_IS_NO_DEFAULT_VALUE,
|
||||
INCORRECT_DATA,
|
||||
TABLE_METADATA_DOESNT_EXIST,
|
||||
ENGINE_REQUIRED,
|
||||
CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE,
|
||||
UNKNOWN_SET_DATA_VARIANT,
|
||||
INCOMPATIBLE_COLUMNS,
|
||||
UNKNOWN_TYPE_OF_AST_NODE,
|
||||
INCORRECT_ELEMENT_OF_SET,
|
||||
INCORRECT_RESULT_OF_SCALAR_SUBQUERY,
|
||||
CANNOT_GET_RETURN_TYPE,
|
||||
ILLEGAL_INDEX,
|
||||
TOO_LARGE_ARRAY_SIZE,
|
||||
FUNCTION_IS_SPECIAL,
|
||||
CANNOT_READ_ARRAY_FROM_TEXT,
|
||||
TOO_LARGE_STRING_SIZE,
|
||||
CANNOT_CREATE_TABLE_FROM_METADATA,
|
||||
AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS,
|
||||
PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS,
|
||||
ZERO_ARRAY_OR_TUPLE_INDEX,
|
||||
ALL_CONNECTION_TRIES_FAILED,
|
||||
UNKNOWN_ELEMENT_IN_CONFIG,
|
||||
EXCESSIVE_ELEMENT_IN_CONFIG,
|
||||
NO_ELEMENTS_IN_CONFIG,
|
||||
ALL_REQUESTED_COLUMNS_ARE_MISSING,
|
||||
SAMPLING_NOT_SUPPORTED,
|
||||
NOT_FOUND_NODE,
|
||||
FOUND_MORE_THAN_ONE_NODE,
|
||||
FIRST_DATE_IS_BIGGER_THAN_LAST_DATE,
|
||||
UNKNOWN_OVERFLOW_MODE,
|
||||
QUERY_SECTION_DOESNT_MAKE_SENSE,
|
||||
NOT_FOUND_FUNCTION_ELEMENT_FOR_AGGREGATE,
|
||||
NOT_FOUND_RELATION_ELEMENT_FOR_CONDITION,
|
||||
NOT_FOUND_RHS_ELEMENT_FOR_CONDITION,
|
||||
NO_ATTRIBUTES_LISTED,
|
||||
INDEX_OF_COLUMN_IN_SORT_CLAUSE_IS_OUT_OF_RANGE,
|
||||
UNKNOWN_DIRECTION_OF_SORTING,
|
||||
ILLEGAL_DIVISION,
|
||||
AGGREGATE_FUNCTION_NOT_APPLICABLE,
|
||||
UNKNOWN_RELATION,
|
||||
DICTIONARIES_WAS_NOT_LOADED,
|
||||
ILLEGAL_OVERFLOW_MODE,
|
||||
TOO_MUCH_ROWS,
|
||||
TIMEOUT_EXCEEDED,
|
||||
TOO_SLOW,
|
||||
TOO_MUCH_COLUMNS,
|
||||
TOO_DEEP_SUBQUERIES,
|
||||
TOO_DEEP_PIPELINE,
|
||||
READONLY,
|
||||
TOO_MUCH_TEMPORARY_COLUMNS,
|
||||
TOO_MUCH_TEMPORARY_NON_CONST_COLUMNS,
|
||||
TOO_DEEP_AST,
|
||||
TOO_BIG_AST,
|
||||
BAD_TYPE_OF_FIELD,
|
||||
BAD_GET,
|
||||
BLOCKS_HAS_DIFFERENT_STRUCTURE,
|
||||
CANNOT_CREATE_DIRECTORY,
|
||||
CANNOT_ALLOCATE_MEMORY,
|
||||
CYCLIC_ALIASES,
|
||||
NEGATIVE_REFCOUNT,
|
||||
CHUNK_NOT_FOUND,
|
||||
DUPLICATE_CHUNK_NAME,
|
||||
MULTIPLE_ALIASES_FOR_EXPRESSION,
|
||||
MULTIPLE_EXPRESSIONS_FOR_ALIAS,
|
||||
THERE_IS_NO_PROFILE,
|
||||
ILLEGAL_FINAL,
|
||||
ILLEGAL_PREWHERE,
|
||||
UNEXPECTED_EXPRESSION,
|
||||
ILLEGAL_AGGREGATION,
|
||||
UNSUPPORTED_MYISAM_BLOCK_TYPE,
|
||||
UNSUPPORTED_COLLATION_LOCALE,
|
||||
COLLATION_COMPARISON_FAILED,
|
||||
UNKNOWN_ACTION,
|
||||
TABLE_MUST_NOT_BE_CREATED_MANUALLY,
|
||||
SIZES_OF_ARRAYS_DOESNT_MATCH,
|
||||
SET_SIZE_LIMIT_EXCEEDED,
|
||||
UNKNOWN_USER,
|
||||
WRONG_PASSWORD,
|
||||
REQUIRED_PASSWORD,
|
||||
IP_ADDRESS_NOT_ALLOWED,
|
||||
UNKNOWN_ADDRESS_PATTERN_TYPE,
|
||||
SERVER_REVISION_IS_TOO_OLD,
|
||||
DNS_ERROR,
|
||||
UNKNOWN_QUOTA,
|
||||
QUOTA_DOESNT_ALLOW_KEYS,
|
||||
QUOTA_EXPIRED,
|
||||
TOO_MUCH_SIMULTANEOUS_QUERIES,
|
||||
NO_FREE_CONNECTION,
|
||||
CANNOT_FSYNC,
|
||||
NESTED_TYPE_TOO_DEEP,
|
||||
ALIAS_REQUIRED,
|
||||
AMBIGUOUS_IDENTIFIER,
|
||||
EMPTY_NESTED_TABLE,
|
||||
SOCKET_TIMEOUT,
|
||||
NETWORK_ERROR,
|
||||
EMPTY_QUERY,
|
||||
UNKNOWN_LOAD_BALANCING,
|
||||
UNKNOWN_TOTALS_MODE,
|
||||
CANNOT_STATVFS,
|
||||
NOT_AN_AGGREGATE,
|
||||
QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING,
|
||||
CLIENT_HAS_CONNECTED_TO_WRONG_PORT,
|
||||
TABLE_IS_DROPPED,
|
||||
DATABASE_NOT_EMPTY,
|
||||
DUPLICATE_INTERSERVER_IO_ENDPOINT,
|
||||
NO_SUCH_INTERSERVER_IO_ENDPOINT,
|
||||
ADDING_REPLICA_TO_NON_EMPTY_TABLE,
|
||||
UNEXPECTED_AST_STRUCTURE,
|
||||
REPLICA_IS_ALREADY_ACTIVE,
|
||||
NO_ZOOKEEPER,
|
||||
NO_FILE_IN_DATA_PART,
|
||||
UNEXPECTED_FILE_IN_DATA_PART,
|
||||
BAD_SIZE_OF_FILE_IN_DATA_PART,
|
||||
QUERY_IS_TOO_LARGE,
|
||||
NOT_FOUND_EXPECTED_DATA_PART,
|
||||
TOO_MANY_UNEXPECTED_DATA_PARTS,
|
||||
NO_SUCH_DATA_PART,
|
||||
BAD_DATA_PART_NAME,
|
||||
NO_REPLICA_HAS_PART,
|
||||
DUPLICATE_DATA_PART,
|
||||
ABORTED,
|
||||
NO_REPLICA_NAME_GIVEN,
|
||||
FORMAT_VERSION_TOO_OLD,
|
||||
CANNOT_MUNMAP,
|
||||
CANNOT_MREMAP,
|
||||
MEMORY_LIMIT_EXCEEDED,
|
||||
TABLE_IS_READ_ONLY,
|
||||
NOT_ENOUGH_SPACE,
|
||||
UNEXPECTED_ZOOKEEPER_ERROR,
|
||||
INVALID_NESTED_NAME,
|
||||
CORRUPTED_DATA,
|
||||
INCORRECT_MARK,
|
||||
INVALID_PARTITION_NAME,
|
||||
NOT_LEADER,
|
||||
NOT_ENOUGH_BLOCK_NUMBERS,
|
||||
NO_SUCH_REPLICA,
|
||||
TOO_MUCH_PARTS,
|
||||
REPLICA_IS_ALREADY_EXIST,
|
||||
NO_ACTIVE_REPLICAS,
|
||||
TOO_MUCH_RETRIES_TO_FETCH_PARTS,
|
||||
PARTITION_ALREADY_EXISTS,
|
||||
PARTITION_DOESNT_EXIST,
|
||||
UNION_ALL_RESULT_STRUCTURES_MISMATCH,
|
||||
UNION_ALL_COLUMN_ALIAS_MISMATCH,
|
||||
CLIENT_OUTPUT_FORMAT_SPECIFIED,
|
||||
UNKNOWN_BLOCK_INFO_FIELD,
|
||||
BAD_COLLATION,
|
||||
CANNOT_COMPILE_CODE,
|
||||
INCOMPATIBLE_TYPE_OF_JOIN,
|
||||
NO_AVAILABLE_REPLICA,
|
||||
MISMATCH_REPLICAS_DATA_SOURCES,
|
||||
STORAGE_DOESNT_SUPPORT_PARALLEL_REPLICAS,
|
||||
CPUID_ERROR,
|
||||
INFINITE_LOOP,
|
||||
CANNOT_COMPRESS,
|
||||
CANNOT_DECOMPRESS,
|
||||
AIO_SUBMIT_ERROR,
|
||||
AIO_COMPLETION_ERROR,
|
||||
AIO_READ_ERROR,
|
||||
AIO_WRITE_ERROR,
|
||||
INDEX_NOT_USED,
|
||||
UNSUPPORTED_PARAMETER = 2,
|
||||
UNEXPECTED_END_OF_FILE = 3,
|
||||
EXPECTED_END_OF_FILE = 4,
|
||||
CANNOT_PARSE_TEXT = 6,
|
||||
INCORRECT_NUMBER_OF_COLUMNS = 7,
|
||||
THERE_IS_NO_COLUMN = 8,
|
||||
SIZES_OF_COLUMNS_DOESNT_MATCH = 9,
|
||||
NOT_FOUND_COLUMN_IN_BLOCK = 10,
|
||||
POSITION_OUT_OF_BOUND = 11,
|
||||
PARAMETER_OUT_OF_BOUND = 12,
|
||||
SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH = 13,
|
||||
EMPTY_TUPLE = 14,
|
||||
DUPLICATE_COLUMN = 15,
|
||||
NO_SUCH_COLUMN_IN_TABLE = 16,
|
||||
DELIMITER_IN_STRING_LITERAL_DOESNT_MATCH = 17,
|
||||
CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN = 18,
|
||||
SIZE_OF_ARRAY_DOESNT_MATCH_SIZE_OF_FIXEDARRAY_COLUMN = 19,
|
||||
NUMBER_OF_COLUMNS_DOESNT_MATCH = 20,
|
||||
CANNOT_READ_ALL_DATA_FROM_TAB_SEPARATED_INPUT = 21,
|
||||
CANNOT_PARSE_ALL_VALUE_FROM_TAB_SEPARATED_INPUT = 22,
|
||||
CANNOT_READ_FROM_ISTREAM = 23,
|
||||
CANNOT_WRITE_TO_OSTREAM = 24,
|
||||
CANNOT_PARSE_ESCAPE_SEQUENCE = 25,
|
||||
CANNOT_PARSE_QUOTED_STRING = 26,
|
||||
CANNOT_PARSE_INPUT_ASSERTION_FAILED = 27,
|
||||
CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER = 28,
|
||||
CANNOT_PRINT_INTEGER = 29,
|
||||
CANNOT_READ_SIZE_OF_COMPRESSED_CHUNK = 30,
|
||||
CANNOT_READ_COMPRESSED_CHUNK = 31,
|
||||
ATTEMPT_TO_READ_AFTER_EOF = 32,
|
||||
CANNOT_READ_ALL_DATA = 33,
|
||||
TOO_MUCH_ARGUMENTS_FOR_FUNCTION = 34,
|
||||
TOO_LESS_ARGUMENTS_FOR_FUNCTION = 35,
|
||||
BAD_ARGUMENTS = 36,
|
||||
UNKNOWN_ELEMENT_IN_AST = 37,
|
||||
CANNOT_PARSE_DATE = 38,
|
||||
TOO_LARGE_SIZE_COMPRESSED = 39,
|
||||
CHECKSUM_DOESNT_MATCH = 40,
|
||||
CANNOT_PARSE_DATETIME = 41,
|
||||
NUMBER_OF_ARGUMENTS_DOESNT_MATCH = 42,
|
||||
ILLEGAL_TYPE_OF_ARGUMENT = 43,
|
||||
ILLEGAL_COLUMN = 44,
|
||||
ILLEGAL_NUMBER_OF_RESULT_COLUMNS = 45,
|
||||
UNKNOWN_FUNCTION = 46,
|
||||
UNKNOWN_IDENTIFIER = 47,
|
||||
NOT_IMPLEMENTED = 48,
|
||||
LOGICAL_ERROR = 49,
|
||||
UNKNOWN_TYPE = 50,
|
||||
EMPTY_LIST_OF_COLUMNS_QUERIED = 51,
|
||||
COLUMN_QUERIED_MORE_THAN_ONCE = 52,
|
||||
TYPE_MISMATCH = 53,
|
||||
STORAGE_DOESNT_ALLOW_PARAMETERS = 54,
|
||||
STORAGE_REQUIRES_PARAMETER = 55,
|
||||
UNKNOWN_STORAGE = 56,
|
||||
TABLE_ALREADY_EXISTS = 57,
|
||||
TABLE_METADATA_ALREADY_EXISTS = 58,
|
||||
ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER = 59,
|
||||
UNKNOWN_TABLE = 60,
|
||||
ONLY_FILTER_COLUMN_IN_BLOCK = 61,
|
||||
SYNTAX_ERROR = 62,
|
||||
UNKNOWN_AGGREGATE_FUNCTION = 63,
|
||||
CANNOT_READ_AGGREGATE_FUNCTION_FROM_TEXT = 64,
|
||||
CANNOT_WRITE_AGGREGATE_FUNCTION_AS_TEXT = 65,
|
||||
NOT_A_COLUMN = 66,
|
||||
ILLEGAL_KEY_OF_AGGREGATION = 67,
|
||||
CANNOT_GET_SIZE_OF_FIELD = 68,
|
||||
ARGUMENT_OUT_OF_BOUND = 69,
|
||||
CANNOT_CONVERT_TYPE = 70,
|
||||
CANNOT_WRITE_AFTER_END_OF_BUFFER = 71,
|
||||
CANNOT_PARSE_NUMBER = 72,
|
||||
UNKNOWN_FORMAT = 73,
|
||||
CANNOT_READ_FROM_FILE_DESCRIPTOR = 74,
|
||||
CANNOT_WRITE_TO_FILE_DESCRIPTOR = 75,
|
||||
CANNOT_OPEN_FILE = 76,
|
||||
CANNOT_CLOSE_FILE = 77,
|
||||
UNKNOWN_TYPE_OF_QUERY = 78,
|
||||
INCORRECT_FILE_NAME = 79,
|
||||
INCORRECT_QUERY = 80,
|
||||
UNKNOWN_DATABASE = 81,
|
||||
DATABASE_ALREADY_EXISTS = 82,
|
||||
DIRECTORY_DOESNT_EXIST = 83,
|
||||
DIRECTORY_ALREADY_EXISTS = 84,
|
||||
FORMAT_IS_NOT_SUITABLE_FOR_INPUT = 85,
|
||||
RECEIVED_ERROR_FROM_REMOTE_IO_SERVER = 86,
|
||||
CANNOT_SEEK_THROUGH_FILE = 87,
|
||||
CANNOT_TRUNCATE_FILE = 88,
|
||||
UNKNOWN_COMPRESSION_METHOD = 89,
|
||||
EMPTY_LIST_OF_COLUMNS_PASSED = 90,
|
||||
SIZES_OF_MARKS_FILES_ARE_INCONSISTENT = 91,
|
||||
EMPTY_DATA_PASSED = 92,
|
||||
UNKNOWN_AGGREGATED_DATA_VARIANT = 93,
|
||||
CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS = 94,
|
||||
CANNOT_READ_FROM_SOCKET = 95,
|
||||
CANNOT_WRITE_TO_SOCKET = 96,
|
||||
CANNOT_READ_ALL_DATA_FROM_CHUNKED_INPUT = 97,
|
||||
CANNOT_WRITE_TO_EMPTY_BLOCK_OUTPUT_STREAM = 98,
|
||||
UNKNOWN_PACKET_FROM_CLIENT = 99,
|
||||
UNKNOWN_PACKET_FROM_SERVER = 100,
|
||||
UNEXPECTED_PACKET_FROM_CLIENT = 101,
|
||||
UNEXPECTED_PACKET_FROM_SERVER = 102,
|
||||
RECEIVED_DATA_FOR_WRONG_QUERY_ID = 103,
|
||||
TOO_SMALL_BUFFER_SIZE = 104,
|
||||
CANNOT_READ_HISTORY = 105,
|
||||
CANNOT_APPEND_HISTORY = 106,
|
||||
FILE_DOESNT_EXIST = 107,
|
||||
NO_DATA_TO_INSERT = 108,
|
||||
CANNOT_BLOCK_SIGNAL = 109,
|
||||
CANNOT_UNBLOCK_SIGNAL = 110,
|
||||
CANNOT_MANIPULATE_SIGSET = 111,
|
||||
CANNOT_WAIT_FOR_SIGNAL = 112,
|
||||
THERE_IS_NO_SESSION = 113,
|
||||
CANNOT_CLOCK_GETTIME = 114,
|
||||
UNKNOWN_SETTING = 115,
|
||||
THERE_IS_NO_DEFAULT_VALUE = 116,
|
||||
INCORRECT_DATA = 117,
|
||||
TABLE_METADATA_DOESNT_EXIST = 118,
|
||||
ENGINE_REQUIRED = 119,
|
||||
CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE = 120,
|
||||
UNKNOWN_SET_DATA_VARIANT = 121,
|
||||
INCOMPATIBLE_COLUMNS = 122,
|
||||
UNKNOWN_TYPE_OF_AST_NODE = 123,
|
||||
INCORRECT_ELEMENT_OF_SET = 124,
|
||||
INCORRECT_RESULT_OF_SCALAR_SUBQUERY = 125,
|
||||
CANNOT_GET_RETURN_TYPE = 126,
|
||||
ILLEGAL_INDEX = 127,
|
||||
TOO_LARGE_ARRAY_SIZE = 128,
|
||||
FUNCTION_IS_SPECIAL = 129,
|
||||
CANNOT_READ_ARRAY_FROM_TEXT = 130,
|
||||
TOO_LARGE_STRING_SIZE = 131,
|
||||
CANNOT_CREATE_TABLE_FROM_METADATA = 132,
|
||||
AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS = 133,
|
||||
PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS = 134,
|
||||
ZERO_ARRAY_OR_TUPLE_INDEX = 135,
|
||||
UNKNOWN_ELEMENT_IN_CONFIG = 137,
|
||||
EXCESSIVE_ELEMENT_IN_CONFIG = 138,
|
||||
NO_ELEMENTS_IN_CONFIG = 139,
|
||||
ALL_REQUESTED_COLUMNS_ARE_MISSING = 140,
|
||||
SAMPLING_NOT_SUPPORTED = 141,
|
||||
NOT_FOUND_NODE = 142,
|
||||
FOUND_MORE_THAN_ONE_NODE = 143,
|
||||
FIRST_DATE_IS_BIGGER_THAN_LAST_DATE = 144,
|
||||
UNKNOWN_OVERFLOW_MODE = 145,
|
||||
QUERY_SECTION_DOESNT_MAKE_SENSE = 146,
|
||||
NOT_FOUND_FUNCTION_ELEMENT_FOR_AGGREGATE = 147,
|
||||
NOT_FOUND_RELATION_ELEMENT_FOR_CONDITION = 148,
|
||||
NOT_FOUND_RHS_ELEMENT_FOR_CONDITION = 149,
|
||||
NO_ATTRIBUTES_LISTED = 150,
|
||||
INDEX_OF_COLUMN_IN_SORT_CLAUSE_IS_OUT_OF_RANGE = 151,
|
||||
UNKNOWN_DIRECTION_OF_SORTING = 152,
|
||||
ILLEGAL_DIVISION = 153,
|
||||
AGGREGATE_FUNCTION_NOT_APPLICABLE = 154,
|
||||
UNKNOWN_RELATION = 155,
|
||||
DICTIONARIES_WAS_NOT_LOADED = 156,
|
||||
ILLEGAL_OVERFLOW_MODE = 157,
|
||||
TOO_MUCH_ROWS = 158,
|
||||
TIMEOUT_EXCEEDED = 159,
|
||||
TOO_SLOW = 160,
|
||||
TOO_MUCH_COLUMNS = 161,
|
||||
TOO_DEEP_SUBQUERIES = 162,
|
||||
TOO_DEEP_PIPELINE = 163,
|
||||
READONLY = 164,
|
||||
TOO_MUCH_TEMPORARY_COLUMNS = 165,
|
||||
TOO_MUCH_TEMPORARY_NON_CONST_COLUMNS = 166,
|
||||
TOO_DEEP_AST = 167,
|
||||
TOO_BIG_AST = 168,
|
||||
BAD_TYPE_OF_FIELD = 169,
|
||||
BAD_GET = 170,
|
||||
BLOCKS_HAS_DIFFERENT_STRUCTURE = 171,
|
||||
CANNOT_CREATE_DIRECTORY = 172,
|
||||
CANNOT_ALLOCATE_MEMORY = 173,
|
||||
CYCLIC_ALIASES = 174,
|
||||
NEGATIVE_REFCOUNT = 175,
|
||||
CHUNK_NOT_FOUND = 176,
|
||||
DUPLICATE_CHUNK_NAME = 177,
|
||||
MULTIPLE_ALIASES_FOR_EXPRESSION = 178,
|
||||
MULTIPLE_EXPRESSIONS_FOR_ALIAS = 179,
|
||||
THERE_IS_NO_PROFILE = 180,
|
||||
ILLEGAL_FINAL = 181,
|
||||
ILLEGAL_PREWHERE = 182,
|
||||
UNEXPECTED_EXPRESSION = 183,
|
||||
ILLEGAL_AGGREGATION = 184,
|
||||
UNSUPPORTED_MYISAM_BLOCK_TYPE = 185,
|
||||
UNSUPPORTED_COLLATION_LOCALE = 186,
|
||||
COLLATION_COMPARISON_FAILED = 187,
|
||||
UNKNOWN_ACTION = 188,
|
||||
TABLE_MUST_NOT_BE_CREATED_MANUALLY = 189,
|
||||
SIZES_OF_ARRAYS_DOESNT_MATCH = 190,
|
||||
SET_SIZE_LIMIT_EXCEEDED = 191,
|
||||
UNKNOWN_USER = 192,
|
||||
WRONG_PASSWORD = 193,
|
||||
REQUIRED_PASSWORD = 194,
|
||||
IP_ADDRESS_NOT_ALLOWED = 195,
|
||||
UNKNOWN_ADDRESS_PATTERN_TYPE = 196,
|
||||
SERVER_REVISION_IS_TOO_OLD = 197,
|
||||
DNS_ERROR = 198,
|
||||
UNKNOWN_QUOTA = 199,
|
||||
QUOTA_DOESNT_ALLOW_KEYS = 200,
|
||||
QUOTA_EXPIRED = 201,
|
||||
TOO_MUCH_SIMULTANEOUS_QUERIES = 202,
|
||||
NO_FREE_CONNECTION = 203,
|
||||
CANNOT_FSYNC = 204,
|
||||
NESTED_TYPE_TOO_DEEP = 205,
|
||||
ALIAS_REQUIRED = 206,
|
||||
AMBIGUOUS_IDENTIFIER = 207,
|
||||
EMPTY_NESTED_TABLE = 208,
|
||||
SOCKET_TIMEOUT = 209,
|
||||
NETWORK_ERROR = 210,
|
||||
EMPTY_QUERY = 211,
|
||||
UNKNOWN_LOAD_BALANCING = 212,
|
||||
UNKNOWN_TOTALS_MODE = 213,
|
||||
CANNOT_STATVFS = 214,
|
||||
NOT_AN_AGGREGATE = 215,
|
||||
QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING = 216,
|
||||
CLIENT_HAS_CONNECTED_TO_WRONG_PORT = 217,
|
||||
TABLE_IS_DROPPED = 218,
|
||||
DATABASE_NOT_EMPTY = 219,
|
||||
DUPLICATE_INTERSERVER_IO_ENDPOINT = 220,
|
||||
NO_SUCH_INTERSERVER_IO_ENDPOINT = 221,
|
||||
ADDING_REPLICA_TO_NON_EMPTY_TABLE = 222,
|
||||
UNEXPECTED_AST_STRUCTURE = 223,
|
||||
REPLICA_IS_ALREADY_ACTIVE = 224,
|
||||
NO_ZOOKEEPER = 225,
|
||||
NO_FILE_IN_DATA_PART = 226,
|
||||
UNEXPECTED_FILE_IN_DATA_PART = 227,
|
||||
BAD_SIZE_OF_FILE_IN_DATA_PART = 228,
|
||||
QUERY_IS_TOO_LARGE = 229,
|
||||
NOT_FOUND_EXPECTED_DATA_PART = 230,
|
||||
TOO_MANY_UNEXPECTED_DATA_PARTS = 231,
|
||||
NO_SUCH_DATA_PART = 232,
|
||||
BAD_DATA_PART_NAME = 233,
|
||||
NO_REPLICA_HAS_PART = 234,
|
||||
DUPLICATE_DATA_PART = 235,
|
||||
ABORTED = 236,
|
||||
NO_REPLICA_NAME_GIVEN = 237,
|
||||
FORMAT_VERSION_TOO_OLD = 238,
|
||||
CANNOT_MUNMAP = 239,
|
||||
CANNOT_MREMAP = 240,
|
||||
MEMORY_LIMIT_EXCEEDED = 241,
|
||||
TABLE_IS_READ_ONLY = 242,
|
||||
NOT_ENOUGH_SPACE = 243,
|
||||
UNEXPECTED_ZOOKEEPER_ERROR = 244,
|
||||
INVALID_NESTED_NAME = 245,
|
||||
CORRUPTED_DATA = 246,
|
||||
INCORRECT_MARK = 247,
|
||||
INVALID_PARTITION_NAME = 248,
|
||||
NOT_LEADER = 249,
|
||||
NOT_ENOUGH_BLOCK_NUMBERS = 250,
|
||||
NO_SUCH_REPLICA = 251,
|
||||
TOO_MUCH_PARTS = 252,
|
||||
REPLICA_IS_ALREADY_EXIST = 253,
|
||||
NO_ACTIVE_REPLICAS = 254,
|
||||
TOO_MUCH_RETRIES_TO_FETCH_PARTS = 255,
|
||||
PARTITION_ALREADY_EXISTS = 256,
|
||||
PARTITION_DOESNT_EXIST = 257,
|
||||
UNION_ALL_RESULT_STRUCTURES_MISMATCH = 258,
|
||||
UNION_ALL_COLUMN_ALIAS_MISMATCH = 259,
|
||||
CLIENT_OUTPUT_FORMAT_SPECIFIED = 260,
|
||||
UNKNOWN_BLOCK_INFO_FIELD = 261,
|
||||
BAD_COLLATION = 262,
|
||||
CANNOT_COMPILE_CODE = 263,
|
||||
INCOMPATIBLE_TYPE_OF_JOIN = 264,
|
||||
NO_AVAILABLE_REPLICA = 265,
|
||||
MISMATCH_REPLICAS_DATA_SOURCES = 266,
|
||||
STORAGE_DOESNT_SUPPORT_PARALLEL_REPLICAS = 267,
|
||||
CPUID_ERROR = 268,
|
||||
INFINITE_LOOP = 269,
|
||||
CANNOT_COMPRESS = 270,
|
||||
CANNOT_DECOMPRESS = 271,
|
||||
AIO_SUBMIT_ERROR = 272,
|
||||
AIO_COMPLETION_ERROR = 273,
|
||||
AIO_READ_ERROR = 274,
|
||||
AIO_WRITE_ERROR = 275,
|
||||
INDEX_NOT_USED = 277,
|
||||
LEADERSHIP_LOST = 278,
|
||||
ALL_CONNECTION_TRIES_FAILED = 279,
|
||||
|
||||
POCO_EXCEPTION = 1000,
|
||||
STD_EXCEPTION,
|
||||
UNKNOWN_EXCEPTION,
|
||||
STD_EXCEPTION = 1001,
|
||||
UNKNOWN_EXCEPTION = 1002,
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -50,7 +50,7 @@ protected:
|
||||
Block readImpl() override;
|
||||
|
||||
private:
|
||||
Logger * log = &Logger::get("SummingSortedBlockInputStream");
|
||||
Logger * log = &Logger::get("AggregatingSortedBlockInputStream");
|
||||
|
||||
/// Прочитали до конца.
|
||||
bool finished = false;
|
||||
@ -68,7 +68,7 @@ private:
|
||||
* Шаблоны используем вместо полиморфных SortCursor'ов и вызовов виртуальных функций.
|
||||
*/
|
||||
template<class TSortCursor>
|
||||
void merge(Block & merged_block, ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
|
||||
void merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
|
||||
|
||||
/// Вставить в результат первую строку для текущей группы.
|
||||
void insertCurrentRow(ColumnPlainPtrs & merged_columns);
|
||||
|
@ -81,7 +81,7 @@ private:
|
||||
* Шаблоны используем вместо полиморфных SortCursor'ов и вызовов виртуальных функций.
|
||||
*/
|
||||
template<class TSortCursor>
|
||||
void merge(Block & merged_block, ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
|
||||
void merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
|
||||
|
||||
/// Вставить в результат строки для текущего идентификатора "визита".
|
||||
void insertRows(ColumnPlainPtrs & merged_columns, size_t & merged_rows, bool last_in_stream = false);
|
||||
|
@ -15,7 +15,7 @@ using Poco::SharedPtr;
|
||||
/** Выполняет над блоком вычисление некоторого выражения.
|
||||
* Выражение состоит из идентификаторов столбцов из блока, констант, обычных функций.
|
||||
* Например: hits * 2 + 3, url LIKE '%yandex%'
|
||||
* Выражение не меняет количество строк в потоке, и обрабатывает каждую строку независимо от других.
|
||||
* Выражение обрабатывает каждую строку независимо от других.
|
||||
*/
|
||||
class ExpressionBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
|
@ -136,7 +136,7 @@ private:
|
||||
void initQueue(std::priority_queue<TSortCursor> & queue);
|
||||
|
||||
template <typename TSortCursor>
|
||||
void merge(Block & merged_block, ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
|
||||
void merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
|
||||
|
||||
Logger * log = &Logger::get("MergingSortedBlockInputStream");
|
||||
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <DB/Client/Connection.h>
|
||||
#include <Yandex/logger_useful.h>
|
||||
|
||||
#include <statdaemons/NetException.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -50,7 +51,7 @@ public:
|
||||
return;
|
||||
}
|
||||
else
|
||||
throw Exception("Unexpected packet from server (expected Data or Exception, got "
|
||||
throw NetException("Unexpected packet from server (expected Data or Exception, got "
|
||||
+ String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
|
||||
}
|
||||
|
||||
@ -97,7 +98,7 @@ public:
|
||||
else if (Protocol::Server::Exception == packet.type)
|
||||
packet.exception->rethrow();
|
||||
else
|
||||
throw Exception("Unexpected packet from server (expected EndOfStream or Exception, got "
|
||||
throw NetException("Unexpected packet from server (expected EndOfStream or Exception, got "
|
||||
+ String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
|
||||
}
|
||||
|
||||
|
@ -99,7 +99,7 @@ private:
|
||||
* Шаблоны используем вместо полиморфных SortCursor'ов и вызовов виртуальных функций.
|
||||
*/
|
||||
template<class TSortCursor>
|
||||
void merge(Block & merged_block, ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
|
||||
void merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
|
||||
|
||||
/// Вставить в результат просуммированную строку для текущей группы.
|
||||
void insertCurrentRow(ColumnPlainPtrs & merged_columns);
|
||||
|
@ -48,6 +48,8 @@ public:
|
||||
|
||||
std::size_t getBytesAllocated() const override { return bytes_allocated; }
|
||||
|
||||
std::size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
|
||||
|
||||
double getHitRate() const override
|
||||
{
|
||||
return static_cast<double>(hit_count.load(std::memory_order_acquire)) /
|
||||
@ -76,6 +78,11 @@ public:
|
||||
return creation_time;
|
||||
}
|
||||
|
||||
bool isInjective(const std::string & attribute_name) const override
|
||||
{
|
||||
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
|
||||
}
|
||||
|
||||
bool hasHierarchy() const override { return hierarchical_attribute; }
|
||||
|
||||
id_t toParent(const id_t id) const override
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/Common/Arena.h>
|
||||
#include <statdaemons/ext/range.hpp>
|
||||
#include <atomic>
|
||||
#include <vector>
|
||||
#include <tuple>
|
||||
|
||||
@ -39,6 +40,8 @@ public:
|
||||
|
||||
std::size_t getBytesAllocated() const override { return bytes_allocated; }
|
||||
|
||||
std::size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
|
||||
|
||||
double getHitRate() const override { return 1.0; }
|
||||
|
||||
std::size_t getElementCount() const override { return element_count; }
|
||||
@ -60,6 +63,11 @@ public:
|
||||
return creation_time;
|
||||
}
|
||||
|
||||
bool isInjective(const std::string & attribute_name) const override
|
||||
{
|
||||
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
|
||||
}
|
||||
|
||||
bool hasHierarchy() const override { return hierarchical_attribute; }
|
||||
|
||||
id_t toParent(const id_t id) const override
|
||||
@ -67,6 +75,8 @@ public:
|
||||
const auto attr = hierarchical_attribute;
|
||||
const auto & array = *std::get<std::unique_ptr<PODArray<UInt64>>>(attr->arrays);
|
||||
|
||||
query_count.fetch_add(1, std::memory_order_relaxed);
|
||||
|
||||
return id < array.size() ? array[id] : std::get<UInt64>(attr->null_values);
|
||||
}
|
||||
|
||||
@ -87,6 +97,8 @@ public:
|
||||
\
|
||||
const auto & array = *std::get<std::unique_ptr<PODArray<TYPE>>>(attribute.arrays);\
|
||||
\
|
||||
query_count.fetch_add(1, std::memory_order_relaxed);\
|
||||
\
|
||||
return id < array.size() ? array[id] : std::get<TYPE>(attribute.null_values);\
|
||||
}
|
||||
DECLARE_INDIVIDUAL_GETTER(UInt8)
|
||||
@ -111,6 +123,8 @@ public:
|
||||
|
||||
const auto & array = *std::get<std::unique_ptr<PODArray<StringRef>>>(attribute.arrays);
|
||||
|
||||
query_count.fetch_add(1, std::memory_order_relaxed);
|
||||
|
||||
return id < array.size() ? String{array[id]} : std::get<String>(attribute.null_values);
|
||||
}
|
||||
|
||||
@ -155,6 +169,8 @@ public:
|
||||
const auto string_ref = id < attr.size() ? attr[id] : StringRef{null_value};
|
||||
out->insertData(string_ref.data, string_ref.size);
|
||||
}
|
||||
|
||||
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -312,6 +328,8 @@ private:
|
||||
const auto id = ids[i];
|
||||
out[i] = id < attr.size() ? attr[id] : null_value;
|
||||
}
|
||||
|
||||
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
@ -382,6 +400,8 @@ private:
|
||||
std::size_t bucket_count = 0;
|
||||
|
||||
std::chrono::time_point<std::chrono::system_clock> creation_time;
|
||||
|
||||
mutable std::atomic<std::size_t> query_count;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <DB/Common/HashTable/HashMap.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <statdaemons/ext/range.hpp>
|
||||
#include <atomic>
|
||||
#include <memory>
|
||||
#include <tuple>
|
||||
|
||||
@ -36,6 +37,8 @@ public:
|
||||
|
||||
std::size_t getBytesAllocated() const override { return bytes_allocated; }
|
||||
|
||||
std::size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
|
||||
|
||||
double getHitRate() const override { return 1.0; }
|
||||
|
||||
std::size_t getElementCount() const override { return element_count; }
|
||||
@ -57,6 +60,11 @@ public:
|
||||
return creation_time;
|
||||
}
|
||||
|
||||
bool isInjective(const std::string & attribute_name) const override
|
||||
{
|
||||
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
|
||||
}
|
||||
|
||||
bool hasHierarchy() const override { return hierarchical_attribute; }
|
||||
|
||||
id_t toParent(const id_t id) const override
|
||||
@ -65,6 +73,8 @@ public:
|
||||
const auto & map = *std::get<std::unique_ptr<HashMap<UInt64, UInt64>>>(attr->maps);
|
||||
const auto it = map.find(id);
|
||||
|
||||
query_count.fetch_add(1, std::memory_order_relaxed);
|
||||
|
||||
return it != map.end() ? it->second : std::get<UInt64>(attr->null_values);
|
||||
}
|
||||
|
||||
@ -86,6 +96,8 @@ public:
|
||||
const auto & map = *std::get<std::unique_ptr<HashMap<UInt64, TYPE>>>(attribute.maps);\
|
||||
const auto it = map.find(id);\
|
||||
\
|
||||
query_count.fetch_add(1, std::memory_order_relaxed);\
|
||||
\
|
||||
return it != map.end() ? TYPE{it->second} : std::get<TYPE>(attribute.null_values);\
|
||||
}
|
||||
DECLARE_INDIVIDUAL_GETTER(UInt8)
|
||||
@ -111,6 +123,8 @@ public:
|
||||
const auto & map = *std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attribute.maps);
|
||||
const auto it = map.find(id);
|
||||
|
||||
query_count.fetch_add(1, std::memory_order_relaxed);
|
||||
|
||||
return it != map.end() ? String{it->second} : std::get<String>(attribute.null_values);
|
||||
}
|
||||
|
||||
@ -155,6 +169,8 @@ public:
|
||||
const auto string_ref = it != attr.end() ? it->second : StringRef{null_value};
|
||||
out->insertData(string_ref.data, string_ref.size);
|
||||
}
|
||||
|
||||
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -310,6 +326,8 @@ private:
|
||||
const auto it = attr.find(ids[i]);
|
||||
out[i] = it != attr.end() ? it->second : null_value;
|
||||
}
|
||||
|
||||
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
@ -368,6 +386,7 @@ private:
|
||||
std::size_t bytes_allocated = 0;
|
||||
std::size_t element_count = 0;
|
||||
std::size_t bucket_count = 0;
|
||||
mutable std::atomic<std::size_t> query_count{};
|
||||
|
||||
std::chrono::time_point<std::chrono::system_clock> creation_time;
|
||||
};
|
||||
|
@ -30,6 +30,8 @@ public:
|
||||
|
||||
virtual std::size_t getBytesAllocated() const = 0;
|
||||
|
||||
virtual std::size_t getQueryCount() const = 0;
|
||||
|
||||
virtual double getHitRate() const = 0;
|
||||
|
||||
virtual std::size_t getElementCount() const = 0;
|
||||
@ -47,6 +49,8 @@ public:
|
||||
|
||||
virtual std::chrono::time_point<std::chrono::system_clock> getCreationTime() const = 0;
|
||||
|
||||
virtual bool isInjective(const std::string & attribute_name) const = 0;
|
||||
|
||||
virtual bool hasHierarchy() const = 0;
|
||||
|
||||
/// do not call unless you ensure that hasHierarchy() returns true
|
||||
|
@ -136,7 +136,7 @@ struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
|
||||
|
||||
|
||||
/** Отдельный случай для преобразования (U)Int32 или (U)Int64 в Date.
|
||||
* Если число меньше 65536, то оно понимается, как DayNum, а если больше - как unix timestamp.
|
||||
* Если число меньше 65536, то оно понимается, как DayNum, а если больше или равно - как unix timestamp.
|
||||
* Немного нелогично, что мы, по сути, помещаем две разные функции в одну.
|
||||
* Но зато это позволяет поддержать распространённый случай,
|
||||
* когда пользователь пишет toDate(UInt32), ожидая, что это - перевод unix timestamp в дату
|
||||
@ -490,8 +490,8 @@ public:
|
||||
* Для неконстантных столбцов arguments[i].column = nullptr.
|
||||
*/
|
||||
void getReturnTypeAndPrerequisites(const ColumnsWithNameAndType & arguments,
|
||||
DataTypePtr & out_return_type,
|
||||
ExpressionActions::Actions & out_prerequisites)
|
||||
DataTypePtr & out_return_type,
|
||||
std::vector<ExpressionAction> & out_prerequisites)
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
|
@ -503,7 +503,7 @@ struct NameRegionToCity { static constexpr auto name = "regionToCity"; };
|
||||
struct NameRegionToArea { static constexpr auto name = "regionToArea"; };
|
||||
struct NameRegionToDistrict { static constexpr auto name = "regionToDistrict"; };
|
||||
struct NameRegionToCountry { static constexpr auto name = "regionToCountry"; };
|
||||
struct NameRegionToContinent { static constexpr auto name = "regionToContient"; };
|
||||
struct NameRegionToContinent { static constexpr auto name = "regionToContinent"; };
|
||||
struct NameRegionToPopulation { static constexpr auto name = "regionToPopulation"; };
|
||||
struct NameOSToRoot { static constexpr auto name = "OSToRoot"; };
|
||||
struct NameSEToRoot { static constexpr auto name = "SEToRoot"; };
|
||||
@ -738,7 +738,7 @@ public:
|
||||
};
|
||||
|
||||
|
||||
class FunctionDictGetString : public IFunction
|
||||
class FunctionDictGetString final : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "dictGetString";
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Poco/Net/DNS.h>
|
||||
|
||||
#include <math.h>
|
||||
#include <mutex>
|
||||
|
||||
#include <DB/IO/WriteBufferFromString.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
@ -21,7 +22,9 @@
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
#include <DB/Columns/ColumnReplicated.h>
|
||||
#include <DB/Common/UnicodeBar.h>
|
||||
#include <DB/Common/HashTable/HashMap.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <DB/Interpreters/ExpressionActions.h>
|
||||
#include <statdaemons/ext/range.hpp>
|
||||
|
||||
|
||||
@ -54,6 +57,8 @@ namespace DB
|
||||
* sleep(n) - спит n секунд каждый блок.
|
||||
*
|
||||
* bar(x, min, max, width) - рисует полосу из количества символов, пропорционального (x - min) и равного width при x == max.
|
||||
*
|
||||
* transform(x, from_array, to_array[, default]) - преобразовать x согласно переданному явным образом соответствию.
|
||||
*/
|
||||
|
||||
|
||||
@ -884,4 +889,506 @@ using FunctionIsFinite = FunctionNumericPredicate<IsFiniteImpl>;
|
||||
using FunctionIsInfinite = FunctionNumericPredicate<IsInfiniteImpl>;
|
||||
using FunctionIsNaN = FunctionNumericPredicate<IsNaNImpl>;
|
||||
|
||||
|
||||
DataTypePtr getSmallestCommonNumericType(const IDataType & t1, const IDataType & t2);
|
||||
|
||||
/** transform(x, [from...], [to...], default)
|
||||
* - преобразует значения согласно явно указанному отображению.
|
||||
*
|
||||
* x - что преобразовывать.
|
||||
* from - константный массив значений для преобразования.
|
||||
* to - константный массив значений, в которые должны быть преобразованы значения из from.
|
||||
* default - константа, какое значение использовать, если x не равен ни одному из значений во from.
|
||||
* from и to - массивы одинаковых размеров.
|
||||
*
|
||||
* Типы:
|
||||
* transform(T, Array(T), Array(U), U) -> U
|
||||
*
|
||||
* transform(x, [from...], [to...])
|
||||
* - eсли default не указан, то для значений x, для которых нет соответствующего элемента во from, возвращается не изменённое значение x.
|
||||
*
|
||||
* Типы:
|
||||
* transform(T, Array(T), Array(T)) -> T
|
||||
*
|
||||
* Замечание: реализация довольно громоздкая.
|
||||
*/
|
||||
class FunctionTransform : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "transform";
|
||||
static IFunction * create(const Context &) { return new FunctionTransform; }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
const auto args_size = arguments.size();
|
||||
if (args_size != 3 && args_size != 4)
|
||||
throw Exception{
|
||||
"Number of arguments for function " + getName() + " doesn't match: passed " +
|
||||
toString(args_size) + ", should be 3 or 4",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
||||
|
||||
const IDataType * type_x = arguments[0].get();
|
||||
|
||||
if (!type_x->isNumeric() && !typeid_cast<const DataTypeString *>(type_x))
|
||||
throw Exception("Unsupported type " + type_x->getName()
|
||||
+ " of first argument of function " + getName()
|
||||
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const DataTypeArray * type_arr_from = typeid_cast<const DataTypeArray *>(arguments[1].get());
|
||||
|
||||
if (!type_arr_from)
|
||||
throw Exception("Second argument of function " + getName()
|
||||
+ ", must be array of source values to transform from.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto type_arr_from_nested = type_arr_from->getNestedType();
|
||||
|
||||
if ((type_x->isNumeric() != type_arr_from_nested->isNumeric())
|
||||
|| (!!typeid_cast<const DataTypeString *>(type_x) != !!typeid_cast<const DataTypeString *>(type_arr_from_nested.get())))
|
||||
throw Exception("First argument and elements of array of second argument of function " + getName()
|
||||
+ " must have compatible types: both numeric or both strings.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const DataTypeArray * type_arr_to = typeid_cast<const DataTypeArray *>(arguments[2].get());
|
||||
|
||||
if (!type_arr_to)
|
||||
throw Exception("Third argument of function " + getName()
|
||||
+ ", must be array of destination values to transform to.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto type_arr_to_nested = type_arr_to->getNestedType();
|
||||
|
||||
if (args_size == 3)
|
||||
{
|
||||
if ((type_x->isNumeric() != type_arr_to_nested->isNumeric())
|
||||
|| (!!typeid_cast<const DataTypeString *>(type_x) != !!typeid_cast<const DataTypeString *>(type_arr_to_nested.get())))
|
||||
throw Exception("Function " + getName()
|
||||
+ " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return type_x->clone();
|
||||
}
|
||||
else
|
||||
{
|
||||
const IDataType * type_default = arguments[3].get();
|
||||
|
||||
if (!type_default->isNumeric() && !typeid_cast<const DataTypeString *>(type_default))
|
||||
throw Exception("Unsupported type " + type_default->getName()
|
||||
+ " of fourth argument (default value) of function " + getName()
|
||||
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if ((type_default->isNumeric() != type_arr_to_nested->isNumeric())
|
||||
|| (!!typeid_cast<const DataTypeString *>(type_default) != !!typeid_cast<const DataTypeString *>(type_arr_to_nested.get())))
|
||||
throw Exception("Function " + getName()
|
||||
+ " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (type_arr_to_nested->behavesAsNumber() && type_default->behavesAsNumber())
|
||||
{
|
||||
/// Берём наименьший общий тип для элементов массива значений to и для default-а.
|
||||
return getSmallestCommonNumericType(*type_arr_to_nested, *type_default);
|
||||
}
|
||||
|
||||
/// TODO Больше проверок.
|
||||
return type_arr_to_nested->clone();
|
||||
}
|
||||
}
|
||||
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const ColumnConstArray * array_from = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[1]).column);
|
||||
const ColumnConstArray * array_to = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[2]).column);
|
||||
|
||||
if (!array_from && !array_to)
|
||||
throw Exception("Second and third arguments of function " + getName() + " must be constant arrays.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
prepare(array_from->getData(), array_to->getData(), block, arguments);
|
||||
|
||||
const auto in = block.getByPosition(arguments.front()).column.get();
|
||||
|
||||
if (in->isConst())
|
||||
{
|
||||
executeConst(block, arguments, result);
|
||||
return;
|
||||
}
|
||||
|
||||
auto column_result = block.getByPosition(result).type->createColumn();
|
||||
auto out = column_result.get();
|
||||
|
||||
if (!executeNum<UInt8>(in, out)
|
||||
&& !executeNum<UInt16>(in, out)
|
||||
&& !executeNum<UInt32>(in, out)
|
||||
&& !executeNum<UInt64>(in, out)
|
||||
&& !executeNum<Int8>(in, out)
|
||||
&& !executeNum<Int16>(in, out)
|
||||
&& !executeNum<Int32>(in, out)
|
||||
&& !executeNum<Int64>(in, out)
|
||||
&& !executeNum<Float32>(in, out)
|
||||
&& !executeNum<Float64>(in, out)
|
||||
&& !executeString(in, out))
|
||||
throw Exception(
|
||||
"Illegal column " + in->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
block.getByPosition(result).column = column_result;
|
||||
}
|
||||
|
||||
private:
|
||||
void executeConst(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
{
|
||||
/// Составим блок из полноценных столбцов размера 1 и вычислим функцию как обычно.
|
||||
|
||||
Block tmp_block;
|
||||
ColumnNumbers tmp_arguments;
|
||||
|
||||
tmp_block.insert(block.getByPosition(arguments[0]));
|
||||
tmp_block.getByPosition(0).column = static_cast<IColumnConst *>(tmp_block.getByPosition(0).column->cloneResized(1).get())->convertToFullColumn();
|
||||
tmp_arguments.push_back(0);
|
||||
|
||||
for (size_t i = 1; i < arguments.size(); ++i)
|
||||
{
|
||||
tmp_block.insert(block.getByPosition(arguments[i]));
|
||||
tmp_arguments.push_back(i);
|
||||
}
|
||||
|
||||
tmp_block.insert(block.getByPosition(result));
|
||||
size_t tmp_result = arguments.size();
|
||||
|
||||
execute(tmp_block, tmp_arguments, tmp_result);
|
||||
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(
|
||||
block.rowsInFirstColumn(),
|
||||
(*tmp_block.getByPosition(tmp_result).column)[0]);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeNum(const IColumn * in_untyped, IColumn * out_untyped)
|
||||
{
|
||||
if (const auto in = typeid_cast<const ColumnVector<T> *>(in_untyped))
|
||||
{
|
||||
if (default_value.isNull())
|
||||
{
|
||||
auto out = typeid_cast<ColumnVector<T> *>(out_untyped);
|
||||
if (!out)
|
||||
throw Exception(
|
||||
"Illegal column " + out_untyped->getName() + " of elements of array of third argument of function " + getName()
|
||||
+ ", must be " + in->getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
executeImplNumToNum<T>(in->getData(), out->getData());
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!executeNumToNumWithDefault<T, UInt8>(in, out_untyped)
|
||||
&& !executeNumToNumWithDefault<T, UInt16>(in, out_untyped)
|
||||
&& !executeNumToNumWithDefault<T, UInt32>(in, out_untyped)
|
||||
&& !executeNumToNumWithDefault<T, UInt64>(in, out_untyped)
|
||||
&& !executeNumToNumWithDefault<T, Int8>(in, out_untyped)
|
||||
&& !executeNumToNumWithDefault<T, Int16>(in, out_untyped)
|
||||
&& !executeNumToNumWithDefault<T, Int32>(in, out_untyped)
|
||||
&& !executeNumToNumWithDefault<T, Int64>(in, out_untyped)
|
||||
&& !executeNumToNumWithDefault<T, Float32>(in, out_untyped)
|
||||
&& !executeNumToNumWithDefault<T, Float64>(in, out_untyped)
|
||||
&& !executeNumToString<T>(in, out_untyped))
|
||||
throw Exception(
|
||||
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
bool executeString(const IColumn * in_untyped, IColumn * out_untyped)
|
||||
{
|
||||
if (const auto in = typeid_cast<const ColumnString *>(in_untyped))
|
||||
{
|
||||
if (!executeStringToNum<UInt8>(in, out_untyped)
|
||||
&& !executeStringToNum<UInt16>(in, out_untyped)
|
||||
&& !executeStringToNum<UInt32>(in, out_untyped)
|
||||
&& !executeStringToNum<UInt64>(in, out_untyped)
|
||||
&& !executeStringToNum<Int8>(in, out_untyped)
|
||||
&& !executeStringToNum<Int16>(in, out_untyped)
|
||||
&& !executeStringToNum<Int32>(in, out_untyped)
|
||||
&& !executeStringToNum<Int64>(in, out_untyped)
|
||||
&& !executeStringToNum<Float32>(in, out_untyped)
|
||||
&& !executeStringToNum<Float64>(in, out_untyped)
|
||||
&& !executeStringToString(in, out_untyped))
|
||||
throw Exception(
|
||||
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
bool executeNumToNumWithDefault(const ColumnVector<T> * in, IColumn * out_untyped)
|
||||
{
|
||||
auto out = typeid_cast<ColumnVector<U> *>(out_untyped);
|
||||
if (!out)
|
||||
return false;
|
||||
|
||||
executeImplNumToNumWithDefault<T, U>(in->getData(), out->getData(), default_value.get<U>());
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeNumToString(const ColumnVector<T> * in, IColumn * out_untyped)
|
||||
{
|
||||
auto out = typeid_cast<ColumnString *>(out_untyped);
|
||||
if (!out)
|
||||
return false;
|
||||
|
||||
const String & default_str = default_value.get<const String &>();
|
||||
StringRef default_string_ref{default_str.data(), default_str.size() + 1};
|
||||
executeImplNumToStringWithDefault<T>(in->getData(), out->getChars(), out->getOffsets(), default_string_ref);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename U>
|
||||
bool executeStringToNum(const ColumnString * in, IColumn * out_untyped)
|
||||
{
|
||||
auto out = typeid_cast<ColumnVector<U> *>(out_untyped);
|
||||
if (!out)
|
||||
return false;
|
||||
|
||||
executeImplStringToNumWithDefault<U>(in->getChars(), in->getOffsets(), out->getData(), default_value.get<U>());
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeStringToString(const ColumnString * in, IColumn * out_untyped)
|
||||
{
|
||||
auto out = typeid_cast<ColumnString *>(out_untyped);
|
||||
if (!out)
|
||||
return false;
|
||||
|
||||
if (default_value.isNull())
|
||||
executeImplStringToString<false>(in->getChars(), in->getOffsets(), out->getChars(), out->getOffsets(), {});
|
||||
else
|
||||
{
|
||||
const String & default_str = default_value.get<const String &>();
|
||||
StringRef default_string_ref{default_str.data(), default_str.size() + 1};
|
||||
executeImplStringToString<true>(in->getChars(), in->getOffsets(), out->getChars(), out->getOffsets(), default_string_ref);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
template <typename T, typename U>
|
||||
void executeImplNumToNumWithDefault(const PODArray<T> & src, PODArray<U> & dst, U dst_default)
|
||||
{
|
||||
const auto & table = *table_num_to_num;
|
||||
size_t size = src.size();
|
||||
dst.resize(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
auto it = table.find(src[i]);
|
||||
if (it != table.end())
|
||||
memcpy(&dst[i], &it->second, sizeof(dst[i])); /// little endian.
|
||||
else
|
||||
dst[i] = dst_default;
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void executeImplNumToNum(const PODArray<T> & src, PODArray<T> & dst)
|
||||
{
|
||||
const auto & table = *table_num_to_num;
|
||||
size_t size = src.size();
|
||||
dst.resize(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
auto it = table.find(src[i]);
|
||||
if (it != table.end())
|
||||
memcpy(&dst[i], &it->second, sizeof(dst[i]));
|
||||
else
|
||||
dst[i] = src[i];
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void executeImplNumToStringWithDefault(const PODArray<T> & src,
|
||||
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets, StringRef dst_default)
|
||||
{
|
||||
const auto & table = *table_num_to_string;
|
||||
size_t size = src.size();
|
||||
dst_offsets.resize(size);
|
||||
ColumnString::Offset_t current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
auto it = table.find(src[i]);
|
||||
StringRef ref = it != table.end() ? it->second : dst_default;
|
||||
dst_data.resize(current_offset + ref.size);
|
||||
memcpy(&dst_data[current_offset], ref.data, ref.size);
|
||||
current_offset += ref.size;
|
||||
dst_offsets[i] = current_offset;
|
||||
}
|
||||
}
|
||||
|
||||
template <typename U>
|
||||
void executeImplStringToNumWithDefault(
|
||||
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
|
||||
PODArray<U> & dst, U dst_default)
|
||||
{
|
||||
const auto & table = *table_string_to_num;
|
||||
size_t size = src_offsets.size();
|
||||
dst.resize(size);
|
||||
ColumnString::Offset_t current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
StringRef ref{&src_data[current_offset], src_offsets[i] - current_offset};
|
||||
current_offset = src_offsets[i];
|
||||
auto it = table.find(ref);
|
||||
if (it != table.end())
|
||||
memcpy(&dst[i], &it->second, sizeof(dst[i]));
|
||||
else
|
||||
dst[i] = dst_default;
|
||||
}
|
||||
}
|
||||
|
||||
template <bool with_default>
|
||||
void executeImplStringToString(
|
||||
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
|
||||
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets, StringRef dst_default)
|
||||
{
|
||||
const auto & table = *table_string_to_string;
|
||||
size_t size = src_offsets.size();
|
||||
dst_offsets.resize(size);
|
||||
ColumnString::Offset_t current_src_offset = 0;
|
||||
ColumnString::Offset_t current_dst_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
StringRef src_ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset};
|
||||
current_src_offset = src_offsets[i];
|
||||
|
||||
auto it = table.find(src_ref);
|
||||
|
||||
StringRef dst_ref = it != table.end() ? it->second : (with_default ? dst_default : src_ref);
|
||||
dst_data.resize(current_dst_offset + dst_ref.size);
|
||||
memcpy(&dst_data[current_dst_offset], dst_ref.data, dst_ref.size);
|
||||
current_dst_offset += dst_ref.size;
|
||||
dst_offsets[i] = current_dst_offset;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/// Разные варианты хэш-таблиц для реализации отображения.
|
||||
|
||||
using NumToNum = HashMap<UInt64, UInt64, HashCRC32<UInt64>>;
|
||||
using NumToString = HashMap<UInt64, StringRef, HashCRC32<UInt64>>; /// Везде StringRef-ы с завершающим нулём.
|
||||
using StringToNum = HashMap<StringRef, UInt64>;
|
||||
using StringToString = HashMap<StringRef, StringRef>;
|
||||
|
||||
std::unique_ptr<NumToNum> table_num_to_num;
|
||||
std::unique_ptr<NumToString> table_num_to_string;
|
||||
std::unique_ptr<StringToNum> table_string_to_num;
|
||||
std::unique_ptr<StringToString> table_string_to_string;
|
||||
|
||||
Arena string_pool;
|
||||
|
||||
Field default_value; /// Null, если не задано.
|
||||
|
||||
bool prepared = false;
|
||||
std::mutex mutex;
|
||||
|
||||
/// Может вызываться из разных потоков. Срабатывает только при первом вызове.
|
||||
void prepare(const Array & from, const Array & to, Block & block, const ColumnNumbers & arguments)
|
||||
{
|
||||
if (prepared)
|
||||
return;
|
||||
|
||||
const size_t size = from.size();
|
||||
if (0 == size)
|
||||
throw Exception("Empty arrays are illegal in function " + getName(), ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
if (prepared)
|
||||
return;
|
||||
|
||||
if (from.size() != to.size())
|
||||
throw Exception("Second and third arguments of function " + getName() + " must be arrays of same size", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
Array converted_to;
|
||||
const Array * used_to = &to;
|
||||
|
||||
/// Задано ли значение по-умолчанию.
|
||||
|
||||
if (arguments.size() == 4)
|
||||
{
|
||||
const IColumnConst * default_col = dynamic_cast<const IColumnConst *>(&*block.getByPosition(arguments[3]).column);
|
||||
|
||||
if (!default_col)
|
||||
throw Exception("Fourth argument of function " + getName() + " (default value) must be constant", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
default_value = (*default_col)[0];
|
||||
|
||||
/// Нужно ли преобразовать элементы to и default_value к наименьшему общему типу, который является Float64?
|
||||
if (default_value.getType() == Field::Types::Float64 && to[0].getType() != Field::Types::Float64)
|
||||
{
|
||||
converted_to.resize(to.size());
|
||||
for (size_t i = 0, size = to.size(); i < size; ++i)
|
||||
converted_to[i] = apply_visitor(FieldVisitorConvertToNumber<Float64>(), to[i]);
|
||||
used_to = &converted_to;
|
||||
}
|
||||
else if (default_value.getType() != Field::Types::Float64 && to[0].getType() == Field::Types::Float64)
|
||||
{
|
||||
default_value = apply_visitor(FieldVisitorConvertToNumber<Float64>(), default_value);
|
||||
}
|
||||
}
|
||||
|
||||
/// Замечание: не делается проверка дубликатов в массиве from.
|
||||
|
||||
if (from[0].getType() != Field::Types::String && to[0].getType() != Field::Types::String)
|
||||
{
|
||||
table_num_to_num.reset(new NumToNum);
|
||||
auto & table = *table_num_to_num;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
table[from[i].get<UInt64>()] = (*used_to)[i].get<UInt64>();
|
||||
}
|
||||
else if (from[0].getType() != Field::Types::String && to[0].getType() == Field::Types::String)
|
||||
{
|
||||
table_num_to_string.reset(new NumToString);
|
||||
auto & table = *table_num_to_string;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const String & str_to = to[i].get<const String &>();
|
||||
StringRef ref{string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1};
|
||||
table[from[i].get<UInt64>()] = ref;
|
||||
}
|
||||
}
|
||||
else if (from[0].getType() == Field::Types::String && to[0].getType() != Field::Types::String)
|
||||
{
|
||||
table_string_to_num.reset(new StringToNum);
|
||||
auto & table = *table_string_to_num;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const String & str_from = from[i].get<const String &>();
|
||||
StringRef ref{string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
|
||||
table[ref] = (*used_to)[i].get<UInt64>();
|
||||
}
|
||||
}
|
||||
else if (from[0].getType() == Field::Types::String && to[0].getType() == Field::Types::String)
|
||||
{
|
||||
table_string_to_string.reset(new StringToString);
|
||||
auto & table = *table_string_to_string;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const String & str_from = from[i].get<const String &>();
|
||||
const String & str_to = to[i].get<const String &>();
|
||||
StringRef ref_from{string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
|
||||
StringRef ref_to{string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1};
|
||||
table[ref_from] = ref_to;
|
||||
}
|
||||
}
|
||||
|
||||
prepared = true;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,7 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Functions/FunctionsArithmetic.h>
|
||||
#include <cmath> // log2()
|
||||
#include <cmath>
|
||||
#include <type_traits>
|
||||
#include <array>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -11,6 +13,9 @@ namespace DB
|
||||
* roundToExp2 - вниз до ближайшей степени двойки;
|
||||
* roundDuration - вниз до ближайшего из: 0, 1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000;
|
||||
* roundAge - вниз до ближайшего из: 0, 18, 25, 35, 45.
|
||||
* round(x, N) - арифметическое округление (N - сколько знаков после запятой оставить; 0 по умолчанию).
|
||||
* ceil(x, N) - наименьшее число, которое не меньше x (N - сколько знаков после запятой оставить; 0 по умолчанию).
|
||||
* floor(x, N) - наибольшее число, которое не больше x (N - сколько знаков после запятой оставить; 0 по умолчанию).
|
||||
*/
|
||||
|
||||
template<typename A>
|
||||
@ -87,13 +92,439 @@ namespace DB
|
||||
}
|
||||
};
|
||||
|
||||
/// Реализация функций округления на низком уровне.
|
||||
|
||||
template<typename T, int rounding_mode, bool with_scale>
|
||||
struct RoundingComputation
|
||||
{
|
||||
};
|
||||
|
||||
template<int rounding_mode, bool with_scale>
|
||||
struct RoundingComputation<Float32, rounding_mode, with_scale>
|
||||
{
|
||||
using Data = std::array<Float32, 4>;
|
||||
using Scale = __m128;
|
||||
|
||||
template<bool with_scale2 = with_scale>
|
||||
static inline void prepareScale(size_t scale, Scale & mm_scale,
|
||||
typename std::enable_if<with_scale2>::type * = nullptr)
|
||||
{
|
||||
Float32 fscale = static_cast<Float32>(scale);
|
||||
mm_scale = _mm_load1_ps(&fscale);
|
||||
}
|
||||
|
||||
template<bool with_scale2 = with_scale>
|
||||
static inline void prepareScale(size_t scale, Scale & mm_scale,
|
||||
typename std::enable_if<!with_scale2>::type * = nullptr)
|
||||
{
|
||||
}
|
||||
|
||||
template<bool with_scale2 = with_scale>
|
||||
static inline void compute(const Data & in, const Scale & mm_scale, Data & out,
|
||||
typename std::enable_if<with_scale2>::type * = nullptr)
|
||||
{
|
||||
__m128 mm_value = _mm_loadu_ps(reinterpret_cast<const Float32 *>(&in));
|
||||
mm_value = _mm_mul_ps(mm_value, mm_scale);
|
||||
mm_value = _mm_round_ps(mm_value, rounding_mode);
|
||||
mm_value = _mm_div_ps(mm_value, mm_scale);
|
||||
_mm_storeu_ps(reinterpret_cast<Float32 *>(&out), mm_value);
|
||||
}
|
||||
|
||||
template<bool with_scale2 = with_scale>
|
||||
static inline void compute(const Data & in, const Scale & mm_scale, Data & out,
|
||||
typename std::enable_if<!with_scale2>::type * = nullptr)
|
||||
{
|
||||
__m128 mm_value = _mm_loadu_ps(reinterpret_cast<const Float32 *>(&in));
|
||||
mm_value = _mm_round_ps(mm_value, rounding_mode);
|
||||
_mm_storeu_ps(reinterpret_cast<Float32 *>(&out), mm_value);
|
||||
}
|
||||
};
|
||||
|
||||
template<int rounding_mode, bool with_scale>
|
||||
struct RoundingComputation<Float64, rounding_mode, with_scale>
|
||||
{
|
||||
using Data = std::array<Float64, 2>;
|
||||
using Scale = __m128d;
|
||||
|
||||
template<bool with_scale2 = with_scale>
|
||||
static inline void prepareScale(size_t scale, Scale & mm_scale,
|
||||
typename std::enable_if<with_scale2>::type * = nullptr)
|
||||
{
|
||||
Float64 fscale = static_cast<Float64>(scale);
|
||||
mm_scale = _mm_load1_pd(&fscale);
|
||||
}
|
||||
|
||||
template<bool with_scale2 = with_scale>
|
||||
static inline void prepareScale(size_t scale, Scale & mm_scale,
|
||||
typename std::enable_if<!with_scale2>::type * = nullptr)
|
||||
{
|
||||
}
|
||||
|
||||
template<bool with_scale2 = with_scale>
|
||||
static inline void compute(const Data & in, const Scale & mm_scale, Data & out,
|
||||
typename std::enable_if<with_scale2>::type * = nullptr)
|
||||
{
|
||||
__m128d mm_value = _mm_loadu_pd(reinterpret_cast<const Float64 *>(&in));
|
||||
mm_value = _mm_mul_pd(mm_value, mm_scale);
|
||||
mm_value = _mm_round_pd(mm_value, rounding_mode);
|
||||
mm_value = _mm_div_pd(mm_value, mm_scale);
|
||||
_mm_storeu_pd(reinterpret_cast<Float64 *>(&out), mm_value);
|
||||
}
|
||||
|
||||
template<bool with_scale2 = with_scale>
|
||||
static inline void compute(const Data & in, const Scale & mm_scale, Data & out,
|
||||
typename std::enable_if<!with_scale2>::type * = nullptr)
|
||||
{
|
||||
__m128d mm_value = _mm_loadu_pd(reinterpret_cast<const Float64 *>(&in));
|
||||
mm_value = _mm_round_pd(mm_value, rounding_mode);
|
||||
_mm_storeu_pd(reinterpret_cast<Float64 *>(&out), mm_value);
|
||||
}
|
||||
};
|
||||
|
||||
/// Реализация функций округления на высоком уровне.
|
||||
|
||||
template<typename T, int rounding_mode, bool with_scale, typename Enable = void>
|
||||
struct FunctionRoundingImpl
|
||||
{
|
||||
};
|
||||
|
||||
/// В случае целочисленных значений не выполяется округления.
|
||||
template<typename T, int rounding_mode, bool with_scale>
|
||||
struct FunctionRoundingImpl<T, rounding_mode, with_scale, typename std::enable_if<std::is_integral<T>::value>::type>
|
||||
{
|
||||
static inline void apply(const PODArray<T> & in, size_t scale, typename ColumnVector<T>::Container_t & out)
|
||||
{
|
||||
size_t size = in.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
out[i] = in[i];
|
||||
}
|
||||
|
||||
static inline T apply(T val, size_t scale)
|
||||
{
|
||||
return val;
|
||||
}
|
||||
};
|
||||
|
||||
template<typename T, int rounding_mode, bool with_scale>
|
||||
struct FunctionRoundingImpl<T, rounding_mode, with_scale, typename std::enable_if<std::is_floating_point<T>::value>::type>
|
||||
{
|
||||
private:
|
||||
using Op = RoundingComputation<T, rounding_mode, with_scale>;
|
||||
using Data = typename Op::Data;
|
||||
using Scale = typename Op::Scale;
|
||||
|
||||
public:
|
||||
static inline void apply(const PODArray<T> & in, size_t scale, typename ColumnVector<T>::Container_t & out)
|
||||
{
|
||||
Scale mm_scale;
|
||||
Op::prepareScale(scale, mm_scale);
|
||||
|
||||
const size_t size = in.size();
|
||||
const size_t data_size = std::tuple_size<Data>();
|
||||
|
||||
size_t i;
|
||||
for (i = 0; i < (size - data_size + 1); i += data_size)
|
||||
{
|
||||
Data tmp;
|
||||
for (size_t j = 0; j < data_size; ++j)
|
||||
tmp[j] = in[i + j];
|
||||
|
||||
Data res;
|
||||
Op::compute(tmp, mm_scale, res);
|
||||
|
||||
for (size_t j = 0; j < data_size; ++j)
|
||||
out[i + j] = res[j];
|
||||
}
|
||||
|
||||
if (i < size)
|
||||
{
|
||||
Data tmp{0};
|
||||
for (size_t j = 0; (j < data_size) && ((i + j) < size); ++j)
|
||||
tmp[j] = in[i + j];
|
||||
|
||||
Data res;
|
||||
Op::compute(tmp, mm_scale, res);
|
||||
|
||||
for (size_t j = 0; (j < data_size) && ((i + j) < size); ++j)
|
||||
out[i + j] = res[j];
|
||||
}
|
||||
}
|
||||
|
||||
static inline T apply(T val, size_t scale)
|
||||
{
|
||||
if (val == 0)
|
||||
return val;
|
||||
else
|
||||
{
|
||||
Scale mm_scale;
|
||||
Op::prepareScale(scale, mm_scale);
|
||||
|
||||
Data tmp{0};
|
||||
tmp[0] = val;
|
||||
|
||||
Data res;
|
||||
Op::compute(tmp, mm_scale, res);
|
||||
return res[0];
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
template<typename T, typename U>
|
||||
struct PrecisionForType
|
||||
{
|
||||
template<typename L = T>
|
||||
static inline bool apply(const ColumnPtr & column, UInt8 & precision,
|
||||
typename std::enable_if<std::is_floating_point<L>::value>::type * = nullptr)
|
||||
{
|
||||
using ColumnType = ColumnConst<U>;
|
||||
|
||||
const ColumnType * precision_col = typeid_cast<const ColumnType *>(&*column);
|
||||
if (precision_col == nullptr)
|
||||
return false;
|
||||
|
||||
U val = precision_col->getData();
|
||||
if (val < 0)
|
||||
val = 0;
|
||||
else if (val >= static_cast<U>(std::numeric_limits<L>::digits10))
|
||||
val = static_cast<U>(std::numeric_limits<L>::digits10);
|
||||
|
||||
precision = static_cast<UInt8>(val);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
/// Для целых чисел точность не имеет значения.
|
||||
template<typename L = T>
|
||||
static inline bool apply(const ColumnPtr & column, UInt8 & precision,
|
||||
typename std::enable_if<std::is_integral<L>::value>::type * = nullptr)
|
||||
{
|
||||
using ColumnType = ColumnConst<U>;
|
||||
|
||||
const ColumnType * precision_col = typeid_cast<const ColumnType *>(&*column);
|
||||
if (precision_col == nullptr)
|
||||
return false;
|
||||
|
||||
precision = 0;
|
||||
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
/// Следующий код генерирует во время сборки таблицу степеней числа 10.
|
||||
|
||||
namespace
|
||||
{
|
||||
/// Отдельные степени числа 10.
|
||||
|
||||
template<size_t N>
|
||||
struct PowerOf10
|
||||
{
|
||||
static const size_t value = 10 * PowerOf10<N - 1>::value;
|
||||
};
|
||||
|
||||
template<>
|
||||
struct PowerOf10<0>
|
||||
{
|
||||
static const size_t value = 1;
|
||||
};
|
||||
}
|
||||
|
||||
/// Объявление и определение контейнера содержащего таблицу степеней числа 10.
|
||||
|
||||
template<size_t... TArgs>
|
||||
struct TableContainer
|
||||
{
|
||||
static const std::array<size_t, sizeof...(TArgs)> values;
|
||||
};
|
||||
|
||||
template<size_t... TArgs>
|
||||
const std::array<size_t, sizeof...(TArgs)> TableContainer<TArgs...>::values = { TArgs... };
|
||||
|
||||
/// Генератор первых N степеней.
|
||||
|
||||
template<size_t N, size_t... TArgs>
|
||||
struct FillArrayImpl
|
||||
{
|
||||
using result = typename FillArrayImpl<N - 1, PowerOf10<N>::value, TArgs...>::result;
|
||||
};
|
||||
|
||||
template<size_t... TArgs>
|
||||
struct FillArrayImpl<0, TArgs...>
|
||||
{
|
||||
using result = TableContainer<PowerOf10<0>::value, TArgs...>;
|
||||
};
|
||||
|
||||
template<size_t N>
|
||||
struct FillArray
|
||||
{
|
||||
using result = typename FillArrayImpl<N - 1>::result;
|
||||
};
|
||||
|
||||
/** Шаблон для функций, которые вычисляют приближенное значение входного параметра
|
||||
* типа (U)Int8/16/32/64 или Float32/64 и принимают дополнительный необязятельный
|
||||
* параметр указывающий сколько знаков после запятой оставить (по умолчанию - 0).
|
||||
* Op - функция (round/floor/ceil)
|
||||
*/
|
||||
template<typename Name, int rounding_mode>
|
||||
class FunctionRounding : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static IFunction * create(const Context & context) { return new FunctionRounding; }
|
||||
|
||||
private:
|
||||
using PowersOf10 = FillArray<std::numeric_limits<Float64>::digits10 + 1>::result;
|
||||
|
||||
private:
|
||||
template<typename T>
|
||||
bool checkType(const IDataType * type) const
|
||||
{
|
||||
return typeid_cast<const T *>(type) != nullptr;
|
||||
}
|
||||
|
||||
template<typename T>
|
||||
bool executeForType(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
using OpWithScale = FunctionRoundingImpl<T, rounding_mode, true>;
|
||||
using OpWithoutScale = FunctionRoundingImpl<T, rounding_mode, false>;
|
||||
|
||||
if (ColumnVector<T> * col = typeid_cast<ColumnVector<T> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
ColumnVector<T> * col_res = new ColumnVector<T>;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
typename ColumnVector<T>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col->getData().size());
|
||||
|
||||
UInt8 precision = 0;
|
||||
if (arguments.size() == 2)
|
||||
precision = getPrecision<T>(block.getByPosition(arguments[1]).column);
|
||||
|
||||
if (precision > 0)
|
||||
OpWithScale::apply(col->getData(), PowersOf10::values[precision], vec_res);
|
||||
else
|
||||
OpWithoutScale::apply(col->getData(), 0, vec_res);
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (ColumnConst<T> * col = typeid_cast<ColumnConst<T> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
UInt8 precision = 0;
|
||||
if (arguments.size() == 2)
|
||||
precision = getPrecision<T>(block.getByPosition(arguments[1]).column);
|
||||
|
||||
T res;
|
||||
if (precision > 0)
|
||||
res = OpWithScale::apply(col->getData(), PowersOf10::values[precision]);
|
||||
else
|
||||
res = OpWithoutScale::apply(col->getData(), 0);
|
||||
|
||||
ColumnConst<T> * col_res = new ColumnConst<T>(col->size(), res);
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
/// В зависимости от входного параметра, определить какая нужна точность
|
||||
/// для результата.
|
||||
template<typename T>
|
||||
UInt8 getPrecision(const ColumnPtr & column)
|
||||
{
|
||||
UInt8 precision = 0;
|
||||
|
||||
if (!( PrecisionForType<T, UInt8>::apply(column, precision)
|
||||
|| PrecisionForType<T, UInt16>::apply(column, precision)
|
||||
|| PrecisionForType<T, UInt16>::apply(column, precision)
|
||||
|| PrecisionForType<T, UInt32>::apply(column, precision)
|
||||
|| PrecisionForType<T, UInt64>::apply(column, precision)
|
||||
|| PrecisionForType<T, Int8>::apply(column, precision)
|
||||
|| PrecisionForType<T, Int16>::apply(column, precision)
|
||||
|| PrecisionForType<T, Int32>::apply(column, precision)
|
||||
|| PrecisionForType<T, Int64>::apply(column, precision)))
|
||||
{
|
||||
throw Exception("Illegal column " + column->getName()
|
||||
+ " of second ('precision') argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
return precision;
|
||||
}
|
||||
|
||||
public:
|
||||
/// Получить имя функции.
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
if ((arguments.size() < 1) || (arguments.size() > 2))
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1 or 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const IDataType * type = &*arguments[1];
|
||||
if (!( checkType<DataTypeUInt8>(type)
|
||||
|| checkType<DataTypeUInt16>(type)
|
||||
|| checkType<DataTypeUInt32>(type)
|
||||
|| checkType<DataTypeUInt64>(type)
|
||||
|| checkType<DataTypeInt8>(type)
|
||||
|| checkType<DataTypeInt16>(type)
|
||||
|| checkType<DataTypeInt32>(type)
|
||||
|| checkType<DataTypeInt64>(type)))
|
||||
{
|
||||
throw Exception("Illegal type in second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
}
|
||||
|
||||
const IDataType * type = &*arguments[0];
|
||||
if (!type->behavesAsNumber())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (!( executeForType<UInt8>(block, arguments, result)
|
||||
|| executeForType<UInt16>(block, arguments, result)
|
||||
|| executeForType<UInt32>(block, arguments, result)
|
||||
|| executeForType<UInt64>(block, arguments, result)
|
||||
|| executeForType<Int8>(block, arguments, result)
|
||||
|| executeForType<Int16>(block, arguments, result)
|
||||
|| executeForType<Int32>(block, arguments, result)
|
||||
|| executeForType<Int64>(block, arguments, result)
|
||||
|| executeForType<Float32>(block, arguments, result)
|
||||
|| executeForType<Float64>(block, arguments, result)))
|
||||
{
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
struct NameRoundToExp2 { static constexpr auto name = "roundToExp2"; };
|
||||
struct NameRoundDuration { static constexpr auto name = "roundDuration"; };
|
||||
struct NameRoundAge { static constexpr auto name = "roundAge"; };
|
||||
struct NameRound { static constexpr auto name = "round"; };
|
||||
struct NameCeil { static constexpr auto name = "ceil"; };
|
||||
struct NameFloor { static constexpr auto name = "floor"; };
|
||||
|
||||
typedef FunctionUnaryArithmetic<RoundToExp2Impl, NameRoundToExp2> FunctionRoundToExp2;
|
||||
typedef FunctionUnaryArithmetic<RoundDurationImpl, NameRoundDuration> FunctionRoundDuration;
|
||||
typedef FunctionUnaryArithmetic<RoundAgeImpl, NameRoundAge> FunctionRoundAge;
|
||||
|
||||
typedef FunctionRounding<NameRound, _MM_FROUND_NINT> FunctionRound;
|
||||
typedef FunctionRounding<NameCeil, _MM_FROUND_CEIL> FunctionCeil;
|
||||
typedef FunctionRounding<NameFloor, _MM_FROUND_FLOOR> FunctionFloor;
|
||||
}
|
||||
|
@ -279,7 +279,6 @@ struct ExtractPathFull
|
||||
|
||||
if (nullptr != (pos = strchr(data, '/')) && pos[1] == '/' && nullptr != (pos = strchr(pos + 2, '/')))
|
||||
{
|
||||
/// no leading slash
|
||||
res_data = pos;
|
||||
res_size = end - res_data;
|
||||
}
|
||||
|
@ -7,12 +7,14 @@
|
||||
#include <DB/Core/ColumnNumbers.h>
|
||||
#include <DB/Core/ColumnsWithNameAndType.h>
|
||||
#include <DB/DataTypes/IDataType.h>
|
||||
#include <DB/Interpreters/ExpressionActions.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ExpressionAction;
|
||||
|
||||
|
||||
/** Интерфейс для обычных функций.
|
||||
* Обычные функции - это функции, которые не меняют количество строк в таблице,
|
||||
* и результат работы которых для каждой строчки не зависит от других строк.
|
||||
@ -54,7 +56,7 @@ public:
|
||||
virtual void getReturnTypeAndPrerequisites(
|
||||
const ColumnsWithNameAndType & arguments,
|
||||
DataTypePtr & out_return_type,
|
||||
ExpressionActions::Actions & out_prerequisites)
|
||||
std::vector<ExpressionAction> & out_prerequisites)
|
||||
{
|
||||
DataTypes types(arguments.size());
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
|
@ -55,7 +55,7 @@ private:
|
||||
BufferWithOwnMemory<ReadBuffer> fill_buffer;
|
||||
|
||||
/// Описание асинхронного запроса на чтение.
|
||||
iocb request;
|
||||
iocb request = { 0 };
|
||||
std::vector<iocb *> request_ptrs{&request};
|
||||
std::vector<io_event> events{1};
|
||||
|
||||
|
@ -26,7 +26,6 @@ public:
|
||||
WriteBufferAIO & operator=(const WriteBufferAIO &) = delete;
|
||||
|
||||
off_t getPositionInFile() override;
|
||||
void truncate(off_t length = 0) override;
|
||||
void sync() override;
|
||||
std::string getFileName() const override { return filename; }
|
||||
int getFD() const override { return fd; }
|
||||
@ -36,6 +35,8 @@ private:
|
||||
void nextImpl() override;
|
||||
///
|
||||
off_t doSeek(off_t off, int whence) override;
|
||||
///
|
||||
void doTruncate(off_t length) override;
|
||||
/// Если в буфере ещё остались данные - запишем их.
|
||||
void flush();
|
||||
/// Ждать окончания текущей асинхронной задачи.
|
||||
@ -50,7 +51,7 @@ private:
|
||||
BufferWithOwnMemory<WriteBuffer> flush_buffer;
|
||||
|
||||
/// Описание асинхронного запроса на запись.
|
||||
iocb request;
|
||||
iocb request = { 0 };
|
||||
std::vector<iocb *> request_ptrs{&request};
|
||||
std::vector<io_event> events{1};
|
||||
|
||||
|
@ -14,15 +14,17 @@ class WriteBufferFromFileBase : public BufferWithOwnMemory<WriteBuffer>
|
||||
public:
|
||||
WriteBufferFromFileBase(size_t buf_size, char * existing_memory, size_t alignment);
|
||||
virtual ~WriteBufferFromFileBase();
|
||||
|
||||
off_t seek(off_t off, int whence = SEEK_SET);
|
||||
void truncate(off_t length = 0);
|
||||
virtual off_t getPositionInFile() = 0;
|
||||
virtual void truncate(off_t length) = 0;
|
||||
virtual void sync() = 0;
|
||||
virtual std::string getFileName() const = 0;
|
||||
virtual int getFD() const = 0;
|
||||
|
||||
protected:
|
||||
virtual off_t doSeek(off_t off, int whence) = 0;
|
||||
virtual void doTruncate(off_t length) = 0;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -85,13 +85,6 @@ public:
|
||||
return seek(0, SEEK_CUR);
|
||||
}
|
||||
|
||||
void truncate(off_t length = 0) override
|
||||
{
|
||||
int res = ftruncate(fd, length);
|
||||
if (-1 == res)
|
||||
throwFromErrno("Cannot truncate file " + getFileName(), ErrorCodes::CANNOT_TRUNCATE_FILE);
|
||||
}
|
||||
|
||||
void sync() override
|
||||
{
|
||||
/// Если в буфере ещё остались данные - запишем их.
|
||||
@ -111,6 +104,13 @@ private:
|
||||
throwFromErrno("Cannot seek through file " + getFileName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
|
||||
return res;
|
||||
}
|
||||
|
||||
void doTruncate(off_t length) override
|
||||
{
|
||||
int res = ftruncate(fd, length);
|
||||
if (-1 == res)
|
||||
throwFromErrno("Cannot truncate file " + getFileName(), ErrorCodes::CANNOT_TRUNCATE_FILE);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -7,6 +7,7 @@
|
||||
|
||||
#include <DB/IO/WriteBuffer.h>
|
||||
#include <DB/IO/BufferWithOwnMemory.h>
|
||||
#include <statdaemons/NetException.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -38,7 +39,7 @@ private:
|
||||
ostr->flush();
|
||||
|
||||
if (!ostr->good())
|
||||
throw Exception("Cannot write to ostream", ErrorCodes::CANNOT_WRITE_TO_OSTREAM);
|
||||
throw NetException("Cannot write to ostream", ErrorCodes::CANNOT_WRITE_TO_OSTREAM);
|
||||
}
|
||||
|
||||
public:
|
||||
|
@ -26,6 +26,10 @@ public:
|
||||
/// к локальным узлам обращаемся напрямую
|
||||
size_t getLocalNodesNum() const { return local_nodes_num; }
|
||||
|
||||
/// используеться для выставления ограничения на размер таймаута
|
||||
static Poco::Timespan saturate(const Poco::Timespan & v, const Poco::Timespan & limit);
|
||||
|
||||
public:
|
||||
/// Соединения с удалёнными серверами.
|
||||
ConnectionPools pools;
|
||||
|
||||
@ -33,15 +37,13 @@ public:
|
||||
{
|
||||
/// contains names of directories for asynchronous write to StorageDistributed
|
||||
std::vector<std::string> dir_names;
|
||||
UInt32 shard_num;
|
||||
int weight;
|
||||
size_t num_local_nodes;
|
||||
};
|
||||
std::vector<ShardInfo> shard_info_vec;
|
||||
std::vector<size_t> slot_to_shard;
|
||||
|
||||
/// используеться для выставления ограничения на размер таймаута
|
||||
static Poco::Timespan saturate(const Poco::Timespan & v, const Poco::Timespan & limit);
|
||||
|
||||
struct Address
|
||||
{
|
||||
/** В конфиге адреса либо находятся в узлах <node>:
|
||||
@ -61,8 +63,10 @@ public:
|
||||
* </shard>
|
||||
*/
|
||||
Poco::Net::SocketAddress host_port;
|
||||
String host_name;
|
||||
String user;
|
||||
String password;
|
||||
UInt32 replica_num;
|
||||
|
||||
Address(const String & config_prefix);
|
||||
Address(const String & host_port_, const String & user_, const String & password_);
|
||||
@ -71,12 +75,18 @@ public:
|
||||
private:
|
||||
static bool isLocal(const Address & address);
|
||||
|
||||
public:
|
||||
/// Массив шардов. Каждый шард - адреса одного сервера.
|
||||
typedef std::vector<Address> Addresses;
|
||||
|
||||
/// Массив шардов. Для каждого шарда - массив адресов реплик (серверов, считающихся идентичными).
|
||||
typedef std::vector<Addresses> AddressesWithFailover;
|
||||
|
||||
public:
|
||||
const Addresses & getShardsInfo() const { return addresses; }
|
||||
const AddressesWithFailover & getShardsWithFailoverInfo() const { return addresses_with_failover; }
|
||||
|
||||
private:
|
||||
Addresses addresses;
|
||||
AddressesWithFailover addresses_with_failover;
|
||||
|
||||
|
@ -39,6 +39,7 @@ class ProcessList;
|
||||
class ProcessListElement;
|
||||
class Macros;
|
||||
class Progress;
|
||||
class Clusters;
|
||||
|
||||
|
||||
/// имя таблицы -> таблица
|
||||
@ -238,6 +239,7 @@ public:
|
||||
|
||||
void initClusters();
|
||||
Cluster & getCluster(const std::string & cluster_name);
|
||||
Poco::SharedPtr<Clusters> getClusters() const;
|
||||
|
||||
Compiler & getCompiler();
|
||||
|
||||
|
@ -134,13 +134,19 @@ private:
|
||||
|
||||
public:
|
||||
/// Справочники будут обновляться в отдельном потоке, каждые reload_period секунд.
|
||||
Dictionaries(const bool throw_on_error, const int reload_period_ = 3600)
|
||||
Dictionaries(const bool throw_on_error, const int reload_period_)
|
||||
: reload_period(reload_period_), log(&Logger::get("Dictionaries"))
|
||||
{
|
||||
reloadImpl(throw_on_error);
|
||||
reloading_thread = std::thread([this] { reloadPeriodically(); });
|
||||
}
|
||||
|
||||
Dictionaries(const bool throw_on_error)
|
||||
: Dictionaries(throw_on_error,
|
||||
Application::instance().config()
|
||||
.getInt("builtin_dictionaries_reload_interval", 3600))
|
||||
{}
|
||||
|
||||
~Dictionaries()
|
||||
{
|
||||
destroy.set();
|
||||
|
@ -2,14 +2,17 @@
|
||||
|
||||
#include <DB/Core/QueryProcessingStage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||
#include <DB/Interpreters/ExpressionActions.h>
|
||||
#include <DB/DataStreams/IBlockInputStream.h>
|
||||
#include <DB/Parsers/ASTSelectQuery.h>
|
||||
#include <DB/TableFunctions/ITableFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ExpressionAnalyzer;
|
||||
class ASTSelectQuery;
|
||||
class SubqueryForSet;
|
||||
|
||||
|
||||
/** Интерпретирует запрос SELECT. Возвращает поток блоков с результатами выполнения запроса до стадии to_stage.
|
||||
*/
|
||||
class InterpreterSelectQuery
|
||||
@ -58,6 +61,8 @@ public:
|
||||
size_t subquery_depth_ = 0,
|
||||
BlockInputStreamPtr input = nullptr);
|
||||
|
||||
~InterpreterSelectQuery();
|
||||
|
||||
/** Выполнить запрос, возможно являющиийся цепочкой UNION ALL.
|
||||
* Получить поток блоков для чтения
|
||||
*/
|
||||
@ -121,9 +126,9 @@ private:
|
||||
void executeLimit( BlockInputStreams & streams);
|
||||
void executeProjection( BlockInputStreams & streams, ExpressionActionsPtr expression);
|
||||
void executeDistinct( BlockInputStreams & streams, bool before_order, Names columns);
|
||||
void executeSubqueriesInSetsAndJoins(BlockInputStreams & streams, SubqueriesForSets & subqueries_for_sets);
|
||||
void executeSubqueriesInSetsAndJoins(BlockInputStreams & streams, std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
|
||||
|
||||
void ignoreWithTotals() { query.group_by_with_totals = false; }
|
||||
void ignoreWithTotals();
|
||||
|
||||
ASTPtr query_ptr;
|
||||
ASTSelectQuery & query;
|
||||
|
@ -61,7 +61,7 @@ struct Limits
|
||||
M(SettingUInt64, max_subquery_depth, 100) \
|
||||
M(SettingUInt64, max_pipeline_depth, 1000) \
|
||||
M(SettingUInt64, max_ast_depth, 1000) /** Проверяются не во время парсинга, */ \
|
||||
M(SettingUInt64, max_ast_elements, 10000) /** а уже после парсинга запроса. */ \
|
||||
M(SettingUInt64, max_ast_elements, 50000) /** а уже после парсинга запроса. */ \
|
||||
\
|
||||
/** 0 - можно всё. 1 - только запросы на чтение. 2 - только запросы на чтение, а также изменение настроек, кроме настройки readonly. */ \
|
||||
M(SettingUInt64, readonly, 0) \
|
||||
|
@ -120,6 +120,15 @@ struct Settings
|
||||
\
|
||||
/** В запросе INSERT с указанием столбцов, заполнять значения по-умолчанию только для столбцов с явными DEFAULT-ами. */ \
|
||||
M(SettingBool, strict_insert_defaults, 0) \
|
||||
\
|
||||
/** В случае превышения максимального размера mark_cache, удалять только записи, старше чем mark_cache_min_lifetime секунд. */ \
|
||||
M(SettingUInt64, mark_cache_min_lifetime, 10000) \
|
||||
\
|
||||
/** Позволяет использовать больше источников, чем количество потоков - для более равномерного распределения работы по потокам. \
|
||||
* Предполагается, что это временное решение, так как можно будет в будущем сделать количество источников равное количеству потоков, \
|
||||
* но чтобы каждый источник динамически выбирал себе доступную работу. \
|
||||
*/ \
|
||||
M(SettingFloat, max_streams_to_max_threads_ratio, 1) \
|
||||
|
||||
/// Всевозможные ограничения на выполнение запроса.
|
||||
Limits limits;
|
||||
|
@ -53,8 +53,8 @@ private:
|
||||
typedef LRUCache<UInt128, MarksInCompressedFile, UInt128TrivialHash, MarksWeightFunction> Base;
|
||||
|
||||
public:
|
||||
MarkCache(size_t max_size_in_bytes)
|
||||
: Base(max_size_in_bytes) {}
|
||||
MarkCache(size_t max_size_in_bytes, const Delay & expiration_delay)
|
||||
: Base(max_size_in_bytes, expiration_delay) {}
|
||||
|
||||
/// Посчитать ключ от пути к файлу и смещения.
|
||||
static UInt128 hash(const String & path_to_file)
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <DB/Core/Exception.h>
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
#include <Yandex/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -95,6 +96,8 @@ public:
|
||||
|
||||
BackgroundProcessingPool(int size_) : size(size_)
|
||||
{
|
||||
LOG_INFO(&Logger::get("BackgroundProcessingPool"), "Create BackgroundProcessingPool with " << size << " threads");
|
||||
|
||||
threads.resize(size);
|
||||
for (auto & thread : threads)
|
||||
thread = std::thread([this] { threadFunction(); });
|
||||
|
@ -75,8 +75,10 @@ public:
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; }
|
||||
|
||||
bool supportsSampling() const override { return true; }
|
||||
bool supportsFinal() const override { return true; }
|
||||
bool supportsPrewhere() const override { return true; }
|
||||
bool supportsFinal() const override { return true; }
|
||||
bool supportsIndexForIn() const override { return true; }
|
||||
bool supportsParallelReplicas() const override { return true; }
|
||||
|
||||
/// Структура подчинённой таблицы не проверяется и не изменяется.
|
||||
void alter(const AlterCommands & params, const String & database_name, const String & table_name, Context & context) override;
|
||||
|
@ -26,9 +26,11 @@ public:
|
||||
NameAndTypePair getColumn(const String & column_name) const override;
|
||||
bool hasColumn(const String & column_name) const override;
|
||||
|
||||
bool supportsSampling() const override { return data->supportsSampling(); }
|
||||
bool supportsFinal() const override { return data->supportsFinal(); }
|
||||
bool supportsPrewhere() const override { return data->supportsPrewhere(); }
|
||||
bool supportsSampling() const override { return data->supportsSampling(); }
|
||||
bool supportsPrewhere() const override { return data->supportsPrewhere(); }
|
||||
bool supportsFinal() const override { return data->supportsFinal(); }
|
||||
bool supportsIndexForIn() const override { return data->supportsIndexForIn(); }
|
||||
bool supportsParallelReplicas() const override { return data->supportsParallelReplicas(); }
|
||||
|
||||
BlockOutputStreamPtr write(ASTPtr query) override;
|
||||
void drop() override;
|
||||
|
@ -37,12 +37,12 @@ public:
|
||||
|
||||
std::string getName() const override { return "Merge"; }
|
||||
std::string getTableName() const override { return name; }
|
||||
bool supportsSampling() const override { return true; }
|
||||
|
||||
/// Проверка откладывается до метода read. Там проверяется поддержка PREWHERE у использующихся таблиц.
|
||||
bool supportsSampling() const override { return true; }
|
||||
bool supportsPrewhere() const override { return true; }
|
||||
|
||||
bool supportsParallelReplicas() const override { return true; }
|
||||
bool supportsFinal() const override { return true; }
|
||||
bool supportsIndexForIn() const override { return true; }
|
||||
|
||||
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
|
||||
NameAndTypePair getColumn(const String &column_name) const override;
|
||||
|
41
dbms/include/DB/Storages/StorageSystemClusters.h
Normal file
41
dbms/include/DB/Storages/StorageSystemClusters.h
Normal file
@ -0,0 +1,41 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
/** Реализует системную таблицу columns, которая позволяет получить информацию
|
||||
* о столбцах каждой таблицы для всех баз данных.
|
||||
*/
|
||||
class StorageSystemClusters : public IStorage
|
||||
{
|
||||
public:
|
||||
StorageSystemClusters(const std::string & name_, Context & context_);
|
||||
static StoragePtr create(const std::string & name_, Context & context_);
|
||||
|
||||
std::string getName() const override { return "SystemColumns"; }
|
||||
std::string getTableName() const override { return name; }
|
||||
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
ASTPtr query,
|
||||
const Context & context,
|
||||
const Settings & settings,
|
||||
QueryProcessingStage::Enum & processed_stage,
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE,
|
||||
unsigned threads = 1) override;
|
||||
|
||||
private:
|
||||
StorageSystemClusters(const std::string & name_);
|
||||
|
||||
private:
|
||||
const std::string name;
|
||||
NamesAndTypesList columns;
|
||||
Context & context;
|
||||
};
|
||||
|
||||
}
|
@ -24,7 +24,7 @@ class TableFunctionRemote : public ITableFunction
|
||||
{
|
||||
public:
|
||||
/// Максимальное количество различных шардов и максимальное количество реплик одного шарда
|
||||
const size_t MAX_ADDRESSES = 200; /// TODO Перенести в Settings.
|
||||
const size_t MAX_ADDRESSES = 1000; /// TODO Перенести в Settings.
|
||||
|
||||
std::string getName() const override { return "remote"; }
|
||||
|
||||
|
@ -15,6 +15,8 @@
|
||||
#include <DB/AggregateFunctions/AggregateFunctionState.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionMerge.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionDebug.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionSequenceMatch.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionsStatistics.h>
|
||||
|
||||
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||
|
||||
@ -536,6 +538,97 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
else
|
||||
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else if (name == "sequenceMatch")
|
||||
{
|
||||
if (!AggregateFunctionSequenceMatch::sufficientArgs(argument_types.size()))
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return new AggregateFunctionSequenceMatch;
|
||||
}
|
||||
else if (name == "varSamp")
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionVarSamp>(*argument_types[0]);
|
||||
|
||||
if (!res)
|
||||
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return res;
|
||||
}
|
||||
else if (name == "varPop")
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionVarPop>(*argument_types[0]);
|
||||
|
||||
if (!res)
|
||||
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return res;
|
||||
}
|
||||
else if (name == "stddevSamp")
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionStdDevSamp>(*argument_types[0]);
|
||||
|
||||
if (!res)
|
||||
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return res;
|
||||
}
|
||||
else if (name == "stddevPop")
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionStdDevPop>(*argument_types[0]);
|
||||
|
||||
if (!res)
|
||||
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return res;
|
||||
}
|
||||
else if (name == "covarSamp")
|
||||
{
|
||||
if (argument_types.size() != 2)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionCovarSamp>(*argument_types[0], *argument_types[1]);
|
||||
if (!res)
|
||||
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
|
||||
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return res;
|
||||
}
|
||||
else if (name == "covarPop")
|
||||
{
|
||||
if (argument_types.size() != 2)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionCovarPop>(*argument_types[0], *argument_types[1]);
|
||||
if (!res)
|
||||
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
|
||||
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return res;
|
||||
}
|
||||
else if (name == "corr")
|
||||
{
|
||||
if (argument_types.size() != 2)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionCorr>(*argument_types[0], *argument_types[1]);
|
||||
if (!res)
|
||||
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
|
||||
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return res;
|
||||
}
|
||||
else if (recursion_level == 0 && name.size() > strlen("State") && !(strcmp(name.data() + name.size() - strlen("State"), "State")))
|
||||
{
|
||||
/// Для агрегатных функций вида aggState, где agg - имя другой агрегатной функции.
|
||||
@ -630,7 +723,15 @@ const AggregateFunctionFactory::FunctionNames & AggregateFunctionFactory::getFun
|
||||
"quantilesTimingWeighted",
|
||||
"medianTimingWeighted",
|
||||
"quantileDeterministic",
|
||||
"quantilesDeterministic"
|
||||
"quantilesDeterministic",
|
||||
"sequenceMatch",
|
||||
"varSamp",
|
||||
"varPop",
|
||||
"stddevSamp",
|
||||
"stddevPop",
|
||||
"covarSamp",
|
||||
"covarPop",
|
||||
"corr"
|
||||
};
|
||||
|
||||
return names;
|
||||
|
@ -60,6 +60,7 @@
|
||||
#include <DB/Common/formatReadable.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
|
||||
#include <statdaemons/NetException.h>
|
||||
|
||||
/// http://en.wikipedia.org/wiki/ANSI_escape_code
|
||||
#define SAVE_CURSOR_POSITION "\033[s"
|
||||
@ -827,7 +828,7 @@ private:
|
||||
return false;
|
||||
|
||||
default:
|
||||
throw Exception("Unexpected packet from server (expected Data, got "
|
||||
throw NetException("Unexpected packet from server (expected Data, got "
|
||||
+ String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
|
||||
}
|
||||
}
|
||||
|
@ -21,6 +21,8 @@
|
||||
|
||||
#include <DB/Client/Connection.h>
|
||||
|
||||
#include <statdaemons/NetException.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -58,14 +60,14 @@ void Connection::connect()
|
||||
disconnect();
|
||||
|
||||
/// Добавляем в сообщение адрес сервера. Также объект Exception запомнит stack trace. Жаль, что более точный тип исключения теряется.
|
||||
throw Exception(e.displayText(), "(" + getServerAddress() + ")", ErrorCodes::NETWORK_ERROR);
|
||||
throw NetException(e.displayText(), "(" + getServerAddress() + ")", ErrorCodes::NETWORK_ERROR);
|
||||
}
|
||||
catch (Poco::TimeoutException & e)
|
||||
{
|
||||
disconnect();
|
||||
|
||||
/// Добавляем в сообщение адрес сервера. Также объект Exception запомнит stack trace. Жаль, что более точный тип исключения теряется.
|
||||
throw Exception(e.displayText(), "(" + getServerAddress() + ")", ErrorCodes::SOCKET_TIMEOUT);
|
||||
throw NetException(e.displayText(), "(" + getServerAddress() + ")", ErrorCodes::SOCKET_TIMEOUT);
|
||||
}
|
||||
}
|
||||
|
||||
@ -125,7 +127,7 @@ void Connection::receiveHello()
|
||||
/// Закроем соединение, чтобы не было рассинхронизации.
|
||||
disconnect();
|
||||
|
||||
throw Exception("Unexpected packet from server " + getServerAddress() + " (expected Hello or Exception, got "
|
||||
throw NetException("Unexpected packet from server " + getServerAddress() + " (expected Hello or Exception, got "
|
||||
+ String(Protocol::Server::toString(packet_type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
|
||||
}
|
||||
}
|
||||
|
@ -1,8 +1,75 @@
|
||||
#include <DB/Common/LRUCache.h>
|
||||
|
||||
#include <iostream>
|
||||
#include <string>
|
||||
#include <thread>
|
||||
#include <chrono>
|
||||
|
||||
using namespace DB;
|
||||
namespace
|
||||
{
|
||||
|
||||
void run();
|
||||
void runTest(unsigned int num, const std::function<bool()> func);
|
||||
bool test1();
|
||||
bool test2();
|
||||
|
||||
#define ASSERT_CHECK(cond, res) \
|
||||
do \
|
||||
{ \
|
||||
if (!(cond)) \
|
||||
{ \
|
||||
std::cout << __FILE__ << ":" << __LINE__ << ":" \
|
||||
<< "Assertion " << #cond << " failed.\n"; \
|
||||
if ((res)) { (res) = false; } \
|
||||
} \
|
||||
} \
|
||||
while (0)
|
||||
|
||||
void run()
|
||||
{
|
||||
const std::vector<std::function<bool()> > tests =
|
||||
{
|
||||
test1,
|
||||
test2
|
||||
};
|
||||
|
||||
unsigned int num = 0;
|
||||
for (const auto & test : tests)
|
||||
{
|
||||
++num;
|
||||
runTest(num, test);
|
||||
}
|
||||
}
|
||||
|
||||
void runTest(unsigned int num, const std::function<bool()> func)
|
||||
{
|
||||
bool ok;
|
||||
|
||||
try
|
||||
{
|
||||
ok = func();
|
||||
}
|
||||
catch (const DB::Exception & ex)
|
||||
{
|
||||
ok = false;
|
||||
std::cout << "Caught exception " << ex.displayText() << "\n";
|
||||
}
|
||||
catch (const std::exception & ex)
|
||||
{
|
||||
ok = false;
|
||||
std::cout << "Caught exception " << ex.what() << "\n";
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
ok = false;
|
||||
std::cout << "Caught unhandled exception\n";
|
||||
}
|
||||
|
||||
if (ok)
|
||||
std::cout << "Test " << num << " passed\n";
|
||||
else
|
||||
std::cout << "Test " << num << " failed\n";
|
||||
}
|
||||
|
||||
struct Weight
|
||||
{
|
||||
@ -12,52 +79,108 @@ struct Weight
|
||||
}
|
||||
};
|
||||
|
||||
void fail()
|
||||
bool test1()
|
||||
{
|
||||
std::cout << "failed" << std::endl;
|
||||
exit(1);
|
||||
using Cache = DB::LRUCache<std::string, std::string, std::hash<std::string>, Weight>;
|
||||
using MappedPtr = Cache::MappedPtr;
|
||||
|
||||
auto ptr = [](const std::string & s)
|
||||
{
|
||||
return MappedPtr(new std::string(s));
|
||||
};
|
||||
|
||||
Cache cache(10);
|
||||
|
||||
bool res = true;
|
||||
|
||||
ASSERT_CHECK(!cache.get("asd"), res);
|
||||
|
||||
cache.set("asd", ptr("qwe"));
|
||||
|
||||
ASSERT_CHECK((*cache.get("asd") == "qwe"), res);
|
||||
|
||||
cache.set("zxcv", ptr("12345"));
|
||||
cache.set("01234567891234567", ptr("--"));
|
||||
|
||||
ASSERT_CHECK((*cache.get("zxcv") == "12345"), res);
|
||||
ASSERT_CHECK((*cache.get("asd") == "qwe"), res);
|
||||
ASSERT_CHECK((*cache.get("01234567891234567") == "--"), res);
|
||||
ASSERT_CHECK(!cache.get("123x"), res);
|
||||
|
||||
cache.set("321x", ptr("+"));
|
||||
|
||||
ASSERT_CHECK(!cache.get("zxcv"), res);
|
||||
ASSERT_CHECK((*cache.get("asd") == "qwe"), res);
|
||||
ASSERT_CHECK((*cache.get("01234567891234567") == "--"), res);
|
||||
ASSERT_CHECK(!cache.get("123x"), res);
|
||||
ASSERT_CHECK((*cache.get("321x") == "+"), res);
|
||||
|
||||
ASSERT_CHECK((cache.weight() == 6), res);
|
||||
ASSERT_CHECK((cache.count() == 3), res);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
typedef LRUCache<std::string, std::string, std::hash<std::string>, Weight> Cache;
|
||||
typedef Cache::MappedPtr MappedPtr;
|
||||
|
||||
MappedPtr ptr(const std::string & s)
|
||||
bool test2()
|
||||
{
|
||||
return MappedPtr(new std::string(s));
|
||||
using namespace std::literals;
|
||||
using Cache = DB::LRUCache<std::string, std::string, std::hash<std::string>, Weight>;
|
||||
using MappedPtr = Cache::MappedPtr;
|
||||
|
||||
auto ptr = [](const std::string & s)
|
||||
{
|
||||
return MappedPtr(new std::string(s));
|
||||
};
|
||||
|
||||
Cache cache(10, 3s);
|
||||
|
||||
bool res = true;
|
||||
|
||||
ASSERT_CHECK(!cache.get("asd"), res);
|
||||
|
||||
cache.set("asd", ptr("qwe"));
|
||||
|
||||
ASSERT_CHECK((*cache.get("asd") == "qwe"), res);
|
||||
|
||||
cache.set("zxcv", ptr("12345"));
|
||||
cache.set("01234567891234567", ptr("--"));
|
||||
|
||||
ASSERT_CHECK((*cache.get("zxcv") == "12345"), res);
|
||||
ASSERT_CHECK((*cache.get("asd") == "qwe"), res);
|
||||
ASSERT_CHECK((*cache.get("01234567891234567") == "--"), res);
|
||||
ASSERT_CHECK(!cache.get("123x"), res);
|
||||
|
||||
cache.set("321x", ptr("+"));
|
||||
|
||||
ASSERT_CHECK((cache.get("zxcv")), res);
|
||||
ASSERT_CHECK((*cache.get("asd") == "qwe"), res);
|
||||
ASSERT_CHECK((*cache.get("01234567891234567") == "--"), res);
|
||||
ASSERT_CHECK(!cache.get("123x"), res);
|
||||
ASSERT_CHECK((*cache.get("321x") == "+"), res);
|
||||
|
||||
ASSERT_CHECK((cache.weight() == 11), res);
|
||||
ASSERT_CHECK((cache.count() == 4), res);
|
||||
|
||||
std::this_thread::sleep_for(5s);
|
||||
|
||||
cache.set("123x", ptr("2769"));
|
||||
|
||||
ASSERT_CHECK(!cache.get("zxcv"), res);
|
||||
ASSERT_CHECK((*cache.get("asd") == "qwe"), res);
|
||||
ASSERT_CHECK((*cache.get("01234567891234567") == "--"), res);
|
||||
ASSERT_CHECK((*cache.get("321x") == "+"), res);
|
||||
|
||||
ASSERT_CHECK((cache.weight() == 10), res);
|
||||
ASSERT_CHECK((cache.count() == 4), res);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
int main()
|
||||
{
|
||||
try
|
||||
{
|
||||
Cache cache(10);
|
||||
|
||||
if (cache.get("asd")) fail();
|
||||
cache.set("asd", ptr("qwe"));
|
||||
if (*cache.get("asd") != "qwe") fail();
|
||||
cache.set("zxcv", ptr("12345"));
|
||||
cache.set("01234567891234567", ptr("--"));
|
||||
if (*cache.get("zxcv") != "12345") fail();
|
||||
if (*cache.get("asd") != "qwe") fail();
|
||||
if (*cache.get("01234567891234567") != "--") fail();
|
||||
if (cache.get("123x")) fail();
|
||||
cache.set("321x", ptr("+"));
|
||||
if (cache.get("zxcv")) fail();
|
||||
|
||||
if (*cache.get("asd") != "qwe") fail();
|
||||
if (*cache.get("01234567891234567") != "--") fail();
|
||||
if (cache.get("123x")) fail();
|
||||
if (*cache.get("321x") != "+") fail();
|
||||
|
||||
if (cache.weight() != 6) fail();
|
||||
if (cache.count() != 3) fail();
|
||||
|
||||
std::cout << "passed" << std::endl;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
fail();
|
||||
}
|
||||
|
||||
run();
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
@ -61,16 +61,16 @@ Block AggregatingSortedBlockInputStream::readImpl()
|
||||
columns_to_aggregate[i] = typeid_cast<ColumnAggregateFunction *>(merged_columns[column_numbers_to_aggregate[i]]);
|
||||
|
||||
if (has_collation)
|
||||
merge(merged_block, merged_columns, queue_with_collation);
|
||||
merge(merged_columns, queue_with_collation);
|
||||
else
|
||||
merge(merged_block, merged_columns, queue);
|
||||
merge(merged_columns, queue);
|
||||
|
||||
return merged_block;
|
||||
}
|
||||
|
||||
|
||||
template<class TSortCursor>
|
||||
void AggregatingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue)
|
||||
void AggregatingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue)
|
||||
{
|
||||
size_t merged_rows = 0;
|
||||
|
||||
@ -81,13 +81,15 @@ void AggregatingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainP
|
||||
|
||||
setPrimaryKey(next_key, current);
|
||||
|
||||
bool key_differs = next_key != current_key;
|
||||
|
||||
/// если накопилось достаточно строк и последняя посчитана полностью
|
||||
if (next_key != current_key && merged_rows >= max_block_size)
|
||||
if (key_differs && merged_rows >= max_block_size)
|
||||
return;
|
||||
|
||||
queue.pop();
|
||||
|
||||
if (next_key != current_key)
|
||||
if (key_differs)
|
||||
{
|
||||
current_key = std::move(next_key);
|
||||
next_key.resize(description.size());
|
||||
|
@ -104,15 +104,15 @@ Block CollapsingSortedBlockInputStream::readImpl()
|
||||
}
|
||||
|
||||
if (has_collation)
|
||||
merge(merged_block, merged_columns, queue_with_collation);
|
||||
merge(merged_columns, queue_with_collation);
|
||||
else
|
||||
merge(merged_block, merged_columns, queue);
|
||||
merge(merged_columns, queue);
|
||||
|
||||
return merged_block;
|
||||
}
|
||||
|
||||
template<class TSortCursor>
|
||||
void CollapsingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue)
|
||||
void CollapsingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue)
|
||||
{
|
||||
size_t merged_rows = 0;
|
||||
|
||||
@ -120,12 +120,22 @@ void CollapsingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPt
|
||||
while (!queue.empty())
|
||||
{
|
||||
TSortCursor current = queue.top();
|
||||
queue.pop();
|
||||
|
||||
Int8 sign = get<Int64>((*current->all_columns[sign_column_number])[current->pos]);
|
||||
setPrimaryKey(next_key, current);
|
||||
|
||||
if (next_key != current_key)
|
||||
bool key_differs = next_key != current_key;
|
||||
|
||||
/// если накопилось достаточно строк и последняя посчитана полностью
|
||||
if (key_differs && merged_rows >= max_block_size)
|
||||
{
|
||||
++blocks_written;
|
||||
return;
|
||||
}
|
||||
|
||||
queue.pop();
|
||||
|
||||
if (key_differs)
|
||||
{
|
||||
/// Запишем данные для предыдущего визита.
|
||||
insertRows(merged_columns, merged_rows);
|
||||
@ -168,12 +178,6 @@ void CollapsingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPt
|
||||
/// Достаём из соответствующего источника следующий блок, если есть.
|
||||
fetchNextBlock(current, queue);
|
||||
}
|
||||
|
||||
if (merged_rows >= max_block_size)
|
||||
{
|
||||
++blocks_written;
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
/// Запишем данные для последнего визита.
|
||||
|
@ -32,16 +32,22 @@ Block DistinctBlockInputStream::readImpl()
|
||||
size_t rows = block.rows();
|
||||
size_t columns = columns_names.empty() ? block.columns() : columns_names.size();
|
||||
|
||||
ConstColumnPlainPtrs column_ptrs(columns);
|
||||
ConstColumnPlainPtrs column_ptrs;
|
||||
column_ptrs.reserve(columns);
|
||||
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
if (columns_names.empty())
|
||||
column_ptrs[i] = block.getByPosition(i).column;
|
||||
else
|
||||
column_ptrs[i] = block.getByName(columns_names[i]).column;
|
||||
auto & column = columns_names.empty()
|
||||
? block.getByPosition(i).column
|
||||
: block.getByName(columns_names[i]).column;
|
||||
|
||||
/// Игнорируем все константные столбцы.
|
||||
if (!column->isConst())
|
||||
column_ptrs.emplace_back(column.get());
|
||||
}
|
||||
|
||||
columns = column_ptrs.size();
|
||||
|
||||
/// Будем фильтровать блок, оставляя там только строки, которых мы ещё не видели.
|
||||
IColumn::Filter filter(rows);
|
||||
|
||||
|
@ -248,7 +248,14 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value)
|
||||
ErrorCodes::TOO_MUCH_ROWS);
|
||||
}
|
||||
else if (limits.read_overflow_mode == OverflowMode::BREAK)
|
||||
cancel();
|
||||
{
|
||||
/// Для break будем останавливаться только если действительно было прочитано столько строк, а не только предполагается к чтению.
|
||||
if ((limits.max_rows_to_read && rows_processed > limits.max_rows_to_read)
|
||||
|| (limits.max_bytes_to_read && bytes_processed > limits.max_bytes_to_read))
|
||||
{
|
||||
cancel();
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
@ -111,15 +111,15 @@ Block MergingSortedBlockInputStream::readImpl()
|
||||
return Block();
|
||||
|
||||
if (has_collation)
|
||||
merge(merged_block, merged_columns, queue_with_collation);
|
||||
merge(merged_columns, queue_with_collation);
|
||||
else
|
||||
merge(merged_block, merged_columns, queue);
|
||||
merge(merged_columns, queue);
|
||||
|
||||
return merged_block;
|
||||
}
|
||||
|
||||
template <typename TSortCursor>
|
||||
void MergingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue)
|
||||
void MergingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue)
|
||||
{
|
||||
size_t merged_rows = 0;
|
||||
|
||||
|
@ -142,16 +142,16 @@ Block SummingSortedBlockInputStream::readImpl()
|
||||
}
|
||||
|
||||
if (has_collation)
|
||||
merge(merged_block, merged_columns, queue_with_collation);
|
||||
merge(merged_columns, queue_with_collation);
|
||||
else
|
||||
merge(merged_block, merged_columns, queue);
|
||||
merge(merged_columns, queue);
|
||||
|
||||
return merged_block;
|
||||
}
|
||||
|
||||
|
||||
template<class TSortCursor>
|
||||
void SummingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue)
|
||||
void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue)
|
||||
{
|
||||
size_t merged_rows = 0;
|
||||
|
||||
@ -159,11 +159,18 @@ void SummingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPtrs
|
||||
while (!queue.empty())
|
||||
{
|
||||
TSortCursor current = queue.top();
|
||||
queue.pop();
|
||||
|
||||
setPrimaryKey(next_key, current);
|
||||
|
||||
if (next_key != current_key)
|
||||
bool key_differs = next_key != current_key;
|
||||
|
||||
/// если накопилось достаточно строк и последняя посчитана полностью
|
||||
if (key_differs && merged_rows >= max_block_size)
|
||||
return;
|
||||
|
||||
queue.pop();
|
||||
|
||||
if (key_differs)
|
||||
{
|
||||
/// Запишем данные для предыдущей группы.
|
||||
if (!current_key[0].isNull() && !current_row_is_zero)
|
||||
@ -194,9 +201,6 @@ void SummingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPtrs
|
||||
/// Достаём из соответствующего источника следующий блок, если есть.
|
||||
fetchNextBlock(current, queue);
|
||||
}
|
||||
|
||||
if (merged_rows >= max_block_size)
|
||||
return;
|
||||
}
|
||||
|
||||
/// Запишем данные для последней группы, если она ненулевая.
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <math.h>
|
||||
|
||||
#include <DB/Functions/NumberTraits.h>
|
||||
#include <DB/Functions/FunctionFactory.h>
|
||||
#include <DB/Functions/FunctionsArithmetic.h>
|
||||
#include <DB/Functions/FunctionsMiscellaneous.h>
|
||||
@ -308,6 +309,71 @@ void FunctionVisibleWidth::execute(Block & block, const ColumnNumbers & argument
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
|
||||
/// TODO: Убрать copy-paste из FunctionsConditional.h
|
||||
template <typename T>
|
||||
struct DataTypeFromFieldTypeOrError
|
||||
{
|
||||
static DataTypePtr getDataType()
|
||||
{
|
||||
return new typename DataTypeFromFieldType<T>::Type;
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct DataTypeFromFieldTypeOrError<NumberTraits::Error>
|
||||
{
|
||||
static DataTypePtr getDataType()
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
};
|
||||
|
||||
template <typename T1, typename T2>
|
||||
DataTypePtr getSmallestCommonNumericTypeImpl()
|
||||
{
|
||||
using ResultType = typename NumberTraits::ResultOfIf<T1, T2>::Type;
|
||||
auto type_res = DataTypeFromFieldTypeOrError<ResultType>::getDataType();
|
||||
if (!type_res)
|
||||
throw Exception("Types " + TypeName<T1>::get() + " and " + TypeName<T2>::get()
|
||||
+ " are not upscalable to a common type without loss of precision", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return type_res;
|
||||
}
|
||||
|
||||
template <typename T1>
|
||||
DataTypePtr getSmallestCommonNumericTypeLeft(const IDataType & t2)
|
||||
{
|
||||
if (typeid_cast<const DataTypeUInt8 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt8>();
|
||||
if (typeid_cast<const DataTypeUInt16 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt16>();
|
||||
if (typeid_cast<const DataTypeUInt32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt32>();
|
||||
if (typeid_cast<const DataTypeUInt64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt64>();
|
||||
if (typeid_cast<const DataTypeInt8 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int8>();
|
||||
if (typeid_cast<const DataTypeInt16 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int16>();
|
||||
if (typeid_cast<const DataTypeInt32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int32>();
|
||||
if (typeid_cast<const DataTypeInt64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int64>();
|
||||
if (typeid_cast<const DataTypeFloat32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Float32>();
|
||||
if (typeid_cast<const DataTypeFloat64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Float64>();
|
||||
|
||||
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
DataTypePtr getSmallestCommonNumericType(const IDataType & t1, const IDataType & t2)
|
||||
{
|
||||
if (typeid_cast<const DataTypeUInt8 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt8>(t2);
|
||||
if (typeid_cast<const DataTypeUInt16 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt16>(t2);
|
||||
if (typeid_cast<const DataTypeUInt32 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt32>(t2);
|
||||
if (typeid_cast<const DataTypeUInt64 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt64>(t2);
|
||||
if (typeid_cast<const DataTypeInt8 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int8>(t2);
|
||||
if (typeid_cast<const DataTypeInt16 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int16>(t2);
|
||||
if (typeid_cast<const DataTypeInt32 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int32>(t2);
|
||||
if (typeid_cast<const DataTypeInt64 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int64>(t2);
|
||||
if (typeid_cast<const DataTypeFloat32 *>(&t1)) return getSmallestCommonNumericTypeLeft<Float32>(t2);
|
||||
if (typeid_cast<const DataTypeFloat64 *>(&t1)) return getSmallestCommonNumericTypeLeft<Float64>(t2);
|
||||
|
||||
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
@ -337,6 +403,8 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionIsFinite>();
|
||||
factory.registerFunction<FunctionIsInfinite>();
|
||||
factory.registerFunction<FunctionIsNaN>();
|
||||
|
||||
factory.registerFunction<FunctionTransform>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -9,6 +9,9 @@ void registerFunctionsRound(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionRoundToExp2>();
|
||||
factory.registerFunction<FunctionRoundDuration>();
|
||||
factory.registerFunction<FunctionRoundAge>();
|
||||
factory.registerFunction<FunctionRound>();
|
||||
factory.registerFunction<FunctionCeil>();
|
||||
factory.registerFunction<FunctionFloor>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -28,8 +28,6 @@ ReadBufferAIO::ReadBufferAIO(const std::string & filename_, size_t buffer_size_,
|
||||
auto error_code = (errno == ENOENT) ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE;
|
||||
throwFromErrno("Cannot open file " + filename, error_code);
|
||||
}
|
||||
|
||||
::memset(&request, 0, sizeof(request));
|
||||
}
|
||||
|
||||
ReadBufferAIO::~ReadBufferAIO()
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
|
||||
#include <DB/IO/ReadBufferFromPocoSocket.h>
|
||||
#include <statdaemons/NetException.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -20,19 +21,19 @@ bool ReadBufferFromPocoSocket::nextImpl()
|
||||
}
|
||||
catch (const Poco::Net::NetException & e)
|
||||
{
|
||||
throw Exception(e.displayText(), "while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR);
|
||||
throw NetException(e.displayText(), "while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR);
|
||||
}
|
||||
catch (const Poco::TimeoutException & e)
|
||||
{
|
||||
throw Exception("Timeout exceeded while reading from socket (" + peer_address.toString() + ")", ErrorCodes::SOCKET_TIMEOUT);
|
||||
throw NetException("Timeout exceeded while reading from socket (" + peer_address.toString() + ")", ErrorCodes::SOCKET_TIMEOUT);
|
||||
}
|
||||
catch (const Poco::IOException & e)
|
||||
{
|
||||
throw Exception(e.displayText(), "while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR);
|
||||
throw NetException(e.displayText(), "while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR);
|
||||
}
|
||||
|
||||
if (bytes_read < 0)
|
||||
throw Exception("Cannot read from socket (" + peer_address.toString() + ")", ErrorCodes::CANNOT_READ_FROM_SOCKET);
|
||||
throw NetException("Cannot read from socket (" + peer_address.toString() + ")", ErrorCodes::CANNOT_READ_FROM_SOCKET);
|
||||
|
||||
if (bytes_read)
|
||||
working_buffer.resize(bytes_read);
|
||||
|
@ -34,10 +34,6 @@ WriteBufferAIO::WriteBufferAIO(const std::string & filename_, size_t buffer_size
|
||||
auto error_code = (errno == ENOENT) ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE;
|
||||
throwFromErrno("Cannot open file " + filename, error_code);
|
||||
}
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::FileOpen);
|
||||
|
||||
::memset(&request, 0, sizeof(request));
|
||||
}
|
||||
|
||||
WriteBufferAIO::~WriteBufferAIO()
|
||||
@ -63,15 +59,6 @@ off_t WriteBufferAIO::getPositionInFile()
|
||||
return seek(0, SEEK_CUR);
|
||||
}
|
||||
|
||||
void WriteBufferAIO::truncate(off_t length)
|
||||
{
|
||||
flush();
|
||||
|
||||
int res = ::ftruncate(fd, length);
|
||||
if (res == -1)
|
||||
throwFromErrno("Cannot truncate file " + filename, ErrorCodes::CANNOT_TRUNCATE_FILE);
|
||||
}
|
||||
|
||||
void WriteBufferAIO::sync()
|
||||
{
|
||||
flush();
|
||||
@ -142,6 +129,15 @@ off_t WriteBufferAIO::doSeek(off_t off, int whence)
|
||||
return pos_in_file;
|
||||
}
|
||||
|
||||
void WriteBufferAIO::doTruncate(off_t length)
|
||||
{
|
||||
flush();
|
||||
|
||||
int res = ::ftruncate(fd, length);
|
||||
if (res == -1)
|
||||
throwFromErrno("Cannot truncate file " + filename, ErrorCodes::CANNOT_TRUNCATE_FILE);
|
||||
}
|
||||
|
||||
void WriteBufferAIO::flush()
|
||||
{
|
||||
next();
|
||||
@ -342,14 +338,21 @@ void WriteBufferAIO::prepare()
|
||||
if (read_count < 0)
|
||||
throw Exception("Read error", ErrorCodes::AIO_READ_ERROR);
|
||||
|
||||
Position truncation_begin;
|
||||
off_t offset = DEFAULT_AIO_FILE_BLOCK_SIZE - region_right_padding;
|
||||
if (read_count > offset)
|
||||
{
|
||||
::memcpy(buffer_end, memory_page + offset, read_count - offset);
|
||||
truncation_begin = buffer_end + (read_count - offset);
|
||||
truncation_count = DEFAULT_AIO_FILE_BLOCK_SIZE - read_count;
|
||||
}
|
||||
else
|
||||
{
|
||||
truncation_begin = buffer_end;
|
||||
truncation_count = region_right_padding;
|
||||
}
|
||||
|
||||
::memset(truncation_begin, 0, truncation_count);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -17,4 +17,9 @@ off_t WriteBufferFromFileBase::seek(off_t off, int whence)
|
||||
return doSeek(off, whence);
|
||||
}
|
||||
|
||||
void WriteBufferFromFileBase::truncate(off_t length)
|
||||
{
|
||||
return doTruncate(length);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
|
||||
#include <DB/IO/WriteBufferFromPocoSocket.h>
|
||||
#include <statdaemons/NetException.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -26,19 +27,19 @@ void WriteBufferFromPocoSocket::nextImpl()
|
||||
}
|
||||
catch (const Poco::Net::NetException & e)
|
||||
{
|
||||
throw Exception(e.displayText() + " while writing to socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR);
|
||||
throw NetException(e.displayText() + " while writing to socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR);
|
||||
}
|
||||
catch (const Poco::TimeoutException & e)
|
||||
{
|
||||
throw Exception("Timeout exceeded while writing to socket (" + peer_address.toString() + ")", ErrorCodes::SOCKET_TIMEOUT);
|
||||
throw NetException("Timeout exceeded while writing to socket (" + peer_address.toString() + ")", ErrorCodes::SOCKET_TIMEOUT);
|
||||
}
|
||||
catch (const Poco::IOException & e)
|
||||
{
|
||||
throw Exception(e.displayText(), " while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR);
|
||||
throw NetException(e.displayText(), " while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR);
|
||||
}
|
||||
|
||||
if (res < 0)
|
||||
throw Exception("Cannot write to socket (" + peer_address.toString() + ")", ErrorCodes::CANNOT_WRITE_TO_SOCKET);
|
||||
throw NetException("Cannot write to socket (" + peer_address.toString() + ")", ErrorCodes::CANNOT_WRITE_TO_SOCKET);
|
||||
bytes_written += res;
|
||||
}
|
||||
}
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <DB/Common/isLocalAddress.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <Poco/Net/NetworkInterface.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -13,8 +12,9 @@ Cluster::Address::Address(const String & config_prefix)
|
||||
{
|
||||
auto & config = Poco::Util::Application::instance().config();
|
||||
|
||||
host_name = config.getString(config_prefix + ".host");
|
||||
host_port = Poco::Net::SocketAddress(
|
||||
config.getString(config_prefix + ".host"),
|
||||
host_name,
|
||||
config.getInt(config_prefix + ".port")
|
||||
);
|
||||
|
||||
@ -68,6 +68,8 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
|
||||
|
||||
const auto & config_prefix = cluster_name + ".";
|
||||
|
||||
UInt32 current_shard_num = 1;
|
||||
|
||||
for (auto it = config_keys.begin(); it != config_keys.end(); ++it)
|
||||
{
|
||||
if (0 == strncmp(it->c_str(), "node", strlen("node")))
|
||||
@ -78,12 +80,13 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
|
||||
continue;
|
||||
|
||||
addresses.emplace_back(prefix);
|
||||
addresses.back().replica_num = 1;
|
||||
|
||||
slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
|
||||
if (const auto is_local = isLocal(addresses.back()))
|
||||
shard_info_vec.push_back({{}, weight, is_local});
|
||||
shard_info_vec.push_back({{}, current_shard_num, weight, is_local});
|
||||
else
|
||||
shard_info_vec.push_back({{addressToDirName(addresses.back())}, weight, is_local});
|
||||
shard_info_vec.push_back({{addressToDirName(addresses.back())}, current_shard_num, weight, is_local});
|
||||
}
|
||||
else if (0 == strncmp(it->c_str(), "shard", strlen("shard")))
|
||||
{
|
||||
@ -92,6 +95,7 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
|
||||
|
||||
addresses_with_failover.emplace_back();
|
||||
Addresses & replica_addresses = addresses_with_failover.back();
|
||||
UInt32 current_replica_num = 1;
|
||||
|
||||
const auto & partial_prefix = config_prefix + *it + ".";
|
||||
const auto weight = config.getInt(partial_prefix + ".weight", 1);
|
||||
@ -116,6 +120,8 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
|
||||
if (0 == strncmp(jt->c_str(), "replica", strlen("replica")))
|
||||
{
|
||||
replica_addresses.emplace_back(partial_prefix + *jt);
|
||||
replica_addresses.back().replica_num = current_replica_num;
|
||||
++current_replica_num;
|
||||
|
||||
if (isLocal(replica_addresses.back()))
|
||||
{
|
||||
@ -142,10 +148,12 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
|
||||
}
|
||||
|
||||
slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
|
||||
shard_info_vec.push_back({std::move(dir_names), weight, num_local_nodes});
|
||||
shard_info_vec.push_back({std::move(dir_names), current_shard_num, weight, num_local_nodes});
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown element in config: " + *it, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
|
||||
++current_shard_num;
|
||||
}
|
||||
|
||||
if (!addresses_with_failover.empty() && !addresses.empty())
|
||||
|
@ -191,6 +191,8 @@ void Compiler::compile(
|
||||
CodeGenerator get_code,
|
||||
ReadyCallback on_ready)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::CompileAttempt);
|
||||
|
||||
std::string prefix = path + "/" + file_name;
|
||||
std::string cpp_file_path = prefix + ".cpp";
|
||||
std::string so_file_path = prefix + ".so";
|
||||
@ -257,6 +259,7 @@ void Compiler::compile(
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Compiled code " << file_name);
|
||||
ProfileEvents::increment(ProfileEvents::CompileSuccess);
|
||||
|
||||
on_ready(lib);
|
||||
}
|
||||
|
@ -1,10 +1,10 @@
|
||||
#include <map>
|
||||
#include <set>
|
||||
#include <chrono>
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
#include <Poco/Mutex.h>
|
||||
#include <Poco/File.h>
|
||||
#include <Poco/Net/NetworkInterface.h>
|
||||
|
||||
#include <Yandex/logger_useful.h>
|
||||
|
||||
@ -730,7 +730,7 @@ void Context::setMarkCache(size_t cache_size_in_bytes)
|
||||
if (shared->mark_cache)
|
||||
throw Exception("Uncompressed cache has been already created.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
shared->mark_cache.reset(new MarkCache(cache_size_in_bytes));
|
||||
shared->mark_cache.reset(new MarkCache(cache_size_in_bytes, std::chrono::seconds(settings.mark_cache_min_lifetime)));
|
||||
}
|
||||
|
||||
MarkCachePtr Context::getMarkCache() const
|
||||
@ -815,6 +815,12 @@ Cluster & Context::getCluster(const std::string & cluster_name)
|
||||
throw Poco::Exception("Failed to find cluster with name = " + cluster_name);
|
||||
}
|
||||
|
||||
Poco::SharedPtr<Clusters> Context::getClusters() const
|
||||
{
|
||||
if (!shared->clusters)
|
||||
throw Poco::Exception("Clusters have not been initialized yet.");
|
||||
return shared->clusters;
|
||||
}
|
||||
|
||||
Compiler & Context::getCompiler()
|
||||
{
|
||||
|
@ -215,7 +215,7 @@ void ExpressionAction::prepare(Block & sample_block)
|
||||
|
||||
void ExpressionAction::execute(Block & block) const
|
||||
{
|
||||
std::cerr << "executing: " << toString() << std::endl;
|
||||
// std::cerr << "executing: " << toString() << std::endl;
|
||||
|
||||
if (type == REMOVE_COLUMN || type == COPY_COLUMN)
|
||||
if (!block.has(source_name))
|
||||
@ -327,7 +327,7 @@ void ExpressionAction::execute(Block & block) const
|
||||
break;
|
||||
|
||||
case ADD_COLUMN:
|
||||
block.insert(ColumnWithNameAndType(added_column->cloneResized(block.rows()), result_type, result_name));
|
||||
block.insert(ColumnWithNameAndType(added_column->cloneResized(block.rowsInFirstColumn()), result_type, result_name));
|
||||
break;
|
||||
|
||||
case COPY_COLUMN:
|
||||
@ -608,6 +608,8 @@ std::string ExpressionActions::getSmallestColumn(const NamesAndTypesList & colum
|
||||
|
||||
void ExpressionActions::finalize(const Names & output_columns)
|
||||
{
|
||||
// std::cerr << "finalize\n";
|
||||
|
||||
NameSet final_columns;
|
||||
for (size_t i = 0; i < output_columns.size(); ++i)
|
||||
{
|
||||
@ -629,7 +631,7 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
unmodified_columns.insert(it->name);
|
||||
}
|
||||
|
||||
/// Будем идти с конца и поодерживать множество нужных на данном этапе столбцов.
|
||||
/// Будем идти с конца и поддерживать множество нужных на данном этапе столбцов.
|
||||
/// Будем выбрасывать ненужные действия, хотя обычно их нет по построению.
|
||||
for (int i = static_cast<int>(actions.size()) - 1; i >= 0; --i)
|
||||
{
|
||||
@ -691,6 +693,23 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
|
||||
unmodified_columns.erase(out);
|
||||
needed_columns.erase(out);
|
||||
|
||||
/** Если функция - константное выражение, то заменим действие на добавление столбца-константы - результата.
|
||||
* То есть, осуществляем constant folding.
|
||||
*/
|
||||
if (action.type == ExpressionAction::APPLY_FUNCTION && sample_block.has(out))
|
||||
{
|
||||
auto & result = sample_block.getByName(out);
|
||||
if (!result.column.isNull())
|
||||
{
|
||||
action.type = ExpressionAction::ADD_COLUMN;
|
||||
action.result_type = result.type;
|
||||
action.added_column = result.column;
|
||||
action.function = nullptr;
|
||||
action.argument_names.clear();
|
||||
in.clear();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
needed_columns.insert(in.begin(), in.end());
|
||||
|
@ -21,6 +21,7 @@
|
||||
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
||||
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||
#include <DB/Interpreters/LogicalExpressionsOptimizer.h>
|
||||
#include <DB/Interpreters/ExternalDictionaries.h>
|
||||
|
||||
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||
|
||||
@ -32,6 +33,8 @@
|
||||
#include <DB/DataStreams/LazyBlockInputStream.h>
|
||||
#include <DB/DataStreams/copyData.h>
|
||||
|
||||
#include <DB/Dictionaries/IDictionary.h>
|
||||
|
||||
#include <DB/Common/typeid_cast.h>
|
||||
|
||||
#include <DB/Parsers/formatAST.h>
|
||||
@ -67,6 +70,23 @@ const std::unordered_set<String> injective_function_names
|
||||
"regionToName",
|
||||
};
|
||||
|
||||
const std::unordered_set<String> possibly_injective_function_names
|
||||
{
|
||||
"dictGetString",
|
||||
"dictGetUInt8",
|
||||
"dictGetUInt16",
|
||||
"dictGetUInt32",
|
||||
"dictGetUInt64",
|
||||
"dictGetInt8",
|
||||
"dictGetInt16",
|
||||
"dictGetInt32",
|
||||
"dictGetInt64",
|
||||
"dictGetFloat32",
|
||||
"dictGetFloat64",
|
||||
"dictGetDate",
|
||||
"dictGetDateTime"
|
||||
};
|
||||
|
||||
void ExpressionAnalyzer::init()
|
||||
{
|
||||
select_query = typeid_cast<ASTSelectQuery *>(&*ast);
|
||||
@ -328,7 +348,8 @@ void ExpressionAnalyzer::normalizeTree()
|
||||
/// finished_asts - уже обработанные вершины (и на что они заменены)
|
||||
/// current_asts - вершины в текущем стеке вызовов этого метода
|
||||
/// current_alias - алиас, повешенный на предка ast (самого глубокого из предков с алиасами)
|
||||
void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOfASTs & current_asts, std::string current_alias)
|
||||
void ExpressionAnalyzer::normalizeTreeImpl(
|
||||
ASTPtr & ast, MapOfASTs & finished_asts, SetOfASTs & current_asts, std::string current_alias)
|
||||
{
|
||||
if (finished_asts.count(ast))
|
||||
{
|
||||
@ -346,25 +367,33 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as
|
||||
/// rewrite правила, которые действуют при обходе сверху-вниз.
|
||||
bool replaced = false;
|
||||
|
||||
if (ASTFunction * node = typeid_cast<ASTFunction *>(&*ast))
|
||||
ASTFunction * func_node = typeid_cast<ASTFunction *>(&*ast);
|
||||
if (func_node)
|
||||
{
|
||||
/** Нет ли в таблице столбца, название которого полностью совпадает с записью функции?
|
||||
* Например, в таблице есть столбец "domain(URL)", и мы запросили domain(URL).
|
||||
*/
|
||||
String function_string = node->getColumnName();
|
||||
String function_string = func_node->getColumnName();
|
||||
NamesAndTypesList::const_iterator it = findColumn(function_string);
|
||||
if (columns.end() != it)
|
||||
{
|
||||
ASTIdentifier * ast_id = new ASTIdentifier(node->range, function_string);
|
||||
ASTIdentifier * ast_id = new ASTIdentifier(func_node->range, function_string);
|
||||
ast = ast_id;
|
||||
current_asts.insert(ast);
|
||||
replaced = true;
|
||||
}
|
||||
|
||||
/// может быть указано IN t, где t - таблица, что равносильно IN (SELECT * FROM t).
|
||||
if (node->name == "in" || node->name == "notIn" || node->name == "globalIn" || node->name == "globalNotIn")
|
||||
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(&*node->arguments->children.at(1)))
|
||||
/// Может быть указано IN t, где t - таблица, что равносильно IN (SELECT * FROM t).
|
||||
if (func_node->name == "in" || func_node->name == "notIn" || func_node->name == "globalIn" || func_node->name == "globalNotIn")
|
||||
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(&*func_node->arguments->children.at(1)))
|
||||
right->kind = ASTIdentifier::Table;
|
||||
|
||||
/// А ещё, в качестве исключения, будем понимать count(*) как count(), а не count(список всех столбцов).
|
||||
if (func_node->name == "count" && func_node->arguments->children.size() == 1
|
||||
&& typeid_cast<const ASTAsterisk *>(func_node->arguments->children[0].get()))
|
||||
{
|
||||
func_node->arguments->children.clear();
|
||||
}
|
||||
}
|
||||
else if (ASTIdentifier * node = typeid_cast<ASTIdentifier *>(&*ast))
|
||||
{
|
||||
@ -427,10 +456,32 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as
|
||||
}
|
||||
|
||||
/// Рекурсивные вызовы. Не опускаемся в подзапросы.
|
||||
/// Также не опускаемся в левый аргумент лямбда-выражений, чтобы не заменять формальные параметры
|
||||
/// по алиасам в выражениях вида 123 AS x, arrayMap(x -> 1, [2]).
|
||||
|
||||
if (func_node && func_node->name == "lambda")
|
||||
{
|
||||
/// Пропускаем первый аргумент. Также предполагаем, что у функции lambda не может быть parameters.
|
||||
for (size_t i = 1, size = func_node->arguments->children.size(); i < size; ++i)
|
||||
{
|
||||
auto & child = func_node->arguments->children[i];
|
||||
|
||||
if (typeid_cast<ASTSelectQuery *>(&*child))
|
||||
continue;
|
||||
|
||||
for (auto & child : ast->children)
|
||||
if (!typeid_cast<ASTSelectQuery *>(&*child))
|
||||
normalizeTreeImpl(child, finished_asts, current_asts, current_alias);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (auto & child : ast->children)
|
||||
{
|
||||
if (typeid_cast<ASTSelectQuery *>(&*child))
|
||||
continue;
|
||||
|
||||
normalizeTreeImpl(child, finished_asts, current_asts, current_alias);
|
||||
}
|
||||
}
|
||||
|
||||
/// Если секция WHERE или HAVING состоит из одного алиаса, ссылку нужно заменить не только в children, но и в where_expression и having_expression.
|
||||
if (ASTSelectQuery * select = typeid_cast<ASTSelectQuery *>(&*ast))
|
||||
@ -496,10 +547,33 @@ void ExpressionAnalyzer::optimizeGroupBy()
|
||||
/// iterate over each GROUP BY expression, eliminate injective function calls and literals
|
||||
for (size_t i = 0; i < group_exprs.size();)
|
||||
{
|
||||
if (const auto function = typeid_cast<ASTFunction*>(group_exprs[i].get()))
|
||||
if (const auto function = typeid_cast<ASTFunction *>(group_exprs[i].get()))
|
||||
{
|
||||
/// assert function is injective
|
||||
if (!injective_function_names.count(function->name))
|
||||
if (possibly_injective_function_names.count(function->name))
|
||||
{
|
||||
/// do not handle semantic errors here
|
||||
if (function->arguments->children.size() < 2)
|
||||
{
|
||||
++i;
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto & dict_name = typeid_cast<const ASTLiteral &>(*function->arguments->children[0])
|
||||
.value.safeGet<String>();
|
||||
|
||||
const auto & dict_ptr = context.getExternalDictionaries().getDictionary(dict_name);
|
||||
|
||||
const auto & attr_name = typeid_cast<const ASTLiteral &>(*function->arguments->children[1])
|
||||
.value.safeGet<String>();
|
||||
|
||||
if (!dict_ptr->isInjective(attr_name))
|
||||
{
|
||||
++i;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
else if (!injective_function_names.count(function->name))
|
||||
{
|
||||
++i;
|
||||
continue;
|
||||
@ -809,76 +883,76 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block)
|
||||
/// Случай явного перечисления значений.
|
||||
void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sample_block, bool create_ordered_set)
|
||||
{
|
||||
IAST & args = *node->arguments;
|
||||
ASTPtr & arg = args.children.at(1);
|
||||
IAST & args = *node->arguments;
|
||||
ASTPtr & arg = args.children.at(1);
|
||||
|
||||
DataTypes set_element_types;
|
||||
ASTPtr & left_arg = args.children.at(0);
|
||||
DataTypes set_element_types;
|
||||
ASTPtr & left_arg = args.children.at(0);
|
||||
|
||||
ASTFunction * left_arg_tuple = typeid_cast<ASTFunction *>(&*left_arg);
|
||||
ASTFunction * left_arg_tuple = typeid_cast<ASTFunction *>(&*left_arg);
|
||||
|
||||
if (left_arg_tuple && left_arg_tuple->name == "tuple")
|
||||
if (left_arg_tuple && left_arg_tuple->name == "tuple")
|
||||
{
|
||||
for (const auto & arg : left_arg_tuple->arguments->children)
|
||||
{
|
||||
for (const auto & arg : left_arg_tuple->arguments->children)
|
||||
{
|
||||
const auto & data_type = sample_block.getByName(arg->getColumnName()).type;
|
||||
const auto & data_type = sample_block.getByName(arg->getColumnName()).type;
|
||||
|
||||
/// @note prevent crash in query: SELECT (1, [1]) in (1, 1)
|
||||
if (const auto array = typeid_cast<const DataTypeArray * >(data_type.get()))
|
||||
throw Exception("Incorrect element of tuple: " + array->getName(), ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
/// @note prevent crash in query: SELECT (1, [1]) in (1, 1)
|
||||
if (const auto array = typeid_cast<const DataTypeArray * >(data_type.get()))
|
||||
throw Exception("Incorrect element of tuple: " + array->getName(), ErrorCodes::INCORRECT_ELEMENT_OF_SET);
|
||||
|
||||
set_element_types.push_back(data_type);
|
||||
}
|
||||
set_element_types.push_back(data_type);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
DataTypePtr left_type = sample_block.getByName(left_arg->getColumnName()).type;
|
||||
if (DataTypeArray * array_type = typeid_cast<DataTypeArray *>(&*left_type))
|
||||
set_element_types.push_back(array_type->getNestedType());
|
||||
else
|
||||
{
|
||||
DataTypePtr left_type = sample_block.getByName(left_arg->getColumnName()).type;
|
||||
if (DataTypeArray * array_type = typeid_cast<DataTypeArray *>(&*left_type))
|
||||
set_element_types.push_back(array_type->getNestedType());
|
||||
else
|
||||
set_element_types.push_back(left_type);
|
||||
}
|
||||
set_element_types.push_back(left_type);
|
||||
}
|
||||
|
||||
/// Отличим случай x in (1, 2) от случая x in 1 (он же x in (1)).
|
||||
bool single_value = false;
|
||||
ASTPtr elements_ast = arg;
|
||||
/// Отличим случай x in (1, 2) от случая x in 1 (он же x in (1)).
|
||||
bool single_value = false;
|
||||
ASTPtr elements_ast = arg;
|
||||
|
||||
if (ASTFunction * set_func = typeid_cast<ASTFunction *>(&*arg))
|
||||
{
|
||||
if (set_func->name != "tuple")
|
||||
throw Exception("Incorrect type of 2nd argument for function " + node->name + ". Must be subquery or set of values.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
/// Отличм случай (x, y) in ((1, 2), (3, 4)) от случая (x, y) in (1, 2).
|
||||
ASTFunction * any_element = typeid_cast<ASTFunction *>(&*set_func->arguments->children.at(0));
|
||||
if (set_element_types.size() >= 2 && (!any_element || any_element->name != "tuple"))
|
||||
single_value = true;
|
||||
else
|
||||
elements_ast = set_func->arguments;
|
||||
}
|
||||
else if (typeid_cast<ASTLiteral *>(&*arg))
|
||||
{
|
||||
single_value = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (ASTFunction * set_func = typeid_cast<ASTFunction *>(&*arg))
|
||||
{
|
||||
if (set_func->name != "tuple")
|
||||
throw Exception("Incorrect type of 2nd argument for function " + node->name + ". Must be subquery or set of values.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
if (single_value)
|
||||
{
|
||||
ASTPtr exp_list = new ASTExpressionList;
|
||||
exp_list->children.push_back(elements_ast);
|
||||
elements_ast = exp_list;
|
||||
}
|
||||
/// Отличм случай (x, y) in ((1, 2), (3, 4)) от случая (x, y) in (1, 2).
|
||||
ASTFunction * any_element = typeid_cast<ASTFunction *>(&*set_func->arguments->children.at(0));
|
||||
if (set_element_types.size() >= 2 && (!any_element || any_element->name != "tuple"))
|
||||
single_value = true;
|
||||
else
|
||||
elements_ast = set_func->arguments;
|
||||
}
|
||||
else if (typeid_cast<ASTLiteral *>(&*arg))
|
||||
{
|
||||
single_value = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Incorrect type of 2nd argument for function " + node->name + ". Must be subquery or set of values.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
ASTSet * ast_set = new ASTSet(arg->getColumnName());
|
||||
ASTPtr ast_set_ptr = ast_set;
|
||||
ast_set->set = new Set(settings.limits);
|
||||
ast_set->is_explicit = true;
|
||||
ast_set->set->createFromAST(set_element_types, elements_ast, create_ordered_set);
|
||||
arg = ast_set_ptr;
|
||||
if (single_value)
|
||||
{
|
||||
ASTPtr exp_list = new ASTExpressionList;
|
||||
exp_list->children.push_back(elements_ast);
|
||||
elements_ast = exp_list;
|
||||
}
|
||||
|
||||
ASTSet * ast_set = new ASTSet(arg->getColumnName());
|
||||
ASTPtr ast_set_ptr = ast_set;
|
||||
ast_set->set = new Set(settings.limits);
|
||||
ast_set->is_explicit = true;
|
||||
ast_set->set->createFromAST(set_element_types, elements_ast, create_ordered_set);
|
||||
arg = ast_set_ptr;
|
||||
}
|
||||
|
||||
|
||||
|
@ -24,6 +24,7 @@
|
||||
#include <DB/Parsers/ASTOrderByElement.h>
|
||||
|
||||
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
||||
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||
#include <DB/Storages/StorageView.h>
|
||||
#include <DB/TableFunctions/ITableFunction.h>
|
||||
#include <DB/TableFunctions/TableFunctionFactory.h>
|
||||
@ -33,6 +34,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
InterpreterSelectQuery::~InterpreterSelectQuery() = default;
|
||||
|
||||
|
||||
void InterpreterSelectQuery::init(BlockInputStreamPtr input, const Names & required_column_names, const NamesAndTypesList & table_column_names)
|
||||
{
|
||||
original_max_threads = settings.max_threads;
|
||||
@ -269,10 +273,9 @@ DataTypes InterpreterSelectQuery::getReturnTypes()
|
||||
{
|
||||
DataTypes res;
|
||||
NamesAndTypesList columns = query_analyzer->getSelectSampleBlock().getColumnsList();
|
||||
for (NamesAndTypesList::iterator it = columns.begin(); it != columns.end(); ++it)
|
||||
{
|
||||
res.push_back(it->type);
|
||||
}
|
||||
for (auto & column : columns)
|
||||
res.push_back(column.type);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -499,7 +502,7 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
|
||||
if (second_stage)
|
||||
{
|
||||
bool need_second_distinct_pass = true;
|
||||
bool need_second_distinct_pass = query.distinct;
|
||||
|
||||
if (need_aggregate)
|
||||
{
|
||||
@ -515,7 +518,7 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
executeExpression(streams, before_order_and_select);
|
||||
executeDistinct(streams, true, selected_columns);
|
||||
|
||||
need_second_distinct_pass = streams.size() > 1;
|
||||
need_second_distinct_pass = query.distinct && (streams.size() > 1);
|
||||
}
|
||||
else if (query.group_by_with_totals && !aggregate_final)
|
||||
{
|
||||
@ -548,6 +551,9 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
if (query.limit_length && streams.size() > 1 && !query.distinct)
|
||||
executePreLimit(streams);
|
||||
|
||||
if (need_second_distinct_pass)
|
||||
union_within_single_query = true;
|
||||
|
||||
if (union_within_single_query)
|
||||
executeUnion(streams);
|
||||
|
||||
@ -610,6 +616,8 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(BlockInpu
|
||||
|
||||
interpreter_subquery = new InterpreterSelectQuery(
|
||||
query.table, subquery_context, required_columns, QueryProcessingStage::Complete, subquery_depth + 1);
|
||||
|
||||
/// Если во внешнем запросе есть аггрегация, то WITH TOTALS игнорируется в подзапросе.
|
||||
if (query_analyzer->hasAggregation())
|
||||
interpreter_subquery->ignoreWithTotals();
|
||||
}
|
||||
@ -640,9 +648,13 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(BlockInpu
|
||||
* - эти настройки будут переданы на удалённые серверы при распределённой обработке запроса,
|
||||
* и там должно быть оригинальное значение max_threads, а не увеличенное.
|
||||
*/
|
||||
bool is_remote = false;
|
||||
Settings settings_for_storage = settings;
|
||||
if (storage && storage->isRemote())
|
||||
{
|
||||
is_remote = true;
|
||||
settings.max_threads = settings.max_distributed_connections;
|
||||
}
|
||||
|
||||
/// Ограничение на количество столбцов для чтения.
|
||||
if (settings.limits.max_columns_to_read && required_columns.size() > settings.limits.max_columns_to_read)
|
||||
@ -657,7 +669,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(BlockInpu
|
||||
|
||||
/** Оптимизация - если не указаны DISTINCT, WHERE, GROUP, HAVING, ORDER, но указан LIMIT, и limit + offset < max_block_size,
|
||||
* то в качестве размера блока будем использовать limit + offset (чтобы не читать из таблицы больше, чем запрошено),
|
||||
* а также установим количество потоков в 1 и отменим асинхронное выполнение конвейера запроса.
|
||||
* а также установим количество потоков в 1.
|
||||
*/
|
||||
if (!query.distinct
|
||||
&& !query.prewhere_expression
|
||||
@ -680,9 +692,15 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(BlockInpu
|
||||
/// Инициализируем изначальные потоки данных, на которые накладываются преобразования запроса. Таблица или подзапрос?
|
||||
if (!interpreter_subquery)
|
||||
{
|
||||
size_t max_streams = settings.max_threads;
|
||||
|
||||
/// Если надо - запрашиваем больше источников, чем количество потоков - для более равномерного распределения работы по потокам.
|
||||
if (max_streams > 1 && !is_remote)
|
||||
max_streams *= settings.max_streams_to_max_threads_ratio;
|
||||
|
||||
streams = storage->read(required_columns, query_ptr,
|
||||
context, settings_for_storage, from_stage,
|
||||
settings.max_block_size, settings.max_threads);
|
||||
settings.max_block_size, max_streams);
|
||||
|
||||
for (auto & stream : streams)
|
||||
stream->addTableLock(table_lock);
|
||||
@ -1004,4 +1022,10 @@ BlockInputStreamPtr InterpreterSelectQuery::executeAndFormat(WriteBuffer & buf)
|
||||
}
|
||||
|
||||
|
||||
void InterpreterSelectQuery::ignoreWithTotals()
|
||||
{
|
||||
query.group_by_with_totals = false;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -73,6 +73,8 @@ bool ParserInsertQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
if (!s_eq.ignore(pos, end, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!id_p.parse(pos, end, id, max_parsed_pos, expected))
|
||||
return false;
|
||||
}
|
||||
@ -82,9 +84,13 @@ bool ParserInsertQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
/// Есть ли список столбцов
|
||||
if (s_lparen.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!columns_p.parse(pos, end, columns, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!s_rparen.ignore(pos, end, max_parsed_pos, expected))
|
||||
return false;
|
||||
}
|
||||
|
@ -181,7 +181,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
/// [GLOBAL] ANY|ALL INNER|LEFT JOIN (subquery) USING (tuple)
|
||||
/// [GLOBAL] ANY|ALL INNER|LEFT JOIN (subquery) USING tuple
|
||||
join.parse(pos, end, select_query->join, max_parsed_pos, expected);
|
||||
|
||||
if (!parse_final_and_sample())
|
||||
|
@ -33,6 +33,7 @@
|
||||
#include <DB/Storages/StorageSystemDictionaries.h>
|
||||
#include <DB/Storages/StorageSystemColumns.h>
|
||||
#include <DB/Storages/StorageSystemFunctions.h>
|
||||
#include <DB/Storages/StorageSystemClusters.h>
|
||||
|
||||
#include <DB/IO/copyData.h>
|
||||
#include <DB/IO/LimitReadBuffer.h>
|
||||
@ -541,6 +542,7 @@ int Server::main(const std::vector<std::string> & args)
|
||||
global_context->addTable("system", "dictionaries", StorageSystemDictionaries::create("dictionaries"));
|
||||
global_context->addTable("system", "columns", StorageSystemColumns::create("columns"));
|
||||
global_context->addTable("system", "functions", StorageSystemFunctions::create("functions"));
|
||||
global_context->addTable("system", "clusters", StorageSystemClusters::create("clusters", *global_context));
|
||||
|
||||
if (has_zookeeper)
|
||||
global_context->addTable("system", "zookeeper", StorageSystemZooKeeper::create("zookeeper"));
|
||||
|
@ -27,6 +27,7 @@
|
||||
|
||||
#include "TCPHandler.h"
|
||||
|
||||
#include <statdaemons/NetException.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -443,7 +444,7 @@ void TCPHandler::receiveHello()
|
||||
throw Exception("Client has connected to wrong port", ErrorCodes::CLIENT_HAS_CONNECTED_TO_WRONG_PORT);
|
||||
}
|
||||
else
|
||||
throw Exception("Unexpected packet from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
|
||||
throw NetException("Unexpected packet from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
|
||||
}
|
||||
|
||||
readStringBinary(client_name, *in);
|
||||
@ -492,13 +493,13 @@ bool TCPHandler::receivePacket()
|
||||
{
|
||||
case Protocol::Client::Query:
|
||||
if (!state.empty())
|
||||
throw Exception("Unexpected packet Query received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
|
||||
throw NetException("Unexpected packet Query received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
|
||||
receiveQuery();
|
||||
return true;
|
||||
|
||||
case Protocol::Client::Data:
|
||||
if (state.empty())
|
||||
throw Exception("Unexpected packet Data received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
|
||||
throw NetException("Unexpected packet Data received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
|
||||
return receiveData();
|
||||
|
||||
case Protocol::Client::Ping:
|
||||
@ -647,13 +648,13 @@ bool TCPHandler::isQueryCancelled()
|
||||
{
|
||||
case Protocol::Client::Cancel:
|
||||
if (state.empty())
|
||||
throw Exception("Unexpected packet Cancel received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
|
||||
throw NetException("Unexpected packet Cancel received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
|
||||
LOG_INFO(log, "Query was cancelled.");
|
||||
state.is_cancelled = true;
|
||||
return true;
|
||||
|
||||
default:
|
||||
throw Exception("Unknown packet from client", ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT);
|
||||
throw NetException("Unknown packet from client", ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <DB/Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeBlockInputStream.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeWhereOptimizer.h>
|
||||
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
#include <DB/DataStreams/ExpressionBlockInputStream.h>
|
||||
@ -63,13 +62,6 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
|
||||
if (real_column_names.empty())
|
||||
real_column_names.push_back(ExpressionActions::getSmallestColumn(data.getColumnsList()));
|
||||
|
||||
ASTSelectQuery & select = *typeid_cast<ASTSelectQuery*>(&*query);
|
||||
|
||||
/// Try transferring some condition from WHERE to PREWHERE if enabled and viable
|
||||
if (settings.optimize_move_to_prewhere)
|
||||
if (select.where_expression && !select.prewhere_expression)
|
||||
MergeTreeWhereOptimizer{select, data, column_names_to_return, log};
|
||||
|
||||
Block virtual_columns_block = getBlockWithVirtualColumns(parts);
|
||||
|
||||
/// Если запрошен хотя бы один виртуальный столбец, пробуем индексировать
|
||||
@ -114,6 +106,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
|
||||
ExpressionActionsPtr filter_expression;
|
||||
double relative_sample_size = 0;
|
||||
|
||||
ASTSelectQuery & select = *typeid_cast<ASTSelectQuery*>(&*query);
|
||||
|
||||
if (select.sample_size)
|
||||
{
|
||||
relative_sample_size = apply_visitor(FieldVisitorConvertToNumber<double>(),
|
||||
|
@ -164,18 +164,21 @@ public:
|
||||
{
|
||||
destination = storage.context.tryGetTable(storage.destination_database, storage.destination_table);
|
||||
|
||||
if (destination.get() == &storage)
|
||||
throw Exception("Destination table is myself. Write will cause infinite loop.", ErrorCodes::INFINITE_LOOP);
|
||||
if (destination)
|
||||
{
|
||||
if (destination.get() == &storage)
|
||||
throw Exception("Destination table is myself. Write will cause infinite loop.", ErrorCodes::INFINITE_LOOP);
|
||||
|
||||
/// Проверяем структуру таблицы.
|
||||
try
|
||||
{
|
||||
destination->check(block, true);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("(when looking at destination table " + storage.destination_database + "." + storage.destination_table + ")");
|
||||
throw;
|
||||
/// Проверяем структуру таблицы.
|
||||
try
|
||||
{
|
||||
destination->check(block, true);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("(when looking at destination table " + storage.destination_database + "." + storage.destination_table + ")");
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -7,9 +7,11 @@
|
||||
#include <DB/Storages/Distributed/DirectoryMonitor.h>
|
||||
#include <DB/Common/escapeForFileName.h>
|
||||
#include <DB/Parsers/ASTInsertQuery.h>
|
||||
#include <DB/Parsers/ASTSelectQuery.h>
|
||||
|
||||
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
||||
#include <DB/Interpreters/InterpreterAlterQuery.h>
|
||||
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||
|
||||
#include <DB/Core/Field.h>
|
||||
|
||||
|
@ -287,18 +287,17 @@ StoragePtr StorageFactory::get(
|
||||
*/
|
||||
ASTs & args_func = typeid_cast<ASTFunction &>(*typeid_cast<ASTCreateQuery &>(*query).storage).children;
|
||||
|
||||
const auto params_error_message = "Storage Distributed requires 3 or 4 parameters"
|
||||
" - name of configuration section with list of remote servers, name of remote database, name of remote table,"
|
||||
" sharding key expression (optional).";
|
||||
|
||||
if (args_func.size() != 1)
|
||||
throw Exception("Storage Distributed requires 3 parameters"
|
||||
" - name of configuration section with list of remote servers, name of remote database, name of remote table.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
throw Exception(params_error_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
ASTs & args = typeid_cast<ASTExpressionList &>(*args_func.at(0)).children;
|
||||
|
||||
if (args.size() != 3 && args.size() != 4)
|
||||
throw Exception("Storage Distributed requires 3 or 4 parameters"
|
||||
" - name of configuration section with list of remote servers, name of remote database, name of remote table,"
|
||||
" sharding key expression (optional).",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
throw Exception(params_error_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
String cluster_name = typeid_cast<ASTIdentifier &>(*args[0]).name;
|
||||
String remote_database = reinterpretAsIdentifier(args[1], local_context).name;
|
||||
@ -433,7 +432,7 @@ SummingMergeTree(EventDate, (OrderID, EventDate, BannerID, PhraseID, ContextType
|
||||
ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/hits', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192)
|
||||
|
||||
|
||||
For further info please read the documentation: http://clickhouse.yandex-team.ru/
|
||||
For further info please read the documentation: https://clickhouse.yandex-team.ru/
|
||||
)";
|
||||
|
||||
String name_part = name.substr(0, name.size() - strlen("MergeTree"));
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <DB/Storages/StorageMerge.h>
|
||||
#include <DB/Common/VirtualColumnUtils.h>
|
||||
#include <DB/Interpreters/InterpreterAlterQuery.h>
|
||||
#include <DB/Interpreters/ExpressionActions.h>
|
||||
#include <DB/Storages/VirtualColumnFactory.h>
|
||||
#include <DB/Parsers/ASTSelectQuery.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <DB/Storages/MergeTree/MergeTreeBlockOutputStream.h>
|
||||
#include <DB/Storages/MergeTree/DiskSpaceMonitor.h>
|
||||
#include <DB/Storages/MergeTree/MergeList.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeWhereOptimizer.h>
|
||||
#include <DB/Common/escapeForFileName.h>
|
||||
#include <DB/Interpreters/InterpreterAlterQuery.h>
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
@ -98,6 +99,13 @@ BlockInputStreams StorageMergeTree::read(
|
||||
const size_t max_block_size,
|
||||
const unsigned threads)
|
||||
{
|
||||
ASTSelectQuery & select = *typeid_cast<ASTSelectQuery*>(&*query);
|
||||
|
||||
/// Try transferring some condition from WHERE to PREWHERE if enabled and viable
|
||||
if (settings.optimize_move_to_prewhere)
|
||||
if (select.where_expression && !select.prewhere_expression)
|
||||
MergeTreeWhereOptimizer{select, data, column_names, log};
|
||||
|
||||
return reader.read(column_names, query, context, settings, processed_stage, max_block_size, threads);
|
||||
}
|
||||
|
||||
@ -262,7 +270,8 @@ void StorageMergeTree::dropPartition(const Field & partition, bool detach, bool
|
||||
/// Просит завершить мерджи и не позволяет им начаться.
|
||||
/// Это защищает от "оживания" данных за удалённую партицию после завершения мерджа.
|
||||
const MergeTreeMergeBlocker merge_blocker{merger};
|
||||
auto structure_lock = lockStructure(true);
|
||||
/// Дожидается завершения мерджей и не даёт начаться новым.
|
||||
auto lock = lockForAlter();
|
||||
|
||||
DayNum_t month = MergeTreeData::getMonthDayNum(partition);
|
||||
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <DB/Storages/MergeTree/ReplicatedMergeTreePartsExchange.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreePartChecker.h>
|
||||
#include <DB/Storages/MergeTree/MergeList.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeWhereOptimizer.h>
|
||||
#include <DB/Parsers/formatAST.h>
|
||||
#include <DB/IO/WriteBufferFromOStream.h>
|
||||
#include <DB/IO/ReadBufferFromString.h>
|
||||
@ -1996,6 +1997,13 @@ BlockInputStreams StorageReplicatedMergeTree::read(
|
||||
else
|
||||
real_column_names.push_back(it);
|
||||
|
||||
ASTSelectQuery & select = *typeid_cast<ASTSelectQuery*>(&*query);
|
||||
|
||||
/// Try transferring some condition from WHERE to PREWHERE if enabled and viable
|
||||
if (settings.optimize_move_to_prewhere)
|
||||
if (select.where_expression && !select.prewhere_expression)
|
||||
MergeTreeWhereOptimizer{select, data, real_column_names, log};
|
||||
|
||||
Block virtual_columns_block;
|
||||
ColumnUInt8 * column = new ColumnUInt8(2);
|
||||
ColumnPtr column_ptr = column;
|
||||
|
128
dbms/src/Storages/StorageSystemClusters.cpp
Normal file
128
dbms/src/Storages/StorageSystemClusters.cpp
Normal file
@ -0,0 +1,128 @@
|
||||
#include <DB/Storages/StorageSystemClusters.h>
|
||||
#include <DB/Interpreters/Cluster.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/Columns/ColumnsNumber.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
StorageSystemClusters::StorageSystemClusters(const std::string & name_, Context & context_)
|
||||
: name(name_)
|
||||
, columns{
|
||||
{ "cluster", new DataTypeString },
|
||||
{ "shard_num", new DataTypeUInt32 },
|
||||
{ "shard_weight", new DataTypeUInt32 },
|
||||
{ "replica_num", new DataTypeUInt32 },
|
||||
{ "host_name", new DataTypeString },
|
||||
{ "host_address", new DataTypeString },
|
||||
{ "port", new DataTypeUInt16 },
|
||||
{ "user", new DataTypeString }
|
||||
}
|
||||
, context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
StoragePtr StorageSystemClusters::create(const std::string & name_, Context & context_)
|
||||
{
|
||||
context_.initClusters();
|
||||
return (new StorageSystemClusters{name_, context_})->thisPtr();
|
||||
}
|
||||
|
||||
BlockInputStreams StorageSystemClusters::read(
|
||||
const Names & column_names,
|
||||
ASTPtr query,
|
||||
const Context & context_,
|
||||
const Settings & settings,
|
||||
QueryProcessingStage::Enum & processed_stage,
|
||||
const size_t max_block_size,
|
||||
const unsigned threads)
|
||||
{
|
||||
check(column_names);
|
||||
processed_stage = QueryProcessingStage::FetchColumns;
|
||||
|
||||
ColumnPtr cluster_column = new ColumnString;
|
||||
ColumnPtr shard_num_column = new ColumnUInt32;
|
||||
ColumnPtr shard_weight_column = new ColumnUInt32;
|
||||
ColumnPtr replica_num_column = new ColumnUInt32;
|
||||
ColumnPtr host_name_column = new ColumnString;
|
||||
ColumnPtr host_address_column = new ColumnString;
|
||||
ColumnPtr port_column = new ColumnUInt16;
|
||||
ColumnPtr user_column = new ColumnString;
|
||||
|
||||
auto updateColumns = [&](const std::string & cluster_name, const Cluster::ShardInfo & shard_info,
|
||||
const Cluster::Address & address)
|
||||
{
|
||||
cluster_column->insert(cluster_name);
|
||||
shard_num_column->insert(static_cast<UInt64>(shard_info.shard_num));
|
||||
shard_weight_column->insert(static_cast<UInt64>(shard_info.weight));
|
||||
replica_num_column->insert(static_cast<UInt64>(address.replica_num));
|
||||
|
||||
host_name_column->insert(address.host_name);
|
||||
host_address_column->insert(address.host_port.host().toString());
|
||||
port_column->insert(static_cast<UInt64>(address.host_port.port()));
|
||||
user_column->insert(address.user);
|
||||
};
|
||||
|
||||
const auto & clusters = context.getClusters();
|
||||
for (const auto & entry : clusters->impl)
|
||||
{
|
||||
const std::string cluster_name = entry.first;
|
||||
const Cluster & cluster = entry.second;
|
||||
const auto & addresses = cluster.getShardsInfo();
|
||||
const auto & addresses_with_failover = cluster.getShardsWithFailoverInfo();
|
||||
const auto & shards_info = cluster.shard_info_vec;
|
||||
|
||||
if (!addresses.empty())
|
||||
{
|
||||
auto it1 = addresses.cbegin();
|
||||
auto it2 = shards_info.cbegin();
|
||||
|
||||
while (it1 != addresses.cend())
|
||||
{
|
||||
const auto & address = *it1;
|
||||
const auto & shard_info = *it2;
|
||||
|
||||
updateColumns(cluster_name, shard_info, address);
|
||||
|
||||
++it1;
|
||||
++it2;
|
||||
}
|
||||
}
|
||||
else if (!addresses_with_failover.empty())
|
||||
{
|
||||
auto it1 = addresses_with_failover.cbegin();
|
||||
auto it2 = shards_info.cbegin();
|
||||
|
||||
while (it1 != addresses_with_failover.cend())
|
||||
{
|
||||
const auto & addresses = *it1;
|
||||
const auto & shard_info = *it2;
|
||||
|
||||
for (const auto & address : addresses)
|
||||
updateColumns(cluster_name, shard_info, address);
|
||||
|
||||
++it1;
|
||||
++it2;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Block block;
|
||||
|
||||
block.insert(ColumnWithNameAndType(cluster_column, new DataTypeString, "cluster"));
|
||||
block.insert(ColumnWithNameAndType(shard_num_column, new DataTypeUInt32, "shard_num"));
|
||||
block.insert(ColumnWithNameAndType(shard_weight_column, new DataTypeUInt32, "shard_weight"));
|
||||
block.insert(ColumnWithNameAndType(replica_num_column, new DataTypeUInt32, "replica_num"));
|
||||
block.insert(ColumnWithNameAndType(host_name_column, new DataTypeString, "host_name"));
|
||||
block.insert(ColumnWithNameAndType(host_address_column, new DataTypeString, "host_address"));
|
||||
block.insert(ColumnWithNameAndType(port_column, new DataTypeUInt16, "port"));
|
||||
block.insert(ColumnWithNameAndType(user_column, new DataTypeString, "user"));
|
||||
|
||||
return BlockInputStreams{ 1, new OneBlockInputStream(block) };
|
||||
}
|
||||
|
||||
}
|
@ -27,6 +27,7 @@ StorageSystemDictionaries::StorageSystemDictionaries(const std::string & name)
|
||||
{ "attribute.types", new DataTypeArray{new DataTypeString} },
|
||||
{ "has_hierarchy", new DataTypeUInt8 },
|
||||
{ "bytes_allocated", new DataTypeUInt64 },
|
||||
{ "query_count", new DataTypeUInt64 },
|
||||
{ "hit_rate", new DataTypeFloat64 },
|
||||
{ "element_count", new DataTypeUInt64 },
|
||||
{ "load_factor", new DataTypeFloat64 },
|
||||
@ -69,6 +70,7 @@ BlockInputStreams StorageSystemDictionaries::read(
|
||||
};
|
||||
ColumnWithNameAndType col_has_hierarchy{new ColumnUInt8, new DataTypeUInt8, "has_hierarchy"};
|
||||
ColumnWithNameAndType col_bytes_allocated{new ColumnUInt64, new DataTypeUInt64, "bytes_allocated"};
|
||||
ColumnWithNameAndType col_query_count{new ColumnUInt64, new DataTypeUInt64, "query_count"};
|
||||
ColumnWithNameAndType col_hit_rate{new ColumnFloat64, new DataTypeFloat64, "hit_rate"};
|
||||
ColumnWithNameAndType col_element_count{new ColumnUInt64, new DataTypeUInt64, "element_count"};
|
||||
ColumnWithNameAndType col_load_factor{new ColumnFloat64, new DataTypeFloat64, "load_factor"};
|
||||
@ -97,6 +99,7 @@ BlockInputStreams StorageSystemDictionaries::read(
|
||||
}));
|
||||
col_has_hierarchy.column->insert(UInt64{dict_ptr->hasHierarchy()});
|
||||
col_bytes_allocated.column->insert(dict_ptr->getBytesAllocated());
|
||||
col_query_count.column->insert(dict_ptr->getQueryCount());
|
||||
col_hit_rate.column->insert(dict_ptr->getHitRate());
|
||||
col_element_count.column->insert(dict_ptr->getElementCount());
|
||||
col_load_factor.column->insert(dict_ptr->getLoadFactor());
|
||||
@ -140,6 +143,7 @@ BlockInputStreams StorageSystemDictionaries::read(
|
||||
col_attribute_types,
|
||||
col_has_hierarchy,
|
||||
col_bytes_allocated,
|
||||
col_query_count,
|
||||
col_hit_rate,
|
||||
col_element_count,
|
||||
col_load_factor,
|
||||
|
104
dbms/tests/queries/0_stateless/00153_transform.reference
Normal file
104
dbms/tests/queries/0_stateless/00153_transform.reference
Normal file
@ -0,0 +1,104 @@
|
||||
0
|
||||
1
|
||||
2
|
||||
111
|
||||
4
|
||||
222
|
||||
6
|
||||
333
|
||||
8
|
||||
9
|
||||
9999
|
||||
9999
|
||||
9999
|
||||
111
|
||||
9999
|
||||
222
|
||||
9999
|
||||
333
|
||||
9999
|
||||
9999
|
||||
|
||||
|
||||
|
||||
hello
|
||||
|
||||
world
|
||||
|
||||
abc
|
||||
|
||||
|
||||
0
|
||||
1
|
||||
2
|
||||
hello
|
||||
4
|
||||
world
|
||||
6
|
||||
abc
|
||||
8
|
||||
9
|
||||
|
||||
|
||||
|
||||
hello
|
||||
|
||||
world
|
||||
|
||||
abc
|
||||
|
||||
|
||||
-
|
||||
-
|
||||
-
|
||||
hello
|
||||
-
|
||||
world
|
||||
-
|
||||
abc
|
||||
-
|
||||
-
|
||||
0
|
||||
0
|
||||
0
|
||||
111
|
||||
0
|
||||
222
|
||||
0
|
||||
333
|
||||
0
|
||||
0
|
||||
-1
|
||||
-1
|
||||
-1
|
||||
111
|
||||
-1
|
||||
222
|
||||
-1
|
||||
333
|
||||
-1
|
||||
-1
|
||||
-1.1
|
||||
-1.1
|
||||
-1.1
|
||||
111
|
||||
-1.1
|
||||
222
|
||||
-1.1
|
||||
333
|
||||
-1.1
|
||||
-1.1
|
||||
1
|
||||
1
|
||||
1
|
||||
111
|
||||
1
|
||||
222.2
|
||||
1
|
||||
333
|
||||
1
|
||||
1
|
||||
Остальные
|
||||
Яндекс
|
||||
Google
|
||||
Остальные
|
14
dbms/tests/queries/0_stateless/00153_transform.sql
Normal file
14
dbms/tests/queries/0_stateless/00153_transform.sql
Normal file
@ -0,0 +1,14 @@
|
||||
SELECT transform(number, [3, 5, 7], [111, 222, 333]) FROM system.numbers LIMIT 10;
|
||||
SELECT transform(number, [3, 5, 7], [111, 222, 333], 9999) FROM system.numbers LIMIT 10;
|
||||
SELECT transform(number, [3, 5, 7], ['hello', 'world', 'abc'], '') FROM system.numbers LIMIT 10;
|
||||
SELECT transform(toString(number), ['3', '5', '7'], ['hello', 'world', 'abc']) FROM system.numbers LIMIT 10;
|
||||
SELECT transform(toString(number), ['3', '5', '7'], ['hello', 'world', 'abc'], '') FROM system.numbers LIMIT 10;
|
||||
SELECT transform(toString(number), ['3', '5', '7'], ['hello', 'world', 'abc'], '-') FROM system.numbers LIMIT 10;
|
||||
SELECT transform(toString(number), ['3', '5', '7'], [111, 222, 333], 0) FROM system.numbers LIMIT 10;
|
||||
SELECT transform(toString(number), ['3', '5', '7'], [111, 222, 333], -1) FROM system.numbers LIMIT 10;
|
||||
SELECT transform(toString(number), ['3', '5', '7'], [111, 222, 333], -1.1) FROM system.numbers LIMIT 10;
|
||||
SELECT transform(toString(number), ['3', '5', '7'], [111, 222.2, 333], 1) FROM system.numbers LIMIT 10;
|
||||
SELECT transform(1, [2, 3], ['Яндекс', 'Google'], 'Остальные') AS title;
|
||||
SELECT transform(2, [2, 3], ['Яндекс', 'Google'], 'Остальные') AS title;
|
||||
SELECT transform(3, [2, 3], ['Яндекс', 'Google'], 'Остальные') AS title;
|
||||
SELECT transform(4, [2, 3], ['Яндекс', 'Google'], 'Остальные') AS title;
|
@ -0,0 +1,10 @@
|
||||
0
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
6
|
||||
7
|
||||
8
|
||||
9
|
@ -0,0 +1 @@
|
||||
SELECT DISTINCT number FROM remote('127.0.0.{1,2}', system.numbers) LIMIT 10
|
120
dbms/tests/queries/0_stateless/00155_merges.reference
Normal file
120
dbms/tests/queries/0_stateless/00155_merges.reference
Normal file
@ -0,0 +1,120 @@
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
1 1
|
||||
1 1
|
||||
|
||||
|
64
dbms/tests/queries/0_stateless/00155_merges.sh
Executable file
64
dbms/tests/queries/0_stateless/00155_merges.sh
Executable file
@ -0,0 +1,64 @@
|
||||
#!/bin/bash
|
||||
|
||||
function create {
|
||||
clickhouse-client --query="DROP TABLE IF EXISTS test.summing"
|
||||
clickhouse-client --query="DROP TABLE IF EXISTS test.collapsing"
|
||||
clickhouse-client --query="DROP TABLE IF EXISTS test.aggregating"
|
||||
|
||||
clickhouse-client --query="CREATE TABLE test.summing (d Date DEFAULT today(), x UInt64, s UInt64 DEFAULT 1) ENGINE = SummingMergeTree(d, x, 8192)"
|
||||
clickhouse-client --query="CREATE TABLE test.collapsing (d Date DEFAULT today(), x UInt64, s UInt64 DEFAULT 1) ENGINE = CollapsingMergeTree(d, x, 8192, s)"
|
||||
clickhouse-client --query="CREATE TABLE test.aggregating (d Date DEFAULT today(), x UInt64, s AggregateFunction(sum, UInt64)) ENGINE = AggregatingMergeTree(d, x, 8192)"
|
||||
}
|
||||
|
||||
|
||||
function cleanup {
|
||||
clickhouse-client --query="DROP TABLE test.summing"
|
||||
clickhouse-client --query="DROP TABLE test.collapsing"
|
||||
clickhouse-client --query="DROP TABLE test.aggregating"
|
||||
}
|
||||
|
||||
|
||||
function test {
|
||||
create
|
||||
|
||||
SUM=$(( $1 + $2 ))
|
||||
MAX=$(( $1 > $2 ? $1 : $2 ))
|
||||
|
||||
clickhouse-client --query="INSERT INTO test.summing (x) SELECT number AS x FROM system.numbers LIMIT $1"
|
||||
clickhouse-client --query="INSERT INTO test.summing (x) SELECT number AS x FROM system.numbers LIMIT $2"
|
||||
|
||||
clickhouse-client --query="INSERT INTO test.collapsing (x) SELECT number AS x FROM system.numbers LIMIT $1"
|
||||
clickhouse-client --query="INSERT INTO test.collapsing (x) SELECT number AS x FROM system.numbers LIMIT $2"
|
||||
|
||||
clickhouse-client --query="INSERT INTO test.aggregating (d, x, s) SELECT today() AS d, number AS x, sumState(materialize(toUInt64(1))) AS s FROM (SELECT number FROM system.numbers LIMIT $1) GROUP BY number"
|
||||
clickhouse-client --query="INSERT INTO test.aggregating (d, x, s) SELECT today() AS d, number AS x, sumState(materialize(toUInt64(1))) AS s FROM (SELECT number FROM system.numbers LIMIT $2) GROUP BY number"
|
||||
|
||||
clickhouse-client --query="SELECT count() = $SUM, sum(s) = $SUM FROM test.summing"
|
||||
clickhouse-client --query="OPTIMIZE TABLE test.summing"
|
||||
clickhouse-client --query="SELECT count() = $MAX, sum(s) = $SUM FROM test.summing"
|
||||
echo
|
||||
clickhouse-client --query="SELECT count() = $SUM, sum(s) = $SUM FROM test.collapsing"
|
||||
clickhouse-client --query="OPTIMIZE TABLE test.collapsing"
|
||||
clickhouse-client --query="SELECT count() = $MAX, sum(s) = $MAX FROM test.collapsing"
|
||||
echo
|
||||
clickhouse-client --query="SELECT count() = $SUM, sumMerge(s) = $SUM FROM test.aggregating"
|
||||
clickhouse-client --query="OPTIMIZE TABLE test.aggregating"
|
||||
clickhouse-client --query="SELECT count() = $MAX, sumMerge(s) = $SUM FROM test.aggregating"
|
||||
echo
|
||||
echo
|
||||
}
|
||||
|
||||
test 8191 8191
|
||||
test 8191 8192
|
||||
test 8192 8191
|
||||
test 8192 8192
|
||||
test 8192 8193
|
||||
test 8193 8192
|
||||
test 8193 8193
|
||||
test 8191 8193
|
||||
test 8193 8191
|
||||
test 8193 8194
|
||||
test 8194 8193
|
||||
test 8194 8194
|
||||
|
||||
cleanup
|
@ -0,0 +1,4 @@
|
||||
[1] 123
|
||||
[3] 123
|
||||
[1,1] 123
|
||||
[3,4] 123
|
@ -0,0 +1,4 @@
|
||||
SELECT arrayMap(x -> 1, [2]), 123 AS y;
|
||||
SELECT arrayMap(x -> x + 1, [2]), 123 AS y;
|
||||
SELECT arrayMap(x -> 1, [2, 3]), 123 AS y;
|
||||
SELECT arrayMap(x -> x + 1, [2, 3]), 123 AS y;
|
@ -0,0 +1 @@
|
||||
[1] 123 124
|
@ -0,0 +1 @@
|
||||
SELECT arrayMap(x -> 1, [2]), 123 AS x, x + 1;
|
@ -0,0 +1,6 @@
|
||||
CREATE DATABASE IF NOT EXISTS test2;
|
||||
DROP TABLE IF EXISTS test2.mt_buffer;
|
||||
CREATE TABLE test2.mt_buffer (d Date DEFAULT today(), x UInt64) ENGINE = Buffer(test2, mt, 16, 100, 100, 1000000, 1000000, 1000000000, 1000000000);
|
||||
INSERT INTO test2.mt_buffer (x) SELECT number AS x FROM system.numbers LIMIT 100000;
|
||||
INSERT INTO test2.mt_buffer (x) SELECT number AS x FROM system.numbers LIMIT 1000000;
|
||||
DROP DATABASE test2;
|
@ -0,0 +1,6 @@
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
6
|
@ -0,0 +1,13 @@
|
||||
DROP TABLE IF EXISTS test.memory;
|
||||
CREATE TABLE test.memory (x UInt8) ENGINE = Memory;
|
||||
|
||||
INSERT INTO test.memory VALUES (1);
|
||||
INSERT INTO test.memory (x) VALUES (2);
|
||||
INSERT INTO test.memory ( x) VALUES (3);
|
||||
INSERT INTO test.memory (x ) VALUES (4);
|
||||
INSERT INTO test.memory ( x ) VALUES (5);
|
||||
INSERT INTO test.memory(x)VALUES(6);
|
||||
|
||||
SELECT * FROM test.memory ORDER BY x;
|
||||
|
||||
DROP TABLE test.memory;
|
@ -0,0 +1,4 @@
|
||||
2015-05-01 12345 1
|
||||
2015-05-01 67890 1
|
||||
2015-05-01 12345 1
|
||||
2015-05-01 67890 1
|
@ -0,0 +1,14 @@
|
||||
DROP TABLE IF EXISTS test.mt;
|
||||
DROP TABLE IF EXISTS test.merge;
|
||||
|
||||
CREATE TABLE test.mt (d Date DEFAULT toDate('2015-05-01'), x UInt64) ENGINE = MergeTree(d, x, 1);
|
||||
CREATE TABLE test.merge (d Date, x UInt64) ENGINE = Merge(test, '^mt$');
|
||||
|
||||
SET max_block_size = 1000000;
|
||||
INSERT INTO test.mt SELECT number AS x FROM system.numbers LIMIT 100000;
|
||||
|
||||
SELECT *, b FROM test.mt WHERE x IN (12345, 67890) AND NOT ignore(blockSize() < 10 AS b) ORDER BY x;
|
||||
SELECT *, b FROM test.merge WHERE x IN (12345, 67890) AND NOT ignore(blockSize() < 10 AS b) ORDER BY x;
|
||||
|
||||
DROP TABLE test.merge;
|
||||
DROP TABLE test.mt;
|
@ -100,6 +100,11 @@ public:
|
||||
*/
|
||||
void remove(const std::string & path, int32_t version = -1);
|
||||
|
||||
/** Удаляет ноду. В случае сетевых ошибок пробует удалять повторно.
|
||||
* Ошибка ZNONODE для второй и последующих попыток игнорируется
|
||||
*/
|
||||
void removeWithRetries(const std::string & path, int32_t version = -1);
|
||||
|
||||
/** Не бросает исключение при следующих ошибках:
|
||||
* - Такой ноды нет.
|
||||
* - У ноды другая версия.
|
||||
|
@ -294,6 +294,15 @@ void ZooKeeper::remove(const std::string & path, int32_t version)
|
||||
check(tryRemove(path, version), path);
|
||||
}
|
||||
|
||||
void ZooKeeper::removeWithRetries(const std::string & path, int32_t version)
|
||||
{
|
||||
size_t attempt;
|
||||
int code = tryRemoveWithRetries(path, version, &attempt);
|
||||
|
||||
if (!(code == ZOK || (code == ZNONODE && attempt > 0)))
|
||||
throw KeeperException(code, path);
|
||||
}
|
||||
|
||||
int32_t ZooKeeper::tryRemove(const std::string & path, int32_t version)
|
||||
{
|
||||
int32_t code = removeImpl(path, version);
|
||||
|
Loading…
Reference in New Issue
Block a user