提交 782fe332 编写于 作者: A Alexey Milovidov

Miscellaneous [#CLICKHOUSE-31].

上级 43fedfec
......@@ -15,6 +15,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
PrettyBlockOutputStream::PrettyBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_, const Context & context_)
: ostr(ostr_), max_rows(max_rows_), no_escapes(no_escapes_), context(context_)
{
......
......@@ -36,7 +36,7 @@ public:
FunctionFactory();
FunctionPtr get(const std::string & name, const Context & context) const; /// Throws an exception if not found.
FunctionPtr tryGet(const std::string & name, const Context & context) const; /// Returns nullptr if not found.
FunctionPtr tryGet(const std::string & name, const Context & context) const; /// Returns nullptr if not found.
/// No locking, you must register all functions before usage of get, tryGet.
template <typename Function> void registerFunction()
......@@ -44,7 +44,7 @@ public:
static_assert(std::is_same<decltype(&Function::create), Creator>::value, "Function::create has incorrect type");
if (!functions.emplace(std::string(Function::name), &Function::create).second)
throw Exception("FunctionFactory: the function name is not unique",
throw Exception("FunctionFactory: the function name '" + std::string(Function::name) + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
}
};
......
......@@ -17,6 +17,7 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_DIVISION;
extern const int ILLEGAL_COLUMN;
}
......
......@@ -22,6 +22,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
extern const int CANNOT_CREATE_CHARSET_CONVERTER;
extern const int CANNOT_CONVERT_CHARSET;
extern const int ILLEGAL_COLUMN;
}
......
......@@ -24,6 +24,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION;
}
/** Encoding functions:
*
* IPv4NumToString (num) - See below.
......@@ -40,10 +46,10 @@ namespace DB
*/
const auto ipv4_bytes_length = 4;
const auto ipv6_bytes_length = 16;
const auto uuid_bytes_length = 16;
const auto uuid_text_length = 36;
constexpr auto ipv4_bytes_length = 4;
constexpr auto ipv6_bytes_length = 16;
constexpr auto uuid_bytes_length = 16;
constexpr auto uuid_text_length = 36;
class IPv6Format
{
......
......@@ -23,12 +23,17 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
/** Functions for working with date and time.
*
* toYear, toMonth, toDayOfMonth, toDayOfWeek, toHour, toMinute, toSecond,
* toMonday, toStartOfMonth, toStartOfYear, toStartOfMinute, toStartOfFiveMinute
* toStartOfHour, toTime,
* now
* now, today, yesterday
* TODO: makeDate, makeDateTime
*
* (toDate - located in FunctionConversion.h file)
......
......@@ -31,6 +31,7 @@ namespace ErrorCodes
{
extern const int DICTIONARIES_WAS_NOT_LOADED;
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
/** Functions using Yandex.Metrica dictionaries
......
......@@ -36,6 +36,7 @@ namespace ErrorCodes
extern const int DICTIONARIES_WAS_NOT_LOADED;
extern const int UNSUPPORTED_METHOD;
extern const int UNKNOWN_TYPE;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
/** Functions that use plug-ins (external) dictionaries.
......
......@@ -13,6 +13,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** Function for an unusual conversion to a string:
*
* bitmaskToList - takes an integer - a bitmask, returns a string of degrees of 2 separated by a comma.
......@@ -63,8 +69,8 @@ public:
|| executeType<Int32>(block, arguments, result)
|| executeType<Int64>(block, arguments, result)))
throw Exception("Illegal column " + block.safeGetByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
private:
......
......@@ -16,6 +16,7 @@ namespace DB
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int ILLEGAL_COLUMN;
}
const Float64 EARTH_RADIUS_IN_METERS = 6372797.560856;
......
......@@ -31,6 +31,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
/** Hashing functions.
*
* Half MD5:
......@@ -716,15 +723,13 @@ public:
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed " +
toString(arg_count) + ", should be 1 or 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto first_arg = arguments.front().get();
if (!typeid_cast<const DataTypeString *>(first_arg))
throw Exception{
"Illegal type " + first_arg->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (arg_count == 2)
{
......@@ -739,8 +744,7 @@ public:
!typeid_cast<const DataTypeInt64 *>(second_arg))
throw Exception{
"Illegal type " + second_arg->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
return std::make_shared<DataTypeUInt64>();
......@@ -755,7 +759,7 @@ public:
else if (arg_count == 2)
executeTwoArgs(block, arguments, result);
else
throw std::logic_error{"got into IFunction::execute with unexpected number of arguments"};
throw Exception{"got into IFunction::execute with unexpected number of arguments", ErrorCodes::LOGICAL_ERROR};
}
private:
......@@ -797,8 +801,7 @@ private:
if (!level_col->isConst())
throw Exception{
"Second argument of function " + getName() + " must be an integral constant",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
const auto level = level_col->get64(0);
......@@ -833,10 +836,10 @@ private:
};
struct NameHalfMD5 { static constexpr auto name = "halfMD5"; };
struct NameSipHash64 { static constexpr auto name = "sipHash64"; };
struct NameIntHash32 { static constexpr auto name = "intHash32"; };
struct NameIntHash64 { static constexpr auto name = "intHash64"; };
struct NameHalfMD5 { static constexpr auto name = "halfMD5"; };
struct NameSipHash64 { static constexpr auto name = "sipHash64"; };
struct NameIntHash32 { static constexpr auto name = "intHash32"; };
struct NameIntHash64 { static constexpr auto name = "intHash64"; };
struct ImplCityHash64
{
......
......@@ -11,6 +11,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
/** Functions are logical links: and, or, not, xor.
* Accept any numeric types, return a UInt8 containing 0 or 1.
*/
......
......@@ -7,10 +7,10 @@
#include <Functions/IFunction.h>
#include <Common/config.h>
/** More effective implementations of mathematical functions are possible when connecting a separate library
* Disabled due licence compatibility limitations
/** More efficient implementations of mathematical functions are possible when using a separate library.
* Disabled due to licence compatibility limitations.
* To enable: download http://www.agner.org/optimize/vectorclass.zip and unpack to contrib/vectorclass
* Then rebuild with -DENABLE_VECTORCLASS=1
* Then rebuild with -DENABLE_VECTORCLASS=1
*/
#if USE_VECTORCLASS
......@@ -32,6 +32,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
template <typename Impl>
class FunctionMathNullaryConstFloat64 : public IFunction
{
......
......@@ -12,6 +12,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
/** Pseudo-random number generation functions.
* The function can be called without arguments or with one argument.
* The argument is ignored and only serves to ensure that several calls to one function are considered different and do not stick together.
......@@ -42,11 +47,11 @@ namespace detail
struct LinearCongruentialGenerator
{
/// Constants from man lrand48_r.
/// Constants from `man lrand48_r`.
static constexpr UInt64 a = 0x5DEECE66D;
static constexpr UInt64 c = 0xB;
/// And this is from `head -c8 /dev/urandom | Xxd -p`
/// And this is from `head -c8 /dev/urandom | xxd -p`
UInt64 current = 0x09826f4a081cee35ULL;
LinearCongruentialGenerator() {}
......@@ -236,11 +241,11 @@ public:
struct NameRand { static constexpr auto name = "rand"; };
struct NameRand64 { static constexpr auto name = "rand64"; };
struct NameRand64 { static constexpr auto name = "rand64"; };
struct NameRandConstant { static constexpr auto name = "randConstant"; };
using FunctionRand = FunctionRandom<RandImpl, NameRand> ;
using FunctionRand64 = FunctionRandom<Rand64Impl, NameRand64>;
using FunctionRand = FunctionRandom<RandImpl, NameRand> ;
using FunctionRand64 = FunctionRandom<Rand64Impl, NameRand64>;
using FunctionRandConstant = FunctionRandomConstant<RandImpl, NameRandConstant>;
......
......@@ -15,8 +15,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** Functions for transforming numbers and dates to strings that contain the same set of bytes in the machine representation, and vice versa.
*/
*/
template<typename Name>
......@@ -104,8 +110,8 @@ public:
|| executeType<Float32>(block, arguments, result)
|| executeType<Float64>(block, arguments, result)))
throw Exception("Illegal column " + block.safeGetByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};
......@@ -196,32 +202,32 @@ public:
};
struct NameReinterpretAsUInt8 { static constexpr auto name = "reinterpretAsUInt8"; };
struct NameReinterpretAsUInt16 { static constexpr auto name = "reinterpretAsUInt16"; };
struct NameReinterpretAsUInt32 { static constexpr auto name = "reinterpretAsUInt32"; };
struct NameReinterpretAsUInt64 { static constexpr auto name = "reinterpretAsUInt64"; };
struct NameReinterpretAsInt8 { static constexpr auto name = "reinterpretAsInt8"; };
struct NameReinterpretAsInt16 { static constexpr auto name = "reinterpretAsInt16"; };
struct NameReinterpretAsInt32 { static constexpr auto name = "reinterpretAsInt32"; };
struct NameReinterpretAsInt64 { static constexpr auto name = "reinterpretAsInt64"; };
struct NameReinterpretAsFloat32 { static constexpr auto name = "reinterpretAsFloat32"; };
struct NameReinterpretAsFloat64 { static constexpr auto name = "reinterpretAsFloat64"; };
struct NameReinterpretAsUInt8 { static constexpr auto name = "reinterpretAsUInt8"; };
struct NameReinterpretAsUInt16 { static constexpr auto name = "reinterpretAsUInt16"; };
struct NameReinterpretAsUInt32 { static constexpr auto name = "reinterpretAsUInt32"; };
struct NameReinterpretAsUInt64 { static constexpr auto name = "reinterpretAsUInt64"; };
struct NameReinterpretAsInt8 { static constexpr auto name = "reinterpretAsInt8"; };
struct NameReinterpretAsInt16 { static constexpr auto name = "reinterpretAsInt16"; };
struct NameReinterpretAsInt32 { static constexpr auto name = "reinterpretAsInt32"; };
struct NameReinterpretAsInt64 { static constexpr auto name = "reinterpretAsInt64"; };
struct NameReinterpretAsFloat32 { static constexpr auto name = "reinterpretAsFloat32"; };
struct NameReinterpretAsFloat64 { static constexpr auto name = "reinterpretAsFloat64"; };
struct NameReinterpretAsDate { static constexpr auto name = "reinterpretAsDate"; };
struct NameReinterpretAsDateTime { static constexpr auto name = "reinterpretAsDateTime"; };
struct NameReinterpretAsString { static constexpr auto name = "reinterpretAsString"; };
using FunctionReinterpretAsUInt8 = FunctionReinterpretStringAs<DataTypeUInt8, NameReinterpretAsUInt8> ;
using FunctionReinterpretAsUInt16 = FunctionReinterpretStringAs<DataTypeUInt16, NameReinterpretAsUInt16>;
using FunctionReinterpretAsUInt32 = FunctionReinterpretStringAs<DataTypeUInt32, NameReinterpretAsUInt32>;
using FunctionReinterpretAsUInt64 = FunctionReinterpretStringAs<DataTypeUInt64, NameReinterpretAsUInt64>;
using FunctionReinterpretAsInt8 = FunctionReinterpretStringAs<DataTypeInt8, NameReinterpretAsInt8> ;
using FunctionReinterpretAsInt16 = FunctionReinterpretStringAs<DataTypeInt16, NameReinterpretAsInt16> ;
using FunctionReinterpretAsInt32 = FunctionReinterpretStringAs<DataTypeInt32, NameReinterpretAsInt32> ;
using FunctionReinterpretAsInt64 = FunctionReinterpretStringAs<DataTypeInt64, NameReinterpretAsInt64> ;
using FunctionReinterpretAsFloat32 = FunctionReinterpretStringAs<DataTypeFloat32, NameReinterpretAsFloat32>;
using FunctionReinterpretAsFloat64 = FunctionReinterpretStringAs<DataTypeFloat64, NameReinterpretAsFloat64>;
using FunctionReinterpretAsDate = FunctionReinterpretStringAs<DataTypeDate, NameReinterpretAsDate> ;
using FunctionReinterpretAsDateTime = FunctionReinterpretStringAs<DataTypeDateTime, NameReinterpretAsDateTime>;
struct NameReinterpretAsString { static constexpr auto name = "reinterpretAsString"; };
using FunctionReinterpretAsUInt8 = FunctionReinterpretStringAs<DataTypeUInt8, NameReinterpretAsUInt8>;
using FunctionReinterpretAsUInt16 = FunctionReinterpretStringAs<DataTypeUInt16, NameReinterpretAsUInt16>;
using FunctionReinterpretAsUInt32 = FunctionReinterpretStringAs<DataTypeUInt32, NameReinterpretAsUInt32>;
using FunctionReinterpretAsUInt64 = FunctionReinterpretStringAs<DataTypeUInt64, NameReinterpretAsUInt64>;
using FunctionReinterpretAsInt8 = FunctionReinterpretStringAs<DataTypeInt8, NameReinterpretAsInt8>;
using FunctionReinterpretAsInt16 = FunctionReinterpretStringAs<DataTypeInt16, NameReinterpretAsInt16>;
using FunctionReinterpretAsInt32 = FunctionReinterpretStringAs<DataTypeInt32, NameReinterpretAsInt32>;
using FunctionReinterpretAsInt64 = FunctionReinterpretStringAs<DataTypeInt64, NameReinterpretAsInt64>;
using FunctionReinterpretAsFloat32 = FunctionReinterpretStringAs<DataTypeFloat32, NameReinterpretAsFloat32>;
using FunctionReinterpretAsFloat64 = FunctionReinterpretStringAs<DataTypeFloat64, NameReinterpretAsFloat64>;
using FunctionReinterpretAsDate = FunctionReinterpretStringAs<DataTypeDate, NameReinterpretAsDate>;
using FunctionReinterpretAsDateTime = FunctionReinterpretStringAs<DataTypeDateTime, NameReinterpretAsDateTime>;
using FunctionReinterpretAsString = FunctionReinterpretAsStringImpl<NameReinterpretAsString>;
......
......@@ -15,6 +15,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
/** Rounding Functions:
* roundToExp2 - down to the nearest power of two;
* roundDuration - down to the nearest of: 0, 1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000;
......
......@@ -15,6 +15,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
template <bool negative = false>
struct EmptyImpl
{
......@@ -894,7 +902,7 @@ public:
{
if (arguments.size() < 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size())
+ ", should be at least 2.",
+ ", should be at least 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (const auto arg_idx : ext::range(0, arguments.size()))
......
......@@ -13,6 +13,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** String functions
*
* length, empty, notEmpty,
......
......@@ -15,6 +15,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
/** Functions that split strings into an array of strings or vice versa.
*
* splitByChar(sep, s)
......@@ -546,9 +552,9 @@ public:
};
using FunctionAlphaTokens = FunctionTokens<AlphaTokensImpl> ;
using FunctionSplitByChar = FunctionTokens<SplitByCharImpl> ;
using FunctionAlphaTokens = FunctionTokens<AlphaTokensImpl>;
using FunctionSplitByChar = FunctionTokens<SplitByCharImpl>;
using FunctionSplitByString = FunctionTokens<SplitByStringImpl>;
using FunctionExtractAll = FunctionTokens<ExtractAllImpl> ;
using FunctionExtractAll = FunctionTokens<ExtractAllImpl>;
}
......@@ -22,6 +22,8 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
}
......
......@@ -8,6 +8,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
......
......@@ -3,9 +3,9 @@
#include <memory>
#include <Core/Names.h>
#include <Core/Field.h>
#include <Core/Block.h>
#include <Core/ColumnNumbers.h>
#include <Core/ColumnsWithTypeAndName.h>
#include <DataTypes/IDataType.h>
......@@ -15,14 +15,9 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int FUNCTION_CANNOT_HAVE_PARAMETERS;
extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION;
extern const int NOT_IMPLEMENTED;
}
struct ExpressionAction;
/** Interface for normal functions.
* Normal functions are functions that do not change the number of rows in the table,
......
......@@ -33,6 +33,7 @@ public:
const Context & context) const;
/// Register a table function by its name.
/// No locking, you must register all functions before usage of get.
template <typename Function>
void registerFunction()
{
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册