mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge
This commit is contained in:
commit
a0e2479383
@ -119,38 +119,7 @@ public:
|
||||
getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + elems);
|
||||
}
|
||||
|
||||
ColumnPtr cut(size_t start, size_t length) const override
|
||||
{
|
||||
if (length == 0)
|
||||
return new ColumnArray(data);
|
||||
|
||||
if (start + length > getOffsets().size())
|
||||
throw Exception("Parameter out of bound in IColumnArray::cut() method.",
|
||||
ErrorCodes::PARAMETER_OUT_OF_BOUND);
|
||||
|
||||
size_t nested_offset = offsetAt(start);
|
||||
size_t nested_length = getOffsets()[start + length - 1] - nested_offset;
|
||||
|
||||
ColumnArray * res_ = new ColumnArray(data);
|
||||
ColumnPtr res = res_;
|
||||
|
||||
res_->data = data->cut(nested_offset, nested_length);
|
||||
Offsets_t & res_offsets = res_->getOffsets();
|
||||
|
||||
if (start == 0)
|
||||
{
|
||||
res_offsets.assign(getOffsets().begin(), getOffsets().begin() + length);
|
||||
}
|
||||
else
|
||||
{
|
||||
res_offsets.resize(length);
|
||||
|
||||
for (size_t i = 0; i < length; ++i)
|
||||
res_offsets[i] = getOffsets()[start + i] - nested_offset;
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
ColumnPtr cut(size_t start, size_t length) const override;
|
||||
|
||||
void insert(const Field & x) override
|
||||
{
|
||||
@ -178,82 +147,9 @@ public:
|
||||
getOffsets().push_back(getOffsets().size() == 0 ? 0 : getOffsets().back());
|
||||
}
|
||||
|
||||
ColumnPtr filter(const Filter & filt) const override
|
||||
{
|
||||
size_t size = getOffsets().size();
|
||||
if (size != filt.size())
|
||||
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
ColumnPtr filter(const Filter & filt) const override;
|
||||
|
||||
if (size == 0)
|
||||
return new ColumnArray(data);
|
||||
|
||||
/// Не слишком оптимально. Можно сделать специализацию для массивов известных типов.
|
||||
Filter nested_filt(getOffsets().back());
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (filt[i])
|
||||
memset(&nested_filt[offsetAt(i)], 1, sizeAt(i));
|
||||
else
|
||||
memset(&nested_filt[offsetAt(i)], 0, sizeAt(i));
|
||||
}
|
||||
|
||||
ColumnArray * res_ = new ColumnArray(data);
|
||||
ColumnPtr res = res_;
|
||||
res_->data = data->filter(nested_filt);
|
||||
|
||||
Offsets_t & res_offsets = res_->getOffsets();
|
||||
res_offsets.reserve(size);
|
||||
|
||||
size_t current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (filt[i])
|
||||
{
|
||||
current_offset += sizeAt(i);
|
||||
res_offsets.push_back(current_offset);
|
||||
}
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override
|
||||
{
|
||||
size_t size = getOffsets().size();
|
||||
|
||||
if (limit == 0)
|
||||
limit = size;
|
||||
else
|
||||
limit = std::min(size, limit);
|
||||
|
||||
if (perm.size() < limit)
|
||||
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
if (limit == 0)
|
||||
return new ColumnArray(data);
|
||||
|
||||
Permutation nested_perm(getOffsets().back());
|
||||
|
||||
ColumnArray * res_ = new ColumnArray(data->cloneEmpty());
|
||||
ColumnPtr res = res_;
|
||||
|
||||
Offsets_t & res_offsets = res_->getOffsets();
|
||||
res_offsets.resize(limit);
|
||||
size_t current_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < limit; ++i)
|
||||
{
|
||||
for (size_t j = 0; j < sizeAt(perm[i]); ++j)
|
||||
nested_perm[current_offset + j] = offsetAt(perm[i]) + j;
|
||||
current_offset += sizeAt(perm[i]);
|
||||
res_offsets[i] = current_offset;
|
||||
}
|
||||
|
||||
if (current_offset != 0)
|
||||
res_->data = data->permute(nested_perm, current_offset);
|
||||
|
||||
return res;
|
||||
}
|
||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
|
||||
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override
|
||||
{
|
||||
@ -290,31 +186,7 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
void getPermutation(bool reverse, size_t limit, Permutation & res) const override
|
||||
{
|
||||
size_t s = size();
|
||||
if (limit >= s)
|
||||
limit = 0;
|
||||
|
||||
res.resize(s);
|
||||
for (size_t i = 0; i < s; ++i)
|
||||
res[i] = i;
|
||||
|
||||
if (limit)
|
||||
{
|
||||
if (reverse)
|
||||
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<false>(*this));
|
||||
else
|
||||
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<true>(*this));
|
||||
}
|
||||
else
|
||||
{
|
||||
if (reverse)
|
||||
std::sort(res.begin(), res.end(), less<false>(*this));
|
||||
else
|
||||
std::sort(res.begin(), res.end(), less<true>(*this));
|
||||
}
|
||||
}
|
||||
void getPermutation(bool reverse, size_t limit, Permutation & res) const override;
|
||||
|
||||
void reserve(size_t n) override
|
||||
{
|
||||
@ -364,24 +236,7 @@ public:
|
||||
const ColumnPtr & getOffsetsColumn() const { return offsets; }
|
||||
|
||||
|
||||
ColumnPtr replicate(const Offsets_t & replicate_offsets) const override
|
||||
{
|
||||
/// Не получается реализовать в общем случае.
|
||||
|
||||
if (typeid_cast<const ColumnUInt8 *>(&*data)) return replicate<UInt8>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnUInt16 *>(&*data)) return replicate<UInt16>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnUInt32 *>(&*data)) return replicate<UInt32>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnUInt64 *>(&*data)) return replicate<UInt64>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnInt8 *>(&*data)) return replicate<Int8>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnInt16 *>(&*data)) return replicate<Int16>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnInt32 *>(&*data)) return replicate<Int32>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnInt64 *>(&*data)) return replicate<Int64>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnFloat32 *>(&*data)) return replicate<Float32>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnFloat64 *>(&*data)) return replicate<Float64>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnString *>(&*data)) return replicateString(replicate_offsets);
|
||||
|
||||
throw Exception("Replication of column " + getName() + " is not implemented.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
ColumnPtr replicate(const Offsets_t & replicate_offsets) const override;
|
||||
|
||||
private:
|
||||
ColumnPtr data;
|
||||
@ -393,129 +248,17 @@ private:
|
||||
|
||||
/// Размножить значения, если вложенный столбец - ColumnArray<T>.
|
||||
template <typename T>
|
||||
ColumnPtr replicate(const Offsets_t & replicate_offsets) const
|
||||
{
|
||||
size_t col_size = size();
|
||||
if (col_size != replicate_offsets.size())
|
||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
ColumnPtr res = cloneEmpty();
|
||||
|
||||
if (0 == col_size)
|
||||
return res;
|
||||
|
||||
ColumnArray & res_ = typeid_cast<ColumnArray &>(*res);
|
||||
|
||||
const typename ColumnVector<T>::Container_t & cur_data = typeid_cast<const ColumnVector<T> &>(*data).getData();
|
||||
const Offsets_t & cur_offsets = getOffsets();
|
||||
|
||||
typename ColumnVector<T>::Container_t & res_data = typeid_cast<ColumnVector<T> &>(res_.getData()).getData();
|
||||
Offsets_t & res_offsets = res_.getOffsets();
|
||||
|
||||
res_data.reserve(data->size() / col_size * replicate_offsets.back());
|
||||
res_offsets.reserve(replicate_offsets.back());
|
||||
|
||||
Offset_t prev_replicate_offset = 0;
|
||||
Offset_t prev_data_offset = 0;
|
||||
Offset_t current_new_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < col_size; ++i)
|
||||
{
|
||||
size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset;
|
||||
size_t value_size = cur_offsets[i] - prev_data_offset;
|
||||
|
||||
for (size_t j = 0; j < size_to_replicate; ++j)
|
||||
{
|
||||
current_new_offset += value_size;
|
||||
res_offsets.push_back(current_new_offset);
|
||||
|
||||
res_data.resize(res_data.size() + value_size);
|
||||
memcpy(&res_data[res_data.size() - value_size], &cur_data[prev_data_offset], value_size * sizeof(T));
|
||||
}
|
||||
|
||||
prev_replicate_offset = replicate_offsets[i];
|
||||
prev_data_offset = cur_offsets[i];
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
ColumnPtr replicate(const Offsets_t & replicate_offsets) const;
|
||||
|
||||
/// Размножить значения, если вложенный столбец - ColumnString. Код слишком сложный.
|
||||
ColumnPtr replicateString(const Offsets_t & replicate_offsets) const
|
||||
{
|
||||
size_t col_size = size();
|
||||
if (col_size != replicate_offsets.size())
|
||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
ColumnPtr replicateString(const Offsets_t & replicate_offsets) const;
|
||||
|
||||
ColumnPtr res = cloneEmpty();
|
||||
|
||||
if (0 == col_size)
|
||||
return res;
|
||||
|
||||
ColumnArray & res_ = typeid_cast<ColumnArray &>(*res);
|
||||
|
||||
const ColumnString & cur_string = typeid_cast<const ColumnString &>(*data);
|
||||
const ColumnString::Chars_t & cur_chars = cur_string.getChars();
|
||||
const Offsets_t & cur_string_offsets = cur_string.getOffsets();
|
||||
const Offsets_t & cur_offsets = getOffsets();
|
||||
|
||||
ColumnString::Chars_t & res_chars = typeid_cast<ColumnString &>(res_.getData()).getChars();
|
||||
Offsets_t & res_string_offsets = typeid_cast<ColumnString &>(res_.getData()).getOffsets();
|
||||
Offsets_t & res_offsets = res_.getOffsets();
|
||||
|
||||
res_chars.reserve(cur_chars.size() / col_size * replicate_offsets.back());
|
||||
res_string_offsets.reserve(cur_string_offsets.size() / col_size * replicate_offsets.back());
|
||||
res_offsets.reserve(replicate_offsets.back());
|
||||
|
||||
Offset_t prev_replicate_offset = 0;
|
||||
|
||||
Offset_t prev_cur_offset = 0;
|
||||
Offset_t prev_cur_string_offset = 0;
|
||||
|
||||
Offset_t current_res_offset = 0;
|
||||
Offset_t current_res_string_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < col_size; ++i)
|
||||
{
|
||||
/// Насколько размножить массив.
|
||||
size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset;
|
||||
/// Количество строк в массиве.
|
||||
size_t value_size = cur_offsets[i] - prev_cur_offset;
|
||||
|
||||
size_t sum_chars_size = 0;
|
||||
|
||||
for (size_t j = 0; j < size_to_replicate; ++j)
|
||||
{
|
||||
current_res_offset += value_size;
|
||||
res_offsets.push_back(current_res_offset);
|
||||
|
||||
sum_chars_size = 0;
|
||||
|
||||
size_t prev_cur_string_offset_local = prev_cur_string_offset;
|
||||
for (size_t k = 0; k < value_size; ++k)
|
||||
{
|
||||
/// Размер одной строки.
|
||||
size_t chars_size = cur_string_offsets[k + prev_cur_offset] - prev_cur_string_offset_local;
|
||||
|
||||
current_res_string_offset += chars_size;
|
||||
res_string_offsets.push_back(current_res_string_offset);
|
||||
|
||||
/// Копирование символов одной строки.
|
||||
res_chars.resize(res_chars.size() + chars_size);
|
||||
memcpy(&res_chars[res_chars.size() - chars_size], &cur_chars[prev_cur_string_offset_local], chars_size);
|
||||
|
||||
sum_chars_size += chars_size;
|
||||
prev_cur_string_offset_local += chars_size;
|
||||
}
|
||||
}
|
||||
|
||||
prev_replicate_offset = replicate_offsets[i];
|
||||
prev_cur_offset = cur_offsets[i];
|
||||
prev_cur_string_offset += sum_chars_size;
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
/** Неконстантные массивы константных значений - довольно редкое явление.
|
||||
* Большинство функций не умеет с ними работать, и не создаёт такие столбцы в качестве результата.
|
||||
* Исключение - функция replicate (см. FunctionsMiscellaneous.h), которая имеет служебное значение для реализации лямбда-функций.
|
||||
* Только ради неё сделана реализация метода replicate для ColumnArray(ColumnConst).
|
||||
*/
|
||||
ColumnPtr replicateConst(const Offsets_t & replicate_offsets) const;
|
||||
};
|
||||
|
||||
|
||||
|
@ -69,7 +69,7 @@ struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
typedef DataTypeDate::FieldType FromFieldType;
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
|
||||
if (const ColumnVector<FromFieldType> * col_from = typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
@ -108,7 +108,7 @@ struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
|
||||
if (const ColumnVector<FromFieldType> * col_from = typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
@ -149,7 +149,7 @@ struct ConvertImpl32Or64ToDate
|
||||
typedef DataTypeDate::FieldType ToFieldType;
|
||||
|
||||
template <typename To, typename From>
|
||||
static To convert(const From & from, const DateLUT & date_lut)
|
||||
static To convert(const From & from, const DateLUTImpl & date_lut)
|
||||
{
|
||||
return from < 0xFFFF
|
||||
? from
|
||||
@ -158,7 +158,7 @@ struct ConvertImpl32Or64ToDate
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
|
||||
if (const ColumnVector<FromFieldType> * col_from
|
||||
= typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
@ -242,6 +242,224 @@ struct ConvertImpl<FromDataType, DataTypeString, Name>
|
||||
}
|
||||
};
|
||||
|
||||
namespace details { namespace {
|
||||
|
||||
/** Пусть source_timestamp представляет дату и время в исходном часовом поясе соответствующем
|
||||
* объекту from_date_lut. Эта функция возвращает timestamp представлящий те же дату и время
|
||||
* в часовом поясе соответствующем объекту to_date_lut.
|
||||
*/
|
||||
time_t convertTimestamp(time_t source_timestamp, DateLUTImpl & from_date_lut, DateLUTImpl & to_date_lut)
|
||||
{
|
||||
if (&from_date_lut == &to_date_lut)
|
||||
return source_timestamp;
|
||||
else
|
||||
{
|
||||
const auto & values = from_date_lut.getValues(source_timestamp);
|
||||
return to_date_lut.makeDateTime(values.year, values.month, values.day_of_month,
|
||||
from_date_lut.toHourInaccurate(source_timestamp),
|
||||
from_date_lut.toMinuteInaccurate(source_timestamp),
|
||||
from_date_lut.toSecondInaccurate(source_timestamp));
|
||||
}
|
||||
}
|
||||
|
||||
/** Функции для преобразования даты + времени в строку.
|
||||
*/
|
||||
struct DateTimeToStringConverter
|
||||
{
|
||||
using FromFieldType = typename DataTypeDateTime::FieldType;
|
||||
|
||||
static void vector_vector(const PODArray<FromFieldType> & vec_from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, ColumnString & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
|
||||
ColumnString::Chars_t & data_to = vec_to.getChars();
|
||||
ColumnString::Offsets_t & offsets_to = vec_to.getOffsets();
|
||||
size_t size = vec_from.size();
|
||||
data_to.resize(size * 2);
|
||||
offsets_to.resize(size);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars_t> write_buffer(data_to);
|
||||
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
|
||||
auto ti = convertTimestamp(vec_from[i], remote_date_lut, local_date_lut);
|
||||
formatImpl<DataTypeDateTime>(ti, write_buffer);
|
||||
writeChar(0, write_buffer);
|
||||
offsets_to[i] = write_buffer.count();
|
||||
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
data_to.resize(write_buffer.count());
|
||||
}
|
||||
|
||||
static void vector_constant(const PODArray<FromFieldType> & vec_from, const std::string & data,
|
||||
ColumnString & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
auto & remote_date_lut = DateLUT::instance(data);
|
||||
|
||||
ColumnString::Chars_t & data_to = vec_to.getChars();
|
||||
ColumnString::Offsets_t & offsets_to = vec_to.getOffsets();
|
||||
size_t size = vec_from.size();
|
||||
data_to.resize(size * 2);
|
||||
offsets_to.resize(size);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars_t> write_buffer(data_to);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
auto ti = convertTimestamp(vec_from[i], remote_date_lut, local_date_lut);
|
||||
formatImpl<DataTypeDateTime>(ti, write_buffer);
|
||||
writeChar(0, write_buffer);
|
||||
offsets_to[i] = write_buffer.count();
|
||||
}
|
||||
data_to.resize(write_buffer.count());
|
||||
}
|
||||
|
||||
static void constant_vector(FromFieldType from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets,
|
||||
ColumnString & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
|
||||
ColumnString::Chars_t & data_to = vec_to.getChars();
|
||||
ColumnString::Offsets_t & offsets_to = vec_to.getOffsets();
|
||||
size_t size = offsets.size();
|
||||
data_to.resize(size * 2);
|
||||
offsets_to.resize(size);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars_t> write_buffer(data_to);
|
||||
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
|
||||
auto ti = convertTimestamp(from, remote_date_lut, local_date_lut);
|
||||
formatImpl<DataTypeDateTime>(ti, write_buffer);
|
||||
writeChar(0, write_buffer);
|
||||
offsets_to[i] = write_buffer.count();
|
||||
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
data_to.resize(write_buffer.count());
|
||||
}
|
||||
|
||||
static void constant_constant(FromFieldType from, const std::string & data, std::string & to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
auto & remote_date_lut = DateLUT::instance(data);
|
||||
|
||||
std::vector<char> buf;
|
||||
WriteBufferFromVector<std::vector<char> > write_buffer(buf);
|
||||
auto ti = convertTimestamp(from, remote_date_lut, local_date_lut);
|
||||
formatImpl<DataTypeDateTime>(ti, write_buffer);
|
||||
to = std::string(&buf[0], write_buffer.count());
|
||||
}
|
||||
};
|
||||
|
||||
}}
|
||||
|
||||
template<typename Name>
|
||||
struct ConvertImpl<DataTypeDateTime, DataTypeString, Name>
|
||||
{
|
||||
using Op = details::DateTimeToStringConverter;
|
||||
using FromFieldType = Op::FromFieldType;
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
||||
const auto * sources = typeid_cast<const ColumnVector<FromFieldType> *>(&*source_col);
|
||||
const auto * const_source = typeid_cast<const ColumnConst<FromFieldType> *>(&*source_col);
|
||||
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (sources)
|
||||
{
|
||||
ColumnString * col_to = new ColumnString;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
auto & vec_from = sources->getData();
|
||||
auto & vec_to = *col_to;
|
||||
|
||||
Op::vector_constant(vec_from, "", vec_to);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
std::string res;
|
||||
Op::constant_constant(const_source->getData(), "", res);
|
||||
block.getByPosition(result).column = new ColumnConstString(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column;
|
||||
const auto * time_zones = typeid_cast<const ColumnString *>(&*time_zone_col);
|
||||
const auto * const_time_zone = typeid_cast<const ColumnConstString *>(&*time_zone_col);
|
||||
|
||||
if (sources)
|
||||
{
|
||||
ColumnString * col_to = new ColumnString;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
auto & vec_from = sources->getData();
|
||||
auto & vec_to = *col_to;
|
||||
|
||||
if (time_zones)
|
||||
Op::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||
else if (const_time_zone)
|
||||
Op::vector_constant(vec_from, const_time_zone->getData(), vec_to);
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
if (time_zones)
|
||||
{
|
||||
ColumnString * col_to = new ColumnString;
|
||||
block.getByPosition(result).column = col_to;
|
||||
auto & vec_to = *col_to;
|
||||
|
||||
Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||
}
|
||||
else if (const_time_zone)
|
||||
{
|
||||
std::string res;
|
||||
Op::constant_constant(const_source->getData(), const_time_zone->getData(), res);
|
||||
block.getByPosition(result).column = new ColumnConstString(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Internal error.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
};
|
||||
|
||||
/** Преобразование строк в числа, даты, даты-с-временем: через парсинг.
|
||||
*/
|
||||
@ -304,6 +522,204 @@ struct ConvertImpl<DataTypeString, ToDataType, Name>
|
||||
}
|
||||
};
|
||||
|
||||
namespace details { namespace {
|
||||
|
||||
/** Функции для преобразования строк в timestamp.
|
||||
*/
|
||||
struct StringToTimestampConverter
|
||||
{
|
||||
using ToFieldType = typename DataTypeInt32::FieldType;
|
||||
|
||||
static void vector_vector(const ColumnString::Chars_t & vec_from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, PODArray<ToFieldType> & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
ReadBuffer read_buffer(const_cast<char *>(reinterpret_cast<const char *>(&vec_from[0])), vec_from.size(), 0);
|
||||
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
|
||||
char zero = 0;
|
||||
for (size_t i = 0; i < vec_to.size(); ++i)
|
||||
{
|
||||
DataTypeDateTime::FieldType x = 0;
|
||||
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
|
||||
auto ti = convertTimestamp(x, local_date_lut, remote_date_lut);
|
||||
|
||||
vec_to[i] = ti;
|
||||
readChar(zero, read_buffer);
|
||||
if (zero != 0)
|
||||
throw Exception("Cannot parse from string.", ErrorCodes::CANNOT_PARSE_NUMBER);
|
||||
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
}
|
||||
|
||||
static void vector_constant(const ColumnString::Chars_t & vec_from, const std::string & data,
|
||||
PODArray<ToFieldType> & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
auto & remote_date_lut = DateLUT::instance(data);
|
||||
ReadBuffer read_buffer(const_cast<char *>(reinterpret_cast<const char *>(&vec_from[0])), vec_from.size(), 0);
|
||||
|
||||
char zero = 0;
|
||||
for (size_t i = 0; i < vec_to.size(); ++i)
|
||||
{
|
||||
DataTypeDateTime::FieldType x = 0;
|
||||
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||
|
||||
auto ti = convertTimestamp(x, local_date_lut, remote_date_lut);
|
||||
|
||||
vec_to[i] = ti;
|
||||
readChar(zero, read_buffer);
|
||||
if (zero != 0)
|
||||
throw Exception("Cannot parse from string.", ErrorCodes::CANNOT_PARSE_NUMBER);
|
||||
}
|
||||
}
|
||||
|
||||
static void constant_vector(const std::string & from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, PODArray<ToFieldType> & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
|
||||
ReadBufferFromString read_buffer(from);
|
||||
DataTypeDateTime::FieldType x = 0;
|
||||
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
|
||||
auto ti = convertTimestamp(x, local_date_lut, remote_date_lut);
|
||||
|
||||
vec_to[i] = ti;
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
}
|
||||
|
||||
static void constant_constant(const std::string & from, const std::string & data, ToFieldType & to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
auto & remote_date_lut = DateLUT::instance(data);
|
||||
|
||||
ReadBufferFromString read_buffer(from);
|
||||
DataTypeDateTime::FieldType x = 0;
|
||||
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||
|
||||
to = convertTimestamp(x, local_date_lut, remote_date_lut);
|
||||
}
|
||||
};
|
||||
|
||||
}}
|
||||
|
||||
struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; };
|
||||
|
||||
template<>
|
||||
struct ConvertImpl<DataTypeString, DataTypeInt32, NameToUnixTimestamp>
|
||||
{
|
||||
using Op = details::StringToTimestampConverter;
|
||||
using ToFieldType = Op::ToFieldType;
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
||||
const auto * sources = typeid_cast<const ColumnString *>(&*source_col);
|
||||
const auto * const_source = typeid_cast<const ColumnConstString *>(&*source_col);
|
||||
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (sources)
|
||||
{
|
||||
auto * col_to = new ColumnVector<ToFieldType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
auto & vec_from = sources->getChars();
|
||||
auto & vec_to = col_to->getData();
|
||||
size_t size = sources->size();
|
||||
vec_to.resize(size);
|
||||
|
||||
Op::vector_constant(vec_from, "", vec_to);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
ToFieldType res;
|
||||
Op::constant_constant(const_source->getData(), "", res);
|
||||
block.getByPosition(result).column = new ColumnConst<ToFieldType>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + NameToUnixTimestamp::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column;
|
||||
const auto * time_zones = typeid_cast<const ColumnString *>(&*time_zone_col);
|
||||
const auto * const_time_zone = typeid_cast<const ColumnConstString *>(&*time_zone_col);
|
||||
|
||||
if (sources)
|
||||
{
|
||||
auto * col_to = new ColumnVector<ToFieldType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
auto & vec_from = sources->getChars();
|
||||
auto & vec_to = col_to->getData();
|
||||
size_t size = sources->size();
|
||||
vec_to.resize(size);
|
||||
|
||||
if (time_zones)
|
||||
Op::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||
else if (const_time_zone)
|
||||
Op::vector_constant(vec_from, const_time_zone->getData(), vec_to);
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + NameToUnixTimestamp::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
if (time_zones)
|
||||
{
|
||||
auto * col_to = new ColumnVector<ToFieldType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
auto & vec_to = col_to->getData();
|
||||
vec_to.resize(time_zones->getOffsets().size());
|
||||
|
||||
Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||
}
|
||||
else if (const_time_zone)
|
||||
{
|
||||
ToFieldType res;
|
||||
Op::constant_constant(const_source->getData(), const_time_zone->getData(), res);
|
||||
block.getByPosition(result).column = new ColumnConst<ToFieldType>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + NameToUnixTimestamp::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + NameToUnixTimestamp::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Internal error.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Если типы совпадают - просто скопируем ссылку на столбец.
|
||||
*/
|
||||
template <typename Name>
|
||||
@ -437,12 +853,7 @@ public:
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return new ToDataType;
|
||||
return getReturnTypeImpl(arguments);
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
@ -468,6 +879,73 @@ public:
|
||||
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
private:
|
||||
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
||||
typename std::enable_if<!(std::is_same<ToDataType2, DataTypeString>::value || std::is_same<Name2, NameToUnixTimestamp>::value), void>::type * = nullptr) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return new ToDataType;
|
||||
}
|
||||
|
||||
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
||||
typename std::enable_if<std::is_same<ToDataType2, DataTypeString>::value>::type * = nullptr) const
|
||||
{
|
||||
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 (typeid_cast<const DataTypeDateTime *>(&*arguments[0]) == nullptr)
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
else if ((arguments.size()) == 2 && typeid_cast<const DataTypeString *>(&*arguments[1]) == nullptr)
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
}
|
||||
|
||||
return new ToDataType2;
|
||||
}
|
||||
|
||||
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
||||
typename std::enable_if<std::is_same<Name2, NameToUnixTimestamp>::value, void>::type * = nullptr) const
|
||||
{
|
||||
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 (typeid_cast<const DataTypeString *>(&*arguments[0]) == nullptr)
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
else if ((arguments.size()) == 2 && typeid_cast<const DataTypeString *>(&*arguments[1]) == nullptr)
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
}
|
||||
|
||||
return new ToDataType2;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@ -628,6 +1106,6 @@ typedef FunctionConvert<DataTypeFloat64, NameToFloat64> FunctionToFloat64;
|
||||
typedef FunctionConvert<DataTypeDate, NameToDate> FunctionToDate;
|
||||
typedef FunctionConvert<DataTypeDateTime, NameToDateTime> FunctionToDateTime;
|
||||
typedef FunctionConvert<DataTypeString, NameToString> FunctionToString;
|
||||
|
||||
typedef FunctionConvert<DataTypeInt32, NameToUnixTimestamp> FunctionToUnixTimestamp;
|
||||
|
||||
}
|
||||
|
@ -4,12 +4,15 @@
|
||||
#include <DB/DataTypes/DataTypeDate.h>
|
||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||
#include <DB/DataTypes/DataTypeArray.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
#include <DB/Columns/ColumnFixedString.h>
|
||||
|
||||
#include <DB/Functions/IFunction.h>
|
||||
|
||||
#include <type_traits>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -48,32 +51,32 @@ namespace DB
|
||||
|
||||
struct ToYearImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toYear(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toYear(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToMonthImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toMonth(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toMonth(DayNum_t(d)); }
|
||||
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toMonth(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toMonth(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToDayOfMonthImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toDayOfMonth(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toDayOfMonth(DayNum_t(d)); }
|
||||
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfMonth(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfMonth(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToDayOfWeekImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toDayOfWeek(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toDayOfWeek(DayNum_t(d)); }
|
||||
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfWeek(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfWeek(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToHourImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toHourInaccurate(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUT & date_lut)
|
||||
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toHourInaccurate(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -81,8 +84,8 @@ struct ToHourImpl
|
||||
|
||||
struct ToMinuteImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toMinuteInaccurate(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUT & date_lut)
|
||||
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toMinuteInaccurate(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -90,8 +93,8 @@ struct ToMinuteImpl
|
||||
|
||||
struct ToSecondImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toSecondInaccurate(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUT & date_lut)
|
||||
static inline UInt8 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toSecondInaccurate(t); }
|
||||
static inline UInt8 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toSecond", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -99,34 +102,49 @@ struct ToSecondImpl
|
||||
|
||||
struct ToMondayImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfWeek(date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfWeek(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfWeek(remote_date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfWeek(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToStartOfMonthImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfMonth(date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfMonth(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfMonth(remote_date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfMonth(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToStartOfQuarterImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfQuarter(date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfQuarter(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfQuarter(remote_date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfQuarter(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToStartOfYearImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfYear(date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfYear(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfYear(remote_date_lut.toDayNum(t)); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfYear(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
|
||||
struct ToTimeImpl
|
||||
{
|
||||
/// При переводе во время, дату будем приравнивать к 1970-01-02.
|
||||
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toTimeInaccurate(t) + 86400; }
|
||||
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
||||
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
{
|
||||
time_t remote_t = remote_date_lut.toTimeInaccurate(t) + 86400;
|
||||
|
||||
if (&remote_date_lut == &local_date_lut)
|
||||
return remote_t;
|
||||
else
|
||||
{
|
||||
const auto & values = remote_date_lut.getValues(remote_t);
|
||||
return local_date_lut.makeDateTime(values.year, values.month, values.day_of_month,
|
||||
remote_date_lut.toHourInaccurate(remote_t),
|
||||
remote_date_lut.toMinuteInaccurate(remote_t),
|
||||
remote_date_lut.toSecondInaccurate(remote_t));
|
||||
}
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -134,8 +152,8 @@ struct ToTimeImpl
|
||||
|
||||
struct ToStartOfMinuteImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toStartOfMinuteInaccurate(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
||||
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfMinuteInaccurate(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toStartOfMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -143,8 +161,8 @@ struct ToStartOfMinuteImpl
|
||||
|
||||
struct ToStartOfFiveMinuteImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toStartOfFiveMinuteInaccurate(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
||||
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfFiveMinuteInaccurate(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toStartOfFiveMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -152,8 +170,8 @@ struct ToStartOfFiveMinuteImpl
|
||||
|
||||
struct ToStartOfHourImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toStartOfHourInaccurate(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
||||
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfHourInaccurate(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toStartOfHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -161,33 +179,33 @@ struct ToStartOfHourImpl
|
||||
|
||||
struct ToRelativeYearNumImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toYear(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toYear(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToRelativeMonthNumImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeMonthNum(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toRelativeMonthNum(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMonthNum(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMonthNum(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToRelativeWeekNumImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeWeekNum(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toRelativeWeekNum(DayNum_t(d)); }
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeWeekNum(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeWeekNum(DayNum_t(d)); }
|
||||
};
|
||||
|
||||
struct ToRelativeDayNumImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toDayNum(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return static_cast<DayNum_t>(d); }
|
||||
static inline UInt16 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toDayNum(t); }
|
||||
static inline UInt16 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return static_cast<DayNum_t>(d); }
|
||||
};
|
||||
|
||||
|
||||
struct ToRelativeHourNumImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeHourNum(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
||||
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeHourNum(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toRelativeHourNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -195,8 +213,8 @@ struct ToRelativeHourNumImpl
|
||||
|
||||
struct ToRelativeMinuteNumImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeMinuteNum(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
||||
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMinuteNum(t); }
|
||||
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toRelativeMinuteNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -204,46 +222,160 @@ struct ToRelativeMinuteNumImpl
|
||||
|
||||
struct ToRelativeSecondNumImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return t; }
|
||||
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
||||
static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { return t; }
|
||||
static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toRelativeSecondNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
};
|
||||
|
||||
template<typename FromType, typename ToType, typename Transform>
|
||||
struct Transformer
|
||||
{
|
||||
static void vector_vector(const PODArray<FromType> & vec_from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, PODArray<ToType> & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < vec_from.size(); ++i)
|
||||
{
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
vec_to[i] = Transform::execute(vec_from[i], remote_date_lut, local_date_lut);
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
}
|
||||
|
||||
static void vector_constant(const PODArray<FromType> & vec_from, const std::string & data,
|
||||
PODArray<ToType> & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
auto & remote_date_lut = DateLUT::instance(data);
|
||||
for (size_t i = 0; i < vec_from.size(); ++i)
|
||||
vec_to[i] = Transform::execute(vec_from[i], remote_date_lut, local_date_lut);
|
||||
}
|
||||
|
||||
static void constant_vector(const FromType & from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, PODArray<ToType> & vec_to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
vec_to[i] = Transform::execute(from, remote_date_lut, local_date_lut);
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
}
|
||||
|
||||
static void constant_constant(const FromType & from, const std::string & data, ToType & to)
|
||||
{
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
auto & remote_date_lut = DateLUT::instance(data);
|
||||
to = Transform::execute(from, remote_date_lut, local_date_lut);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType, typename Transform, typename Name>
|
||||
struct DateTimeTransformImpl
|
||||
{
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
using Op = Transformer<FromType, ToType, Transform>;
|
||||
|
||||
if (const ColumnVector<FromType> * col_from = typeid_cast<const ColumnVector<FromType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
||||
const auto * sources = typeid_cast<const ColumnVector<FromType> *>(&*source_col);
|
||||
const auto * const_source = typeid_cast<const ColumnConst<FromType> *>(&*source_col);
|
||||
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
ColumnVector<ToType> * col_to = new ColumnVector<ToType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
if (sources)
|
||||
{
|
||||
auto * col_to = new ColumnVector<ToType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<FromType>::Container_t & vec_from = col_from->getData();
|
||||
typename ColumnVector<ToType>::Container_t & vec_to = col_to->getData();
|
||||
size_t size = vec_from.size();
|
||||
vec_to.resize(size);
|
||||
auto & vec_from = sources->getData();
|
||||
auto & vec_to = col_to->getData();
|
||||
size_t size = vec_from.size();
|
||||
vec_to.resize(size);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::execute(vec_from[i], date_lut);
|
||||
Op::vector_constant(vec_from, "", vec_to);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
ToType res;
|
||||
Op::constant_constant(const_source->getData(), "", res);
|
||||
block.getByPosition(result).column = new ColumnConst<ToType>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
else if (const ColumnConst<FromType> * col_from = typeid_cast<const ColumnConst<FromType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
block.getByPosition(result).column = new ColumnConst<ToType>(col_from->size(), Transform::execute(col_from->getData(), date_lut));
|
||||
const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column;
|
||||
const auto * time_zones = typeid_cast<const ColumnString *>(&*time_zone_col);
|
||||
const auto * const_time_zone = typeid_cast<const ColumnConstString *>(&*time_zone_col);
|
||||
|
||||
if (sources)
|
||||
{
|
||||
auto * col_to = new ColumnVector<ToType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
auto & vec_from = sources->getData();
|
||||
auto & vec_to = col_to->getData();
|
||||
vec_to.resize(vec_from.size());
|
||||
|
||||
if (time_zones)
|
||||
Op::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||
else if (const_time_zone)
|
||||
Op::vector_constant(vec_from, const_time_zone->getData(), vec_to);
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
if (time_zones)
|
||||
{
|
||||
auto * col_to = new ColumnVector<ToType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
auto & vec_to = col_to->getData();
|
||||
vec_to.resize(time_zones->getOffsets().size());
|
||||
|
||||
Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||
}
|
||||
else if (const_time_zone)
|
||||
{
|
||||
ToType res;
|
||||
Op::constant_constant(const_source->getData(), const_time_zone->getData(), res);
|
||||
block.getByPosition(result).column = new ColumnConst<ToType>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception("Internal error.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename ToDataType, typename Transform, typename Name>
|
||||
class FunctionDateOrDateTimeToSomething : public IFunction
|
||||
{
|
||||
@ -257,15 +389,9 @@ public:
|
||||
return name;
|
||||
}
|
||||
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return new ToDataType;
|
||||
return getReturnTypeImpl(arguments);
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
@ -281,6 +407,54 @@ public:
|
||||
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
private:
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
|
||||
template<typename ToDataType2 = ToDataType, typename Transform2 = Transform>
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
||||
typename std::enable_if<
|
||||
!(std::is_same<ToDataType2, DataTypeDate>::value
|
||||
|| (std::is_same<ToDataType2, DataTypeDateTime>::value && std::is_same<Transform2, ToTimeImpl>::value))
|
||||
, void>::type * = nullptr) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return new ToDataType;
|
||||
}
|
||||
|
||||
template<typename ToDataType2 = ToDataType, typename Transform2 = Transform>
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
||||
typename std::enable_if<
|
||||
std::is_same<ToDataType2, DataTypeDate>::value
|
||||
|| (std::is_same<ToDataType2, DataTypeDateTime>::value && std::is_same<Transform2, ToTimeImpl>::value)
|
||||
, void>::type * = nullptr) const
|
||||
{
|
||||
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 (typeid_cast<const DataTypeDateTime *>(&*arguments[0]) == nullptr)
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
else if ((arguments.size()) == 2 && typeid_cast<const DataTypeString *>(&*arguments[1]) == nullptr)
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
}
|
||||
|
||||
return new ToDataType;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
@ -332,7 +332,7 @@ inline void writeDateText(DayNum_t date, WriteBuffer & buf)
|
||||
return;
|
||||
}
|
||||
|
||||
const DateLUT::Values & values = DateLUT::instance().getValues(date);
|
||||
const auto & values = DateLUT::instance().getValues(date);
|
||||
|
||||
s[0] += values.year / 1000;
|
||||
s[1] += (values.year / 100) % 10;
|
||||
@ -364,7 +364,8 @@ inline void writeDateText(mysqlxx::Date date, WriteBuffer & buf)
|
||||
|
||||
|
||||
/// в формате YYYY-MM-DD HH:MM:SS, согласно текущему часовому поясу
|
||||
inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, char date_delimeter = '-', char time_delimeter = ':')
|
||||
template <char date_delimeter = '-', char time_delimeter = ':'>
|
||||
inline void writeDateTimeText(time_t datetime, WriteBuffer & buf)
|
||||
{
|
||||
char s[19] = {'0', '0', '0', '0', date_delimeter, '0', '0', date_delimeter, '0', '0', ' ', '0', '0', time_delimeter, '0', '0', time_delimeter, '0', '0'};
|
||||
|
||||
@ -374,8 +375,8 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, char date_deli
|
||||
return;
|
||||
}
|
||||
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
const DateLUT::Values & values = date_lut.getValues(datetime);
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & values = date_lut.getValues(datetime);
|
||||
|
||||
s[0] += values.year / 1000;
|
||||
s[1] += (values.year / 100) % 10;
|
||||
@ -400,7 +401,8 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, char date_deli
|
||||
buf.write(s, 19);
|
||||
}
|
||||
|
||||
inline void writeDateTimeText(mysqlxx::DateTime datetime, WriteBuffer & buf, char date_delimeter = '-', char time_delimeter = ':')
|
||||
template <char date_delimeter = '-', char time_delimeter = ':'>
|
||||
inline void writeDateTimeText(mysqlxx::DateTime datetime, WriteBuffer & buf)
|
||||
{
|
||||
char s[19] = {'0', '0', '0', '0', date_delimeter, '0', '0', date_delimeter, '0', '0', ' ', '0', '0', time_delimeter, '0', '0', time_delimeter, '0', '0'};
|
||||
|
||||
|
334
dbms/src/Columns/ColumnArray.cpp
Normal file
334
dbms/src/Columns/ColumnArray.cpp
Normal file
@ -0,0 +1,334 @@
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
ColumnPtr ColumnArray::cut(size_t start, size_t length) const
|
||||
{
|
||||
if (length == 0)
|
||||
return new ColumnArray(data);
|
||||
|
||||
if (start + length > getOffsets().size())
|
||||
throw Exception("Parameter out of bound in IColumnArray::cut() method.",
|
||||
ErrorCodes::PARAMETER_OUT_OF_BOUND);
|
||||
|
||||
size_t nested_offset = offsetAt(start);
|
||||
size_t nested_length = getOffsets()[start + length - 1] - nested_offset;
|
||||
|
||||
ColumnArray * res_ = new ColumnArray(data);
|
||||
ColumnPtr res = res_;
|
||||
|
||||
res_->data = data->cut(nested_offset, nested_length);
|
||||
Offsets_t & res_offsets = res_->getOffsets();
|
||||
|
||||
if (start == 0)
|
||||
{
|
||||
res_offsets.assign(getOffsets().begin(), getOffsets().begin() + length);
|
||||
}
|
||||
else
|
||||
{
|
||||
res_offsets.resize(length);
|
||||
|
||||
for (size_t i = 0; i < length; ++i)
|
||||
res_offsets[i] = getOffsets()[start + i] - nested_offset;
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
ColumnPtr ColumnArray::filter(const Filter & filt) const
|
||||
{
|
||||
size_t size = getOffsets().size();
|
||||
if (size != filt.size())
|
||||
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
if (size == 0)
|
||||
return new ColumnArray(data);
|
||||
|
||||
/// Не слишком оптимально. Можно сделать специализацию для массивов известных типов.
|
||||
Filter nested_filt(getOffsets().back());
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (filt[i])
|
||||
memset(&nested_filt[offsetAt(i)], 1, sizeAt(i));
|
||||
else
|
||||
memset(&nested_filt[offsetAt(i)], 0, sizeAt(i));
|
||||
}
|
||||
|
||||
ColumnArray * res_ = new ColumnArray(data);
|
||||
ColumnPtr res = res_;
|
||||
res_->data = data->filter(nested_filt);
|
||||
|
||||
Offsets_t & res_offsets = res_->getOffsets();
|
||||
res_offsets.reserve(size);
|
||||
|
||||
size_t current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (filt[i])
|
||||
{
|
||||
current_offset += sizeAt(i);
|
||||
res_offsets.push_back(current_offset);
|
||||
}
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
ColumnPtr ColumnArray::permute(const Permutation & perm, size_t limit) const
|
||||
{
|
||||
size_t size = getOffsets().size();
|
||||
|
||||
if (limit == 0)
|
||||
limit = size;
|
||||
else
|
||||
limit = std::min(size, limit);
|
||||
|
||||
if (perm.size() < limit)
|
||||
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
if (limit == 0)
|
||||
return new ColumnArray(data);
|
||||
|
||||
Permutation nested_perm(getOffsets().back());
|
||||
|
||||
ColumnArray * res_ = new ColumnArray(data->cloneEmpty());
|
||||
ColumnPtr res = res_;
|
||||
|
||||
Offsets_t & res_offsets = res_->getOffsets();
|
||||
res_offsets.resize(limit);
|
||||
size_t current_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < limit; ++i)
|
||||
{
|
||||
for (size_t j = 0; j < sizeAt(perm[i]); ++j)
|
||||
nested_perm[current_offset + j] = offsetAt(perm[i]) + j;
|
||||
current_offset += sizeAt(perm[i]);
|
||||
res_offsets[i] = current_offset;
|
||||
}
|
||||
|
||||
if (current_offset != 0)
|
||||
res_->data = data->permute(nested_perm, current_offset);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void ColumnArray::getPermutation(bool reverse, size_t limit, Permutation & res) const
|
||||
{
|
||||
size_t s = size();
|
||||
if (limit >= s)
|
||||
limit = 0;
|
||||
|
||||
res.resize(s);
|
||||
for (size_t i = 0; i < s; ++i)
|
||||
res[i] = i;
|
||||
|
||||
if (limit)
|
||||
{
|
||||
if (reverse)
|
||||
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<false>(*this));
|
||||
else
|
||||
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<true>(*this));
|
||||
}
|
||||
else
|
||||
{
|
||||
if (reverse)
|
||||
std::sort(res.begin(), res.end(), less<false>(*this));
|
||||
else
|
||||
std::sort(res.begin(), res.end(), less<true>(*this));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
ColumnPtr ColumnArray::replicate(const Offsets_t & replicate_offsets) const
|
||||
{
|
||||
/// Не получается реализовать в общем случае.
|
||||
|
||||
if (typeid_cast<const ColumnUInt8 *>(&*data)) return replicate<UInt8>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnUInt16 *>(&*data)) return replicate<UInt16>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnUInt32 *>(&*data)) return replicate<UInt32>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnUInt64 *>(&*data)) return replicate<UInt64>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnInt8 *>(&*data)) return replicate<Int8>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnInt16 *>(&*data)) return replicate<Int16>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnInt32 *>(&*data)) return replicate<Int32>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnInt64 *>(&*data)) return replicate<Int64>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnFloat32 *>(&*data)) return replicate<Float32>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnFloat64 *>(&*data)) return replicate<Float64>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnString *>(&*data)) return replicateString(replicate_offsets);
|
||||
if (dynamic_cast<const IColumnConst *>(&*data)) return replicateConst(replicate_offsets);
|
||||
|
||||
throw Exception("Replication of column " + getName() + " is not implemented.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
ColumnPtr ColumnArray::replicate(const Offsets_t & replicate_offsets) const
|
||||
{
|
||||
size_t col_size = size();
|
||||
if (col_size != replicate_offsets.size())
|
||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
ColumnPtr res = cloneEmpty();
|
||||
|
||||
if (0 == col_size)
|
||||
return res;
|
||||
|
||||
ColumnArray & res_ = typeid_cast<ColumnArray &>(*res);
|
||||
|
||||
const typename ColumnVector<T>::Container_t & cur_data = typeid_cast<const ColumnVector<T> &>(*data).getData();
|
||||
const Offsets_t & cur_offsets = getOffsets();
|
||||
|
||||
typename ColumnVector<T>::Container_t & res_data = typeid_cast<ColumnVector<T> &>(res_.getData()).getData();
|
||||
Offsets_t & res_offsets = res_.getOffsets();
|
||||
|
||||
res_data.reserve(data->size() / col_size * replicate_offsets.back());
|
||||
res_offsets.reserve(replicate_offsets.back());
|
||||
|
||||
Offset_t prev_replicate_offset = 0;
|
||||
Offset_t prev_data_offset = 0;
|
||||
Offset_t current_new_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < col_size; ++i)
|
||||
{
|
||||
size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset;
|
||||
size_t value_size = cur_offsets[i] - prev_data_offset;
|
||||
|
||||
for (size_t j = 0; j < size_to_replicate; ++j)
|
||||
{
|
||||
current_new_offset += value_size;
|
||||
res_offsets.push_back(current_new_offset);
|
||||
|
||||
res_data.resize(res_data.size() + value_size);
|
||||
memcpy(&res_data[res_data.size() - value_size], &cur_data[prev_data_offset], value_size * sizeof(T));
|
||||
}
|
||||
|
||||
prev_replicate_offset = replicate_offsets[i];
|
||||
prev_data_offset = cur_offsets[i];
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
ColumnPtr ColumnArray::replicateString(const Offsets_t & replicate_offsets) const
|
||||
{
|
||||
size_t col_size = size();
|
||||
if (col_size != replicate_offsets.size())
|
||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
ColumnPtr res = cloneEmpty();
|
||||
|
||||
if (0 == col_size)
|
||||
return res;
|
||||
|
||||
ColumnArray & res_ = typeid_cast<ColumnArray &>(*res);
|
||||
|
||||
const ColumnString & cur_string = typeid_cast<const ColumnString &>(*data);
|
||||
const ColumnString::Chars_t & cur_chars = cur_string.getChars();
|
||||
const Offsets_t & cur_string_offsets = cur_string.getOffsets();
|
||||
const Offsets_t & cur_offsets = getOffsets();
|
||||
|
||||
ColumnString::Chars_t & res_chars = typeid_cast<ColumnString &>(res_.getData()).getChars();
|
||||
Offsets_t & res_string_offsets = typeid_cast<ColumnString &>(res_.getData()).getOffsets();
|
||||
Offsets_t & res_offsets = res_.getOffsets();
|
||||
|
||||
res_chars.reserve(cur_chars.size() / col_size * replicate_offsets.back());
|
||||
res_string_offsets.reserve(cur_string_offsets.size() / col_size * replicate_offsets.back());
|
||||
res_offsets.reserve(replicate_offsets.back());
|
||||
|
||||
Offset_t prev_replicate_offset = 0;
|
||||
|
||||
Offset_t prev_cur_offset = 0;
|
||||
Offset_t prev_cur_string_offset = 0;
|
||||
|
||||
Offset_t current_res_offset = 0;
|
||||
Offset_t current_res_string_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < col_size; ++i)
|
||||
{
|
||||
/// Насколько размножить массив.
|
||||
size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset;
|
||||
/// Количество строк в массиве.
|
||||
size_t value_size = cur_offsets[i] - prev_cur_offset;
|
||||
|
||||
size_t sum_chars_size = 0;
|
||||
|
||||
for (size_t j = 0; j < size_to_replicate; ++j)
|
||||
{
|
||||
current_res_offset += value_size;
|
||||
res_offsets.push_back(current_res_offset);
|
||||
|
||||
sum_chars_size = 0;
|
||||
|
||||
size_t prev_cur_string_offset_local = prev_cur_string_offset;
|
||||
for (size_t k = 0; k < value_size; ++k)
|
||||
{
|
||||
/// Размер одной строки.
|
||||
size_t chars_size = cur_string_offsets[k + prev_cur_offset] - prev_cur_string_offset_local;
|
||||
|
||||
current_res_string_offset += chars_size;
|
||||
res_string_offsets.push_back(current_res_string_offset);
|
||||
|
||||
/// Копирование символов одной строки.
|
||||
res_chars.resize(res_chars.size() + chars_size);
|
||||
memcpy(&res_chars[res_chars.size() - chars_size], &cur_chars[prev_cur_string_offset_local], chars_size);
|
||||
|
||||
sum_chars_size += chars_size;
|
||||
prev_cur_string_offset_local += chars_size;
|
||||
}
|
||||
}
|
||||
|
||||
prev_replicate_offset = replicate_offsets[i];
|
||||
prev_cur_offset = cur_offsets[i];
|
||||
prev_cur_string_offset += sum_chars_size;
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
ColumnPtr ColumnArray::replicateConst(const Offsets_t & replicate_offsets) const
|
||||
{
|
||||
size_t col_size = size();
|
||||
if (col_size != replicate_offsets.size())
|
||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
if (0 == col_size)
|
||||
return cloneEmpty();
|
||||
|
||||
const Offsets_t & cur_offsets = getOffsets();
|
||||
|
||||
ColumnOffsets_t * res_column_offsets = new ColumnOffsets_t;
|
||||
ColumnPtr res_column_offsets_holder = res_column_offsets;
|
||||
Offsets_t & res_offsets = res_column_offsets->getData();
|
||||
res_offsets.reserve(replicate_offsets.back());
|
||||
|
||||
Offset_t prev_replicate_offset = 0;
|
||||
Offset_t prev_data_offset = 0;
|
||||
Offset_t current_new_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < col_size; ++i)
|
||||
{
|
||||
size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset;
|
||||
size_t value_size = cur_offsets[i] - prev_data_offset;
|
||||
|
||||
for (size_t j = 0; j < size_to_replicate; ++j)
|
||||
{
|
||||
current_new_offset += value_size;
|
||||
res_offsets.push_back(current_new_offset);
|
||||
}
|
||||
|
||||
prev_replicate_offset = replicate_offsets[i];
|
||||
prev_data_offset = cur_offsets[i];
|
||||
}
|
||||
|
||||
return new ColumnArray(getData().cloneResized(current_new_offset), res_column_offsets_holder);
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -20,6 +20,7 @@ void registerFunctionsConversion(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionToDateTime>();
|
||||
factory.registerFunction<FunctionToString>();
|
||||
factory.registerFunction<FunctionToFixedString>();
|
||||
factory.registerFunction<FunctionToUnixTimestamp>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -210,7 +210,7 @@ void QueryLog::flush()
|
||||
{
|
||||
LOG_TRACE(log, "Flushing query log");
|
||||
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
|
||||
Block block = createBlock();
|
||||
|
||||
|
@ -95,7 +95,7 @@ QueryParseResult QueryParser::parse(std::istream & s)
|
||||
if (result.date_first > result.date_last)
|
||||
throw Exception("First date is bigger than last date.", ErrorCodes::FIRST_DATE_IS_BIGGER_THAN_LAST_DATE);
|
||||
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
result.days = 1 + date_lut.toDayNum(result.date_last) - date_lut.toDayNum(result.date_first);
|
||||
|
||||
result.cut_date_last = false;
|
||||
|
@ -105,7 +105,7 @@ size_t ActiveDataPartSet::size() const
|
||||
|
||||
String ActiveDataPartSet::getPartName(DayNum_t left_date, DayNum_t right_date, UInt64 left_id, UInt64 right_id, UInt64 level)
|
||||
{
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
|
||||
/// Имя директории для куска иммет вид: YYYYMMDD_YYYYMMDD_N_N_L.
|
||||
String res;
|
||||
@ -153,7 +153,7 @@ void ActiveDataPartSet::parsePartName(const String & file_name, Part & part, con
|
||||
|
||||
const Poco::RegularExpression::MatchVec & matches = *matches_p;
|
||||
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
|
||||
part.left_date = date_lut.YYYYMMDDToDayNum(parse<UInt32>(file_name.substr(matches[1].offset, matches[1].length)));
|
||||
part.right_date = date_lut.YYYYMMDDToDayNum(parse<UInt32>(file_name.substr(matches[2].offset, matches[2].length)));
|
||||
|
@ -46,7 +46,7 @@ bool MergeTreeDataMerger::selectPartsToMerge(MergeTreeData::DataPartsVector & pa
|
||||
{
|
||||
MergeTreeData::DataParts data_parts = data.getDataParts();
|
||||
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
|
||||
size_t min_max = -1U;
|
||||
size_t min_min = -1U;
|
||||
|
@ -12,7 +12,7 @@ BlocksWithDateIntervals MergeTreeDataWriter::splitBlockIntoParts(const Block & b
|
||||
{
|
||||
data.check(block, true);
|
||||
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
|
||||
size_t rows = block.rows();
|
||||
size_t columns = block.columns();
|
||||
@ -77,7 +77,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa
|
||||
UInt16 min_date = block_with_dates.min_date;
|
||||
UInt16 max_date = block_with_dates.max_date;
|
||||
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
|
||||
size_t part_size = (block.rows() + data.index_granularity - 1) / data.index_granularity;
|
||||
|
||||
|
@ -2214,8 +2214,8 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params,
|
||||
static String getFakePartNameForDrop(const String & month_name, UInt64 left, UInt64 right)
|
||||
{
|
||||
/// Диапазон дат - весь месяц.
|
||||
DateLUT & lut = DateLUT::instance();
|
||||
time_t start_time = DateLUT::instance().YYYYMMDDToDate(parse<UInt32>(month_name + "01"));
|
||||
auto & lut = DateLUT::instance();
|
||||
time_t start_time = lut.YYYYMMDDToDate(parse<UInt32>(month_name + "01"));
|
||||
DayNum_t left_date = lut.toDayNum(start_time);
|
||||
DayNum_t right_date = DayNum_t(static_cast<size_t>(left_date) + lut.daysInMonth(start_time) - 1);
|
||||
|
||||
|
@ -1 +1 @@
|
||||
1000
|
||||
995
|
||||
|
File diff suppressed because one or more lines are too long
@ -1,43 +1,43 @@
|
||||
100
|
||||
2014-06-01 10
|
||||
2014-06-02 10
|
||||
2014-06-03 10
|
||||
2014-06-04 10
|
||||
2014-06-05 10
|
||||
2014-06-06 10
|
||||
2014-06-07 10
|
||||
2014-06-08 10
|
||||
2014-06-09 10
|
||||
2014-06-10 10
|
||||
155
|
||||
2014-06-01 10
|
||||
2014-06-02 10
|
||||
2014-06-03 10
|
||||
2014-06-04 10
|
||||
2014-06-05 10
|
||||
2014-06-06 15
|
||||
2014-06-07 15
|
||||
2014-06-08 15
|
||||
2014-06-09 15
|
||||
2014-06-10 15
|
||||
2014-06-11 10
|
||||
2014-06-12 10
|
||||
2014-06-13 10
|
||||
2014-06-14 10
|
||||
2014-06-15 10
|
||||
155
|
||||
2014-06-01 10
|
||||
2014-06-02 10
|
||||
2014-06-03 10
|
||||
2014-06-04 10
|
||||
2014-06-05 10
|
||||
2014-06-06 15
|
||||
2014-06-07 15
|
||||
2014-06-08 15
|
||||
2014-06-09 15
|
||||
2014-06-10 15
|
||||
2014-06-11 10
|
||||
2014-06-12 10
|
||||
2014-06-13 10
|
||||
2014-06-14 10
|
||||
2014-06-15 10
|
||||
99
|
||||
2014-06-01 9
|
||||
2014-06-02 9
|
||||
2014-06-03 9
|
||||
2014-06-04 9
|
||||
2014-06-05 9
|
||||
2014-06-06 9
|
||||
2014-06-07 9
|
||||
2014-06-08 9
|
||||
2014-06-09 9
|
||||
2014-06-10 9
|
||||
154
|
||||
2014-06-01 9
|
||||
2014-06-02 9
|
||||
2014-06-03 9
|
||||
2014-06-04 9
|
||||
2014-06-05 9
|
||||
2014-06-06 14
|
||||
2014-06-07 14
|
||||
2014-06-08 14
|
||||
2014-06-09 14
|
||||
2014-06-10 14
|
||||
2014-06-11 9
|
||||
2014-06-12 9
|
||||
2014-06-13 9
|
||||
2014-06-14 9
|
||||
2014-06-15 9
|
||||
154
|
||||
2014-06-01 9
|
||||
2014-06-02 9
|
||||
2014-06-03 9
|
||||
2014-06-04 9
|
||||
2014-06-05 9
|
||||
2014-06-06 14
|
||||
2014-06-07 14
|
||||
2014-06-08 14
|
||||
2014-06-09 14
|
||||
2014-06-10 14
|
||||
2014-06-11 9
|
||||
2014-06-12 9
|
||||
2014-06-13 9
|
||||
2014-06-14 9
|
||||
2014-06-15 9
|
||||
|
@ -1,43 +1,43 @@
|
||||
100
|
||||
2014-06-01 10
|
||||
2014-06-02 10
|
||||
2014-06-03 10
|
||||
2014-06-04 10
|
||||
2014-06-05 10
|
||||
2014-06-06 10
|
||||
2014-06-07 10
|
||||
2014-06-08 10
|
||||
2014-06-09 10
|
||||
2014-06-10 10
|
||||
155
|
||||
2014-06-01 10
|
||||
2014-06-02 10
|
||||
2014-06-03 10
|
||||
2014-06-04 10
|
||||
2014-06-05 10
|
||||
2014-06-06 15
|
||||
2014-06-07 15
|
||||
2014-06-08 15
|
||||
2014-06-09 15
|
||||
2014-06-10 15
|
||||
2014-06-11 10
|
||||
2014-06-12 10
|
||||
2014-06-13 10
|
||||
2014-06-14 10
|
||||
2014-06-15 10
|
||||
155
|
||||
2014-06-01 10
|
||||
2014-06-02 10
|
||||
2014-06-03 10
|
||||
2014-06-04 10
|
||||
2014-06-05 10
|
||||
2014-06-06 15
|
||||
2014-06-07 15
|
||||
2014-06-08 15
|
||||
2014-06-09 15
|
||||
2014-06-10 15
|
||||
2014-06-11 10
|
||||
2014-06-12 10
|
||||
2014-06-13 10
|
||||
2014-06-14 10
|
||||
2014-06-15 10
|
||||
99
|
||||
2014-06-01 9
|
||||
2014-06-02 9
|
||||
2014-06-03 9
|
||||
2014-06-04 9
|
||||
2014-06-05 9
|
||||
2014-06-06 9
|
||||
2014-06-07 9
|
||||
2014-06-08 9
|
||||
2014-06-09 9
|
||||
2014-06-10 9
|
||||
154
|
||||
2014-06-01 9
|
||||
2014-06-02 9
|
||||
2014-06-03 9
|
||||
2014-06-04 9
|
||||
2014-06-05 9
|
||||
2014-06-06 14
|
||||
2014-06-07 14
|
||||
2014-06-08 14
|
||||
2014-06-09 14
|
||||
2014-06-10 14
|
||||
2014-06-11 9
|
||||
2014-06-12 9
|
||||
2014-06-13 9
|
||||
2014-06-14 9
|
||||
2014-06-15 9
|
||||
154
|
||||
2014-06-01 9
|
||||
2014-06-02 9
|
||||
2014-06-03 9
|
||||
2014-06-04 9
|
||||
2014-06-05 9
|
||||
2014-06-06 14
|
||||
2014-06-07 14
|
||||
2014-06-08 14
|
||||
2014-06-09 14
|
||||
2014-06-10 14
|
||||
2014-06-11 9
|
||||
2014-06-12 9
|
||||
2014-06-13 9
|
||||
2014-06-14 9
|
||||
2014-06-15 9
|
||||
|
@ -1,2 +1,2 @@
|
||||
1 1
|
||||
1 1
|
||||
1 0
|
||||
1 0
|
||||
|
@ -1 +1 @@
|
||||
10 10 1
|
||||
10 10 0
|
||||
|
85
dbms/tests/queries/0_stateless/00189_time_zones.reference
Normal file
85
dbms/tests/queries/0_stateless/00189_time_zones.reference
Normal file
@ -0,0 +1,85 @@
|
||||
2014-12-29
|
||||
2014-12-22
|
||||
2014-12-22
|
||||
2014-12-29
|
||||
2014-12-22
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-07-01
|
||||
2014-07-01
|
||||
2014-07-01
|
||||
2014-10-01
|
||||
2014-07-01
|
||||
1970-01-02 12:00:00 1970-01-02 12:00:00
|
||||
1970-01-02 10:00:00 1970-01-02 11:00:00
|
||||
1970-01-02 09:00:00 1970-01-02 10:00:00
|
||||
1970-01-02 18:00:00 1970-01-02 18:00:00
|
||||
1970-01-02 01:30:00 1970-01-02 01:30:00
|
||||
2015-07-13 2015-07-01 2015-07-01 1970-01-02 19:30:00
|
||||
2014-12-22 2014-12-01 2014-10-01 1970-01-02 21:00:00
|
||||
2014-12-29 2015-01-01 2015-01-01 1970-01-02 12:00:00
|
||||
2014-09-29 2014-09-01 2014-07-01 1970-01-02 21:50:00
|
||||
2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00
|
||||
2014-09-29 2014-09-01 2014-10-01 1970-01-02 11:20:00
|
||||
2014-12-22 2014-12-01 2014-10-01 1970-01-02 13:30:00
|
||||
2014-12-29 2015-01-01 2015-01-01 1970-01-02 01:30:00
|
||||
2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00
|
||||
2015-07-13 2015-07-01 2015-07-01 1970-01-02 02:00:00
|
||||
2015-03-09 2015-03-01 2015-01-01 1970-01-02 19:30:00
|
||||
2015-03-09 2015-03-01 2015-01-01 1970-01-02 10:30:00
|
||||
2015-03-09 2015-03-01 2015-01-01 1970-01-02 13:30:00
|
||||
2015-03-09 2015-03-01 2015-01-01 1970-01-02 11:30:00
|
||||
2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00
|
||||
2015-07-15 13:30:00
|
||||
2015-07-15 12:30:00
|
||||
2015-07-15 11:30:00
|
||||
2015-07-15 19:30:00
|
||||
2015-07-15 02:30:00
|
||||
2015-07-15 19:30:00
|
||||
2014-12-28 21:00:00
|
||||
2015-01-01 12:00:00
|
||||
2014-09-30 21:50:00
|
||||
2015-03-15 02:30:00
|
||||
2015-07-15 19:30:00
|
||||
2015-07-15 11:30:00
|
||||
2015-07-15 13:30:00
|
||||
2015-07-15 12:30:00
|
||||
2015-07-15 02:30:00
|
||||
2014-09-30 20:50:00
|
||||
2014-12-28 21:00:00
|
||||
2015-01-01 09:00:00
|
||||
2015-03-15 10:30:00
|
||||
2015-07-15 11:30:00
|
||||
1426415400
|
||||
1426422600
|
||||
1426426200
|
||||
1426393800
|
||||
1426455000
|
||||
1426415400
|
||||
1426415400
|
||||
1426415400
|
||||
1426415400
|
||||
1426415400
|
||||
1436934600
|
||||
1419811200
|
||||
1420102800
|
||||
1412113800
|
||||
1426455000
|
||||
1426393800
|
||||
1426426200
|
||||
1426415400
|
||||
1426422600
|
||||
1426455000
|
||||
1412113800
|
||||
1419807600
|
||||
1420110000
|
||||
1426422600
|
||||
1436959800
|
89
dbms/tests/queries/0_stateless/00189_time_zones.sql
Normal file
89
dbms/tests/queries/0_stateless/00189_time_zones.sql
Normal file
@ -0,0 +1,89 @@
|
||||
|
||||
/* timestamp 1419800400 == 2014-12-29 00:00:00 (Europe/Moscow) */
|
||||
/* timestamp 1412106600 == 2014-09-30 23:50:00 (Europe/Moscow) */
|
||||
/* timestamp 1420102800 == 2015-01-01 12:00:00 (Europe/Moscow) */
|
||||
/* timestamp 1428310800 == 2015-04-06 12:00:00 (Europe/Moscow) */
|
||||
/* timestamp 1436956200 == 2015-07-15 13:30:00 (Europe/Moscow) */
|
||||
/* timestamp 1426415400 == 2015-03-15 13:30:00 (Europe/Moscow) */
|
||||
|
||||
/* toMonday */
|
||||
|
||||
SELECT toMonday(toDateTime(1419800400), 'Europe/Moscow');
|
||||
SELECT toMonday(toDateTime(1419800400), 'Europe/Paris');
|
||||
SELECT toMonday(toDateTime(1419800400), 'Europe/London');
|
||||
SELECT toMonday(toDateTime(1419800400), 'Asia/Tokyo');
|
||||
SELECT toMonday(toDateTime(1419800400), 'Pacific/Pitcairn');
|
||||
|
||||
/* toStartOfMonth */
|
||||
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Moscow');
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Paris');
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/London');
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Asia/Tokyo');
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Pacific/Pitcairn');
|
||||
|
||||
/* toStartOfQuarter */
|
||||
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Moscow');
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Paris');
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/London');
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Asia/Tokyo');
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Pacific/Pitcairn');
|
||||
|
||||
/* toStartOfYear */
|
||||
|
||||
SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toStartOfQuarter(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toStartOfQuarter(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toTime */
|
||||
|
||||
SELECT toTime(toDateTime(1420102800), 'Europe/Moscow'), toTime(toDateTime(1428310800), 'Europe/Moscow');
|
||||
SELECT toTime(toDateTime(1420102800), 'Europe/Paris'), toTime(toDateTime(1428310800), 'Europe/Paris');
|
||||
SELECT toTime(toDateTime(1420102800), 'Europe/London'), toTime(toDateTime(1428310800), 'Europe/London');
|
||||
SELECT toTime(toDateTime(1420102800), 'Asia/Tokyo'), toTime(toDateTime(1428310800), 'Asia/Tokyo');
|
||||
SELECT toTime(toDateTime(1420102800), 'Pacific/Pitcairn'), toTime(toDateTime(1428310800), 'Pacific/Pitcairn');
|
||||
|
||||
DROP TABLE IF EXISTS foo;
|
||||
CREATE TABLE foo(x Int32, y String) ENGINE=Memory;
|
||||
INSERT INTO foo(x, y) VALUES(1420102800, 'Europe/Moscow');
|
||||
INSERT INTO foo(x, y) VALUES(1412106600, 'Europe/Paris');
|
||||
INSERT INTO foo(x, y) VALUES(1419800400, 'Europe/London');
|
||||
INSERT INTO foo(x, y) VALUES(1436956200, 'Asia/Tokyo');
|
||||
INSERT INTO foo(x, y) VALUES(1426415400, 'Pacific/Pitcairn');
|
||||
|
||||
SELECT toMonday(toDateTime(x), y), toStartOfMonth(toDateTime(x), y), toStartOfQuarter(toDateTime(x), y), toTime(toDateTime(x), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toMonday(toDateTime(x), 'Europe/Paris'), toStartOfMonth(toDateTime(x), 'Europe/London'), toStartOfQuarter(toDateTime(x), 'Asia/Tokyo'), toTime(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY x ASC;
|
||||
SELECT toMonday(toDateTime(1426415400), y), toStartOfMonth(toDateTime(1426415400), y), toStartOfQuarter(toDateTime(1426415400), y), toTime(toDateTime(1426415400), y) FROM foo ORDER BY y ASC;
|
||||
|
||||
/* toString */
|
||||
|
||||
SELECT toString(toDateTime(1436956200), 'Europe/Moscow');
|
||||
SELECT toString(toDateTime(1436956200), 'Europe/Paris');
|
||||
SELECT toString(toDateTime(1436956200), 'Europe/London');
|
||||
SELECT toString(toDateTime(1436956200), 'Asia/Tokyo');
|
||||
SELECT toString(toDateTime(1436956200), 'Pacific/Pitcairn');
|
||||
|
||||
SELECT toString(toDateTime(x), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toString(toDateTime(1436956200), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toString(toDateTime(x), 'Europe/London') FROM foo ORDER BY x ASC;
|
||||
|
||||
/* toUnixTimestamp */
|
||||
|
||||
SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Europe/Moscow');
|
||||
SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Europe/Paris');
|
||||
SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Europe/London');
|
||||
SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Asia/Tokyo');
|
||||
SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Pacific/Pitcairn');
|
||||
|
||||
SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/Moscow'), 'Europe/Moscow');
|
||||
SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/Paris'), 'Europe/Paris');
|
||||
SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/London'), 'Europe/London');
|
||||
SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Asia/Tokyo'), 'Asia/Tokyo');
|
||||
SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Pacific/Pitcairn'), 'Pacific/Pitcairn');
|
||||
|
||||
SELECT toUnixTimestamp(toString(toDateTime(x)), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toUnixTimestamp(toString(toDateTime(1426415400)), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toUnixTimestamp(toString(toDateTime(x)), 'Europe/Paris') FROM foo ORDER BY x ASC;
|
@ -0,0 +1,8 @@
|
||||
['']
|
||||
---
|
||||
---
|
||||
0 ['hello']
|
||||
---
|
||||
---
|
||||
0 []
|
||||
---
|
@ -0,0 +1,11 @@
|
||||
SELECT arrayFilter(x -> notEmpty(concat(x, 'hello')), ['']) ARRAY JOIN [0] AS elem, arrayMap(x -> concat(x, 'hello'), ['']) AS unused WHERE NOT ignore(elem);
|
||||
SELECT '---';
|
||||
SELECT arrayFilter(x -> x = 'hello', ['']) ARRAY JOIN [0] AS elem WHERE NOT ignore(elem) AND arrayExists(x -> x = 'hello', ['']);
|
||||
SELECT '---';
|
||||
SELECT arrayJoin([0]), replicate('hello', [1]) WHERE NOT ignore(replicate('hello', [1]));
|
||||
SELECT '---';
|
||||
SELECT arrayJoin([0]), replicate('hello', emptyArrayString()) ARRAY JOIN emptyArrayString() AS unused WHERE NOT ignore(replicate('hello', emptyArrayString()));
|
||||
SELECT '---';
|
||||
SELECT arrayJoin([0]), replicate('hello', emptyArrayString()) WHERE NOT ignore(replicate('hello', emptyArrayString()));
|
||||
SELECT '---';
|
||||
SELECT replicate('hello', emptyArrayString()) ARRAY JOIN emptyArrayString() AS unused WHERE NOT ignore(replicate('hello', emptyArrayString()));
|
@ -1,57 +1,115 @@
|
||||
#include <cstring>
|
||||
#include <Yandex/DateLUT.h>
|
||||
#include <Poco/Exception.h>
|
||||
|
||||
#include <unicode/timezone.h>
|
||||
#include <unicode/unistr.h>
|
||||
|
||||
std::string DateLUT::default_time_zone;
|
||||
|
||||
DateLUT::DateLUT()
|
||||
{
|
||||
size_t i = 0;
|
||||
time_t start_of_day = DATE_LUT_MIN;
|
||||
using namespace icu;
|
||||
|
||||
do
|
||||
std::unique_ptr<TimeZone> tz(TimeZone::createDefault());
|
||||
if (tz == nullptr)
|
||||
throw Poco::Exception("Failed to determine the host time zone.");
|
||||
|
||||
UnicodeString u_out;
|
||||
tz->getID(u_out);
|
||||
u_out.toUTF8String(default_time_zone);
|
||||
|
||||
std::unique_ptr<StringEnumeration> time_zone_ids(TimeZone::createEnumeration());
|
||||
if (time_zone_ids == nullptr)
|
||||
throw Poco::Exception("Failed to query the list of time zones.");
|
||||
|
||||
UErrorCode status = U_ZERO_ERROR;
|
||||
const UnicodeString * zone_id = time_zone_ids->snext(status);
|
||||
if (zone_id == nullptr)
|
||||
throw Poco::Exception("No time zone available.");
|
||||
|
||||
std::vector<UnicodeString> time_zones;
|
||||
while ((zone_id != nullptr) && (status == U_ZERO_ERROR))
|
||||
{
|
||||
if (i > DATE_LUT_MAX_DAY_NUM)
|
||||
throw Poco::Exception("Cannot create DateLUT: i > DATE_LUT_MAX_DAY_NUM.");
|
||||
|
||||
tm time_descr;
|
||||
localtime_r(&start_of_day, &time_descr);
|
||||
|
||||
time_descr.tm_hour = 0;
|
||||
time_descr.tm_min = 0;
|
||||
time_descr.tm_sec = 0;
|
||||
time_descr.tm_isdst = -1;
|
||||
|
||||
start_of_day = mktime(&time_descr);
|
||||
|
||||
Values & values = lut[i];
|
||||
|
||||
values.year = time_descr.tm_year + 1900;
|
||||
values.month = time_descr.tm_mon + 1;
|
||||
values.day_of_week = time_descr.tm_wday == 0 ? 7 : time_descr.tm_wday;
|
||||
values.day_of_month = time_descr.tm_mday;
|
||||
|
||||
values.date = start_of_day;
|
||||
|
||||
/// Переходим на следующий день.
|
||||
++time_descr.tm_mday;
|
||||
|
||||
/** Обратите внимание, что в 1981-1984 году в России,
|
||||
* 1 апреля начиналось в час ночи, а не в полночь.
|
||||
* Если здесь оставить час равным нулю, то прибавление единицы к дню, привело бы к 23 часам того же дня.
|
||||
*/
|
||||
time_descr.tm_hour = 12;
|
||||
start_of_day = mktime(&time_descr);
|
||||
|
||||
++i;
|
||||
} while (start_of_day <= DATE_LUT_MAX);
|
||||
|
||||
/// Заполняем lookup таблицу для годов
|
||||
memset(years_lut, 0, DATE_LUT_YEARS * sizeof(years_lut[0]));
|
||||
for (size_t day = 0; day < i && lut[day].year <= DATE_LUT_MAX_YEAR; ++day)
|
||||
{
|
||||
if (lut[day].month == 1 && lut[day].day_of_month == 1)
|
||||
years_lut[lut[day].year - DATE_LUT_MIN_YEAR] = day;
|
||||
time_zones.push_back(*zone_id);
|
||||
zone_id = time_zone_ids->snext(status);
|
||||
}
|
||||
|
||||
offset_at_start_of_epoch = 86400 - lut[findIndex(86400)].date;
|
||||
size_t group_id = 0;
|
||||
|
||||
for (const auto & time_zone : time_zones)
|
||||
{
|
||||
const UnicodeString & u_group_name = TimeZone::getEquivalentID(time_zone, 0);
|
||||
std::string group_name;
|
||||
|
||||
if (u_group_name.isEmpty())
|
||||
{
|
||||
time_zone.toUTF8String(group_name);
|
||||
|
||||
auto res = time_zone_to_group.insert(std::make_pair(group_name, group_id));
|
||||
if (!res.second)
|
||||
throw Poco::Exception("Failed to initialize time zone information.");
|
||||
++group_id;
|
||||
}
|
||||
else
|
||||
{
|
||||
u_group_name.toUTF8String(group_name);
|
||||
|
||||
auto it = time_zone_to_group.find(group_name);
|
||||
if (it == time_zone_to_group.end())
|
||||
{
|
||||
auto count = TimeZone::countEquivalentIDs(time_zone);
|
||||
if (count == 0)
|
||||
throw Poco::Exception("Inconsistent time zone information.");
|
||||
|
||||
for (auto i = 0; i < count; ++i)
|
||||
{
|
||||
const UnicodeString & u_name = TimeZone::getEquivalentID(time_zone, i);
|
||||
std::string name;
|
||||
u_name.toUTF8String(name);
|
||||
auto res = time_zone_to_group.insert(std::make_pair(name, group_id));
|
||||
if (!res.second)
|
||||
throw Poco::Exception("Failed to initialize time zone information.");
|
||||
}
|
||||
++group_id;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (group_id == 0)
|
||||
throw Poco::Exception("Could not find any time zone information.");
|
||||
|
||||
date_lut_impl_list = std::make_unique<DateLUTImplList>(group_id);
|
||||
}
|
||||
|
||||
DateLUTImpl & DateLUT::instance(const std::string & time_zone)
|
||||
{
|
||||
auto & date_lut = Singleton<DateLUT>::instance();
|
||||
return date_lut.get(time_zone);
|
||||
}
|
||||
|
||||
DateLUTImpl & DateLUT::get(const std::string & time_zone)
|
||||
{
|
||||
const std::string & actual_time_zone = time_zone.empty() ? default_time_zone : time_zone;
|
||||
|
||||
auto it = time_zone_to_group.find(actual_time_zone);
|
||||
if (it == time_zone_to_group.end())
|
||||
throw Poco::Exception("Invalid time zone " + actual_time_zone);
|
||||
|
||||
const auto & group_id = it->second;
|
||||
auto & wrapper = (*date_lut_impl_list)[group_id];
|
||||
|
||||
DateLUTImpl * tmp = wrapper.load(std::memory_order_acquire);
|
||||
if (tmp == nullptr)
|
||||
{
|
||||
std::lock_guard<std::mutex> guard(mutex);
|
||||
tmp = wrapper.load(std::memory_order_acquire);
|
||||
if (tmp == nullptr)
|
||||
{
|
||||
tmp = new DateLUTImpl(actual_time_zone);
|
||||
wrapper.store(tmp, std::memory_order_release);
|
||||
}
|
||||
}
|
||||
|
||||
return *tmp;
|
||||
}
|
||||
|
||||
|
136
libs/libcommon/src/DateLUTImpl.cpp
Normal file
136
libs/libcommon/src/DateLUTImpl.cpp
Normal file
@ -0,0 +1,136 @@
|
||||
#include <Yandex/DateLUTImpl.h>
|
||||
#include <Poco/Exception.h>
|
||||
|
||||
#include <memory>
|
||||
#include <cstring>
|
||||
#include <glib.h>
|
||||
|
||||
namespace details { namespace {
|
||||
|
||||
struct GTimeZoneUnref
|
||||
{
|
||||
void operator()(GTimeZone * tz) const
|
||||
{
|
||||
g_time_zone_unref(tz);
|
||||
}
|
||||
};
|
||||
|
||||
using GTimeZonePtr = std::unique_ptr<GTimeZone, GTimeZoneUnref>;
|
||||
|
||||
struct GDateTimeUnref
|
||||
{
|
||||
void operator()(GDateTime * dt) const
|
||||
{
|
||||
g_date_time_unref(dt);
|
||||
}
|
||||
};
|
||||
|
||||
using GDateTimePtr = std::unique_ptr<GDateTime, GDateTimeUnref>;
|
||||
|
||||
GTimeZonePtr createGTimeZone(const std::string & description)
|
||||
{
|
||||
GTimeZone * tz = g_time_zone_new(description.c_str());
|
||||
if (tz == nullptr)
|
||||
throw Poco::Exception("Failed to create GTimeZone object.");
|
||||
|
||||
return GTimeZonePtr(tz);
|
||||
}
|
||||
|
||||
GDateTimePtr createGDateTime(time_t timestamp)
|
||||
{
|
||||
GDateTime * dt= g_date_time_new_from_unix_utc(timestamp);
|
||||
if (dt == nullptr)
|
||||
throw Poco::Exception("Failed to create GDateTime object.");
|
||||
|
||||
return GDateTimePtr(dt);
|
||||
}
|
||||
|
||||
GDateTimePtr createGDateTime(const GTimeZonePtr & p_tz, const GDateTimePtr & p_dt)
|
||||
{
|
||||
GDateTime * dt = p_dt.get();
|
||||
if (dt == nullptr)
|
||||
throw Poco::Exception("Null pointer.");
|
||||
|
||||
GDateTime * local_dt = g_date_time_new(p_tz.get(),
|
||||
g_date_time_get_year(dt),
|
||||
g_date_time_get_month(dt),
|
||||
g_date_time_get_day_of_month(dt),
|
||||
g_date_time_get_hour(dt),
|
||||
g_date_time_get_minute(dt),
|
||||
g_date_time_get_second(dt));
|
||||
if (local_dt == nullptr)
|
||||
throw Poco::Exception("Failed to create GDateTime object.");
|
||||
|
||||
return GDateTimePtr(local_dt);
|
||||
}
|
||||
|
||||
GDateTimePtr toNextDay(const GTimeZonePtr & p_tz, const GDateTimePtr & p_dt)
|
||||
{
|
||||
GDateTime * dt = p_dt.get();
|
||||
if (dt == nullptr)
|
||||
throw Poco::Exception("Null pointer.");
|
||||
|
||||
dt = g_date_time_add_days(dt, 1);
|
||||
if (dt == nullptr)
|
||||
throw Poco::Exception("Failed to create GDateTime object.");
|
||||
|
||||
GDateTimePtr p_next_dt = GDateTimePtr(dt);
|
||||
GDateTime * next_dt = p_next_dt.get();
|
||||
|
||||
dt = g_date_time_new(p_tz.get(),
|
||||
g_date_time_get_year(next_dt),
|
||||
g_date_time_get_month(next_dt),
|
||||
g_date_time_get_day_of_month(next_dt),
|
||||
0, 0, 0);
|
||||
if (dt == nullptr)
|
||||
throw Poco::Exception("Failed to create GDateTime object.");
|
||||
|
||||
return GDateTimePtr(dt);
|
||||
}
|
||||
|
||||
}}
|
||||
|
||||
DateLUTImpl::DateLUTImpl(const std::string & time_zone)
|
||||
{
|
||||
details::GTimeZonePtr p_tz = details::createGTimeZone(time_zone);
|
||||
|
||||
size_t i = 0;
|
||||
time_t start_of_day = DATE_LUT_MIN;
|
||||
|
||||
details::GDateTimePtr p_dt = details::createGDateTime(start_of_day);
|
||||
|
||||
p_dt = details::createGDateTime(p_tz, p_dt);
|
||||
|
||||
do
|
||||
{
|
||||
if (i > DATE_LUT_MAX_DAY_NUM)
|
||||
throw Poco::Exception("Cannot create DateLUTImpl: i > DATE_LUT_MAX_DAY_NUM.");
|
||||
|
||||
GDateTime * dt = p_dt.get();
|
||||
|
||||
start_of_day = g_date_time_to_unix(dt);
|
||||
|
||||
Values & values = lut[i];
|
||||
|
||||
values.year = g_date_time_get_year(dt);
|
||||
values.month = g_date_time_get_month(dt);
|
||||
values.day_of_week = g_date_time_get_day_of_week(dt);
|
||||
values.day_of_month = g_date_time_get_day_of_month(dt);
|
||||
values.date = start_of_day;
|
||||
|
||||
/// Переходим на следующий день.
|
||||
p_dt = details::toNextDay(p_tz, p_dt);
|
||||
++i;
|
||||
}
|
||||
while (start_of_day <= DATE_LUT_MAX);
|
||||
|
||||
/// Заполняем lookup таблицу для годов
|
||||
::memset(years_lut, 0, DATE_LUT_YEARS * sizeof(years_lut[0]));
|
||||
for (size_t day = 0; day < i && lut[day].year <= DATE_LUT_MAX_YEAR; ++day)
|
||||
{
|
||||
if (lut[day].month == 1 && lut[day].day_of_month == 1)
|
||||
years_lut[lut[day].year - DATE_LUT_MIN_YEAR] = day;
|
||||
}
|
||||
|
||||
offset_at_start_of_epoch = g_time_zone_get_offset(p_tz.get(), g_time_zone_find_interval(p_tz.get(), G_TIME_TYPE_UNIVERSAL, 0));
|
||||
}
|
@ -33,7 +33,7 @@ static time_t orderedIdentifierToDate(unsigned value)
|
||||
|
||||
void loop(time_t begin, time_t end, int step)
|
||||
{
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
|
||||
for (time_t t = begin; t < end; t += step)
|
||||
std::cout << toString(t)
|
||||
|
@ -35,7 +35,7 @@ static time_t orderedIdentifierToDate(unsigned value)
|
||||
|
||||
void loop(time_t begin, time_t end, int step)
|
||||
{
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
|
||||
for (time_t t = begin; t < end; t += step)
|
||||
{
|
||||
|
@ -7,12 +7,12 @@ int main(int argc, char ** argv)
|
||||
/** В DateLUT был глюк - для времён из дня 1970-01-01, возвращался номер часа больше 23. */
|
||||
static const time_t TIME = 66130;
|
||||
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
|
||||
std::cerr << date_lut.toHourInaccurate(TIME) << std::endl;
|
||||
std::cerr << date_lut.toDayNum(TIME) << std::endl;
|
||||
|
||||
const DateLUT::Values * values = reinterpret_cast<const DateLUT::Values *>(&date_lut);
|
||||
const auto * values = reinterpret_cast<const DateLUTImpl::Values *>(&date_lut);
|
||||
|
||||
std::cerr << values[0].date << ", " << time_t(values[1].date - values[0].date) << std::endl;
|
||||
|
||||
|
@ -29,8 +29,8 @@ private:
|
||||
|
||||
void init(time_t time)
|
||||
{
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
const DateLUT::Values & values = date_lut.getValues(time);
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & values = date_lut.getValues(time);
|
||||
|
||||
m_year = values.year;
|
||||
m_month = values.month;
|
||||
@ -66,7 +66,7 @@ public:
|
||||
|
||||
Date(DayNum_t day_num)
|
||||
{
|
||||
const DateLUT::Values & values = DateLUT::instance().getValues(day_num);
|
||||
const auto & values = DateLUT::instance().getValues(day_num);
|
||||
m_year = values.year;
|
||||
m_month = values.month;
|
||||
m_day = values.day_of_month;
|
||||
|
@ -43,8 +43,8 @@ private:
|
||||
return;
|
||||
}
|
||||
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
const DateLUT::Values & values = date_lut.getValues(time);
|
||||
auto & date_lut = DateLUT::instance();
|
||||
const auto & values = date_lut.getValues(time);
|
||||
|
||||
m_year = values.year;
|
||||
m_month = values.month;
|
||||
|
@ -145,7 +145,7 @@ private:
|
||||
|
||||
time_t getDateTimeImpl() const
|
||||
{
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
|
||||
if (m_length == 10)
|
||||
{
|
||||
@ -173,7 +173,7 @@ private:
|
||||
|
||||
time_t getDateImpl() const
|
||||
{
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
|
||||
if (m_length == 10 || m_length == 19)
|
||||
{
|
||||
@ -216,7 +216,7 @@ private:
|
||||
return getDateImpl();
|
||||
else
|
||||
{
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
auto & date_lut = DateLUT::instance();
|
||||
return date_lut.toDate(getIntImpl());
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user