mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-17 20:02:05 +00:00
dbms: Server: Support for multiple time zones: development [#METR-15618]
This commit is contained in:
parent
7968c06699
commit
5a3d5970c5
@ -256,6 +256,111 @@ time_t convert_time(time_t remote_time, DateLUTImpl & remote_date_lut, DateLUTIm
|
||||
}
|
||||
}
|
||||
|
||||
struct DateTimeConverter
|
||||
{
|
||||
static void vector_vector(const typename ColumnVector<DataTypeDateTime::FieldType>::Container_t & 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 = convert_time(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 typename ColumnVector<DataTypeDateTime::FieldType>::Container_t & 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 = convert_time(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(DataTypeDateTime::FieldType 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 = convert_time(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(DataTypeDateTime::FieldType 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 = convert_time(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>
|
||||
{
|
||||
@ -263,57 +368,87 @@ struct ConvertImpl<DataTypeDateTime, DataTypeString, Name>
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
std::string time_zone;
|
||||
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
||||
const ColumnVector<DataTypeDateTime::FieldType> * sources = typeid_cast<const ColumnVector<DataTypeDateTime::FieldType> *>(&*source_col);
|
||||
const ColumnConst<DataTypeDateTime::FieldType> * const_source = typeid_cast<const ColumnConst<DataTypeDateTime::FieldType> *>(&*source_col);
|
||||
|
||||
if (arguments.size() == 2)
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[1]).column;
|
||||
if (const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*column))
|
||||
time_zone = col->getData();
|
||||
}
|
||||
|
||||
auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
|
||||
if (const ColumnVector<FromFieldType> * col_from = typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
ColumnString * col_to = new ColumnString;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<FromFieldType>::Container_t & vec_from = col_from->getData();
|
||||
ColumnString::Chars_t & data_to = col_to->getChars();
|
||||
ColumnString::Offsets_t & offsets_to = col_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)
|
||||
if (sources)
|
||||
{
|
||||
auto ti = convert_time(vec_from[i], remote_date_lut, local_date_lut);
|
||||
formatImpl<DataTypeDateTime>(ti, write_buffer);
|
||||
writeChar(0, write_buffer);
|
||||
offsets_to[i] = write_buffer.count();
|
||||
ColumnString * col_to = new ColumnString;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
auto & vec_from = sources->getData();
|
||||
auto & vec_to = *col_to;
|
||||
|
||||
DateTimeConverter::vector_constant(vec_from, "", vec_to);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
std::string res;
|
||||
DateTimeConverter::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);
|
||||
}
|
||||
data_to.resize(write_buffer.count());
|
||||
}
|
||||
else if (const ColumnConst<FromFieldType> * col_from = typeid_cast<const ColumnConst<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
std::vector<char> buf;
|
||||
WriteBufferFromVector<std::vector<char> > write_buffer(buf);
|
||||
auto ti = convert_time(col_from->getData(), remote_date_lut, local_date_lut);
|
||||
formatImpl<DataTypeDateTime>(ti, write_buffer);
|
||||
block.getByPosition(result).column = new ColumnConstString(col_from->size(), std::string(&buf[0], write_buffer.count()));
|
||||
const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column;
|
||||
const ColumnString * time_zones = typeid_cast<const ColumnString *>(&*time_zone_col);
|
||||
const ColumnConstString * 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)
|
||||
DateTimeConverter::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||
else if (const_time_zone)
|
||||
DateTimeConverter::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;
|
||||
|
||||
DateTimeConverter::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||
}
|
||||
else if (const_time_zone)
|
||||
{
|
||||
std::string res;
|
||||
DateTimeConverter::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("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Преобразование строк в числа, даты, даты-с-временем: через парсинг.
|
||||
*/
|
||||
template <typename DataType> void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb) { readText(x,rb); }
|
||||
@ -375,6 +510,98 @@ struct ConvertImpl<DataTypeString, ToDataType, Name>
|
||||
}
|
||||
};
|
||||
|
||||
struct TimeConverter
|
||||
{
|
||||
static void vector_vector(const ColumnString::Chars_t & vec_from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets,
|
||||
typename ColumnVector<DataTypeInt32::FieldType>::Container_t & 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 = convert_time(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,
|
||||
typename ColumnVector<DataTypeInt32::FieldType>::Container_t & 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 = convert_time(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,
|
||||
typename ColumnVector<DataTypeInt32::FieldType>::Container_t & 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 = convert_time(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, DataTypeInt32::FieldType & 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 = convert_time(x, local_date_lut, remote_date_lut);
|
||||
}
|
||||
};
|
||||
|
||||
struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; };
|
||||
|
||||
template<>
|
||||
@ -384,58 +611,94 @@ struct ConvertImpl<DataTypeString, DataTypeInt32, NameToUnixTimestamp>
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
std::string time_zone;
|
||||
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
||||
const ColumnString * sources = typeid_cast<const ColumnString *>(&*source_col);
|
||||
const ColumnConstString * const_source = typeid_cast<const ColumnConstString *>(&*source_col);
|
||||
|
||||
if (arguments.size() == 2)
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[1]).column;
|
||||
if (const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*column))
|
||||
time_zone = col->getData();
|
||||
}
|
||||
|
||||
auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
auto & local_date_lut = DateLUT::instance();
|
||||
|
||||
if (const ColumnString * col_from = typeid_cast<const ColumnString *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
ColumnVector<ToFieldType> * col_to = new ColumnVector<ToFieldType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const ColumnString::Chars_t & data_from = col_from->getChars();
|
||||
typename ColumnVector<ToFieldType>::Container_t & vec_to = col_to->getData();
|
||||
size_t size = col_from->size();
|
||||
vec_to.resize(size);
|
||||
|
||||
ReadBuffer read_buffer(const_cast<char *>(reinterpret_cast<const char *>(&data_from[0])), data_from.size(), 0);
|
||||
|
||||
char zero = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
if (sources)
|
||||
{
|
||||
DataTypeDateTime::FieldType x = 0;
|
||||
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||
auto ti = convert_time(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);
|
||||
ColumnVector<DataTypeInt32::FieldType> * col_to = new ColumnVector<DataTypeInt32::FieldType>;
|
||||
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);
|
||||
|
||||
TimeConverter::vector_constant(vec_from, "", vec_to);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
DataTypeInt32::FieldType res;
|
||||
TimeConverter::constant_constant(const_source->getData(), "", res);
|
||||
block.getByPosition(result).column = new ColumnConst<DataTypeInt32::FieldType>(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 (const ColumnConstString * col_from = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column))
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
const String & s = col_from->getData();
|
||||
ReadBufferFromString read_buffer(s);
|
||||
DataTypeDateTime::FieldType x = 0;
|
||||
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||
auto ti = convert_time(x, local_date_lut, remote_date_lut);
|
||||
block.getByPosition(result).column = new ColumnConst<ToFieldType>(col_from->size(), ti);
|
||||
const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column;
|
||||
const ColumnString * time_zones = typeid_cast<const ColumnString *>(&*time_zone_col);
|
||||
const ColumnConstString * const_time_zone = typeid_cast<const ColumnConstString *>(&*time_zone_col);
|
||||
|
||||
if (sources)
|
||||
{
|
||||
ColumnVector<DataTypeInt32::FieldType> * col_to = new ColumnVector<DataTypeInt32::FieldType>;
|
||||
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)
|
||||
TimeConverter::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||
else if (const_time_zone)
|
||||
TimeConverter::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)
|
||||
{
|
||||
ColumnVector<DataTypeInt32::FieldType> * col_to = new ColumnVector<DataTypeInt32::FieldType>;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
auto & vec_to = col_to->getData();
|
||||
vec_to.resize(time_zones->getOffsets().size());
|
||||
|
||||
TimeConverter::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||
}
|
||||
else if (const_time_zone)
|
||||
{
|
||||
DataTypeInt32::FieldType res;
|
||||
TimeConverter::constant_constant(const_source->getData(), const_time_zone->getData(), res);
|
||||
block.getByPosition(result).column = new ColumnConst<DataTypeInt32::FieldType>(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("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + NameToUnixTimestamp::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Если типы совпадают - просто скопируем ссылку на столбец.
|
||||
*/
|
||||
template <typename Name>
|
||||
|
@ -33,16 +33,31 @@
|
||||
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 11:30: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-03-09 2015-03-01 2015-01-01 1970-01-02 19:30: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
|
||||
@ -53,3 +68,18 @@
|
||||
1426415400
|
||||
1426415400
|
||||
1426415400
|
||||
1412113800
|
||||
1419811200
|
||||
1420102800
|
||||
1426455000
|
||||
1436934600
|
||||
1426393800
|
||||
1426426200
|
||||
1426415400
|
||||
1426422600
|
||||
1426455000
|
||||
1412113800
|
||||
1419807600
|
||||
1420110000
|
||||
1426422600
|
||||
1436959800
|
||||
|
@ -56,7 +56,7 @@ 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 x 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 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 */
|
||||
|
||||
@ -66,6 +66,10 @@ 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');
|
||||
@ -79,3 +83,7 @@ SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/Paris'), 'Europe
|
||||
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 x 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;
|
||||
|
Loading…
Reference in New Issue
Block a user