Merge branch 'master' into new-block-for-functions-3

This commit is contained in:
Nikolai Kochetov 2020-10-14 23:37:10 +03:00
commit d0112c5dc4
23 changed files with 746 additions and 315 deletions

View File

@ -313,13 +313,4 @@ namespace ZeroTraits
}
inline bool operator==(StringRef lhs, const char * rhs)
{
for (size_t pos = 0; pos < lhs.size; ++pos)
if (!rhs[pos] || lhs.data[pos] != rhs[pos])
return false;
return true;
}
std::ostream & operator<<(std::ostream & os, const StringRef & str);

View File

@ -33,6 +33,12 @@ server_pid=none
function stop_server
{
if ! kill -0 -- "$server_pid"
then
echo "ClickHouse server pid '$server_pid' is not running"
return 0
fi
for _ in {1..60}
do
if ! pkill -f "clickhouse-server" && ! kill -- "$server_pid" ; then break ; fi
@ -168,8 +174,8 @@ clickhouse-test --help
mkdir -p "$FASTTEST_DATA"{,/client-config}
cp -a "$FASTTEST_SOURCE/programs/server/"{config,users}.xml "$FASTTEST_DATA"
cp -a "$FASTTEST_SOURCE/programs/server/"{config,users}.xml "$FASTTEST_DATA"
"$FASTTEST_SOURCE/tests/config/install.sh" "$FASTTEST_DATA" "$FASTTEST_DATA/client-config"
cp -a "$FASTTEST_SOURCE/programs/server/config.d/log_to_console.xml" "$FASTTEST_DATA/config.d"
# doesn't support SSL
rm -f "$FASTTEST_DATA/config.d/secure_ports.xml"
}
@ -269,7 +275,7 @@ then
stop_server ||:
# Clean the data so that there is no interference from the previous test run.
rm -rf "$FASTTEST_DATA"/{meta,}data ||:
rm -rf "$FASTTEST_DATA"/{{meta,}data,user_files} ||:
start_server
@ -329,7 +335,10 @@ case "$stage" in
;&
"run_tests")
run_tests
;&
;;
*)
echo "Unknown test stage '$stage'"
exit 1
esac
pstree -apgT

View File

@ -538,6 +538,51 @@ SELECT
└────────────────┴───────────────────┘
```
## formatReadableTimeDelta {#formatreadabletimedelta}
Accepts the time delta in seconds. Returns a time delta with (year, month, day, hour, minute, second) as a string.
**Syntax**
``` sql
formatReadableTimeDelta(column[, maximum_unit])
```
**Parameters**
- `column` — A column with numeric time delta.
- `maximum_unit` — Optional. Maximum unit to show. Acceptable values seconds, minutes, hours, days, months, years.
Example:
``` sql
SELECT
arrayJoin([100, 12345, 432546534]) AS elapsed,
formatReadableTimeDelta(elapsed) AS time_delta
```
``` text
┌────elapsed─┬─time_delta ─────────────────────────────────────────────────────┐
│ 100 │ 1 minute and 40 seconds │
│ 12345 │ 3 hours, 25 minutes and 45 seconds │
│ 432546534 │ 13 years, 8 months, 17 days, 7 hours, 48 minutes and 54 seconds │
└────────────┴─────────────────────────────────────────────────────────────────┘
```
``` sql
SELECT
arrayJoin([100, 12345, 432546534]) AS elapsed,
formatReadableTimeDelta(elapsed, 'minutes') AS time_delta
```
``` text
┌────elapsed─┬─time_delta ─────────────────────────────────────────────────────┐
│ 100 │ 1 minute and 40 seconds │
│ 12345 │ 205 minutes and 45 seconds │
│ 432546534 │ 7209108 minutes and 54 seconds │
└────────────┴─────────────────────────────────────────────────────────────────┘
```
## least(a, b) {#leasta-b}
Returns the smallest value from a and b.

View File

@ -1,15 +0,0 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsFormatting.h>
namespace DB
{
void registerFunctionsFormatting(FunctionFactory & factory)
{
factory.registerFunction<FunctionBitmaskToList>();
factory.registerFunction<FunctionFormatReadableSize>();
factory.registerFunction<FunctionFormatReadableQuantity>();
}
}

View File

@ -1,282 +0,0 @@
#pragma once
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnConst.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/WriteBufferFromVector.h>
#include <IO/WriteHelpers.h>
#include <Common/formatReadable.h>
#include <Common/typeid_cast.h>
#include <type_traits>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
/** Function for an unusual conversion to a string:
*
* bitmaskToList - takes an integer - a bitmask, returns a string of degrees of 2 separated by a comma.
* for example, bitmaskToList(50) = '2,16,32'
*
* formatReadableSize - prints the transferred size in bytes in form `123.45 GiB`.
*/
class FunctionBitmaskToList : public IFunction
{
public:
static constexpr auto name = "bitmaskToList";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionBitmaskToList>(); }
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 1; }
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const DataTypePtr & type = arguments[0];
if (!isInteger(type))
throw Exception("Cannot format " + type->getName() + " as bitmask string", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>();
}
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
if (!(executeType<UInt8>(columns, arguments, result)
|| executeType<UInt16>(columns, arguments, result)
|| executeType<UInt32>(columns, arguments, result)
|| executeType<UInt64>(columns, arguments, result)
|| executeType<Int8>(columns, arguments, result)
|| executeType<Int16>(columns, arguments, result)
|| executeType<Int32>(columns, arguments, result)
|| executeType<Int64>(columns, arguments, result)))
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
private:
template <typename T>
inline static void writeBitmask(T x, WriteBuffer & out)
{
using UnsignedT = make_unsigned_t<T>;
UnsignedT u_x = x;
bool first = true;
while (u_x)
{
UnsignedT y = u_x & (u_x - 1);
UnsignedT bit = u_x ^ y;
u_x = y;
if (!first)
writeChar(',', out);
first = false;
writeIntText(T(bit), out);
}
}
template <typename T>
bool executeType(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result) const
{
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(columns[arguments[0]].column.get()))
{
auto col_to = ColumnString::create();
const typename ColumnVector<T>::Container & vec_from = col_from->getData();
ColumnString::Chars & data_to = col_to->getChars();
ColumnString::Offsets & offsets_to = col_to->getOffsets();
size_t size = vec_from.size();
data_to.resize(size * 2);
offsets_to.resize(size);
WriteBufferFromVector<ColumnString::Chars> buf_to(data_to);
for (size_t i = 0; i < size; ++i)
{
writeBitmask<T>(vec_from[i], buf_to);
writeChar(0, buf_to);
offsets_to[i] = buf_to.count();
}
buf_to.finalize();
columns[result].column = std::move(col_to);
}
else
{
return false;
}
return true;
}
};
class FunctionFormatReadableSize : public IFunction
{
public:
static constexpr auto name = "formatReadableSize";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionFormatReadableSize>(); }
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 1; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const IDataType & type = *arguments[0];
if (!isNativeNumber(type))
throw Exception("Cannot format " + type.getName() + " as size in bytes", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>();
}
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
if (!(executeType<UInt8>(columns, arguments, result)
|| executeType<UInt16>(columns, arguments, result)
|| executeType<UInt32>(columns, arguments, result)
|| executeType<UInt64>(columns, arguments, result)
|| executeType<Int8>(columns, arguments, result)
|| executeType<Int16>(columns, arguments, result)
|| executeType<Int32>(columns, arguments, result)
|| executeType<Int64>(columns, arguments, result)
|| executeType<Float32>(columns, arguments, result)
|| executeType<Float64>(columns, arguments, result)))
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
private:
template <typename T>
bool executeType(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result) const
{
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(columns[arguments[0]].column.get()))
{
auto col_to = ColumnString::create();
const typename ColumnVector<T>::Container & vec_from = col_from->getData();
ColumnString::Chars & data_to = col_to->getChars();
ColumnString::Offsets & offsets_to = col_to->getOffsets();
size_t size = vec_from.size();
data_to.resize(size * 2);
offsets_to.resize(size);
WriteBufferFromVector<ColumnString::Chars> buf_to(data_to);
for (size_t i = 0; i < size; ++i)
{
formatReadableSizeWithBinarySuffix(static_cast<double>(vec_from[i]), buf_to);
writeChar(0, buf_to);
offsets_to[i] = buf_to.count();
}
buf_to.finalize();
columns[result].column = std::move(col_to);
return true;
}
return false;
}
};
class FunctionFormatReadableQuantity : public IFunction
{
public:
static constexpr auto name = "formatReadableQuantity";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionFormatReadableQuantity>(); }
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 1; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const IDataType & type = *arguments[0];
if (!isNativeNumber(type))
throw Exception("Cannot format " + type.getName() + " as quantity", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>();
}
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
if (!(executeType<UInt8>(columns, arguments, result)
|| executeType<UInt16>(columns, arguments, result)
|| executeType<UInt32>(columns, arguments, result)
|| executeType<UInt64>(columns, arguments, result)
|| executeType<Int8>(columns, arguments, result)
|| executeType<Int16>(columns, arguments, result)
|| executeType<Int32>(columns, arguments, result)
|| executeType<Int64>(columns, arguments, result)
|| executeType<Float32>(columns, arguments, result)
|| executeType<Float64>(columns, arguments, result)))
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
private:
template <typename T>
bool executeType(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result) const
{
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(columns[arguments[0]].column.get()))
{
auto col_to = ColumnString::create();
const typename ColumnVector<T>::Container & vec_from = col_from->getData();
ColumnString::Chars & data_to = col_to->getChars();
ColumnString::Offsets & offsets_to = col_to->getOffsets();
size_t size = vec_from.size();
data_to.resize(size * 2);
offsets_to.resize(size);
WriteBufferFromVector<ColumnString::Chars> buf_to(data_to);
for (size_t i = 0; i < size; ++i)
{
formatReadableQuantity(static_cast<double>(vec_from[i]), buf_to);
writeChar(0, buf_to);
offsets_to[i] = buf_to.count();
}
buf_to.finalize();
columns[result].column = std::move(col_to);
return true;
}
return false;
}
};
}

View File

@ -0,0 +1,134 @@
#include <Functions/FunctionFactory.h>
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeString.h>
#include <IO/WriteBufferFromVector.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
/** Function for an unusual conversion to a string:
*
* bitmaskToList - takes an integer - a bitmask, returns a string of degrees of 2 separated by a comma.
* for example, bitmaskToList(50) = '2,16,32'
*/
namespace
{
class FunctionBitmaskToList : public IFunction
{
public:
static constexpr auto name = "bitmaskToList";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionBitmaskToList>(); }
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 1; }
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const DataTypePtr & type = arguments[0];
if (!isInteger(type))
throw Exception("Cannot format " + type->getName() + " as bitmask string", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>();
}
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers &, size_t result, size_t /*input_rows_count*/) const override
{
if (!(executeType<UInt8>(columns, result)
|| executeType<UInt16>(columns, result)
|| executeType<UInt32>(columns, result)
|| executeType<UInt64>(columns, result)
|| executeType<Int8>(columns, result)
|| executeType<Int16>(columns, result)
|| executeType<Int32>(columns, result)
|| executeType<Int64>(columns, result)))
throw Exception("Illegal column " + columns[0].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
private:
template <typename T>
inline static void writeBitmask(T x, WriteBuffer & out)
{
using UnsignedT = make_unsigned_t<T>;
UnsignedT u_x = x;
bool first = true;
while (u_x)
{
UnsignedT y = u_x & (u_x - 1);
UnsignedT bit = u_x ^ y;
u_x = y;
if (!first)
writeChar(',', out);
first = false;
writeIntText(T(bit), out);
}
}
template <typename T>
bool executeType(ColumnsWithTypeAndName & columns, size_t result) const
{
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(columns[0].column.get()))
{
auto col_to = ColumnString::create();
const typename ColumnVector<T>::Container & vec_from = col_from->getData();
ColumnString::Chars & data_to = col_to->getChars();
ColumnString::Offsets & offsets_to = col_to->getOffsets();
size_t size = vec_from.size();
data_to.resize(size * 2);
offsets_to.resize(size);
WriteBufferFromVector<ColumnString::Chars> buf_to(data_to);
for (size_t i = 0; i < size; ++i)
{
writeBitmask<T>(vec_from[i], buf_to);
writeChar(0, buf_to);
offsets_to[i] = buf_to.count();
}
buf_to.finalize();
columns[result].column = std::move(col_to);
}
else
{
return false;
}
return true;
}
};
}
void registerFunctionBitmaskToList(FunctionFactory & factory)
{
factory.registerFunction<FunctionBitmaskToList>();
}
}

View File

@ -0,0 +1,102 @@
#pragma once
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeString.h>
#include <IO/WriteBufferFromVector.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
/** formatReadableSize - prints the transferred size in bytes in form `123.45 GiB`.
* formatReadableQuantity - prints the quantity in form of 123 million.
*/
template <typename Impl>
class FunctionFormatReadable : public IFunction
{
public:
static constexpr auto name = Impl::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionFormatReadable<Impl>>(); }
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 1; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const IDataType & type = *arguments[0];
if (!isNativeNumber(type))
throw Exception("Cannot format " + type.getName() + " because it's not a native numeric type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>();
}
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
if (!(executeType<UInt8>(block, arguments, result)
|| executeType<UInt16>(block, arguments, result)
|| executeType<UInt32>(block, arguments, result)
|| executeType<UInt64>(block, arguments, result)
|| executeType<Int8>(block, arguments, result)
|| executeType<Int16>(block, arguments, result)
|| executeType<Int32>(block, arguments, result)
|| executeType<Int64>(block, arguments, result)
|| executeType<Float32>(block, arguments, result)
|| executeType<Float64>(block, arguments, result)))
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
private:
template <typename T>
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result) const
{
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block[arguments[0]].column.get()))
{
auto col_to = ColumnString::create();
const typename ColumnVector<T>::Container & vec_from = col_from->getData();
ColumnString::Chars & data_to = col_to->getChars();
ColumnString::Offsets & offsets_to = col_to->getOffsets();
size_t size = vec_from.size();
data_to.resize(size * 2);
offsets_to.resize(size);
WriteBufferFromVector<ColumnString::Chars> buf_to(data_to);
for (size_t i = 0; i < size; ++i)
{
Impl::format(static_cast<double>(vec_from[i]), buf_to);
writeChar(0, buf_to);
offsets_to[i] = buf_to.count();
}
buf_to.finalize();
block[result].column = std::move(col_to);
return true;
}
return false;
}
};
}

View File

@ -0,0 +1,26 @@
#include <Functions/FunctionFactory.h>
#include <Functions/formatReadable.h>
namespace DB
{
namespace
{
struct Impl
{
static constexpr auto name = "formatReadableQuantity";
static void format(double value, DB::WriteBuffer & out)
{
formatReadableQuantity(value, out);
}
};
}
void registerFunctionFormatReadableQuantity(FunctionFactory & factory)
{
factory.registerFunction<FunctionFormatReadable<Impl>>();
}
}

View File

@ -0,0 +1,26 @@
#include <Functions/FunctionFactory.h>
#include <Functions/formatReadable.h>
namespace DB
{
namespace
{
struct Impl
{
static constexpr auto name = "formatReadableSize";
static void format(double value, DB::WriteBuffer & out)
{
formatReadableSizeWithBinarySuffix(value, out);
}
};
}
void registerFunctionFormatReadableSize(FunctionFactory & factory)
{
factory.registerFunction<FunctionFormatReadable<Impl>>();
}
}

View File

@ -0,0 +1,220 @@
#include <Functions/FunctionFactory.h>
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeString.h>
#include <IO/WriteBufferFromVector.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
}
namespace
{
/** Prints amount of seconds in form of:
* "1 year, 2 months, 12 days, 3 hours, 1 minute and 33 seconds".
* Maximum unit can be specified as a second argument: for example, you can specify "days",
* and it will avoid using years and months.
*
* The length of years and months (and even days in presence of time adjustments) are rough:
* year is just 365 days, month is 30.5 days, day is 86400 seconds.
*
* You may think that the choice of constants and the whole purpose of this function is very ignorant...
* And you're right. But actually it's made similar to a random Python library from the internet:
* https://github.com/jmoiron/humanize/blob/b37dc30ba61c2446eecb1a9d3e9ac8c9adf00f03/src/humanize/time.py#L462
*/
class FunctionFormatReadableTimeDelta : public IFunction
{
public:
static constexpr auto name = "formatReadableTimeDelta";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionFormatReadableTimeDelta>(); }
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.empty())
throw Exception(
"Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size())
+ ", should be at least 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (arguments.size() > 2)
throw Exception(
"Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size())
+ ", should be at most 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & type = *arguments[0];
if (!isNativeNumber(type))
throw Exception("Cannot format " + type.getName() + " as time delta", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (arguments.size() == 2)
{
const auto * maximum_unit_arg = arguments[1].get();
if (!isStringOrFixedString(maximum_unit_arg))
throw Exception("Illegal type " + maximum_unit_arg->getName() + " of argument maximum_unit of function "
+ getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
return std::make_shared<DataTypeString>();
}
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
bool useDefaultImplementationForConstants() const override { return true; }
enum Unit
{
Seconds,
Minutes,
Hours,
Days,
Months,
Years
};
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
StringRef maximum_unit_str;
if (arguments.size() == 2)
{
const ColumnPtr & maximum_unit_column = block[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);
}
Unit max_unit;
/// Default means "use all available units".
if (maximum_unit_str.size == 0 || 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.toString(), getName());
auto col_to = ColumnString::create();
ColumnString::Chars & data_to = col_to->getChars();
ColumnString::Offsets & offsets_to = col_to->getOffsets();
offsets_to.resize(input_rows_count);
WriteBufferFromVector<ColumnString::Chars> buf_to(data_to);
for (size_t i = 0; i < input_rows_count; ++i)
{
/// Virtual call is Ok (neglible comparing to the rest of calculations).
Float64 value = block[arguments[0]].column->getFloat64(i);
bool is_negative = value < 0;
if (is_negative)
{
writeChar('-', buf_to);
value = -value;
}
/// To output separators between parts: ", " and " and ".
bool has_output = false;
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(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);
}
writeChar(0, buf_to);
offsets_to[i] = buf_to.count();
}
buf_to.finalize();
block[result].column = std::move(col_to);
}
static void processUnit(
UInt64 unit_size, const char * unit_name, size_t unit_name_size,
Float64 & value, WriteBuffer & buf_to, bool & has_output)
{
if (unlikely(value + 1.0 == value))
{
/// 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);
buf_to.write(unit_name, unit_name_size);
writeChar('s', buf_to);
has_output = true;
value = 0;
return;
}
UInt64 num_units = value / unit_size;
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_size > 1 || has_output)
return;
}
/// Remaining value to print on next iteration.
value -= num_units * unit_size;
if (has_output)
{
/// Need delimiter between values. The last delimiter is " and ", all previous are comma.
if (value < 1)
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.
if (num_units != 1)
writeChar('s', buf_to);
has_output = true;
}
};
}
void registerFunctionFormatReadableTimeDelta(FunctionFactory & factory)
{
factory.registerFunction<FunctionFormatReadableTimeDelta>();
}
}

View File

@ -0,0 +1,19 @@
namespace DB
{
class FunctionFactory;
void registerFunctionBitmaskToList(FunctionFactory &);
void registerFunctionFormatReadableSize(FunctionFactory &);
void registerFunctionFormatReadableQuantity(FunctionFactory &);
void registerFunctionFormatReadableTimeDelta(FunctionFactory &);
void registerFunctionsFormatting(FunctionFactory & factory)
{
registerFunctionBitmaskToList(factory);
registerFunctionFormatReadableSize(factory);
registerFunctionFormatReadableQuantity(factory);
registerFunctionFormatReadableTimeDelta(factory);
}
}

View File

@ -112,6 +112,7 @@ SRCS(
bitBoolMaskAnd.cpp
bitBoolMaskOr.cpp
bitCount.cpp
bitmaskToList.cpp
bitNot.cpp
bitOr.cpp
bitRotateLeft.cpp
@ -163,6 +164,9 @@ SRCS(
filesystem.cpp
finalizeAggregation.cpp
formatDateTime.cpp
formatReadableQuantity.cpp
formatReadableSize.cpp
formatReadableTimeDelta.cpp
formatRow.cpp
formatString.cpp
fromUnixTimestamp64Micro.cpp
@ -177,7 +181,6 @@ SRCS(
FunctionsEmbeddedDictionaries.cpp
FunctionsExternalDictionaries.cpp
FunctionsExternalModels.cpp
FunctionsFormatting.cpp
FunctionsHashing.cpp
FunctionsJSON.cpp
FunctionsLogical.cpp
@ -335,6 +338,7 @@ SRCS(
registerFunctionsConsistentHashing.cpp
registerFunctions.cpp
registerFunctionsDateTime.cpp
registerFunctionsFormatting.cpp
registerFunctionsGeo.cpp
registerFunctionsHigherOrder.cpp
registerFunctionsIntrospection.cpp

View File

@ -0,0 +1,5 @@
<test>
<query>SELECT count() FROM numbers(1000000) WHERE NOT ignore(formatReadableSize(number))</query>
<query>SELECT count() FROM numbers(1000000) WHERE NOT ignore(formatReadableQuantity(number))</query>
<query>SELECT count() FROM numbers(1000000) WHERE NOT ignore(formatReadableTimeDelta(number))</query>
</test>

View File

@ -32,14 +32,14 @@
<substitution>
<name>table</name>
<values>
<value>zeros(100000000)</value>
<value>zeros (100000000)</value>
<value>zeros_mt(1000000000)</value>
</values>
</substitution>
<substitution>
<name>table_slow</name>
<values>
<value>numbers(1000000)</value>
<value>numbers (10000000)</value>
<value>numbers_mt(100000000)</value>
</values>
</substitution>

View File

@ -175,6 +175,7 @@ SELECT protocol(NULL);
SELECT toInt16OrZero(NULL);
SELECT formatReadableSize(NULL);
SELECT formatReadableQuantity(NULL);
SELECT formatReadableTimeDelta(NULL);
SELECT concatAssumeInjective(NULL);
SELECT toString(NULL);
SELECT MACStringToNum(NULL);

View File

@ -0,0 +1,30 @@
1 5 seconds
60 5 minutes and 30 seconds
3600 5 hours and 30 minutes
86400 5 days and 12 hours
2592000 5 months, 12 days and 12 hours
31536000 5 years, 5 months and 30 days
minutes 1 5 seconds
minutes 60 5 minutes and 30 seconds
minutes 3600 330 minutes
minutes 86400 7920 minutes
minutes 2592000 237600 minutes
minutes 31536000 2890800 minutes
hours 1 5 seconds
hours 60 5 minutes and 30 seconds
hours 3600 5 hours and 30 minutes
hours 86400 132 hours
hours 2592000 3960 hours
hours 31536000 48180 hours
days 1 5 seconds
days 60 5 minutes and 30 seconds
days 3600 5 hours and 30 minutes
days 86400 5 days and 12 hours
days 2592000 165 days
days 31536000 2007 days and 12 hours
months 1 5 seconds
months 60 5 minutes and 30 seconds
months 3600 5 hours and 30 minutes
months 86400 5 days and 12 hours
months 2592000 5 months, 12 days and 12 hours
months 31536000 65 months and 25 days

View File

@ -0,0 +1,19 @@
SELECT
arrayJoin([1, 60, 60*60, 60*60*24, 60*60*24*30, 60*60*24*365]) AS elapsed,
formatReadableTimeDelta(elapsed*5.5) AS time_delta;
SELECT
'minutes' AS maximum_unit,
arrayJoin([1, 60, 60*60, 60*60*24, 60*60*24*30, 60*60*24*365]) AS elapsed,
formatReadableTimeDelta(elapsed*5.5, maximum_unit) AS time_delta;
SELECT
'hours' AS maximum_unit,
arrayJoin([1, 60, 60*60, 60*60*24, 60*60*24*30, 60*60*24*365]) AS elapsed,
formatReadableTimeDelta(elapsed*5.5, maximum_unit) AS time_delta;
SELECT
'days' AS maximum_unit,
arrayJoin([1, 60, 60*60, 60*60*24, 60*60*24*30, 60*60*24*365]) AS elapsed,
formatReadableTimeDelta(elapsed*5.5, maximum_unit) AS time_delta;
SELECT
'months' AS maximum_unit,
arrayJoin([1, 60, 60*60, 60*60*24, 60*60*24*30, 60*60*24*365]) AS elapsed,
formatReadableTimeDelta(elapsed*5.5, maximum_unit) AS time_delta;

View File

@ -0,0 +1,20 @@
-1 year, 1 month, 1 day, 1 hour, 1 minute and 1 second
-1 year, 1 month, 1 day, 1 hour, 1 minute and 1 second
-13 months, 1 hour, 1 minute and 1 second
-396 days, 13 hours, 1 minute and 1 second
-9517 hours, 1 minute and 1 second
-571021 minutes and 1 second
-34261261 seconds
-1 year, 1 month, 1 day, 1 hour and 1 minute
-1 year, 1 month, 1 day, 1 hour and 1 second
-1 year, 1 month, 1 day, 1 minute and 1 second
-1 year, 1 month, 1 hour, 1 minute and 1 second
-1 year, 1 day, 1 hour, 1 minute and 1 second
-1 month, 1 day, 1 hour, 1 minute and 1 second
3.1709791983764585e92 years
3.7947783849423193e93 months
1.1574074074074074e95 days
2.777777777777778e96 hours
1.6666666666666666e98 minutes
1e100 seconds
36558901084 years

View File

@ -0,0 +1,24 @@
SELECT formatReadableTimeDelta(-(1 + 60 + 3600 + 86400 + 30.5 * 86400 + 365 * 86400));
SELECT formatReadableTimeDelta(-(1 + 60 + 3600 + 86400 + 30.5 * 86400 + 365 * 86400), 'years');
SELECT formatReadableTimeDelta(-(1 + 60 + 3600 + 86400 + 30.5 * 86400 + 365 * 86400), 'months');
SELECT formatReadableTimeDelta(-(1 + 60 + 3600 + 86400 + 30.5 * 86400 + 365 * 86400), 'days');
SELECT formatReadableTimeDelta(-(1 + 60 + 3600 + 86400 + 30.5 * 86400 + 365 * 86400), 'hours');
SELECT formatReadableTimeDelta(-(1 + 60 + 3600 + 86400 + 30.5 * 86400 + 365 * 86400), 'minutes');
SELECT formatReadableTimeDelta(-(1 + 60 + 3600 + 86400 + 30.5 * 86400 + 365 * 86400), 'seconds');
SELECT formatReadableTimeDelta(-(1 + 60 + 3600 + 86400 + 30.5 * 86400 + 365 * 86400), 'second'); -- { serverError 36 }
SELECT formatReadableTimeDelta(-(60 + 3600 + 86400 + 30.5 * 86400 + 365 * 86400));
SELECT formatReadableTimeDelta(-(1 + 3600 + 86400 + 30.5 * 86400 + 365 * 86400));
SELECT formatReadableTimeDelta(-(1 + 60 + 86400 + 30.5 * 86400 + 365 * 86400));
SELECT formatReadableTimeDelta(-(1 + 60 + 3600 + 30.5 * 86400 + 365 * 86400));
SELECT formatReadableTimeDelta(-(1 + 60 + 3600 + 86400 + 365 * 86400));
SELECT formatReadableTimeDelta(-(1 + 60 + 3600 + 86400 + 30.5 * 86400));
SELECT formatReadableTimeDelta(1e100);
SELECT formatReadableTimeDelta(1e100, 'months');
SELECT formatReadableTimeDelta(1e100, 'days');
SELECT formatReadableTimeDelta(1e100, 'hours');
SELECT formatReadableTimeDelta(1e100, 'minutes');
SELECT formatReadableTimeDelta(1e100, 'seconds');
SELECT formatReadableTimeDelta(0x1000000000000000);

View File

@ -0,0 +1 @@
11 1 1 8 8 7367

View File

@ -0,0 +1,14 @@
DROP TABLE IF EXISTS max_length_alias_14053;
CREATE TABLE max_length_alias_14053
(`a` Date,`b` UInt16,`c.d` Array(Date),`dcount` UInt16 ALIAS length(c.d))
ENGINE = MergeTree PARTITION BY toMonday(a) ORDER BY (a, b)
SETTINGS index_granularity = 8192;
INSERT INTO max_length_alias_14053 VALUES ('2020-10-06',7367,['2020-10-06','2020-10-06','2020-10-06','2020-10-06','2020-10-06']),('2020-10-06',7367,['2020-10-06','2020-10-06','2020-10-06']),('2020-10-06',7367,['2020-10-06','2020-10-06']),('2020-10-07',7367,['2020-10-07','2020-10-07','2020-10-07','2020-10-07','2020-10-07']),('2020-10-08',7367,['2020-10-08','2020-10-08','2020-10-08','2020-10-08']),('2020-10-11',7367,['2020-10-11','2020-10-11','2020-10-11','2020-10-11','2020-10-11','2020-10-11','2020-10-11','2020-10-11']),('2020-10-11',7367,['2020-10-11']),('2020-08-26',7367,['2020-08-26','2020-08-26']),('2020-08-28',7367,['2020-08-28','2020-08-28','2020-08-28']),('2020-08-29',7367,['2020-08-29']),('2020-09-22',7367,['2020-09-22','2020-09-22','2020-09-22','2020-09-22','2020-09-22','2020-09-22','2020-09-22']);
SELECT count(), min(length(c.d)) AS minExpr, min(dcount) AS minAlias,
max(length(c.d)) AS maxExpr, max(dcount) AS maxAlias, b
FROM max_length_alias_14053 GROUP BY b;
DROP TABLE max_length_alias_14053;

View File

@ -1,2 +1,2 @@
add_executable (query_db_generator query_db_generator.cpp)
target_link_libraries(query_db_generator PRIVATE clickhouse_parsers)
target_link_libraries(query_db_generator PRIVATE clickhouse_parsers boost::program_options)

View File

@ -1,7 +1,8 @@
#include <map>
#include <boost/algorithm/string.hpp>
#include <cstdlib>
#include <stdio.h>
#include <iostream>
#include <string>
#include <pcg_random.hpp>
#include <Core/Field.h>
@ -16,6 +17,10 @@
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <boost/algorithm/string.hpp>
#include <boost/program_options.hpp>
namespace po = boost::program_options;
using ColumnType = uint32_t;
using TableAndColumn = std::pair<std::string, std::string>;
@ -1264,8 +1269,41 @@ TableList getTablesFromSelect(std::vector<std::string> queries)
return result;
}
int main(int, char **)
int main(int argc, const char *argv[])
{
try
{
po::options_description desc("Allowed options");
desc.add_options()
("help,h", "Display greeting and allowed options.")
("input,i", po::value<std::string>(), "Input filename.")
("output,o", po::value<std::string>(), "Output filename.");
po::variables_map vm;
po::store(po::parse_command_line(argc, argv, desc), vm);
po::notify(vm);
if (vm.count("help") || vm.count("h"))
{
std::cout << "Hello! It is datasets generator for ClickHouse's queries." << std::endl;
std::cout << "Put some query as an input and it will produce queries for table creating and filling." << std::endl;
std::cout << "After that your query could be executed on this tables." << std::endl;
std::cout << desc << std::endl;
return 1;
}
if (vm.count("input"))
freopen(vm["input"].as<std::string>().c_str(), "r", stdin);
if (vm.count("output"))
freopen(vm["output"].as<std::string>().c_str(), "w", stdout);
if (vm.empty())
std::cout << "Copy your queries (with semicolons) here, press Enter and Ctrl+D." << std::endl;
}
catch (...)
{
std::cerr << "Got error while parse command line arguments: " << DB::getCurrentExceptionMessage(true) << std::endl;
throw;
}
handlers["plus"] = arithmeticFunc;
handlers["minus"] = arithmeticFunc;
handlers["like"] = likeFunc;