diff --git a/dbms/src/AggregateFunctions/AggregateFunctionIntersectionsMax.cpp b/dbms/src/AggregateFunctions/AggregateFunctionIntersectionsMax.cpp index 1292bcb8561..e9e6c739422 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionIntersectionsMax.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionIntersectionsMax.cpp @@ -7,62 +7,66 @@ namespace DB { - template -typename Intersections::PointsMap::iterator -Intersections::insert_point(const T &v) +typename Intersections::PointsMap::iterator Intersections::insert_point(const T & v) { - auto res = points.emplace(v,0); - auto &i = res.first; - if(!res.second) return i; - if(i==points.begin()) return i; + auto res = points.emplace(v, 0); + auto & i = res.first; + if (!res.second) + return i; + if (i == points.begin()) + return i; auto prev = i; prev--; - i->second=prev->second; + i->second = prev->second; return i; } template -void Intersections::add(const T &start, const T &end, T weight) +void Intersections::add(const T & start, const T & end, T weight) { auto sp = insert_point(start); auto ep = end ? insert_point(end) : points.end(); - do { - sp->second+=weight; - if(sp->second > max_weight) { + do + { + sp->second += weight; + if (sp->second > max_weight) + { max_weight = sp->second; max_weight_pos = sp->first; } - } while(++sp != ep); + } while (++sp != ep); } template -void Intersections::merge(const Intersections &other) +void Intersections::merge(const Intersections & other) { - if(other.points.empty()) + if (other.points.empty()) return; typename PointsMap::const_iterator prev, i = other.points.begin(); prev = i; i++; - while(i != other.points.end()) { - add(prev->first,i->first,prev->second); + while (i != other.points.end()) + { + add(prev->first, i->first, prev->second); prev = i; i++; } - if(prev != other.points.end()) - add(prev->first,0,prev->second); + if (prev != other.points.end()) + add(prev->first, 0, prev->second); } template void Intersections::serialize(WriteBuffer & buf) const { - writeBinary(points.size(),buf); - for(const auto &p: points) { - writeBinary(p.first,buf); - writeBinary(p.second,buf); + writeBinary(points.size(), buf); + for (const auto & p : points) + { + writeBinary(p.first, buf); + writeBinary(p.second, buf); } } @@ -74,72 +78,71 @@ void Intersections::deserialize(ReadBuffer & buf) T weight; readBinary(size, buf); - for (std::size_t i = 0; i < size; ++i) { + for (std::size_t i = 0; i < size; ++i) + { readBinary(point, buf); - readBinary(weight,buf); - points.emplace(point,weight); + readBinary(weight, buf); + points.emplace(point, weight); } } void AggregateFunctionIntersectionsMax::_add( - AggregateDataPtr place, - const IColumn & column_start, - const IColumn & column_end, - size_t row_num) const + AggregateDataPtr place, const IColumn & column_start, const IColumn & column_end, size_t row_num) const { PointType start_time, end_time; Field tmp_start_time_field, tmp_end_time_field; - column_start.get(row_num,tmp_start_time_field); - if(tmp_start_time_field.isNull()) + column_start.get(row_num, tmp_start_time_field); + if (tmp_start_time_field.isNull()) return; start_time = tmp_start_time_field.template get(); - if(0==start_time) + if (0 == start_time) return; - column_end.get(row_num,tmp_end_time_field); - if(tmp_end_time_field.isNull()) { + column_end.get(row_num, tmp_end_time_field); + if (tmp_end_time_field.isNull()) + { end_time = 0; - } else { + } + else + { end_time = tmp_end_time_field.template get(); - if(0!=end_time) { - if(end_time==start_time) { + if (0 != end_time) + { + if (end_time == start_time) + { end_time = 0; - } else if(end_time < start_time) { + } + else if (end_time < start_time) + { return; } } } - data(place).add(start_time,end_time); + data(place).add(start_time, end_time); } namespace { + AggregateFunctionPtr createAggregateFunctionIntersectionsMax( + const std::string & name, const DataTypes & argument_types, const Array & parameters) + { + assertBinary(name, argument_types); + return std::make_shared(argument_types, parameters, false); + } -AggregateFunctionPtr createAggregateFunctionIntersectionsMax(const std::string & name, const DataTypes & argument_types, const Array & parameters) -{ - assertBinary(name, argument_types); - return std::make_shared(argument_types,parameters,false); -} - -AggregateFunctionPtr createAggregateFunctionIntersectionsMaxPos(const std::string & name, const DataTypes & argument_types, const Array & parameters) -{ - assertBinary(name, argument_types); - return std::make_shared(argument_types,parameters,true); -} - + AggregateFunctionPtr createAggregateFunctionIntersectionsMaxPos( + const std::string & name, const DataTypes & argument_types, const Array & parameters) + { + assertBinary(name, argument_types); + return std::make_shared(argument_types, parameters, true); + } } void registerAggregateFunctionIntersectionsMax(AggregateFunctionFactory & factory) { - factory.registerFunction("intersectionsMax", - createAggregateFunctionIntersectionsMax, - AggregateFunctionFactory::CaseInsensitive); - factory.registerFunction("intersectionsMaxPos", - createAggregateFunctionIntersectionsMaxPos, - AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("intersectionsMax", createAggregateFunctionIntersectionsMax, AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("intersectionsMaxPos", createAggregateFunctionIntersectionsMaxPos, AggregateFunctionFactory::CaseInsensitive); } - } - diff --git a/dbms/src/AggregateFunctions/AggregateFunctionIntersectionsMax.h b/dbms/src/AggregateFunctions/AggregateFunctionIntersectionsMax.h index ca97986e678..c15ae1ae33d 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionIntersectionsMax.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionIntersectionsMax.h @@ -2,10 +2,10 @@ #include -#include -#include #include #include +#include +#include #include @@ -15,7 +15,6 @@ namespace DB { - namespace ErrorCodes { extern const int AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS; @@ -26,71 +25,68 @@ namespace ErrorCodes template class Intersections final { - using PointsMap = std::map; PointsMap points; T max_weight; T max_weight_pos; - typename PointsMap::iterator insert_point(const T &v); - public: + typename PointsMap::iterator insert_point(const T & v); - Intersections() - : max_weight(0) - { } +public: + Intersections() : max_weight(0) {} - void add(const T &start, const T &end, T weight = 1); - void merge(const Intersections &other); + void add(const T & start, const T & end, T weight = 1); + void merge(const Intersections & other); void serialize(WriteBuffer & buf) const; void deserialize(ReadBuffer & buf); - T max() const { return max_weight; } - T max_pos() const { return max_weight_pos; } + T max() const + { + return max_weight; + } + T max_pos() const + { + return max_weight_pos; + } }; class AggregateFunctionIntersectionsMax final - : public IAggregateFunctionDataHelper, AggregateFunctionIntersectionsMax> + : public IAggregateFunctionDataHelper, AggregateFunctionIntersectionsMax> { using PointType = UInt64; bool return_position; void _add(AggregateDataPtr place, const IColumn & column_start, const IColumn & column_end, size_t row_num) const; - public: +public: AggregateFunctionIntersectionsMax(const DataTypes & arguments, const Array & params, bool return_position) - : return_position(return_position) + : return_position(return_position) { - if (!params.empty()) { + if (!params.empty()) + { throw Exception( - "Aggregate function " + getName() + " does not allow paremeters.", - ErrorCodes::AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS); + "Aggregate function " + getName() + " does not allow paremeters.", ErrorCodes::AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS); } if (arguments.size() != 2) throw Exception("Aggregate function " + getName() + " requires two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if(!arguments[0]->isValueRepresentedByInteger()) - throw Exception { - getName() + ": first argument must be represented by integer", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT - }; + if (!arguments[0]->isValueRepresentedByInteger()) + throw Exception{getName() + ": first argument must be represented by integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if(!arguments[1]->isValueRepresentedByInteger()) - throw Exception { - getName() + ": second argument must be represented by integer", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT - }; + if (!arguments[1]->isValueRepresentedByInteger()) + throw Exception{getName() + ": second argument must be represented by integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if(!arguments[0]->equals(*arguments[1])) - throw Exception { - getName() + ": arguments must have the same type", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT - }; + if (!arguments[0]->equals(*arguments[1])) + throw Exception{getName() + ": arguments must have the same type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } - String getName() const override { return "IntersectionsMax"; } + String getName() const override + { + return "IntersectionsMax"; + } DataTypePtr getReturnType() const override { @@ -99,7 +95,7 @@ class AggregateFunctionIntersectionsMax final void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - _add(place,*columns[0],*columns[1],row_num); + _add(place, *columns[0], *columns[1], row_num); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override @@ -119,14 +115,15 @@ class AggregateFunctionIntersectionsMax final void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - auto &ret = static_cast(to).getData(); + auto & ret = static_cast(to).getData(); ret.push_back(data(place).max()); - if(return_position) + if (return_position) ret.push_back(data(place).max_pos()); } - const char * getHeaderFilePath() const override { return __FILE__; } + const char * getHeaderFilePath() const override + { + return __FILE__; + } }; - } -