Merge pull request #15408 from ClickHouse/random_timezone_for_stateless_tests

Use random timezone in stateless and stateful tests
This commit is contained in:
alexey-milovidov 2021-07-27 16:18:14 +03:00 committed by GitHub
commit d7b2404119
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
76 changed files with 1174 additions and 1190 deletions

View File

@ -61,4 +61,7 @@ ENV TSAN_OPTIONS='halt_on_error=1 history_size=7'
ENV UBSAN_OPTIONS='print_stacktrace=1'
ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1'
ENV TZ=Europe/Moscow
RUN ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone
CMD sleep 1

View File

@ -2,6 +2,11 @@
set -e -x
# Choose random timezone for this test run
TZ="$(grep -v '#' /usr/share/zoneinfo/zone.tab | awk '{print $3}' | shuf | head -n1)"
echo "Choosen random timezone $TZ"
ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone
dpkg -i package_folder/clickhouse-common-static_*.deb;
dpkg -i package_folder/clickhouse-common-static-dbg_*.deb
dpkg -i package_folder/clickhouse-server_*.deb

View File

@ -3,6 +3,11 @@
# fail on errors, verbose and export all env variables
set -e -x -a
# Choose random timezone for this test run.
TZ="$(grep -v '#' /usr/share/zoneinfo/zone.tab | awk '{print $3}' | shuf | head -n1)"
echo "Choosen random timezone $TZ"
ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone
dpkg -i package_folder/clickhouse-common-static_*.deb
dpkg -i package_folder/clickhouse-common-static-dbg_*.deb
dpkg -i package_folder/clickhouse-server_*.deb

View File

@ -77,9 +77,6 @@ RUN mkdir -p /tmp/clickhouse-odbc-tmp \
&& odbcinst -i -s -l -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbc.ini.sample \
&& rm -rf /tmp/clickhouse-odbc-tmp
ENV TZ=Europe/Moscow
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
COPY run.sh /
CMD ["/bin/bash", "/run.sh"]

View File

@ -1,8 +1,6 @@
# docker build -t yandex/clickhouse-unit-test .
FROM yandex/clickhouse-stateless-test
ENV TZ=Europe/Moscow
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
RUN apt-get install gdb
COPY run.sh /

View File

@ -1423,18 +1423,23 @@ namespace
};
/// Serializes a ColumnVector<UInt32> containing dates to a field of any type except TYPE_MESSAGE, TYPE_GROUP, TYPE_BOOL, TYPE_ENUM.
/// Serializes a ColumnVector<UInt32> containing datetimes to a field of any type except TYPE_MESSAGE, TYPE_GROUP, TYPE_BOOL, TYPE_ENUM.
class ProtobufSerializerDateTime : public ProtobufSerializerNumber<UInt32>
{
public:
ProtobufSerializerDateTime(
const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_)
: ProtobufSerializerNumber<UInt32>(field_descriptor_, reader_or_writer_)
const DataTypeDateTime & type,
const FieldDescriptor & field_descriptor_,
const ProtobufReaderOrWriter & reader_or_writer_)
: ProtobufSerializerNumber<UInt32>(field_descriptor_, reader_or_writer_),
date_lut(type.getTimeZone())
{
setFunctions();
}
protected:
const DateLUTImpl & date_lut;
void setFunctions()
{
switch (field_typeid)
@ -1458,17 +1463,17 @@ namespace
{
write_function = [this](UInt32 value)
{
dateTimeToString(value, text_buffer);
dateTimeToString(value, text_buffer, date_lut);
writeStr(text_buffer);
};
read_function = [this]() -> UInt32
{
readStr(text_buffer);
return stringToDateTime(text_buffer);
return stringToDateTime(text_buffer, date_lut);
};
default_function = [this]() -> UInt32 { return stringToDateTime(field_descriptor.default_value_string()); };
default_function = [this]() -> UInt32 { return stringToDateTime(field_descriptor.default_value_string(), date_lut); };
break;
}
@ -1477,17 +1482,17 @@ namespace
}
}
static void dateTimeToString(time_t tm, String & str)
static void dateTimeToString(time_t tm, String & str, const DateLUTImpl & lut)
{
WriteBufferFromString buf{str};
writeDateTimeText(tm, buf);
writeDateTimeText(tm, buf, lut);
}
static time_t stringToDateTime(const String & str)
static time_t stringToDateTime(const String & str, const DateLUTImpl & lut)
{
ReadBufferFromString buf{str};
time_t tm = 0;
readDateTimeText(tm, buf);
readDateTimeText(tm, buf, lut);
if (tm < 0)
tm = 0;
return tm;
@ -2833,7 +2838,7 @@ namespace
case TypeIndex::Float32: return std::make_unique<ProtobufSerializerNumber<Float32>>(field_descriptor, reader_or_writer);
case TypeIndex::Float64: return std::make_unique<ProtobufSerializerNumber<Float64>>(field_descriptor, reader_or_writer);
case TypeIndex::Date: return std::make_unique<ProtobufSerializerDate>(field_descriptor, reader_or_writer);
case TypeIndex::DateTime: return std::make_unique<ProtobufSerializerDateTime>(field_descriptor, reader_or_writer);
case TypeIndex::DateTime: return std::make_unique<ProtobufSerializerDateTime>(assert_cast<const DataTypeDateTime &>(*data_type), field_descriptor, reader_or_writer);
case TypeIndex::DateTime64: return std::make_unique<ProtobufSerializerDateTime64>(assert_cast<const DataTypeDateTime64 &>(*data_type), field_descriptor, reader_or_writer);
case TypeIndex::String: return std::make_unique<ProtobufSerializerString<false>>(field_descriptor, reader_or_writer);
case TypeIndex::FixedString: return std::make_unique<ProtobufSerializerString<true>>(typeid_cast<std::shared_ptr<const DataTypeFixedString>>(data_type), field_descriptor, reader_or_writer);

File diff suppressed because it is too large Load Diff

View File

@ -1,24 +1,19 @@
#pragma once
#include "config_formats.h"
#if USE_ARROW || USE_ORC || USE_PARQUET
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeString.h>
#include <arrow/type.h>
#include <Columns/ColumnVector.h>
#include <arrow/table.h>
#include <arrow/array.h>
#include <arrow/buffer.h>
#include <Processors/Chunk.h>
#include <Core/Block.h>
namespace DB
{
class Block;
class Chunk;
class ArrowColumnToCHColumn
{
public:
@ -27,37 +22,16 @@ public:
void arrowTableToCHChunk(Chunk & res, std::shared_ptr<arrow::Table> & table);
private:
#define FOR_ARROW_NUMERIC_TYPES(M) \
M(arrow::Type::UINT8, DB::UInt8) \
M(arrow::Type::INT8, DB::Int8) \
M(arrow::Type::UINT16, DB::UInt16) \
M(arrow::Type::INT16, DB::Int16) \
M(arrow::Type::UINT32, DB::UInt32) \
M(arrow::Type::INT32, DB::Int32) \
M(arrow::Type::UINT64, DB::UInt64) \
M(arrow::Type::INT64, DB::Int64) \
M(arrow::Type::HALF_FLOAT, DB::Float32) \
M(arrow::Type::FLOAT, DB::Float32) \
M(arrow::Type::DOUBLE, DB::Float64)
#define FOR_ARROW_INDEXES_TYPES(M) \
M(arrow::Type::UINT8, DB::UInt8) \
M(arrow::Type::INT8, DB::UInt8) \
M(arrow::Type::UINT16, DB::UInt16) \
M(arrow::Type::INT16, DB::UInt16) \
M(arrow::Type::UINT32, DB::UInt32) \
M(arrow::Type::INT32, DB::UInt32) \
M(arrow::Type::UINT64, DB::UInt64) \
M(arrow::Type::INT64, DB::UInt64)
const Block & header;
std::unordered_map<std::string, DataTypePtr> name_to_internal_type;
const std::string format_name;
/// Map {column name : dictionary column}.
/// To avoid converting dictionary from Arrow Dictionary
/// to LowCardinality every chunk we save it and reuse.
std::unordered_map<std::string, ColumnPtr> dictionary_values;
};
}
#endif

View File

@ -23,6 +23,30 @@
#include <arrow/type.h>
#include <arrow/util/decimal.h>
#define FOR_INTERNAL_NUMERIC_TYPES(M) \
M(UInt8, arrow::UInt8Builder) \
M(Int8, arrow::Int8Builder) \
M(UInt16, arrow::UInt16Builder) \
M(Int16, arrow::Int16Builder) \
M(UInt32, arrow::UInt32Builder) \
M(Int32, arrow::Int32Builder) \
M(UInt64, arrow::UInt64Builder) \
M(Int64, arrow::Int64Builder) \
M(Float32, arrow::FloatBuilder) \
M(Float64, arrow::DoubleBuilder)
#define FOR_ARROW_TYPES(M) \
M(UINT8, arrow::UInt8Type) \
M(INT8, arrow::Int8Type) \
M(UINT16, arrow::UInt16Type) \
M(INT16, arrow::Int16Type) \
M(UINT32, arrow::UInt32Type) \
M(INT32, arrow::Int32Type) \
M(UINT64, arrow::UInt64Type) \
M(INT64, arrow::Int64Type) \
M(FLOAT, arrow::FloatType) \
M(DOUBLE, arrow::DoubleType) \
M(STRING, arrow::StringType)
namespace DB
{
@ -46,11 +70,8 @@ namespace DB
{"Float32", arrow::float32()},
{"Float64", arrow::float64()},
//{"Date", arrow::date64()},
//{"Date", arrow::date32()},
{"Date", arrow::uint16()}, // CHECK
//{"DateTime", arrow::date64()}, // BUG! saves as date32
{"DateTime", arrow::uint32()},
{"Date", arrow::uint16()}, /// uint16 is used instead of date32, because Apache Arrow cannot correctly serialize Date32Array.
{"DateTime", arrow::uint32()}, /// uint32 is used instead of date64, because we don't need milliseconds.
{"String", arrow::binary()},
{"FixedString", arrow::binary()},
@ -265,11 +286,11 @@ namespace DB
auto value_type = assert_cast<arrow::DictionaryType *>(array_builder->type().get())->value_type();
#define DISPATCH(ARROW_TYPE_ID, ARROW_TYPE) \
if (arrow::Type::ARROW_TYPE_ID == value_type->id()) \
{ \
fillArrowArrayWithLowCardinalityColumnDataImpl<ARROW_TYPE>(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values); \
return; \
}
if (arrow::Type::ARROW_TYPE_ID == value_type->id()) \
{ \
fillArrowArrayWithLowCardinalityColumnDataImpl<ARROW_TYPE>(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values); \
return; \
}
FOR_ARROW_TYPES(DISPATCH)
#undef DISPATCH
@ -337,7 +358,6 @@ namespace DB
size_t end)
{
const auto & internal_data = assert_cast<const ColumnVector<UInt32> &>(*write_column).getData();
//arrow::Date64Builder builder;
arrow::UInt32Builder & builder = assert_cast<arrow::UInt32Builder &>(*array_builder);
arrow::Status status;
@ -346,8 +366,6 @@ namespace DB
if (null_bytemap && (*null_bytemap)[value_i])
status = builder.AppendNull();
else
/// Implicitly converts UInt16 to Int32
//status = date_builder.Append(static_cast<int64_t>(internal_data[value_i]) * 1000); // now ms. TODO check other units
status = builder.Append(internal_data[value_i]);
checkStatus(status, write_column->getName(), format_name);
@ -367,7 +385,7 @@ namespace DB
{
const String column_type_name = column_type->getFamilyName();
if ("Nullable" == column_type_name)
if (column_type->isNullable())
{
const ColumnNullable * column_nullable = assert_cast<const ColumnNullable *>(column.get());
ColumnPtr nested_column = column_nullable->getNestedColumnPtr();
@ -376,35 +394,35 @@ namespace DB
const PaddedPODArray<UInt8> & bytemap = assert_cast<const ColumnVector<UInt8> &>(*null_column).getData();
fillArrowArray(column_name, nested_column, nested_type, &bytemap, array_builder, format_name, start, end, dictionary_values);
}
else if ("String" == column_type_name)
else if (isString(column_type))
{
fillArrowArrayWithStringColumnData<ColumnString>(column, null_bytemap, format_name, array_builder, start, end);
}
else if ("FixedString" == column_type_name)
else if (isFixedString(column_type))
{
fillArrowArrayWithStringColumnData<ColumnFixedString>(column, null_bytemap, format_name, array_builder, start, end);
}
else if ("Date" == column_type_name)
else if (isDate(column_type))
{
fillArrowArrayWithDateColumnData(column, null_bytemap, format_name, array_builder, start, end);
}
else if ("DateTime" == column_type_name)
else if (isDateTime(column_type))
{
fillArrowArrayWithDateTimeColumnData(column, null_bytemap, format_name, array_builder, start, end);
}
else if ("Array" == column_type_name)
else if (isArray(column_type))
{
fillArrowArrayWithArrayColumnData<arrow::ListBuilder>(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values);
}
else if ("Tuple" == column_type_name)
else if (isTuple(column_type))
{
fillArrowArrayWithTupleColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values);
}
else if ("LowCardinality" == column_type_name)
else if (column_type->getTypeId() == TypeIndex::LowCardinality)
{
fillArrowArrayWithLowCardinalityColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values);
}
else if ("Map" == column_type_name)
else if (isMap(column_type))
{
ColumnPtr column_array = assert_cast<const ColumnMap *>(column.get())->getNestedColumnPtr();
DataTypePtr array_type = assert_cast<const DataTypeMap *>(column_type.get())->getNestedType();
@ -437,10 +455,10 @@ namespace DB
throw Exception{ErrorCodes::LOGICAL_ERROR, "Cannot fill arrow array with decimal data with type {}", column_type_name};
}
#define DISPATCH(CPP_NUMERIC_TYPE, ARROW_BUILDER_TYPE) \
else if (#CPP_NUMERIC_TYPE == column_type_name) \
{ \
fillArrowArrayWithNumericColumnData<CPP_NUMERIC_TYPE, ARROW_BUILDER_TYPE>(column, null_bytemap, format_name, array_builder, start, end); \
}
else if (#CPP_NUMERIC_TYPE == column_type_name) \
{ \
fillArrowArrayWithNumericColumnData<CPP_NUMERIC_TYPE, ARROW_BUILDER_TYPE>(column, null_bytemap, format_name, array_builder, start, end); \
}
FOR_INTERNAL_NUMERIC_TYPES(DISPATCH)
#undef DISPATCH
@ -448,7 +466,7 @@ namespace DB
{
throw Exception
{
fmt::format(R"(Internal type "{}" of a column "{}" is not supported for conversion into a {} data format.)", column_type_name, column_name, format_name),
fmt::format("Internal type '{}' of a column '{}' is not supported for conversion into {} data format.", column_type_name, column_name, format_name),
ErrorCodes::UNKNOWN_TYPE
};
}
@ -502,14 +520,15 @@ namespace DB
}
}
static std::shared_ptr<arrow::DataType> getArrowType(DataTypePtr column_type, ColumnPtr column, const std::string & column_name, const std::string & format_name, bool * is_column_nullable)
static std::shared_ptr<arrow::DataType> getArrowType(
DataTypePtr column_type, ColumnPtr column, const std::string & column_name, const std::string & format_name, bool * out_is_column_nullable)
{
if (column_type->isNullable())
{
DataTypePtr nested_type = assert_cast<const DataTypeNullable *>(column_type.get())->getNestedType();
ColumnPtr nested_column = assert_cast<const ColumnNullable *>(column.get())->getNestedColumnPtr();
auto arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, is_column_nullable);
*is_column_nullable = true;
auto arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, out_is_column_nullable);
*out_is_column_nullable = true;
return arrow_type;
}
@ -542,7 +561,7 @@ namespace DB
{
auto nested_type = assert_cast<const DataTypeArray *>(column_type.get())->getNestedType();
auto nested_column = assert_cast<const ColumnArray *>(column.get())->getDataPtr();
auto nested_arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, is_column_nullable);
auto nested_arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, out_is_column_nullable);
return arrow::list(nested_arrow_type);
}
@ -554,8 +573,8 @@ namespace DB
for (size_t i = 0; i != nested_types.size(); ++i)
{
String name = column_name + "." + std::to_string(i);
auto nested_arrow_type = getArrowType(nested_types[i], tuple_column->getColumnPtr(i), name, format_name, is_column_nullable);
nested_fields.push_back(std::make_shared<arrow::Field>(name, nested_arrow_type, *is_column_nullable));
auto nested_arrow_type = getArrowType(nested_types[i], tuple_column->getColumnPtr(i), name, format_name, out_is_column_nullable);
nested_fields.push_back(std::make_shared<arrow::Field>(name, nested_arrow_type, *out_is_column_nullable));
}
return arrow::struct_(std::move(nested_fields));
}
@ -568,7 +587,7 @@ namespace DB
const auto & indexes_column = lc_column->getIndexesPtr();
return arrow::dictionary(
getArrowTypeForLowCardinalityIndexes(indexes_column),
getArrowType(nested_type, nested_column, column_name, format_name, is_column_nullable));
getArrowType(nested_type, nested_column, column_name, format_name, out_is_column_nullable));
}
if (isMap(column_type))
@ -579,9 +598,8 @@ namespace DB
const auto & columns = assert_cast<const ColumnMap *>(column.get())->getNestedData().getColumns();
return arrow::map(
getArrowType(key_type, columns[0], column_name, format_name, is_column_nullable),
getArrowType(val_type, columns[1], column_name, format_name, is_column_nullable)
);
getArrowType(key_type, columns[0], column_name, format_name, out_is_column_nullable),
getArrowType(val_type, columns[1], column_name, format_name, out_is_column_nullable));
}
const std::string type_name = column_type->getFamilyName();
@ -594,8 +612,9 @@ namespace DB
return arrow_type_it->second;
}
throw Exception{fmt::format(R"(The type "{}" of a column "{}" is not supported for conversion into a {} data format.)", column_type->getName(), column_name, format_name),
ErrorCodes::UNKNOWN_TYPE};
throw Exception(ErrorCodes::UNKNOWN_TYPE,
"The type '{}' of a column '{}' is not supported for conversion into {} data format.",
column_type->getName(), column_name, format_name);
}
CHColumnToArrowColumn::CHColumnToArrowColumn(const Block & header, const std::string & format_name_, bool low_cardinality_as_dictionary_)
@ -638,7 +657,8 @@ namespace DB
arrow::Status status = MakeBuilder(pool, arrow_fields[column_i]->type(), &array_builder);
checkStatus(status, column->getName(), format_name);
fillArrowArray(header_column.name, column, header_column.type, nullptr, array_builder.get(), format_name, 0, column->size(), dictionary_values);
fillArrowArray(
header_column.name, column, header_column.type, nullptr, array_builder.get(), format_name, 0, column->size(), dictionary_values);
std::shared_ptr<arrow::Array> arrow_array;
status = array_builder->Finish(&arrow_array);

View File

@ -7,42 +7,18 @@
#include <Processors/Chunk.h>
#include <arrow/table.h>
namespace DB
{
class CHColumnToArrowColumn
{
public:
CHColumnToArrowColumn(const Block & header, const std::string & format_name_, bool low_cardinality_as_dictionary_ = false);
CHColumnToArrowColumn(const Block & header, const std::string & format_name_, bool low_cardinality_as_dictionary_);
void chChunkToArrowTable(std::shared_ptr<arrow::Table> & res, const Chunk & chunk, size_t columns_num);
private:
#define FOR_INTERNAL_NUMERIC_TYPES(M) \
M(UInt8, arrow::UInt8Builder) \
M(Int8, arrow::Int8Builder) \
M(UInt16, arrow::UInt16Builder) \
M(Int16, arrow::Int16Builder) \
M(UInt32, arrow::UInt32Builder) \
M(Int32, arrow::Int32Builder) \
M(UInt64, arrow::UInt64Builder) \
M(Int64, arrow::Int64Builder) \
M(Float32, arrow::FloatBuilder) \
M(Float64, arrow::DoubleBuilder)
#define FOR_ARROW_TYPES(M) \
M(UINT8, arrow::UInt8Type) \
M(INT8, arrow::Int8Type) \
M(UINT16, arrow::UInt16Type) \
M(INT16, arrow::Int16Type) \
M(UINT32, arrow::UInt32Type) \
M(INT32, arrow::Int32Type) \
M(UINT64, arrow::UInt64Type) \
M(INT64, arrow::Int64Type) \
M(FLOAT, arrow::FloatType) \
M(DOUBLE, arrow::DoubleType) \
M(STRING, arrow::StringType)
ColumnsWithTypeAndName header_columns;
std::vector<std::shared_ptr<arrow::Field>> arrow_fields;
const std::string format_name;
@ -52,5 +28,7 @@ private:
/// Dictionary every chunk we save it and reuse.
std::unordered_map<std::string, std::shared_ptr<arrow::Array>> dictionary_values;
};
}
#endif

View File

@ -2,16 +2,7 @@
#if USE_PARQUET
// TODO: clean includes
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Common/assert_cast.h>
#include <Core/callOnTypeIndex.h>
#include <DataStreams/SquashingBlockOutputStream.h>
#include <Formats/FormatFactory.h>
#include <IO/WriteHelpers.h>
#include <arrow/api.h>
#include <arrow/util/memory.h>
#include <parquet/arrow/writer.h>
#include "ArrowBufferedStreams.h"
#include "CHColumnToArrowColumn.h"
@ -19,6 +10,7 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_EXCEPTION;
@ -37,7 +29,7 @@ void ParquetBlockOutputFormat::consume(Chunk chunk)
if (!ch_column_to_arrow_column)
{
const Block & header = getPort(PortKind::Main).getHeader();
ch_column_to_arrow_column = std::make_unique<CHColumnToArrowColumn>(header, "Parquet");
ch_column_to_arrow_column = std::make_unique<CHColumnToArrowColumn>(header, "Parquet", false);
}
ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, chunk, columns_num);
@ -91,11 +83,7 @@ void registerOutputFormatProcessorParquet(FormatFactory & factory)
const RowOutputFormatParams &,
const FormatSettings & format_settings)
{
auto impl = std::make_shared<ParquetBlockOutputFormat>(buf, sample, format_settings);
/// TODO
// auto res = std::make_shared<SquashingBlockOutputStream>(impl, impl->getHeader(), format_settings.parquet.row_group_size, 0);
// res->disableFlush();
return impl;
return std::make_shared<ParquetBlockOutputFormat>(buf, sample, format_settings);
});
}

View File

@ -2,6 +2,8 @@
#include <AggregateFunctions/IAggregateFunction.h>
#include <common/DateLUTImpl.h>
#include <common/DateLUT.h>
#include <DataTypes/DataTypeDateTime.h>
namespace DB
{
@ -16,6 +18,8 @@ static GraphiteRollupSortedAlgorithm::ColumnsDefinition defineColumns(
def.value_column_num = header.getPositionByName(params.value_column_name);
def.version_column_num = header.getPositionByName(params.version_column_name);
def.time_column_type = header.getByPosition(def.time_column_num).type;
size_t num_columns = header.columns();
for (size_t i = 0; i < num_columns; ++i)
if (i != def.time_column_num && i != def.value_column_num && i != def.version_column_num)
@ -122,8 +126,8 @@ UInt32 GraphiteRollupSortedAlgorithm::selectPrecision(const Graphite::Retentions
* In this case, the date should not change. The date is calculated using the local time zone.
*
* If the rounding value is less than an hour,
* then, assuming that time zones that differ from UTC by a non-integer number of hours are not supported,
* just simply round the unix timestamp down to a multiple of 3600.
* then, assuming that time zones that differ from UTC by a multiple of 15-minute intervals
* (that is true for all modern timezones but not true for historical timezones).
* And if the rounding value is greater,
* then we will round down the number of seconds from the beginning of the day in the local time zone.
*
@ -131,7 +135,7 @@ UInt32 GraphiteRollupSortedAlgorithm::selectPrecision(const Graphite::Retentions
*/
static time_t roundTimeToPrecision(const DateLUTImpl & date_lut, time_t time, UInt32 precision)
{
if (precision <= 3600)
if (precision <= 900)
{
return time / precision * precision;
}
@ -145,7 +149,10 @@ static time_t roundTimeToPrecision(const DateLUTImpl & date_lut, time_t time, UI
IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge()
{
const DateLUTImpl & date_lut = DateLUT::instance();
/// Timestamp column can be DateTime or UInt32. If it is DateTime, we can use its timezone for calculations.
const TimezoneMixin * timezone = dynamic_cast<const TimezoneMixin *>(columns_definition.time_column_type.get());
const DateLUTImpl & date_lut = timezone ? timezone->getTimeZone() : DateLUT::instance();
/// Take rows in needed order and put them into `merged_data` until we get `max_block_size` rows.
///

View File

@ -35,6 +35,8 @@ public:
size_t value_column_num;
size_t version_column_num;
DataTypePtr time_column_type;
/// All columns other than 'time', 'value', 'version'. They are unmodified during rollup.
ColumnNumbers unmodified_column_numbers;
};

View File

@ -480,7 +480,10 @@ static StoragePtr create(const StorageFactory::Arguments & args)
"No replica name in config" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::NO_REPLICA_NAME_GIVEN);
++arg_num;
}
else if (is_extended_storage_def && (arg_cnt == 0 || !engine_args[arg_num]->as<ASTLiteral>() || (arg_cnt == 1 && merging_params.mode == MergeTreeData::MergingParams::Graphite)))
else if (is_extended_storage_def
&& (arg_cnt == 0
|| !engine_args[arg_num]->as<ASTLiteral>()
|| (arg_cnt == 1 && merging_params.mode == MergeTreeData::MergingParams::Graphite)))
{
/// Try use default values if arguments are not specified.
/// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic.

View File

@ -15,27 +15,27 @@ SELECT dateDiff('week', toDate('2017-12-31'), toDate('2018-01-01'));
SELECT dateDiff('day', toDate('2017-12-31'), toDate('2016-01-01'));
SELECT dateDiff('day', toDate('2017-12-31'), toDate('2017-01-01'));
SELECT dateDiff('day', toDate('2017-12-31'), toDate('2018-01-01'));
SELECT dateDiff('hour', toDate('2017-12-31'), toDate('2016-01-01'));
SELECT dateDiff('hour', toDate('2017-12-31'), toDate('2017-01-01'));
SELECT dateDiff('hour', toDate('2017-12-31'), toDate('2018-01-01'));
SELECT dateDiff('minute', toDate('2017-12-31'), toDate('2016-01-01'));
SELECT dateDiff('minute', toDate('2017-12-31'), toDate('2017-01-01'));
SELECT dateDiff('minute', toDate('2017-12-31'), toDate('2018-01-01'));
SELECT dateDiff('second', toDate('2017-12-31'), toDate('2016-01-01'));
SELECT dateDiff('second', toDate('2017-12-31'), toDate('2017-01-01'));
SELECT dateDiff('second', toDate('2017-12-31'), toDate('2018-01-01'));
SELECT dateDiff('hour', toDate('2017-12-31'), toDate('2016-01-01'), 'UTC');
SELECT dateDiff('hour', toDate('2017-12-31'), toDate('2017-01-01'), 'UTC');
SELECT dateDiff('hour', toDate('2017-12-31'), toDate('2018-01-01'), 'UTC');
SELECT dateDiff('minute', toDate('2017-12-31'), toDate('2016-01-01'), 'UTC');
SELECT dateDiff('minute', toDate('2017-12-31'), toDate('2017-01-01'), 'UTC');
SELECT dateDiff('minute', toDate('2017-12-31'), toDate('2018-01-01'), 'UTC');
SELECT dateDiff('second', toDate('2017-12-31'), toDate('2016-01-01'), 'UTC');
SELECT dateDiff('second', toDate('2017-12-31'), toDate('2017-01-01'), 'UTC');
SELECT dateDiff('second', toDate('2017-12-31'), toDate('2018-01-01'), 'UTC');
SELECT 'Date and DateTime arguments';
SELECT dateDiff('second', toDate('2017-12-31'), toDateTime('2016-01-01 00:00:00'));
SELECT dateDiff('second', toDateTime('2017-12-31 00:00:00'), toDate('2017-01-01'));
SELECT dateDiff('second', toDateTime('2017-12-31 00:00:00'), toDateTime('2018-01-01 00:00:00'));
SELECT dateDiff('second', toDate('2017-12-31'), toDateTime('2016-01-01 00:00:00', 'UTC'), 'UTC');
SELECT dateDiff('second', toDateTime('2017-12-31 00:00:00', 'UTC'), toDate('2017-01-01'), 'UTC');
SELECT dateDiff('second', toDateTime('2017-12-31 00:00:00', 'UTC'), toDateTime('2018-01-01 00:00:00', 'UTC'));
SELECT 'Constant and non-constant arguments';
SELECT dateDiff('minute', materialize(toDate('2017-12-31')), toDate('2016-01-01'));
SELECT dateDiff('minute', toDate('2017-12-31'), materialize(toDate('2017-01-01')));
SELECT dateDiff('minute', materialize(toDate('2017-12-31')), materialize(toDate('2018-01-01')));
SELECT dateDiff('minute', materialize(toDate('2017-12-31')), toDate('2016-01-01'), 'UTC');
SELECT dateDiff('minute', toDate('2017-12-31'), materialize(toDate('2017-01-01')), 'UTC');
SELECT dateDiff('minute', materialize(toDate('2017-12-31')), materialize(toDate('2018-01-01')), 'UTC');
SELECT 'Case insensitive';

View File

@ -5,4 +5,7 @@ WITH toDateTime(1 + rand() % 0xFFFFFFFF) AS t SELECT count() FROM numbers(100000
WITH toDateTime(1 + rand() % 0xFFFFFFFF) AS t SELECT count() FROM numbers(1000000) WHERE formatDateTime(t, '%F %R:%S') != toString(t);
WITH toDate(today() + rand() % 4096) AS t SELECT count() FROM numbers(1000000) WHERE formatDateTime(t, '%F') != toString(t);
WITH toDate(today() + rand() % 4096) AS t SELECT count() FROM numbers(1000000) WHERE formatDateTime(t, '%F %T') != toString(toDateTime(t));
-- Note: in some other timezones, daylight saving time change happens in midnight, so the first time of day is 01:00:00 instead of 00:00:00.
-- Stick to Moscow timezone to avoid this issue.
WITH toDate(today() + rand() % 4096) AS t SELECT count() FROM numbers(1000000) WHERE formatDateTime(t, '%F %T', 'Europe/Moscow') != toString(toDateTime(t, 'Europe/Moscow'));

View File

@ -27,7 +27,7 @@ CREATE TABLE persons_00825 (uuid UUID,
photo Nullable(String),
phoneNumber Nullable(FixedString(13)),
isOnline UInt8,
visitTime Nullable(DateTime),
visitTime Nullable(DateTime('Europe/Moscow')),
age UInt8,
zodiacSign Enum16('aries'=321, 'taurus'=420, 'gemini'=521, 'cancer'=621, 'leo'=723, 'virgo'=823,
'libra'=923, 'scorpius'=1023, 'sagittarius'=1122, 'capricorn'=1222, 'aquarius'=120,
@ -43,12 +43,12 @@ CREATE TABLE persons_00825 (uuid UUID,
randomBigNumber Int64,
measureUnits Nested(unit String, coef Float32),
nestiness_a_b_c_d Nullable(UInt32),
\`nestiness_a_B.c_E\` Array(UInt32)
"nestiness_a_B.c_E" Array(UInt32)
) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO persons_00825 VALUES (toUUID('a7522158-3d41-4b77-ad69-6c598ee55c49'), 'Ivan', 'Petrov', 'male', toDate('1980-12-29'), 'png', '+74951234567', 1, toDateTime('2019-01-05 18:45:00'), 38, 'capricorn', ['Yesterday', 'Flowers'], [255, 0, 0], 'Moscow', [55.753215, 37.622504], 3.14, 214.10, 0.1, 5.8, 17060000000, ['meter', 'centimeter', 'kilometer'], [1, 0.01, 1000], 500, [501, 502]);
INSERT INTO persons_00825 VALUES (toUUID('a7522158-3d41-4b77-ad69-6c598ee55c49'), 'Ivan', 'Petrov', 'male', toDate('1980-12-29'), 'png', '+74951234567', 1, toDateTime('2019-01-05 18:45:00', 'Europe/Moscow'), 38, 'capricorn', ['Yesterday', 'Flowers'], [255, 0, 0], 'Moscow', [55.753215, 37.622504], 3.14, 214.10, 0.1, 5.8, 17060000000, ['meter', 'centimeter', 'kilometer'], [1, 0.01, 1000], 500, [501, 502]);
INSERT INTO persons_00825 VALUES (toUUID('c694ad8a-f714-4ea3-907d-fd54fb25d9b5'), 'Natalia', 'Sokolova', 'female', toDate('1992-03-08'), 'jpg', NULL, 0, NULL, 26, 'pisces', [], [100, 200, 50], 'Plymouth', [50.403724, -4.142123], 3.14159, NULL, 0.007, 5.4, -20000000000000, [], [], NULL, []);
INSERT INTO persons_00825 VALUES (toUUID('a7da1aa6-f425-4789-8947-b034786ed374'), 'Vasily', 'Sidorov', 'male', toDate('1995-07-28'), 'bmp', '+442012345678', 1, toDateTime('2018-12-30 00:00:00'), 23, 'leo', ['Sunny'], [250, 244, 10], 'Murmansk', [68.970682, 33.074981], 3.14159265358979, 100000000000, 800, -3.2, 154400000, ['pound'], [16], 503, []);
INSERT INTO persons_00825 VALUES (toUUID('a7da1aa6-f425-4789-8947-b034786ed374'), 'Vasily', 'Sidorov', 'male', toDate('1995-07-28'), 'bmp', '+442012345678', 1, toDateTime('2018-12-30 00:00:00', 'Europe/Moscow'), 23, 'leo', ['Sunny'], [250, 244, 10], 'Murmansk', [68.970682, 33.074981], 3.14159265358979, 100000000000, 800, -3.2, 154400000, ['pound'], [16], 503, []);
SELECT * FROM persons_00825 ORDER BY name;
EOF

View File

@ -43,7 +43,7 @@ converted:
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06
diff:
dest:
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 1970-01-01 06:29:04
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 00:00:00
80 81 82 83 84 85 86 87 88 89 str02 fstr2\0\0\0\0\0\0\0\0\0\0 2005-03-04 2006-08-09 10:11:12
min:
-128 0 0 0 0 0 0 0 -1 -1 string-1\0\0\0\0\0\0\0 fixedstring-1\0\0 2003-04-05 2003-02-03
@ -51,10 +51,10 @@ min:
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06
127 -1 -1 -1 -1 -1 -1 -1 -1 -1 string-2\0\0\0\0\0\0\0 fixedstring-2\0\0 2004-06-07 2004-02-03
max:
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 1970-01-01 06:22:27 2003-02-03 04:05:06
-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 1970-01-01 06:09:16 2002-02-03 04:05:06
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 2003-04-05 00:00:00 2003-02-03 04:05:06
-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 2001-02-03 00:00:00 2002-02-03 04:05:06
80 81 82 83 84 85 86 87 88 89 str02 fstr2 2005-03-04 05:06:07 2006-08-09 10:11:12
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2\0\0 1970-01-01 06:29:36 2004-02-03 04:05:06
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2\0\0 2004-06-07 00:00:00 2004-02-03 04:05:06
dest from null:
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06

View File

@ -6,15 +6,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
#${CLICKHOUSE_CLIENT} --max_block_size=1 --query="SELECT * FROM system.numbers LIMIT 10 FORMAT Parquet" > ${CLICKHOUSE_TMP}/t1.pq
#${CLICKHOUSE_CLIENT} --max_block_size=5 --query="SELECT * FROM system.numbers LIMIT 10 FORMAT Parquet" > ${CLICKHOUSE_TMP}/t5.pq
#${CLICKHOUSE_CLIENT} --max_block_size=15 --query="SELECT * FROM system.numbers LIMIT 10 FORMAT Parquet" > ${CLICKHOUSE_TMP}/t15.pq
#${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 100000 FORMAT Parquet" > ${CLICKHOUSE_TMP}/t100000.pq
#${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 1000000000 FORMAT Parquet" > ${CLICKHOUSE_TMP}/t1g.pq
#${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 100000000 FORMAT Parquet" > ${CLICKHOUSE_TMP}/t100m.pq
#${CLICKHOUSE_CLIENT} --max_block_size=100000000 --query="SELECT * FROM system.numbers LIMIT 100000000 FORMAT Parquet" > ${CLICKHOUSE_TMP}/t100m-100mbs.pq
#valgrind --tool=massif ${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 1000000 FORMAT Parquet" > ${CLICKHOUSE_TMP}/t1g.pq
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS contributors"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE contributors (name String) ENGINE = Memory"
@ -35,11 +26,6 @@ ${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 100000 FORMAT P
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10"
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_numbers"
#${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 10000000 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet"
#${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10"
#${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_numbers"
#${CLICKHOUSE_CLIENT} --max_block_size=2 --query="SELECT * FROM system.numbers LIMIT 3 FORMAT Parquet" > ${CLICKHOUSE_TMP}/bs2.pq
${CLICKHOUSE_CLIENT} --max_block_size=2 --query="SELECT * FROM system.numbers LIMIT 3 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10"
@ -55,16 +41,6 @@ ${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.events FORMAT Parquet" | ${CL
${CLICKHOUSE_CLIENT} --query="SELECT event, description FROM parquet_events WHERE event IN ('ContextLock', 'Query') ORDER BY event"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_events"
#${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types1"
#${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types2"
#${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types1 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String ) ENGINE = Memory"
#${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types2 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String ) ENGINE = Memory"
#${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( -108, 108, -1016, 1116, -1032, -1064, 1164, -1.032, -1.064, 'string' )"
#${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types1"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types2"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types3"
@ -84,8 +60,6 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( -128,
# max
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06')"
# 'SELECT -127,-128,-129,126,127,128,255,256,257,-32767,-32768,-32769,32766,32767,32768,65535,65536,65537, -2147483647,-2147483648,-2147483649,2147483646,2147483647,2147483648,4294967295,4294967296,4294967297, -9223372036854775807,-9223372036854775808,9223372036854775806,9223372036854775807,9223372036854775808,18446744073709551615';
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet"
echo original:
@ -121,7 +95,6 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types5 (int8 Nullable(I
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime)) ENGINE = Memory"
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types5 ORDER BY int8 FORMAT Parquet" > "${CLICKHOUSE_TMP}"/parquet_all_types_5.parquet
#${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types5 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types6 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types5 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types6 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types6 FORMAT Parquet"
echo dest from null:
@ -140,7 +113,7 @@ ${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types4"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_arrays"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_arrays (id UInt32, a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(Datetime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) engine=Memory()"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_arrays (id UInt32, a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(DateTime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) engine=Memory()"
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays VALUES (1, [1,-2,3], [1,2,3], [100, -200, 300], [100, 200, 300], [10000000, -20000000, 30000000], [10000000, 2000000, 3000000], [100000000000000, -200000000000, 3000000000000], [100000000000000, 20000000000000, 3000000000000], ['Some string', 'Some string', 'Some string'], ['0000', '1111', '2222'], [42.42, 424.2, 0.4242], [424242.424242, 4242042420.242424, 42], ['2000-01-01', '2001-01-01', '2002-01-01'], ['2000-01-01', '2001-01-01', '2002-01-01'], [0.2, 10.003, 4.002], [4.000000001, 10000.10000, 10000.100001], [1000000000.000000001123, 90.0000000010010101, 0101001.0112341001])"

View File

@ -9,69 +9,69 @@ import argparse
# Create SQL statement to verify dateTime64 is accepted as argument to functions taking DateTime.
FUNCTIONS="""
toTimeZone(N, 'UTC')
toYear(N)
toQuarter(N)
toMonth(N)
toDayOfYear(N)
toDayOfMonth(N)
toDayOfWeek(N)
toHour(N)
toMinute(N)
toSecond(N)
toYear(N, 'Europe/Moscow')
toQuarter(N, 'Europe/Moscow')
toMonth(N, 'Europe/Moscow')
toDayOfYear(N, 'Europe/Moscow')
toDayOfMonth(N, 'Europe/Moscow')
toDayOfWeek(N, 'Europe/Moscow')
toHour(N, 'Europe/Moscow')
toMinute(N, 'Europe/Moscow')
toSecond(N, 'Europe/Moscow')
toUnixTimestamp(N)
toStartOfYear(N)
toStartOfISOYear(N)
toStartOfQuarter(N)
toStartOfMonth(N)
toMonday(N)
toStartOfWeek(N)
toStartOfDay(N)
toStartOfHour(N)
toStartOfMinute(N)
toStartOfFiveMinute(N)
toStartOfTenMinutes(N)
toStartOfFifteenMinutes(N)
toStartOfInterval(N, INTERVAL 1 year)
toStartOfInterval(N, INTERVAL 1 month)
toStartOfInterval(N, INTERVAL 1 day)
toStartOfInterval(N, INTERVAL 15 minute)
date_trunc('year', N)
date_trunc('month', N)
date_trunc('day', N)
date_trunc('minute', N)
toTime(N)
toRelativeYearNum(N)
toRelativeQuarterNum(N)
toRelativeMonthNum(N)
toRelativeWeekNum(N)
toRelativeDayNum(N)
toRelativeHourNum(N)
toRelativeMinuteNum(N)
toRelativeSecondNum(N)
toISOYear(N)
toISOWeek(N)
toWeek(N)
toYearWeek(N)
timeSlot(N)
toYYYYMM(N)
toYYYYMMDD(N)
toYYYYMMDDhhmmss(N)
addYears(N, 1)
addMonths(N, 1)
addWeeks(N, 1)
addDays(N, 1)
addHours(N, 1)
addMinutes(N, 1)
addSeconds(N, 1)
addQuarters(N, 1)
subtractYears(N, 1)
subtractMonths(N, 1)
subtractWeeks(N, 1)
subtractDays(N, 1)
subtractHours(N, 1)
subtractMinutes(N, 1)
subtractSeconds(N, 1)
subtractQuarters(N, 1)
toStartOfYear(N, 'Europe/Moscow')
toStartOfISOYear(N, 'Europe/Moscow')
toStartOfQuarter(N, 'Europe/Moscow')
toStartOfMonth(N, 'Europe/Moscow')
toMonday(N, 'Europe/Moscow')
toStartOfWeek(N, 'Europe/Moscow')
toStartOfDay(N, 'Europe/Moscow')
toStartOfHour(N, 'Europe/Moscow')
toStartOfMinute(N, 'Europe/Moscow')
toStartOfFiveMinute(N, 'Europe/Moscow')
toStartOfTenMinutes(N, 'Europe/Moscow')
toStartOfFifteenMinutes(N, 'Europe/Moscow')
toStartOfInterval(N, INTERVAL 1 year, 'Europe/Moscow')
toStartOfInterval(N, INTERVAL 1 month, 'Europe/Moscow')
toStartOfInterval(N, INTERVAL 1 day, 'Europe/Moscow')
toStartOfInterval(N, INTERVAL 15 minute, 'Europe/Moscow')
date_trunc('year', N, 'Europe/Moscow')
date_trunc('month', N, 'Europe/Moscow')
date_trunc('day', N, 'Europe/Moscow')
date_trunc('minute', N, 'Europe/Moscow')
toTime(N, 'Europe/Moscow')
toRelativeYearNum(N, 'Europe/Moscow')
toRelativeQuarterNum(N, 'Europe/Moscow')
toRelativeMonthNum(N, 'Europe/Moscow')
toRelativeWeekNum(N, 'Europe/Moscow')
toRelativeDayNum(N, 'Europe/Moscow')
toRelativeHourNum(N, 'Europe/Moscow')
toRelativeMinuteNum(N, 'Europe/Moscow')
toRelativeSecondNum(N, 'Europe/Moscow')
toISOYear(N, 'Europe/Moscow')
toISOWeek(N, 'Europe/Moscow')
toWeek(N, 'Europe/Moscow')
toYearWeek(N, 'Europe/Moscow')
timeSlot(N, 'Europe/Moscow')
toYYYYMM(N, 'Europe/Moscow')
toYYYYMMDD(N, 'Europe/Moscow')
toYYYYMMDDhhmmss(N, 'Europe/Moscow')
addYears(N, 1, 'Europe/Moscow')
addMonths(N, 1, 'Europe/Moscow')
addWeeks(N, 1, 'Europe/Moscow')
addDays(N, 1, 'Europe/Moscow')
addHours(N, 1, 'Europe/Moscow')
addMinutes(N, 1, 'Europe/Moscow')
addSeconds(N, 1, 'Europe/Moscow')
addQuarters(N, 1, 'Europe/Moscow')
subtractYears(N, 1, 'Europe/Moscow')
subtractMonths(N, 1, 'Europe/Moscow')
subtractWeeks(N, 1, 'Europe/Moscow')
subtractDays(N, 1, 'Europe/Moscow')
subtractHours(N, 1, 'Europe/Moscow')
subtractMinutes(N, 1, 'Europe/Moscow')
subtractSeconds(N, 1, 'Europe/Moscow')
subtractQuarters(N, 1, 'Europe/Moscow')
CAST(N as DateTime('Europe/Minsk'))
CAST(N as Date)
CAST(N as UInt64)
@ -80,10 +80,10 @@ CAST(N as DateTime64(3, 'Europe/Minsk'))
CAST(N as DateTime64(6, 'Europe/Minsk'))
CAST(N as DateTime64(9, 'Europe/Minsk'))
# Casting our test values to DateTime(12) will cause an overflow and hence will fail the test under UB sanitizer.
# CAST(N as DateTime64(12))
# CAST(N as DateTime64(12, 'Europe/Moscow'))
# DateTime64(18) will always fail due to zero precision, but it is Ok to test here:
# CAST(N as DateTime64(18))
formatDateTime(N, '%C %d %D %e %F %H %I %j %m %M %p %R %S %T %u %V %w %y %Y %%')
# CAST(N as DateTime64(18, 'Europe/Moscow'))
formatDateTime(N, '%C %d %D %e %F %H %I %j %m %M %p %R %S %T %u %V %w %y %Y %%', 'Europe/Moscow')
""".splitlines()
# Expanded later to cartesian product of all arguments, using format string.

View File

@ -3,47 +3,47 @@ Code: 43
"DateTime('UTC')","2019-09-16 16:20:11"
"DateTime64(3, 'UTC')","2019-09-16 16:20:11.234"
------------------------------------------
SELECT toYear(N)
SELECT toYear(N, \'Europe/Moscow\')
"UInt16",2019
"UInt16",2019
"UInt16",2019
------------------------------------------
SELECT toQuarter(N)
SELECT toQuarter(N, \'Europe/Moscow\')
"UInt8",3
"UInt8",3
"UInt8",3
------------------------------------------
SELECT toMonth(N)
SELECT toMonth(N, \'Europe/Moscow\')
"UInt8",9
"UInt8",9
"UInt8",9
------------------------------------------
SELECT toDayOfYear(N)
SELECT toDayOfYear(N, \'Europe/Moscow\')
"UInt16",259
"UInt16",259
"UInt16",259
------------------------------------------
SELECT toDayOfMonth(N)
SELECT toDayOfMonth(N, \'Europe/Moscow\')
"UInt8",16
"UInt8",16
"UInt8",16
------------------------------------------
SELECT toDayOfWeek(N)
SELECT toDayOfWeek(N, \'Europe/Moscow\')
"UInt8",1
"UInt8",1
"UInt8",1
------------------------------------------
SELECT toHour(N)
SELECT toHour(N, \'Europe/Moscow\')
Code: 43
"UInt8",19
"UInt8",19
------------------------------------------
SELECT toMinute(N)
SELECT toMinute(N, \'Europe/Moscow\')
Code: 43
"UInt8",20
"UInt8",20
------------------------------------------
SELECT toSecond(N)
SELECT toSecond(N, \'Europe/Moscow\')
Code: 43
"UInt8",11
"UInt8",11
@ -53,270 +53,270 @@ Code: 44
"UInt32",1568650811
"UInt32",1568650811
------------------------------------------
SELECT toStartOfYear(N)
"Date","2019-01-01"
SELECT toStartOfYear(N, \'Europe/Moscow\')
Code: 43
"Date","2019-01-01"
"Date","2019-01-01"
------------------------------------------
SELECT toStartOfISOYear(N)
"Date","2018-12-31"
SELECT toStartOfISOYear(N, \'Europe/Moscow\')
Code: 43
"Date","2018-12-31"
"Date","2018-12-31"
------------------------------------------
SELECT toStartOfQuarter(N)
"Date","2019-07-01"
SELECT toStartOfQuarter(N, \'Europe/Moscow\')
Code: 43
"Date","2019-07-01"
"Date","2019-07-01"
------------------------------------------
SELECT toStartOfMonth(N)
"Date","2019-09-01"
SELECT toStartOfMonth(N, \'Europe/Moscow\')
Code: 43
"Date","2019-09-01"
"Date","2019-09-01"
------------------------------------------
SELECT toMonday(N)
"Date","2019-09-16"
SELECT toMonday(N, \'Europe/Moscow\')
Code: 43
"Date","2019-09-16"
"Date","2019-09-16"
------------------------------------------
SELECT toStartOfWeek(N)
"Date","2019-09-15"
"Date","2019-09-15"
"Date","2019-09-15"
------------------------------------------
SELECT toStartOfDay(N)
"DateTime","2019-09-16 00:00:00"
"DateTime('Europe/Minsk')","2019-09-16 00:00:00"
"DateTime('Europe/Minsk')","2019-09-16 00:00:00"
------------------------------------------
SELECT toStartOfHour(N)
SELECT toStartOfWeek(N, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:00:00"
"DateTime('Europe/Minsk')","2019-09-16 19:00:00"
------------------------------------------
SELECT toStartOfMinute(N)
Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:20:00"
"DateTime('Europe/Minsk')","2019-09-16 19:20:00"
------------------------------------------
SELECT toStartOfFiveMinute(N)
Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:20:00"
"DateTime('Europe/Minsk')","2019-09-16 19:20:00"
------------------------------------------
SELECT toStartOfTenMinutes(N)
SELECT toStartOfDay(N, \'Europe/Moscow\')
"DateTime('Europe/Moscow')","2019-09-16 00:00:00"
"DateTime('Europe/Moscow')","2019-09-16 00:00:00"
"DateTime('Europe/Moscow')","2019-09-16 00:00:00"
------------------------------------------
SELECT toStartOfHour(N, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:20:00"
"DateTime('Europe/Minsk')","2019-09-16 19:20:00"
"DateTime('Europe/Moscow')","2019-09-16 19:00:00"
"DateTime('Europe/Moscow')","2019-09-16 19:00:00"
------------------------------------------
SELECT toStartOfFifteenMinutes(N)
SELECT toStartOfMinute(N, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:15:00"
"DateTime('Europe/Minsk')","2019-09-16 19:15:00"
"DateTime('Europe/Moscow')","2019-09-16 19:20:00"
"DateTime('Europe/Moscow')","2019-09-16 19:20:00"
------------------------------------------
SELECT toStartOfInterval(N, INTERVAL 1 year)
"Date","2019-01-01"
SELECT toStartOfFiveMinute(N, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-16 19:20:00"
"DateTime('Europe/Moscow')","2019-09-16 19:20:00"
------------------------------------------
SELECT toStartOfTenMinutes(N, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-16 19:20:00"
"DateTime('Europe/Moscow')","2019-09-16 19:20:00"
------------------------------------------
SELECT toStartOfFifteenMinutes(N, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-16 19:15:00"
"DateTime('Europe/Moscow')","2019-09-16 19:15:00"
------------------------------------------
SELECT toStartOfInterval(N, INTERVAL 1 year, \'Europe/Moscow\')
Code: 43
"Date","2019-01-01"
"Date","2019-01-01"
------------------------------------------
SELECT toStartOfInterval(N, INTERVAL 1 month)
"Date","2019-09-01"
"Date","2019-09-01"
"Date","2019-09-01"
------------------------------------------
SELECT toStartOfInterval(N, INTERVAL 1 day)
"DateTime","2019-09-16 00:00:00"
"DateTime('Europe/Minsk')","2019-09-16 00:00:00"
"DateTime('Europe/Minsk')","2019-09-16 00:00:00"
------------------------------------------
SELECT toStartOfInterval(N, INTERVAL 15 minute)
SELECT toStartOfInterval(N, INTERVAL 1 month, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:15:00"
"DateTime('Europe/Minsk')","2019-09-16 19:15:00"
------------------------------------------
SELECT date_trunc(\'year\', N)
"Date","2019-01-01"
"Date","2019-01-01"
"Date","2019-01-01"
------------------------------------------
SELECT date_trunc(\'month\', N)
"Date","2019-09-01"
"Date","2019-09-01"
"Date","2019-09-01"
------------------------------------------
SELECT date_trunc(\'day\', N)
"DateTime","2019-09-16 00:00:00"
"DateTime('Europe/Minsk')","2019-09-16 00:00:00"
"DateTime('Europe/Minsk')","2019-09-16 00:00:00"
SELECT toStartOfInterval(N, INTERVAL 1 day, \'Europe/Moscow\')
"DateTime('Europe/Moscow')","2019-09-16 00:00:00"
"DateTime('Europe/Moscow')","2019-09-16 00:00:00"
"DateTime('Europe/Moscow')","2019-09-16 00:00:00"
------------------------------------------
SELECT date_trunc(\'minute\', N)
SELECT toStartOfInterval(N, INTERVAL 15 minute, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:20:00"
"DateTime('Europe/Minsk')","2019-09-16 19:20:00"
"DateTime('Europe/Moscow')","2019-09-16 19:15:00"
"DateTime('Europe/Moscow')","2019-09-16 19:15:00"
------------------------------------------
SELECT toTime(N)
SELECT date_trunc(\'year\', N, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Minsk')","1970-01-02 19:20:11"
"DateTime('Europe/Minsk')","1970-01-02 19:20:11"
"Date","2019-01-01"
"Date","2019-01-01"
------------------------------------------
SELECT toRelativeYearNum(N)
SELECT date_trunc(\'month\', N, \'Europe/Moscow\')
Code: 43
"Date","2019-09-01"
"Date","2019-09-01"
------------------------------------------
SELECT date_trunc(\'day\', N, \'Europe/Moscow\')
"DateTime('Europe/Moscow')","2019-09-16 00:00:00"
"DateTime('Europe/Moscow')","2019-09-16 00:00:00"
"DateTime('Europe/Moscow')","2019-09-16 00:00:00"
------------------------------------------
SELECT date_trunc(\'minute\', N, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-16 19:20:00"
"DateTime('Europe/Moscow')","2019-09-16 19:20:00"
------------------------------------------
SELECT toTime(N, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","1970-01-02 19:20:11"
"DateTime('Europe/Moscow')","1970-01-02 19:20:11"
------------------------------------------
SELECT toRelativeYearNum(N, \'Europe/Moscow\')
"UInt16",2019
"UInt16",2019
"UInt16",2019
------------------------------------------
SELECT toRelativeQuarterNum(N)
SELECT toRelativeQuarterNum(N, \'Europe/Moscow\')
"UInt32",8078
"UInt32",8078
"UInt32",8078
------------------------------------------
SELECT toRelativeMonthNum(N)
SELECT toRelativeMonthNum(N, \'Europe/Moscow\')
"UInt32",24237
"UInt32",24237
"UInt32",24237
------------------------------------------
SELECT toRelativeWeekNum(N)
SELECT toRelativeWeekNum(N, \'Europe/Moscow\')
"UInt32",2594
"UInt32",2594
"UInt32",2594
------------------------------------------
SELECT toRelativeDayNum(N)
SELECT toRelativeDayNum(N, \'Europe/Moscow\')
"UInt32",18155
"UInt32",18155
"UInt32",18155
------------------------------------------
SELECT toRelativeHourNum(N)
SELECT toRelativeHourNum(N, \'Europe/Moscow\')
"UInt32",435717
"UInt32",435736
"UInt32",435736
------------------------------------------
SELECT toRelativeMinuteNum(N)
SELECT toRelativeMinuteNum(N, \'Europe/Moscow\')
"UInt32",26143020
"UInt32",26144180
"UInt32",26144180
------------------------------------------
SELECT toRelativeSecondNum(N)
SELECT toRelativeSecondNum(N, \'Europe/Moscow\')
"UInt32",1568581200
"UInt32",1568650811
"UInt32",1568650811
------------------------------------------
SELECT toISOYear(N)
SELECT toISOYear(N, \'Europe/Moscow\')
"UInt16",2019
"UInt16",2019
"UInt16",2019
------------------------------------------
SELECT toISOWeek(N)
SELECT toISOWeek(N, \'Europe/Moscow\')
"UInt8",38
"UInt8",38
"UInt8",38
------------------------------------------
SELECT toWeek(N)
"UInt8",37
"UInt8",37
"UInt8",37
------------------------------------------
SELECT toYearWeek(N)
"UInt32",201937
"UInt32",201937
"UInt32",201937
------------------------------------------
SELECT timeSlot(N)
SELECT toWeek(N, \'Europe/Moscow\')
Code: 43
Code: 43
Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:00:00"
"DateTime('Europe/Minsk')","2019-09-16 19:00:00"
------------------------------------------
SELECT toYYYYMM(N)
SELECT toYearWeek(N, \'Europe/Moscow\')
Code: 43
Code: 43
Code: 43
------------------------------------------
SELECT timeSlot(N, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-16 19:00:00"
"DateTime('Europe/Moscow')","2019-09-16 19:00:00"
------------------------------------------
SELECT toYYYYMM(N, \'Europe/Moscow\')
"UInt32",201909
"UInt32",201909
"UInt32",201909
------------------------------------------
SELECT toYYYYMMDD(N)
SELECT toYYYYMMDD(N, \'Europe/Moscow\')
"UInt32",20190916
"UInt32",20190916
"UInt32",20190916
------------------------------------------
SELECT toYYYYMMDDhhmmss(N)
SELECT toYYYYMMDDhhmmss(N, \'Europe/Moscow\')
"UInt64",20190916000000
"UInt64",20190916192011
"UInt64",20190916192011
------------------------------------------
SELECT addYears(N, 1)
"Date","2020-09-16"
"DateTime('Europe/Minsk')","2020-09-16 19:20:11"
"DateTime64(3, 'Europe/Minsk')","2020-09-16 19:20:11.234"
SELECT addYears(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2020-09-16 19:20:11"
Code: 43
------------------------------------------
SELECT addMonths(N, 1)
"Date","2019-10-16"
"DateTime('Europe/Minsk')","2019-10-16 19:20:11"
"DateTime64(3, 'Europe/Minsk')","2019-10-16 19:20:11.234"
SELECT addMonths(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-10-16 19:20:11"
Code: 43
------------------------------------------
SELECT addWeeks(N, 1)
"Date","2019-09-23"
"DateTime('Europe/Minsk')","2019-09-23 19:20:11"
"DateTime64(3, 'Europe/Minsk')","2019-09-23 19:20:11.234"
SELECT addWeeks(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-23 19:20:11"
Code: 43
------------------------------------------
SELECT addDays(N, 1)
"Date","2019-09-17"
"DateTime('Europe/Minsk')","2019-09-17 19:20:11"
"DateTime64(3, 'Europe/Minsk')","2019-09-17 19:20:11.234"
SELECT addDays(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-17 19:20:11"
Code: 43
------------------------------------------
SELECT addHours(N, 1)
"DateTime","2019-09-16 01:00:00"
"DateTime('Europe/Minsk')","2019-09-16 20:20:11"
"DateTime64(3, 'Europe/Minsk')","2019-09-16 20:20:11.234"
SELECT addHours(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-16 20:20:11"
Code: 43
------------------------------------------
SELECT addMinutes(N, 1)
"DateTime","2019-09-16 00:01:00"
"DateTime('Europe/Minsk')","2019-09-16 19:21:11"
"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:21:11.234"
SELECT addMinutes(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-16 19:21:11"
Code: 43
------------------------------------------
SELECT addSeconds(N, 1)
"DateTime","2019-09-16 00:00:01"
"DateTime('Europe/Minsk')","2019-09-16 19:20:12"
"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234"
SELECT addSeconds(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-16 19:20:12"
Code: 43
------------------------------------------
SELECT addQuarters(N, 1)
"Date","2019-12-16"
"DateTime('Europe/Minsk')","2019-12-16 19:20:11"
"DateTime64(3, 'Europe/Minsk')","2019-12-16 19:20:11.234"
SELECT addQuarters(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-12-16 19:20:11"
Code: 43
------------------------------------------
SELECT subtractYears(N, 1)
"Date","2018-09-16"
"DateTime('Europe/Minsk')","2018-09-16 19:20:11"
"DateTime64(3, 'Europe/Minsk')","2018-09-16 19:20:11.234"
SELECT subtractYears(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2018-09-16 19:20:11"
Code: 43
------------------------------------------
SELECT subtractMonths(N, 1)
"Date","2019-08-16"
"DateTime('Europe/Minsk')","2019-08-16 19:20:11"
"DateTime64(3, 'Europe/Minsk')","2019-08-16 19:20:11.234"
SELECT subtractMonths(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-08-16 19:20:11"
Code: 43
------------------------------------------
SELECT subtractWeeks(N, 1)
"Date","2019-09-09"
"DateTime('Europe/Minsk')","2019-09-09 19:20:11"
"DateTime64(3, 'Europe/Minsk')","2019-09-09 19:20:11.234"
SELECT subtractWeeks(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-09 19:20:11"
Code: 43
------------------------------------------
SELECT subtractDays(N, 1)
"Date","2019-09-15"
"DateTime('Europe/Minsk')","2019-09-15 19:20:11"
"DateTime64(3, 'Europe/Minsk')","2019-09-15 19:20:11.234"
SELECT subtractDays(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-15 19:20:11"
Code: 43
------------------------------------------
SELECT subtractHours(N, 1)
"DateTime","2019-09-15 23:00:00"
"DateTime('Europe/Minsk')","2019-09-16 18:20:11"
"DateTime64(3, 'Europe/Minsk')","2019-09-16 18:20:11.234"
SELECT subtractHours(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-16 18:20:11"
Code: 43
------------------------------------------
SELECT subtractMinutes(N, 1)
"DateTime","2019-09-15 23:59:00"
"DateTime('Europe/Minsk')","2019-09-16 19:19:11"
"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:19:11.234"
SELECT subtractMinutes(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-16 19:19:11"
Code: 43
------------------------------------------
SELECT subtractSeconds(N, 1)
"DateTime","2019-09-15 23:59:59"
"DateTime('Europe/Minsk')","2019-09-16 19:20:10"
"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234"
SELECT subtractSeconds(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-09-16 19:20:10"
Code: 43
------------------------------------------
SELECT subtractQuarters(N, 1)
"Date","2019-06-16"
"DateTime('Europe/Minsk')","2019-06-16 19:20:11"
"DateTime64(3, 'Europe/Minsk')","2019-06-16 19:20:11.234"
SELECT subtractQuarters(N, 1, \'Europe/Moscow\')
Code: 43
"DateTime('Europe/Moscow')","2019-06-16 19:20:11"
Code: 43
------------------------------------------
SELECT CAST(N as DateTime(\'Europe/Minsk\'))
"DateTime('Europe/Minsk')","2019-09-16 00:00:00"
@ -353,7 +353,7 @@ SELECT CAST(N as DateTime64(9, \'Europe/Minsk\'))
"DateTime64(9, 'Europe/Minsk')","2019-09-16 19:20:11.000000000"
"DateTime64(9, 'Europe/Minsk')","2019-09-16 19:20:11.234000000"
------------------------------------------
SELECT formatDateTime(N, \'%C %d %D %e %F %H %I %j %m %M %p %R %S %T %u %V %w %y %Y %%\')
SELECT formatDateTime(N, \'%C %d %D %e %F %H %I %j %m %M %p %R %S %T %u %V %w %y %Y %%\', \'Europe/Moscow\')
"String","20 16 09/16/19 16 2019-09-16 00 12 259 09 00 AM 00:00 00 00:00:00 1 38 1 19 2019 %"
"String","20 16 09/16/19 16 2019-09-16 19 07 259 09 20 PM 19:20 11 19:20:11 1 38 1 19 2019 %"
"String","20 16 09/16/19 16 2019-09-16 19 07 259 09 20 PM 19:20 11 19:20:11 1 38 1 19 2019 %"

View File

@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
for typename in "UInt32" "UInt64" "Float64" "Float32" "DateTime" "Decimal32(5)" "Decimal64(5)" "Decimal128(5)" "DateTime64(3)"
for typename in "UInt32" "UInt64" "Float64" "Float32" "DateTime('Europe/Moscow')" "Decimal32(5)" "Decimal64(5)" "Decimal128(5)" "DateTime64(3, 'Europe/Moscow')"
do
$CLICKHOUSE_CLIENT -mn <<EOF
DROP TABLE IF EXISTS A;
@ -24,4 +24,4 @@ DROP TABLE A;
DROP TABLE B;
EOF
done
done

View File

@ -9,7 +9,7 @@ $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS movement"
$CLICKHOUSE_CLIENT -n --query "CREATE TABLE movement (date DateTime('Europe/Moscow')) Engine = MergeTree ORDER BY (toStartOfHour(date));"
$CLICKHOUSE_CLIENT --query "insert into movement select toDateTime('2020-01-22 00:00:00') + number%(23*3600) from numbers(1000000);"
$CLICKHOUSE_CLIENT --query "insert into movement select toDateTime('2020-01-22 00:00:00', 'Europe/Moscow') + number%(23*3600) from numbers(1000000);"
$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE movement FINAL"
@ -18,20 +18,20 @@ SELECT
count(),
toStartOfHour(date) AS Hour
FROM movement
WHERE (date >= toDateTime('2020-01-22T10:00:00')) AND (date <= toDateTime('2020-01-22T23:00:00'))
WHERE (date >= toDateTime('2020-01-22T10:00:00', 'Europe/Moscow')) AND (date <= toDateTime('2020-01-22T23:00:00', 'Europe/Moscow'))
GROUP BY Hour
ORDER BY Hour DESC
" | grep "16:00:00" | cut -f1
$CLICKHOUSE_CLIENT --query "alter table movement delete where date >= toDateTime('2020-01-22T16:00:00') and date < toDateTime('2020-01-22T17:00:00') SETTINGS mutations_sync = 2"
$CLICKHOUSE_CLIENT --query "alter table movement delete where date >= toDateTime('2020-01-22T16:00:00', 'Europe/Moscow') and date < toDateTime('2020-01-22T17:00:00', 'Europe/Moscow') SETTINGS mutations_sync = 2"
$CLICKHOUSE_CLIENT -n --query "
SELECT
count(),
toStartOfHour(date) AS Hour
FROM movement
WHERE (date >= toDateTime('2020-01-22T10:00:00')) AND (date <= toDateTime('2020-01-22T23:00:00'))
WHERE (date >= toDateTime('2020-01-22T10:00:00', 'Europe/Moscow')) AND (date <= toDateTime('2020-01-22T23:00:00', 'Europe/Moscow'))
GROUP BY Hour
ORDER BY Hour DESC
" | grep "16:00:00" | wc -l
@ -42,7 +42,7 @@ SELECT
count(),
toStartOfHour(date) AS Hour
FROM movement
WHERE (date >= toDateTime('2020-01-22T10:00:00')) AND (date <= toDateTime('2020-01-22T23:00:00'))
WHERE (date >= toDateTime('2020-01-22T10:00:00', 'Europe/Moscow')) AND (date <= toDateTime('2020-01-22T23:00:00', 'Europe/Moscow'))
GROUP BY Hour
ORDER BY Hour DESC
" | grep "22:00:00" | cut -f1

View File

@ -7,7 +7,7 @@ DROP TABLE IF EXISTS test_table;
SELECT '-';
DROP TABLE IF EXISTS test_table_2;
CREATE TABLE test_table_2(a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)) ENGINE=GenerateRandom(10, 5, 3);
CREATE TABLE test_table_2(a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3, 'Europe/Moscow'), UUID)) ENGINE=GenerateRandom(10, 5, 3);
SELECT * FROM test_table_2 LIMIT 100;

View File

@ -46,7 +46,7 @@ h
\N
o
-
Date DateTime DateTime(\'Europe/Moscow\')
Date DateTime(\'Europe/Moscow\') DateTime(\'Europe/Moscow\')
2113-06-12 2050-12-17 02:46:35 2096-02-16 22:18:22
2141-08-09 2013-10-17 23:35:26 1976-01-24 12:52:48
2039-08-16 1974-11-17 23:22:46 1980-03-04 21:02:50
@ -58,7 +58,7 @@ Date DateTime DateTime(\'Europe/Moscow\')
2008-03-16 2047-05-16 23:28:36 2103-02-11 16:44:39
2000-07-07 2105-07-19 19:29:06 1980-01-02 05:18:22
-
DateTime64(3) DateTime64(6) DateTime64(6, \'Europe/Moscow\')
DateTime64(3, \'Europe/Moscow\') DateTime64(6, \'Europe/Moscow\') DateTime64(6, \'Europe/Moscow\')
1978-06-07 23:50:57.320 2013-08-28 10:21:54.010758 1991-08-25 16:23:26.140215
1978-08-25 17:07:25.427 2034-05-02 20:49:42.148578 2015-08-26 15:26:31.783160
2037-04-04 10:50:56.898 2055-05-28 11:12:48.819271 2068-12-26 09:58:49.635722

View File

@ -42,20 +42,20 @@ LIMIT 10;
SELECT '-';
SELECT
toTypeName(d), toTypeName(dt), toTypeName(dtm)
FROM generateRandom('d Date, dt DateTime, dtm DateTime(\'Europe/Moscow\')')
FROM generateRandom('d Date, dt DateTime(\'Europe/Moscow\'), dtm DateTime(\'Europe/Moscow\')')
LIMIT 1;
SELECT
d, dt, dtm
FROM generateRandom('d Date, dt DateTime, dtm DateTime(\'Europe/Moscow\')', 1, 10, 10)
FROM generateRandom('d Date, dt DateTime(\'Europe/Moscow\'), dtm DateTime(\'Europe/Moscow\')', 1, 10, 10)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(dt64), toTypeName(dts64), toTypeName(dtms64)
FROM generateRandom('dt64 DateTime64, dts64 DateTime64(6), dtms64 DateTime64(6 ,\'Europe/Moscow\')')
FROM generateRandom('dt64 DateTime64(3, \'Europe/Moscow\'), dts64 DateTime64(6, \'Europe/Moscow\'), dtms64 DateTime64(6 ,\'Europe/Moscow\')')
LIMIT 1;
SELECT
dt64, dts64, dtms64
FROM generateRandom('dt64 DateTime64, dts64 DateTime64(6), dtms64 DateTime64(6 ,\'Europe/Moscow\')', 1, 10, 10)
FROM generateRandom('dt64 DateTime64(3, \'Europe/Moscow\'), dts64 DateTime64(6, \'Europe/Moscow\'), dtms64 DateTime64(6 ,\'Europe/Moscow\')', 1, 10, 10)
LIMIT 10;
SELECT '-';
SELECT
@ -168,8 +168,8 @@ FROM generateRandom('i String', 1, 10, 10)
LIMIT 10;
SELECT '-';
DROP TABLE IF EXISTS test_table;
CREATE TABLE test_table(a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)) ENGINE=Memory;
INSERT INTO test_table SELECT * FROM generateRandom('a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)', 1, 10, 2)
CREATE TABLE test_table(a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3, 'Europe/Moscow'), UUID)) ENGINE=Memory;
INSERT INTO test_table SELECT * FROM generateRandom('a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3, \'Europe/Moscow\'), UUID)', 1, 10, 2)
LIMIT 10;
SELECT * FROM test_table ORDER BY a, d, c;
@ -179,8 +179,8 @@ DROP TABLE IF EXISTS test_table;
SELECT '-';
DROP TABLE IF EXISTS test_table_2;
CREATE TABLE test_table_2(a Array(Int8), b UInt32, c Nullable(String), d Decimal32(4), e Nullable(Enum16('h' = 1, 'w' = 5 , 'o' = -200)), f Float64, g Tuple(Date, DateTime, DateTime64, UUID), h FixedString(2)) ENGINE=Memory;
INSERT INTO test_table_2 SELECT * FROM generateRandom('a Array(Int8), b UInt32, c Nullable(String), d Decimal32(4), e Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)), f Float64, g Tuple(Date, DateTime, DateTime64, UUID), h FixedString(2)', 10, 5, 3)
CREATE TABLE test_table_2(a Array(Int8), b UInt32, c Nullable(String), d Decimal32(4), e Nullable(Enum16('h' = 1, 'w' = 5 , 'o' = -200)), f Float64, g Tuple(Date, DateTime('Europe/Moscow'), DateTime64(3, 'Europe/Moscow'), UUID), h FixedString(2)) ENGINE=Memory;
INSERT INTO test_table_2 SELECT * FROM generateRandom('a Array(Int8), b UInt32, c Nullable(String), d Decimal32(4), e Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)), f Float64, g Tuple(Date, DateTime(\'Europe/Moscow\'), DateTime64(3, \'Europe/Moscow\'), UUID), h FixedString(2)', 10, 5, 3)
LIMIT 10;
SELECT a, b, c, d, e, f, g, hex(h) FROM test_table_2 ORDER BY a, b, c, d, e, f, g, h;

View File

@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS msgpack";
$CLICKHOUSE_CLIENT --query="CREATE TABLE msgpack (uint8 UInt8, uint16 UInt16, uint32 UInt32, uint64 UInt64, int8 Int8, int16 Int16, int32 Int32, int64 Int64, float Float32, double Float64, string String, date Date, datetime DateTime, datetime64 DateTime64, array Array(UInt32)) ENGINE = Memory";
$CLICKHOUSE_CLIENT --query="CREATE TABLE msgpack (uint8 UInt8, uint16 UInt16, uint32 UInt32, uint64 UInt64, int8 Int8, int16 Int16, int32 Int32, int64 Int64, float Float32, double Float64, string String, date Date, datetime DateTime('Europe/Moscow'), datetime64 DateTime64(3, 'Europe/Moscow'), array Array(UInt32)) ENGINE = Memory";
$CLICKHOUSE_CLIENT --query="INSERT INTO msgpack VALUES (255, 65535, 4294967295, 100000000000, -128, -32768, -2147483648, -100000000000, 2.02, 10000.0000001, 'String', 18980, 1639872000, 1639872000000, [1,2,3,4,5]), (4, 1234, 3244467295, 500000000000, -1, -256, -14741221, -7000000000, 100.1, 14321.032141201, 'Another string', 20000, 1839882000, 1639872891123, [5,4,3,2,1]), (42, 42, 42, 42, 42, 42, 42, 42, 42.42, 42.42, '42', 42, 42, 42, [42])";

View File

@ -14,7 +14,7 @@ select timestamp '2001-09-28 01:00:00' + interval 23 hour;
select timestamp '2001-09-28 23:00:00' - interval 23 hour;
-- TODO: return interval
select (timestamp '2001-09-29 03:00:00' - timestamp '2001-09-27 12:00:00') x, toTypeName(x); -- interval '1 day 15:00:00'
select (timestamp '2001-12-29 03:00:00' - timestamp '2001-12-27 12:00:00') x, toTypeName(x); -- interval '1 day 15:00:00'
-- select -interval 23 hour; -- interval '-23:00:00'
-- select interval 1 day + interval 1 hour; -- interval '1 day 01:00:00'

View File

@ -2,9 +2,9 @@ select toUInt8(x) from values('x Nullable(String)', '42', NULL, '0', '', '256');
select toInt64(x) from values('x Nullable(String)', '42', NULL, '0', '', '256');
select toDate(x) from values('x Nullable(String)', '2020-12-24', NULL, '0000-00-00', '', '9999-01-01');
select toDateTime(x) from values('x Nullable(String)', '2020-12-24 01:02:03', NULL, '0000-00-00 00:00:00', '');
select toDateTime64(x, 2) from values('x Nullable(String)', '2020-12-24 01:02:03', NULL, '0000-00-00 00:00:00', '');
select toUnixTimestamp(x) from values ('x Nullable(String)', '2000-01-01 13:12:12', NULL, '');
select toDateTime(x, 'Europe/Moscow') from values('x Nullable(String)', '2020-12-24 01:02:03', NULL, '0000-00-00 00:00:00', '');
select toDateTime64(x, 2, 'Europe/Moscow') from values('x Nullable(String)', '2020-12-24 01:02:03', NULL, '0000-00-00 00:00:00', '');
select toUnixTimestamp(x, 'Europe/Moscow') from values ('x Nullable(String)', '2000-01-01 13:12:12', NULL, '');
select toDecimal32(x, 2) from values ('x Nullable(String)', '42', NULL, '3.14159');
select toDecimal64(x, 8) from values ('x Nullable(String)', '42', NULL, '3.14159');

View File

@ -12,7 +12,7 @@ ENGINE = MergeTree()
PARTITION BY date
ORDER BY key;
INSERT INTO table_rename_with_default (date, key, value1) SELECT toDate('2019-10-01') + number % 3, number, toString(number) from numbers(9);
INSERT INTO table_rename_with_default (date, key, value1) SELECT toDateTime(toDate('2019-10-01') + number % 3, 'Europe/Moscow'), number, toString(number) from numbers(9);
SELECT * FROM table_rename_with_default WHERE key = 1 FORMAT TSVWithNames;
@ -42,7 +42,7 @@ ENGINE = ReplicatedMergeTree('/clickhouse/test_01213/table_rename_with_ttl', '1'
ORDER BY tuple()
TTL date2 + INTERVAL 10000 MONTH;
INSERT INTO table_rename_with_ttl SELECT toDate('2019-10-01') + number % 3, toDate('2018-10-01') + number % 3, toString(number), toString(number) from numbers(9);
INSERT INTO table_rename_with_ttl SELECT toDateTime(toDate('2019-10-01') + number % 3, 'Europe/Moscow'), toDateTime(toDate('2018-10-01') + number % 3, 'Europe/Moscow'), toString(number), toString(number) from numbers(9);
SELECT * FROM table_rename_with_ttl WHERE value1 = '1' FORMAT TSVWithNames;

View File

@ -1,5 +1,6 @@
drop table if exists test_graphite;
create table test_graphite (key UInt32, Path String, Time DateTime, Value Float64, Version UInt32, col UInt64) engine = GraphiteMergeTree('graphite_rollup') order by key settings index_granularity=10;
create table test_graphite (key UInt32, Path String, Time DateTime, Value Float64, Version UInt32, col UInt64)
engine = GraphiteMergeTree('graphite_rollup') order by key settings index_granularity=10;
insert into test_graphite
select 1, 'sum_1', toDateTime(today()) - number * 60 - 30, number, 1, number from numbers(300) union all
@ -21,7 +22,7 @@ select 2, 'max_1', toDateTime(today() - 3) - number * 60 - 30, number, 1, number
select 1, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200) union all
select 2, 'max_2', toDateTime(today() - 3) - number * 60 - 30, number, 1, number from numbers(1200);
optimize table test_graphite;
optimize table test_graphite final;
select key, Path, Value, Version, col from test_graphite order by key, Path, Time desc;

View File

@ -4,10 +4,10 @@ SELECT toStartOfSecond(now()); -- {serverError 43}
SELECT toStartOfSecond(); -- {serverError 42}
SELECT toStartOfSecond(now64(), 123); -- {serverError 43}
WITH toDateTime64('2019-09-16 19:20:11', 3) AS dt64 SELECT toStartOfSecond(dt64, 'UTC') AS res, toTypeName(res);
WITH toDateTime64('2019-09-16 19:20:11', 3, 'Europe/Moscow') AS dt64 SELECT toStartOfSecond(dt64, 'UTC') AS res, toTypeName(res);
WITH toDateTime64('2019-09-16 19:20:11', 0, 'UTC') AS dt64 SELECT toStartOfSecond(dt64) AS res, toTypeName(res);
WITH toDateTime64('2019-09-16 19:20:11.123', 3, 'UTC') AS dt64 SELECT toStartOfSecond(dt64) AS res, toTypeName(res);
WITH toDateTime64('2019-09-16 19:20:11.123', 9, 'UTC') AS dt64 SELECT toStartOfSecond(dt64) AS res, toTypeName(res);
SELECT 'non-const column';
WITH toDateTime64('2019-09-16 19:20:11.123', 3, 'UTC') AS dt64 SELECT toStartOfSecond(materialize(dt64)) AS res, toTypeName(res);
WITH toDateTime64('2019-09-16 19:20:11.123', 3, 'UTC') AS dt64 SELECT toStartOfSecond(materialize(dt64)) AS res, toTypeName(res);

View File

@ -41,7 +41,7 @@ converted:
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06
diff:
dest:
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 1970-01-01 06:29:04
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 00:00:00
80 81 82 83 84 85 86 87 88 89 str02 fstr2\0\0\0\0\0\0\0\0\0\0 2005-03-04 2006-08-09 10:11:12
min:
-128 0 0 0 0 0 0 0 -1 -1 string-1\0\0\0\0\0\0\0 fixedstring-1\0\0 2003-04-05 2003-02-03
@ -49,10 +49,10 @@ min:
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06
127 -1 -1 -1 -1 -1 -1 -1 -1 -1 string-2\0\0\0\0\0\0\0 fixedstring-2\0\0 2004-06-07 2004-02-03
max:
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 1970-01-01 06:22:27 2003-02-03 04:05:06
-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 1970-01-01 06:09:16 2002-02-03 04:05:06
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 2003-04-05 00:00:00 2003-02-03 04:05:06
-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 2001-02-03 00:00:00 2002-02-03 04:05:06
80 81 82 83 84 85 86 87 88 89 str02 fstr2 2005-03-04 05:06:07 2006-08-09 10:11:12
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2\0\0 1970-01-01 06:29:36 2004-02-03 04:05:06
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2\0\0 2004-06-07 00:00:00 2004-02-03 04:05:06
dest from null:
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06

View File

@ -11,7 +11,7 @@ CB_DIR=$(dirname "$CLICKHOUSE_CLIENT_BINARY")
DATA_FILE=$CUR_DIR/data_arrow/test.arrow
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_load"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_load (bool UInt8, int8 Int8, int16 Int16, int32 Int32, int64 Int64, uint8 UInt8, uint16 UInt16, uint32 UInt32, uint64 UInt64, halffloat Float32, float Float32, double Float64, string String, date32 Date, date64 DateTime, timestamp DateTime) ENGINE = Memory"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_load (bool UInt8, int8 Int8, int16 Int16, int32 Int32, int64 Int64, uint8 UInt8, uint16 UInt16, uint32 UInt32, uint64 UInt64, halffloat Float32, float Float32, double Float64, string String, date32 Date, date64 DateTime('Europe/Moscow'), timestamp DateTime('Europe/Moscow')) ENGINE = Memory"
cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into arrow_load format Arrow"
${CLICKHOUSE_CLIENT} --query="select * from arrow_load"

View File

@ -41,7 +41,7 @@ converted:
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06
diff:
dest:
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 1970-01-01 06:29:04
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 00:00:00
80 81 82 83 84 85 86 87 88 89 str02 fstr2\0\0\0\0\0\0\0\0\0\0 2005-03-04 2006-08-09 10:11:12
min:
-128 0 0 0 0 0 0 0 -1 -1 string-1\0\0\0\0\0\0\0 fixedstring-1\0\0 2003-04-05 2003-02-03
@ -49,10 +49,10 @@ min:
79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06
127 -1 -1 -1 -1 -1 -1 -1 -1 -1 string-2\0\0\0\0\0\0\0 fixedstring-2\0\0 2004-06-07 2004-02-03
max:
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 1970-01-01 06:22:27 2003-02-03 04:05:06
-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 1970-01-01 06:09:16 2002-02-03 04:05:06
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 2003-04-05 00:00:00 2003-02-03 04:05:06
-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 2001-02-03 00:00:00 2002-02-03 04:05:06
80 81 82 83 84 85 86 87 88 89 str02 fstr2 2005-03-04 05:06:07 2006-08-09 10:11:12
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2\0\0 1970-01-01 06:29:36 2004-02-03 04:05:06
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2\0\0 2004-06-07 00:00:00 2004-02-03 04:05:06
dest from null:
-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06

View File

@ -12,22 +12,22 @@ SELECT toUnixTimestamp64Micro('abc', 123); -- {serverError 42}
SELECT toUnixTimestamp64Nano('abc', 123); -- {serverError 42}
SELECT 'const column';
WITH toDateTime64('2019-09-16 19:20:12.345678910', 3) AS dt64
WITH toDateTime64('2019-09-16 19:20:12.345678910', 3, 'Europe/Moscow') AS dt64
SELECT dt64, toUnixTimestamp64Milli(dt64), toUnixTimestamp64Micro(dt64), toUnixTimestamp64Nano(dt64);
WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64
WITH toDateTime64('2019-09-16 19:20:12.345678910', 6, 'Europe/Moscow') AS dt64
SELECT dt64, toUnixTimestamp64Milli(dt64), toUnixTimestamp64Micro(dt64), toUnixTimestamp64Nano(dt64);
WITH toDateTime64('2019-09-16 19:20:12.345678910', 9) AS dt64
WITH toDateTime64('2019-09-16 19:20:12.345678910', 9, 'Europe/Moscow') AS dt64
SELECT dt64, toUnixTimestamp64Milli(dt64), toUnixTimestamp64Micro(dt64), toUnixTimestamp64Nano(dt64);
SELECT 'non-const column';
WITH toDateTime64('2019-09-16 19:20:12.345678910', 3) AS x
WITH toDateTime64('2019-09-16 19:20:12.345678910', 3, 'Europe/Moscow') AS x
SELECT materialize(x) as dt64, toUnixTimestamp64Milli(dt64), toUnixTimestamp64Micro(dt64), toUnixTimestamp64Nano(dt64);
WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS x
WITH toDateTime64('2019-09-16 19:20:12.345678910', 6, 'Europe/Moscow') AS x
SELECT materialize(x) as dt64, toUnixTimestamp64Milli(dt64), toUnixTimestamp64Micro(dt64), toUnixTimestamp64Nano(dt64);
WITH toDateTime64('2019-09-16 19:20:12.345678910', 9) AS x
WITH toDateTime64('2019-09-16 19:20:12.345678910', 9, 'Europe/Moscow') AS x
SELECT materialize(x) as dt64, toUnixTimestamp64Milli(dt64), toUnixTimestamp64Micro(dt64), toUnixTimestamp64Nano(dt64);

View File

@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS orc";
$CLICKHOUSE_CLIENT --query="CREATE TABLE orc (uint8 UInt8, uint16 UInt16, uint32 UInt32, uint64 UInt64, int8 Int8, int16 Int16, int32 Int32, int64 Int64, float Float32, double Float64, string String, fixed FixedString(4), date Date, datetime DateTime, decimal32 Decimal32(4), decimal64 Decimal64(10), decimal128 Decimal128(20), nullable Nullable(Int32)) ENGINE = Memory";
$CLICKHOUSE_CLIENT --query="CREATE TABLE orc (uint8 UInt8, uint16 UInt16, uint32 UInt32, uint64 UInt64, int8 Int8, int16 Int16, int32 Int32, int64 Int64, float Float32, double Float64, string String, fixed FixedString(4), date Date, datetime DateTime('Europe/Moscow'), decimal32 Decimal32(4), decimal64 Decimal64(10), decimal128 Decimal128(20), nullable Nullable(Int32)) ENGINE = Memory";
$CLICKHOUSE_CLIENT --query="INSERT INTO orc VALUES (255, 65535, 4294967295, 100000000000, -128, -32768, -2147483648, -100000000000, 2.02, 10000.0000001, 'String', '2020', 18980, 1639872000, 1.0001, 1.00000001, 100000.00000000000001, 1), (4, 1234, 3244467295, 500000000000, -1, -256, -14741221, -7000000000, 100.1, 14321.032141201, 'Another string', '2000', 20000, 1839882000, 34.1234, 123123.123123123, 123123123.123123123123123, NULL), (42, 42, 42, 42, 42, 42, 42, 42, 42.42, 42.42, '42', '4242', 42, 42, 42.42, 42.42424242, 424242.42424242424242, 42)";

View File

@ -1,6 +1,6 @@
SELECT
toDate((number * 10) * 86400) AS d1,
toDate(number * 86400) AS d2,
toDate(toDateTime((number * 10) * 86400, 'Europe/Moscow')) AS d1,
toDate(toDateTime(number * 86400, 'Europe/Moscow')) AS d2,
'original' AS source
FROM numbers(10)
WHERE (number % 3) = 1
@ -11,11 +11,11 @@ ORDER BY
SELECT '===============';
SELECT
toDate((number * 10) * 86400) AS d1,
toDate(number * 86400) AS d2,
toDate(toDateTime((number * 10) * 86400, 'Europe/Moscow')) AS d1,
toDate(toDateTime(number * 86400, 'Europe/Moscow')) AS d2,
'original' AS source
FROM numbers(10)
WHERE (number % 3) = 1
ORDER BY
d1 WITH FILL STEP 5,
d2 WITH FILL;
d2 WITH FILL;

View File

@ -1,14 +1,16 @@
DROP TABLE IF EXISTS tdm;
DROP TABLE IF EXISTS tdm2;
CREATE TABLE tdm (x DateTime) ENGINE = MergeTree ORDER BY x SETTINGS write_final_mark = 0;
CREATE TABLE tdm (x DateTime('Europe/Moscow')) ENGINE = MergeTree ORDER BY x SETTINGS write_final_mark = 0;
INSERT INTO tdm VALUES (now());
SELECT count(x) FROM tdm WHERE toDate(x) < today() SETTINGS max_rows_to_read = 1;
SELECT count(x) FROM tdm WHERE toDate(x) < toDate(now(), 'Europe/Moscow') SETTINGS max_rows_to_read = 1;
SELECT toDate(-1), toDate(10000000000000), toDate(100), toDate(65536), toDate(65535);
SELECT toDateTime(-1), toDateTime(10000000000000), toDateTime(1000);
SELECT toDate(-1), toDate(10000000000000, 'Europe/Moscow'), toDate(100), toDate(65536, 'UTC'), toDate(65535, 'Europe/Moscow');
SELECT toDateTime(-1, 'Europe/Moscow'), toDateTime(10000000000000, 'Europe/Moscow'), toDateTime(1000, 'Europe/Moscow');
CREATE TABLE tdm2 (timestamp UInt32) ENGINE = MergeTree ORDER BY timestamp SETTINGS index_granularity = 1;
INSERT INTO tdm2 VALUES (toUnixTimestamp('2000-01-01 13:12:12')), (toUnixTimestamp('2000-01-01 14:12:12')), (toUnixTimestamp('2000-01-01 15:12:12'));
SET max_rows_to_read = 1;
SELECT toDateTime(timestamp) FROM tdm2 WHERE toHour(toDateTime(timestamp)) = 13;

View File

@ -2,20 +2,20 @@ DROP TABLE IF EXISTS tMM;
DROP TABLE IF EXISTS tDD;
DROP TABLE IF EXISTS sDD;
DROP TABLE IF EXISTS xMM;
CREATE TABLE tMM(d DateTime,a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() SETTINGS index_granularity = 8192;
CREATE TABLE tMM(d DateTime('Europe/Moscow'), a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() SETTINGS index_granularity = 8192;
SYSTEM STOP MERGES tMM;
INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00', 'Europe/Moscow') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00', 'Europe/Moscow') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00', 'Europe/Moscow') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00', 'Europe/Moscow') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00', 'Europe/Moscow') + number*60, number FROM numbers(5000);
INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00', 'Europe/Moscow') + number*60, number FROM numbers(5000);
CREATE TABLE tDD(d DateTime,a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() SETTINGS index_granularity = 8192;
CREATE TABLE tDD(d DateTime('Europe/Moscow'),a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() SETTINGS index_granularity = 8192;
SYSTEM STOP MERGES tDD;
insert into tDD select toDateTime(toDate('2020-09-23')), number from numbers(10000) UNION ALL select toDateTime(toDateTime('2020-09-23 11:00:00')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-24')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-25')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-08-15')), number from numbers(10000);
insert into tDD select toDateTime(toDate('2020-09-23'), 'Europe/Moscow'), number from numbers(10000) UNION ALL select toDateTime(toDateTime('2020-09-23 11:00:00', 'Europe/Moscow')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-24'), 'Europe/Moscow'), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-25'), 'Europe/Moscow'), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-08-15'), 'Europe/Moscow'), number from numbers(10000);
CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000))) ORDER BY tuple() SETTINGS index_granularity = 8192;
CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000), 'Europe/Moscow')) ORDER BY tuple() SETTINGS index_granularity = 8192;
SYSTEM STOP MERGES sDD;
insert into sDD select (1597536000+number*60)*1000, number from numbers(5000);
insert into sDD select (1597536000+number*60)*1000, number from numbers(5000);
@ -24,14 +24,14 @@ insert into sDD select (1598918400+number*60)*1000, number from numbers(5000);
insert into sDD select (1601510400+number*60)*1000, number from numbers(5000);
insert into sDD select (1602720000+number*60)*1000, number from numbers(5000);
CREATE TABLE xMM(d DateTime,a Int64, f Int64) ENGINE = MergeTree PARTITION BY (toYYYYMM(d), a) ORDER BY tuple() SETTINGS index_granularity = 8192;
CREATE TABLE xMM(d DateTime('Europe/Moscow'),a Int64, f Int64) ENGINE = MergeTree PARTITION BY (toYYYYMM(d), a) ORDER BY tuple() SETTINGS index_granularity = 8192;
SYSTEM STOP MERGES xMM;
INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 1, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 2, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 3, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 2, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, 1, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, 1, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00', 'Europe/Moscow') + number*60, 1, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00', 'Europe/Moscow') + number*60, 2, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00', 'Europe/Moscow') + number*60, 3, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00', 'Europe/Moscow') + number*60, 2, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00', 'Europe/Moscow') + number*60, 1, number FROM numbers(5000);
INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00', 'Europe/Moscow') + number*60, 1, number FROM numbers(5000);
SELECT '--------- tMM ----------------------------';
@ -44,8 +44,8 @@ select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816;
select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015;
select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15';
select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00';
select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00');
select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00');
select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00', 'Europe/Moscow');
select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00', 'Europe/Moscow');
select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00';
select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00';
select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00';

View File

@ -35,11 +35,11 @@ select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d
3 15000
Selected 3/6 parts by partition key, 3 parts by primary key, 3/3 marks by primary key, 3 marks to read from 3 ranges
select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00');
select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00', 'Europe/Moscow');
6 30000
Selected 6/6 parts by partition key, 6 parts by primary key, 6/6 marks by primary key, 6 marks to read from 6 ranges
select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00');
select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00', 'Europe/Moscow');
0 0
Selected 0/6 parts by partition key, 0 parts by primary key, 0/0 marks by primary key, 0 marks to read from 0 ranges

View File

@ -1,22 +1,15 @@
#!/usr/bin/env bash
#-------------------------------------------------------------------------------------------
# Description of test result:
# Test the correctness of the partition
# pruning
# Test the correctness of the partition pruning
#
# Script executes queries from a file 01508_partition_pruning_long.queries (1 line = 1 query)
# Queries are started with 'select' (but NOT with 'SELECT') are executed with log_level=debug
#-------------------------------------------------------------------------------------------
# Script executes queries from a file 01508_partition_pruning_long.queries (1 line = 1 query)
# Queries are started with 'select' (but NOT with 'SELECT') are executed with log_level=debug
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
#export CLICKHOUSE_CLIENT="clickhouse-client --send_logs_level=none"
#export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none
#export CURDIR=.
queries="${CURDIR}/01508_partition_pruning_long.queries"
while IFS= read -r sql

View File

@ -4,30 +4,30 @@ DROP TABLE IF EXISTS test_sequenceNextNode_Nullable;
CREATE TABLE IF NOT EXISTS test_sequenceNextNode_Nullable (dt DateTime, id int, action Nullable(String)) ENGINE = MergeTree() PARTITION BY dt ORDER BY id;
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',1,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',1,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',1,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',1,'D');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',2,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',2,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',2,'D');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',2,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',3,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',3,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',4,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',4,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',4,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',4,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',4,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',5,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',5,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',5,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',5,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',6,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',6,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',6,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',6,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',6,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:01',1,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:02',1,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:03',1,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:04',1,'D');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:01',2,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:02',2,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:03',2,'D');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:04',2,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:01',3,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:02',3,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:01',4,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:02',4,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:03',4,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:04',4,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:05',4,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:01',5,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:02',5,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:03',5,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:04',5,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:01',6,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:02',6,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:03',6,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:04',6,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:05',6,'C');
SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
@ -50,11 +50,11 @@ SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, a
SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',10,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,NULL);
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',10,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',10,'D');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:01',10,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:02',10,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:02',10,NULL);
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:03',10,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:04',10,'D');
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
@ -63,10 +63,10 @@ SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(d
SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'D');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:01',11,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:01',11,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:01',11,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('2000-01-02 09:00:01',11,'D');
SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B');
SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D');
@ -100,30 +100,30 @@ DROP TABLE IF EXISTS test_sequenceNextNode;
CREATE TABLE IF NOT EXISTS test_sequenceNextNode (dt DateTime, id int, action String) ENGINE = MergeTree() PARTITION BY dt ORDER BY id;
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',1,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',1,'B');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',1,'C');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',1,'D');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',2,'B');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',2,'B');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',2,'D');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',2,'C');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',3,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',3,'B');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',4,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',4,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',4,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',4,'B');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',4,'C');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',5,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',5,'B');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',5,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',5,'C');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',6,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',6,'B');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',6,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',6,'B');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',6,'C');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:01',1,'A');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:02',1,'B');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:03',1,'C');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:04',1,'D');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:01',2,'B');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:02',2,'B');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:03',2,'D');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:04',2,'C');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:01',3,'A');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:02',3,'B');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:01',4,'A');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:02',4,'A');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:03',4,'A');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:04',4,'B');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:05',4,'C');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:01',5,'A');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:02',5,'B');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:03',5,'A');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:04',5,'C');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:01',6,'A');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:02',6,'B');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:03',6,'A');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:04',6,'B');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:05',6,'C');
SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
@ -146,10 +146,10 @@ SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, a
SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',10,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',10,'B');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',10,'C');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',10,'D');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:01',10,'A');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:02',10,'B');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:03',10,'C');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:04',10,'D');
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
@ -158,10 +158,10 @@ SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(d
SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'B');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'C');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'D');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:01',11,'A');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:01',11,'B');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:01',11,'C');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:01',11,'D');
SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B');
SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D');
@ -189,8 +189,8 @@ SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_
SELECT '(max_args)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = '0', action = '1', action = '2', action = '3', action = '4', action = '5', action = '6', action = '7', action = '8', action = '9', action = '10', action = '11', action = '12', action = '13', action = '14', action = '15', action = '16', action = '17', action = '18', action = '19', action = '20', action = '21', action = '22', action = '23', action = '24', action = '25', action = '26', action = '27', action = '28', action = '29', action = '30', action = '31', action = '32', action = '33', action = '34', action = '35', action = '36', action = '37', action = '38', action = '39', action = '40', action = '41', action = '42', action = '43', action = '44', action = '45', action = '46', action = '47', action = '48', action = '49', action = '50', action = '51', action = '52', action = '53', action = '54', action = '55', action = '56', action = '57', action = '58', action = '59', action = '60', action = '61', action = '62', action = '63') from test_sequenceNextNode GROUP BY id ORDER BY id;
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:01',12,'A');
INSERT INTO test_sequenceNextNode values ('2000-01-02 09:00:01',12,'A');
SELECT '(forward, head, A) id = 12', sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 12;
@ -200,18 +200,18 @@ DROP TABLE IF EXISTS test_base_condition;
CREATE TABLE IF NOT EXISTS test_base_condition (dt DateTime, id int, action String, referrer String) ENGINE = MergeTree() PARTITION BY dt ORDER BY id;
INSERT INTO test_base_condition values ('1970-01-01 09:00:01',1,'A','1');
INSERT INTO test_base_condition values ('1970-01-01 09:00:02',1,'B','2');
INSERT INTO test_base_condition values ('1970-01-01 09:00:03',1,'C','3');
INSERT INTO test_base_condition values ('1970-01-01 09:00:04',1,'D','4');
INSERT INTO test_base_condition values ('1970-01-01 09:00:01',2,'D','4');
INSERT INTO test_base_condition values ('1970-01-01 09:00:02',2,'C','3');
INSERT INTO test_base_condition values ('1970-01-01 09:00:03',2,'B','2');
INSERT INTO test_base_condition values ('1970-01-01 09:00:04',2,'A','1');
INSERT INTO test_base_condition values ('1970-01-01 09:00:01',3,'B','10');
INSERT INTO test_base_condition values ('1970-01-01 09:00:02',3,'B','2');
INSERT INTO test_base_condition values ('1970-01-01 09:00:03',3,'D','3');
INSERT INTO test_base_condition values ('1970-01-01 09:00:04',3,'C','4');
INSERT INTO test_base_condition values ('2000-01-02 09:00:01',1,'A','1');
INSERT INTO test_base_condition values ('2000-01-02 09:00:02',1,'B','2');
INSERT INTO test_base_condition values ('2000-01-02 09:00:03',1,'C','3');
INSERT INTO test_base_condition values ('2000-01-02 09:00:04',1,'D','4');
INSERT INTO test_base_condition values ('2000-01-02 09:00:01',2,'D','4');
INSERT INTO test_base_condition values ('2000-01-02 09:00:02',2,'C','3');
INSERT INTO test_base_condition values ('2000-01-02 09:00:03',2,'B','2');
INSERT INTO test_base_condition values ('2000-01-02 09:00:04',2,'A','1');
INSERT INTO test_base_condition values ('2000-01-02 09:00:01',3,'B','10');
INSERT INTO test_base_condition values ('2000-01-02 09:00:02',3,'B','2');
INSERT INTO test_base_condition values ('2000-01-02 09:00:03',3,'D','3');
INSERT INTO test_base_condition values ('2000-01-02 09:00:04',3,'C','4');
SELECT '(forward, head, 1)', id, sequenceNextNode('forward', 'head')(dt, action, referrer = '1') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(forward, head, 1, A)', id, sequenceNextNode('forward', 'head')(dt, action, referrer = '1', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;

View File

@ -30,8 +30,8 @@ SELECT reinterpret(a, 'String'), reinterpretAsString(a), reinterpretAsUInt8('11'
SELECT reinterpret(a, 'String'), reinterpretAsString(a), reinterpretAsUInt16('11') as a;
SELECT 'Dates';
SELECT reinterpret(0, 'Date'), reinterpret('', 'Date');
SELECT reinterpret(0, 'DateTime'), reinterpret('', 'DateTime');
SELECT reinterpret(0, 'DateTime64'), reinterpret('', 'DateTime64');
SELECT reinterpret(0, 'DateTime(''Europe/Moscow'')'), reinterpret('', 'DateTime(''Europe/Moscow'')');
SELECT reinterpret(0, 'DateTime64(3, ''Europe/Moscow'')'), reinterpret('', 'DateTime64(3, ''Europe/Moscow'')');
SELECT 'Decimals';
SELECT reinterpret(toDecimal32(5, 2), 'Decimal32(2)'), reinterpret('1', 'Decimal32(2)');
SELECT reinterpret(toDecimal64(5, 2), 'Decimal64(2)'), reinterpret('1', 'Decimal64(2)');;

View File

@ -17,11 +17,11 @@ SELECT toDateTime64(toFloat32(bitShiftLeft(toUInt64(1),33)), 2, 'Europe/Moscow')
2106-02-07 09:28:16.00
SELECT toDateTime64(toFloat64(bitShiftLeft(toUInt64(1),33)), 2, 'Europe/Moscow') FORMAT Null;
-- These are outsize of extended range and hence clamped
SELECT toDateTime64(-1 * bitShiftLeft(toUInt64(1), 35), 2);
SELECT toDateTime64(-1 * bitShiftLeft(toUInt64(1), 35), 2, 'Europe/Moscow');
1925-01-01 02:00:00.00
SELECT CAST(-1 * bitShiftLeft(toUInt64(1), 35) AS DateTime64);
SELECT CAST(-1 * bitShiftLeft(toUInt64(1), 35) AS DateTime64(3, 'Europe/Moscow'));
1925-01-01 02:00:00.000
SELECT CAST(bitShiftLeft(toUInt64(1), 35) AS DateTime64);
SELECT CAST(bitShiftLeft(toUInt64(1), 35) AS DateTime64(3, 'Europe/Moscow'));
2282-12-31 03:00:00.000
SELECT toDateTime64(bitShiftLeft(toUInt64(1), 35), 2);
SELECT toDateTime64(bitShiftLeft(toUInt64(1), 35), 2, 'Europe/Moscow');
2282-12-31 03:00:00.00

View File

@ -11,7 +11,7 @@ SELECT toDateTime64(toFloat32(bitShiftLeft(toUInt64(1),33)), 2, 'Europe/Moscow')
SELECT toDateTime64(toFloat64(bitShiftLeft(toUInt64(1),33)), 2, 'Europe/Moscow') FORMAT Null;
-- These are outsize of extended range and hence clamped
SELECT toDateTime64(-1 * bitShiftLeft(toUInt64(1), 35), 2);
SELECT CAST(-1 * bitShiftLeft(toUInt64(1), 35) AS DateTime64);
SELECT CAST(bitShiftLeft(toUInt64(1), 35) AS DateTime64);
SELECT toDateTime64(bitShiftLeft(toUInt64(1), 35), 2);
SELECT toDateTime64(-1 * bitShiftLeft(toUInt64(1), 35), 2, 'Europe/Moscow');
SELECT CAST(-1 * bitShiftLeft(toUInt64(1), 35) AS DateTime64(3, 'Europe/Moscow'));
SELECT CAST(bitShiftLeft(toUInt64(1), 35) AS DateTime64(3, 'Europe/Moscow'));
SELECT toDateTime64(bitShiftLeft(toUInt64(1), 35), 2, 'Europe/Moscow');

View File

@ -1,9 +1,5 @@
-- { echo }
select toDateTime64(toDateTime(1), 2);
1970-01-01 03:00:01.00
select toDateTime64(toDate(1), 2);
1970-01-01 03:00:01.00
1970-01-02 00:00:00.00
select toDateTime64(toDateTime(1), 2, 'GMT');
1970-01-01 00:00:01.00
select toDateTime64(toDate(1), 2, 'GMT');
1970-01-02 00:00:00.00

View File

@ -1,5 +1,7 @@
-- { echo }
select toDateTime64(toDateTime(1), 2);
select toDateTime64(toDate(1), 2);
select toDateTime64(toDateTime(1, 'Europe/Moscow'), 2);
select toDateTime64(toDate(1), 2) FORMAT Null; -- Unknown timezone
select toDateTime64(toDateTime(1), 2) FORMAT Null; -- Unknown timezone
select toDateTime64(toDateTime(1), 2, 'Europe/Moscow');
select toDateTime64(toDate(1), 2, 'Europe/Moscow');
select toDateTime64(toDateTime(1), 2, 'GMT');
select toDateTime64(toDate(1), 2, 'GMT');

View File

@ -1,9 +1,4 @@
-- { echo }
SELECT toString(toDateTime('-922337203.6854775808', 1));
1940-10-09 22:13:17.6
SELECT toString(toDateTime('9922337203.6854775808', 1));
2283-11-11 23:46:43.6
SELECT toDateTime64(CAST('10000000000.1' AS Decimal64(1)), 1);
2283-11-11 23:46:40.1
SELECT toDateTime64(CAST('-10000000000.1' AS Decimal64(1)), 1);
1925-01-01 00:00:00.1

View File

@ -1,5 +1,4 @@
-- { echo }
SELECT toString(toDateTime('-922337203.6854775808', 1));
SELECT toString(toDateTime('9922337203.6854775808', 1));
SELECT toDateTime64(CAST('10000000000.1' AS Decimal64(1)), 1);
SELECT toDateTime64(CAST('-10000000000.1' AS Decimal64(1)), 1);
SELECT toString(toDateTime('-922337203.6854775808', 1, 'Europe/Moscow'));
SELECT toString(toDateTime('9922337203.6854775808', 1, 'Europe/Moscow'));
SELECT toDateTime64(CAST('10000000000.1' AS Decimal64(1)), 1, 'Europe/Moscow');
SELECT toDateTime64(CAST('-10000000000.1' AS Decimal64(1)), 1, 'Europe/Moscow');

View File

@ -5,7 +5,7 @@ INSERT INTO t VALUES (3, '1111111111222');
INSERT INTO t VALUES (4, '1111111111.222');
SELECT * FROM t ORDER BY i;
SELECT toDateTime64(1111111111.222, 3);
SELECT toDateTime64('1111111111.222', 3);
SELECT toDateTime64('1111111111222', 3);
SELECT ignore(toDateTime64(1111111111222, 3)); -- This gives somewhat correct but unexpected result
SELECT toDateTime64(1111111111.222, 3, 'Europe/Moscow');
SELECT toDateTime64('1111111111.222', 3, 'Europe/Moscow');
SELECT toDateTime64('1111111111222', 3, 'Europe/Moscow');
SELECT ignore(toDateTime64(1111111111222, 3, 'Europe/Moscow')); -- This gives somewhat correct but unexpected result

View File

@ -1,7 +1,3 @@
-- { echo }
SELECT CAST(1111111111.222 AS DateTime64(3));
2005-03-18 04:58:31.222
SELECT toDateTime(1111111111.222, 3);
2005-03-18 04:58:31.222
SELECT toDateTime64(1111111111.222, 3);
2005-03-18 04:58:31.222

View File

@ -1,4 +1,3 @@
-- { echo }
SELECT CAST(1111111111.222 AS DateTime64(3));
SELECT toDateTime(1111111111.222, 3);
SELECT toDateTime64(1111111111.222, 3);
SELECT CAST(1111111111.222 AS DateTime64(3, 'Europe/Moscow'));
SELECT toDateTime(1111111111.222, 3, 'Europe/Moscow');
SELECT toDateTime64(1111111111.222, 3, 'Europe/Moscow');

View File

@ -1,5 +1,5 @@
drop table if exists replacing;
create table replacing( `A` Int64, `D` DateTime64(9), `S` String) ENGINE = ReplacingMergeTree(D) ORDER BY A;
create table replacing( `A` Int64, `D` DateTime64(9, 'Europe/Moscow'), `S` String) ENGINE = ReplacingMergeTree(D) ORDER BY A;
insert into replacing values (1,'1970-01-01 08:25:46.300800000','a');
insert into replacing values (2,'1970-01-01 08:25:46.300800002','b');

View File

@ -1,5 +1,5 @@
drop table if exists tab;
create table tab (i8 Int8, i16 Int16, i32 Int32, i64 Int64, i128 Int128, i256 Int256, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, u128 UInt128, u256 UInt256, id UUID, s String, fs FixedString(33), a Array(UInt8), t Tuple(UInt16, UInt32), d Date, dt DateTime, dt64 DateTime64, dec128 Decimal128(3), dec256 Decimal256(4), lc LowCardinality(String)) engine = MergeTree PARTITION BY (i8, i16, i32, i64, i128, i256, u8, u16, u32, u64, u128, u256, id, s, fs, a, t, d, dt, dt64, dec128, dec256, lc) order by tuple();
create table tab (i8 Int8, i16 Int16, i32 Int32, i64 Int64, i128 Int128, i256 Int256, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, u128 UInt128, u256 UInt256, id UUID, s String, fs FixedString(33), a Array(UInt8), t Tuple(UInt16, UInt32), d Date, dt DateTime('Europe/Moscow'), dt64 DateTime64(3, 'Europe/Moscow'), dec128 Decimal128(3), dec256 Decimal256(4), lc LowCardinality(String)) engine = MergeTree PARTITION BY (i8, i16, i32, i64, i128, i256, u8, u16, u32, u64, u128, u256, id, s, fs, a, t, d, dt, dt64, dec128, dec256, lc) order by tuple();
insert into tab values (-1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, '61f0c404-5cb3-11e7-907b-a6006ad3dba0', 'a', 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', [1, 2, 3], (-1, -2), '2020-01-01', '2020-01-01 01:01:01', '2020-01-01 01:01:01', '123.456', '78.9101', 'a');
-- Here we check that partition id did not change.
-- Different result means Backward Incompatible Change. Old partitions will not be accepted by new server.

View File

@ -1,5 +1,5 @@
drop table if exists tab;
create table tab (i8 Int8, i16 Int16, i32 Int32, i64 Int64, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, id UUID, s String, fs FixedString(33), a Array(UInt8), t Tuple(UInt16, UInt32), d Date, dt DateTime, dt64 DateTime64, dec128 Decimal128(3), lc LowCardinality(String)) engine = MergeTree PARTITION BY (i8, i16, i32, i64, u8, u16, u32, u64, id, s, fs, a, t, d, dt, dt64, dec128, lc) order by tuple();
create table tab (i8 Int8, i16 Int16, i32 Int32, i64 Int64, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, id UUID, s String, fs FixedString(33), a Array(UInt8), t Tuple(UInt16, UInt32), d Date, dt DateTime('Europe/Moscow'), dt64 DateTime64(3, 'Europe/Moscow'), dec128 Decimal128(3), lc LowCardinality(String)) engine = MergeTree PARTITION BY (i8, i16, i32, i64, u8, u16, u32, u64, id, s, fs, a, t, d, dt, dt64, dec128, lc) order by tuple();
insert into tab values (-1, -1, -1, -1, -1, -1, -1, -1, '61f0c404-5cb3-11e7-907b-a6006ad3dba0', 'a', 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', [1, 2, 3], (-1, -2), '2020-01-01', '2020-01-01 01:01:01', '2020-01-01 01:01:01', '123.456', 'a');
-- Here we check that partition id did not change.
-- Different result means Backward Incompatible Change. Old partitions will not be accepted by new server.

View File

@ -1,10 +1,17 @@
drop table if exists t;
create table t engine Memory as select * from generateRandom('a Array(Int8), b UInt32, c Nullable(String), d Decimal32(4), e Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)), f Float64, g Tuple(Date, DateTime, DateTime64, UUID), h FixedString(2), i Array(Nullable(UUID))', 10, 5, 3) limit 2;
create temporary table t engine Memory as select * from generateRandom(
$$
a Array(Int8),
b UInt32,
c Nullable(String),
d Decimal32(4),
e Nullable(Enum16('h' = 1, 'w' = 5 , 'o' = -200)),
f Float64,
g Tuple(Date, DateTime('Europe/Moscow'), DateTime64(3, 'Europe/Moscow'), UUID),
h FixedString(2),
i Array(Nullable(UUID))
$$, 10, 5, 3) limit 2;
select * apply toJSONString from t;
drop table t;
set allow_experimental_map_type = 1;
select toJSONString(map('1234', '5678'));

View File

@ -15,7 +15,7 @@ WITH toDate('2000-01-01') as a, toDateTime('2000-01-01', 'Europe/Moscow') as b
SELECT if(value, b, a) as result, toTypeName(result)
FROM predicate_table;
WITH toDateTime('2000-01-01') as a, toDateTime64('2000-01-01', 5, 'Europe/Moscow') as b
WITH toDateTime('2000-01-01', 'Europe/Moscow') as a, toDateTime64('2000-01-01', 5, 'Europe/Moscow') as b
SELECT if(value, b, a) as result, toTypeName(result)
FROM predicate_table;

View File

@ -1 +1 @@
`a1` Array(Int8), `a2` Array(UInt8), `a3` Array(Int16), `a4` Array(UInt16), `a5` Array(Int32), `a6` Array(UInt32), `a7` Array(Int64), `a8` Array(UInt64), `a9` Array(String), `a10` Array(FixedString(4)), `a11` Array(Float32), `a12` Array(Float64), `a13` Array(Date), `a14` Array(Datetime), `a15` Array(Decimal(4, 2)), `a16` Array(Decimal(10, 2)), `a17` Array(Decimal(25, 2))
`a1` Array(Int8), `a2` Array(UInt8), `a3` Array(Int16), `a4` Array(UInt16), `a5` Array(Int32), `a6` Array(UInt32), `a7` Array(Int64), `a8` Array(UInt64), `a9` Array(String), `a10` Array(FixedString(4)), `a11` Array(Float32), `a12` Array(Float64), `a13` Array(Date), `a14` Array(Datetime('Europe/Moscow')), `a15` Array(Decimal(4, 2)), `a16` Array(Decimal(10, 2)), `a17` Array(Decimal(25, 2))

View File

@ -1 +1 @@
`a` Nullable(Int64), `b` Nullable(Float64), `c` Nullable(DateTime), `index` Nullable(String), `__index_level_1__` Nullable(DateTime)
`a` Nullable(Int64), `b` Nullable(Float64), `c` Nullable(DateTime('Europe/Moscow')), `index` Nullable(String), `__index_level_1__` Nullable(DateTime('Europe/Moscow'))

View File

@ -4,8 +4,8 @@ import json
import sys
TYPE_PARQUET_CONVERTED_TO_CLICKHOUSE = {
"TIMESTAMP_MICROS": "DateTime",
"TIMESTAMP_MILLIS": "DateTime",
"TIMESTAMP_MICROS": "DateTime('Europe/Moscow')",
"TIMESTAMP_MILLIS": "DateTime('Europe/Moscow')",
"UTF8": "String",
}

View File

@ -1 +1 @@
SELECT EventTime FROM test.hits ORDER BY EventTime DESC LIMIT 10
SELECT EventTime::DateTime('Europe/Moscow') FROM test.hits ORDER BY EventTime DESC LIMIT 10

View File

@ -1 +1 @@
SELECT EventTime FROM remote('127.0.0.{1,2}', test, hits) ORDER BY EventTime DESC LIMIT 10
SELECT EventTime::DateTime('Europe/Moscow') FROM remote('127.0.0.{1,2}', test, hits) ORDER BY EventTime DESC LIMIT 10

View File

@ -1,2 +1,2 @@
SET max_parallel_replicas = 2;
SELECT EventTime FROM remote('127.0.0.{1|2}', test, hits) ORDER BY EventTime DESC LIMIT 10
SELECT EventTime::DateTime('Europe/Moscow') FROM remote('127.0.0.{1|2}', test, hits) ORDER BY EventTime DESC LIMIT 10

View File

@ -1,6 +1,6 @@
DROP TABLE IF EXISTS test.hits_snippet;
CREATE TABLE test.hits_snippet(EventTime DateTime, EventDate Date, CounterID UInt32, UserID UInt64, URL String, Referer String) ENGINE = MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192);
CREATE TABLE test.hits_snippet(EventTime DateTime('Europe/Moscow'), EventDate Date, CounterID UInt32, UserID UInt64, URL String, Referer String) ENGINE = MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192);
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
SET max_block_size = 4096;

View File

@ -15,7 +15,7 @@ SELECT count() FROM test.hits WHERE EventDate IN (toDate('2014-03-18'), toDate('
SELECT count() FROM test.hits WHERE EventDate = concat('2014-0', '3-18');
DROP TABLE IF EXISTS test.hits_indexed_by_time;
CREATE TABLE test.hits_indexed_by_time (EventDate Date, EventTime DateTime) ENGINE = MergeTree(EventDate, EventTime, 8192);
CREATE TABLE test.hits_indexed_by_time (EventDate Date, EventTime DateTime('Europe/Moscow')) ENGINE = MergeTree ORDER BY (EventDate, EventTime);
INSERT INTO test.hits_indexed_by_time SELECT EventDate, EventTime FROM test.hits;
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime = '2014-03-18 01:02:03';
@ -25,12 +25,12 @@ SELECT count() FROM test.hits_indexed_by_time WHERE EventTime <= '2014-03-18 01:
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime >= '2014-03-18 01:02:03';
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime IN ('2014-03-18 01:02:03', '2014-03-19 04:05:06');
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime = toDateTime('2014-03-18 01:02:03');
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime < toDateTime('2014-03-18 01:02:03');
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime > toDateTime('2014-03-18 01:02:03');
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime <= toDateTime('2014-03-18 01:02:03');
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime >= toDateTime('2014-03-18 01:02:03');
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime IN (toDateTime('2014-03-18 01:02:03'), toDateTime('2014-03-19 04:05:06'));
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime = toDateTime('2014-03-18 01:02:03', 'Europe/Moscow');
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime < toDateTime('2014-03-18 01:02:03', 'Europe/Moscow');
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime > toDateTime('2014-03-18 01:02:03', 'Europe/Moscow');
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime <= toDateTime('2014-03-18 01:02:03', 'Europe/Moscow');
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime >= toDateTime('2014-03-18 01:02:03', 'Europe/Moscow');
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime IN (toDateTime('2014-03-18 01:02:03', 'Europe/Moscow'), toDateTime('2014-03-19 04:05:06', 'Europe/Moscow'));
SELECT count() FROM test.hits_indexed_by_time WHERE EventTime = concat('2014-03-18 ', '01:02:03');

View File

@ -1,2 +1,2 @@
SELECT UserID, EventTime, pp.Key1, pp.Key2, ParsedParams.Key1 FROM test.hits ARRAY JOIN ParsedParams AS pp WHERE CounterID = 1704509 ORDER BY UserID, EventTime, pp.Key1, pp.Key2 LIMIT 100;
SELECT UserID, EventTime, pp.Key1, pp.Key2, ParsedParams.Key1 FROM test.hits LEFT ARRAY JOIN ParsedParams AS pp WHERE CounterID = 1704509 ORDER BY UserID, EventTime, pp.Key1, pp.Key2 LIMIT 100;
SELECT UserID, EventTime::DateTime('Europe/Moscow'), pp.Key1, pp.Key2, ParsedParams.Key1 FROM test.hits ARRAY JOIN ParsedParams AS pp WHERE CounterID = 1704509 ORDER BY UserID, EventTime, pp.Key1, pp.Key2 LIMIT 100;
SELECT UserID, EventTime::DateTime('Europe/Moscow'), pp.Key1, pp.Key2, ParsedParams.Key1 FROM test.hits LEFT ARRAY JOIN ParsedParams AS pp WHERE CounterID = 1704509 ORDER BY UserID, EventTime, pp.Key1, pp.Key2 LIMIT 100;

View File

@ -2,12 +2,12 @@ SET max_bytes_to_read = 600000000;
SET optimize_move_to_prewhere = 1;
SELECT uniq(URL) FROM test.hits WHERE EventTime >= '2014-03-20 00:00:00' AND EventTime < '2014-03-21 00:00:00';
SELECT uniq(URL) FROM test.hits WHERE EventTime >= '2014-03-20 00:00:00' AND URL != '' AND EventTime < '2014-03-21 00:00:00';
SELECT uniq(*) FROM test.hits WHERE EventTime >= '2014-03-20 00:00:00' AND EventTime < '2014-03-21 00:00:00' AND EventDate = '2014-03-21';
WITH EventTime AS xyz SELECT uniq(*) FROM test.hits WHERE xyz >= '2014-03-20 00:00:00' AND xyz < '2014-03-21 00:00:00' AND EventDate = '2014-03-21';
SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Europe/Moscow') >= '2014-03-20 00:00:00' AND toTimeZone(EventTime, 'Europe/Moscow') < '2014-03-21 00:00:00';
SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Europe/Moscow') >= '2014-03-20 00:00:00' AND URL != '' AND toTimeZone(EventTime, 'Europe/Moscow') < '2014-03-21 00:00:00';
SELECT uniq(*) FROM test.hits WHERE toTimeZone(EventTime, 'Europe/Moscow') >= '2014-03-20 00:00:00' AND toTimeZone(EventTime, 'Europe/Moscow') < '2014-03-21 00:00:00' AND EventDate = '2014-03-21';
WITH toTimeZone(EventTime, 'Europe/Moscow') AS xyz SELECT uniq(*) FROM test.hits WHERE xyz >= '2014-03-20 00:00:00' AND xyz < '2014-03-21 00:00:00' AND EventDate = '2014-03-21';
SET optimize_move_to_prewhere = 0;
SELECT uniq(URL) FROM test.hits WHERE EventTime >= '2014-03-20 00:00:00' AND EventTime < '2014-03-21 00:00:00'; -- { serverError 307 }
SELECT uniq(URL) FROM test.hits WHERE EventTime >= '2014-03-20 00:00:00' AND URL != '' AND EventTime < '2014-03-21 00:00:00'; -- { serverError 307 }
SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Europe/Moscow') >= '2014-03-20 00:00:00' AND toTimeZone(EventTime, 'Europe/Moscow') < '2014-03-21 00:00:00'; -- { serverError 307 }
SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Europe/Moscow') >= '2014-03-20 00:00:00' AND URL != '' AND toTimeZone(EventTime, 'Europe/Moscow') < '2014-03-21 00:00:00'; -- { serverError 307 }

View File

@ -10,10 +10,10 @@ for format in "${FORMATS[@]}"
do
echo "$format, false";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
"SELECT ClientEventTime as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum
"SELECT ClientEventTime::DateTime('Europe/Moscow') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum
echo "$format, true";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=true -q \
"SELECT ClientEventTime as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum
"SELECT ClientEventTime::DateTime('Europe/Moscow') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum
done

View File

@ -10,8 +10,8 @@ FORMATS=('TSV' 'CSV' 'JSONCompactEachRow')
for format in "${FORMATS[@]}"
do
echo "$format, false";
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+ClientEventTime+as+a,MobilePhoneModel+as+b,ClientIP6+as+c+FROM+test.hits+ORDER+BY+a,b,c+LIMIT+1000000+Format+$format&output_format_parallel_formatting=false" -d' ' | md5sum
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+ClientEventTime::DateTime('Europe/Moscow')+as+a,MobilePhoneModel+as+b,ClientIP6+as+c+FROM+test.hits+ORDER+BY+a,b,c+LIMIT+1000000+Format+$format&output_format_parallel_formatting=false" -d' ' | md5sum
echo "$format, true";
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+ClientEventTime+as+a,MobilePhoneModel+as+b,ClientIP6+as+c+FROM+test.hits+ORDER+BY+a,b,c+LIMIT+1000000+Format+$format&output_format_parallel_formatting=true" -d' ' | md5sum
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+ClientEventTime::DateTime('Europe/Moscow')+as+a,MobilePhoneModel+as+b,ClientIP6+as+c+FROM+test.hits+ORDER+BY+a,b,c+LIMIT+1000000+Format+$format&output_format_parallel_formatting=true" -d' ' | md5sum
done

View File

@ -7,6 +7,6 @@ ba1081a754a06ef6563840b2d8d4d327 -
JSONCompactEachRow, true
ba1081a754a06ef6563840b2d8d4d327 -
JSONCompactStringsEachRowWithNamesAndTypes, false
902e53f621d5336aa7f702a5d6b64b42 -
31ded3cd9971b124450fb5a44a8bce63 -
JSONCompactStringsEachRowWithNamesAndTypes, true
902e53f621d5336aa7f702a5d6b64b42 -
31ded3cd9971b124450fb5a44a8bce63 -

View File

@ -11,9 +11,9 @@ for format in "${FORMATS[@]}"
do
echo "$format, false";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
"SELECT ClientEventTime as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum
"SELECT ClientEventTime::DateTime('Europe/Moscow') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum
echo "$format, true";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=true -q \
"SELECT ClientEventTime as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum
"SELECT ClientEventTime::DateTime('Europe/Moscow') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum
done

View File

@ -11,9 +11,9 @@ for format in "${FORMATS[@]}"
do
echo "$format, false";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
"SELECT ClientEventTime as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum
"SELECT ClientEventTime::DateTime('Europe/Moscow') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum
echo "$format, true";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=true -q \
"SELECT ClientEventTime as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum
"SELECT ClientEventTime::DateTime('Europe/Moscow') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum
done

View File

@ -10,23 +10,23 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names"
for format in "${FORMATS[@]}"
do
# Columns are permuted
$CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime, b String) ENGINE=Memory()"
$CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Europe/Moscow'), b String) ENGINE=Memory()"
echo "$format, false";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
"SELECT URLRegions as d, ClientEventTime as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 50000 Format $format" | \
"SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Europe/Moscow') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 50000 Format $format" | \
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names FORMAT $format"
$CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names"
$CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime, b String) ENGINE=Memory()"
$CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Europe/Moscow'), b String) ENGINE=Memory()"
echo "$format, true";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
"SELECT URLRegions as d, ClientEventTime as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 50000 Format $format" | \
"SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Europe/Moscow') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 50000 Format $format" | \
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names FORMAT $format"
$CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names"
done
done

View File

@ -11,7 +11,7 @@ for format in "${FORMATS[@]}"
do
echo $format
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 00163_column_oriented SYNC"
$CLICKHOUSE_CLIENT -q "CREATE TABLE 00163_column_oriented(ClientEventTime DateTime, MobilePhoneModel String, ClientIP6 FixedString(16)) ENGINE=File($format)"
$CLICKHOUSE_CLIENT -q "CREATE TABLE 00163_column_oriented(ClientEventTime DateTime('Europe/Moscow'), MobilePhoneModel String, ClientIP6 FixedString(16)) ENGINE=File($format)"
$CLICKHOUSE_CLIENT -q "INSERT INTO 00163_column_oriented SELECT ClientEventTime, MobilePhoneModel, ClientIP6 FROM test.hits ORDER BY ClientEventTime, MobilePhoneModel, ClientIP6 LIMIT 100"
$CLICKHOUSE_CLIENT -q "SELECT ClientEventTime from 00163_column_oriented" | md5sum
$CLICKHOUSE_CLIENT -q "SELECT MobilePhoneModel from 00163_column_oriented" | md5sum

View File

@ -521,6 +521,7 @@
"01999_grant_with_replace",
"01902_table_function_merge_db_repr",
"01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer",
"01946_test_wrong_host_name_access"
"01946_test_wrong_host_name_access",
"01213_alter_rename_with_default_zookeeper" /// Warning: Removing leftovers from table.
]
}