mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
Merge remote-tracking branch 'upstream/master' into fix4
This commit is contained in:
commit
956e1513c5
@ -1,6 +1,6 @@
|
||||
# This strings autochanged from release_lib.sh:
|
||||
set(VERSION_DESCRIBE v1.1.54300-testing)
|
||||
set(VERSION_REVISION 54300)
|
||||
set(VERSION_DESCRIBE v1.1.54306-testing)
|
||||
set(VERSION_REVISION 54306)
|
||||
# end of autochange
|
||||
|
||||
set (VERSION_MAJOR 1)
|
||||
|
@ -92,7 +92,7 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
AggregateFunctionForEach(AggregateFunctionPtr nested_)
|
||||
explicit AggregateFunctionForEach(AggregateFunctionPtr nested_)
|
||||
: nested_func_owner(nested_), nested_func(nested_func_owner.get())
|
||||
{
|
||||
}
|
||||
|
@ -26,6 +26,8 @@ private:
|
||||
Array parameters;
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
DataTypeAggregateFunction(const AggregateFunctionPtr & function_, const DataTypes & argument_types_, const Array & parameters_)
|
||||
: function(function_), argument_types(argument_types_), parameters(parameters_)
|
||||
{
|
||||
|
@ -19,6 +19,8 @@ private:
|
||||
DataTypePtr offsets;
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
DataTypeArray(const DataTypePtr & nested_);
|
||||
DataTypeArray(const DataTypeTraits::EnrichedDataTypePtr & enriched_nested_);
|
||||
|
||||
|
@ -36,6 +36,8 @@ public:
|
||||
using NameToValueMap = HashMap<StringRef, FieldType, StringRefHash>;
|
||||
using ValueToNameMap = std::unordered_map<FieldType, StringRef>;
|
||||
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
private:
|
||||
Values values;
|
||||
NameToValueMap name_to_value_map;
|
||||
|
@ -15,6 +15,8 @@ private:
|
||||
DataTypePtr return_type;
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
/// Some types could be still unknown.
|
||||
DataTypeExpression(const DataTypes & argument_types_ = DataTypes(), const DataTypePtr & return_type_ = nullptr)
|
||||
: argument_types(argument_types_), return_type(return_type_) {}
|
||||
|
@ -117,6 +117,7 @@ void registerDataTypeNull(DataTypeFactory & factory);
|
||||
void registerDataTypeUUID(DataTypeFactory & factory);
|
||||
void registerDataTypeAggregateFunction(DataTypeFactory & factory);
|
||||
void registerDataTypeNested(DataTypeFactory & factory);
|
||||
void registerDataTypeInterval(DataTypeFactory & factory);
|
||||
|
||||
|
||||
DataTypeFactory::DataTypeFactory()
|
||||
@ -134,6 +135,7 @@ DataTypeFactory::DataTypeFactory()
|
||||
registerDataTypeUUID(*this);
|
||||
registerDataTypeAggregateFunction(*this);
|
||||
registerDataTypeNested(*this);
|
||||
registerDataTypeInterval(*this);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -18,6 +18,8 @@ private:
|
||||
size_t n;
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
DataTypeFixedString(size_t n_) : n(n_)
|
||||
{
|
||||
if (n == 0)
|
||||
|
19
dbms/src/DataTypes/DataTypeInterval.cpp
Normal file
19
dbms/src/DataTypes/DataTypeInterval.cpp
Normal file
@ -0,0 +1,19 @@
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerDataTypeInterval(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataType("IntervalSecond", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(DataTypeInterval::Second)); });
|
||||
factory.registerSimpleDataType("IntervalMinute", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(DataTypeInterval::Minute)); });
|
||||
factory.registerSimpleDataType("IntervalHour", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(DataTypeInterval::Hour)); });
|
||||
factory.registerSimpleDataType("IntervalDay", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(DataTypeInterval::Day)); });
|
||||
factory.registerSimpleDataType("IntervalWeek", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(DataTypeInterval::Week)); });
|
||||
factory.registerSimpleDataType("IntervalMonth", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(DataTypeInterval::Month)); });
|
||||
factory.registerSimpleDataType("IntervalYear", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(DataTypeInterval::Year)); });
|
||||
}
|
||||
|
||||
}
|
66
dbms/src/DataTypes/DataTypeInterval.h
Normal file
66
dbms/src/DataTypes/DataTypeInterval.h
Normal file
@ -0,0 +1,66 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Data type to deal with INTERVAL in SQL (arithmetic on time intervals).
|
||||
*
|
||||
* Mostly the same as Int64.
|
||||
* But also tagged with interval kind.
|
||||
*
|
||||
* Intended isage is for temporary elements in expressions,
|
||||
* not for storing values in tables.
|
||||
*/
|
||||
class DataTypeInterval final : public DataTypeNumberBase<Int64>
|
||||
{
|
||||
public:
|
||||
enum Kind
|
||||
{
|
||||
Second,
|
||||
Minute,
|
||||
Hour,
|
||||
Day,
|
||||
Week,
|
||||
Month,
|
||||
Year
|
||||
};
|
||||
|
||||
private:
|
||||
Kind kind;
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
Kind getKind() const { return kind; }
|
||||
|
||||
const char * kindToString() const
|
||||
{
|
||||
switch (kind)
|
||||
{
|
||||
case Second: return "Second";
|
||||
case Minute: return "Minute";
|
||||
case Hour: return "Hour";
|
||||
case Day: return "Day";
|
||||
case Week: return "Week";
|
||||
case Month: return "Month";
|
||||
case Year: return "Year";
|
||||
default: __builtin_unreachable();
|
||||
}
|
||||
}
|
||||
|
||||
DataTypeInterval(Kind kind) : kind(kind) {};
|
||||
|
||||
std::string getName() const override { return std::string("Interval") + kindToString(); }
|
||||
const char * getFamilyName() const override { return "Interval"; }
|
||||
|
||||
bool behavesAsNumber() const override { return false; }
|
||||
bool notForTables() const override { return true; }
|
||||
|
||||
DataTypePtr clone() const override { return std::make_shared<DataTypeInterval>(kind); }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -17,6 +17,8 @@ private:
|
||||
NamesAndTypesListPtr nested;
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
DataTypeNested(const NamesAndTypesListPtr & nested_);
|
||||
|
||||
std::string getName() const override;
|
||||
|
@ -16,6 +16,8 @@ public:
|
||||
using FieldType = Null;
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = false;
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return "Null";
|
||||
|
@ -11,6 +11,8 @@ namespace DB
|
||||
class DataTypeNullable final : public IDataType
|
||||
{
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
DataTypeNullable(const DataTypePtr & nested_data_type_);
|
||||
std::string getName() const override { return "Nullable(" + nested_data_type->getName() + ")"; }
|
||||
const char * getFamilyName() const override { return "Nullable"; }
|
||||
|
@ -12,6 +12,7 @@ template <typename T>
|
||||
class DataTypeNumberBase : public IDataType
|
||||
{
|
||||
public:
|
||||
static constexpr bool is_parametric = false;
|
||||
using FieldType = T;
|
||||
|
||||
std::string getName() const override { return TypeName<T>::get(); }
|
||||
|
@ -12,6 +12,7 @@ namespace DB
|
||||
class DataTypeSet final : public IDataTypeDummy
|
||||
{
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
std::string getName() const override { return "Set"; }
|
||||
const char * getFamilyName() const override { return "Set"; }
|
||||
DataTypePtr clone() const override { return std::make_shared<DataTypeSet>(); }
|
||||
|
@ -12,6 +12,7 @@ class DataTypeString final : public IDataType
|
||||
{
|
||||
public:
|
||||
using FieldType = String;
|
||||
static constexpr bool is_parametric = false;
|
||||
|
||||
std::string getName() const override
|
||||
{
|
||||
|
@ -16,6 +16,7 @@ class DataTypeTuple final : public IDataType
|
||||
private:
|
||||
DataTypes elems;
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
DataTypeTuple(const DataTypes & elems_) : elems(elems_) {}
|
||||
|
||||
std::string getName() const override;
|
||||
|
@ -33,6 +33,7 @@ class DataTypeNumber<void> final : public IDataTypeDummy
|
||||
{
|
||||
public:
|
||||
using FieldType = void;
|
||||
static constexpr bool is_parametric = false;
|
||||
|
||||
std::string getName() const override { return "Void"; }
|
||||
const char * getFamilyName() const override { return "Void"; }
|
||||
@ -46,6 +47,7 @@ class DataTypeNumber<Null> final : public IDataTypeDummy
|
||||
{
|
||||
public:
|
||||
using FieldType = Null;
|
||||
static constexpr bool is_parametric = false;
|
||||
|
||||
std::string getName() const override { return "Null"; }
|
||||
const char * getFamilyName() const override { return "Null"; }
|
||||
|
@ -29,6 +29,12 @@ using DataTypes = std::vector<DataTypePtr>;
|
||||
class IDataType
|
||||
{
|
||||
public:
|
||||
/// Compile time flag. If false, then if C++ types are the same, then SQL types are also the same.
|
||||
/// Example: DataTypeString is not parametric: thus all instances of DataTypeString are the same SQL type.
|
||||
/// Example: DataTypeFixedString is parametric: different instances of DataTypeFixedString may be different SQL types.
|
||||
/// Place it in descendants:
|
||||
/// static constexpr bool is_parametric = false;
|
||||
|
||||
/// Name of data type (examples: UInt64, Array(String)).
|
||||
virtual String getName() const = 0;
|
||||
|
||||
|
@ -13,6 +13,11 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
||||
}
|
||||
|
||||
|
||||
String getTableDefinitionFromCreateQuery(const ASTPtr & query)
|
||||
{
|
||||
@ -22,8 +27,6 @@ String getTableDefinitionFromCreateQuery(const ASTPtr & query)
|
||||
/// We remove everything that is not needed for ATTACH from the query.
|
||||
create.attach = true;
|
||||
create.database.clear();
|
||||
create.as_database.clear();
|
||||
create.as_table.clear();
|
||||
create.if_not_exists = false;
|
||||
create.is_populate = false;
|
||||
|
||||
@ -31,6 +34,13 @@ String getTableDefinitionFromCreateQuery(const ASTPtr & query)
|
||||
if (!create.is_view && !create.is_materialized_view)
|
||||
create.select = nullptr;
|
||||
|
||||
/// For "MATERIALIZED VIEW x TO y" it's necessary to save destination table
|
||||
if (!(create.is_materialized_view && !create.storage))
|
||||
{
|
||||
create.as_database.clear();
|
||||
create.as_table.clear();
|
||||
}
|
||||
|
||||
std::ostringstream statement_stream;
|
||||
formatAST(create, statement_stream, 0, false);
|
||||
statement_stream << '\n';
|
||||
@ -56,6 +66,8 @@ std::pair<String, StoragePtr> createTableFromDefinition(
|
||||
/// We do not directly use `InterpreterCreateQuery::execute`, because
|
||||
/// - the database has not been created yet;
|
||||
/// - the code is simpler, since the query is already brought to a suitable form.
|
||||
if (!ast_create_query.columns)
|
||||
throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
|
||||
|
||||
InterpreterCreateQuery::ColumnsInfo columns_info = InterpreterCreateQuery::getColumnsInfo(*ast_create_query.columns, context);
|
||||
|
||||
|
@ -3,10 +3,12 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/NumberTraits.h>
|
||||
#include <Core/AccurateComparison.h>
|
||||
#include <Core/FieldVisitors.h>
|
||||
@ -542,9 +544,13 @@ class FunctionBinaryArithmetic : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionBinaryArithmetic>(); }
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionBinaryArithmetic>(context); }
|
||||
|
||||
FunctionBinaryArithmetic(const Context & context) : context(context) {}
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
|
||||
/// Overload for InvalidType
|
||||
template <typename ResultDataType,
|
||||
typename std::enable_if<std::is_same<ResultDataType, InvalidType>::value>::type * = nullptr>
|
||||
@ -745,6 +751,47 @@ private:
|
||||
return false;
|
||||
}
|
||||
|
||||
FunctionPtr getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1) const
|
||||
{
|
||||
/// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval.
|
||||
/// We construct another function (example: addMonths) and call it.
|
||||
|
||||
bool function_is_plus = std::is_same<Op<UInt8, UInt8>, PlusImpl<UInt8, UInt8>>::value;
|
||||
bool function_is_minus = std::is_same<Op<UInt8, UInt8>, MinusImpl<UInt8, UInt8>>::value;
|
||||
|
||||
if (!function_is_plus && !function_is_minus)
|
||||
return {};
|
||||
|
||||
int interval_arg = 1;
|
||||
const DataTypeInterval * interval_data_type = checkAndGetDataType<DataTypeInterval>(type1.get());
|
||||
if (!interval_data_type)
|
||||
{
|
||||
interval_arg = 0;
|
||||
interval_data_type = checkAndGetDataType<DataTypeInterval>(type0.get());
|
||||
}
|
||||
if (!interval_data_type)
|
||||
return {};
|
||||
|
||||
if (interval_arg == 0 && function_is_minus)
|
||||
throw Exception("Wrong order of arguments for function " + getName() + ": argument of type Interval cannot be first.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const DataTypeDate * date_data_type = checkAndGetDataType<DataTypeDate>(interval_arg == 0 ? type1.get() : type0.get());
|
||||
const DataTypeDateTime * date_time_data_type = nullptr;
|
||||
if (!date_data_type)
|
||||
{
|
||||
date_time_data_type = checkAndGetDataType<DataTypeDateTime>(interval_arg == 0 ? type1.get() : type0.get());
|
||||
if (!date_time_data_type)
|
||||
throw Exception("Wrong argument types for function " + getName() + ": if one argument is Interval, then another must be Date or DateTime.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
std::stringstream function_name;
|
||||
function_name << (function_is_plus ? "add" : "subtract") << interval_data_type->kindToString() << 's';
|
||||
|
||||
return FunctionFactory::instance().get(function_name.str(), context);
|
||||
}
|
||||
|
||||
public:
|
||||
String getName() const override
|
||||
{
|
||||
@ -755,6 +802,21 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
/// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval.
|
||||
if (FunctionPtr function = getFunctionForIntervalArithmetic(arguments[0], arguments[1]))
|
||||
{
|
||||
DataTypes new_arguments = arguments;
|
||||
|
||||
/// Interval argument must be second.
|
||||
if (checkDataType<DataTypeInterval>(new_arguments[0].get()))
|
||||
std::swap(new_arguments[0], new_arguments[1]);
|
||||
|
||||
/// Change interval argument to its representation
|
||||
new_arguments[1] = std::make_shared<DataTypeNumber<DataTypeInterval::FieldType>>();
|
||||
|
||||
return function->getReturnTypeImpl(new_arguments);
|
||||
}
|
||||
|
||||
DataTypePtr type_res;
|
||||
|
||||
if (!( checkLeftType<DataTypeDate>(arguments, type_res)
|
||||
@ -777,6 +839,25 @@ public:
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
/// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval.
|
||||
if (FunctionPtr function = getFunctionForIntervalArithmetic(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type))
|
||||
{
|
||||
ColumnNumbers new_arguments = arguments;
|
||||
|
||||
/// Interval argument must be second.
|
||||
if (checkDataType<DataTypeInterval>(block.getByPosition(arguments[0]).type.get()))
|
||||
std::swap(new_arguments[0], new_arguments[1]);
|
||||
|
||||
/// Change interval argument type to its representation
|
||||
Block new_block = block;
|
||||
new_block.getByPosition(new_arguments[1]).type = std::make_shared<DataTypeNumber<DataTypeInterval::FieldType>>();
|
||||
|
||||
function->executeImpl(new_block, new_arguments, result);
|
||||
block.getByPosition(result).column = new_block.getByPosition(result).column;
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
if (!( executeLeftType<DataTypeDate>(block, arguments, result)
|
||||
|| executeLeftType<DataTypeDateTime>(block, arguments, result)
|
||||
|| executeLeftType<DataTypeUInt8>(block, arguments, result)
|
||||
|
@ -57,13 +57,16 @@ void registerFunctionsConversion(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionToInt64>();
|
||||
factory.registerFunction<FunctionToFloat32>();
|
||||
factory.registerFunction<FunctionToFloat64>();
|
||||
|
||||
factory.registerFunction<FunctionToDate>();
|
||||
factory.registerFunction<FunctionToDateTime>();
|
||||
factory.registerFunction<FunctionToUUID>();
|
||||
factory.registerFunction<FunctionToString>();
|
||||
factory.registerFunction<FunctionToFixedString>();
|
||||
|
||||
factory.registerFunction<FunctionToUnixTimestamp>();
|
||||
factory.registerFunction<FunctionCast>();
|
||||
|
||||
factory.registerFunction<FunctionToUInt8OrZero>();
|
||||
factory.registerFunction<FunctionToUInt16OrZero>();
|
||||
factory.registerFunction<FunctionToUInt32OrZero>();
|
||||
@ -74,6 +77,14 @@ void registerFunctionsConversion(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionToInt64OrZero>();
|
||||
factory.registerFunction<FunctionToFloat32OrZero>();
|
||||
factory.registerFunction<FunctionToFloat64OrZero>();
|
||||
|
||||
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalSecond, PositiveMonotonicity>>();
|
||||
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalMinute, PositiveMonotonicity>>();
|
||||
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalHour, PositiveMonotonicity>>();
|
||||
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalDay, PositiveMonotonicity>>();
|
||||
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalWeek, PositiveMonotonicity>>();
|
||||
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalMonth, PositiveMonotonicity>>();
|
||||
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalYear, PositiveMonotonicity>>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,6 +19,7 @@
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
@ -94,6 +95,8 @@ struct ConvertImpl
|
||||
*/
|
||||
struct ToDateTimeImpl
|
||||
{
|
||||
static constexpr auto name = "toDateTime";
|
||||
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(DayNum_t(d));
|
||||
@ -101,7 +104,7 @@ struct ToDateTimeImpl
|
||||
};
|
||||
|
||||
template <typename Name> struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
|
||||
: DateTimeTransformImpl<UInt16, UInt32, ToDateTimeImpl, Name> {};
|
||||
: DateTimeTransformImpl<UInt16, UInt32, ToDateTimeImpl> {};
|
||||
|
||||
|
||||
/// Implementation of toDate function.
|
||||
@ -109,6 +112,8 @@ template <typename Name> struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name
|
||||
template <typename FromType, typename ToType>
|
||||
struct ToDateTransform32Or64
|
||||
{
|
||||
static constexpr auto name = "toDate";
|
||||
|
||||
static inline ToType execute(const FromType & from, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return (from < 0xFFFF) ? from : time_zone.toDayNum(from);
|
||||
@ -118,7 +123,7 @@ struct ToDateTransform32Or64
|
||||
/** Conversion of DateTime to Date: throw off time component.
|
||||
*/
|
||||
template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
|
||||
: DateTimeTransformImpl<UInt32, UInt16, ToDateImpl, Name> {};
|
||||
: DateTimeTransformImpl<UInt32, UInt16, ToDateImpl> {};
|
||||
|
||||
/** Special case of converting (U)Int32 or (U)Int64 (and also, for convenience, Float32, Float64) to Date.
|
||||
* If number is less than 65536, then it is treated as DayNum, and if greater or equals, then as unix timestamp.
|
||||
@ -128,17 +133,17 @@ template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name
|
||||
* (otherwise such usage would be frequent mistake).
|
||||
*/
|
||||
template <typename Name> struct ConvertImpl<DataTypeUInt32, DataTypeDate, Name>
|
||||
: DateTimeTransformImpl<UInt32, UInt16, ToDateTransform32Or64<UInt32, UInt16>, Name> {};
|
||||
: DateTimeTransformImpl<UInt32, UInt16, ToDateTransform32Or64<UInt32, UInt16>> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeUInt64, DataTypeDate, Name>
|
||||
: DateTimeTransformImpl<UInt64, UInt16, ToDateTransform32Or64<UInt64, UInt16>, Name> {};
|
||||
: DateTimeTransformImpl<UInt64, UInt16, ToDateTransform32Or64<UInt64, UInt16>> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeInt32, DataTypeDate, Name>
|
||||
: DateTimeTransformImpl<Int32, UInt16, ToDateTransform32Or64<Int32, UInt16>, Name> {};
|
||||
: DateTimeTransformImpl<Int32, UInt16, ToDateTransform32Or64<Int32, UInt16>> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeInt64, DataTypeDate, Name>
|
||||
: DateTimeTransformImpl<Int64, UInt16, ToDateTransform32Or64<Int64, UInt16>, Name> {};
|
||||
: DateTimeTransformImpl<Int64, UInt16, ToDateTransform32Or64<Int64, UInt16>> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeFloat32, DataTypeDate, Name>
|
||||
: DateTimeTransformImpl<Float32, UInt16, ToDateTransform32Or64<Float32, UInt16>, Name> {};
|
||||
: DateTimeTransformImpl<Float32, UInt16, ToDateTransform32Or64<Float32, UInt16>> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeFloat64, DataTypeDate, Name>
|
||||
: DateTimeTransformImpl<Float64, UInt16, ToDateTransform32Or64<Float64, UInt16>, Name> {};
|
||||
: DateTimeTransformImpl<Float64, UInt16, ToDateTransform32Or64<Float64, UInt16>> {};
|
||||
|
||||
|
||||
/** Transformation of numbers, dates, datetimes to strings: through formatting.
|
||||
@ -196,7 +201,7 @@ const DateLUTImpl * extractTimeZoneFromFunctionArguments(Block & block, const Co
|
||||
|
||||
|
||||
template <typename FromDataType, typename Name>
|
||||
struct ConvertImpl<FromDataType, DataTypeString, Name>
|
||||
struct ConvertImpl<FromDataType, typename std::enable_if<!std::is_same<FromDataType, DataTypeString>::value, DataTypeString>::type, Name>
|
||||
{
|
||||
using FromFieldType = typename FromDataType::FieldType;
|
||||
|
||||
@ -261,7 +266,7 @@ struct ConvertImplGenericToString
|
||||
ColumnString::Chars_t & data_to = col_to->getChars();
|
||||
ColumnString::Offsets_t & offsets_to = col_to->getOffsets();
|
||||
|
||||
data_to.resize(size * 2); /// Using coefficient 2 for initial size is arbitary.
|
||||
data_to.resize(size * 2); /// Using coefficient 2 for initial size is arbitary.
|
||||
offsets_to.resize(size);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars_t> write_buffer(data_to);
|
||||
@ -326,7 +331,7 @@ void throwExceptionForIncompletelyParsedValue(
|
||||
|
||||
|
||||
template <typename ToDataType, typename Name>
|
||||
struct ConvertImpl<DataTypeString, ToDataType, Name>
|
||||
struct ConvertImpl<typename std::enable_if<!std::is_same<ToDataType, DataTypeString>::value, DataTypeString>::type, ToDataType, Name>
|
||||
{
|
||||
using ToFieldType = typename ToDataType::FieldType;
|
||||
|
||||
@ -479,7 +484,7 @@ struct ConvertImplGenericFromString
|
||||
|
||||
/// Function toUnixTimestamp has exactly the same implementation as toDateTime of String type.
|
||||
/// Note that time zone argument could be passed only for toUnixTimestamp function.
|
||||
struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; };
|
||||
struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; };
|
||||
|
||||
template <>
|
||||
struct ConvertImpl<DataTypeString, DataTypeUInt32, NameToUnixTimestamp>
|
||||
@ -488,8 +493,8 @@ struct ConvertImpl<DataTypeString, DataTypeUInt32, NameToUnixTimestamp>
|
||||
|
||||
/** If types are identical, just take reference to column.
|
||||
*/
|
||||
template <typename Name>
|
||||
struct ConvertImpl<DataTypeString, DataTypeString, Name>
|
||||
template <typename T, typename Name>
|
||||
struct ConvertImpl<typename std::enable_if<!T::is_parametric, T>::type, T, Name>
|
||||
{
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
@ -566,7 +571,7 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
|
||||
ColumnString::Offsets_t & offsets_to = col_to->getOffsets();
|
||||
size_t size = col_from->size();
|
||||
size_t n = col_from->getN();
|
||||
data_to.resize(size * (n + 1)); /// + 1 - zero terminator
|
||||
data_to.resize(size * (n + 1)); /// + 1 - zero terminator
|
||||
offsets_to.resize(size);
|
||||
|
||||
size_t offset_from = 0;
|
||||
@ -596,8 +601,26 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
|
||||
|
||||
|
||||
/// Declared early because used below.
|
||||
struct NameToDate { static constexpr auto name = "toDate"; };
|
||||
struct NameToString { static constexpr auto name = "toString"; };
|
||||
struct NameToDate { static constexpr auto name = "toDate"; };
|
||||
struct NameToString { static constexpr auto name = "toString"; };
|
||||
|
||||
|
||||
#define DEFINE_NAME_TO_INTERVAL(INTERVAL_KIND) \
|
||||
struct NameToInterval ## INTERVAL_KIND \
|
||||
{ \
|
||||
static constexpr auto name = "toInterval" #INTERVAL_KIND; \
|
||||
static constexpr auto kind = DataTypeInterval::INTERVAL_KIND; \
|
||||
};
|
||||
|
||||
DEFINE_NAME_TO_INTERVAL(Second)
|
||||
DEFINE_NAME_TO_INTERVAL(Minute)
|
||||
DEFINE_NAME_TO_INTERVAL(Hour)
|
||||
DEFINE_NAME_TO_INTERVAL(Day)
|
||||
DEFINE_NAME_TO_INTERVAL(Week)
|
||||
DEFINE_NAME_TO_INTERVAL(Month)
|
||||
DEFINE_NAME_TO_INTERVAL(Year)
|
||||
|
||||
#undef DEFINE_NAME_TO_INTERVAL
|
||||
|
||||
|
||||
template <typename ToDataType, typename Name, typename MonotonicityImpl>
|
||||
@ -707,9 +730,11 @@ private:
|
||||
|
||||
template <typename ToDataType2 = ToDataType, typename Name2 = Name>
|
||||
DataTypePtr getReturnTypeInternal(const DataTypes & arguments,
|
||||
typename std::enable_if<!(std::is_same<ToDataType2, DataTypeString>::value ||
|
||||
std::is_same<Name2, NameToUnixTimestamp>::value ||
|
||||
std::is_same<Name2, NameToDate>::value)>::type * = nullptr) const
|
||||
typename std::enable_if<!(
|
||||
std::is_same<ToDataType2, DataTypeString>::value
|
||||
|| std::is_same<ToDataType2, DataTypeInterval>::value
|
||||
|| std::is_same<Name2, NameToUnixTimestamp>::value
|
||||
|| std::is_same<Name2, NameToDate>::value)>::type * = nullptr) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
@ -791,6 +816,18 @@ private:
|
||||
|
||||
return std::make_shared<ToDataType2>();
|
||||
}
|
||||
|
||||
template <typename ToDataType2 = ToDataType, typename Name2 = Name>
|
||||
DataTypePtr getReturnTypeInternal(const DataTypes & arguments,
|
||||
typename std::enable_if<std::is_same<ToDataType2, DataTypeInterval>::value>::type * = nullptr) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return std::make_shared<DataTypeInterval>(Name::kind);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@ -1066,36 +1103,39 @@ struct ToStringMonotonicity
|
||||
}
|
||||
};
|
||||
|
||||
struct NameToUInt8 { static constexpr auto name = "toUInt8"; };
|
||||
struct NameToUInt16 { static constexpr auto name = "toUInt16"; };
|
||||
struct NameToUInt32 { static constexpr auto name = "toUInt32"; };
|
||||
struct NameToUInt64 { static constexpr auto name = "toUInt64"; };
|
||||
struct NameToInt8 { static constexpr auto name = "toInt8"; };
|
||||
struct NameToInt16 { static constexpr auto name = "toInt16"; };
|
||||
struct NameToInt32 { static constexpr auto name = "toInt32"; };
|
||||
struct NameToInt64 { static constexpr auto name = "toInt64"; };
|
||||
struct NameToFloat32 { static constexpr auto name = "toFloat32"; };
|
||||
struct NameToFloat64 { static constexpr auto name = "toFloat64"; };
|
||||
struct NameToDateTime { static constexpr auto name = "toDateTime"; };
|
||||
struct NameToUUID { static constexpr auto name = "toUUID"; };
|
||||
|
||||
using FunctionToUInt8 = FunctionConvert<DataTypeUInt8, NameToUInt8, ToIntMonotonicity<UInt8>>;
|
||||
using FunctionToUInt16 = FunctionConvert<DataTypeUInt16, NameToUInt16, ToIntMonotonicity<UInt16>>;
|
||||
using FunctionToUInt32 = FunctionConvert<DataTypeUInt32, NameToUInt32, ToIntMonotonicity<UInt32>>;
|
||||
using FunctionToUInt64 = FunctionConvert<DataTypeUInt64, NameToUInt64, ToIntMonotonicity<UInt64>>;
|
||||
using FunctionToInt8 = FunctionConvert<DataTypeInt8, NameToInt8, ToIntMonotonicity<Int8>>;
|
||||
using FunctionToInt16 = FunctionConvert<DataTypeInt16, NameToInt16, ToIntMonotonicity<Int16>>;
|
||||
using FunctionToInt32 = FunctionConvert<DataTypeInt32, NameToInt32, ToIntMonotonicity<Int32>>;
|
||||
using FunctionToInt64 = FunctionConvert<DataTypeInt64, NameToInt64, ToIntMonotonicity<Int64>>;
|
||||
using FunctionToFloat32 = FunctionConvert<DataTypeFloat32, NameToFloat32, PositiveMonotonicity>;
|
||||
using FunctionToFloat64 = FunctionConvert<DataTypeFloat64, NameToFloat64, PositiveMonotonicity>;
|
||||
using FunctionToDate = FunctionConvert<DataTypeDate, NameToDate, ToIntMonotonicity<UInt16>>;
|
||||
using FunctionToDateTime = FunctionConvert<DataTypeDateTime, NameToDateTime, ToIntMonotonicity<UInt32>>;
|
||||
using FunctionToUUID = FunctionConvert<DataTypeUUID, NameToUUID, ToIntMonotonicity<UInt128>>;
|
||||
using FunctionToString = FunctionConvert<DataTypeString, NameToString, ToStringMonotonicity>;
|
||||
using FunctionToUnixTimestamp = FunctionConvert<DataTypeUInt32, NameToUnixTimestamp, ToIntMonotonicity<UInt32>>;
|
||||
struct NameToUInt8 { static constexpr auto name = "toUInt8"; };
|
||||
struct NameToUInt16 { static constexpr auto name = "toUInt16"; };
|
||||
struct NameToUInt32 { static constexpr auto name = "toUInt32"; };
|
||||
struct NameToUInt64 { static constexpr auto name = "toUInt64"; };
|
||||
struct NameToInt8 { static constexpr auto name = "toInt8"; };
|
||||
struct NameToInt16 { static constexpr auto name = "toInt16"; };
|
||||
struct NameToInt32 { static constexpr auto name = "toInt32"; };
|
||||
struct NameToInt64 { static constexpr auto name = "toInt64"; };
|
||||
struct NameToFloat32 { static constexpr auto name = "toFloat32"; };
|
||||
struct NameToFloat64 { static constexpr auto name = "toFloat64"; };
|
||||
struct NameToDateTime { static constexpr auto name = "toDateTime"; };
|
||||
struct NameToUUID { static constexpr auto name = "toUUID"; };
|
||||
|
||||
using FunctionToUInt8 = FunctionConvert<DataTypeUInt8, NameToUInt8, ToIntMonotonicity<UInt8>>;
|
||||
using FunctionToUInt16 = FunctionConvert<DataTypeUInt16, NameToUInt16, ToIntMonotonicity<UInt16>>;
|
||||
using FunctionToUInt32 = FunctionConvert<DataTypeUInt32, NameToUInt32, ToIntMonotonicity<UInt32>>;
|
||||
using FunctionToUInt64 = FunctionConvert<DataTypeUInt64, NameToUInt64, ToIntMonotonicity<UInt64>>;
|
||||
using FunctionToInt8 = FunctionConvert<DataTypeInt8, NameToInt8, ToIntMonotonicity<Int8>>;
|
||||
using FunctionToInt16 = FunctionConvert<DataTypeInt16, NameToInt16, ToIntMonotonicity<Int16>>;
|
||||
using FunctionToInt32 = FunctionConvert<DataTypeInt32, NameToInt32, ToIntMonotonicity<Int32>>;
|
||||
using FunctionToInt64 = FunctionConvert<DataTypeInt64, NameToInt64, ToIntMonotonicity<Int64>>;
|
||||
using FunctionToFloat32 = FunctionConvert<DataTypeFloat32, NameToFloat32, PositiveMonotonicity>;
|
||||
using FunctionToFloat64 = FunctionConvert<DataTypeFloat64, NameToFloat64, PositiveMonotonicity>;
|
||||
using FunctionToDate = FunctionConvert<DataTypeDate, NameToDate, ToIntMonotonicity<UInt16>>;
|
||||
using FunctionToDateTime = FunctionConvert<DataTypeDateTime, NameToDateTime, ToIntMonotonicity<UInt32>>;
|
||||
using FunctionToUUID = FunctionConvert<DataTypeUUID, NameToUUID, ToIntMonotonicity<UInt128>>;
|
||||
using FunctionToString = FunctionConvert<DataTypeString, NameToString, ToStringMonotonicity>;
|
||||
using FunctionToUnixTimestamp = FunctionConvert<DataTypeUInt32, NameToUnixTimestamp, ToIntMonotonicity<UInt32>>;
|
||||
|
||||
|
||||
template <typename DataType> struct FunctionTo;
|
||||
|
||||
template <> struct FunctionTo<DataTypeUInt8> { using Type = FunctionToUInt8; };
|
||||
template <> struct FunctionTo<DataTypeUInt16> { using Type = FunctionToUInt16; };
|
||||
template <> struct FunctionTo<DataTypeUInt32> { using Type = FunctionToUInt32; };
|
||||
@ -1111,32 +1151,33 @@ template <> struct FunctionTo<DataTypeDateTime> { using Type = FunctionToDateTim
|
||||
template <> struct FunctionTo<DataTypeUUID> { using Type = FunctionToUUID; };
|
||||
template <> struct FunctionTo<DataTypeString> { using Type = FunctionToString; };
|
||||
template <> struct FunctionTo<DataTypeFixedString> { using Type = FunctionToFixedString; };
|
||||
|
||||
template <typename FieldType> struct FunctionTo<DataTypeEnum<FieldType>>
|
||||
: FunctionTo<DataTypeNumber<FieldType>>
|
||||
{
|
||||
};
|
||||
|
||||
struct NameToUInt8OrZero { static constexpr auto name = "toUInt8OrZero"; };
|
||||
struct NameToUInt16OrZero { static constexpr auto name = "toUInt16OrZero"; };
|
||||
struct NameToUInt32OrZero { static constexpr auto name = "toUInt32OrZero"; };
|
||||
struct NameToUInt64OrZero { static constexpr auto name = "toUInt64OrZero"; };
|
||||
struct NameToInt8OrZero { static constexpr auto name = "toInt8OrZero"; };
|
||||
struct NameToInt16OrZero { static constexpr auto name = "toInt16OrZero"; };
|
||||
struct NameToInt32OrZero { static constexpr auto name = "toInt32OrZero"; };
|
||||
struct NameToInt64OrZero { static constexpr auto name = "toInt64OrZero"; };
|
||||
struct NameToFloat32OrZero { static constexpr auto name = "toFloat32OrZero"; };
|
||||
struct NameToFloat64OrZero { static constexpr auto name = "toFloat64OrZero"; };
|
||||
struct NameToUInt8OrZero { static constexpr auto name = "toUInt8OrZero"; };
|
||||
struct NameToUInt16OrZero { static constexpr auto name = "toUInt16OrZero"; };
|
||||
struct NameToUInt32OrZero { static constexpr auto name = "toUInt32OrZero"; };
|
||||
struct NameToUInt64OrZero { static constexpr auto name = "toUInt64OrZero"; };
|
||||
struct NameToInt8OrZero { static constexpr auto name = "toInt8OrZero"; };
|
||||
struct NameToInt16OrZero { static constexpr auto name = "toInt16OrZero"; };
|
||||
struct NameToInt32OrZero { static constexpr auto name = "toInt32OrZero"; };
|
||||
struct NameToInt64OrZero { static constexpr auto name = "toInt64OrZero"; };
|
||||
struct NameToFloat32OrZero { static constexpr auto name = "toFloat32OrZero"; };
|
||||
struct NameToFloat64OrZero { static constexpr auto name = "toFloat64OrZero"; };
|
||||
|
||||
using FunctionToUInt8OrZero = FunctionConvertOrZero<DataTypeUInt8, NameToUInt8OrZero>;
|
||||
using FunctionToUInt16OrZero = FunctionConvertOrZero<DataTypeUInt16, NameToUInt16OrZero>;
|
||||
using FunctionToUInt32OrZero = FunctionConvertOrZero<DataTypeUInt32, NameToUInt32OrZero>;
|
||||
using FunctionToUInt64OrZero = FunctionConvertOrZero<DataTypeUInt64, NameToUInt64OrZero>;
|
||||
using FunctionToInt8OrZero = FunctionConvertOrZero<DataTypeInt8, NameToInt8OrZero>;
|
||||
using FunctionToInt16OrZero = FunctionConvertOrZero<DataTypeInt16, NameToInt16OrZero>;
|
||||
using FunctionToInt32OrZero = FunctionConvertOrZero<DataTypeInt32, NameToInt32OrZero>;
|
||||
using FunctionToInt64OrZero = FunctionConvertOrZero<DataTypeInt64, NameToInt64OrZero>;
|
||||
using FunctionToFloat32OrZero = FunctionConvertOrZero<DataTypeFloat32, NameToFloat32OrZero>;
|
||||
using FunctionToFloat64OrZero = FunctionConvertOrZero<DataTypeFloat64, NameToFloat64OrZero>;
|
||||
using FunctionToUInt8OrZero = FunctionConvertOrZero<DataTypeUInt8, NameToUInt8OrZero>;
|
||||
using FunctionToUInt16OrZero = FunctionConvertOrZero<DataTypeUInt16, NameToUInt16OrZero>;
|
||||
using FunctionToUInt32OrZero = FunctionConvertOrZero<DataTypeUInt32, NameToUInt32OrZero>;
|
||||
using FunctionToUInt64OrZero = FunctionConvertOrZero<DataTypeUInt64, NameToUInt64OrZero>;
|
||||
using FunctionToInt8OrZero = FunctionConvertOrZero<DataTypeInt8, NameToInt8OrZero>;
|
||||
using FunctionToInt16OrZero = FunctionConvertOrZero<DataTypeInt16, NameToInt16OrZero>;
|
||||
using FunctionToInt32OrZero = FunctionConvertOrZero<DataTypeInt32, NameToInt32OrZero>;
|
||||
using FunctionToInt64OrZero = FunctionConvertOrZero<DataTypeInt64, NameToInt64OrZero>;
|
||||
using FunctionToFloat32OrZero = FunctionConvertOrZero<DataTypeFloat32, NameToFloat32OrZero>;
|
||||
using FunctionToFloat64OrZero = FunctionConvertOrZero<DataTypeFloat64, NameToFloat64OrZero>;
|
||||
|
||||
|
||||
class FunctionCast final : public IFunction
|
||||
|
@ -37,6 +37,22 @@ void registerFunctionsDateTime(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionToYYYYMM>();
|
||||
factory.registerFunction<FunctionToYYYYMMDD>();
|
||||
factory.registerFunction<FunctionToYYYYMMDDhhmmss>();
|
||||
|
||||
factory.registerFunction<FunctionAddSeconds>();
|
||||
factory.registerFunction<FunctionAddMinutes>();
|
||||
factory.registerFunction<FunctionAddHours>();
|
||||
factory.registerFunction<FunctionAddDays>();
|
||||
factory.registerFunction<FunctionAddWeeks>();
|
||||
factory.registerFunction<FunctionAddMonths>();
|
||||
factory.registerFunction<FunctionAddYears>();
|
||||
|
||||
factory.registerFunction<FunctionSubtractSeconds>();
|
||||
factory.registerFunction<FunctionSubtractMinutes>();
|
||||
factory.registerFunction<FunctionSubtractHours>();
|
||||
factory.registerFunction<FunctionSubtractDays>();
|
||||
factory.registerFunction<FunctionSubtractWeeks>();
|
||||
factory.registerFunction<FunctionSubtractMonths>();
|
||||
factory.registerFunction<FunctionSubtractYears>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -85,6 +85,8 @@ struct ZeroTransform
|
||||
|
||||
struct ToDateImpl
|
||||
{
|
||||
static constexpr auto name = "toDate";
|
||||
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return UInt16(time_zone.toDayNum(t));
|
||||
@ -99,6 +101,8 @@ struct ToDateImpl
|
||||
|
||||
struct ToStartOfDayImpl
|
||||
{
|
||||
static constexpr auto name = "toStartOfDay";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDate(t);
|
||||
@ -113,6 +117,8 @@ struct ToStartOfDayImpl
|
||||
|
||||
struct ToMondayImpl
|
||||
{
|
||||
static constexpr auto name = "toMonday";
|
||||
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t));
|
||||
@ -127,6 +133,8 @@ struct ToMondayImpl
|
||||
|
||||
struct ToStartOfMonthImpl
|
||||
{
|
||||
static constexpr auto name = "toStartOfMonth";
|
||||
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfMonth(time_zone.toDayNum(t));
|
||||
@ -141,6 +149,8 @@ struct ToStartOfMonthImpl
|
||||
|
||||
struct ToStartOfQuarterImpl
|
||||
{
|
||||
static constexpr auto name = "toStartOfQuarter";
|
||||
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfQuarter(time_zone.toDayNum(t));
|
||||
@ -155,6 +165,8 @@ struct ToStartOfQuarterImpl
|
||||
|
||||
struct ToStartOfYearImpl
|
||||
{
|
||||
static constexpr auto name = "toStartOfYear";
|
||||
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfYear(time_zone.toDayNum(t));
|
||||
@ -170,6 +182,8 @@ struct ToStartOfYearImpl
|
||||
|
||||
struct ToTimeImpl
|
||||
{
|
||||
static constexpr auto name = "toTime";
|
||||
|
||||
/// When transforming to time, the date will be equated to 1970-01-02.
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
@ -186,9 +200,11 @@ struct ToTimeImpl
|
||||
|
||||
struct ToStartOfMinuteImpl
|
||||
{
|
||||
static constexpr auto name = "toStartOfMinute";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfMinuteInaccurate(t);
|
||||
return time_zone.toStartOfMinute(t);
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
@ -200,9 +216,11 @@ struct ToStartOfMinuteImpl
|
||||
|
||||
struct ToStartOfFiveMinuteImpl
|
||||
{
|
||||
static constexpr auto name = "toStartOfFiveMinute";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfFiveMinuteInaccurate(t);
|
||||
return time_zone.toStartOfFiveMinute(t);
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
@ -214,10 +232,13 @@ struct ToStartOfFiveMinuteImpl
|
||||
|
||||
struct ToStartOfHourImpl
|
||||
{
|
||||
static constexpr auto name = "toStartOfHour";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfHourInaccurate(t);
|
||||
return time_zone.toStartOfHour(t);
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toStartOfHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -228,6 +249,8 @@ struct ToStartOfHourImpl
|
||||
|
||||
struct ToYearImpl
|
||||
{
|
||||
static constexpr auto name = "toYear";
|
||||
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toYear(t);
|
||||
@ -242,6 +265,8 @@ struct ToYearImpl
|
||||
|
||||
struct ToMonthImpl
|
||||
{
|
||||
static constexpr auto name = "toMonth";
|
||||
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toMonth(t);
|
||||
@ -256,6 +281,8 @@ struct ToMonthImpl
|
||||
|
||||
struct ToDayOfMonthImpl
|
||||
{
|
||||
static constexpr auto name = "toDayOfMonth";
|
||||
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfMonth(t);
|
||||
@ -270,6 +297,8 @@ struct ToDayOfMonthImpl
|
||||
|
||||
struct ToDayOfWeekImpl
|
||||
{
|
||||
static constexpr auto name = "toDayOfWeek";
|
||||
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfWeek(t);
|
||||
@ -284,10 +313,13 @@ struct ToDayOfWeekImpl
|
||||
|
||||
struct ToHourImpl
|
||||
{
|
||||
static constexpr auto name = "toHour";
|
||||
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toHour(t);
|
||||
}
|
||||
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -298,9 +330,11 @@ struct ToHourImpl
|
||||
|
||||
struct ToMinuteImpl
|
||||
{
|
||||
static constexpr auto name = "toMinute";
|
||||
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toMinuteInaccurate(t);
|
||||
return time_zone.toMinute(t);
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
@ -312,9 +346,11 @@ struct ToMinuteImpl
|
||||
|
||||
struct ToSecondImpl
|
||||
{
|
||||
static constexpr auto name = "toSecond";
|
||||
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toSecondInaccurate(t);
|
||||
return time_zone.toSecond(t);
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
@ -326,6 +362,8 @@ struct ToSecondImpl
|
||||
|
||||
struct ToRelativeYearNumImpl
|
||||
{
|
||||
static constexpr auto name = "toRelativeYearNum";
|
||||
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toYear(t);
|
||||
@ -340,6 +378,8 @@ struct ToRelativeYearNumImpl
|
||||
|
||||
struct ToRelativeMonthNumImpl
|
||||
{
|
||||
static constexpr auto name = "toRelativeMonthNum";
|
||||
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeMonthNum(t);
|
||||
@ -354,6 +394,8 @@ struct ToRelativeMonthNumImpl
|
||||
|
||||
struct ToRelativeWeekNumImpl
|
||||
{
|
||||
static constexpr auto name = "toRelativeWeekNum";
|
||||
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeWeekNum(t);
|
||||
@ -368,6 +410,8 @@ struct ToRelativeWeekNumImpl
|
||||
|
||||
struct ToRelativeDayNumImpl
|
||||
{
|
||||
static constexpr auto name = "toRelativeDayNum";
|
||||
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayNum(t);
|
||||
@ -383,6 +427,8 @@ struct ToRelativeDayNumImpl
|
||||
|
||||
struct ToRelativeHourNumImpl
|
||||
{
|
||||
static constexpr auto name = "toRelativeHourNum";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeHourNum(t);
|
||||
@ -397,6 +443,8 @@ struct ToRelativeHourNumImpl
|
||||
|
||||
struct ToRelativeMinuteNumImpl
|
||||
{
|
||||
static constexpr auto name = "toRelativeMinuteNum";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeMinuteNum(t);
|
||||
@ -411,6 +459,8 @@ struct ToRelativeMinuteNumImpl
|
||||
|
||||
struct ToRelativeSecondNumImpl
|
||||
{
|
||||
static constexpr auto name = "toRelativeSecondNum";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return t;
|
||||
@ -425,6 +475,8 @@ struct ToRelativeSecondNumImpl
|
||||
|
||||
struct ToYYYYMMImpl
|
||||
{
|
||||
static constexpr auto name = "toYYYYMM";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMM(t);
|
||||
@ -439,6 +491,8 @@ struct ToYYYYMMImpl
|
||||
|
||||
struct ToYYYYMMDDImpl
|
||||
{
|
||||
static constexpr auto name = "toYYYYMMDD";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMMDD(t);
|
||||
@ -453,6 +507,8 @@ struct ToYYYYMMDDImpl
|
||||
|
||||
struct ToYYYYMMDDhhmmssImpl
|
||||
{
|
||||
static constexpr auto name = "toYYYYMMDDhhmmss";
|
||||
|
||||
static inline UInt64 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMMDDhhmmss(t);
|
||||
@ -477,24 +533,16 @@ struct Transformer
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::execute(vec_from[i], time_zone);
|
||||
}
|
||||
|
||||
static void constant(const FromType & from, ToType & to, const DateLUTImpl & time_zone)
|
||||
{
|
||||
to = Transform::execute(from, time_zone);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename FromType, typename ToType, typename Transform, typename Name>
|
||||
template <typename FromType, typename ToType, typename Transform>
|
||||
struct DateTimeTransformImpl
|
||||
{
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
using Op = Transformer<FromType, ToType, Transform>;
|
||||
|
||||
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
||||
const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get());
|
||||
|
||||
const ColumnConst * time_zone_column = nullptr;
|
||||
|
||||
if (arguments.size() == 2)
|
||||
@ -503,7 +551,7 @@ struct DateTimeTransformImpl
|
||||
|
||||
if (!time_zone_column)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of second (time zone) argument of function " + Name::name + ", must be constant string",
|
||||
+ " of second (time zone) argument of function " + Transform::name + ", must be constant string",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
@ -511,7 +559,8 @@ struct DateTimeTransformImpl
|
||||
? DateLUT::instance(time_zone_column->getValue<String>())
|
||||
: DateLUT::instance();
|
||||
|
||||
if (sources)
|
||||
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
||||
if (const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get()))
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnVector<ToType>>();
|
||||
block.getByPosition(result).column = col_to;
|
||||
@ -520,18 +569,18 @@ struct DateTimeTransformImpl
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
+ " of first argument of function " + Transform::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename ToDataType, typename Transform, typename Name>
|
||||
template <typename ToDataType, typename Transform>
|
||||
class FunctionDateOrDateTimeToSomething : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static constexpr auto name = Transform::name;
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionDateOrDateTimeToSomething>(); };
|
||||
|
||||
String getName() const override
|
||||
@ -576,12 +625,12 @@ public:
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
||||
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
||||
|
||||
if (checkDataType<DataTypeDate>(from_type))
|
||||
DateTimeTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform, Name>::execute(block, arguments, result);
|
||||
DateTimeTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result);
|
||||
else if (checkDataType<DataTypeDateTime>(from_type))
|
||||
DateTimeTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform, Name>::execute(block, arguments, result);
|
||||
DateTimeTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result);
|
||||
else
|
||||
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -628,6 +677,296 @@ public:
|
||||
};
|
||||
|
||||
|
||||
struct AddSecondsImpl
|
||||
{
|
||||
static constexpr auto name = "addSeconds";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return t + delta;
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(DayNum_t(d)) + delta;
|
||||
}
|
||||
};
|
||||
|
||||
struct AddMinutesImpl
|
||||
{
|
||||
static constexpr auto name = "addMinutes";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return t + delta * 60;
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(DayNum_t(d)) + delta * 60;
|
||||
}
|
||||
};
|
||||
|
||||
struct AddHoursImpl
|
||||
{
|
||||
static constexpr auto name = "addHours";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return t + delta * 3600;
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(DayNum_t(d)) + delta * 3600;
|
||||
}
|
||||
};
|
||||
|
||||
struct AddDaysImpl
|
||||
{
|
||||
static constexpr auto name = "addDays";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addDays(t, delta);
|
||||
}
|
||||
|
||||
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return d + delta;
|
||||
}
|
||||
};
|
||||
|
||||
struct AddWeeksImpl
|
||||
{
|
||||
static constexpr auto name = "addWeeks";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addWeeks(t, delta);
|
||||
}
|
||||
|
||||
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return d + delta * 7;
|
||||
}
|
||||
};
|
||||
|
||||
struct AddMonthsImpl
|
||||
{
|
||||
static constexpr auto name = "addMonths";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addMonths(t, delta);
|
||||
}
|
||||
|
||||
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addMonths(DayNum_t(d), delta);
|
||||
}
|
||||
};
|
||||
|
||||
struct AddYearsImpl
|
||||
{
|
||||
static constexpr auto name = "addYears";
|
||||
|
||||
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addYears(t, delta);
|
||||
}
|
||||
|
||||
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addYears(DayNum_t(d), delta);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename Transform>
|
||||
struct SubtractIntervalImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return Transform::execute(t, -delta, time_zone);
|
||||
}
|
||||
|
||||
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return Transform::execute(d, -delta, time_zone);
|
||||
}
|
||||
};
|
||||
|
||||
struct SubtractSecondsImpl : SubtractIntervalImpl<AddSecondsImpl> { static constexpr auto name = "subtractSeconds"; };
|
||||
struct SubtractMinutesImpl : SubtractIntervalImpl<AddMinutesImpl> { static constexpr auto name = "subtractMinutes"; };
|
||||
struct SubtractHoursImpl : SubtractIntervalImpl<AddHoursImpl> { static constexpr auto name = "subtractHours"; };
|
||||
struct SubtractDaysImpl : SubtractIntervalImpl<AddDaysImpl> { static constexpr auto name = "subtractDays"; };
|
||||
struct SubtractWeeksImpl : SubtractIntervalImpl<AddWeeksImpl> { static constexpr auto name = "subtractWeeks"; };
|
||||
struct SubtractMonthsImpl : SubtractIntervalImpl<AddMonthsImpl> { static constexpr auto name = "subtractMonths"; };
|
||||
struct SubtractYearsImpl : SubtractIntervalImpl<AddYearsImpl> { static constexpr auto name = "subtractYears"; };
|
||||
|
||||
|
||||
template <typename FromType, typename ToType, typename Transform>
|
||||
struct Adder
|
||||
{
|
||||
static void vector_vector(const PaddedPODArray<FromType> & vec_from, PaddedPODArray<ToType> & vec_to, const IColumn & delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
size_t size = vec_from.size();
|
||||
vec_to.resize(size);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::execute(vec_from[i], delta.getInt(i), time_zone);
|
||||
}
|
||||
|
||||
static void vector_constant(const PaddedPODArray<FromType> & vec_from, PaddedPODArray<ToType> & vec_to, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
size_t size = vec_from.size();
|
||||
vec_to.resize(size);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::execute(vec_from[i], delta, time_zone);
|
||||
}
|
||||
|
||||
static void constant_vector(const FromType & from, PaddedPODArray<ToType> & vec_to, const IColumn & delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
size_t size = delta.size();
|
||||
vec_to.resize(size);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::execute(from, delta.getInt(i), time_zone);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename FromType, typename Transform>
|
||||
struct DateTimeAddIntervalImpl
|
||||
{
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
using ToType = decltype(Transform::execute(FromType(), 0, std::declval<DateLUTImpl>()));
|
||||
using Op = Adder<FromType, ToType, Transform>;
|
||||
|
||||
const ColumnConst * time_zone_column = nullptr;
|
||||
|
||||
if (arguments.size() == 3)
|
||||
{
|
||||
time_zone_column = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[2]).column.get());
|
||||
|
||||
if (!time_zone_column)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
|
||||
+ " of second (time zone) argument of function " + Transform::name + ", must be constant string",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
const DateLUTImpl & time_zone = time_zone_column
|
||||
? DateLUT::instance(time_zone_column->getValue<String>())
|
||||
: DateLUT::instance();
|
||||
|
||||
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
||||
|
||||
if (const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get()))
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnVector<ToType>>();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const IColumn & delta_column = *block.getByPosition(arguments[1]).column;
|
||||
|
||||
if (const auto * delta_const_column = typeid_cast<const ColumnConst *>(&delta_column))
|
||||
Op::vector_constant(sources->getData(), col_to->getData(), delta_const_column->getValue<Int64>(), time_zone);
|
||||
else
|
||||
Op::vector_vector(sources->getData(), col_to->getData(), delta_column, time_zone);
|
||||
}
|
||||
else if (const auto * sources = checkAndGetColumnConst<ColumnVector<FromType>>(source_col.get()))
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnVector<ToType>>();
|
||||
block.getByPosition(result).column = col_to;
|
||||
Op::constant_vector(sources->template getValue<FromType>(), col_to->getData(), *block.getByPosition(arguments[1]).column, time_zone);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Transform::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename Transform>
|
||||
class FunctionDateOrDateTimeAddInterval : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Transform::name;
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionDateOrDateTimeAddInterval>(); };
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 2 && arguments.size() != 3)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 2 or 3",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!arguments[1]->behavesAsNumber())
|
||||
throw Exception("Second argument for function " + getName() + " (delta) must be number",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
if (!checkDataType<DataTypeDate>(arguments[0].get())
|
||||
&& !checkDataType<DataTypeDateTime>(arguments[0].get()))
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName() +
|
||||
". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!checkDataType<DataTypeDateTime>(arguments[0].get())
|
||||
|| !checkDataType<DataTypeString>(arguments[1].get()))
|
||||
throw Exception{
|
||||
"Function " + getName() + " supports 2 or 3 arguments. The 1st argument "
|
||||
"must be of type Date or DateTime. The 2nd argument must be number. "
|
||||
"The 3rd argument (optional) must be "
|
||||
"a constant string with timezone name. The timezone argument is allowed "
|
||||
"only when the 1st argument has the type DateTime",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
|
||||
if (checkDataType<DataTypeDate>(arguments[0].get()))
|
||||
return std::make_shared<typename std::conditional<std::is_same<
|
||||
decltype(Transform::execute(DataTypeDate::FieldType(), 0, std::declval<DateLUTImpl>())), UInt16>::value,
|
||||
DataTypeDate, DataTypeDateTime>::type>();
|
||||
else
|
||||
return std::make_shared<typename std::conditional<std::is_same<
|
||||
decltype(Transform::execute(DataTypeDateTime::FieldType(), 0, std::declval<DateLUTImpl>())), UInt16>::value,
|
||||
DataTypeDate, DataTypeDateTime>::type>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {2}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
||||
|
||||
if (checkDataType<DataTypeDate>(from_type))
|
||||
DateTimeAddIntervalImpl<DataTypeDate::FieldType, Transform>::execute(block, arguments, result);
|
||||
else if (checkDataType<DataTypeDateTime>(from_type))
|
||||
DateTimeAddIntervalImpl<DataTypeDateTime::FieldType, Transform>::execute(block, arguments, result);
|
||||
else
|
||||
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Get the current time. (It is a constant, it is evaluated once for the entire query.)
|
||||
class FunctionNow : public IFunction
|
||||
{
|
||||
@ -906,62 +1245,50 @@ public:
|
||||
};
|
||||
|
||||
|
||||
struct NameToYear { static constexpr auto name = "toYear"; };
|
||||
struct NameToMonth { static constexpr auto name = "toMonth"; };
|
||||
struct NameToDayOfMonth { static constexpr auto name = "toDayOfMonth"; };
|
||||
struct NameToDayOfWeek { static constexpr auto name = "toDayOfWeek"; };
|
||||
struct NameToHour { static constexpr auto name = "toHour"; };
|
||||
struct NameToMinute { static constexpr auto name = "toMinute"; };
|
||||
struct NameToSecond { static constexpr auto name = "toSecond"; };
|
||||
struct NameToStartOfDay { static constexpr auto name = "toStartOfDay"; };
|
||||
struct NameToMonday { static constexpr auto name = "toMonday"; };
|
||||
struct NameToStartOfMonth { static constexpr auto name = "toStartOfMonth"; };
|
||||
struct NameToStartOfQuarter { static constexpr auto name = "toStartOfQuarter"; };
|
||||
struct NameToStartOfYear { static constexpr auto name = "toStartOfYear"; };
|
||||
struct NameToStartOfMinute { static constexpr auto name = "toStartOfMinute"; };
|
||||
struct NameToStartOfFiveMinute { static constexpr auto name = "toStartOfFiveMinute"; };
|
||||
struct NameToStartOfHour { static constexpr auto name = "toStartOfHour"; };
|
||||
struct NameToTime { static constexpr auto name = "toTime"; };
|
||||
struct NameToRelativeYearNum { static constexpr auto name = "toRelativeYearNum"; };
|
||||
struct NameToRelativeMonthNum { static constexpr auto name = "toRelativeMonthNum"; };
|
||||
struct NameToRelativeWeekNum { static constexpr auto name = "toRelativeWeekNum"; };
|
||||
struct NameToRelativeDayNum { static constexpr auto name = "toRelativeDayNum"; };
|
||||
struct NameToRelativeHourNum { static constexpr auto name = "toRelativeHourNum"; };
|
||||
struct NameToRelativeMinuteNum { static constexpr auto name = "toRelativeMinuteNum"; };
|
||||
struct NameToRelativeSecondNum { static constexpr auto name = "toRelativeSecondNum"; };
|
||||
struct NameToYYYYMM { static constexpr auto name = "toYYYYMM"; };
|
||||
struct NameToYYYYMMDD { static constexpr auto name = "toYYYYMMDD"; };
|
||||
struct NameToYYYYMMDDhhmmss { static constexpr auto name = "toYYYYMMDDhhmmss"; };
|
||||
using FunctionToYear = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToYearImpl>;
|
||||
using FunctionToMonth = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToMonthImpl>;
|
||||
using FunctionToDayOfMonth = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfMonthImpl>;
|
||||
using FunctionToDayOfWeek = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfWeekImpl>;
|
||||
using FunctionToHour = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToHourImpl>;
|
||||
using FunctionToMinute = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToMinuteImpl>;
|
||||
using FunctionToSecond = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToSecondImpl>;
|
||||
using FunctionToStartOfDay = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfDayImpl>;
|
||||
using FunctionToMonday = FunctionDateOrDateTimeToSomething<DataTypeDate, ToMondayImpl>;
|
||||
using FunctionToStartOfMonth = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfMonthImpl>;
|
||||
using FunctionToStartOfQuarter = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfQuarterImpl>;
|
||||
using FunctionToStartOfYear = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfYearImpl>;
|
||||
using FunctionToStartOfMinute = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfMinuteImpl>;
|
||||
using FunctionToStartOfFiveMinute = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfFiveMinuteImpl>;
|
||||
using FunctionToStartOfHour = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfHourImpl>;
|
||||
using FunctionToTime = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToTimeImpl>;
|
||||
|
||||
using FunctionToRelativeYearNum = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToRelativeYearNumImpl>;
|
||||
using FunctionToRelativeMonthNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeMonthNumImpl>;
|
||||
using FunctionToRelativeWeekNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeWeekNumImpl>;
|
||||
using FunctionToRelativeDayNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeDayNumImpl>;
|
||||
|
||||
using FunctionToYear = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToYearImpl, NameToYear>;
|
||||
using FunctionToMonth = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToMonthImpl, NameToMonth>;
|
||||
using FunctionToDayOfMonth = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfMonthImpl, NameToDayOfMonth>;
|
||||
using FunctionToDayOfWeek = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfWeekImpl, NameToDayOfWeek>;
|
||||
using FunctionToHour = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToHourImpl, NameToHour>;
|
||||
using FunctionToMinute = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToMinuteImpl, NameToMinute>;
|
||||
using FunctionToSecond = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToSecondImpl, NameToSecond>;
|
||||
using FunctionToStartOfDay = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfDayImpl, NameToStartOfDay>;
|
||||
using FunctionToMonday = FunctionDateOrDateTimeToSomething<DataTypeDate, ToMondayImpl, NameToMonday>;
|
||||
using FunctionToStartOfMonth = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfMonthImpl, NameToStartOfMonth>;
|
||||
using FunctionToStartOfQuarter = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfQuarterImpl, NameToStartOfQuarter>;
|
||||
using FunctionToStartOfYear = FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfYearImpl, NameToStartOfYear>;
|
||||
using FunctionToStartOfMinute = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfMinuteImpl, NameToStartOfMinute>;
|
||||
using FunctionToStartOfFiveMinute = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfFiveMinuteImpl, NameToStartOfFiveMinute>;
|
||||
using FunctionToStartOfHour = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfHourImpl, NameToStartOfHour>;
|
||||
using FunctionToTime = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToTimeImpl, NameToTime>;
|
||||
using FunctionToRelativeHourNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeHourNumImpl>;
|
||||
using FunctionToRelativeMinuteNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeMinuteNumImpl>;
|
||||
using FunctionToRelativeSecondNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeSecondNumImpl>;
|
||||
|
||||
using FunctionToRelativeYearNum = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToRelativeYearNumImpl, NameToRelativeYearNum>;
|
||||
using FunctionToRelativeMonthNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeMonthNumImpl, NameToRelativeMonthNum>;
|
||||
using FunctionToRelativeWeekNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeWeekNumImpl, NameToRelativeWeekNum>;
|
||||
using FunctionToRelativeDayNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeDayNumImpl, NameToRelativeDayNum>;
|
||||
using FunctionToYYYYMM = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToYYYYMMImpl>;
|
||||
using FunctionToYYYYMMDD = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToYYYYMMDDImpl>;
|
||||
using FunctionToYYYYMMDDhhmmss = FunctionDateOrDateTimeToSomething<DataTypeUInt64, ToYYYYMMDDhhmmssImpl>;
|
||||
|
||||
using FunctionToRelativeHourNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeHourNumImpl, NameToRelativeHourNum>;
|
||||
using FunctionToRelativeMinuteNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeMinuteNumImpl, NameToRelativeMinuteNum>;
|
||||
using FunctionToRelativeSecondNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeSecondNumImpl, NameToRelativeSecondNum>;
|
||||
using FunctionAddSeconds = FunctionDateOrDateTimeAddInterval<AddSecondsImpl>;
|
||||
using FunctionAddMinutes = FunctionDateOrDateTimeAddInterval<AddMinutesImpl>;
|
||||
using FunctionAddHours = FunctionDateOrDateTimeAddInterval<AddHoursImpl>;
|
||||
using FunctionAddDays = FunctionDateOrDateTimeAddInterval<AddDaysImpl>;
|
||||
using FunctionAddWeeks = FunctionDateOrDateTimeAddInterval<AddWeeksImpl>;
|
||||
using FunctionAddMonths = FunctionDateOrDateTimeAddInterval<AddMonthsImpl>;
|
||||
using FunctionAddYears = FunctionDateOrDateTimeAddInterval<AddYearsImpl>;
|
||||
|
||||
using FunctionToYYYYMM = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToYYYYMMImpl, NameToYYYYMM>;
|
||||
using FunctionToYYYYMMDD = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToYYYYMMDDImpl, NameToYYYYMMDD>;
|
||||
using FunctionToYYYYMMDDhhmmss = FunctionDateOrDateTimeToSomething<DataTypeUInt64, ToYYYYMMDDhhmmssImpl, NameToYYYYMMDDhhmmss>;
|
||||
using FunctionSubtractSeconds = FunctionDateOrDateTimeAddInterval<SubtractSecondsImpl>;
|
||||
using FunctionSubtractMinutes = FunctionDateOrDateTimeAddInterval<SubtractMinutesImpl>;
|
||||
using FunctionSubtractHours = FunctionDateOrDateTimeAddInterval<SubtractHoursImpl>;
|
||||
using FunctionSubtractDays = FunctionDateOrDateTimeAddInterval<SubtractDaysImpl>;
|
||||
using FunctionSubtractWeeks = FunctionDateOrDateTimeAddInterval<SubtractWeeksImpl>;
|
||||
using FunctionSubtractMonths = FunctionDateOrDateTimeAddInterval<SubtractMonthsImpl>;
|
||||
using FunctionSubtractYears = FunctionDateOrDateTimeAddInterval<SubtractYearsImpl>;
|
||||
|
||||
}
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Interpreters/Context.h>
|
||||
@ -215,6 +216,51 @@ public:
|
||||
};
|
||||
|
||||
|
||||
/// Returns number of fields in Enum data type of passed value.
|
||||
class FunctionGetSizeOfEnumType : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "getSizeOfEnumType";
|
||||
static FunctionPtr create(const Context & context)
|
||||
{
|
||||
return std::make_shared<FunctionGetSizeOfEnumType>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (checkDataType<DataTypeEnum8>(arguments[0].get()))
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
else if (checkDataType<DataTypeEnum16>(arguments[0].get()))
|
||||
return std::make_shared<DataTypeUInt16>();
|
||||
|
||||
throw Exception("The argument for function " + getName() + " must be Enum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (auto type = checkAndGetDataType<DataTypeEnum8>(block.getByPosition(arguments[0]).type.get()))
|
||||
block.getByPosition(result).column = DataTypeUInt8().createConstColumn(block.rows(), UInt64(type->getValues().size()));
|
||||
else if (auto type = checkAndGetDataType<DataTypeEnum16>(block.getByPosition(arguments[0]).type.get()))
|
||||
block.getByPosition(result).column = DataTypeUInt16().createConstColumn(block.rows(), UInt64(type->getValues().size()));
|
||||
else
|
||||
throw Exception("The argument for function " + getName() + " must be Enum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
||||
/// Returns name of IColumn instance.
|
||||
class FunctionToColumnTypeName : public IFunction
|
||||
{
|
||||
@ -1733,6 +1779,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionHostName>();
|
||||
factory.registerFunction<FunctionVisibleWidth>();
|
||||
factory.registerFunction<FunctionToTypeName>();
|
||||
factory.registerFunction<FunctionGetSizeOfEnumType>();
|
||||
factory.registerFunction<FunctionToColumnTypeName>();
|
||||
factory.registerFunction<FunctionDefaultValueOfArgumentType>();
|
||||
factory.registerFunction<FunctionBlockSize>();
|
||||
|
@ -3,8 +3,5 @@ include_directories (${CMAKE_CURRENT_BINARY_DIR})
|
||||
add_executable (number_traits number_traits.cpp)
|
||||
target_link_libraries (number_traits dbms)
|
||||
|
||||
add_executable (functions_arithmetic functions_arithmetic.cpp)
|
||||
target_link_libraries (functions_arithmetic dbms clickhouse_functions)
|
||||
|
||||
add_executable (logical_functions_performance logical_functions_performance.cpp)
|
||||
target_link_libraries (logical_functions_performance dbms)
|
||||
|
@ -1,83 +0,0 @@
|
||||
#include <iostream>
|
||||
#include <iomanip>
|
||||
|
||||
#include <Common/Stopwatch.h>
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
#include <Functions/FunctionsArithmetic.h>
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
try
|
||||
{
|
||||
using namespace DB;
|
||||
|
||||
size_t n = atoi(argv[1]);
|
||||
|
||||
ColumnWithTypeAndName descr1;
|
||||
auto col1 = std::make_shared<ColumnUInt8>();
|
||||
descr1.type = std::make_shared<DataTypeUInt8>();
|
||||
descr1.column = col1;
|
||||
descr1.name = "x";
|
||||
col1->getData().resize(n);
|
||||
|
||||
ColumnWithTypeAndName descr2;
|
||||
auto col2 = std::make_shared<ColumnInt16>();
|
||||
descr2.type = std::make_shared<DataTypeInt16>();
|
||||
descr2.column = col2;
|
||||
descr2.name = "x";
|
||||
|
||||
Block block;
|
||||
block.insert(descr1);
|
||||
block.insert(descr2);
|
||||
col2->getData().resize(n);
|
||||
|
||||
for (size_t i = 0; i < n; ++i)
|
||||
{
|
||||
col1->getData()[i] = 10;
|
||||
col2->getData()[i] = 3;
|
||||
}
|
||||
|
||||
FunctionDivideFloating f;
|
||||
DataTypes arg_types;
|
||||
arg_types.push_back(descr1.type);
|
||||
arg_types.push_back(descr2.type);
|
||||
|
||||
ColumnNumbers arg_nums;
|
||||
arg_nums.push_back(0);
|
||||
arg_nums.push_back(1);
|
||||
|
||||
DataTypePtr res_type = f.getReturnType(arg_types);
|
||||
|
||||
ColumnWithTypeAndName descr_res;
|
||||
descr_res.type = res_type;
|
||||
descr_res.name = "z";
|
||||
|
||||
{
|
||||
Stopwatch stopwatch;
|
||||
stopwatch.start();
|
||||
|
||||
size_t res_num = 2;
|
||||
f.execute(block, arg_nums, res_num);
|
||||
|
||||
stopwatch.stop();
|
||||
std::cout << std::fixed << std::setprecision(2)
|
||||
<< "Elapsed " << stopwatch.elapsedSeconds() << " sec."
|
||||
<< ", " << n / stopwatch.elapsedSeconds() << " rows/sec."
|
||||
<< std::endl;
|
||||
}
|
||||
|
||||
Float64 x = 0;
|
||||
for (size_t i = 0; i < n; ++i)
|
||||
x += get<Float64>((*block.getByPosition(2).column)[i]);
|
||||
|
||||
std::cout << x << std::endl;
|
||||
return 0;
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
std::cerr << e.displayText() << std::endl;
|
||||
throw;
|
||||
}
|
@ -563,8 +563,8 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTI
|
||||
s[9] += values.day_of_month % 10;
|
||||
|
||||
UInt8 hour = date_lut.toHour(datetime);
|
||||
UInt8 minute = date_lut.toMinuteInaccurate(datetime);
|
||||
UInt8 second = date_lut.toSecondInaccurate(datetime);
|
||||
UInt8 minute = date_lut.toMinute(datetime);
|
||||
UInt8 second = date_lut.toSecond(datetime);
|
||||
|
||||
s[11] += hour / 10;
|
||||
s[12] += hour % 10;
|
||||
|
@ -5,17 +5,23 @@
|
||||
#include <DataStreams/LazyBlockInputStream.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event DistributedConnectionMissingTable;
|
||||
extern const Event DistributedConnectionStaleReplica;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
|
||||
extern const int ALL_REPLICAS_ARE_STALE;
|
||||
|
||||
extern const int ALL_REPLICAS_ARE_STALE;
|
||||
}
|
||||
|
||||
namespace ClusterProxy
|
||||
@ -72,6 +78,7 @@ void SelectStreamFactory::createForShard(
|
||||
StoragePtr main_table_storage = context.tryGetTable(main_table.database, main_table.table);
|
||||
if (!main_table_storage) /// Table is absent on a local server.
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable);
|
||||
if (shard_info.pool)
|
||||
{
|
||||
LOG_WARNING(
|
||||
@ -117,6 +124,10 @@ void SelectStreamFactory::createForShard(
|
||||
}
|
||||
|
||||
/// If we reached this point, local replica is stale.
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica);
|
||||
LOG_WARNING(
|
||||
&Logger::get("ClusterProxy::SelectStreamFactory"),
|
||||
"Local replica of shard " << shard_info.shard_num << " is stale (delay: " << local_delay << "s.)");
|
||||
|
||||
if (!settings.fallback_to_stale_replicas_for_distributed_queries)
|
||||
{
|
||||
@ -128,9 +139,9 @@ void SelectStreamFactory::createForShard(
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Local replica for shard " + toString(shard_info.shard_num)
|
||||
+ " is stale (delay: " + toString(local_delay) + "), but no other replica configured.",
|
||||
ErrorCodes::ALL_REPLICAS_ARE_STALE);
|
||||
"Local replica of shard " + toString(shard_info.shard_num)
|
||||
+ " is stale (delay: " + toString(local_delay) + "s.), but no other replica configured",
|
||||
ErrorCodes::ALL_REPLICAS_ARE_STALE);
|
||||
}
|
||||
|
||||
if (!shard_info.pool)
|
||||
@ -144,13 +155,25 @@ void SelectStreamFactory::createForShard(
|
||||
/// Do it lazily to avoid connecting in the main thread.
|
||||
|
||||
auto lazily_create_stream = [
|
||||
pool = shard_info.pool, query, query_ast, context, throttler,
|
||||
pool = shard_info.pool, shard_num = shard_info.shard_num, query, query_ast, context, throttler,
|
||||
main_table = main_table, external_tables = external_tables, stage = processed_stage,
|
||||
local_delay]()
|
||||
-> BlockInputStreamPtr
|
||||
{
|
||||
std::vector<ConnectionPoolWithFailover::TryResult> try_results =
|
||||
pool->getManyChecked(&context.getSettingsRef(), PoolMode::GET_MANY, main_table);
|
||||
std::vector<ConnectionPoolWithFailover::TryResult> try_results;
|
||||
try
|
||||
{
|
||||
try_results = pool->getManyChecked(&context.getSettingsRef(), PoolMode::GET_MANY, main_table);
|
||||
}
|
||||
catch (const Exception & ex)
|
||||
{
|
||||
if (ex.code() == ErrorCodes::ALL_CONNECTION_TRIES_FAILED)
|
||||
LOG_WARNING(
|
||||
&Logger::get("ClusterProxy::SelectStreamFactory"),
|
||||
"Connections to remote replicas of local shard " << shard_num << " failed, will use stale local replica");
|
||||
else
|
||||
throw;
|
||||
}
|
||||
|
||||
double max_remote_delay = 0.0;
|
||||
for (const auto & try_result : try_results)
|
||||
@ -159,7 +182,7 @@ void SelectStreamFactory::createForShard(
|
||||
max_remote_delay = std::max(try_result.staleness, max_remote_delay);
|
||||
}
|
||||
|
||||
if (local_delay < max_remote_delay)
|
||||
if (try_results.empty() || local_delay < max_remote_delay)
|
||||
return createLocalStream(query_ast, context, stage);
|
||||
else
|
||||
{
|
||||
|
@ -470,6 +470,16 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
String data_path = path + "data/" + database_name_escaped + "/";
|
||||
String metadata_path = path + "metadata/" + database_name_escaped + "/" + table_name_escaped + ".sql";
|
||||
|
||||
// If this is a stub ATTACH query, read the query definition from the database
|
||||
if (create.attach && !create.storage && !create.columns)
|
||||
{
|
||||
// Table SQL definition is available even if the table is detached
|
||||
auto query = context.getCreateQuery(database_name, table_name);
|
||||
auto & as_create = typeid_cast<const ASTCreateQuery &>(*query);
|
||||
create = as_create; // Copy the saved create query, but use ATTACH instead of CREATE
|
||||
create.attach = true;
|
||||
}
|
||||
|
||||
std::unique_ptr<InterpreterSelectQuery> interpreter_select;
|
||||
Block as_select_sample;
|
||||
/// For `view` type tables, you may need `sample_block` to get the columns.
|
||||
|
@ -168,7 +168,8 @@ protected:
|
||||
|
||||
if (!as_table.empty())
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "")
|
||||
std::string what = (!is_materialized_view ? " AS " : " TO ");
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << what << (settings.hilite ? hilite_none : "")
|
||||
<< (!as_database.empty() ? backQuoteIfNeed(as_database) + "." : "") << backQuoteIfNeed(as_table);
|
||||
}
|
||||
|
||||
|
@ -607,4 +607,58 @@ bool ParserNullityChecking::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
}
|
||||
|
||||
|
||||
bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
Pos begin = pos;
|
||||
|
||||
/// If no INTERVAL keyword, go to nested parser.
|
||||
if (!ParserKeyword("INTERVAL").ignore(pos, expected))
|
||||
return next_parser.parse(pos, node, expected);
|
||||
|
||||
ASTPtr expr;
|
||||
/// Any expression can be inside, because operator surrounds it.
|
||||
if (!ParserExpressionWithOptionalAlias(false).parse(pos, expr, expected))
|
||||
return false;
|
||||
|
||||
const char * function_name = nullptr;
|
||||
|
||||
if (ParserKeyword("SECOND").ignore(pos, expected))
|
||||
function_name = "toIntervalSecond";
|
||||
else if (ParserKeyword("MINUTE").ignore(pos, expected))
|
||||
function_name = "toIntervalMinute";
|
||||
else if (ParserKeyword("HOUR").ignore(pos, expected))
|
||||
function_name = "toIntervalHour";
|
||||
else if (ParserKeyword("DAY").ignore(pos, expected))
|
||||
function_name = "toIntervalDay";
|
||||
else if (ParserKeyword("WEEK").ignore(pos, expected))
|
||||
function_name = "toIntervalWeek";
|
||||
else if (ParserKeyword("MONTH").ignore(pos, expected))
|
||||
function_name = "toIntervalMonth";
|
||||
else if (ParserKeyword("YEAR").ignore(pos, expected))
|
||||
function_name = "toIntervalYear";
|
||||
else
|
||||
return false;
|
||||
|
||||
/// the function corresponding to the operator
|
||||
auto function = std::make_shared<ASTFunction>();
|
||||
|
||||
/// function arguments
|
||||
auto exp_list = std::make_shared<ASTExpressionList>();
|
||||
|
||||
/// the first argument of the function is the previous element, the second is the next one
|
||||
function->range.first = begin->begin;
|
||||
function->range.second = pos->begin;
|
||||
function->name = function_name;
|
||||
function->arguments = exp_list;
|
||||
function->children.push_back(exp_list);
|
||||
|
||||
exp_list->children.push_back(expr);
|
||||
exp_list->range.first = begin->begin;
|
||||
exp_list->range.second = pos->begin;
|
||||
|
||||
node = function;
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -164,11 +164,22 @@ protected:
|
||||
};
|
||||
|
||||
|
||||
/// Optional conversion to INTERVAL data type. Example: "INTERVAL x SECOND" parsed as "toIntervalSecond(x)".
|
||||
class ParserIntervalOperatorExpression : public IParserBase
|
||||
{
|
||||
protected:
|
||||
ParserMultiplicativeExpression next_parser;
|
||||
|
||||
const char * getName() const { return "INTERVAL operator expression"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
||||
};
|
||||
|
||||
|
||||
class ParserAdditiveExpression : public IParserBase
|
||||
{
|
||||
private:
|
||||
static const char * operators[];
|
||||
ParserLeftAssociativeBinaryOperatorList operator_parser {operators, std::make_unique<ParserMultiplicativeExpression>()};
|
||||
ParserLeftAssociativeBinaryOperatorList operator_parser {operators, std::make_unique<ParserIntervalOperatorExpression>()};
|
||||
|
||||
protected:
|
||||
const char * getName() const { return "additive expression"; }
|
||||
@ -323,6 +334,7 @@ protected:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class ParserExpressionInCastExpression : public IParserBase
|
||||
{
|
||||
public:
|
||||
@ -344,7 +356,7 @@ class ParserExpressionList : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserExpressionList(bool allow_alias_without_as_keyword_, bool prefer_alias_to_column_name_ = false)
|
||||
: allow_alias_without_as_keyword(allow_alias_without_as_keyword_), prefer_alias_to_column_name(prefer_alias_to_column_name_) {}
|
||||
: allow_alias_without_as_keyword(allow_alias_without_as_keyword_), prefer_alias_to_column_name(prefer_alias_to_column_name_) {}
|
||||
|
||||
protected:
|
||||
bool allow_alias_without_as_keyword;
|
||||
@ -359,7 +371,7 @@ class ParserNotEmptyExpressionList : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserNotEmptyExpressionList(bool allow_alias_without_as_keyword, bool prefer_alias_to_column_name = false)
|
||||
: nested_parser(allow_alias_without_as_keyword, prefer_alias_to_column_name) {}
|
||||
: nested_parser(allow_alias_without_as_keyword, prefer_alias_to_column_name) {}
|
||||
private:
|
||||
ParserExpressionList nested_parser;
|
||||
protected:
|
||||
|
@ -219,6 +219,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
bool is_materialized_view = false;
|
||||
bool is_populate = false;
|
||||
bool is_temporary = false;
|
||||
bool to_table = false;
|
||||
|
||||
if (!s_create.ignore(pos, expected))
|
||||
{
|
||||
@ -254,6 +255,23 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
}
|
||||
|
||||
// Shortcut for ATTACH a previously detached table
|
||||
if (attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon))
|
||||
{
|
||||
auto query = std::make_shared<ASTCreateQuery>(StringRange(begin, pos));
|
||||
node = query;
|
||||
|
||||
query->attach = attach;
|
||||
query->if_not_exists = if_not_exists;
|
||||
|
||||
if (database)
|
||||
query->database = typeid_cast<ASTIdentifier &>(*database).name;
|
||||
if (table)
|
||||
query->table = typeid_cast<ASTIdentifier &>(*table).name;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
/// List of columns.
|
||||
if (s_lparen.ignore(pos, expected))
|
||||
{
|
||||
@ -341,6 +359,22 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
}
|
||||
|
||||
// TO [db.]table
|
||||
if (ParserKeyword{"TO"}.ignore(pos, expected))
|
||||
{
|
||||
to_table = true;
|
||||
|
||||
if (!name_p.parse(pos, as_table, expected))
|
||||
return false;
|
||||
|
||||
if (s_dot.ignore(pos, expected))
|
||||
{
|
||||
as_database = as_table;
|
||||
if (!name_p.parse(pos, as_table, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/// Optional - a list of columns can be specified. It must fully comply with SELECT.
|
||||
if (s_lparen.ignore(pos, expected))
|
||||
{
|
||||
@ -351,7 +385,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
}
|
||||
|
||||
if (is_materialized_view)
|
||||
if (is_materialized_view && !to_table)
|
||||
{
|
||||
/// Internal ENGINE for MATERIALIZED VIEW must be specified.
|
||||
if (!storage_p.parse(pos, storage, expected))
|
||||
|
@ -214,7 +214,7 @@ protected:
|
||||
* CREATE|ATTACH DATABASE db [ENGINE = engine]
|
||||
*
|
||||
* Or:
|
||||
* CREATE|ATTACH [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [ENGINE = engine] [POPULATE] AS SELECT ...
|
||||
* CREATE|ATTACH [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ...
|
||||
*/
|
||||
class ParserCreateQuery : public IParserBase
|
||||
{
|
||||
|
@ -70,8 +70,8 @@ StorageMaterializedView::StorageMaterializedView(
|
||||
if (!query.select)
|
||||
throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
if (!query.storage)
|
||||
throw Exception("ENGINE of MaterializedView must be specified explicitly", ErrorCodes::INCORRECT_QUERY);
|
||||
if (!query.storage && query.as_table.empty())
|
||||
throw Exception("ENGINE of MaterializedView should be specified explicitly", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
extractDependentTable(*query.select, select_database_name, select_table_name);
|
||||
|
||||
@ -80,16 +80,28 @@ StorageMaterializedView::StorageMaterializedView(
|
||||
DatabaseAndTableName(select_database_name, select_table_name),
|
||||
DatabaseAndTableName(database_name, table_name));
|
||||
|
||||
String inner_table_name = getInnerTableName();
|
||||
// If the destination table is not set, use inner table
|
||||
if (!query.storage)
|
||||
{
|
||||
target_database_name = query.as_database;
|
||||
target_table_name = query.as_table;
|
||||
}
|
||||
else
|
||||
{
|
||||
target_database_name = database_name;
|
||||
target_table_name = ".inner." + table_name;
|
||||
has_inner_table = true;
|
||||
}
|
||||
|
||||
inner_query = query.select->ptr();
|
||||
|
||||
/// If there is an ATTACH request, then the internal table must already be connected.
|
||||
if (!attach_)
|
||||
if (!attach_ && has_inner_table)
|
||||
{
|
||||
/// We will create a query to create an internal table.
|
||||
auto manual_create_query = std::make_shared<ASTCreateQuery>();
|
||||
manual_create_query->database = database_name;
|
||||
manual_create_query->table = inner_table_name;
|
||||
manual_create_query->database = target_database_name;
|
||||
manual_create_query->table = target_table_name;
|
||||
manual_create_query->set(manual_create_query->columns, query.columns->ptr());
|
||||
manual_create_query->set(manual_create_query->storage, query.storage->ptr());
|
||||
|
||||
@ -114,12 +126,12 @@ StorageMaterializedView::StorageMaterializedView(
|
||||
|
||||
NameAndTypePair StorageMaterializedView::getColumn(const String & column_name) const
|
||||
{
|
||||
return getInnerTable()->getColumn(column_name);
|
||||
return getTargetTable()->getColumn(column_name);
|
||||
}
|
||||
|
||||
bool StorageMaterializedView::hasColumn(const String & column_name) const
|
||||
{
|
||||
return getInnerTable()->hasColumn(column_name);
|
||||
return getTargetTable()->hasColumn(column_name);
|
||||
}
|
||||
|
||||
BlockInputStreams StorageMaterializedView::read(
|
||||
@ -130,12 +142,12 @@ BlockInputStreams StorageMaterializedView::read(
|
||||
const size_t max_block_size,
|
||||
const unsigned num_streams)
|
||||
{
|
||||
return getInnerTable()->read(column_names, query_info, context, processed_stage, max_block_size, num_streams);
|
||||
return getTargetTable()->read(column_names, query_info, context, processed_stage, max_block_size, num_streams);
|
||||
}
|
||||
|
||||
BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const Settings & settings)
|
||||
{
|
||||
return getInnerTable()->write(query, settings);
|
||||
return getTargetTable()->write(query, settings);
|
||||
}
|
||||
|
||||
void StorageMaterializedView::drop()
|
||||
@ -144,14 +156,12 @@ void StorageMaterializedView::drop()
|
||||
DatabaseAndTableName(select_database_name, select_table_name),
|
||||
DatabaseAndTableName(database_name, table_name));
|
||||
|
||||
auto inner_table_name = getInnerTableName();
|
||||
|
||||
if (global_context.tryGetTable(database_name, inner_table_name))
|
||||
if (has_inner_table && global_context.tryGetTable(target_database_name, target_table_name))
|
||||
{
|
||||
/// We create and execute `drop` query for internal table.
|
||||
auto drop_query = std::make_shared<ASTDropQuery>();
|
||||
drop_query->database = database_name;
|
||||
drop_query->table = inner_table_name;
|
||||
drop_query->database = target_database_name;
|
||||
drop_query->table = target_table_name;
|
||||
ASTPtr ast_drop_query = drop_query;
|
||||
InterpreterDropQuery drop_interpreter(ast_drop_query, global_context);
|
||||
drop_interpreter.execute();
|
||||
@ -160,12 +170,12 @@ void StorageMaterializedView::drop()
|
||||
|
||||
bool StorageMaterializedView::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
|
||||
{
|
||||
return getInnerTable()->optimize(query, partition, final, deduplicate, context);
|
||||
return getTargetTable()->optimize(query, partition, final, deduplicate, context);
|
||||
}
|
||||
|
||||
StoragePtr StorageMaterializedView::getInnerTable() const
|
||||
StoragePtr StorageMaterializedView::getTargetTable() const
|
||||
{
|
||||
return global_context.getTable(database_name, getInnerTableName());
|
||||
return global_context.getTable(target_database_name, target_table_name);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -20,18 +20,17 @@ public:
|
||||
std::string getName() const override { return "MaterializedView"; }
|
||||
std::string getTableName() const override { return table_name; }
|
||||
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
|
||||
std::string getInnerTableName() const { return ".inner." + table_name; }
|
||||
ASTPtr getInnerQuery() const { return inner_query->clone(); };
|
||||
StoragePtr getInnerTable() const;
|
||||
StoragePtr getTargetTable() const;
|
||||
|
||||
NameAndTypePair getColumn(const String & column_name) const override;
|
||||
bool hasColumn(const String & column_name) const override;
|
||||
|
||||
bool supportsSampling() const override { return getInnerTable()->supportsSampling(); }
|
||||
bool supportsPrewhere() const override { return getInnerTable()->supportsPrewhere(); }
|
||||
bool supportsFinal() const override { return getInnerTable()->supportsFinal(); }
|
||||
bool supportsParallelReplicas() const override { return getInnerTable()->supportsParallelReplicas(); }
|
||||
bool supportsIndexForIn() const override { return getInnerTable()->supportsIndexForIn(); }
|
||||
bool supportsSampling() const override { return getTargetTable()->supportsSampling(); }
|
||||
bool supportsPrewhere() const override { return getTargetTable()->supportsPrewhere(); }
|
||||
bool supportsFinal() const override { return getTargetTable()->supportsFinal(); }
|
||||
bool supportsParallelReplicas() const override { return getTargetTable()->supportsParallelReplicas(); }
|
||||
bool supportsIndexForIn() const override { return getTargetTable()->supportsIndexForIn(); }
|
||||
|
||||
BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
|
||||
void drop() override;
|
||||
@ -48,11 +47,14 @@ public:
|
||||
private:
|
||||
String select_database_name;
|
||||
String select_table_name;
|
||||
String target_database_name;
|
||||
String target_table_name;
|
||||
String table_name;
|
||||
String database_name;
|
||||
ASTPtr inner_query;
|
||||
Context & global_context;
|
||||
NamesAndTypesListPtr columns;
|
||||
bool has_inner_table = false;
|
||||
|
||||
StorageMaterializedView(
|
||||
const String & table_name_,
|
||||
|
@ -11,8 +11,8 @@ TEST(RangeFiltered, simple)
|
||||
for (int i = 0; i < 10; ++i)
|
||||
v.push_back(i);
|
||||
|
||||
auto v30 = createRangeFiltered([] (int i) { return i % 3 == 0;}, v);
|
||||
auto v31 = createRangeFiltered([] (int i) { return i % 3 != 0;}, v);
|
||||
auto v30 = createRangeFiltered([] (int i) { return i % 3 == 0; }, v);
|
||||
auto v31 = createRangeFiltered([] (int i) { return i % 3 != 0; }, v);
|
||||
|
||||
for (const int & i : v30)
|
||||
ASSERT_EQ(i % 3, 0);
|
||||
@ -27,7 +27,6 @@ TEST(RangeFiltered, simple)
|
||||
auto it2 = std::next(it);
|
||||
ASSERT_EQ(*it2, 3);
|
||||
|
||||
auto it3 = it;
|
||||
it = std::next(it2);
|
||||
ASSERT_EQ(*it, 6);
|
||||
}
|
@ -8,11 +8,12 @@ make -j24
|
||||
sudo make install
|
||||
|
||||
# Perform analysis
|
||||
# cd ClickHouse_clean/build
|
||||
|
||||
cd ClickHouse_clean/build
|
||||
cppcheck -j24 --project=compile_commands.json --enable=all 2> cppcheck-errors.txt
|
||||
# or
|
||||
# cppcheck -j24 -i contrib -i build --enable=all . 2> cppcheck-errors.txt
|
||||
|
||||
# or (from directory with sources)
|
||||
# cppcheck -i contrib -i build --enable=all . 2> cppcheck-errors.txt
|
||||
|
||||
# Check is pretty fast.
|
||||
# It gives many false positives.
|
||||
|
@ -58,7 +58,7 @@ def test(started_cluster):
|
||||
assert node_2_1.query("SELECT sum(x) FROM replicated").strip() == ''
|
||||
assert node_2_2.query("SELECT sum(x) FROM replicated").strip() == '2'
|
||||
|
||||
# With in_order balancing first replicas chosen.
|
||||
# With in_order balancing first replicas are chosen.
|
||||
assert instance_with_dist_table.query(
|
||||
"SELECT count() FROM distributed SETTINGS load_balancing='in_order'").strip() == ''
|
||||
|
||||
@ -89,3 +89,12 @@ SELECT count() FROM distributed SETTINGS
|
||||
max_replica_delay_for_distributed_queries=1,
|
||||
fallback_to_stale_replicas_for_distributed_queries=0
|
||||
''')
|
||||
|
||||
# Now partition off the remote replica of the local shard and test that failover still works.
|
||||
pm.partition_instances(node_1_1, node_1_2, port=9000)
|
||||
|
||||
assert instance_with_dist_table.query('''
|
||||
SELECT sum(x) FROM distributed SETTINGS
|
||||
load_balancing='in_order',
|
||||
max_replica_delay_for_distributed_queries=1
|
||||
''').strip() == '2'
|
||||
|
96
dbms/tests/performance/date_time/date_time.xml
Normal file
96
dbms/tests/performance/date_time/date_time.xml
Normal file
@ -0,0 +1,96 @@
|
||||
<test>
|
||||
<name>Functions for working with date and time</name>
|
||||
|
||||
<type>once</type>
|
||||
|
||||
<stop_conditions>
|
||||
<any_of>
|
||||
<average_speed_not_changing_for_ms>10000</average_speed_not_changing_for_ms>
|
||||
<total_time_ms>1000</total_time_ms>
|
||||
</any_of>
|
||||
</stop_conditions>
|
||||
|
||||
<metrics>
|
||||
<max_rows_per_second />
|
||||
<max_bytes_per_second />
|
||||
<avg_rows_per_second />
|
||||
<avg_bytes_per_second />
|
||||
</metrics>
|
||||
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>datetime_transform</name>
|
||||
<values>
|
||||
<value>toSecond</value>
|
||||
<value>toMinute</value>
|
||||
<value>toHour</value>
|
||||
<value>toDayOfWeek</value>
|
||||
<value>toDayOfMonth</value>
|
||||
<value>toMonth</value>
|
||||
<value>toQuarter</value>
|
||||
<value>toYear</value>
|
||||
|
||||
<value>toStartOfMinute</value>
|
||||
<value>toStartOfFiveMinute</value>
|
||||
<value>toStartOfHour</value>
|
||||
<value>toStartOfDay</value>
|
||||
<value>toDate</value>
|
||||
<value>toMonday</value>
|
||||
<value>toStartOfMonth</value>
|
||||
<value>toStartOfQuarter</value>
|
||||
<value>toYear</value>
|
||||
|
||||
<value>toTime</value>
|
||||
|
||||
<value>toRelativeYearNum</value>
|
||||
<value>toRelativeMonthNum</value>
|
||||
<value>toRelativeWeekNum</value>
|
||||
<value>toRelativeDayNum</value>
|
||||
<value>toRelativeHourNum</value>
|
||||
<value>toRelativeMinuteNum</value>
|
||||
<value>toRelativeSecondNum</value>
|
||||
|
||||
<value>toYYYYMM</value>
|
||||
<value>toYYYYMMDD</value>
|
||||
<value>toYYYYMMDDhhmmss</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>date_transform</name>
|
||||
<values>
|
||||
<value>toDayOfWeek</value>
|
||||
<value>toDayOfMonth</value>
|
||||
<value>toMonth</value>
|
||||
<value>toQuarter</value>
|
||||
<value>toYear</value>
|
||||
|
||||
<value>toStartOfDay</value>
|
||||
<value>toDate</value>
|
||||
<value>toMonday</value>
|
||||
<value>toStartOfMonth</value>
|
||||
<value>toStartOfQuarter</value>
|
||||
<value>toYear</value>
|
||||
|
||||
<value>toRelativeYearNum</value>
|
||||
<value>toRelativeMonthNum</value>
|
||||
<value>toRelativeWeekNum</value>
|
||||
<value>toRelativeDayNum</value>
|
||||
|
||||
<value>toYYYYMM</value>
|
||||
<value>toYYYYMMDD</value>
|
||||
<value>toYYYYMMDDhhmmss</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>time_zone</name>
|
||||
<values>
|
||||
<value>UTC</value>
|
||||
<value>Europe/Moscow</value>
|
||||
<value>Asia/Kolkata</value>
|
||||
</values>
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
||||
<query>SELECT count() FROM system.numbers WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, {datetime_transform}(t, '{time_zone}'))</query>
|
||||
<query>SELECT count() FROM system.numbers WHERE NOT ignore(toDate('2017-01-01') + number % 1000 + rand() % 10 AS t, {date_transform}(t))</query>
|
||||
</test>
|
41
dbms/tests/performance/set/set.xml
Normal file
41
dbms/tests/performance/set/set.xml
Normal file
@ -0,0 +1,41 @@
|
||||
<test>
|
||||
<name>set_lookup_random</name>
|
||||
<type>once</type>
|
||||
|
||||
<stop_conditions>
|
||||
<all_of>
|
||||
<total_time_ms>10000</total_time_ms>
|
||||
</all_of>
|
||||
<any_of>
|
||||
<average_speed_not_changing_for_ms>5000</average_speed_not_changing_for_ms>
|
||||
<total_time_ms>20000</total_time_ms>
|
||||
</any_of>
|
||||
</stop_conditions>
|
||||
|
||||
<main_metric>
|
||||
<avg_rows_per_second/>
|
||||
</main_metric>
|
||||
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>table</name>
|
||||
<values>
|
||||
<value>system.numbers</value>
|
||||
<value>system.numbers_mt</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>size</name>
|
||||
<values>
|
||||
<value>1</value>
|
||||
<value>16</value>
|
||||
<value>1024</value>
|
||||
<value>16384</value>
|
||||
<value>1048576</value>
|
||||
<value>10000000</value>
|
||||
</values>
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
||||
<query>SELECT count() FROM {table} WHERE rand64() IN (SELECT number FROM system.numbers LIMIT {size})</query>
|
||||
</test>
|
25
dbms/tests/performance/set/set_hits.xml
Normal file
25
dbms/tests/performance/set/set_hits.xml
Normal file
@ -0,0 +1,25 @@
|
||||
<test>
|
||||
<name>set_lookup_hits</name>
|
||||
<type>once</type>
|
||||
|
||||
<stop_conditions>
|
||||
<all_of>
|
||||
<total_time_ms>10000</total_time_ms>
|
||||
</all_of>
|
||||
<any_of>
|
||||
<average_speed_not_changing_for_ms>5000</average_speed_not_changing_for_ms>
|
||||
<total_time_ms>20000</total_time_ms>
|
||||
</any_of>
|
||||
</stop_conditions>
|
||||
|
||||
<main_metric>
|
||||
<avg_rows_per_second/>
|
||||
</main_metric>
|
||||
|
||||
<query>SELECT count() FROM hits_100m_single WHERE UserID IN (SELECT UserID FROM hits_100m_single WHERE AdvEngineID != 0)</query>
|
||||
<query>SELECT count() FROM hits_100m_single WHERE UserID IN (SELECT UserID FROM hits_100m_single)</query>
|
||||
<query>SELECT count() FROM hits_100m_single WHERE SearchPhrase IN (SELECT SearchPhrase FROM hits_100m_single)</query>
|
||||
<query>SELECT count() FROM hits_100m_single WHERE URL IN (SELECT URL FROM hits_100m_single WHERE AdvEngineID != 0)</query>
|
||||
<query>SELECT count() FROM hits_100m_single WHERE URL IN (SELECT URL FROM hits_100m_single WHERE SearchEngineID != 0)</query>
|
||||
<query>SELECT count() FROM hits_100m_single WHERE RegionID IN (SELECT RegionID FROM hits_100m_single)</query>
|
||||
</test>
|
@ -8,21 +8,21 @@
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-12-01
|
||||
2014-07-01
|
||||
2014-07-01
|
||||
2014-07-01
|
||||
2014-10-01
|
||||
2014-07-01
|
||||
2014-01-01
|
||||
2014-01-01
|
||||
2014-01-01
|
||||
2014-01-01
|
||||
2014-01-01
|
||||
1970-01-02 12:00:00 1970-01-02 12:00:00
|
||||
1970-01-02 10:00:00 1970-01-02 11:00:00
|
||||
1970-01-02 09:00:00 1970-01-02 10:00:00
|
||||
1970-01-02 18:00:00 1970-01-02 18:00:00
|
||||
1970-01-02 01:30:00 1970-01-02 01:30:00
|
||||
1970-01-02 01:00:00 1970-01-02 01:00:00
|
||||
2014
|
||||
2014
|
||||
2014
|
||||
@ -92,7 +92,6 @@
|
||||
392251
|
||||
392251
|
||||
392251
|
||||
392251
|
||||
23535110
|
||||
23535110
|
||||
23535110
|
||||
|
@ -24,20 +24,20 @@ SELECT toStartOfMonth(toDateTime(1419800400), 'Pacific/Pitcairn');
|
||||
|
||||
/* toStartOfQuarter */
|
||||
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Moscow');
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Paris');
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/London');
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Asia/Tokyo');
|
||||
SELECT toStartOfMonth(toDateTime(1419800400), 'Pacific/Pitcairn');
|
||||
|
||||
/* toStartOfYear */
|
||||
|
||||
SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/Moscow');
|
||||
SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/Paris');
|
||||
SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/London');
|
||||
SELECT toStartOfQuarter(toDateTime(1412106600), 'Asia/Tokyo');
|
||||
SELECT toStartOfQuarter(toDateTime(1412106600), 'Pacific/Pitcairn');
|
||||
|
||||
/* toStartOfYear */
|
||||
|
||||
SELECT toStartOfYear(toDateTime(1419800400), 'Europe/Moscow');
|
||||
SELECT toStartOfYear(toDateTime(1419800400), 'Europe/Paris');
|
||||
SELECT toStartOfYear(toDateTime(1419800400), 'Europe/London');
|
||||
SELECT toStartOfYear(toDateTime(1419800400), 'Asia/Tokyo');
|
||||
SELECT toStartOfYear(toDateTime(1419800400), 'Pacific/Pitcairn');
|
||||
|
||||
/* toTime */
|
||||
|
||||
SELECT toString(toTime(toDateTime(1420102800), 'Europe/Moscow'), 'Europe/Moscow'), toString(toTime(toDateTime(1428310800), 'Europe/Moscow'), 'Europe/Moscow');
|
||||
@ -156,7 +156,7 @@ SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/Moscow') - toRelativeHo
|
||||
SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/Paris') - toRelativeHourNum(toDateTime(0), 'Europe/Paris');
|
||||
SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/London') - toRelativeHourNum(toDateTime(0), 'Europe/London');
|
||||
SELECT toRelativeHourNum(toDateTime(1412106600), 'Asia/Tokyo') - toRelativeHourNum(toDateTime(0), 'Asia/Tokyo');
|
||||
SELECT toRelativeHourNum(toDateTime(1412106600), 'Pacific/Pitcairn') - toRelativeHourNum(toDateTime(0), 'Pacific/Pitcairn');
|
||||
-- known wrong result: SELECT toRelativeHourNum(toDateTime(1412106600), 'Pacific/Pitcairn') - toRelativeHourNum(toDateTime(0), 'Pacific/Pitcairn');
|
||||
|
||||
/* toRelativeMinuteNum */
|
||||
|
||||
|
@ -61,104 +61,104 @@ FROM
|
||||
(
|
||||
SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.00
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.01
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.02
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.03
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.04
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.05
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.06
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.07
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.08
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.09
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.02
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.03
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.04
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.05
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.06
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.07
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.08
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.09
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.10
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.11
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.12
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.13
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.14
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.15
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.16
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.17
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.18
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.19
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.20
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.12
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.13
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.14
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.15
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.16
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.17
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.18
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.19
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.20
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.21
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.22
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.23
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.24
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.25
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.26
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.27
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.28
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.29
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.30
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.22
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.23
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.24
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.25
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.26
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.27
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.28
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.29
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.30
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.31
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.32
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.33
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.34
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.35
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.36
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.37
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.38
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.39
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.40
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.32
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.33
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.34
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.35
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.36
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.37
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.38
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.39
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.40
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.41
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.42
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.43
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.44
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.45
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.46
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.47
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.48
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.49
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.50
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.42
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.43
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.44
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.45
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.46
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.47
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.48
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.49
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.50
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.51
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.52
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.53
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.54
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.55
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.56
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.57
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.58
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.59
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.60
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.52
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.53
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.54
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.55
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.56
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.57
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.58
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.59
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.60
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.61
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.62
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.63
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.64
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.65
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.66
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.67
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.68
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.69
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.70
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.62
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.63
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.64
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.65
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.66
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.67
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.68
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.69
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.70
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.71
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.72
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.73
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.74
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.75
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.76
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.77
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.78
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.79
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.80
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.72
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.73
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.74
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.75
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.76
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.77
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.78
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.79
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.80
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.81
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.82
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.83
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.84
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.85
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.86
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.87
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.88
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.89
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.90
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.82
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.83
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.84
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.85
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.86
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.87
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.88
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.89
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.90
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.91
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.92
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.93
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.94
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.95
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.96
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.97
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.98
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.99
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.92
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.93
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.94
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.95
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.96
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.97
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.98
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.99
|
||||
)
|
||||
GROUP BY x
|
||||
HAVING c = 1
|
||||
@ -182,104 +182,104 @@ FROM
|
||||
(
|
||||
SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.00
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.01
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.02
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.03
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.04
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.05
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.06
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.07
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.08
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.09
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.02
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.03
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.04
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.05
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.06
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.07
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.08
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.09
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.10
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.11
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.12
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.13
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.14
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.15
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.16
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.17
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.18
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.19
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.20
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.12
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.13
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.14
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.15
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.16
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.17
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.18
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.19
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.20
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.21
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.22
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.23
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.24
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.25
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.26
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.27
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.28
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.29
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.30
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.22
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.23
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.24
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.25
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.26
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.27
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.28
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.29
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.30
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.31
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.32
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.33
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.34
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.35
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.36
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.37
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.38
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.39
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.40
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.32
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.33
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.34
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.35
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.36
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.37
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.38
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.39
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.40
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.41
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.42
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.43
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.44
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.45
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.46
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.47
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.48
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.49
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.50
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.42
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.43
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.44
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.45
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.46
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.47
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.48
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.49
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.50
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.51
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.52
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.53
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.54
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.55
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.56
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.57
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.58
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.59
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.60
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.52
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.53
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.54
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.55
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.56
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.57
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.58
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.59
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.60
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.61
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.62
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.63
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.64
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.65
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.66
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.67
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.68
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.69
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.70
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.62
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.63
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.64
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.65
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.66
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.67
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.68
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.69
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.70
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.71
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.72
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.73
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.74
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.75
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.76
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.77
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.78
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.79
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.80
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.72
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.73
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.74
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.75
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.76
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.77
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.78
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.79
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.80
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.81
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.82
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.83
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.84
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.85
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.86
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.87
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.88
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.89
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.90
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.82
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.83
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.84
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.85
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.86
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.87
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.88
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.89
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.90
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.91
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.92
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.93
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.94
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.95
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.96
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.97
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.98
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.99
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.92
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.93
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.94
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.95
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.96
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.97
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.98
|
||||
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.99
|
||||
)
|
||||
GROUP BY x
|
||||
HAVING c = 1
|
||||
|
@ -0,0 +1,4 @@
|
||||
1
|
||||
2
|
||||
1
|
||||
2
|
@ -0,0 +1,23 @@
|
||||
DROP TABLE IF EXISTS test.src;
|
||||
DROP TABLE IF EXISTS test.dst;
|
||||
DROP TABLE IF EXISTS test.mv;
|
||||
|
||||
CREATE TABLE test.src (x UInt8) ENGINE = Null;
|
||||
CREATE TABLE test.dst (x UInt8) ENGINE = Memory();
|
||||
|
||||
CREATE MATERIALIZED VIEW test.mv TO test.dst AS SELECT * FROM test.src;
|
||||
INSERT INTO test.src VALUES (1), (2);
|
||||
|
||||
-- Detach MV and see if the data is still readable
|
||||
DETACH TABLE test.mv;
|
||||
SELECT * FROM test.dst;
|
||||
|
||||
-- Reattach MV (shortcut)
|
||||
ATTACH TABLE test.mv;
|
||||
|
||||
-- Drop the MV and see if the data is still readable
|
||||
DROP TABLE test.mv;
|
||||
SELECT * FROM test.dst;
|
||||
|
||||
DROP TABLE test.src;
|
||||
DROP TABLE test.dst;
|
@ -0,0 +1,2 @@
|
||||
2
|
||||
3
|
@ -0,0 +1,2 @@
|
||||
SELECT getSizeOfEnumType(CAST(1 AS Enum8('a' = 1, 'b' = 2)));
|
||||
SELECT getSizeOfEnumType(CAST('b' AS Enum16('a' = 1, 'b' = 2, 'x' = 10)));
|
@ -0,0 +1,3 @@
|
||||
1990-10-19 00:00:00
|
||||
1990-10-19 00:00:00
|
||||
1990-10-19 00:00:00
|
5
dbms/tests/queries/0_stateless/00512_fractional_time_zones.sh
Executable file
5
dbms/tests/queries/0_stateless/00512_fractional_time_zones.sh
Executable file
@ -0,0 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
TZ=Europe/Moscow clickhouse-local --query="SELECT toDateTime('1990-10-19 00:00:00')" 2>/dev/null
|
||||
TZ=Asia/Colombo clickhouse-local --query="SELECT toDateTime('1990-10-19 00:00:00')" 2>/dev/null
|
||||
TZ=Asia/Kathmandu clickhouse-local --query="SELECT toDateTime('1990-10-19 00:00:00')" 2>/dev/null
|
@ -0,0 +1,12 @@
|
||||
2 2017-10-28 02:00:00
|
||||
2 2017-10-28 02:00:00
|
||||
2 2017-10-28 02:00:00
|
||||
2 2017-10-28 02:00:00
|
||||
2 2017-10-28 02:00:00
|
||||
2 2017-10-28 02:00:00
|
||||
3 2017-10-28 03:00:00
|
||||
3 2017-10-28 03:00:00
|
||||
3 2017-10-28 03:00:00
|
||||
3 2017-10-28 03:00:00
|
||||
3 2017-10-28 03:00:00
|
||||
3 2017-10-28 03:00:00
|
@ -0,0 +1 @@
|
||||
WITH toDateTime(1509138000) + number * 300 AS t SELECT toHour(t, 'Asia/Kolkata') AS h, toString(toStartOfHour(t, 'Asia/Kolkata'), 'Asia/Kolkata') AS h_start FROM system.numbers LIMIT 12;
|
@ -0,0 +1,38 @@
|
||||
2016-11-30 08:18:19
|
||||
2017-10-31 01:58:29
|
||||
2017-10-31 08:18:19
|
||||
2017-11-30 08:18:19
|
||||
2017-12-31 08:18:19
|
||||
2018-01-31 08:18:19
|
||||
2018-02-28 08:18:19
|
||||
2018-03-31 08:18:19
|
||||
2018-04-30 08:18:19
|
||||
2018-05-31 08:18:19
|
||||
2018-06-30 08:18:19
|
||||
2018-07-31 08:18:19
|
||||
2018-08-31 08:18:19
|
||||
2018-09-30 08:18:19
|
||||
2018-10-31 08:18:19
|
||||
2018-11-30 08:18:19
|
||||
2018-12-31 08:18:19
|
||||
2019-01-31 08:18:19
|
||||
2019-02-28 08:18:19
|
||||
2019-03-31 08:18:19
|
||||
2019-04-30 08:18:19
|
||||
2019-05-31 08:18:19
|
||||
2016-02-29 01:02:03 2016-02-29 01:02:03
|
||||
2017-02-28 01:02:03 2016-03-29 01:02:03
|
||||
2018-02-28 01:02:03 2016-04-29 01:02:03
|
||||
2019-02-28 01:02:03 2016-05-29 01:02:03
|
||||
2020-02-29 01:02:03 2016-06-29 01:02:03
|
||||
2021-02-28 01:02:03 2016-07-29 01:02:03
|
||||
2022-02-28 01:02:03 2016-08-29 01:02:03
|
||||
2023-02-28 01:02:03 2016-09-29 01:02:03
|
||||
2024-02-29 01:02:03 2016-10-29 01:02:03
|
||||
2025-02-28 01:02:03 2016-11-29 01:02:03
|
||||
2026-02-28 01:02:03 2016-12-29 01:02:03
|
||||
2027-02-28 01:02:03 2017-01-29 01:02:03
|
||||
2028-02-29 01:02:03 2017-02-28 01:02:03
|
||||
2029-02-28 01:02:03 2017-03-29 01:02:03
|
||||
2030-02-28 01:02:03 2017-04-29 01:02:03
|
||||
2031-02-28 01:02:03 2017-05-29 01:02:03
|
@ -0,0 +1,4 @@
|
||||
SELECT toDateTime('2017-10-30 08:18:19') + INTERVAL 1 DAY + INTERVAL 1 MONTH - INTERVAL 1 YEAR;
|
||||
SELECT toDateTime('2017-10-30 08:18:19') + INTERVAL 1 HOUR + INTERVAL 1000 MINUTE + INTERVAL 10 SECOND;
|
||||
SELECT toDateTime('2017-10-30 08:18:19') + INTERVAL 1 DAY + INTERVAL number MONTH FROM system.numbers LIMIT 20;
|
||||
SELECT toDateTime('2016-02-29 01:02:03') + INTERVAL number YEAR, toDateTime('2016-02-29 01:02:03') + INTERVAL number MONTH FROM system.numbers LIMIT 16;
|
4
debian/changelog
vendored
4
debian/changelog
vendored
@ -1,5 +1,5 @@
|
||||
clickhouse (1.1.54300) unstable; urgency=low
|
||||
clickhouse (1.1.54306) unstable; urgency=low
|
||||
|
||||
* Modified source code
|
||||
|
||||
-- <robot-metrika-test@yandex-team.ru> Mon, 16 Oct 2017 13:49:50 +0300
|
||||
-- <robot-metrika-test@yandex-team.ru> Sat, 28 Oct 2017 00:23:24 +0300
|
||||
|
@ -107,7 +107,7 @@ At the moment, ``ALTER`` queries for replicated tables are not supported yet.
|
||||
|
||||
CREATE VIEW
|
||||
~~~~~~~~~~~
|
||||
``CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [ENGINE = engine] [POPULATE] AS SELECT ...``
|
||||
``CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ...``
|
||||
|
||||
Creates a view. There are two types of views: normal and MATERIALIZED.
|
||||
|
||||
@ -133,7 +133,7 @@ This query is fully equivalent to using the subquery:
|
||||
|
||||
Materialized views store data transformed by the corresponding SELECT query.
|
||||
|
||||
When creating a materialized view, you have to specify ENGINE - the table engine for storing data.
|
||||
When creating a materialized view, you have to either specify ENGINE - the table engine for storing data, or target table for materialized results. By default, it uses the same engine as for the table that the SELECT query is made from.
|
||||
|
||||
A materialized view is arranged as follows: when inserting data to the table specified in SELECT, part of the inserted data is converted by this SELECT query, and the result is inserted in the view.
|
||||
|
||||
@ -142,6 +142,7 @@ If you specify POPULATE, the existing table data is inserted in the view when cr
|
||||
The SELECT query can contain DISTINCT, GROUP BY, ORDER BY, LIMIT ... Note that the corresponding conversions are performed independently on each block of inserted data. For example, if GROUP BY is set, data is aggregated during insertion, but only within a single packet of inserted data. The data won't be further aggregated. The exception is when using an ENGINE that independently performs data aggregation, such as SummingMergeTree.
|
||||
|
||||
The execution of ALTER queries on materialized views has not been fully developed, so they might be inconvenient.
|
||||
If the materialized view uses a ``TO [db.]name`` to specify a target table, it is possible to DETACH the view, ALTER the target table, and ATTACH the view again.
|
||||
|
||||
Views look the same as normal tables. For example, they are listed in the result of the SHOW TABLES query.
|
||||
|
||||
@ -154,6 +155,12 @@ The query is exactly the same as CREATE, except
|
||||
- The query doesn't create data on the disk, but assumes that data is already in the appropriate places, and just adds information about the table to the server.
|
||||
After executing an ATTACH query, the server will know about the existence of the table.
|
||||
|
||||
If the table has been previously detached and it's structure is known, it's possible to use shorthand form and omit structure definition:
|
||||
|
||||
.. code-block:: sql
|
||||
|
||||
ATTACH TABLE [IF NOT EXISTS] [db.]name
|
||||
|
||||
This query is used when starting the server. The server stores table metadata as files with ATTACH queries, which it simply runs at launch (with the exception of system tables, which are explicitly created on the server).
|
||||
|
||||
DROP
|
||||
|
@ -1,21 +1,18 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/Types.h>
|
||||
#include <ext/singleton.h>
|
||||
#include <common/likely.h>
|
||||
#include <common/strong_typedef.h>
|
||||
|
||||
#include <iostream>
|
||||
#include <vector>
|
||||
#include <unordered_map>
|
||||
#include <ctime>
|
||||
|
||||
#define DATE_LUT_MIN 0
|
||||
#define DATE_LUT_MAX (0x7FFFFFFF - 86400)
|
||||
#define DATE_LUT_MAX_DAY_NUM (0x7FFFFFFF / 86400)
|
||||
#define DATE_LUT_MAX (0xFFFFFFFFU - 86400)
|
||||
#define DATE_LUT_MAX_DAY_NUM (0xFFFFFFFFU / 86400)
|
||||
/// Table size is bigger than DATE_LUT_MAX_DAY_NUM to fill all indices within UInt16 range: this allows to remove extra check.
|
||||
#define DATE_LUT_SIZE 0x10000
|
||||
#define DATE_LUT_MIN_YEAR 1970
|
||||
#define DATE_LUT_MAX_YEAR 2037 /// Last supported year
|
||||
#define DATE_LUT_YEARS 68 /// Number of years in lookup table
|
||||
#define DATE_LUT_MAX_YEAR 2105 /// Last supported year
|
||||
#define DATE_LUT_YEARS (1 + DATE_LUT_MAX_YEAR - DATE_LUT_MIN_YEAR) /// Number of years in lookup table
|
||||
|
||||
|
||||
STRONG_TYPEDEF(UInt16, DayNum_t);
|
||||
@ -33,9 +30,9 @@ public:
|
||||
struct Values
|
||||
{
|
||||
/// Least significat 32 bits from time_t at beginning of the day.
|
||||
/// Signedness is important to support 1970-01-01 MSK, where time_t == -10800.
|
||||
/// Change to time_t; change constants above; and recompile the sources if you need to support time after 2038 year.
|
||||
Int32 date;
|
||||
/// If the unix timestamp of beginning of the day is negative (example: 1970-01-01 MSK, where time_t == -10800), then value is zero.
|
||||
/// Change to time_t; change constants above; and recompile the sources if you need to support time after 2105 year.
|
||||
UInt32 date;
|
||||
|
||||
/// Properties of the day.
|
||||
UInt16 year;
|
||||
@ -43,6 +40,10 @@ public:
|
||||
UInt8 day_of_month;
|
||||
UInt8 day_of_week;
|
||||
|
||||
/// Total number of days in current month. Actually we can use separate table that is independent of time zone.
|
||||
/// But due to alignment, this field is totally zero cost.
|
||||
UInt8 days_in_month;
|
||||
|
||||
/// For days, when offset from UTC was changed due to daylight saving time or permanent change, following values could be non zero.
|
||||
UInt16 time_at_offset_change; /// In seconds from beginning of the day. Assuming offset never changed close to the end of day (so, value < 65536).
|
||||
Int16 amount_of_offset_change; /// Usually -3600 or 3600, but look at Lord Howe Island.
|
||||
@ -51,15 +52,16 @@ public:
|
||||
private:
|
||||
/// Lookup table is indexed by DayNum.
|
||||
/// Day nums are the same in all time zones. 1970-01-01 is 0 and so on.
|
||||
/// Table is relatively large (~30 000 elements), so better not to place object on stack.
|
||||
/// Table is relatively large, so better not to place the object on stack.
|
||||
/// In comparison to std::vector, plain array is cheaper by one indirection.
|
||||
Values lut[DATE_LUT_MAX_DAY_NUM + 1];
|
||||
Values lut[DATE_LUT_SIZE];
|
||||
|
||||
/// Year number after DATE_LUT_MIN_YEAR -> day num for start of year.
|
||||
DayNum_t years_lut[DATE_LUT_YEARS];
|
||||
|
||||
/// UTC offset at beginning of the Unix epoch.
|
||||
/// UTC offset at beginning of the Unix epoch. The same as unix timestamp of 1970-01-01 00:00:00 local time.
|
||||
time_t offset_at_start_of_epoch;
|
||||
bool offset_is_whole_number_of_hours_everytime;
|
||||
|
||||
/// Time zone name.
|
||||
std::string time_zone;
|
||||
@ -92,15 +94,10 @@ private:
|
||||
return lut[findIndex(t)];
|
||||
}
|
||||
|
||||
static inline DayNum_t fixDay(DayNum_t day)
|
||||
{
|
||||
return day > DATE_LUT_MAX_DAY_NUM ? static_cast<DayNum_t>(0) : day;
|
||||
}
|
||||
|
||||
public:
|
||||
const std::string & getTimeZone() const { return time_zone; }
|
||||
|
||||
/// всё ниже thread-safe; корректность входных данных не проверяется
|
||||
/// All functions below are thread-safe; arguments are not checked.
|
||||
|
||||
inline time_t toDate(time_t t) const { return find(t).date; }
|
||||
inline unsigned toMonth(time_t t) const { return find(t).month; }
|
||||
@ -108,47 +105,7 @@ public:
|
||||
inline unsigned toDayOfWeek(time_t t) const { return find(t).day_of_week; }
|
||||
inline unsigned toDayOfMonth(time_t t) const { return find(t).day_of_month; }
|
||||
|
||||
/// номер недели, начиная с какой-то недели в прошлом; неделя начинается с понедельника
|
||||
/// (переводим к понедельнику и делим DayNum на 7; будем исходить из допущения,
|
||||
/// что в области применения этой функции не было и не будет недель, состоящих не из семи дней)
|
||||
inline unsigned toRelativeWeekNum(DayNum_t d) const
|
||||
{
|
||||
return (d + 8 - lut[d].day_of_week) / 7;
|
||||
}
|
||||
|
||||
inline unsigned toRelativeWeekNum(time_t t) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
return (index + 8 - lut[index].day_of_week) / 7;
|
||||
}
|
||||
|
||||
/// номер месяца, начиная с какого-то месяца в прошлом (год * 12 + номер месяца в году)
|
||||
inline unsigned toRelativeMonthNum(DayNum_t d) const
|
||||
{
|
||||
return lut[d].year * 12 + lut[d].month;
|
||||
}
|
||||
|
||||
inline unsigned toRelativeMonthNum(time_t t) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
return lut[index].year * 12 + lut[index].month;
|
||||
}
|
||||
|
||||
/// делим unix timestamp на 3600;
|
||||
/// (таким образом, учитываются прошедшие интервалы времени длительностью в час, не зависимо от перевода стрелок;
|
||||
/// поддерживаются только часовые пояса, в которых перевод стрелок осуществлялся только на целое число часов)
|
||||
inline time_t toRelativeHourNum(time_t t) const
|
||||
{
|
||||
return t / 3600;
|
||||
}
|
||||
|
||||
/// делим unix timestamp на 60
|
||||
inline time_t toRelativeMinuteNum(time_t t) const
|
||||
{
|
||||
return t / 60;
|
||||
}
|
||||
|
||||
/// округление вниз до понедельника
|
||||
/// Round down to start of monday.
|
||||
inline time_t toFirstDayOfWeek(time_t t) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
@ -166,7 +123,7 @@ public:
|
||||
return DayNum_t(index - (lut[index].day_of_week - 1));
|
||||
}
|
||||
|
||||
/// округление вниз до первого числа месяца
|
||||
/// Round down to start of month.
|
||||
inline time_t toFirstDayOfMonth(time_t t) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
@ -175,7 +132,7 @@ public:
|
||||
|
||||
inline DayNum_t toFirstDayNumOfMonth(DayNum_t d) const
|
||||
{
|
||||
return DayNum_t(d - (lut[fixDay(d)].day_of_month - 1));
|
||||
return DayNum_t(d - (lut[d].day_of_month - 1));
|
||||
}
|
||||
|
||||
inline DayNum_t toFirstDayNumOfMonth(time_t t) const
|
||||
@ -184,53 +141,33 @@ public:
|
||||
return DayNum_t(index - (lut[index].day_of_month - 1));
|
||||
}
|
||||
|
||||
/// округление до первого числа квартала
|
||||
inline time_t toFirstDayOfQuarter(time_t t) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
switch (lut[index].month % 3)
|
||||
{
|
||||
case 0:
|
||||
index = index - lut[index].day_of_month;
|
||||
case 2:
|
||||
index = index - lut[index].day_of_month;
|
||||
case 1:
|
||||
index = index - lut[index].day_of_month + 1;
|
||||
}
|
||||
return DayNum_t(index);
|
||||
}
|
||||
|
||||
/// Round down to start of quarter.
|
||||
inline DayNum_t toFirstDayNumOfQuarter(DayNum_t d) const
|
||||
{
|
||||
size_t index = fixDay(d);
|
||||
switch (lut[index].month % 3)
|
||||
size_t index = d;
|
||||
size_t month_inside_quarter = (lut[index].month - 1) % 3;
|
||||
|
||||
index = index - lut[index].day_of_month;
|
||||
while (month_inside_quarter)
|
||||
{
|
||||
case 0:
|
||||
index = index - lut[index].day_of_month;
|
||||
case 2:
|
||||
index = index - lut[index].day_of_month;
|
||||
case 1:
|
||||
index = index - lut[index].day_of_month + 1;
|
||||
index = index - lut[index].day_of_month;
|
||||
--month_inside_quarter;
|
||||
}
|
||||
return DayNum_t(index);
|
||||
|
||||
return DayNum_t(index + 1);
|
||||
}
|
||||
|
||||
inline DayNum_t toFirstDayNumOfQuarter(time_t t) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
switch (lut[index].month % 3)
|
||||
{
|
||||
case 0:
|
||||
index = index - lut[index].day_of_month;
|
||||
case 2:
|
||||
index = index - lut[index].day_of_month;
|
||||
case 1:
|
||||
index = index - lut[index].day_of_month + 1;
|
||||
}
|
||||
return DayNum_t(index);
|
||||
return toFirstDayNumOfQuarter(toDayNum(t));
|
||||
}
|
||||
|
||||
/// округление вниз до первого числа года
|
||||
inline time_t toFirstDayOfQuarter(time_t t) const
|
||||
{
|
||||
return fromDayNum(toFirstDayNumOfQuarter(t));
|
||||
}
|
||||
|
||||
/// Round down to start of year.
|
||||
inline time_t toFirstDayOfYear(time_t t) const
|
||||
{
|
||||
return lut[years_lut[lut[findIndex(t)].year - DATE_LUT_MIN_YEAR]].date;
|
||||
@ -238,7 +175,7 @@ public:
|
||||
|
||||
inline DayNum_t toFirstDayNumOfYear(DayNum_t d) const
|
||||
{
|
||||
return years_lut[lut[fixDay(d)].year - DATE_LUT_MIN_YEAR];
|
||||
return years_lut[lut[d].year - DATE_LUT_MIN_YEAR];
|
||||
}
|
||||
|
||||
inline time_t toFirstDayNumOfYear(time_t t) const
|
||||
@ -246,7 +183,6 @@ public:
|
||||
return lut[years_lut[lut[findIndex(t)].year - DATE_LUT_MIN_YEAR]].date;
|
||||
}
|
||||
|
||||
/// первое число следующего месяца
|
||||
inline time_t toFirstDayOfNextMonth(time_t t) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
@ -254,7 +190,6 @@ public:
|
||||
return lut[index - (lut[index].day_of_month - 1)].date;
|
||||
}
|
||||
|
||||
/// первое число предыдущего месяца
|
||||
inline time_t toFirstDayOfPrevMonth(time_t t) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
@ -262,20 +197,25 @@ public:
|
||||
return lut[index - (lut[index].day_of_month - 1)].date;
|
||||
}
|
||||
|
||||
/// количество дней в месяце
|
||||
inline size_t daysInMonth(time_t t) const
|
||||
inline UInt8 daysInMonth(DayNum_t d) const
|
||||
{
|
||||
size_t today = findIndex(t);
|
||||
size_t start_of_month = today - (lut[today].day_of_month - 1);
|
||||
size_t next_month = start_of_month + 31;
|
||||
size_t start_of_next_month = next_month - (lut[next_month].day_of_month - 1);
|
||||
return start_of_next_month - start_of_month;
|
||||
return lut[d].days_in_month;
|
||||
}
|
||||
|
||||
/** Округление до даты; затем сдвиг на указанное количество дней.
|
||||
* Замечание: результат сдвига должен находиться в пределах LUT.
|
||||
inline UInt8 daysInMonth(time_t t) const
|
||||
{
|
||||
return find(t).days_in_month;
|
||||
}
|
||||
|
||||
inline UInt8 daysInMonth(UInt16 year, UInt8 month) const
|
||||
{
|
||||
auto any_day_of_month = years_lut[year - DATE_LUT_MIN_YEAR] + 31 * (month - 1);
|
||||
return lut[any_day_of_month].days_in_month;
|
||||
}
|
||||
|
||||
/** Round to start of day, then shift for specified amount of days.
|
||||
*/
|
||||
inline time_t toDateAndShift(time_t t, int days = 1) const
|
||||
inline time_t toDateAndShift(time_t t, Int32 days) const
|
||||
{
|
||||
return lut[findIndex(t) + days].date;
|
||||
}
|
||||
@ -283,17 +223,25 @@ public:
|
||||
inline time_t toTime(time_t t) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
|
||||
if (unlikely(index == 0))
|
||||
return t + offset_at_start_of_epoch;
|
||||
|
||||
time_t res = t - lut[index].date;
|
||||
|
||||
if (res >= lut[index].time_at_offset_change)
|
||||
res += lut[index].amount_of_offset_change;
|
||||
|
||||
return res - offset_at_start_of_epoch; /// Отсчёт от 1970-01-01 00:00:00 по локальному времени
|
||||
return res - offset_at_start_of_epoch; /// Starting at 1970-01-01 00:00:00 local time.
|
||||
}
|
||||
|
||||
inline unsigned toHour(time_t t) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
|
||||
if (unlikely(index == 0))
|
||||
return (t + offset_at_start_of_epoch) / 3600;
|
||||
|
||||
time_t res = t - lut[index].date;
|
||||
|
||||
if (res >= lut[index].time_at_offset_change)
|
||||
@ -302,46 +250,103 @@ public:
|
||||
return res / 3600;
|
||||
}
|
||||
|
||||
inline unsigned toMinute(time_t t) const { return ((t - find(t).date) % 3600) / 60; }
|
||||
inline unsigned toSecond(time_t t) const { return (t - find(t).date) % 60; }
|
||||
/** Only for time zones with/when offset from UTC is multiple of five minutes.
|
||||
* This is true for all time zones: right now, all time zones have an offset that is multiple of 15 minutes.
|
||||
*
|
||||
* "By 1929, most major countries had adopted hourly time zones. Nepal was the last
|
||||
* country to adopt a standard offset, shifting slightly to UTC+5:45 in 1986."
|
||||
* - https://en.wikipedia.org/wiki/Time_zone#Offsets_from_UTC
|
||||
*
|
||||
* Also please note, that unix timestamp doesn't count "leap seconds":
|
||||
* each minute, with added or subtracted leap second, spans exactly 60 unix timestamps.
|
||||
*/
|
||||
|
||||
inline unsigned toStartOfMinute(time_t t) const
|
||||
inline unsigned toSecond(time_t t) const { return t % 60; }
|
||||
|
||||
inline unsigned toMinute(time_t t) const
|
||||
{
|
||||
if (offset_is_whole_number_of_hours_everytime)
|
||||
return (t / 60) % 60;
|
||||
|
||||
time_t date = find(t).date;
|
||||
return date + (t - date) / 60 * 60;
|
||||
return (t - date) / 60 % 60;
|
||||
}
|
||||
|
||||
inline unsigned toStartOfHour(time_t t) const
|
||||
inline time_t toStartOfMinute(time_t t) const { return t / 60 * 60; }
|
||||
inline time_t toStartOfFiveMinute(time_t t) const { return t / 300 * 300; }
|
||||
|
||||
inline time_t toStartOfHour(time_t t) const
|
||||
{
|
||||
if (offset_is_whole_number_of_hours_everytime)
|
||||
return t / 3600 * 3600;
|
||||
|
||||
time_t date = find(t).date;
|
||||
/// Still can return wrong values for time at 1970-01-01 if the UTC offset was non-whole number of hours.
|
||||
return date + (t - date) / 3600 * 3600;
|
||||
}
|
||||
|
||||
/** Только для часовых поясов, отличающихся от UTC на значение, кратное часу и без перевода стрелок не значение не кратное часу */
|
||||
|
||||
inline unsigned toMinuteInaccurate(time_t t) const { return (t / 60) % 60; }
|
||||
inline unsigned toSecondInaccurate(time_t t) const { return t % 60; }
|
||||
|
||||
inline unsigned toStartOfMinuteInaccurate(time_t t) const { return t / 60 * 60; }
|
||||
inline unsigned toStartOfFiveMinuteInaccurate(time_t t) const { return t / 300 * 300; }
|
||||
inline unsigned toStartOfHourInaccurate(time_t t) const { return t / 3600 * 3600; }
|
||||
|
||||
/// Номер дня в пределах UNIX эпохи (и немного больше) - позволяет хранить дату в двух байтах
|
||||
/** Number of calendar day since the beginning of UNIX epoch (1970-01-01 is zero)
|
||||
* We use just two bytes for it. It covers the range up to 2105 and slightly more.
|
||||
*
|
||||
* This is "calendar" day, it itself is independent of time zone
|
||||
* (conversion from/to unix timestamp will depend on time zone,
|
||||
* because the same calendar day starts/ends at different timestamps in different time zones)
|
||||
*/
|
||||
|
||||
inline DayNum_t toDayNum(time_t t) const { return static_cast<DayNum_t>(findIndex(t)); }
|
||||
inline time_t fromDayNum(DayNum_t d) const { return lut[fixDay(d)].date; }
|
||||
inline time_t fromDayNum(DayNum_t d) const { return lut[d].date; }
|
||||
|
||||
inline time_t toDate(DayNum_t d) const { return lut[fixDay(d)].date; }
|
||||
inline unsigned toMonth(DayNum_t d) const { return lut[fixDay(d)].month; }
|
||||
inline unsigned toYear(DayNum_t d) const { return lut[fixDay(d)].year; }
|
||||
inline unsigned toDayOfWeek(DayNum_t d) const { return lut[fixDay(d)].day_of_week; }
|
||||
inline unsigned toDayOfMonth(DayNum_t d) const { return lut[fixDay(d)].day_of_month; }
|
||||
inline time_t toDate(DayNum_t d) const { return lut[d].date; }
|
||||
inline unsigned toMonth(DayNum_t d) const { return lut[d].month; }
|
||||
inline unsigned toYear(DayNum_t d) const { return lut[d].year; }
|
||||
inline unsigned toDayOfWeek(DayNum_t d) const { return lut[d].day_of_week; }
|
||||
inline unsigned toDayOfMonth(DayNum_t d) const { return lut[d].day_of_month; }
|
||||
|
||||
inline const Values & getValues(DayNum_t d) const { return lut[fixDay(d)]; }
|
||||
inline const Values & getValues(time_t t) const { return lut[findIndex(t)]; }
|
||||
/// Number of week from some fixed moment in the past. Week begins at monday.
|
||||
/// (round down to monday and divide DayNum by 7; we made an assumption,
|
||||
/// that in domain of the function there was no weeks with any other number of days than 7)
|
||||
inline unsigned toRelativeWeekNum(DayNum_t d) const
|
||||
{
|
||||
/// We add 8 to avoid underflow at beginning of unix epoch.
|
||||
return (d + 8 - lut[d].day_of_week) / 7;
|
||||
}
|
||||
|
||||
/// получает DayNum_t из года, месяца, дня
|
||||
inline DayNum_t makeDayNum(short year, char month, char day_of_month) const
|
||||
inline unsigned toRelativeWeekNum(time_t t) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
return (index + 8 - lut[index].day_of_week) / 7;
|
||||
}
|
||||
|
||||
/// Number of month from some fixed moment in the past (year * 12 + month)
|
||||
inline unsigned toRelativeMonthNum(DayNum_t d) const
|
||||
{
|
||||
return lut[d].year * 12 + lut[d].month;
|
||||
}
|
||||
|
||||
inline unsigned toRelativeMonthNum(time_t t) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
return lut[index].year * 12 + lut[index].month;
|
||||
}
|
||||
|
||||
/// We calculate all hour-length intervals, unrelated to offset changes.
|
||||
inline time_t toRelativeHourNum(time_t t) const
|
||||
{
|
||||
if (offset_is_whole_number_of_hours_everytime)
|
||||
return t / 3600;
|
||||
|
||||
/// Assume that if offset was fractional, then the fraction is the same as at the beginning of epoch.
|
||||
/// NOTE This assumption is false for "Pacific/Pitcairn" time zone.
|
||||
return (t + 86400 - offset_at_start_of_epoch) / 3600;
|
||||
}
|
||||
|
||||
inline time_t toRelativeMinuteNum(time_t t) const
|
||||
{
|
||||
return t / 60;
|
||||
}
|
||||
|
||||
/// Create DayNum_t from year, month, day of month.
|
||||
inline DayNum_t makeDayNum(UInt16 year, UInt8 month, UInt8 day_of_month) const
|
||||
{
|
||||
if (unlikely(year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31))
|
||||
return DayNum_t(0);
|
||||
@ -349,14 +354,14 @@ public:
|
||||
return DayNum_t(any_day_of_month - toDayOfMonth(any_day_of_month) + day_of_month);
|
||||
}
|
||||
|
||||
inline time_t makeDate(short year, char month, char day_of_month) const
|
||||
inline time_t makeDate(UInt16 year, UInt8 month, UInt8 day_of_month) const
|
||||
{
|
||||
return lut[makeDayNum(year, month, day_of_month)].date;
|
||||
}
|
||||
|
||||
/** Does not accept daylight saving time as argument: in case of ambiguity, it choose greater timestamp.
|
||||
*/
|
||||
inline time_t makeDateTime(short year, char month, char day_of_month, char hour, char minute, char second) const
|
||||
inline time_t makeDateTime(UInt16 year, UInt8 month, UInt8 day_of_month, UInt8 hour, UInt8 minute, UInt8 second) const
|
||||
{
|
||||
size_t index = makeDayNum(year, month, day_of_month);
|
||||
time_t time_offset = hour * 3600 + minute * 60 + second;
|
||||
@ -367,6 +372,9 @@ public:
|
||||
return lut[index].date + time_offset;
|
||||
}
|
||||
|
||||
inline const Values & getValues(DayNum_t d) const { return lut[d]; }
|
||||
inline const Values & getValues(time_t t) const { return lut[findIndex(t)]; }
|
||||
|
||||
inline UInt32 toNumYYYYMM(time_t t) const
|
||||
{
|
||||
const Values & values = find(t);
|
||||
@ -375,7 +383,7 @@ public:
|
||||
|
||||
inline UInt32 toNumYYYYMM(DayNum_t d) const
|
||||
{
|
||||
const Values & values = lut[fixDay(d)];
|
||||
const Values & values = lut[d];
|
||||
return values.year * 100 + values.month;
|
||||
}
|
||||
|
||||
@ -387,7 +395,7 @@ public:
|
||||
|
||||
inline UInt32 toNumYYYYMMDD(DayNum_t d) const
|
||||
{
|
||||
const Values & values = lut[fixDay(d)];
|
||||
const Values & values = lut[d];
|
||||
return values.year * 10000 + values.month * 100 + values.day_of_month;
|
||||
}
|
||||
|
||||
@ -406,8 +414,8 @@ public:
|
||||
{
|
||||
const Values & values = find(t);
|
||||
return
|
||||
toSecondInaccurate(t)
|
||||
+ toMinuteInaccurate(t) * 100
|
||||
toSecond(t)
|
||||
+ toMinute(t) * 100
|
||||
+ toHour(t) * 10000
|
||||
+ UInt64(values.day_of_month) * 1000000
|
||||
+ UInt64(values.month) * 100000000
|
||||
@ -425,6 +433,113 @@ public:
|
||||
num % 100);
|
||||
}
|
||||
|
||||
/// Adding calendar intervals.
|
||||
/// Implementation specific behaviour when delta is too big.
|
||||
|
||||
inline time_t addDays(time_t t, Int64 delta) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t);
|
||||
|
||||
index += delta;
|
||||
|
||||
if (time_offset >= lut[index].time_at_offset_change)
|
||||
time_offset -= lut[index].amount_of_offset_change;
|
||||
|
||||
return lut[index].date + time_offset;
|
||||
}
|
||||
|
||||
inline time_t addWeeks(time_t t, Int64 delta) const
|
||||
{
|
||||
return addDays(t, delta * 7);
|
||||
}
|
||||
|
||||
inline UInt8 saturateDayOfMonth(UInt16 year, UInt8 month, UInt8 day_of_month) const
|
||||
{
|
||||
if (likely(day_of_month <= 28))
|
||||
return day_of_month;
|
||||
|
||||
UInt8 days_in_month = daysInMonth(year, month);
|
||||
|
||||
if (day_of_month > days_in_month)
|
||||
day_of_month = days_in_month;
|
||||
|
||||
return day_of_month;
|
||||
}
|
||||
|
||||
/// If resulting month has less deys than source month, then saturation can happen.
|
||||
/// Example: 31 Aug + 1 month = 30 Sep.
|
||||
inline time_t addMonths(time_t t, Int64 delta) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
const Values & values = lut[index];
|
||||
|
||||
time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t);
|
||||
|
||||
auto month = values.month + delta;
|
||||
auto year = values.year + (month - 1) / 12;
|
||||
month = ((month - 1) % 12) + 1;
|
||||
auto day_of_month = saturateDayOfMonth(year, month, values.day_of_month);
|
||||
|
||||
DayNum_t result_day = makeDayNum(year, month, day_of_month);
|
||||
|
||||
if (time_offset >= lut[result_day].time_at_offset_change)
|
||||
time_offset -= lut[result_day].amount_of_offset_change;
|
||||
|
||||
return lut[result_day].date + time_offset;
|
||||
}
|
||||
|
||||
inline DayNum_t addMonths(DayNum_t d, Int64 delta) const
|
||||
{
|
||||
const Values & values = lut[d];
|
||||
|
||||
auto month = values.month + delta;
|
||||
auto year = values.year + (month - 1) / 12;
|
||||
month = ((month - 1) % 12) + 1;
|
||||
auto day_of_month = saturateDayOfMonth(year, month, values.day_of_month);
|
||||
|
||||
return makeDayNum(year, month, day_of_month);
|
||||
}
|
||||
|
||||
/// Saturation can occur if 29 Feb is mapped to non-leap year.
|
||||
inline time_t addYears(time_t t, Int64 delta) const
|
||||
{
|
||||
size_t index = findIndex(t);
|
||||
const Values & values = lut[index];
|
||||
|
||||
time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t);
|
||||
|
||||
auto year = values.year + delta;
|
||||
auto month = values.month;
|
||||
auto day_of_month = values.day_of_month;
|
||||
|
||||
/// Saturation to 28 Feb can happen.
|
||||
if (unlikely(day_of_month == 29 && month == 2))
|
||||
day_of_month = saturateDayOfMonth(year, month, day_of_month);
|
||||
|
||||
DayNum_t result_day = makeDayNum(year, month, day_of_month);
|
||||
|
||||
if (time_offset >= lut[result_day].time_at_offset_change)
|
||||
time_offset -= lut[result_day].amount_of_offset_change;
|
||||
|
||||
return lut[result_day].date + time_offset;
|
||||
}
|
||||
|
||||
inline DayNum_t addYears(DayNum_t d, Int64 delta) const
|
||||
{
|
||||
const Values & values = lut[d];
|
||||
|
||||
auto year = values.year + delta;
|
||||
auto month = values.month;
|
||||
auto day_of_month = values.day_of_month;
|
||||
|
||||
/// Saturation to 28 Feb can happen.
|
||||
if (unlikely(day_of_month == 29 && month == 2))
|
||||
day_of_month = saturateDayOfMonth(year, month, day_of_month);
|
||||
|
||||
return makeDayNum(year, month, day_of_month);
|
||||
}
|
||||
|
||||
|
||||
inline std::string timeToString(time_t t) const
|
||||
{
|
||||
@ -442,8 +557,8 @@ public:
|
||||
s[9] += values.day_of_month % 10;
|
||||
|
||||
auto hour = toHour(t);
|
||||
auto minute = toMinuteInaccurate(t);
|
||||
auto second = toSecondInaccurate(t);
|
||||
auto minute = toMinute(t);
|
||||
auto second = toSecond(t);
|
||||
|
||||
s[11] += hour / 10;
|
||||
s[12] += hour % 10;
|
||||
@ -475,7 +590,7 @@ public:
|
||||
|
||||
inline std::string dateToString(DayNum_t d) const
|
||||
{
|
||||
const Values & values = lut[fixDay(d)];
|
||||
const Values & values = lut[d];
|
||||
|
||||
std::string s {"0000-00-00"};
|
||||
|
||||
@ -490,4 +605,6 @@ public:
|
||||
|
||||
return s;
|
||||
}
|
||||
|
||||
inline bool isOffsetWholeNumberOfHoursEveryTime() const { return offset_is_whole_number_of_hours_everytime; }
|
||||
};
|
||||
|
@ -48,8 +48,8 @@ private:
|
||||
m_month = values.month;
|
||||
m_day = values.day_of_month;
|
||||
m_hour = date_lut.toHour(time);
|
||||
m_minute = date_lut.toMinuteInaccurate(time);
|
||||
m_second = date_lut.toSecondInaccurate(time);
|
||||
m_minute = date_lut.toMinute(time);
|
||||
m_second = date_lut.toSecond(time);
|
||||
}
|
||||
|
||||
void init(const char * s, size_t length)
|
||||
|
@ -6,28 +6,30 @@
|
||||
template <typename F, typename C>
|
||||
struct RangeFiltered
|
||||
{
|
||||
using RawIterator = typename C:: iterator;
|
||||
/// Template parameter C may be const. Then const_iterator is used.
|
||||
using RawIterator = decltype(std::declval<C>().begin());
|
||||
class Iterator;
|
||||
|
||||
/// Will iterate over elements for which filter(*it) == true
|
||||
RangeFiltered(F && filter, const C & container)
|
||||
: filter(std::move(filter)), container(container) {}
|
||||
template <typename F_, typename C_> /// Another template for universal references to work.
|
||||
RangeFiltered(F_ && filter, C_ && container)
|
||||
: filter(std::move(filter)), container(container) {}
|
||||
|
||||
Iterator begin() const
|
||||
{
|
||||
return {*this, std::begin(container)};
|
||||
return Iterator{*this, std::begin(container)};
|
||||
}
|
||||
|
||||
Iterator end() const
|
||||
{
|
||||
return {*this, std::end(container)};
|
||||
return Iterator{*this, std::end(container)};
|
||||
}
|
||||
|
||||
/// Convert ordinary iterator to filtered one
|
||||
/// Real position will be in range [ordinary_iterator; end()], so it is suitable to use with lower[upper]_bound()
|
||||
inline Iterator convert(RawIterator ordinary_iterator) const
|
||||
{
|
||||
return {*this, ordinary_iterator};
|
||||
return Iterator{*this, ordinary_iterator};
|
||||
}
|
||||
|
||||
|
||||
@ -114,12 +116,12 @@ struct RangeFiltered
|
||||
|
||||
protected:
|
||||
F filter;
|
||||
const C & container;
|
||||
C & container;
|
||||
};
|
||||
|
||||
|
||||
template <typename F, typename C>
|
||||
inline RangeFiltered<std::decay_t<F>, std::decay_t<C>> createRangeFiltered(F && filter, C && container)
|
||||
inline RangeFiltered<std::decay_t<F>, std::remove_reference_t<C>> createRangeFiltered(F && filter, C && container)
|
||||
{
|
||||
return {std::forward<F>(filter), std::forward<C>(container)};
|
||||
};
|
||||
|
@ -13,7 +13,27 @@ using UInt32 = Poco::UInt32;
|
||||
using UInt64 = Poco::UInt64;
|
||||
|
||||
|
||||
/// Обход проблемы с тем, что KDevelop не видит time_t и size_t (для подсветки синтаксиса).
|
||||
/** This is not the best way to overcome an issue of different definitions
|
||||
* of uint64_t and size_t on Linux and Mac OS X (both 64 bit).
|
||||
*
|
||||
* Note that on both platforms, long and long long are 64 bit types.
|
||||
* But they are always different types (with the same physical representation).
|
||||
*/
|
||||
namespace std
|
||||
{
|
||||
inline UInt64 max(unsigned long x, unsigned long long y) { return x > y ? x : y; }
|
||||
inline UInt64 max(unsigned long long x, unsigned long y) { return x > y ? x : y; }
|
||||
inline UInt64 min(unsigned long x, unsigned long long y) { return x < y ? x : y; }
|
||||
inline UInt64 min(unsigned long long x, unsigned long y) { return x < y ? x : y; }
|
||||
|
||||
inline Int64 max(long x, long long y) { return x > y ? x : y; }
|
||||
inline Int64 max(long long x, long y) { return x > y ? x : y; }
|
||||
inline Int64 min(long x, long long y) { return x < y ? x : y; }
|
||||
inline Int64 min(long long x, long y) { return x < y ? x : y; }
|
||||
}
|
||||
|
||||
|
||||
/// Workaround for the issue, that KDevelop doesn't see time_t and size_t types (for syntax highlight).
|
||||
#ifdef IN_KDEVELOP_PARSER
|
||||
using time_t = Int64;
|
||||
using size_t = UInt64;
|
||||
|
@ -9,6 +9,8 @@
|
||||
#include <cstring>
|
||||
#include <iostream>
|
||||
|
||||
#define DATE_LUT_MIN 0
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
@ -45,6 +47,7 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_)
|
||||
|
||||
cctz::time_zone::absolute_lookup start_of_epoch_lookup = cctz_time_zone.lookup(std::chrono::system_clock::from_time_t(start_of_day));
|
||||
offset_at_start_of_epoch = start_of_epoch_lookup.offset;
|
||||
offset_is_whole_number_of_hours_everytime = true;
|
||||
|
||||
cctz::civil_day date{1970, 1, 1};
|
||||
|
||||
@ -61,9 +64,20 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_)
|
||||
values.day_of_week = getDayOfWeek(date);
|
||||
values.date = start_of_day;
|
||||
|
||||
if (values.day_of_month == 1)
|
||||
{
|
||||
cctz::civil_month month(date);
|
||||
values.days_in_month = cctz::civil_day(month + 1) - cctz::civil_day(month);
|
||||
}
|
||||
else
|
||||
values.days_in_month = i != 0 ? lut[i - 1].days_in_month : 31;
|
||||
|
||||
values.time_at_offset_change = 0;
|
||||
values.amount_of_offset_change = 0;
|
||||
|
||||
if (start_of_day % 3600)
|
||||
offset_is_whole_number_of_hours_everytime = false;
|
||||
|
||||
/// If UTC offset was changed in previous day.
|
||||
if (i != 0)
|
||||
{
|
||||
@ -102,7 +116,14 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_)
|
||||
}
|
||||
while (start_of_day <= DATE_LUT_MAX && i <= DATE_LUT_MAX_DAY_NUM);
|
||||
|
||||
/// Заполняем lookup таблицу для годов
|
||||
/// Fill excessive part of lookup table. This is needed only to simplify handling of overflow cases.
|
||||
while (i < DATE_LUT_SIZE)
|
||||
{
|
||||
lut[i] = lut[0];
|
||||
++i;
|
||||
}
|
||||
|
||||
/// Fill lookup table for years.
|
||||
::memset(years_lut, 0, DATE_LUT_YEARS * sizeof(years_lut[0]));
|
||||
for (size_t day = 0; day < i && lut[day].year <= DATE_LUT_MAX_YEAR; ++day)
|
||||
{
|
||||
|
@ -14,9 +14,8 @@ target_link_libraries (date_lut2 common ${PLATFORM_LIBS})
|
||||
target_link_libraries (date_lut3 common ${PLATFORM_LIBS})
|
||||
target_link_libraries (date_lut4 common ${PLATFORM_LIBS})
|
||||
target_link_libraries (date_lut_default_timezone common ${PLATFORM_LIBS})
|
||||
target_link_libraries (multi_version common ${Boost_SYSTEM_LIBRARY} ${Boost_FILESYSTEM_LIBRARY} ${RT_LIBRARIES})
|
||||
target_link_libraries (multi_version common ${Boost_FILESYSTEM_LIBRARY} ${Boost_SYSTEM_LIBRARY} ${RT_LIBRARIES})
|
||||
|
||||
add_executable (unit_tests_libcommon gtest_json_test.cpp gtest_strong_typedef.cpp)
|
||||
target_link_libraries (unit_tests_libcommon gtest_main)
|
||||
target_link_libraries (unit_tests_libcommon ${Boost_SYSTEM_LIBRARY} ${RT_LIBRARIES} common)
|
||||
target_link_libraries (unit_tests_libcommon gtest_main common ${Boost_FILESYSTEM_LIBRARY} ${Boost_SYSTEM_LIBRARY} ${RT_LIBRARIES})
|
||||
add_check(unit_tests_libcommon)
|
||||
|
@ -3,7 +3,7 @@
|
||||
set -e
|
||||
|
||||
BRANCH=trunk
|
||||
#BRANCH=tags/RELEASE_400/final
|
||||
#BRANCH=tags/RELEASE_500/final
|
||||
|
||||
THREADS=$(grep -c ^processor /proc/cpuinfo)
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user