2013-10-26 19:00:13 +00:00
|
|
|
#include <math.h>
|
|
|
|
|
2014-08-22 00:57:20 +00:00
|
|
|
#include <DB/Functions/FunctionFactory.h>
|
2013-10-26 19:00:13 +00:00
|
|
|
#include <DB/Functions/FunctionsArithmetic.h>
|
|
|
|
#include <DB/Functions/FunctionsMiscellaneous.h>
|
2016-09-21 11:00:04 +00:00
|
|
|
#include <DB/Functions/DataTypeTraits.h>
|
2015-12-21 13:15:13 +00:00
|
|
|
#include <DB/DataTypes/DataTypeEnum.h>
|
2016-08-17 13:38:33 +00:00
|
|
|
#include <DB/DataTypes/DataTypeNullable.h>
|
|
|
|
#include <DB/Columns/ColumnNullable.h>
|
2016-12-12 05:14:46 +00:00
|
|
|
#include <common/ClickHouseRevision.h>
|
2015-12-21 13:15:13 +00:00
|
|
|
#include <ext/enumerate.hpp>
|
2013-10-26 19:00:13 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
|
2016-12-30 01:53:21 +00:00
|
|
|
static size_t widthOfUTF8String(const String & s)
|
2016-12-30 00:19:05 +00:00
|
|
|
{
|
2016-12-30 01:53:21 +00:00
|
|
|
size_t res = 0;
|
|
|
|
for (auto c : s) /// Skip UTF-8 continuation bytes.
|
|
|
|
res += (UInt8(c) <= 0x7F || UInt8(c) >= 0xC0);
|
2016-12-30 00:19:05 +00:00
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
2013-10-26 19:00:13 +00:00
|
|
|
|
2016-07-06 09:47:55 +00:00
|
|
|
void FunctionVisibleWidth::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
|
2016-08-17 13:38:33 +00:00
|
|
|
{
|
2017-01-02 20:12:12 +00:00
|
|
|
auto & src = block.safeGetByPosition(arguments[0]);
|
2017-01-02 20:42:49 +00:00
|
|
|
size_t size = block.rows();
|
2016-08-17 13:38:33 +00:00
|
|
|
|
2016-12-30 01:53:21 +00:00
|
|
|
if (!src.column->isConst())
|
2016-08-17 13:38:33 +00:00
|
|
|
{
|
2016-12-30 01:53:21 +00:00
|
|
|
auto res_col = std::make_shared<ColumnUInt64>(size);
|
2016-08-17 13:38:33 +00:00
|
|
|
auto & res_data = static_cast<ColumnUInt64 &>(*res_col).getData();
|
2017-01-02 20:12:12 +00:00
|
|
|
block.safeGetByPosition(result).column = res_col;
|
2016-08-17 13:38:33 +00:00
|
|
|
|
2016-12-30 01:53:21 +00:00
|
|
|
/// For simplicity reasons, function is implemented by serializing into temporary buffer.
|
2016-09-21 11:00:04 +00:00
|
|
|
|
2016-12-30 01:53:21 +00:00
|
|
|
String tmp;
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
|
|
|
{
|
2016-12-30 01:53:21 +00:00
|
|
|
WriteBufferFromString out(tmp);
|
|
|
|
src.type->serializeTextEscaped(*src.column, i, out);
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
2016-09-21 11:00:04 +00:00
|
|
|
|
2016-12-30 01:53:21 +00:00
|
|
|
res_data[i] = widthOfUTF8String(tmp);
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
|
|
|
}
|
2016-12-30 01:53:21 +00:00
|
|
|
else
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
2016-12-30 01:53:21 +00:00
|
|
|
String tmp;
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
2016-12-30 01:53:21 +00:00
|
|
|
WriteBufferFromString out(tmp);
|
|
|
|
src.type->serializeTextEscaped(*src.column->cut(0, 1)->convertToFullColumnIfConst(), 0, out);
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
|
|
|
|
2017-01-02 20:12:12 +00:00
|
|
|
block.safeGetByPosition(result).column = std::make_shared<ColumnConstUInt64>(size, widthOfUTF8String(tmp));
|
2016-06-21 21:43:43 +00:00
|
|
|
}
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
|
|
|
|
2015-04-28 04:47:27 +00:00
|
|
|
|
2016-12-10 06:12:21 +00:00
|
|
|
void FunctionHasColumnInTable::getReturnTypeAndPrerequisitesImpl(
|
2016-11-15 10:57:11 +00:00
|
|
|
const ColumnsWithTypeAndName & arguments,
|
|
|
|
DataTypePtr & out_return_type,
|
|
|
|
ExpressionActions::Actions & out_prerequisites)
|
|
|
|
{
|
|
|
|
static const std::string arg_pos_description[] = {"First", "Second", "Third"};
|
2016-12-29 19:38:10 +00:00
|
|
|
for (size_t i = 0; i < getNumberOfArguments(); ++i)
|
2016-11-15 10:57:11 +00:00
|
|
|
{
|
|
|
|
const ColumnWithTypeAndName & argument = arguments[i];
|
|
|
|
|
|
|
|
const ColumnConstString * column = typeid_cast<const ColumnConstString *>(argument.column.get());
|
|
|
|
if (!column)
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
arg_pos_description[i] + " argument for function " + getName() + " must be const String.",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
out_return_type = std::make_shared<DataTypeUInt8>();
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-12-10 06:12:21 +00:00
|
|
|
void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
|
2016-11-15 10:57:11 +00:00
|
|
|
{
|
|
|
|
auto get_string_from_block =
|
2016-11-16 11:30:50 +00:00
|
|
|
[&](size_t column_pos) -> const String &
|
2016-11-15 10:57:11 +00:00
|
|
|
{
|
2017-01-02 20:12:12 +00:00
|
|
|
ColumnPtr column = block.safeGetByPosition(column_pos).column;
|
2016-11-15 10:57:11 +00:00
|
|
|
const ColumnConstString * const_column = typeid_cast<const ColumnConstString *>(column.get());
|
|
|
|
return const_column->getData();
|
|
|
|
};
|
|
|
|
|
2016-12-30 01:53:21 +00:00
|
|
|
const String & database_name = get_string_from_block(arguments[0]);
|
|
|
|
const String & table_name = get_string_from_block(arguments[1]);
|
|
|
|
const String & column_name = get_string_from_block(arguments[2]);
|
2016-11-15 10:57:11 +00:00
|
|
|
|
2016-12-30 01:53:21 +00:00
|
|
|
const StoragePtr & table = global_context.getTable(database_name, table_name);
|
2016-11-16 11:30:50 +00:00
|
|
|
const bool has_column = table->hasColumn(column_name);
|
2016-11-15 10:57:11 +00:00
|
|
|
|
2017-01-02 20:12:12 +00:00
|
|
|
block.safeGetByPosition(result).column = std::make_shared<ColumnConstUInt8>(
|
2017-01-02 20:42:49 +00:00
|
|
|
block.rows(), has_column);
|
2016-11-15 10:57:11 +00:00
|
|
|
}
|
|
|
|
|
2014-08-22 00:57:20 +00:00
|
|
|
|
2016-12-12 05:14:46 +00:00
|
|
|
std::string FunctionVersion::getVersion() const
|
2014-08-22 00:57:20 +00:00
|
|
|
{
|
2016-12-12 05:14:46 +00:00
|
|
|
std::ostringstream os;
|
|
|
|
os << DBMS_VERSION_MAJOR << "." << DBMS_VERSION_MINOR << "." << ClickHouseRevision::get();
|
|
|
|
return os.str();
|
|
|
|
}
|
|
|
|
|
2014-08-22 00:57:20 +00:00
|
|
|
|
|
|
|
void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
|
|
|
{
|
2014-11-12 17:23:26 +00:00
|
|
|
factory.registerFunction<FunctionCurrentDatabase>();
|
|
|
|
factory.registerFunction<FunctionHostName>();
|
|
|
|
factory.registerFunction<FunctionVisibleWidth>();
|
|
|
|
factory.registerFunction<FunctionToTypeName>();
|
2016-07-10 02:14:23 +00:00
|
|
|
factory.registerFunction<FunctionToColumnTypeName>();
|
2014-11-12 17:23:26 +00:00
|
|
|
factory.registerFunction<FunctionBlockSize>();
|
2016-05-17 18:25:47 +00:00
|
|
|
factory.registerFunction<FunctionBlockNumber>();
|
2016-07-08 21:46:18 +00:00
|
|
|
factory.registerFunction<FunctionRowNumberInBlock>();
|
|
|
|
factory.registerFunction<FunctionRowNumberInAllBlocks>();
|
2014-11-12 17:23:26 +00:00
|
|
|
factory.registerFunction<FunctionSleep>();
|
|
|
|
factory.registerFunction<FunctionMaterialize>();
|
|
|
|
factory.registerFunction<FunctionIgnore>();
|
2016-04-15 23:10:29 +00:00
|
|
|
factory.registerFunction<FunctionIndexHint>();
|
2015-07-13 21:27:08 +00:00
|
|
|
factory.registerFunction<FunctionIdentity>();
|
2014-11-12 17:23:26 +00:00
|
|
|
factory.registerFunction<FunctionArrayJoin>();
|
2015-06-19 02:34:47 +00:00
|
|
|
factory.registerFunction<FunctionReplicate>();
|
2014-11-12 17:23:26 +00:00
|
|
|
factory.registerFunction<FunctionBar>();
|
2016-11-15 10:57:11 +00:00
|
|
|
factory.registerFunction<FunctionHasColumnInTable>();
|
2014-11-12 17:23:26 +00:00
|
|
|
|
|
|
|
factory.registerFunction<FunctionTuple>();
|
|
|
|
factory.registerFunction<FunctionTupleElement>();
|
|
|
|
factory.registerFunction<FunctionIn<false, false>>();
|
|
|
|
factory.registerFunction<FunctionIn<false, true>>();
|
|
|
|
factory.registerFunction<FunctionIn<true, false>>();
|
|
|
|
factory.registerFunction<FunctionIn<true, true>>();
|
2015-04-03 13:30:00 +00:00
|
|
|
|
|
|
|
factory.registerFunction<FunctionIsFinite>();
|
|
|
|
factory.registerFunction<FunctionIsInfinite>();
|
2015-04-03 14:52:33 +00:00
|
|
|
factory.registerFunction<FunctionIsNaN>();
|
2015-06-03 13:11:59 +00:00
|
|
|
|
|
|
|
factory.registerFunction<FunctionVersion>();
|
2016-04-09 07:47:08 +00:00
|
|
|
factory.registerFunction<FunctionUptime>();
|
2015-06-06 00:28:37 +00:00
|
|
|
|
|
|
|
factory.registerFunction<FunctionRunningAccumulate>();
|
2016-09-16 22:00:30 +00:00
|
|
|
factory.registerFunction<FunctionRunningDifference>();
|
2015-06-06 00:28:37 +00:00
|
|
|
factory.registerFunction<FunctionFinalizeAggregation>();
|
2014-08-22 00:57:20 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|