Apply ./utils/check-style/fix-style #2352

This commit is contained in:
Alexey Milovidov 2018-05-13 12:36:51 +03:00
parent c79d4fd0fd
commit cecffcd5ab

View File

@ -1,23 +1,22 @@
#pragma once #pragma once
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <unordered_set>
#include <sstream>
#include <iostream> #include <iostream>
#include <DataTypes/DataTypesNumber.h> #include <sstream>
#include <DataTypes/DataTypeDateTime.h> #include <unordered_set>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/ArenaAllocator.h> #include <Common/ArenaAllocator.h>
#include <ext/range.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <ext/range.h>
#include <AggregateFunctions/IAggregateFunction.h> #include <AggregateFunctions/IAggregateFunction.h>
namespace DB namespace DB
{ {
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
@ -39,8 +38,8 @@ struct AggregateFunctionWindowFunnelData
using TimestampEvent = std::pair<UInt32, UInt8>; using TimestampEvent = std::pair<UInt32, UInt8>;
static constexpr size_t bytes_on_stack = 64; static constexpr size_t bytes_on_stack = 64;
using TimestampEvents = PODArray<TimestampEvent, bytes_on_stack, AllocatorWithStackMemory<Allocator<false>, bytes_on_stack>>; using TimestampEvents = PODArray<TimestampEvent, bytes_on_stack, AllocatorWithStackMemory<Allocator<false>, bytes_on_stack>>;
using Comparator = ComparePairFirst; using Comparator = ComparePairFirst;
bool sorted = true; bool sorted = true;
@ -113,7 +112,7 @@ struct AggregateFunctionWindowFunnelData
size_t size; size_t size;
readBinary(size, buf); readBinary(size, buf);
/// TODO Protection against huge size /// TODO Protection against huge size
events_list.clear(); events_list.clear();
@ -129,7 +128,6 @@ struct AggregateFunctionWindowFunnelData
events_list.emplace_back(timestamp, event); events_list.emplace_back(timestamp, event);
} }
} }
}; };
/** Calculates the max event level in a sliding window. /** Calculates the max event level in a sliding window.
@ -138,11 +136,12 @@ struct AggregateFunctionWindowFunnelData
* Usage: * Usage:
* - windowFunnel(window)(timestamp, cond1, cond2, cond3, ....) * - windowFunnel(window)(timestamp, cond1, cond2, cond3, ....)
*/ */
class AggregateFunctionWindowFunnel final : public IAggregateFunctionDataHelper<AggregateFunctionWindowFunnelData, AggregateFunctionWindowFunnel> class AggregateFunctionWindowFunnel final
: public IAggregateFunctionDataHelper<AggregateFunctionWindowFunnelData, AggregateFunctionWindowFunnel>
{ {
private: private:
UInt32 window; UInt32 window;
UInt8 events_size; UInt8 events_size;
// Loop through the entire events_list, update the event timestamp value // Loop through the entire events_list, update the event timestamp value
@ -161,54 +160,59 @@ private:
// events_timestamp stores the timestamp that lastest level 1 happen. // events_timestamp stores the timestamp that lastest level 1 happen.
// timestamp defaults to -1, which unsigned timestamp value never meet // timestamp defaults to -1, which unsigned timestamp value never meet
std::vector<Int32> events_timestamp(events_size, -1); std::vector<Int32> events_timestamp(events_size, -1);
for(const auto i : ext::range(0, data.size())) for (const auto i : ext::range(0, data.size()))
{ {
const auto & timestamp = (data.events_list)[i].first; const auto & timestamp = (data.events_list)[i].first;
const auto & event_idx = (data.events_list)[i].second - 1; const auto & event_idx = (data.events_list)[i].second - 1;
if(event_idx == 0) if (event_idx == 0)
events_timestamp[0] = timestamp; events_timestamp[0] = timestamp;
else if(events_timestamp[event_idx - 1] >= 0 && timestamp <= events_timestamp[event_idx - 1] + window) else if (events_timestamp[event_idx - 1] >= 0 && timestamp <= events_timestamp[event_idx - 1] + window)
{ {
events_timestamp[event_idx] = events_timestamp[event_idx - 1]; events_timestamp[event_idx] = events_timestamp[event_idx - 1];
if(event_idx + 1 == events_size) return events_size; if (event_idx + 1 == events_size)
return events_size;
} }
} }
for(size_t event = events_timestamp.size(); event > 0; --event) for (size_t event = events_timestamp.size(); event > 0; --event)
{ {
if(events_timestamp[event - 1] >= 0) return event; if (events_timestamp[event - 1] >= 0)
return event;
} }
return 0; return 0;
} }
public: public:
String getName() const override
String getName() const override { return "windowFunnel"; } {
return "windowFunnel";
}
AggregateFunctionWindowFunnel(const DataTypes & arguments, const Array & params) AggregateFunctionWindowFunnel(const DataTypes & arguments, const Array & params)
{ {
DataTypePtr windowType = arguments[0]; DataTypePtr windowType = arguments[0];
const auto time_arg = arguments.front().get(); const auto time_arg = arguments.front().get();
if (!typeid_cast<const DataTypeDateTime *>(time_arg) && !typeid_cast<const DataTypeUInt32 *>(time_arg) ) if (!typeid_cast<const DataTypeDateTime *>(time_arg) && !typeid_cast<const DataTypeUInt32 *>(time_arg))
throw Exception{"Illegal type " + time_arg->getName() + " of first argument of aggregate function " throw Exception{"Illegal type " + time_arg->getName() + " of first argument of aggregate function " + getName()
+ getName() + ", must be DateTime or UInt32"}; + ", must be DateTime or UInt32"};
if (arguments.size() - 1 > AggregateFunctionWindowFunnelData::max_events) if (arguments.size() - 1 > AggregateFunctionWindowFunnelData::max_events)
throw Exception{"Aggregate function " + getName() + " supports up to " + throw Exception{"Aggregate function " + getName() + " supports up to " + toString(AggregateFunctionWindowFunnelData::max_events)
toString(AggregateFunctionWindowFunnelData::max_events) + " event arguments.", + " event arguments.",
ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION}; ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION};
for(const auto i : ext::range(1, arguments.size())) for (const auto i : ext::range(1, arguments.size()))
{ {
auto cond_arg = arguments[i].get(); auto cond_arg = arguments[i].get();
if (!typeid_cast<const DataTypeUInt8 *>(cond_arg)) if (!typeid_cast<const DataTypeUInt8 *>(cond_arg))
throw Exception{"Illegal type " + cond_arg->getName() + " of argument " + toString(i + 1) + throw Exception{"Illegal type " + cond_arg->getName() + " of argument " + toString(i + 1) + " of aggregate function "
" of aggregate function " + getName() + ", must be UInt8", + getName() + ", must be UInt8",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
} }
if (params.size() != 1) if (params.size() != 1)
throw Exception("Aggregate function " + getName() + " requires exactly 1 args(timestamp_window).", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); throw Exception("Aggregate function " + getName() + " requires exactly 1 args(timestamp_window).",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
events_size = arguments.size() - 1; events_size = arguments.size() - 1;
window = params[0].safeGet<UInt64>(); window = params[0].safeGet<UInt64>();
@ -223,20 +227,18 @@ public:
void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override
{ {
UInt8 event_level = 0; UInt8 event_level = 0;
for(const auto i : ext::range(1, events_size + 1)) for (const auto i : ext::range(1, events_size + 1))
{ {
auto event = static_cast<const ColumnVector<UInt8> *>(columns[i])->getData()[row_num]; auto event = static_cast<const ColumnVector<UInt8> *>(columns[i])->getData()[row_num];
if(event){ if (event)
event_level = i; {
break; event_level = i;
} break;
}
} }
if(event_level) if (event_level)
{ {
this->data(place).add( this->data(place).add(static_cast<const ColumnVector<UInt32> *>(columns[0])->getData()[row_num], event_level);
static_cast<const ColumnVector<UInt32> *>(columns[0])->getData()[row_num],
event_level
);
} }
} }
@ -260,7 +262,9 @@ public:
static_cast<ColumnUInt8 &>(to).getData().push_back(getEventLevel(this->data(place))); static_cast<ColumnUInt8 &>(to).getData().push_back(getEventLevel(this->data(place)));
} }
const char * getHeaderFilePath() const override { return __FILE__; } const char * getHeaderFilePath() const override
{
return __FILE__;
}
}; };
} }