mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge pull request #21888 from rf/rf/deltasum-with-timestamp
add `deltaSumTimestamp` + docs, tests & minor improvement to `deltaSum`
This commit is contained in:
commit
cb0bd3df5f
@ -6,6 +6,10 @@ toc_priority: 141
|
||||
|
||||
Sums the arithmetic difference between consecutive rows. If the difference is negative, it is ignored.
|
||||
|
||||
Note that the underlying data must be sorted in order for this function to work properly.
|
||||
If you would like to use this function in a materialized view, you most likely want to use the
|
||||
[deltaSumTimestamp](deltasumtimestamp.md) method instead.
|
||||
|
||||
**Syntax**
|
||||
|
||||
``` sql
|
||||
|
@ -0,0 +1,41 @@
|
||||
---
|
||||
toc_priority: 141
|
||||
---
|
||||
|
||||
# deltaSumTimestamp {#agg_functions-deltasum}
|
||||
|
||||
Syntax: `deltaSumTimestamp(value, timestamp)`
|
||||
|
||||
Adds the differences between consecutive rows. If the difference is negative, it is ignored.
|
||||
Uses `timestamp` to order values.
|
||||
|
||||
This function is primarily for materialized views that are ordered by some time bucket aligned
|
||||
timestamp, for example a `toStartOfMinute` bucket. Because the rows in such a materialized view
|
||||
will all have the same timestamp, it is impossible for them to be merged in the "right" order. This
|
||||
function keeps track of the `timestamp` of the values it's seen, so it's possible to order the states
|
||||
correctly during merging.
|
||||
|
||||
To calculate the delta sum across an ordered collection you can simply use the
|
||||
[deltaSum](./deltasum.md) function.
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `value` must be some [Integer](../../data-types/int-uint.md) type or [Float](../../data-types/float.md) type or a [Date](../../data-types/date.md) or [DateTime](../../data-types/datetime.md).
|
||||
- `timestamp` must be some [Integer](../../data-types/int-uint.md) type or [Float](../../data-types/float.md) type or a [Date](../../data-types/date.md) or [DateTime](../../data-types/datetime.md).
|
||||
|
||||
**Returned value**
|
||||
|
||||
- Accumulated differences between consecutive values, ordered by the `timestamp` parameter.
|
||||
|
||||
**Example**
|
||||
|
||||
```sql
|
||||
SELECT deltaSumTimestamp(value, timestamp)
|
||||
FROM (select number as timestamp, [0, 4, 8, 3, 0, 0, 0, 1, 3, 5][number] as value from numbers(1, 10))
|
||||
```
|
||||
|
||||
``` text
|
||||
┌─deltaSumTimestamp(value, timestamp)─┐
|
||||
│ 13 │
|
||||
└─────────────────────────────────────┘
|
||||
```
|
@ -22,8 +22,7 @@ struct AggregationFunctionDeltaSumData
|
||||
T sum = 0;
|
||||
T last = 0;
|
||||
T first = 0;
|
||||
bool seen_last = false;
|
||||
bool seen_first = false;
|
||||
bool seen = false;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
@ -49,18 +48,17 @@ public:
|
||||
{
|
||||
auto value = assert_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num];
|
||||
|
||||
if ((this->data(place).last < value) && this->data(place).seen_last)
|
||||
if ((this->data(place).last < value) && this->data(place).seen)
|
||||
{
|
||||
this->data(place).sum += (value - this->data(place).last);
|
||||
}
|
||||
|
||||
this->data(place).last = value;
|
||||
this->data(place).seen_last = true;
|
||||
|
||||
if (!this->data(place).seen_first)
|
||||
if (!this->data(place).seen)
|
||||
{
|
||||
this->data(place).first = value;
|
||||
this->data(place).seen_first = true;
|
||||
this->data(place).seen = true;
|
||||
}
|
||||
}
|
||||
|
||||
@ -69,7 +67,7 @@ public:
|
||||
auto place_data = &this->data(place);
|
||||
auto rhs_data = &this->data(rhs);
|
||||
|
||||
if ((place_data->last < rhs_data->first) && place_data->seen_last && rhs_data->seen_first)
|
||||
if ((place_data->last < rhs_data->first) && place_data->seen && rhs_data->seen)
|
||||
{
|
||||
// If the lhs last number seen is less than the first number the rhs saw, the lhs is before
|
||||
// the rhs, for example [0, 2] [4, 7]. So we want to add the deltasums, but also add the
|
||||
@ -79,7 +77,7 @@ public:
|
||||
place_data->sum += rhs_data->sum + (rhs_data->first - place_data->last);
|
||||
place_data->last = rhs_data->last;
|
||||
}
|
||||
else if ((rhs_data->first < place_data->last && rhs_data->seen_last && place_data->seen_first))
|
||||
else if ((rhs_data->first < place_data->last && rhs_data->seen && place_data->seen))
|
||||
{
|
||||
// In the opposite scenario, the lhs comes after the rhs, e.g. [4, 6] [1, 2]. Since we
|
||||
// assume the input interval states are sorted by time, we assume this is a counter
|
||||
@ -89,16 +87,15 @@ public:
|
||||
place_data->sum += rhs_data->sum;
|
||||
place_data->last = rhs_data->last;
|
||||
}
|
||||
else if (rhs_data->seen_first && !place_data->seen_first)
|
||||
else if (rhs_data->seen && !place_data->seen)
|
||||
{
|
||||
// If we're here then the lhs is an empty state and the rhs does have some state, so
|
||||
// we'll just take that state.
|
||||
|
||||
place_data->first = rhs_data->first;
|
||||
place_data->seen_first = rhs_data->seen_first;
|
||||
place_data->last = rhs_data->last;
|
||||
place_data->seen_last = rhs_data->seen_last;
|
||||
place_data->sum = rhs_data->sum;
|
||||
place_data->seen = rhs_data->seen;
|
||||
}
|
||||
|
||||
// Otherwise lhs either has data or is uninitialized, so we don't need to modify its values.
|
||||
@ -109,8 +106,7 @@ public:
|
||||
writeIntBinary(this->data(place).sum, buf);
|
||||
writeIntBinary(this->data(place).first, buf);
|
||||
writeIntBinary(this->data(place).last, buf);
|
||||
writePODBinary<bool>(this->data(place).seen_first, buf);
|
||||
writePODBinary<bool>(this->data(place).seen_last, buf);
|
||||
writePODBinary<bool>(this->data(place).seen, buf);
|
||||
}
|
||||
|
||||
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override
|
||||
@ -118,8 +114,7 @@ public:
|
||||
readIntBinary(this->data(place).sum, buf);
|
||||
readIntBinary(this->data(place).first, buf);
|
||||
readIntBinary(this->data(place).last, buf);
|
||||
readPODBinary<bool>(this->data(place).seen_first, buf);
|
||||
readPODBinary<bool>(this->data(place).seen_last, buf);
|
||||
readPODBinary<bool>(this->data(place).seen, buf);
|
||||
}
|
||||
|
||||
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
|
||||
|
@ -0,0 +1,51 @@
|
||||
#include <AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
AggregateFunctionPtr createAggregateFunctionDeltaSumTimestamp(
|
||||
const String & name,
|
||||
const DataTypes & arguments,
|
||||
const Array & params)
|
||||
{
|
||||
assertNoParameters(name, params);
|
||||
|
||||
if (arguments.size() != 2)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name,
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!isInteger(arguments[0]) && !isFloat(arguments[0]) && !isDateOrDateTime(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument for aggregate function " +
|
||||
name + ", must be Int, Float, Date, DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!isInteger(arguments[1]) && !isFloat(arguments[1]) && !isDateOrDateTime(arguments[1]))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of argument for aggregate function " +
|
||||
name + ", must be Int, Float, Date, DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return AggregateFunctionPtr(createWithTwoNumericOrDateTypes<AggregationFunctionDeltaSumTimestamp>(
|
||||
*arguments[0], *arguments[1], arguments, params));
|
||||
}
|
||||
}
|
||||
|
||||
void registerAggregateFunctionDeltaSumTimestamp(AggregateFunctionFactory & factory)
|
||||
{
|
||||
AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = true };
|
||||
|
||||
factory.registerFunction("deltaSumTimestamp", { createAggregateFunctionDeltaSumTimestamp, properties });
|
||||
}
|
||||
|
||||
}
|
172
src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h
Normal file
172
src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h
Normal file
@ -0,0 +1,172 @@
|
||||
#pragma once
|
||||
|
||||
#include <type_traits>
|
||||
#include <experimental/type_traits>
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename ValueType, typename TimestampType>
|
||||
struct AggregationFunctionDeltaSumTimestampData
|
||||
{
|
||||
ValueType sum = 0;
|
||||
ValueType first = 0;
|
||||
ValueType last = 0;
|
||||
TimestampType first_ts = 0;
|
||||
TimestampType last_ts = 0;
|
||||
bool seen = false;
|
||||
};
|
||||
|
||||
template <typename ValueType, typename TimestampType>
|
||||
class AggregationFunctionDeltaSumTimestamp final
|
||||
: public IAggregateFunctionDataHelper<
|
||||
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
|
||||
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>
|
||||
>
|
||||
{
|
||||
public:
|
||||
AggregationFunctionDeltaSumTimestamp(const DataTypes & arguments, const Array & params)
|
||||
: IAggregateFunctionDataHelper<
|
||||
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
|
||||
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>
|
||||
>{arguments, params}
|
||||
{}
|
||||
|
||||
AggregationFunctionDeltaSumTimestamp()
|
||||
: IAggregateFunctionDataHelper<
|
||||
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
|
||||
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>
|
||||
>{}
|
||||
{}
|
||||
|
||||
bool allocatesMemoryInArena() const override { return false; }
|
||||
|
||||
String getName() const override { return "deltaSumTimestamp"; }
|
||||
|
||||
DataTypePtr getReturnType() const override { return std::make_shared<DataTypeNumber<ValueType>>(); }
|
||||
|
||||
void NO_SANITIZE_UNDEFINED ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
|
||||
{
|
||||
auto value = assert_cast<const ColumnVector<ValueType> &>(*columns[0]).getData()[row_num];
|
||||
auto ts = assert_cast<const ColumnVector<TimestampType> &>(*columns[1]).getData()[row_num];
|
||||
|
||||
if ((this->data(place).last < value) && this->data(place).seen)
|
||||
{
|
||||
this->data(place).sum += (value - this->data(place).last);
|
||||
}
|
||||
|
||||
this->data(place).last = value;
|
||||
this->data(place).last_ts = ts;
|
||||
|
||||
if (!this->data(place).seen)
|
||||
{
|
||||
this->data(place).first = value;
|
||||
this->data(place).seen = true;
|
||||
this->data(place).first_ts = ts;
|
||||
}
|
||||
}
|
||||
|
||||
// before returns true if lhs is before rhs or false if it is not or can't be determined
|
||||
bool ALWAYS_INLINE before (
|
||||
const AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType> * lhs,
|
||||
const AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType> * rhs
|
||||
) const
|
||||
{
|
||||
if (lhs->last_ts < rhs->first_ts)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
if (lhs->last_ts == rhs->first_ts && (lhs->last_ts < rhs->last_ts || lhs->first_ts < rhs->first_ts))
|
||||
{
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
void NO_SANITIZE_UNDEFINED ALWAYS_INLINE merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
|
||||
{
|
||||
auto place_data = &this->data(place);
|
||||
auto rhs_data = &this->data(rhs);
|
||||
|
||||
if (!place_data->seen && rhs_data->seen)
|
||||
{
|
||||
place_data->sum = rhs_data->sum;
|
||||
place_data->seen = true;
|
||||
place_data->first = rhs_data->first;
|
||||
place_data->first_ts = rhs_data->first_ts;
|
||||
place_data->last = rhs_data->last;
|
||||
place_data->last_ts = rhs_data->last_ts;
|
||||
}
|
||||
else if (place_data->seen && !rhs_data->seen)
|
||||
return;
|
||||
else if (before(place_data, rhs_data))
|
||||
{
|
||||
// This state came before the rhs state
|
||||
|
||||
if (rhs_data->first > place_data->last)
|
||||
place_data->sum += (rhs_data->first - place_data->last);
|
||||
place_data->sum += rhs_data->sum;
|
||||
place_data->last = rhs_data->last;
|
||||
place_data->last_ts = rhs_data->last_ts;
|
||||
}
|
||||
else if (before(rhs_data, place_data))
|
||||
{
|
||||
// This state came after the rhs state
|
||||
|
||||
if (place_data->first > rhs_data->last)
|
||||
place_data->sum += (place_data->first - rhs_data->last);
|
||||
place_data->sum += rhs_data->sum;
|
||||
place_data->first = rhs_data->first;
|
||||
place_data->first_ts = rhs_data->first_ts;
|
||||
}
|
||||
else
|
||||
{
|
||||
// If none of those conditions matched, it means both states we are merging have all
|
||||
// same timestamps. We have to pick either the smaller or larger value so that the
|
||||
// result is deterministic.
|
||||
|
||||
if (place_data->first < rhs_data->first)
|
||||
{
|
||||
place_data->first = rhs_data->first;
|
||||
place_data->last = rhs_data->last;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override
|
||||
{
|
||||
writeIntBinary(this->data(place).sum, buf);
|
||||
writeIntBinary(this->data(place).first, buf);
|
||||
writeIntBinary(this->data(place).first_ts, buf);
|
||||
writeIntBinary(this->data(place).last, buf);
|
||||
writeIntBinary(this->data(place).last_ts, buf);
|
||||
writePODBinary<bool>(this->data(place).seen, buf);
|
||||
}
|
||||
|
||||
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override
|
||||
{
|
||||
readIntBinary(this->data(place).sum, buf);
|
||||
readIntBinary(this->data(place).first, buf);
|
||||
readIntBinary(this->data(place).first_ts, buf);
|
||||
readIntBinary(this->data(place).last, buf);
|
||||
readIntBinary(this->data(place).last_ts, buf);
|
||||
readPODBinary<bool>(this->data(place).seen, buf);
|
||||
}
|
||||
|
||||
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
|
||||
{
|
||||
assert_cast<ColumnVector<ValueType> &>(to).getData().push_back(this->data(place).sum);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -184,6 +184,46 @@ static IAggregateFunction * createWithTwoNumericTypes(const IDataType & first_ty
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
template <typename FirstType, template <typename, typename> class AggregateFunctionTemplate, typename... TArgs>
|
||||
static IAggregateFunction * createWithTwoNumericOrDateTypesSecond(const IDataType & second_type, TArgs && ... args)
|
||||
{
|
||||
WhichDataType which(second_type);
|
||||
#define DISPATCH(TYPE) \
|
||||
if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate<FirstType, TYPE>(std::forward<TArgs>(args)...);
|
||||
FOR_NUMERIC_TYPES(DISPATCH)
|
||||
#undef DISPATCH
|
||||
if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate<FirstType, Int8>(std::forward<TArgs>(args)...);
|
||||
if (which.idx == TypeIndex::Enum16) return new AggregateFunctionTemplate<FirstType, Int16>(std::forward<TArgs>(args)...);
|
||||
|
||||
/// expects that DataTypeDate based on UInt16, DataTypeDateTime based on UInt32 and UUID based on UInt128
|
||||
if (which.idx == TypeIndex::Date) return new AggregateFunctionTemplate<FirstType, UInt16>(std::forward<TArgs>(args)...);
|
||||
if (which.idx == TypeIndex::DateTime) return new AggregateFunctionTemplate<FirstType, UInt32>(std::forward<TArgs>(args)...);
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
template <template <typename, typename> class AggregateFunctionTemplate, typename... TArgs>
|
||||
static IAggregateFunction * createWithTwoNumericOrDateTypes(const IDataType & first_type, const IDataType & second_type, TArgs && ... args)
|
||||
{
|
||||
WhichDataType which(first_type);
|
||||
#define DISPATCH(TYPE) \
|
||||
if (which.idx == TypeIndex::TYPE) \
|
||||
return createWithTwoNumericOrDateTypesSecond<TYPE, AggregateFunctionTemplate>(second_type, std::forward<TArgs>(args)...);
|
||||
FOR_NUMERIC_TYPES(DISPATCH)
|
||||
#undef DISPATCH
|
||||
if (which.idx == TypeIndex::Enum8)
|
||||
return createWithTwoNumericOrDateTypesSecond<Int8, AggregateFunctionTemplate>(second_type, std::forward<TArgs>(args)...);
|
||||
if (which.idx == TypeIndex::Enum16)
|
||||
return createWithTwoNumericOrDateTypesSecond<Int16, AggregateFunctionTemplate>(second_type, std::forward<TArgs>(args)...);
|
||||
|
||||
/// expects that DataTypeDate based on UInt16, DataTypeDateTime based on UInt32 and UUID based on UInt128
|
||||
if (which.idx == TypeIndex::Date)
|
||||
return createWithTwoNumericOrDateTypesSecond<UInt16, AggregateFunctionTemplate>(second_type, std::forward<TArgs>(args)...);
|
||||
if (which.idx == TypeIndex::DateTime)
|
||||
return createWithTwoNumericOrDateTypesSecond<UInt32, AggregateFunctionTemplate>(second_type, std::forward<TArgs>(args)...);
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
template <template <typename> class AggregateFunctionTemplate, typename... TArgs>
|
||||
static IAggregateFunction * createWithStringType(const IDataType & argument_type, TArgs && ... args)
|
||||
{
|
||||
|
@ -12,6 +12,7 @@ void registerAggregateFunctionAvg(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionAvgWeighted(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionCount(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionDeltaSum(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionDeltaSumTimestamp(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionGroupArray(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionGroupArrayInsertAt(AggregateFunctionFactory &);
|
||||
@ -71,6 +72,7 @@ void registerAggregateFunctions()
|
||||
registerAggregateFunctionAvgWeighted(factory);
|
||||
registerAggregateFunctionCount(factory);
|
||||
registerAggregateFunctionDeltaSum(factory);
|
||||
registerAggregateFunctionDeltaSumTimestamp(factory);
|
||||
registerAggregateFunctionGroupArray(factory);
|
||||
registerAggregateFunctionGroupUniqArray(factory);
|
||||
registerAggregateFunctionGroupArrayInsertAt(factory);
|
||||
|
@ -20,6 +20,7 @@ SRCS(
|
||||
AggregateFunctionCombinatorFactory.cpp
|
||||
AggregateFunctionCount.cpp
|
||||
AggregateFunctionDeltaSum.cpp
|
||||
AggregateFunctionDeltaSumTimestamp.cpp
|
||||
AggregateFunctionDistinct.cpp
|
||||
AggregateFunctionEntropy.cpp
|
||||
AggregateFunctionFactory.cpp
|
||||
|
@ -0,0 +1,5 @@
|
||||
10
|
||||
10
|
||||
8
|
||||
8
|
||||
13
|
5
tests/queries/0_stateless/01762_deltasumtimestamp.sql
Normal file
5
tests/queries/0_stateless/01762_deltasumtimestamp.sql
Normal file
@ -0,0 +1,5 @@
|
||||
select deltaSumTimestampMerge(state) from (select deltaSumTimestampState(value, timestamp) as state from (select toDate(number) as timestamp, [4, 5, 5, 5][number-4] as value from numbers(5, 4)) UNION ALL select deltaSumTimestampState(value, timestamp) as state from (select toDate(number) as timestamp, [0, 4, 8, 3][number] as value from numbers(1, 4)));
|
||||
select deltaSumTimestampMerge(state) from (select deltaSumTimestampState(value, timestamp) as state from (select number as timestamp, [0, 4, 8, 3][number] as value from numbers(1, 4)) UNION ALL select deltaSumTimestampState(value, timestamp) as state from (select number as timestamp, [4, 5, 5, 5][number-4] as value from numbers(5, 4)));
|
||||
select deltaSumTimestamp(value, timestamp) from (select toDateTime(number) as timestamp, [0, 4, 8, 3][number] as value from numbers(1, 4));
|
||||
select deltaSumTimestamp(value, timestamp) from (select toDateTime(number) as timestamp, [0, 4.5, 8, 3][number] as value from numbers(1, 4));
|
||||
select deltaSumTimestamp(value, timestamp) from (select number as timestamp, [0, 4, 8, 3, 0, 0, 0, 1, 3, 5][number] as value from numbers(1, 10));
|
Loading…
Reference in New Issue
Block a user