mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-19 14:11:58 +00:00
fix wrong interval
This commit is contained in:
parent
e19653618c
commit
551d1ea875
@ -104,18 +104,23 @@ struct TimeSlotsImpl
|
||||
|
||||
/// The following three methods process DateTime64 type
|
||||
static void vectorVector(
|
||||
const PaddedPODArray<DateTime64> & starts, const PaddedPODArray<Decimal64> & durations, UInt32 time_slot_size,
|
||||
PaddedPODArray<DateTime64> & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale)
|
||||
const PaddedPODArray<DateTime64> & starts, const PaddedPODArray<Decimal64> & durations, Decimal64 time_slot_size,
|
||||
PaddedPODArray<DateTime64> & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale)
|
||||
{
|
||||
size_t size = starts.size();
|
||||
|
||||
result_offsets.resize(size);
|
||||
result_values.reserve(size);
|
||||
|
||||
int dt_multiplier = dt_scale < duration_scale ? DecimalUtils::scaleMultiplier<DateTime64>(std::abs(duration_scale - dt_scale)) : 1;
|
||||
int dur_multiplier = dt_scale > duration_scale ? DecimalUtils::scaleMultiplier<DateTime64>(std::abs(dt_scale - duration_scale)) : 1;
|
||||
UInt16 max_scale = dt_scale > duration_scale ? dt_scale : duration_scale;
|
||||
max_scale = time_slot_scale > max_scale ? time_slot_scale : max_scale;
|
||||
|
||||
Int64 dt_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(max_scale - dt_scale);
|
||||
Int64 dur_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(max_scale - duration_scale);
|
||||
Int64 ts_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(max_scale - time_slot_scale);
|
||||
|
||||
ColumnArray::Offset current_offset = 0;
|
||||
time_slot_size = time_slot_size.value * ts_multiplier;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
for (DateTime64 value = (starts[i] * dt_multiplier) / time_slot_size, end = (starts[i] + durations[i] * dur_multiplier) / time_slot_size; value <= end; value += 1)
|
||||
@ -128,19 +133,24 @@ struct TimeSlotsImpl
|
||||
}
|
||||
|
||||
static void vectorConstant(
|
||||
const PaddedPODArray<DateTime64> & starts, Decimal64 duration, UInt32 time_slot_size,
|
||||
PaddedPODArray<DateTime64> & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale)
|
||||
const PaddedPODArray<DateTime64> & starts, Decimal64 duration, Decimal64 time_slot_size,
|
||||
PaddedPODArray<DateTime64> & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale)
|
||||
{
|
||||
size_t size = starts.size();
|
||||
|
||||
result_offsets.resize(size);
|
||||
result_values.reserve(size);
|
||||
|
||||
int dt_multiplier = dt_scale < duration_scale ? DecimalUtils::scaleMultiplier<DateTime64>(std::abs(duration_scale - dt_scale)) : 1;
|
||||
int dur_multiplier = dt_scale > duration_scale ? DecimalUtils::scaleMultiplier<DateTime64>(std::abs(dt_scale - duration_scale)) : 1;
|
||||
UInt16 max_scale = dt_scale > duration_scale ? dt_scale : duration_scale;
|
||||
max_scale = time_slot_scale > max_scale ? time_slot_scale : max_scale;
|
||||
|
||||
Int64 dt_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(max_scale - dt_scale);
|
||||
Int64 dur_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(max_scale - duration_scale);
|
||||
Int64 ts_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(max_scale - time_slot_scale);
|
||||
|
||||
ColumnArray::Offset current_offset = 0;
|
||||
duration = duration * dur_multiplier;
|
||||
time_slot_size = time_slot_size.value * ts_multiplier;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
for (DateTime64 value = (starts[i] * dt_multiplier) / time_slot_size, end = (starts[i] + duration) / time_slot_size; value <= end; value += 1)
|
||||
@ -153,19 +163,24 @@ struct TimeSlotsImpl
|
||||
}
|
||||
|
||||
static void constantVector(
|
||||
DateTime64 start, const PaddedPODArray<Decimal64> & durations, UInt32 time_slot_size,
|
||||
PaddedPODArray<DateTime64> & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale)
|
||||
DateTime64 start, const PaddedPODArray<Decimal64> & durations, Decimal64 time_slot_size,
|
||||
PaddedPODArray<DateTime64> & result_values, ColumnArray::Offsets & result_offsets, UInt16 dt_scale, UInt16 duration_scale, UInt16 time_slot_scale)
|
||||
{
|
||||
size_t size = durations.size();
|
||||
|
||||
result_offsets.resize(size);
|
||||
result_values.reserve(size);
|
||||
|
||||
int dt_multiplier = dt_scale < duration_scale ? DecimalUtils::scaleMultiplier<DateTime64>(std::abs(duration_scale - dt_scale)) : 1;
|
||||
int dur_multiplier = dt_scale > duration_scale ? DecimalUtils::scaleMultiplier<DateTime64>(std::abs(dt_scale - duration_scale)) : 1;
|
||||
UInt16 max_scale = dt_scale > duration_scale ? dt_scale : duration_scale;
|
||||
max_scale = time_slot_scale > max_scale ? time_slot_scale : max_scale;
|
||||
|
||||
Int64 dt_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(max_scale - dt_scale);
|
||||
Int64 dur_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(max_scale - duration_scale);
|
||||
Int64 ts_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(max_scale - time_slot_scale);
|
||||
|
||||
ColumnArray::Offset current_offset = 0;
|
||||
start = dt_multiplier * start;
|
||||
time_slot_size = time_slot_size.value * ts_multiplier;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
for (DateTime64 value = start / time_slot_size, end = (start + durations[i] * dur_multiplier) / time_slot_size; value <= end; value += 1)
|
||||
@ -203,43 +218,33 @@ public:
|
||||
+ toString(arguments.size()) + ", should be 2 or 3",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName()
|
||||
+ ". Must be DateTime or DateTime64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!isNumber(arguments[1].type))
|
||||
{
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() + ". Must be numeric type.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
if (isDateTime(arguments[0].type))
|
||||
{
|
||||
if (!isUnsignedInteger(arguments[1].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() + ". Must be unsigned integer when first argument is DateTime.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 3 && !isUnsignedInteger(arguments[2].type))
|
||||
if (!WhichDataType(arguments[1].type).isUInt32())
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[2].type->getName() + " of third argument of function " + getName() + ". Must be unsigned integer when first argument is DateTime.",
|
||||
"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() + ". Must be UInt32 when first argument is DateTime.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isUInt32())
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[2].type->getName() + " of third argument of function " + getName() + ". Must be UInt32 when first argument is DateTime.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else if (isDateTime64(arguments[0].type))
|
||||
{
|
||||
if (!WhichDataType(arguments[1].type).isDecimal64())
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() + ". Must be Decimal64 when first argument is DateTime64.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isDecimal64())
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[2].type->getName() + " of third argument of function " + getName() + ". Must be Decimal64 when first argument is DateTime64.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!isNumber(arguments[1].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() + ". Must be numeric when first argument is DateTime64.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 3 && !isNumber(arguments[2].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[2].type->getName() + " of third argument of function " + getName() + ". Must be numeric when first argument is DateTime64.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
|
||||
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName()
|
||||
+ ". Must be DateTime or DateTime64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
/// If time zone is specified for source data type, attach it to the resulting type.
|
||||
/// Note that there is no explicit time zone argument for this function (we specify 2 as an argument number with explicit time zone).
|
||||
@ -259,20 +264,19 @@ public:
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override
|
||||
{
|
||||
UInt32 time_slot_size = 1800;
|
||||
|
||||
if (arguments.size() == 3)
|
||||
{
|
||||
const auto * time_slot_column = checkAndGetColumnConst<ColumnConst>(arguments[2].column.get());
|
||||
if (!time_slot_column)
|
||||
throw Exception("Third argument for function " + getName() + " must be constant integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (time_slot_size = time_slot_column->getValue<UInt32>(); time_slot_size == 0)
|
||||
throw Exception("Third argument for function " + getName() + " must be greater than zero", ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
if (isDateTime(arguments[0].type))
|
||||
{
|
||||
UInt32 time_slot_size = 1800;
|
||||
if (arguments.size() == 3)
|
||||
{
|
||||
const auto * time_slot_column = checkAndGetColumnConst<ColumnConst>(arguments[2].column.get());
|
||||
if (!time_slot_column)
|
||||
throw Exception("Third argument for function " + getName() + " must be constant integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (time_slot_size = time_slot_column->getValue<UInt32>(); time_slot_size == 0)
|
||||
throw Exception("Third argument for function " + getName() + " must be greater than zero", ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
const auto * dt_starts = checkAndGetColumn<ColumnUInt32>(arguments[0].column.get());
|
||||
const auto * dt_const_starts = checkAndGetColumnConst<ColumnUInt32>(arguments[0].column.get());
|
||||
|
||||
@ -300,6 +304,19 @@ public:
|
||||
}
|
||||
else if (isDateTime64(arguments[0].type))
|
||||
{
|
||||
Decimal64 time_slot_size = Decimal64(1800);
|
||||
UInt16 time_slot_scale = 0;
|
||||
if (arguments.size() == 3)
|
||||
{
|
||||
const auto * time_slot_column = checkAndGetColumnConst<ColumnDecimal<Decimal64>>(arguments[2].column.get());
|
||||
if (!time_slot_column)
|
||||
throw Exception("Third argument for function " + getName() + " must be constant integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (time_slot_size = time_slot_column->getValue<Decimal64>(); time_slot_size == 0)
|
||||
throw Exception("Third argument for function " + getName() + " must be greater than zero", ErrorCodes::ILLEGAL_COLUMN);
|
||||
time_slot_scale = assert_cast<const DataTypeDecimalBase<Decimal64> *>(arguments[2].type.get())->getScale();
|
||||
}
|
||||
|
||||
const auto * dt64_starts = checkAndGetColumn<DataTypeDateTime64::ColumnType>(arguments[0].column.get());
|
||||
const auto * dt64_const_starts = checkAndGetColumnConst<DataTypeDateTime64::ColumnType>(arguments[0].column.get());
|
||||
|
||||
@ -314,19 +331,19 @@ public:
|
||||
|
||||
if (dt64_starts && durations)
|
||||
{
|
||||
TimeSlotsImpl::vectorVector(dt64_starts->getData(), durations->getData(), time_slot_size, res_values, res->getOffsets(), dt64_scale, duration_scale);
|
||||
TimeSlotsImpl::vectorVector(dt64_starts->getData(), durations->getData(), time_slot_size, res_values, res->getOffsets(), dt64_scale, duration_scale, time_slot_scale);
|
||||
return res;
|
||||
}
|
||||
else if (dt64_starts && const_durations)
|
||||
{
|
||||
TimeSlotsImpl::vectorConstant(
|
||||
dt64_starts->getData(), const_durations->getValue<Decimal64>(), time_slot_size, res_values, res->getOffsets(), dt64_scale, duration_scale);
|
||||
dt64_starts->getData(), const_durations->getValue<Decimal64>(), time_slot_size, res_values, res->getOffsets(), dt64_scale, duration_scale, time_slot_scale);
|
||||
return res;
|
||||
}
|
||||
else if (dt64_const_starts && durations)
|
||||
{
|
||||
TimeSlotsImpl::constantVector(
|
||||
dt64_const_starts->getValue<DateTime64>(), durations->getData(), time_slot_size, res_values, res->getOffsets(), dt64_scale, duration_scale);
|
||||
dt64_const_starts->getValue<DateTime64>(), durations->getData(), time_slot_size, res_values, res->getOffsets(), dt64_scale, duration_scale, time_slot_scale);
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user