#pragma once #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } /// See DateTimeTransforms.h template class FunctionDateOrDateTimeToSomething : public IFunction { public: static constexpr auto name = Transform::name; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } bool isVariadic() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.size() == 1) { if (!isDateOrDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception( "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } else if (arguments.size() == 2) { if (!isDateOrDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception( "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (!isString(arguments[1].type)) throw Exception( "Function " + getName() + " supports 1 or 2 arguments. The 1st argument " "must be of type Date or DateTime. The 2nd argument (optional) must be " "a constant string with timezone name", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if ((isDate(arguments[0].type) || isDate32(arguments[0].type)) && (std::is_same_v || std::is_same_v)) throw Exception( "The timezone argument of function " + getName() + " is allowed only when the 1st argument has the type DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } else throw Exception( "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 1 or 2", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); /// For DateTime, if time zone is specified, attach it to type. /// If the time zone is specified but empty, throw an exception. if constexpr (std::is_same_v) { std::string time_zone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0); /// only validate the time_zone part if the number of arguments is 2. This is mainly /// to accommodate functions like toStartOfDay(today()), toStartOfDay(yesterday()) etc. if (arguments.size() == 2 && time_zone.empty()) throw Exception( "Function " + getName() + " supports a 2nd argument (optional) that must be non-empty and be a valid time zone", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(time_zone); } if constexpr (std::is_same_v) { Int64 scale = DataTypeDateTime64::default_scale; if (const auto * dt64 = checkAndGetDataType(arguments[0].type.get())) scale = dt64->getScale(); return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, 1, 0)); } else return std::make_shared(); } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { const IDataType * from_type = arguments[0].type.get(); WhichDataType which(from_type); if (which.isDate()) return DateTimeTransformImpl::execute(arguments, result_type, input_rows_count); else if (which.isDate32()) return DateTimeTransformImpl::execute(arguments, result_type, input_rows_count); else if (which.isDateTime()) return DateTimeTransformImpl::execute(arguments, result_type, input_rows_count); else if (which.isDateTime64()) { const auto scale = static_cast(from_type)->getScale(); const TransformDateTime64 transformer(scale); return DateTimeTransformImpl::execute(arguments, result_type, input_rows_count, transformer); } else throw Exception("Illegal type " + arguments[0].type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } bool hasInformationAboutMonotonicity() const override { return true; } Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override { IFunction::Monotonicity is_monotonic { true }; IFunction::Monotonicity is_not_monotonic; if (std::is_same_v) { is_monotonic.is_always_monotonic = true; return is_monotonic; } /// This method is called only if the function has one argument. Therefore, we do not care about the non-local time zone. const DateLUTImpl & date_lut = DateLUT::instance(); if (left.isNull() || right.isNull()) return is_not_monotonic; /// The function is monotonous on the [left, right] segment, if the factor transformation returns the same values for them. if (checkAndGetDataType(&type)) { return Transform::FactorTransform::execute(UInt16(left.get()), date_lut) == Transform::FactorTransform::execute(UInt16(right.get()), date_lut) ? is_monotonic : is_not_monotonic; } else if (checkAndGetDataType(&type)) { return Transform::FactorTransform::execute(Int32(left.get()), date_lut) == Transform::FactorTransform::execute(Int32(right.get()), date_lut) ? is_monotonic : is_not_monotonic; } else { return Transform::FactorTransform::execute(UInt32(left.get()), date_lut) == Transform::FactorTransform::execute(UInt32(right.get()), date_lut) ? is_monotonic : is_not_monotonic; } } }; }