From 09926ebb9397287b3e334bd071f3c79cd1a6809c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Sep 2018 01:04:39 +0300 Subject: [PATCH] Every function in its own file, part 2 [#CLICKHOUSE-2] --- dbms/src/Columns/ColumnArray.h | 2 +- dbms/src/Functions/FunctionNumericPredicate.h | 88 + dbms/src/Functions/FunctionsMiscellaneous.cpp | 2039 ----------------- dbms/src/Functions/arrayJoin.cpp | 75 + dbms/src/Functions/bar.cpp | 170 ++ dbms/src/Functions/blockNumber.cpp | 59 + dbms/src/Functions/blockSize.cpp | 56 + dbms/src/Functions/currentDatabase.cpp | 53 + .../Functions/defaultValueOfArgumentType.cpp | 48 + dbms/src/Functions/dumpColumnStructure.cpp | 54 + dbms/src/Functions/finalizeAggregation.cpp | 71 + dbms/src/Functions/getSizeOfEnumType.cpp | 68 + dbms/src/Functions/hasColumnInTable.cpp | 144 ++ dbms/src/Functions/hostName.cpp | 58 + dbms/src/Functions/identity.cpp | 44 + dbms/src/Functions/ignore.cpp | 53 + dbms/src/Functions/in.cpp | 131 ++ dbms/src/Functions/indexHint.cpp | 63 + dbms/src/Functions/isFinite.cpp | 44 + dbms/src/Functions/isInfinite.cpp | 40 + dbms/src/Functions/isNaN.cpp | 27 + dbms/src/Functions/lowCardinalityIndices.cpp | 63 + dbms/src/Functions/lowCardinalityKeys.cpp | 57 + dbms/src/Functions/materialize.cpp | 51 + .../registerFunctionsMiscellaneous.cpp | 85 + dbms/src/Functions/rowNumberInAllBlocks.cpp | 66 + dbms/src/Functions/rowNumberInBlock.cpp | 59 + dbms/src/Functions/runningAccumulate.cpp | 107 + dbms/src/Functions/runningDifference.cpp | 13 + dbms/src/Functions/runningDifference.h | 194 ++ ...unningDifferenceStartingWithFirstValue.cpp | 13 + dbms/src/Functions/sleep.cpp | 13 + dbms/src/Functions/sleep.h | 99 + dbms/src/Functions/sleepEachRow.cpp | 13 + dbms/src/Functions/throwIf.cpp | 91 + dbms/src/Functions/timezone.cpp | 50 + dbms/src/Functions/toColumnTypeName.cpp | 49 + dbms/src/Functions/toLowCardinality.cpp | 58 + dbms/src/Functions/toTypeName.cpp | 53 + dbms/src/Functions/uptime.cpp | 57 + dbms/src/Functions/version.cpp | 48 + dbms/src/Functions/visibleWidth.cpp | 79 + 42 files changed, 2665 insertions(+), 2040 deletions(-) create mode 100644 dbms/src/Functions/FunctionNumericPredicate.h delete mode 100644 dbms/src/Functions/FunctionsMiscellaneous.cpp create mode 100644 dbms/src/Functions/arrayJoin.cpp create mode 100644 dbms/src/Functions/bar.cpp create mode 100644 dbms/src/Functions/blockNumber.cpp create mode 100644 dbms/src/Functions/blockSize.cpp create mode 100644 dbms/src/Functions/currentDatabase.cpp create mode 100644 dbms/src/Functions/defaultValueOfArgumentType.cpp create mode 100644 dbms/src/Functions/dumpColumnStructure.cpp create mode 100644 dbms/src/Functions/finalizeAggregation.cpp create mode 100644 dbms/src/Functions/getSizeOfEnumType.cpp create mode 100644 dbms/src/Functions/hasColumnInTable.cpp create mode 100644 dbms/src/Functions/hostName.cpp create mode 100644 dbms/src/Functions/identity.cpp create mode 100644 dbms/src/Functions/ignore.cpp create mode 100644 dbms/src/Functions/in.cpp create mode 100644 dbms/src/Functions/indexHint.cpp create mode 100644 dbms/src/Functions/isFinite.cpp create mode 100644 dbms/src/Functions/isInfinite.cpp create mode 100644 dbms/src/Functions/isNaN.cpp create mode 100644 dbms/src/Functions/lowCardinalityIndices.cpp create mode 100644 dbms/src/Functions/lowCardinalityKeys.cpp create mode 100644 dbms/src/Functions/materialize.cpp create mode 100644 dbms/src/Functions/registerFunctionsMiscellaneous.cpp create mode 100644 dbms/src/Functions/rowNumberInAllBlocks.cpp create mode 100644 dbms/src/Functions/rowNumberInBlock.cpp create mode 100644 dbms/src/Functions/runningAccumulate.cpp create mode 100644 dbms/src/Functions/runningDifference.cpp create mode 100644 dbms/src/Functions/runningDifference.h create mode 100644 dbms/src/Functions/runningDifferenceStartingWithFirstValue.cpp create mode 100644 dbms/src/Functions/sleep.cpp create mode 100644 dbms/src/Functions/sleep.h create mode 100644 dbms/src/Functions/sleepEachRow.cpp create mode 100644 dbms/src/Functions/throwIf.cpp create mode 100644 dbms/src/Functions/timezone.cpp create mode 100644 dbms/src/Functions/toColumnTypeName.cpp create mode 100644 dbms/src/Functions/toLowCardinality.cpp create mode 100644 dbms/src/Functions/toTypeName.cpp create mode 100644 dbms/src/Functions/uptime.cpp create mode 100644 dbms/src/Functions/version.cpp create mode 100644 dbms/src/Functions/visibleWidth.cpp diff --git a/dbms/src/Columns/ColumnArray.h b/dbms/src/Columns/ColumnArray.h index 642d3da2327..c2c17c17ed7 100644 --- a/dbms/src/Columns/ColumnArray.h +++ b/dbms/src/Columns/ColumnArray.h @@ -137,7 +137,7 @@ private: /** Non-constant arrays of constant values are quite rare. * Most functions can not work with them, and does not create such columns as a result. - * An exception is the function `replicate`(see FunctionsMiscellaneous.h), which has service meaning for the implementation of lambda functions. + * An exception is the function `replicate` (see FunctionsMiscellaneous.h), which has service meaning for the implementation of lambda functions. * Only for its sake is the implementation of the `replicate` method for ColumnArray(ColumnConst). */ ColumnPtr replicateConst(const Offsets & replicate_offsets) const; diff --git a/dbms/src/Functions/FunctionNumericPredicate.h b/dbms/src/Functions/FunctionNumericPredicate.h new file mode 100644 index 00000000000..ed26d3c5ee0 --- /dev/null +++ b/dbms/src/Functions/FunctionNumericPredicate.h @@ -0,0 +1,88 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +template +class FunctionNumericPredicate : public IFunction +{ +public: + static constexpr auto name = Impl::name; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isNumber(arguments.front())) + throw Exception{"Argument for function " + getName() + " must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const auto in = block.getByPosition(arguments.front()).column.get(); + + if ( !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result)) + throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + } + + template + bool execute(Block & block, const IColumn * in_untyped, const size_t result) + { + if (const auto in = checkAndGetColumn>(in_untyped)) + { + const auto size = in->size(); + + auto out = ColumnUInt8::create(size); + + const auto & in_data = in->getData(); + auto & out_data = out->getData(); + + for (const auto i : ext::range(0, size)) + out_data[i] = Impl::execute(in_data[i]); + + block.getByPosition(result).column = std::move(out); + return true; + } + + return false; + } +}; + +} diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp deleted file mode 100644 index fc95d7d6d50..00000000000 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ /dev/null @@ -1,2039 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int FUNCTION_IS_SPECIAL; - extern const int ARGUMENT_OUT_OF_BOUND; - extern const int TOO_SLOW; - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int FUNCTION_THROW_IF_VALUE_IS_NON_ZERO; -} - -/** Helper functions - * - * visibleWidth(x) - calculates the approximate width when outputting the value in a text (tab-separated) form to the console. - * - * toTypeName(x) - get the type name - * blockSize() - get the block size - * materialize(x) - materialize the constant - * ignore(...) is a function that takes any arguments, and always returns 0. - * sleep(seconds) - the specified number of seconds sleeps each block. - * - * in(x, set) - function for evaluating the IN - * notIn(x, set) - and NOT IN. - * - * arrayJoin(arr) - a special function - it can not be executed directly; - * is used only to get the result type of the corresponding expression. - * - * replicate(x, arr) - creates an array of the same size as arr, all elements of which are equal to x; - * for example: replicate(1, ['a', 'b', 'c']) = [1, 1, 1]. - * - * sleep(n) - sleeps n seconds for each block. - * - * bar(x, min, max, width) - draws a strip from the number of characters proportional to (x - min) and equal to width for x == max. - * - * version() - returns the current version of the server on the line. - * - * finalizeAggregation(agg_state) - get the result from the aggregation state. - * - * runningAccumulate(agg_state) - takes the states of the aggregate function and returns a column with values, - * are the result of the accumulation of these states for a set of block lines, from the first to the current line. - */ - - -class FunctionCurrentDatabase : public IFunction -{ - const String db_name; - -public: - static constexpr auto name = "currentDatabase"; - static FunctionPtr create(const Context & context) - { - return std::make_shared(context.getCurrentDatabase()); - } - - explicit FunctionCurrentDatabase(const String & db_name) : db_name{db_name} - { - } - - String getName() const override - { - return name; - } - size_t getNumberOfArguments() const override - { - return 0; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - bool isDeterministic() const override { return false; } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, db_name); - } -}; - - -/// Get the host name. Is is constant on single server, but is not constant in distributed queries. -class FunctionHostName : public IFunction -{ -public: - static constexpr auto name = "hostName"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - /** convertToFullColumn needed because in distributed query processing, - * each server returns its own value. - */ - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst( - input_rows_count, DNSResolver::instance().getHostName())->convertToFullColumnIfConst(); - } -}; - - -class FunctionVisibleWidth : public IFunction -{ -public: - static constexpr auto name = "visibleWidth"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - bool useDefaultImplementationForNulls() const override { return false; } - - /// Get the name of the function. - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - - /// Execute the function on the block. - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override; -}; - - -/// Returns name of IDataType instance (name of data type). -class FunctionToTypeName : public IFunction -{ -public: - static constexpr auto name = "toTypeName"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - /// Execute the function on the block. - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column - = DataTypeString().createColumnConst(input_rows_count, block.getByPosition(arguments[0]).type->getName()); - } -}; - - -/// 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 &) - { - return std::make_shared(); - } - - 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 - { - WhichDataType which(arguments[0]); - - if (which.isEnum8()) - return std::make_shared(); - else if (which.isEnum16()) - return std::make_shared(); - - 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, size_t input_rows_count) override - { - if (auto type = checkAndGetDataType(block.getByPosition(arguments[0]).type.get())) - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(type->getValues().size())); - else if (auto type = checkAndGetDataType(block.getByPosition(arguments[0]).type.get())) - block.getByPosition(result).column = DataTypeUInt16().createColumnConst(input_rows_count, 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 -{ -public: - static constexpr auto name = "toColumnTypeName"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - 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 - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column - = DataTypeString().createColumnConst(input_rows_count, block.getByPosition(arguments[0]).column->getName()); - } -}; - - - -/// Dump the structure of type and column. -class FunctionDumpColumnStructure : public IFunction -{ -public: - static constexpr auto name = "dumpColumnStructure"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - 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 - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - const auto & elem = block.getByPosition(arguments[0]); - - /// Note that the result is not a constant, because it contains block size. - - block.getByPosition(result).column - = DataTypeString().createColumnConst(input_rows_count, - elem.type->getName() + ", " + elem.column->dumpStructure())->convertToFullColumnIfConst(); - } -}; - - -/// Returns global default value for type of passed argument (example: 0 for numeric types, '' for String). -class FunctionDefaultValueOfArgumentType : public IFunction -{ -public: - static constexpr auto name = "defaultValueOfArgumentType"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - 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 - { - return arguments[0]; - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - const IDataType & type = *block.getByPosition(arguments[0]).type; - block.getByPosition(result).column = type.createColumnConst(input_rows_count, type.getDefault()); - } -}; - - -class FunctionBlockSize : public IFunction -{ -public: - static constexpr auto name = "blockSize"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - /// Get the function name. - String getName() const override - { - return name; - } - - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = ColumnUInt64::create(input_rows_count, input_rows_count); - } -}; - - -class FunctionRowNumberInBlock : public IFunction -{ -public: - static constexpr auto name = "rowNumberInBlock"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - /// Get the name of the function. - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - auto column = ColumnUInt64::create(); - auto & data = column->getData(); - data.resize(input_rows_count); - for (size_t i = 0; i < input_rows_count; ++i) - data[i] = i; - - block.getByPosition(result).column = std::move(column); - } -}; - - -/** Incremental block number among calls of this function. */ -class FunctionBlockNumber : public IFunction -{ -private: - std::atomic block_number{0}; - -public: - static constexpr auto name = "blockNumber"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - /// Get the function name. - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - size_t current_block_number = block_number++; - block.getByPosition(result).column = ColumnUInt64::create(input_rows_count, current_block_number); - } -}; - - -/** Incremental number of row within all blocks passed to this function. */ -class FunctionRowNumberInAllBlocks : public IFunction -{ -private: - std::atomic rows{0}; - -public: - static constexpr auto name = "rowNumberInAllBlocks"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - /// Get the name of the function. - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - size_t current_row_number = rows.fetch_add(input_rows_count); - - auto column = ColumnUInt64::create(); - auto & data = column->getData(); - data.resize(input_rows_count); - for (size_t i = 0; i < input_rows_count; ++i) - data[i] = current_row_number + i; - - block.getByPosition(result).column = std::move(column); - } -}; - - -enum class FunctionSleepVariant -{ - PerBlock, - PerRow -}; - -template -class FunctionSleep : public IFunction -{ -public: - static constexpr auto name = variant == FunctionSleepVariant::PerBlock ? "sleep" : "sleepEachRow"; - static FunctionPtr create(const Context &) - { - return std::make_shared>(); - } - - /// Get the name of the function. - String getName() const override - { - return name; - } - - /// Do not sleep during query analysis. - bool isSuitableForConstantFolding() const override - { - return false; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - WhichDataType which(arguments[0]); - - if (!which.isFloat() - && !which.isNativeUInt()) - throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected Float64", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const IColumn * col = block.getByPosition(arguments[0]).column.get(); - - if (!col->isColumnConst()) - throw Exception("The argument of function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN); - - Float64 seconds = applyVisitor(FieldVisitorConvertToNumber(), static_cast(*col).getField()); - - if (seconds < 0) - throw Exception("Cannot sleep negative amount of time (not implemented)", ErrorCodes::BAD_ARGUMENTS); - - size_t size = col->size(); - - /// We do not sleep if the block is empty. - if (size > 0) - { - unsigned useconds = seconds * (variant == FunctionSleepVariant::PerBlock ? 1 : size) * 1e6; - - /// When sleeping, the query cannot be cancelled. For abitily to cancel query, we limit sleep time. - if (useconds > 3000000) /// The choice is arbitary - throw Exception("The maximum sleep time is 3000000 microseconds. Requested: " + toString(useconds), ErrorCodes::TOO_SLOW); - - usleep(useconds); - } - - /// convertToFullColumn needed, because otherwise (constant expression case) function will not get called on each block. - block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(size, UInt64(0))->convertToFullColumnIfConst(); - } -}; - - -class FunctionMaterialize : public IFunction -{ -public: - static constexpr auto name = "materialize"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - /// Get the function name. - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - return arguments[0]; - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const auto & src = block.getByPosition(arguments[0]).column; - if (ColumnPtr converted = src->convertToFullColumnIfConst()) - block.getByPosition(result).column = converted; - else - block.getByPosition(result).column = src; - } -}; - -template -struct FunctionInName; -template <> -struct FunctionInName -{ - static constexpr auto name = "in"; -}; -template <> -struct FunctionInName -{ - static constexpr auto name = "globalIn"; -}; -template <> -struct FunctionInName -{ - static constexpr auto name = "notIn"; -}; -template <> -struct FunctionInName -{ - static constexpr auto name = "globalNotIn"; -}; - -template -class FunctionIn : public IFunction -{ -public: - static constexpr auto name = FunctionInName::name; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 2; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - bool useDefaultImplementationForNulls() const override - { - return false; - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - /// Second argument must be ColumnSet. - ColumnPtr column_set_ptr = block.getByPosition(arguments[1]).column; - const ColumnSet * column_set = typeid_cast(&*column_set_ptr); - if (!column_set) - throw Exception("Second argument for function '" + getName() + "' must be Set; found " + column_set_ptr->getName(), - ErrorCodes::ILLEGAL_COLUMN); - - Block block_of_key_columns; - - /// First argument may be tuple or single column. - const ColumnWithTypeAndName & left_arg = block.getByPosition(arguments[0]); - const ColumnTuple * tuple = typeid_cast(left_arg.column.get()); - const ColumnConst * const_tuple = checkAndGetColumnConst(left_arg.column.get()); - const DataTypeTuple * type_tuple = typeid_cast(left_arg.type.get()); - - ColumnPtr materialized_tuple; - if (const_tuple) - { - materialized_tuple = const_tuple->convertToFullColumn(); - tuple = typeid_cast(materialized_tuple.get()); - } - - auto set = column_set->getData(); - auto set_types = set->getDataTypes(); - if (tuple && (set_types.size() != 1 || !set_types[0]->equals(*type_tuple))) - { - const Columns & tuple_columns = tuple->getColumns(); - const DataTypes & tuple_types = type_tuple->getElements(); - size_t tuple_size = tuple_columns.size(); - for (size_t i = 0; i < tuple_size; ++i) - block_of_key_columns.insert({ tuple_columns[i], tuple_types[i], "" }); - } - else - block_of_key_columns.insert(left_arg); - - block.getByPosition(result).column = set->execute(block_of_key_columns, negative); - } -}; - - -class FunctionIgnore : public IFunction -{ -public: - static constexpr auto name = "ignore"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - bool isVariadic() const override - { - return true; - } - size_t getNumberOfArguments() const override - { - return 0; - } - - bool useDefaultImplementationForNulls() const override { return false; } - - String getName() const override - { - return name; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(0)); - } -}; - - -/** The `indexHint` function takes any number of any arguments and always returns one. - * - * This function has a special meaning (see ExpressionAnalyzer, KeyCondition) - * - the expressions inside it are not evaluated; - * - but when analyzing the index (selecting ranges for reading), this function is treated the same way, - * as if instead of using it the expression itself would be. - * - * Example: WHERE something AND indexHint(CounterID = 34) - * - do not read or calculate CounterID = 34, but select ranges in which the CounterID = 34 expression can be true. - * - * The function can be used for debugging purposes, as well as for (hidden from the user) query conversions. - */ -class FunctionIndexHint : public IFunction -{ -public: - static constexpr auto name = "indexHint"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - bool isVariadic() const override - { - return true; - } - size_t getNumberOfArguments() const override - { - return 0; - } - - bool useDefaultImplementationForNulls() const override { return false; } - - String getName() const override - { - return name; - } - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(1)); - } -}; - - -class FunctionIdentity : public IFunction -{ -public: - static constexpr auto name = "identity"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - return arguments.front(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - block.getByPosition(result).column = block.getByPosition(arguments.front()).column; - } -}; - - -class FunctionArrayJoin : public IFunction -{ -public: - static constexpr auto name = "arrayJoin"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - - /// Get the function name. - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - /** It could return many different values for single argument. */ - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - const DataTypeArray * arr = checkAndGetDataType(arguments[0].get()); - if (!arr) - throw Exception("Argument for function " + getName() + " must be Array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return arr->getNestedType(); - } - - void executeImpl(Block &, const ColumnNumbers &, size_t, size_t /*input_rows_count*/) override - { - throw Exception("Function " + getName() + " must not be executed directly.", ErrorCodes::FUNCTION_IS_SPECIAL); - } - - /// Because of function cannot be executed directly. - bool isSuitableForConstantFolding() const override - { - return false; - } -}; - - -FunctionPtr FunctionReplicate::create(const Context &) -{ - return std::make_shared(); -} - -DataTypePtr FunctionReplicate::getReturnTypeImpl(const DataTypes & arguments) const -{ - const DataTypeArray * array_type = checkAndGetDataType(arguments[1].get()); - if (!array_type) - throw Exception("Second argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(arguments[0]); -} - -void FunctionReplicate::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - ColumnPtr first_column = block.getByPosition(arguments[0]).column; - - const ColumnArray * array_column = checkAndGetColumn(block.getByPosition(arguments[1]).column.get()); - ColumnPtr temp_column; - - if (!array_column) - { - auto const_array_column = checkAndGetColumnConst(block.getByPosition(arguments[1]).column.get()); - if (!const_array_column) - throw Exception("Unexpected column for replicate", ErrorCodes::ILLEGAL_COLUMN); - temp_column = const_array_column->convertToFullColumn(); - array_column = checkAndGetColumn(temp_column.get()); - } - - block.getByPosition(result).column - = ColumnArray::create(first_column->replicate(array_column->getOffsets()), array_column->getOffsetsPtr()); -} - -/** Returns a string with nice Unicode-art bar with resolution of 1/8 part of symbol. - */ -class FunctionBar : public IFunction -{ -public: - static constexpr auto name = "bar"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - 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() != 3 && arguments.size() != 4) - throw Exception("Function " + getName() - + " requires from 3 or 4 parameters: value, min_value, max_value, [max_width_of_bar = 80]. Passed " - + toString(arguments.size()) - + ".", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - if (!isNumber(arguments[0]) || !isNumber(arguments[1]) || !isNumber(arguments[2]) - || (arguments.size() == 4 && !isNumber(arguments[3]))) - throw Exception("All arguments for function " + getName() + " must be numeric.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - Int64 min = extractConstant(block, arguments, 1, "Second"); /// The level at which the line has zero length. - Int64 max = extractConstant(block, arguments, 2, "Third"); /// The level at which the line has the maximum length. - - /// The maximum width of the bar in characters, by default. - Float64 max_width = arguments.size() == 4 ? extractConstant(block, arguments, 3, "Fourth") : 80; - - if (max_width < 1) - throw Exception("Max_width argument must be >= 1.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - if (max_width > 1000) - throw Exception("Too large max_width.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - const auto & src = *block.getByPosition(arguments[0]).column; - - auto res_column = ColumnString::create(); - - if (executeNumber(src, *res_column, min, max, max_width) || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width) - || executeNumber(src, *res_column, min, max, max_width)) - { - block.getByPosition(result).column = std::move(res_column); - } - else - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } - -private: - template - T extractConstant(Block & block, const ColumnNumbers & arguments, size_t argument_pos, const char * which_argument) const - { - const auto & column = *block.getByPosition(arguments[argument_pos]).column; - - if (!column.isColumnConst()) - throw Exception( - which_argument + String(" argument for function ") + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN); - - return applyVisitor(FieldVisitorConvertToNumber(), column[0]); - } - - template - static void fill(const PaddedPODArray & src, - ColumnString::Chars_t & dst_chars, - ColumnString::Offsets & dst_offsets, - Int64 min, - Int64 max, - Float64 max_width) - { - size_t size = src.size(); - size_t current_offset = 0; - - dst_offsets.resize(size); - dst_chars.reserve(size * (UnicodeBar::getWidthInBytes(max_width) + 1)); /// lines 0-terminated. - - for (size_t i = 0; i < size; ++i) - { - Float64 width = UnicodeBar::getWidth(src[i], min, max, max_width); - size_t next_size = current_offset + UnicodeBar::getWidthInBytes(width) + 1; - dst_chars.resize(next_size); - UnicodeBar::render(width, reinterpret_cast(&dst_chars[current_offset])); - current_offset = next_size; - dst_offsets[i] = current_offset; - } - } - - template - static void fill(T src, String & dst_chars, Int64 min, Int64 max, Float64 max_width) - { - Float64 width = UnicodeBar::getWidth(src, min, max, max_width); - dst_chars.resize(UnicodeBar::getWidthInBytes(width)); - UnicodeBar::render(width, dst_chars.data()); - } - - template - static bool executeNumber(const IColumn & src, ColumnString & dst, Int64 min, Int64 max, Float64 max_width) - { - if (const ColumnVector * col = checkAndGetColumn>(&src)) - { - fill(col->getData(), dst.getChars(), dst.getOffsets(), min, max, max_width); - return true; - } - else - return false; - } -}; - - -template -class FunctionNumericPredicate : public IFunction -{ -public: - static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isNumber(arguments.front())) - throw Exception{"Argument for function " + getName() + " must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const auto in = block.getByPosition(arguments.front()).column.get(); - - if ( !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result)) - throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; - } - - template - bool execute(Block & block, const IColumn * in_untyped, const size_t result) - { - if (const auto in = checkAndGetColumn>(in_untyped)) - { - const auto size = in->size(); - - auto out = ColumnUInt8::create(size); - - const auto & in_data = in->getData(); - auto & out_data = out->getData(); - - for (const auto i : ext::range(0, size)) - out_data[i] = Impl::execute(in_data[i]); - - block.getByPosition(result).column = std::move(out); - return true; - } - - return false; - } -}; - -struct IsFiniteImpl -{ - /// Better implementation, because isinf, isfinite, isnan are not inlined for unknown reason. - /// Assuming IEEE 754. - /// NOTE gcc 7 doesn't vectorize this loop. - - static constexpr auto name = "isFinite"; - template - static bool execute(const T t) - { - if constexpr (std::is_same_v) - return (ext::bit_cast(t) - & 0b01111111100000000000000000000000) - != 0b01111111100000000000000000000000; - else if constexpr (std::is_same_v) - return (ext::bit_cast(t) - & 0b0111111111110000000000000000000000000000000000000000000000000000) - != 0b0111111111110000000000000000000000000000000000000000000000000000; - else - { - (void)t; - return true; - } - } -}; - -struct IsInfiniteImpl -{ - static constexpr auto name = "isInfinite"; - template - static bool execute(const T t) - { - if constexpr (std::is_same_v) - return (ext::bit_cast(t) - & 0b01111111111111111111111111111111) - == 0b01111111100000000000000000000000; - else if constexpr (std::is_same_v) - return (ext::bit_cast(t) - & 0b0111111111111111111111111111111111111111111111111111111111111111) - == 0b0111111111110000000000000000000000000000000000000000000000000000; - else - { - (void)t; - return false; - } - } -}; - -struct IsNaNImpl -{ - static constexpr auto name = "isNaN"; - template - static bool execute(const T t) - { - return t != t; //-V501 - } -}; - -using FunctionIsFinite = FunctionNumericPredicate; -using FunctionIsInfinite = FunctionNumericPredicate; -using FunctionIsNaN = FunctionNumericPredicate; - - -/** Returns server version (constant). - */ -class FunctionVersion : public IFunction -{ -public: - static constexpr auto name = "version"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - static const std::string version = getVersion(); - block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, version); - } - -private: - std::string getVersion() const; -}; - - -/** Returns server uptime in seconds. - */ -class FunctionUptime : public IFunction -{ -public: - static constexpr auto name = "uptime"; - static FunctionPtr create(const Context & context) - { - return std::make_shared(context.getUptimeSeconds()); - } - - explicit FunctionUptime(time_t uptime_) : uptime(uptime_) - { - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - bool isDeterministic() const override { return false; } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = DataTypeUInt32().createColumnConst(input_rows_count, static_cast(uptime)); - } - -private: - time_t uptime; -}; - - -/** Returns the server time zone. - */ -class FunctionTimeZone : public IFunction -{ -public: - static constexpr auto name = "timezone"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - size_t getNumberOfArguments() const override - { - return 0; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - bool isDeterministic() const override { return false; } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, DateLUT::instance().getTimeZone()); - } -}; - - -/** Quite unusual function. - * Takes state of aggregate function (example runningAccumulate(uniqState(UserID))), - * and for each row of block, return result of aggregate function on merge of states of all previous rows and current row. - * - * So, result of function depends on partition of data to blocks and on order of data in block. - */ -class FunctionRunningAccumulate : public IFunction -{ -public: - static constexpr auto name = "runningAccumulate"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - bool isDeterministic() const override { return false; } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - const DataTypeAggregateFunction * type = checkAndGetDataType(arguments[0].get()); - if (!type) - throw Exception("Argument for function " + getName() + " must have type AggregateFunction - state of aggregate function.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return type->getReturnType(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const ColumnAggregateFunction * column_with_states - = typeid_cast(&*block.getByPosition(arguments.at(0)).column); - if (!column_with_states) - throw Exception("Illegal column " + block.getByPosition(arguments.at(0)).column->getName() - + " of first argument of function " - + getName(), - ErrorCodes::ILLEGAL_COLUMN); - - AggregateFunctionPtr aggregate_function_ptr = column_with_states->getAggregateFunction(); - const IAggregateFunction & agg_func = *aggregate_function_ptr; - - AlignedBuffer place(agg_func.sizeOfData(), agg_func.alignOfData()); - agg_func.create(place.data()); - SCOPE_EXIT(agg_func.destroy(place.data())); - - std::unique_ptr arena = agg_func.allocatesMemoryInArena() ? std::make_unique() : nullptr; - - auto result_column_ptr = agg_func.getReturnType()->createColumn(); - IColumn & result_column = *result_column_ptr; - result_column.reserve(column_with_states->size()); - - const auto & states = column_with_states->getData(); - for (const auto & state_to_add : states) - { - /// Will pass empty arena if agg_func does not allocate memory in arena - agg_func.merge(place.data(), state_to_add, arena.get()); - agg_func.insertResultInto(place.data(), result_column); - } - - block.getByPosition(result).column = std::move(result_column_ptr); - } -}; - -template -struct FunctionRunningDifferenceName; - -template <> -struct FunctionRunningDifferenceName -{ - static constexpr auto name = "runningDifference"; -}; - -template <> -struct FunctionRunningDifferenceName -{ - static constexpr auto name = "runningDifferenceStartingWithFirstValue"; -}; - -/** Calculate difference of consecutive values in block. - * So, result of function depends on partition of data to blocks and on order of data in block. - */ -template -class FunctionRunningDifferenceImpl : public IFunction -{ -private: - /// It is possible to track value from previous block, to calculate continuously across all blocks. Not implemented. - - template - static void process(const PaddedPODArray & src, PaddedPODArray & dst, const NullMap * null_map) - { - size_t size = src.size(); - dst.resize(size); - - if (size == 0) - return; - - /// It is possible to SIMD optimize this loop. By no need for that in practice. - - Src prev; - bool has_prev_value = false; - - for (size_t i = 0; i < size; ++i) - { - if (null_map && (*null_map)[i]) - continue; - - if (!has_prev_value) - { - dst[i] = is_first_line_zero ? 0 : src[i]; - prev = src[i]; - has_prev_value = true; - } - else - { - auto cur = src[i]; - dst[i] = static_cast(cur) - prev; - prev = cur; - } - } - } - - /// Result type is same as result of subtraction of argument types. - template - using DstFieldType = typename NumberTraits::ResultOfSubtraction::Type; - - /// Call polymorphic lambda with tag argument of concrete field type of src_type. - template - void dispatchForSourceType(const IDataType & src_type, F && f) const - { - WhichDataType which(src_type); - - if (which.isUInt8()) - f(UInt8()); - else if (which.isUInt16()) - f(UInt16()); - else if (which.isUInt32()) - f(UInt32()); - else if (which.isUInt64()) - f(UInt64()); - else if (which.isInt8()) - f(Int8()); - else if (which.isInt16()) - f(Int16()); - else if (which.isInt32()) - f(Int32()); - else if (which.isInt64()) - f(Int64()); - else if (which.isFloat32()) - f(Float32()); - else if (which.isFloat64()) - f(Float64()); - else if (which.isDate()) - f(DataTypeDate::FieldType()); - else if (which.isDateTime()) - f(DataTypeDateTime::FieldType()); - else - throw Exception("Argument for function " + getName() + " must have numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - -public: - static constexpr auto name = FunctionRunningDifferenceName::name; - - static FunctionPtr create(const Context &) - { - return std::make_shared>(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - bool isDeterministicInScopeOfQuery() const override - { - return false; - } - - bool useDefaultImplementationForNulls() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - DataTypePtr res; - dispatchForSourceType(*removeNullable(arguments[0]), [&](auto field_type_tag) - { - res = std::make_shared>>(); - }); - - if (arguments[0]->isNullable()) - res = makeNullable(res); - - return res; - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - auto & src = block.getByPosition(arguments.at(0)); - const auto & res_type = block.getByPosition(result).type; - - /// When column is constant, its difference is zero. - if (src.column->isColumnConst()) - { - block.getByPosition(result).column = res_type->createColumnConstWithDefaultValue(input_rows_count); - return; - } - - auto res_column = removeNullable(res_type)->createColumn(); - auto * src_column = src.column.get(); - ColumnPtr null_map_column = nullptr; - const NullMap * null_map = nullptr; - if (auto * nullable_column = checkAndGetColumn(src_column)) - { - src_column = &nullable_column->getNestedColumn(); - null_map_column = nullable_column->getNullMapColumnPtr(); - null_map = &nullable_column->getNullMapData(); - } - - dispatchForSourceType(*removeNullable(src.type), [&](auto field_type_tag) - { - using SrcFieldType = decltype(field_type_tag); - - process(static_cast &>(*src_column).getData(), - static_cast> &>(*res_column).getData(), null_map); - }); - - if (null_map_column) - block.getByPosition(result).column = ColumnNullable::create(std::move(res_column), null_map_column); - else - block.getByPosition(result).column = std::move(res_column); - } -}; - -using FunctionRunningDifference = FunctionRunningDifferenceImpl; -using FunctionRunningIncome = FunctionRunningDifferenceImpl; - - -/** Takes state of aggregate function. Returns result of aggregation (finalized state). - */ -class FunctionFinalizeAggregation : public IFunction -{ -public: - static constexpr auto name = "finalizeAggregation"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - const DataTypeAggregateFunction * type = checkAndGetDataType(arguments[0].get()); - if (!type) - throw Exception("Argument for function " + getName() + " must have type AggregateFunction - state of aggregate function.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return type->getReturnType(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const ColumnAggregateFunction * column_with_states - = typeid_cast(&*block.getByPosition(arguments.at(0)).column); - if (!column_with_states) - throw Exception("Illegal column " + block.getByPosition(arguments.at(0)).column->getName() - + " of first argument of function " - + getName(), - ErrorCodes::ILLEGAL_COLUMN); - - block.getByPosition(result).column = column_with_states->convertToValues(); - } -}; - - -/** Usage: - * hasColumnInTable(['hostname'[, 'username'[, 'password']],] 'database', 'table', 'column') - */ -class FunctionHasColumnInTable : public IFunction -{ -public: - static constexpr auto name = "hasColumnInTable"; - static FunctionPtr create(const Context & context) - { - return std::make_shared(context.getGlobalContext()); - } - - explicit FunctionHasColumnInTable(const Context & global_context_) : global_context(global_context_) - { - } - - bool isVariadic() const override - { - return true; - } - size_t getNumberOfArguments() const override - { - return 0; - } - - String getName() const override - { - return name; - } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; - - bool isDeterministic() const override { return false; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; - -private: - const Context & global_context; -}; - - -void FunctionVisibleWidth::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - auto & src = block.getByPosition(arguments[0]); - size_t size = input_rows_count; - - auto res_col = ColumnUInt64::create(size); - auto & res_data = static_cast(*res_col).getData(); - - /// For simplicity reasons, function is implemented by serializing into temporary buffer. - - String tmp; - FormatSettings format_settings; - for (size_t i = 0; i < size; ++i) - { - { - WriteBufferFromString out(tmp); - src.type->serializeText(*src.column, i, out, format_settings); - } - - res_data[i] = UTF8::countCodePoints(reinterpret_cast(tmp.data()), tmp.size()); - } - - block.getByPosition(result).column = std::move(res_col); -} - - -DataTypePtr FunctionHasColumnInTable::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const -{ - if (arguments.size() < 3 || arguments.size() > 6) - throw Exception{"Invalid number of arguments for function " + getName(), - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - static const std::string arg_pos_description[] = {"First", "Second", "Third", "Fourth", "Fifth", "Sixth"}; - for (size_t i = 0; i < arguments.size(); ++i) - { - const ColumnWithTypeAndName & argument = arguments[i]; - - if (!checkColumnConst(argument.column.get())) - { - throw Exception(arg_pos_description[i] + " argument for function " + getName() + " must be const String.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - } - - return std::make_shared(); -} - - -void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - auto get_string_from_block = [&](size_t column_pos) -> String - { - ColumnPtr column = block.getByPosition(column_pos).column; - const ColumnConst * const_column = checkAndGetColumnConst(column.get()); - return const_column->getValue(); - }; - - size_t arg = 0; - String host_name; - String user_name; - String password; - - if (arguments.size() > 3) - host_name = get_string_from_block(arguments[arg++]); - - if (arguments.size() > 4) - user_name = get_string_from_block(arguments[arg++]); - - if (arguments.size() > 5) - password = get_string_from_block(arguments[arg++]); - - String database_name = get_string_from_block(arguments[arg++]); - String table_name = get_string_from_block(arguments[arg++]); - String column_name = get_string_from_block(arguments[arg++]); - - bool has_column; - if (host_name.empty()) - { - const StoragePtr & table = global_context.getTable(database_name, table_name); - has_column = table->hasColumn(column_name); - } - else - { - std::vector> host_names = {{ host_name }}; - auto cluster = std::make_shared(global_context.getSettings(), host_names, !user_name.empty() ? user_name : "default", password, global_context.getTCPPort(), false); - auto remote_columns = getStructureOfRemoteTable(*cluster, database_name, table_name, global_context); - has_column = remote_columns.hasPhysical(column_name); - } - - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(has_column)); -} - - -/// Throw an exception if the argument is non zero. -class FunctionThrowIf : public IFunction -{ -public: - static constexpr auto name = "throwIf"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isNumber(arguments.front())) - throw Exception{"Argument for function " + getName() + " must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const auto in = block.getByPosition(arguments.front()).column.get(); - - if ( !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result) - && !execute(block, in, result)) - throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; - } - - template - bool execute(Block & block, const IColumn * in_untyped, const size_t result) - { - if (const auto in = checkAndGetColumn>(in_untyped)) - { - const auto & in_data = in->getData(); - if (!memoryIsZero(in_data.data(), in_data.size() * sizeof(in_data[0]))) - throw Exception("Value passed to 'throwIf' function is non zero", ErrorCodes::FUNCTION_THROW_IF_VALUE_IS_NON_ZERO); - - /// We return non constant to avoid constant folding. - block.getByPosition(result).column = ColumnUInt8::create(in_data.size(), 0); - return true; - } - - return false; - } -}; - - -class FunctionToLowCardinality: public IFunction -{ -public: - static constexpr auto name = "toLowCardinality"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 1; } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (arguments[0]->withDictionary()) - return arguments[0]; - - return std::make_shared(arguments[0]); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - auto arg_num = arguments[0]; - const auto & arg = block.getByPosition(arg_num); - auto & res = block.getByPosition(result); - - if (arg.type->withDictionary()) - res.column = arg.column; - else - { - auto column = res.type->createColumn(); - typeid_cast(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size()); - res.column = std::move(column); - } - } -}; - -class FunctionLowCardinalityIndexes: public IFunction -{ -public: - static constexpr auto name = "lowCardinalityIndexes"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 1; } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - auto * type = typeid_cast(arguments[0].get()); - if (!type) - throw Exception("First first argument of function lowCardinalityIndexes must be ColumnWithDictionary, but got" - + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - auto arg_num = arguments[0]; - const auto & arg = block.getByPosition(arg_num); - auto & res = block.getByPosition(result); - auto indexes_col = typeid_cast(arg.column.get())->getIndexesPtr(); - auto new_indexes_col = ColumnUInt64::create(indexes_col->size()); - auto & data = new_indexes_col->getData(); - for (size_t i = 0; i < data.size(); ++i) - data[i] = indexes_col->getUInt(i); - - res.column = std::move(new_indexes_col); - } -}; - -class FunctionLowCardinalityKeys: public IFunction -{ -public: - static constexpr auto name = "lowCardinalityKeys"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 1; } - - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - auto * type = typeid_cast(arguments[0].get()); - if (!type) - throw Exception("First first argument of function lowCardinalityKeys must be ColumnWithDictionary, but got" - + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return type->getDictionaryType(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - auto arg_num = arguments[0]; - const auto & arg = block.getByPosition(arg_num); - auto & res = block.getByPosition(result); - const auto * column_with_dictionary = typeid_cast(arg.column.get()); - res.column = column_with_dictionary->getDictionary().getNestedColumn()->cloneResized(arg.column->size()); - } -}; - - -std::string FunctionVersion::getVersion() const -{ - return VERSION_STRING; -} - - -void registerFunctionsMiscellaneous(FunctionFactory & factory) -{ - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); -} -} diff --git a/dbms/src/Functions/arrayJoin.cpp b/dbms/src/Functions/arrayJoin.cpp new file mode 100644 index 00000000000..7e7dba8e71a --- /dev/null +++ b/dbms/src/Functions/arrayJoin.cpp @@ -0,0 +1,75 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FUNCTION_IS_SPECIAL; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +/** arrayJoin(arr) - a special function - it can not be executed directly; + * is used only to get the result type of the corresponding expression. + */ +class FunctionArrayJoin : public IFunction +{ +public: + static constexpr auto name = "arrayJoin"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + + /// Get the function name. + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + /** It could return many different values for single argument. */ + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const DataTypeArray * arr = checkAndGetDataType(arguments[0].get()); + if (!arr) + throw Exception("Argument for function " + getName() + " must be Array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return arr->getNestedType(); + } + + void executeImpl(Block &, const ColumnNumbers &, size_t, size_t /*input_rows_count*/) override + { + throw Exception("Function " + getName() + " must not be executed directly.", ErrorCodes::FUNCTION_IS_SPECIAL); + } + + /// Because of function cannot be executed directly. + bool isSuitableForConstantFolding() const override + { + return false; + } +}; + + +void registerFunctionArrayJoin(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/bar.cpp b/dbms/src/Functions/bar.cpp new file mode 100644 index 00000000000..6cb40154ad7 --- /dev/null +++ b/dbms/src/Functions/bar.cpp @@ -0,0 +1,170 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +/** bar(x, min, max, width) - draws a strip from the number of characters proportional to (x - min) and equal to width for x == max. + * Returns a string with nice Unicode-art bar with resolution of 1/8 part of symbol. + */ +class FunctionBar : public IFunction +{ +public: + static constexpr auto name = "bar"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + 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() != 3 && arguments.size() != 4) + throw Exception("Function " + getName() + + " requires from 3 or 4 parameters: value, min_value, max_value, [max_width_of_bar = 80]. Passed " + + toString(arguments.size()) + + ".", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!isNumber(arguments[0]) || !isNumber(arguments[1]) || !isNumber(arguments[2]) + || (arguments.size() == 4 && !isNumber(arguments[3]))) + throw Exception("All arguments for function " + getName() + " must be numeric.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + Int64 min = extractConstant(block, arguments, 1, "Second"); /// The level at which the line has zero length. + Int64 max = extractConstant(block, arguments, 2, "Third"); /// The level at which the line has the maximum length. + + /// The maximum width of the bar in characters, by default. + Float64 max_width = arguments.size() == 4 ? extractConstant(block, arguments, 3, "Fourth") : 80; + + if (max_width < 1) + throw Exception("Max_width argument must be >= 1.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + if (max_width > 1000) + throw Exception("Too large max_width.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + const auto & src = *block.getByPosition(arguments[0]).column; + + auto res_column = ColumnString::create(); + + if (executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width) + || executeNumber(src, *res_column, min, max, max_width)) + { + block.getByPosition(result).column = std::move(res_column); + } + else + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + +private: + template + T extractConstant(Block & block, const ColumnNumbers & arguments, size_t argument_pos, const char * which_argument) const + { + const auto & column = *block.getByPosition(arguments[argument_pos]).column; + + if (!column.isColumnConst()) + throw Exception( + which_argument + String(" argument for function ") + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN); + + return applyVisitor(FieldVisitorConvertToNumber(), column[0]); + } + + template + static void fill(const PaddedPODArray & src, + ColumnString::Chars_t & dst_chars, + ColumnString::Offsets & dst_offsets, + Int64 min, + Int64 max, + Float64 max_width) + { + size_t size = src.size(); + size_t current_offset = 0; + + dst_offsets.resize(size); + dst_chars.reserve(size * (UnicodeBar::getWidthInBytes(max_width) + 1)); /// lines 0-terminated. + + for (size_t i = 0; i < size; ++i) + { + Float64 width = UnicodeBar::getWidth(src[i], min, max, max_width); + size_t next_size = current_offset + UnicodeBar::getWidthInBytes(width) + 1; + dst_chars.resize(next_size); + UnicodeBar::render(width, reinterpret_cast(&dst_chars[current_offset])); + current_offset = next_size; + dst_offsets[i] = current_offset; + } + } + + template + static void fill(T src, String & dst_chars, Int64 min, Int64 max, Float64 max_width) + { + Float64 width = UnicodeBar::getWidth(src, min, max, max_width); + dst_chars.resize(UnicodeBar::getWidthInBytes(width)); + UnicodeBar::render(width, dst_chars.data()); + } + + template + static bool executeNumber(const IColumn & src, ColumnString & dst, Int64 min, Int64 max, Float64 max_width) + { + if (const ColumnVector * col = checkAndGetColumn>(&src)) + { + fill(col->getData(), dst.getChars(), dst.getOffsets(), min, max, max_width); + return true; + } + else + return false; + } +}; + + +void registerFunctionBar(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/blockNumber.cpp b/dbms/src/Functions/blockNumber.cpp new file mode 100644 index 00000000000..5e1cd412443 --- /dev/null +++ b/dbms/src/Functions/blockNumber.cpp @@ -0,0 +1,59 @@ +#include +#include +#include +#include + + +namespace DB +{ + +/** Incremental block number among calls of this function. */ +class FunctionBlockNumber : public IFunction +{ +private: + std::atomic block_number{0}; + +public: + static constexpr auto name = "blockNumber"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + /// Get the function name. + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + size_t current_block_number = block_number++; + block.getByPosition(result).column = ColumnUInt64::create(input_rows_count, current_block_number); + } +}; + + +void registerFunctionBlockNumber(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/blockSize.cpp b/dbms/src/Functions/blockSize.cpp new file mode 100644 index 00000000000..4b4c18edbf9 --- /dev/null +++ b/dbms/src/Functions/blockSize.cpp @@ -0,0 +1,56 @@ +#include +#include +#include +#include + + +namespace DB +{ + +/** blockSize() - get the block size in number of rows. + */ +class FunctionBlockSize : public IFunction +{ +public: + static constexpr auto name = "blockSize"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + /// Get the function name. + String getName() const override + { + return name; + } + + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = ColumnUInt64::create(input_rows_count, input_rows_count); + } +}; + + +void registerFunctionBlockSize(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/currentDatabase.cpp b/dbms/src/Functions/currentDatabase.cpp new file mode 100644 index 00000000000..6c5137fe2ee --- /dev/null +++ b/dbms/src/Functions/currentDatabase.cpp @@ -0,0 +1,53 @@ +#include +#include +#include +#include + + +namespace DB +{ + +class FunctionCurrentDatabase : public IFunction +{ + const String db_name; + +public: + static constexpr auto name = "currentDatabase"; + static FunctionPtr create(const Context & context) + { + return std::make_shared(context.getCurrentDatabase()); + } + + explicit FunctionCurrentDatabase(const String & db_name) : db_name{db_name} + { + } + + String getName() const override + { + return name; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, db_name); + } +}; + + +void registerFunctionCurrentDatabase(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/defaultValueOfArgumentType.cpp b/dbms/src/Functions/defaultValueOfArgumentType.cpp new file mode 100644 index 00000000000..452f25b51d8 --- /dev/null +++ b/dbms/src/Functions/defaultValueOfArgumentType.cpp @@ -0,0 +1,48 @@ +#include +#include + + +namespace DB +{ + +/// Returns global default value for type of passed argument (example: 0 for numeric types, '' for String). +class FunctionDefaultValueOfArgumentType : public IFunction +{ +public: + static constexpr auto name = "defaultValueOfArgumentType"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + 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 + { + return arguments[0]; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const IDataType & type = *block.getByPosition(arguments[0]).type; + block.getByPosition(result).column = type.createColumnConst(input_rows_count, type.getDefault()); + } +}; + + +void registerFunctionDefaultValueOfArgumentType(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/dumpColumnStructure.cpp b/dbms/src/Functions/dumpColumnStructure.cpp new file mode 100644 index 00000000000..95a2c37951b --- /dev/null +++ b/dbms/src/Functions/dumpColumnStructure.cpp @@ -0,0 +1,54 @@ +#include +#include +#include + + +namespace DB +{ + +/// Dump the structure of type and column. +class FunctionDumpColumnStructure : public IFunction +{ +public: + static constexpr auto name = "dumpColumnStructure"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + 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 + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const auto & elem = block.getByPosition(arguments[0]); + + /// Note that the result is not a constant, because it contains block size. + + block.getByPosition(result).column + = DataTypeString().createColumnConst(input_rows_count, + elem.type->getName() + ", " + elem.column->dumpStructure())->convertToFullColumnIfConst(); + } +}; + + +void registerFunctionDumpColumnStructure(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/finalizeAggregation.cpp b/dbms/src/Functions/finalizeAggregation.cpp new file mode 100644 index 00000000000..3f7ba9eb4c5 --- /dev/null +++ b/dbms/src/Functions/finalizeAggregation.cpp @@ -0,0 +1,71 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/** finalizeAggregation(agg_state) - get the result from the aggregation state. + * Takes state of aggregate function. Returns result of aggregation (finalized state). + */ +class FunctionFinalizeAggregation : public IFunction +{ +public: + static constexpr auto name = "finalizeAggregation"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const DataTypeAggregateFunction * type = checkAndGetDataType(arguments[0].get()); + if (!type) + throw Exception("Argument for function " + getName() + " must have type AggregateFunction - state of aggregate function.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return type->getReturnType(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const ColumnAggregateFunction * column_with_states + = typeid_cast(&*block.getByPosition(arguments.at(0)).column); + if (!column_with_states) + throw Exception("Illegal column " + block.getByPosition(arguments.at(0)).column->getName() + + " of first argument of function " + + getName(), + ErrorCodes::ILLEGAL_COLUMN); + + block.getByPosition(result).column = column_with_states->convertToValues(); + } +}; + + +void registerFunctionFinalizeAggregation(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/getSizeOfEnumType.cpp b/dbms/src/Functions/getSizeOfEnumType.cpp new file mode 100644 index 00000000000..4aef8492243 --- /dev/null +++ b/dbms/src/Functions/getSizeOfEnumType.cpp @@ -0,0 +1,68 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/// 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 &) + { + return std::make_shared(); + } + + 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 + { + WhichDataType which(arguments[0]); + + if (which.isEnum8()) + return std::make_shared(); + else if (which.isEnum16()) + return std::make_shared(); + + 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, size_t input_rows_count) override + { + if (auto type = checkAndGetDataType(block.getByPosition(arguments[0]).type.get())) + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(type->getValues().size())); + else if (auto type = checkAndGetDataType(block.getByPosition(arguments[0]).type.get())) + block.getByPosition(result).column = DataTypeUInt16().createColumnConst(input_rows_count, UInt64(type->getValues().size())); + else + throw Exception("The argument for function " + getName() + " must be Enum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } +}; + + +void registerFunctionGetSizeOfEnumType(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/hasColumnInTable.cpp b/dbms/src/Functions/hasColumnInTable.cpp new file mode 100644 index 00000000000..5d539281bf6 --- /dev/null +++ b/dbms/src/Functions/hasColumnInTable.cpp @@ -0,0 +1,144 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/** Usage: + * hasColumnInTable(['hostname'[, 'username'[, 'password']],] 'database', 'table', 'column') + */ +class FunctionHasColumnInTable : public IFunction +{ +public: + static constexpr auto name = "hasColumnInTable"; + static FunctionPtr create(const Context & context) + { + return std::make_shared(context.getGlobalContext()); + } + + explicit FunctionHasColumnInTable(const Context & global_context_) : global_context(global_context_) + { + } + + bool isVariadic() const override + { + return true; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + String getName() const override + { + return name; + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; + + bool isDeterministic() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; + +private: + const Context & global_context; +}; + + +DataTypePtr FunctionHasColumnInTable::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const +{ + if (arguments.size() < 3 || arguments.size() > 6) + throw Exception{"Invalid number of arguments for function " + getName(), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + + static const std::string arg_pos_description[] = {"First", "Second", "Third", "Fourth", "Fifth", "Sixth"}; + for (size_t i = 0; i < arguments.size(); ++i) + { + const ColumnWithTypeAndName & argument = arguments[i]; + + if (!checkColumnConst(argument.column.get())) + { + throw Exception(arg_pos_description[i] + " argument for function " + getName() + " must be const String.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + } + + return std::make_shared(); +} + + +void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) +{ + auto get_string_from_block = [&](size_t column_pos) -> String + { + ColumnPtr column = block.getByPosition(column_pos).column; + const ColumnConst * const_column = checkAndGetColumnConst(column.get()); + return const_column->getValue(); + }; + + size_t arg = 0; + String host_name; + String user_name; + String password; + + if (arguments.size() > 3) + host_name = get_string_from_block(arguments[arg++]); + + if (arguments.size() > 4) + user_name = get_string_from_block(arguments[arg++]); + + if (arguments.size() > 5) + password = get_string_from_block(arguments[arg++]); + + String database_name = get_string_from_block(arguments[arg++]); + String table_name = get_string_from_block(arguments[arg++]); + String column_name = get_string_from_block(arguments[arg++]); + + bool has_column; + if (host_name.empty()) + { + const StoragePtr & table = global_context.getTable(database_name, table_name); + has_column = table->hasColumn(column_name); + } + else + { + std::vector> host_names = {{ host_name }}; + + auto cluster = std::make_shared( + global_context.getSettings(), + host_names, + !user_name.empty() ? user_name : "default", + password, + global_context.getTCPPort(), + false); + + auto remote_columns = getStructureOfRemoteTable(*cluster, database_name, table_name, global_context); + has_column = remote_columns.hasPhysical(column_name); + } + + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(has_column)); +} + + +void registerFunctionHasColumnInTable(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/hostName.cpp b/dbms/src/Functions/hostName.cpp new file mode 100644 index 00000000000..489305c3108 --- /dev/null +++ b/dbms/src/Functions/hostName.cpp @@ -0,0 +1,58 @@ +#include +#include +#include +#include + + +namespace DB +{ + +/// Get the host name. Is is constant on single server, but is not constant in distributed queries. +class FunctionHostName : public IFunction +{ +public: + static constexpr auto name = "hostName"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + /** convertToFullColumn needed because in distributed query processing, + * each server returns its own value. + */ + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst( + input_rows_count, DNSResolver::instance().getHostName())->convertToFullColumnIfConst(); + } +}; + + +void registerFunctionHostName(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/identity.cpp b/dbms/src/Functions/identity.cpp new file mode 100644 index 00000000000..24977091d07 --- /dev/null +++ b/dbms/src/Functions/identity.cpp @@ -0,0 +1,44 @@ +#include +#include + + +namespace DB +{ + +class FunctionIdentity : public IFunction +{ +public: + static constexpr auto name = "identity"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return arguments.front(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + block.getByPosition(result).column = block.getByPosition(arguments.front()).column; + } +}; + + +void registerFunctionIdentity(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/ignore.cpp b/dbms/src/Functions/ignore.cpp new file mode 100644 index 00000000000..09a7ac2653f --- /dev/null +++ b/dbms/src/Functions/ignore.cpp @@ -0,0 +1,53 @@ +#include +#include +#include + + +namespace DB +{ + +/** ignore(...) is a function that takes any arguments, and always returns 0. + */ +class FunctionIgnore : public IFunction +{ +public: + static constexpr auto name = "ignore"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + bool isVariadic() const override + { + return true; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + bool useDefaultImplementationForNulls() const override { return false; } + + String getName() const override + { + return name; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(0)); + } +}; + + +void registerFunctionIgnore(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/in.cpp b/dbms/src/Functions/in.cpp new file mode 100644 index 00000000000..9a984da1bac --- /dev/null +++ b/dbms/src/Functions/in.cpp @@ -0,0 +1,131 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +/** in(x, set) - function for evaluating the IN + * notIn(x, set) - and NOT IN. + */ + +template +struct FunctionInName; + +template <> +struct FunctionInName +{ + static constexpr auto name = "in"; +}; + +template <> +struct FunctionInName +{ + static constexpr auto name = "globalIn"; +}; + +template <> +struct FunctionInName +{ + static constexpr auto name = "notIn"; +}; + +template <> +struct FunctionInName +{ + static constexpr auto name = "globalNotIn"; +}; + +template +class FunctionIn : public IFunction +{ +public: + static constexpr auto name = FunctionInName::name; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 2; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + bool useDefaultImplementationForNulls() const override + { + return false; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + /// Second argument must be ColumnSet. + ColumnPtr column_set_ptr = block.getByPosition(arguments[1]).column; + const ColumnSet * column_set = typeid_cast(&*column_set_ptr); + if (!column_set) + throw Exception("Second argument for function '" + getName() + "' must be Set; found " + column_set_ptr->getName(), + ErrorCodes::ILLEGAL_COLUMN); + + Block block_of_key_columns; + + /// First argument may be tuple or single column. + const ColumnWithTypeAndName & left_arg = block.getByPosition(arguments[0]); + const ColumnTuple * tuple = typeid_cast(left_arg.column.get()); + const ColumnConst * const_tuple = checkAndGetColumnConst(left_arg.column.get()); + const DataTypeTuple * type_tuple = typeid_cast(left_arg.type.get()); + + ColumnPtr materialized_tuple; + if (const_tuple) + { + materialized_tuple = const_tuple->convertToFullColumn(); + tuple = typeid_cast(materialized_tuple.get()); + } + + auto set = column_set->getData(); + auto set_types = set->getDataTypes(); + if (tuple && (set_types.size() != 1 || !set_types[0]->equals(*type_tuple))) + { + const Columns & tuple_columns = tuple->getColumns(); + const DataTypes & tuple_types = type_tuple->getElements(); + size_t tuple_size = tuple_columns.size(); + for (size_t i = 0; i < tuple_size; ++i) + block_of_key_columns.insert({ tuple_columns[i], tuple_types[i], "" }); + } + else + block_of_key_columns.insert(left_arg); + + block.getByPosition(result).column = set->execute(block_of_key_columns, negative); + } +}; + + +void registerFunctionsIn(FunctionFactory & factory) +{ + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); +} + +} diff --git a/dbms/src/Functions/indexHint.cpp b/dbms/src/Functions/indexHint.cpp new file mode 100644 index 00000000000..2b1e2608df6 --- /dev/null +++ b/dbms/src/Functions/indexHint.cpp @@ -0,0 +1,63 @@ +#include +#include +#include + + +namespace DB +{ + + +/** The `indexHint` function takes any number of any arguments and always returns one. + * + * This function has a special meaning (see ExpressionAnalyzer, KeyCondition) + * - the expressions inside it are not evaluated; + * - but when analyzing the index (selecting ranges for reading), this function is treated the same way, + * as if instead of using it the expression itself would be. + * + * Example: WHERE something AND indexHint(CounterID = 34) + * - do not read or calculate CounterID = 34, but select ranges in which the CounterID = 34 expression can be true. + * + * The function can be used for debugging purposes, as well as for (hidden from the user) query conversions. + */ +class FunctionIndexHint : public IFunction +{ +public: + static constexpr auto name = "indexHint"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + bool isVariadic() const override + { + return true; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + bool useDefaultImplementationForNulls() const override { return false; } + + String getName() const override + { + return name; + } + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(1)); + } +}; + + +void registerFunctionIndexHint(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/isFinite.cpp b/dbms/src/Functions/isFinite.cpp new file mode 100644 index 00000000000..e898ddc8304 --- /dev/null +++ b/dbms/src/Functions/isFinite.cpp @@ -0,0 +1,44 @@ +#include +#include +#include +#include + + +namespace DB +{ + +struct IsFiniteImpl +{ + /// Better implementation, because isinf, isfinite, isnan are not inlined for unknown reason. + /// Assuming IEEE 754. + /// NOTE gcc 7 doesn't vectorize this loop. + + static constexpr auto name = "isFinite"; + template + static bool execute(const T t) + { + if constexpr (std::is_same_v) + return (ext::bit_cast(t) + & 0b01111111100000000000000000000000) + != 0b01111111100000000000000000000000; + else if constexpr (std::is_same_v) + return (ext::bit_cast(t) + & 0b0111111111110000000000000000000000000000000000000000000000000000) + != 0b0111111111110000000000000000000000000000000000000000000000000000; + else + { + (void)t; + return true; + } + } +}; + +using FunctionIsFinite = FunctionNumericPredicate; + + +void registerFunctionIsFinite(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/isInfinite.cpp b/dbms/src/Functions/isInfinite.cpp new file mode 100644 index 00000000000..17208373cb3 --- /dev/null +++ b/dbms/src/Functions/isInfinite.cpp @@ -0,0 +1,40 @@ +#include +#include +#include +#include + + +namespace DB +{ + +struct IsInfiniteImpl +{ + static constexpr auto name = "isInfinite"; + template + static bool execute(const T t) + { + if constexpr (std::is_same_v) + return (ext::bit_cast(t) + & 0b01111111111111111111111111111111) + == 0b01111111100000000000000000000000; + else if constexpr (std::is_same_v) + return (ext::bit_cast(t) + & 0b0111111111111111111111111111111111111111111111111111111111111111) + == 0b0111111111110000000000000000000000000000000000000000000000000000; + else + { + (void)t; + return false; + } + } +}; + +using FunctionIsInfinite = FunctionNumericPredicate; + + +void registerFunctionIsInfinite(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/isNaN.cpp b/dbms/src/Functions/isNaN.cpp new file mode 100644 index 00000000000..5146f7cfc0d --- /dev/null +++ b/dbms/src/Functions/isNaN.cpp @@ -0,0 +1,27 @@ +#include +#include + + +namespace DB +{ + +struct IsNaNImpl +{ + static constexpr auto name = "isNaN"; + template + static bool execute(const T t) + { + /// Suppression for PVS-Studio. + return t != t; //-V501 + } +}; + +using FunctionIsNaN = FunctionNumericPredicate; + + +void registerFunctionIsNaN(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/lowCardinalityIndices.cpp b/dbms/src/Functions/lowCardinalityIndices.cpp new file mode 100644 index 00000000000..1a0c05d738c --- /dev/null +++ b/dbms/src/Functions/lowCardinalityIndices.cpp @@ -0,0 +1,63 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +class FunctionLowCardinalityIndices: public IFunction +{ +public: + static constexpr auto name = "lowCardinalityIndices"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + auto * type = typeid_cast(arguments[0].get()); + if (!type) + throw Exception("First first argument of function lowCardinalityIndexes must be ColumnWithDictionary, but got" + + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + auto arg_num = arguments[0]; + const auto & arg = block.getByPosition(arg_num); + auto & res = block.getByPosition(result); + auto indexes_col = typeid_cast(arg.column.get())->getIndexesPtr(); + auto new_indexes_col = ColumnUInt64::create(indexes_col->size()); + auto & data = new_indexes_col->getData(); + for (size_t i = 0; i < data.size(); ++i) + data[i] = indexes_col->getUInt(i); + + res.column = std::move(new_indexes_col); + } +}; + + +void registerFunctionLowCardinalityIndices(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/lowCardinalityKeys.cpp b/dbms/src/Functions/lowCardinalityKeys.cpp new file mode 100644 index 00000000000..e4654a35a68 --- /dev/null +++ b/dbms/src/Functions/lowCardinalityKeys.cpp @@ -0,0 +1,57 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +class FunctionLowCardinalityKeys: public IFunction +{ +public: + static constexpr auto name = "lowCardinalityKeys"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + auto * type = typeid_cast(arguments[0].get()); + if (!type) + throw Exception("First first argument of function lowCardinalityKeys must be ColumnWithDictionary, but got" + + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return type->getDictionaryType(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + auto arg_num = arguments[0]; + const auto & arg = block.getByPosition(arg_num); + auto & res = block.getByPosition(result); + const auto * column_with_dictionary = typeid_cast(arg.column.get()); + res.column = column_with_dictionary->getDictionary().getNestedColumn()->cloneResized(arg.column->size()); + } +}; + + +void registerFunctionLowCardinalityKeys(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/materialize.cpp b/dbms/src/Functions/materialize.cpp new file mode 100644 index 00000000000..4d3a1a57a77 --- /dev/null +++ b/dbms/src/Functions/materialize.cpp @@ -0,0 +1,51 @@ +#include +#include + + +namespace DB +{ + +/** materialize(x) - materialize the constant + */ +class FunctionMaterialize : public IFunction +{ +public: + static constexpr auto name = "materialize"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + /// Get the function name. + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return arguments[0]; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const auto & src = block.getByPosition(arguments[0]).column; + if (ColumnPtr converted = src->convertToFullColumnIfConst()) + block.getByPosition(result).column = converted; + else + block.getByPosition(result).column = src; + } +}; + + +void registerFunctionMaterialize(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp new file mode 100644 index 00000000000..69f1d01a45f --- /dev/null +++ b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp @@ -0,0 +1,85 @@ +#include + +namespace DB +{ + +void registerFunctionCurrentDatabase(FunctionFactory &); +void registerFunctionHostName(FunctionFactory &); +void registerFunctionVisibleWidth(FunctionFactory &); +void registerFunctionToTypeName(FunctionFactory &); +void registerFunctionGetSizeOfEnumType(FunctionFactory &); +void registerFunctionToColumnTypeName(FunctionFactory &); +void registerFunctionDumpColumnStructure(FunctionFactory &); +void registerFunctionDefaultValueOfArgumentType(FunctionFactory &); +void registerFunctionBlockSize(FunctionFactory &); +void registerFunctionBlockNumber(FunctionFactory &); +void registerFunctionRowNumberInBlock(FunctionFactory &); +void registerFunctionRowNumberInAllBlocks(FunctionFactory &); +void registerFunctionSleep(FunctionFactory &); +void registerFunctionSleepEachRow(FunctionFactory &); +void registerFunctionMaterialize(FunctionFactory &); +void registerFunctionIgnore(FunctionFactory &); +void registerFunctionIndexHint(FunctionFactory &); +void registerFunctionIdentity(FunctionFactory &); +void registerFunctionArrayJoin(FunctionFactory &); +void registerFunctionReplicate(FunctionFactory &); +void registerFunctionBar(FunctionFactory &); +void registerFunctionHasColumnInTable(FunctionFactory &); +void registerFunctionIsFinite(FunctionFactory &); +void registerFunctionIsInfinite(FunctionFactory &); +void registerFunctionIsNaN(FunctionFactory &); +void registerFunctionThrowIf(FunctionFactory &); +void registerFunctionVersion(FunctionFactory &); +void registerFunctionUptime(FunctionFactory &); +void registerFunctionTimeZone(FunctionFactory &); +void registerFunctionRunningAccumulate(FunctionFactory &); +void registerFunctionRunningDifference(FunctionFactory &); +void registerFunctionRunningIncome(FunctionFactory &); +void registerFunctionFinalizeAggregation(FunctionFactory &); +void registerFunctionToLowCardinality(FunctionFactory &); +void registerFunctionLowCardinalityIndexes(FunctionFactory &); +void registerFunctionLowCardinalityKeys(FunctionFactory &); +void registerFunctionsIn(FunctionFactory &); + +void registerFunctionsMiscellaneous(FunctionFactory & factory) +{ + registerFunctionCurrentDatabase(factory); + registerFunctionHostName(factory); + registerFunctionVisibleWidth(factory); + registerFunctionToTypeName(factory); + registerFunctionGetSizeOfEnumType(factory); + registerFunctionToColumnTypeName(factory); + registerFunctionDumpColumnStructure(factory); + registerFunctionDefaultValueOfArgumentType(factory); + registerFunctionBlockSize(factory); + registerFunctionBlockNumber(factory); + registerFunctionRowNumberInBlock(factory); + registerFunctionRowNumberInAllBlocks(factory); + registerFunctionSleep(factory); + registerFunctionSleepEachRow(factory); + registerFunctionMaterialize(factory); + registerFunctionIgnore(factory); + registerFunctionIndexHint(factory); + registerFunctionIdentity(factory); + registerFunctionArrayJoin(factory); + registerFunctionReplicate(factory); + registerFunctionBar(factory); + registerFunctionHasColumnInTable(factory); + registerFunctionIsFinite(factory); + registerFunctionIsInfinite(factory); + registerFunctionIsNaN(factory); + registerFunctionThrowIf(factory); + registerFunctionVersion(factory); + registerFunctionUptime(factory); + registerFunctionTimeZone(factory); + registerFunctionRunningAccumulate(factory); + registerFunctionRunningDifference(factory); + registerFunctionRunningIncome(factory); + registerFunctionFinalizeAggregation(factory); + registerFunctionToLowCardinality(factory); + registerFunctionLowCardinalityIndexes(factory); + registerFunctionLowCardinalityKeys(factory); + registerFunctionsIn(factory); +} + +} diff --git a/dbms/src/Functions/rowNumberInAllBlocks.cpp b/dbms/src/Functions/rowNumberInAllBlocks.cpp new file mode 100644 index 00000000000..aa6e7d228bd --- /dev/null +++ b/dbms/src/Functions/rowNumberInAllBlocks.cpp @@ -0,0 +1,66 @@ +#include +#include +#include +#include + + +namespace DB +{ + +/** Incremental number of row within all blocks passed to this function. */ +class FunctionRowNumberInAllBlocks : public IFunction +{ +private: + std::atomic rows{0}; + +public: + static constexpr auto name = "rowNumberInAllBlocks"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + /// Get the name of the function. + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + size_t current_row_number = rows.fetch_add(input_rows_count); + + auto column = ColumnUInt64::create(); + auto & data = column->getData(); + data.resize(input_rows_count); + for (size_t i = 0; i < input_rows_count; ++i) + data[i] = current_row_number + i; + + block.getByPosition(result).column = std::move(column); + } +}; + + +void registerFunctionRowNumberInAllBlocks(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/rowNumberInBlock.cpp b/dbms/src/Functions/rowNumberInBlock.cpp new file mode 100644 index 00000000000..05ae8add35c --- /dev/null +++ b/dbms/src/Functions/rowNumberInBlock.cpp @@ -0,0 +1,59 @@ +#include +#include +#include +#include + + +namespace DB +{ + +class FunctionRowNumberInBlock : public IFunction +{ +public: + static constexpr auto name = "rowNumberInBlock"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + /// Get the name of the function. + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + auto column = ColumnUInt64::create(); + auto & data = column->getData(); + data.resize(input_rows_count); + for (size_t i = 0; i < input_rows_count; ++i) + data[i] = i; + + block.getByPosition(result).column = std::move(column); + } +}; + +void registerFunctionRowNumberInBlock(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/runningAccumulate.cpp b/dbms/src/Functions/runningAccumulate.cpp new file mode 100644 index 00000000000..0434c90120d --- /dev/null +++ b/dbms/src/Functions/runningAccumulate.cpp @@ -0,0 +1,107 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/** runningAccumulate(agg_state) - takes the states of the aggregate function and returns a column with values, + * are the result of the accumulation of these states for a set of block lines, from the first to the current line. + * + * Quite unusual function. + * Takes state of aggregate function (example runningAccumulate(uniqState(UserID))), + * and for each row of block, return result of aggregate function on merge of states of all previous rows and current row. + * + * So, result of function depends on partition of data to blocks and on order of data in block. + */ +class FunctionRunningAccumulate : public IFunction +{ +public: + static constexpr auto name = "runningAccumulate"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + bool isDeterministic() const override { return false; } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const DataTypeAggregateFunction * type = checkAndGetDataType(arguments[0].get()); + if (!type) + throw Exception("Argument for function " + getName() + " must have type AggregateFunction - state of aggregate function.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return type->getReturnType(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const ColumnAggregateFunction * column_with_states + = typeid_cast(&*block.getByPosition(arguments.at(0)).column); + if (!column_with_states) + throw Exception("Illegal column " + block.getByPosition(arguments.at(0)).column->getName() + + " of first argument of function " + + getName(), + ErrorCodes::ILLEGAL_COLUMN); + + AggregateFunctionPtr aggregate_function_ptr = column_with_states->getAggregateFunction(); + const IAggregateFunction & agg_func = *aggregate_function_ptr; + + AlignedBuffer place(agg_func.sizeOfData(), agg_func.alignOfData()); + agg_func.create(place.data()); + SCOPE_EXIT(agg_func.destroy(place.data())); + + std::unique_ptr arena = agg_func.allocatesMemoryInArena() ? std::make_unique() : nullptr; + + auto result_column_ptr = agg_func.getReturnType()->createColumn(); + IColumn & result_column = *result_column_ptr; + result_column.reserve(column_with_states->size()); + + const auto & states = column_with_states->getData(); + for (const auto & state_to_add : states) + { + /// Will pass empty arena if agg_func does not allocate memory in arena + agg_func.merge(place.data(), state_to_add, arena.get()); + agg_func.insertResultInto(place.data(), result_column); + } + + block.getByPosition(result).column = std::move(result_column_ptr); + } +}; + + +void registerFunctionRunningAccumulate(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/runningDifference.cpp b/dbms/src/Functions/runningDifference.cpp new file mode 100644 index 00000000000..07acaf6522b --- /dev/null +++ b/dbms/src/Functions/runningDifference.cpp @@ -0,0 +1,13 @@ +#include +#include + + +namespace DB +{ + +void registerFunctionRunningDifference(FunctionFactory & factory) +{ + factory.registerFunction>(); +} + +} diff --git a/dbms/src/Functions/runningDifference.h b/dbms/src/Functions/runningDifference.h new file mode 100644 index 00000000000..4059d6f3ada --- /dev/null +++ b/dbms/src/Functions/runningDifference.h @@ -0,0 +1,194 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +template +struct FunctionRunningDifferenceName; + +template <> +struct FunctionRunningDifferenceName +{ + static constexpr auto name = "runningDifference"; +}; + +template <> +struct FunctionRunningDifferenceName +{ + static constexpr auto name = "runningDifferenceStartingWithFirstValue"; +}; + +/** Calculate difference of consecutive values in block. + * So, result of function depends on partition of data to blocks and on order of data in block. + */ +template +class FunctionRunningDifferenceImpl : public IFunction +{ +private: + /// It is possible to track value from previous block, to calculate continuously across all blocks. Not implemented. + + template + static void process(const PaddedPODArray & src, PaddedPODArray & dst, const NullMap * null_map) + { + size_t size = src.size(); + dst.resize(size); + + if (size == 0) + return; + + /// It is possible to SIMD optimize this loop. By no need for that in practice. + + Src prev; + bool has_prev_value = false; + + for (size_t i = 0; i < size; ++i) + { + if (null_map && (*null_map)[i]) + continue; + + if (!has_prev_value) + { + dst[i] = is_first_line_zero ? 0 : src[i]; + prev = src[i]; + has_prev_value = true; + } + else + { + auto cur = src[i]; + dst[i] = static_cast(cur) - prev; + prev = cur; + } + } + } + + /// Result type is same as result of subtraction of argument types. + template + using DstFieldType = typename NumberTraits::ResultOfSubtraction::Type; + + /// Call polymorphic lambda with tag argument of concrete field type of src_type. + template + void dispatchForSourceType(const IDataType & src_type, F && f) const + { + WhichDataType which(src_type); + + if (which.isUInt8()) + f(UInt8()); + else if (which.isUInt16()) + f(UInt16()); + else if (which.isUInt32()) + f(UInt32()); + else if (which.isUInt64()) + f(UInt64()); + else if (which.isInt8()) + f(Int8()); + else if (which.isInt16()) + f(Int16()); + else if (which.isInt32()) + f(Int32()); + else if (which.isInt64()) + f(Int64()); + else if (which.isFloat32()) + f(Float32()); + else if (which.isFloat64()) + f(Float64()); + else if (which.isDate()) + f(DataTypeDate::FieldType()); + else if (which.isDateTime()) + f(DataTypeDateTime::FieldType()); + else + throw Exception("Argument for function " + getName() + " must have numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + +public: + static constexpr auto name = FunctionRunningDifferenceName::name; + + static FunctionPtr create(const Context &) + { + return std::make_shared>(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + bool isDeterministicInScopeOfQuery() const override + { + return false; + } + + bool useDefaultImplementationForNulls() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + DataTypePtr res; + dispatchForSourceType(*removeNullable(arguments[0]), [&](auto field_type_tag) + { + res = std::make_shared>>(); + }); + + if (arguments[0]->isNullable()) + res = makeNullable(res); + + return res; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + auto & src = block.getByPosition(arguments.at(0)); + const auto & res_type = block.getByPosition(result).type; + + /// When column is constant, its difference is zero. + if (src.column->isColumnConst()) + { + block.getByPosition(result).column = res_type->createColumnConstWithDefaultValue(input_rows_count); + return; + } + + auto res_column = removeNullable(res_type)->createColumn(); + auto * src_column = src.column.get(); + ColumnPtr null_map_column = nullptr; + const NullMap * null_map = nullptr; + if (auto * nullable_column = checkAndGetColumn(src_column)) + { + src_column = &nullable_column->getNestedColumn(); + null_map_column = nullable_column->getNullMapColumnPtr(); + null_map = &nullable_column->getNullMapData(); + } + + dispatchForSourceType(*removeNullable(src.type), [&](auto field_type_tag) + { + using SrcFieldType = decltype(field_type_tag); + + process(static_cast &>(*src_column).getData(), + static_cast> &>(*res_column).getData(), null_map); + }); + + if (null_map_column) + block.getByPosition(result).column = ColumnNullable::create(std::move(res_column), null_map_column); + else + block.getByPosition(result).column = std::move(res_column); + } +}; + +} diff --git a/dbms/src/Functions/runningDifferenceStartingWithFirstValue.cpp b/dbms/src/Functions/runningDifferenceStartingWithFirstValue.cpp new file mode 100644 index 00000000000..4419e413da7 --- /dev/null +++ b/dbms/src/Functions/runningDifferenceStartingWithFirstValue.cpp @@ -0,0 +1,13 @@ +#include +#include + + +namespace DB +{ + +void registerFunctionRunningDifferenceStartingWithFirstValue(FunctionFactory & factory) +{ + factory.registerFunction>(); +} + +} diff --git a/dbms/src/Functions/sleep.cpp b/dbms/src/Functions/sleep.cpp new file mode 100644 index 00000000000..ad1c6680363 --- /dev/null +++ b/dbms/src/Functions/sleep.cpp @@ -0,0 +1,13 @@ +#include +#include + + +namespace DB +{ + +void registerFunctionSleep(FunctionFactory & factory) +{ + factory.registerFunction>(); +} + +} diff --git a/dbms/src/Functions/sleep.h b/dbms/src/Functions/sleep.h new file mode 100644 index 00000000000..1dfdbf3b5f5 --- /dev/null +++ b/dbms/src/Functions/sleep.h @@ -0,0 +1,99 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TOO_SLOW; + extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; +} + +/** sleep(seconds) - the specified number of seconds sleeps each block. + */ + +enum class FunctionSleepVariant +{ + PerBlock, + PerRow +}; + +template +class FunctionSleep : public IFunction +{ +public: + static constexpr auto name = variant == FunctionSleepVariant::PerBlock ? "sleep" : "sleepEachRow"; + static FunctionPtr create(const Context &) + { + return std::make_shared>(); + } + + /// Get the name of the function. + String getName() const override + { + return name; + } + + /// Do not sleep during query analysis. + bool isSuitableForConstantFolding() const override + { + return false; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + WhichDataType which(arguments[0]); + + if (!which.isFloat() + && !which.isNativeUInt()) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected Float64", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const IColumn * col = block.getByPosition(arguments[0]).column.get(); + + if (!col->isColumnConst()) + throw Exception("The argument of function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN); + + Float64 seconds = applyVisitor(FieldVisitorConvertToNumber(), static_cast(*col).getField()); + + if (seconds < 0) + throw Exception("Cannot sleep negative amount of time (not implemented)", ErrorCodes::BAD_ARGUMENTS); + + size_t size = col->size(); + + /// We do not sleep if the block is empty. + if (size > 0) + { + unsigned useconds = seconds * (variant == FunctionSleepVariant::PerBlock ? 1 : size) * 1e6; + + /// When sleeping, the query cannot be cancelled. For abitily to cancel query, we limit sleep time. + if (useconds > 3000000) /// The choice is arbitary + throw Exception("The maximum sleep time is 3000000 microseconds. Requested: " + toString(useconds), ErrorCodes::TOO_SLOW); + + ::usleep(useconds); + } + + /// convertToFullColumn needed, because otherwise (constant expression case) function will not get called on each block. + block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(size, UInt64(0))->convertToFullColumnIfConst(); + } +}; + +} diff --git a/dbms/src/Functions/sleepEachRow.cpp b/dbms/src/Functions/sleepEachRow.cpp new file mode 100644 index 00000000000..c1c983e850e --- /dev/null +++ b/dbms/src/Functions/sleepEachRow.cpp @@ -0,0 +1,13 @@ +#include +#include + + +namespace DB +{ + +void registerFunctionSleepEachRow(FunctionFactory & factory) +{ + factory.registerFunction>(); +} + +} diff --git a/dbms/src/Functions/throwIf.cpp b/dbms/src/Functions/throwIf.cpp new file mode 100644 index 00000000000..a6d7504414b --- /dev/null +++ b/dbms/src/Functions/throwIf.cpp @@ -0,0 +1,91 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int FUNCTION_THROW_IF_VALUE_IS_NON_ZERO; +} + + +/// Throw an exception if the argument is non zero. +class FunctionThrowIf : public IFunction +{ +public: + static constexpr auto name = "throwIf"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isNumber(arguments.front())) + throw Exception{"Argument for function " + getName() + " must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const auto in = block.getByPosition(arguments.front()).column.get(); + + if ( !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result) + && !execute(block, in, result)) + throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + } + + template + bool execute(Block & block, const IColumn * in_untyped, const size_t result) + { + if (const auto in = checkAndGetColumn>(in_untyped)) + { + const auto & in_data = in->getData(); + if (!memoryIsZero(in_data.data(), in_data.size() * sizeof(in_data[0]))) + throw Exception("Value passed to 'throwIf' function is non zero", ErrorCodes::FUNCTION_THROW_IF_VALUE_IS_NON_ZERO); + + /// We return non constant to avoid constant folding. + block.getByPosition(result).column = ColumnUInt8::create(in_data.size(), 0); + return true; + } + + return false; + } +}; + + +void registerFunctionThrowIf(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/timezone.cpp b/dbms/src/Functions/timezone.cpp new file mode 100644 index 00000000000..3b3a71c1545 --- /dev/null +++ b/dbms/src/Functions/timezone.cpp @@ -0,0 +1,50 @@ +#include +#include +#include +#include + + +namespace DB +{ + + +/** Returns the server time zone. + */ +class FunctionTimeZone : public IFunction +{ +public: + static constexpr auto name = "timezone"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, DateLUT::instance().getTimeZone()); + } +}; + + +void registerFunctionTimeZone(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/toColumnTypeName.cpp b/dbms/src/Functions/toColumnTypeName.cpp new file mode 100644 index 00000000000..35d332d082f --- /dev/null +++ b/dbms/src/Functions/toColumnTypeName.cpp @@ -0,0 +1,49 @@ +#include +#include +#include + + +namespace DB +{ + +/// Returns name of IColumn instance. +class FunctionToColumnTypeName : public IFunction +{ +public: + static constexpr auto name = "toColumnTypeName"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + 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 + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column + = DataTypeString().createColumnConst(input_rows_count, block.getByPosition(arguments[0]).column->getName()); + } +}; + + +void registerFunctionToColumnTypeName(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/toLowCardinality.cpp b/dbms/src/Functions/toLowCardinality.cpp new file mode 100644 index 00000000000..046eb717114 --- /dev/null +++ b/dbms/src/Functions/toLowCardinality.cpp @@ -0,0 +1,58 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +class FunctionToLowCardinality: public IFunction +{ +public: + static constexpr auto name = "toLowCardinality"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments[0]->withDictionary()) + return arguments[0]; + + return std::make_shared(arguments[0]); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + auto arg_num = arguments[0]; + const auto & arg = block.getByPosition(arg_num); + auto & res = block.getByPosition(result); + + if (arg.type->withDictionary()) + res.column = arg.column; + else + { + auto column = res.type->createColumn(); + typeid_cast(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size()); + res.column = std::move(column); + } + } +}; + + +void registerFunctionToLowCardinality(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/toTypeName.cpp b/dbms/src/Functions/toTypeName.cpp new file mode 100644 index 00000000000..560fdaf8dd4 --- /dev/null +++ b/dbms/src/Functions/toTypeName.cpp @@ -0,0 +1,53 @@ +#include +#include +#include + + +namespace DB +{ + +/** toTypeName(x) - get the type name + * Returns name of IDataType instance (name of data type). + */ +class FunctionToTypeName : public IFunction +{ +public: + static constexpr auto name = "toTypeName"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + /// Execute the function on the block. + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column + = DataTypeString().createColumnConst(input_rows_count, block.getByPosition(arguments[0]).type->getName()); + } +}; + + +void registerFunctionToTypeName(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/uptime.cpp b/dbms/src/Functions/uptime.cpp new file mode 100644 index 00000000000..1f54223eebb --- /dev/null +++ b/dbms/src/Functions/uptime.cpp @@ -0,0 +1,57 @@ +#include +#include +#include +#include + + +namespace DB +{ + +/** Returns server uptime in seconds. + */ +class FunctionUptime : public IFunction +{ +public: + static constexpr auto name = "uptime"; + static FunctionPtr create(const Context & context) + { + return std::make_shared(context.getUptimeSeconds()); + } + + explicit FunctionUptime(time_t uptime_) : uptime(uptime_) + { + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeUInt32().createColumnConst(input_rows_count, static_cast(uptime)); + } + +private: + time_t uptime; +}; + + +void registerFunctionUptime(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/version.cpp b/dbms/src/Functions/version.cpp new file mode 100644 index 00000000000..3467788a96c --- /dev/null +++ b/dbms/src/Functions/version.cpp @@ -0,0 +1,48 @@ +#include +#include +#include +#include + + +namespace DB +{ + +/** version() - returns the current version as a string. + */ +class FunctionVersion : public IFunction +{ +public: + static constexpr auto name = "version"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, String(VERSION_STRING)); + } +}; + + +void registerFunctionVersion(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/visibleWidth.cpp b/dbms/src/Functions/visibleWidth.cpp new file mode 100644 index 00000000000..2380bf9a993 --- /dev/null +++ b/dbms/src/Functions/visibleWidth.cpp @@ -0,0 +1,79 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** visibleWidth(x) - calculates the approximate width when outputting the value in a text form to the console. + * In fact it calculate the number of Unicode code points. + * It does not support zero width and full width characters, combining characters, etc. + */ +class FunctionVisibleWidth : public IFunction +{ +public: + static constexpr auto name = "visibleWidth"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + bool useDefaultImplementationForNulls() const override { return false; } + + /// Get the name of the function. + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + /// Execute the function on the block. + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + auto & src = block.getByPosition(arguments[0]); + size_t size = input_rows_count; + + auto res_col = ColumnUInt64::create(size); + auto & res_data = static_cast(*res_col).getData(); + + /// For simplicity reasons, function is implemented by serializing into temporary buffer. + + String tmp; + FormatSettings format_settings; + for (size_t i = 0; i < size; ++i) + { + { + WriteBufferFromString out(tmp); + src.type->serializeText(*src.column, i, out, format_settings); + } + + res_data[i] = UTF8::countCodePoints(reinterpret_cast(tmp.data()), tmp.size()); + } + + block.getByPosition(result).column = std::move(res_col); + } +}; + + +void registerFunctionVisibleWidth(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +}