diff --git a/src/Functions/formatReadableTimeDelta.cpp b/src/Functions/formatReadableTimeDelta.cpp index 5fd48c01e8f..96d3a6fd966 100644 --- a/src/Functions/formatReadableTimeDelta.cpp +++ b/src/Functions/formatReadableTimeDelta.cpp @@ -2,11 +2,11 @@ #include #include #include -#include #include #include #include #include +#include namespace DB @@ -14,9 +14,10 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int BAD_ARGUMENTS; +extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int BAD_ARGUMENTS; } namespace @@ -52,75 +53,88 @@ public: { if (arguments.empty()) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be at least 1.", - getName(), arguments.size()); + "Number of arguments for function {} doesn't match: passed {}, should be at least 1.", + getName(), arguments.size()); - if (arguments.size() > 2) + if (arguments.size() > 3) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be at most 2.", - getName(), arguments.size()); + "Number of arguments for function {} doesn't match: passed {}, should be 1, 2 or 3.", + getName(), arguments.size()); const IDataType & type = *arguments[0]; if (!isNativeNumber(type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot format {} as time delta", type.getName()); - if (arguments.size() == 2) + if (arguments.size() >= 2) { const auto * maximum_unit_arg = arguments[1].get(); if (!isStringOrFixedString(maximum_unit_arg)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument maximum_unit of function {}", maximum_unit_arg->getName(), getName()); + + if (arguments.size() == 3) + { + const auto * minimum_unit_arg = arguments[2].get(); + if (!isStringOrFixedString(minimum_unit_arg)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument minimum_unit of function {}", + minimum_unit_arg->getName(), getName()); + } } return std::make_shared(); } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } bool useDefaultImplementationForConstants() const override { return true; } enum Unit { - Seconds, - Minutes, - Hours, - Days, - Months, - Years + Nanoseconds = 1, + Microseconds = 2, + Milliseconds = 3, + Seconds = 4, + Minutes = 5, + Hours = 6, + Days = 7, + Months = 8, + Years = 9 }; ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - std::string_view maximum_unit_str; - if (arguments.size() == 2) + std::string_view maximum_unit_str, minimum_unit_str; + if (arguments.size() >= 2) { const ColumnPtr & maximum_unit_column = arguments[1].column; const ColumnConst * maximum_unit_const_col = checkAndGetColumnConstStringOrFixedString(maximum_unit_column.get()); if (maximum_unit_const_col) maximum_unit_str = maximum_unit_const_col->getDataColumn().getDataAt(0).toView(); + + if (arguments.size() == 3) + { + const ColumnPtr & minimum_unit_column = arguments[2].column; + const ColumnConst * minimum_unit_const_col = checkAndGetColumnConstStringOrFixedString(minimum_unit_column.get()); + if (minimum_unit_const_col) + minimum_unit_str = minimum_unit_const_col->getDataColumn().getDataAt(0).toView(); + } } + /// Default means "use all available whole units". + Unit max_unit = dispatchUnit(maximum_unit_str, Years, "maximum"); + /// Set seconds as min_unit by default not to ruin old use cases + Unit min_unit = dispatchUnit(minimum_unit_str, Seconds, "minimum"); - Unit max_unit; - - /// Default means "use all available units". - if (maximum_unit_str.empty() || maximum_unit_str == "years") - max_unit = Years; - else if (maximum_unit_str == "months") - max_unit = Months; - else if (maximum_unit_str == "days") - max_unit = Days; - else if (maximum_unit_str == "hours") - max_unit = Hours; - else if (maximum_unit_str == "minutes") - max_unit = Minutes; - else if (maximum_unit_str == "seconds") - max_unit = Seconds; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unexpected value of maximum unit argument ({}) for function {}, the only allowed values are:" - " 'seconds', 'minutes', 'hours', 'days', 'months', 'years'.", - maximum_unit_str, getName()); + if (min_unit > max_unit) + { + if (minimum_unit_str.empty()) + min_unit = Nanoseconds; /// User wants sub-second max_unit. Show him all sub-second units unless other specified. + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Explicitly specified value of minimum unit argument ({}) for function {} " + "must not be greater than maximum unit value ({}).", + minimum_unit_str, getName(), maximum_unit_str); + } auto col_to = ColumnString::create(); @@ -152,14 +166,61 @@ public: /// To output separators between parts: ", " and " and ". bool has_output = false; + Float64 whole_part; + std::string fractional_str = getFractionalString(std::modf(value, &whole_part)); + switch (max_unit) /// A kind of Duff Device. { - case Years: processUnit(365 * 24 * 3600, " year", 5, value, buf_to, has_output); [[fallthrough]]; - case Months: processUnit(static_cast(30.5 * 24 * 3600), " month", 6, value, buf_to, has_output); [[fallthrough]]; - case Days: processUnit(24 * 3600, " day", 4, value, buf_to, has_output); [[fallthrough]]; - case Hours: processUnit(3600, " hour", 5, value, buf_to, has_output); [[fallthrough]]; - case Minutes: processUnit(60, " minute", 7, value, buf_to, has_output); [[fallthrough]]; - case Seconds: processUnit(1, " second", 7, value, buf_to, has_output); + case Years: + processUnit(365 * 24 * 3600, 0, " year", 5, whole_part, fractional_str, buf_to, has_output, min_unit, min_unit == Years); + if (min_unit == Years) + break; + [[fallthrough]]; + + case Months: + processUnit(static_cast(30.5 * 24 * 3600), 0, " month", 6, whole_part, fractional_str, buf_to, has_output, min_unit, min_unit == Months); + if (min_unit == Months) + break; + [[fallthrough]]; + + case Days: + processUnit(24 * 3600, 0, " day", 4, whole_part, fractional_str, buf_to, has_output, min_unit, min_unit == Days); + if (min_unit == Days) + break; + [[fallthrough]]; + + case Hours: + processUnit(3600, 0, " hour", 5, whole_part, fractional_str, buf_to, has_output, min_unit, min_unit == Hours); + if (min_unit == Hours) + break; + [[fallthrough]]; + + case Minutes: + processUnit(60, 0, " minute", 7, whole_part, fractional_str, buf_to, has_output, min_unit, min_unit == Minutes); + if (min_unit == Minutes) + break; + [[fallthrough]]; + + case Seconds: + processUnit(1, 0, " second", 7, whole_part, fractional_str, buf_to, has_output, min_unit, min_unit == Seconds); + if (min_unit == Seconds) + break; + [[fallthrough]]; + + case Milliseconds: + processUnit(1, 3, " millisecond", 12, whole_part, fractional_str, buf_to, has_output, min_unit, min_unit == Milliseconds); + if (min_unit == Milliseconds) + break; + [[fallthrough]]; + + case Microseconds: + processUnit(1, 6, " microsecond", 12, whole_part, fractional_str, buf_to, has_output, min_unit, min_unit == Microseconds); + if (min_unit == Microseconds) + break; + [[fallthrough]]; + + case Nanoseconds: + processUnit(1, 9, " nanosecond", 11, whole_part, fractional_str, buf_to, has_output, min_unit, true); } } @@ -172,42 +233,68 @@ public: } static void processUnit( - UInt64 unit_size, const char * unit_name, size_t unit_name_size, - Float64 & value, WriteBuffer & buf_to, bool & has_output) + UInt64 unit_multiplier, UInt32 unit_scale, const char * unit_name, size_t unit_name_size, Float64 & whole_part, + String & fractional_str, WriteBuffer & buf_to, bool & has_output, Unit min_unit, bool is_minimum_unit) { - if (unlikely(value + 1.0 == value)) + if (unlikely(whole_part + 1.0 == whole_part)) { /// The case when value is too large so exact representation for subsequent smaller units is not possible. - writeText(std::floor(value / unit_size), buf_to); + writeText(std::floor(whole_part * DecimalUtils::scaleMultiplier(unit_scale) / unit_multiplier), buf_to); buf_to.write(unit_name, unit_name_size); writeChar('s', buf_to); has_output = true; - value = 0; + whole_part = 0; return; } - - UInt64 num_units = static_cast(value / unit_size); - - if (!num_units) + UInt64 num_units = 0; + if (unit_scale == 0) /// dealing with whole number of seconds { - /// Zero units, no need to print. But if it's the last (seconds) and the only unit, print "0 seconds" nevertheless. - if (unit_size > 1 || has_output) - return; + num_units = static_cast(std::floor(whole_part / unit_multiplier)); + + if (!num_units) + { + /// Zero units, no need to print. But if it's the last (seconds) and the only unit, print "0 seconds" nevertheless. + if (unit_multiplier != 1 || has_output) + return; + } + + /// Remaining value to print on next iteration. + whole_part -= num_units * unit_multiplier; + } + else /// dealing with sub-seconds, a bit more peculiar to avoid more precision issues + { + if (whole_part >= 1) /// There were no whole units printed + { + num_units += static_cast(whole_part) * DecimalUtils::scaleMultiplier(unit_scale); + whole_part = 0; + } + + for (UInt32 i = 0; i < unit_scale; ++i) + { + num_units += (fractional_str[i] - '0') * DecimalUtils::scaleMultiplier(unit_scale - i - 1); + fractional_str[i] = '0'; + } + + if (!num_units) + { + /// Zero units, no need to print. But if it's the last (nanoseconds) and the only unit, print "0 nanoseconds" nevertheless. + if (!is_minimum_unit || has_output) + return; + } } - /// Remaining value to print on next iteration. - value -= num_units * unit_size; - + /// Write number of units if (has_output) { /// Need delimiter between values. The last delimiter is " and ", all previous are comma. - if (value < 1) + if (is_minimum_unit || (whole_part < 1 && fractional_str.substr(0, (4 - min_unit) * 3) == std::string((4 - min_unit) * 3, '0'))) writeCString(" and ", buf_to); else writeCString(", ", buf_to); } writeText(num_units, buf_to); + buf_to.write(unit_name, unit_name_size); /// If we just leave strlen(unit_name) here, clang-11 fails to make it compile-time. /// How to pronounce: unit vs. units. @@ -216,6 +303,47 @@ public: has_output = true; } + +private: + static std::string getFractionalString(const Float64 & fractional_part) + { + DB::DoubleConverter::BufferType buffer; + double_conversion::StringBuilder builder{buffer, sizeof(buffer)}; + + if (!DB::DoubleConverter::instance().ToFixed(fractional_part, 9, &builder)) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, "Cannot print double number: {}", fractional_part); + + return std::string(buffer, builder.position()).substr(2); /// do not return `0.` -- we don't need it + } + + Unit dispatchUnit(const std::string_view & unit_str, const Unit default_unit, const std::string & bound_name) const + { + if (unit_str.empty()) + return default_unit; + else if (unit_str == "years") + return Years; + else if (unit_str == "months") + return Months; + else if (unit_str == "days") + return Days; + else if (unit_str == "hours") + return Hours; + else if (unit_str == "minutes") + return Minutes; + else if (unit_str == "seconds") + return Seconds; + else if (unit_str == "milliseconds") + return Milliseconds; + else if (unit_str == "microseconds") + return Microseconds; + else if (unit_str == "nanoseconds") + return Nanoseconds; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Unexpected value of {} unit argument ({}) for function {}, the only allowed values are:" + " 'nanoseconds', 'microseconds', 'nanoseconds', 'seconds', 'minutes', 'hours', 'days', 'months', 'years'.", + bound_name, unit_str, getName()); + } }; } diff --git a/tests/queries/0_stateless/02887_format_readable_timedelta_subseconds.reference b/tests/queries/0_stateless/02887_format_readable_timedelta_subseconds.reference new file mode 100644 index 00000000000..389479eb529 --- /dev/null +++ b/tests/queries/0_stateless/02887_format_readable_timedelta_subseconds.reference @@ -0,0 +1,24 @@ +1 second +1 minute +2 minutes and 3 seconds +24 seconds +35 seconds +1 minute and 6 seconds +1 minute and 7 seconds +48 seconds +1 minute and 39 seconds +1 hour +1120 milliseconds +60200 milliseconds +123330 milliseconds +24450 milliseconds +35570 milliseconds +66640 milliseconds +67797 milliseconds, 979 microseconds and 790 nanoseconds +48888 milliseconds and 888 microseconds +99960 milliseconds +3600000 milliseconds +0 seconds +1 hour and 1 second +0 nanoseconds +1000 milliseconds and 500 microseconds diff --git a/tests/queries/0_stateless/02887_format_readable_timedelta_subseconds.sql b/tests/queries/0_stateless/02887_format_readable_timedelta_subseconds.sql new file mode 100644 index 00000000000..0a8a76d54fd --- /dev/null +++ b/tests/queries/0_stateless/02887_format_readable_timedelta_subseconds.sql @@ -0,0 +1,30 @@ +-- max_unit bigger than second, min_unit omitted (and considered 'seconds') +WITH + 'hours' AS maximum_unit, + arrayJoin([1.12, 60.2, 123.33, 24.45, 35.57, 66.64, 67.79, 48.88, 99.96, 3600]) AS elapsed +SELECT + formatReadableTimeDelta(elapsed, maximum_unit) AS time_delta; + +-- max_unit smaller than second, min_unit omitted (and considered 'nanoseconds') +WITH + 'milliseconds' AS maximum_unit, + arrayJoin([1.12, 60.2, 123.33, 24.45, 35.57, 66.64, 67.79797979, 48.888888, 99.96, 3600]) AS elapsed +SELECT + formatReadableTimeDelta(elapsed, maximum_unit) AS time_delta; + +-- Check exception is thrown +SELECT formatReadableTimeDelta(1.1, 'seconds', 'hours'); -- { serverError BAD_ARGUMENTS } + +-- Check empty units are omitted unless they are the only one +WITH + 'hours' AS maximum_unit, + 'microseconds' as minimum_unit, + arrayJoin([0, 3601.000000003]) AS elapsed +SELECT + formatReadableTimeDelta(elapsed, maximum_unit, minimum_unit); + +WITH + 'milliseconds' AS maximum_unit, + arrayJoin([0, 1.0005]) AS elapsed +SELECT + formatReadableTimeDelta(elapsed, maximum_unit);