Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2016-12-02 23:27:42 +03:00
commit 2def6778e7
4 changed files with 2159 additions and 1908 deletions

File diff suppressed because it is too large Load Diff

View File

@ -13,7 +13,6 @@
#include <DB/Columns/ColumnArray.h> #include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnFixedString.h> #include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnConst.h> #include <DB/Columns/ColumnConst.h>
#include <DB/Functions/FunctionsArray.h>
#include <DB/Functions/IFunction.h> #include <DB/Functions/IFunction.h>
#include <ext/range.hpp> #include <ext/range.hpp>
@ -932,61 +931,13 @@ public:
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionReverse>(); } static FunctionPtr create(const Context & context) { return std::make_shared<FunctionReverse>(); }
/// Получить имя функции. /// Получить имя функции.
String getName() const override String getName() const override;
{
return name;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override DataTypePtr getReturnType(const DataTypes & arguments) const override;
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!typeid_cast<const DataTypeString *>(&*arguments[0]) && !typeid_cast<const DataTypeFixedString *>(&*arguments[0])
&& !typeid_cast<const DataTypeArray *>(&*arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return arguments[0]->clone();
}
/// Выполнить функцию над блоком. /// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override void execute(Block & block, const ColumnNumbers & arguments, size_t result) override;
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
if (const ColumnString * col = typeid_cast<const ColumnString *>(column.get()))
{
std::shared_ptr<ColumnString> col_res = std::make_shared<ColumnString>();
block.getByPosition(result).column = col_res;
ReverseImpl::vector(col->getChars(), col->getOffsets(),
col_res->getChars(), col_res->getOffsets());
}
else if (const ColumnFixedString * col = typeid_cast<const ColumnFixedString *>(column.get()))
{
auto col_res = std::make_shared<ColumnFixedString>(col->getN());
block.getByPosition(result).column = col_res;
ReverseImpl::vector_fixed(col->getChars(), col->getN(),
col_res->getChars());
}
else if (const ColumnConstString * col = typeid_cast<const ColumnConstString *>(column.get()))
{
String res;
ReverseImpl::constant(col->getData(), res);
auto col_res = std::make_shared<ColumnConstString>(col->size(), res);
block.getByPosition(result).column = col_res;
}
else if (typeid_cast<const ColumnArray *>(column.get()) || typeid_cast<const ColumnConstArray *>(column.get()))
{
FunctionArrayReverse().execute(block, arguments, result);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
}; };
@ -1566,108 +1517,12 @@ public:
static constexpr auto name = "appendTrailingCharIfAbsent"; static constexpr auto name = "appendTrailingCharIfAbsent";
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionAppendTrailingCharIfAbsent>(); } static FunctionPtr create(const Context & context) { return std::make_shared<FunctionAppendTrailingCharIfAbsent>(); }
String getName() const override String getName() const override;
{
return name;
}
private: private:
DataTypePtr getReturnType(const DataTypes & arguments) const override DataTypePtr getReturnType(const DataTypes & arguments) const override;
{
if (arguments.size() != 2)
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
if (!typeid_cast<const DataTypeString *>(arguments[0].get())) void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override;
throw Exception{
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
if (!typeid_cast<const DataTypeString *>(arguments[1].get()))
throw Exception{
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
return std::make_shared<DataTypeString>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto & column = block.getByPosition(arguments[0]).column;
const auto & column_char = block.getByPosition(arguments[1]).column;
if (!typeid_cast<const ColumnConstString *>(column_char.get()))
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
const auto & trailing_char_str = static_cast<const ColumnConstString &>(*column_char).getData();
if (trailing_char_str.size() != 1)
throw Exception{
"Second argument of function " + getName() + " must be a one-character string",
ErrorCodes::BAD_ARGUMENTS
};
if (const auto col = typeid_cast<const ColumnString *>(&*column))
{
auto col_res = std::make_shared<ColumnString>();
block.getByPosition(result).column = col_res;
const auto & src_data = col->getChars();
const auto & src_offsets = col->getOffsets();
auto & dst_data = col_res->getChars();
auto & dst_offsets = col_res->getOffsets();
const auto size = src_offsets.size();
dst_data.resize(src_data.size() + size);
dst_offsets.resize(size);
ColumnString::Offset_t src_offset{};
ColumnString::Offset_t dst_offset{};
for (const auto i : ext::range(0, size))
{
const auto src_length = src_offsets[i] - src_offset;
memcpySmallAllowReadWriteOverflow15(&dst_data[dst_offset], &src_data[src_offset], src_length);
src_offset = src_offsets[i];
dst_offset += src_length;
if (src_length > 1 && dst_data[dst_offset - 2] != trailing_char_str.front())
{
dst_data[dst_offset - 1] = trailing_char_str.front();
dst_data[dst_offset] = 0;
++dst_offset;
}
dst_offsets[i] = dst_offset;
}
dst_data.resize_assume_reserved(dst_offset);
}
else if (const auto col = typeid_cast<const ColumnConstString *>(&*column))
{
const auto & in_data = col->getData();
block.getByPosition(result).column = std::make_shared<ColumnConstString>(
col->size(),
in_data.size() == 0 ? in_data :
in_data.back() == trailing_char_str.front() ? in_data : in_data + trailing_char_str);
}
else
throw Exception{
"Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN
};
}
}; };

File diff suppressed because it is too large Load Diff

View File

@ -1,9 +1,173 @@
#include <DB/Functions/FunctionsArray.h>
#include <DB/Functions/FunctionFactory.h> #include <DB/Functions/FunctionFactory.h>
#include <DB/Functions/FunctionsString.h> #include <DB/Functions/FunctionsString.h>
namespace DB namespace DB
{ {
String FunctionReverse::getName() const
{
return name;
}
DataTypePtr FunctionReverse::getReturnType(const DataTypes & arguments) const
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!typeid_cast<const DataTypeString *>(&*arguments[0]) && !typeid_cast<const DataTypeFixedString *>(&*arguments[0])
&& !typeid_cast<const DataTypeArray *>(&*arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return arguments[0]->clone();
}
void FunctionReverse::execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
if (const ColumnString * col = typeid_cast<const ColumnString *>(column.get()))
{
std::shared_ptr<ColumnString> col_res = std::make_shared<ColumnString>();
block.getByPosition(result).column = col_res;
ReverseImpl::vector(col->getChars(), col->getOffsets(),
col_res->getChars(), col_res->getOffsets());
}
else if (const ColumnFixedString * col = typeid_cast<const ColumnFixedString *>(column.get()))
{
auto col_res = std::make_shared<ColumnFixedString>(col->getN());
block.getByPosition(result).column = col_res;
ReverseImpl::vector_fixed(col->getChars(), col->getN(),
col_res->getChars());
}
else if (const ColumnConstString * col = typeid_cast<const ColumnConstString *>(column.get()))
{
String res;
ReverseImpl::constant(col->getData(), res);
auto col_res = std::make_shared<ColumnConstString>(col->size(), res);
block.getByPosition(result).column = col_res;
}
else if (typeid_cast<const ColumnArray *>(column.get()) || typeid_cast<const ColumnConstArray *>(column.get()))
{
FunctionArrayReverse().execute(block, arguments, result);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
String FunctionAppendTrailingCharIfAbsent::getName() const
{
return name;
}
DataTypePtr FunctionAppendTrailingCharIfAbsent::getReturnType(const DataTypes & arguments) const
{
if (arguments.size() != 2)
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
if (!typeid_cast<const DataTypeString *>(arguments[0].get()))
throw Exception{
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
if (!typeid_cast<const DataTypeString *>(arguments[1].get()))
throw Exception{
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
return std::make_shared<DataTypeString>();
}
void FunctionAppendTrailingCharIfAbsent::execute(Block & block, const ColumnNumbers & arguments, const size_t result)
{
const auto & column = block.getByPosition(arguments[0]).column;
const auto & column_char = block.getByPosition(arguments[1]).column;
if (!typeid_cast<const ColumnConstString *>(column_char.get()))
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
const auto & trailing_char_str = static_cast<const ColumnConstString &>(*column_char).getData();
if (trailing_char_str.size() != 1)
throw Exception{
"Second argument of function " + getName() + " must be a one-character string",
ErrorCodes::BAD_ARGUMENTS
};
if (const auto col = typeid_cast<const ColumnString *>(&*column))
{
auto col_res = std::make_shared<ColumnString>();
block.getByPosition(result).column = col_res;
const auto & src_data = col->getChars();
const auto & src_offsets = col->getOffsets();
auto & dst_data = col_res->getChars();
auto & dst_offsets = col_res->getOffsets();
const auto size = src_offsets.size();
dst_data.resize(src_data.size() + size);
dst_offsets.resize(size);
ColumnString::Offset_t src_offset{};
ColumnString::Offset_t dst_offset{};
for (const auto i : ext::range(0, size))
{
const auto src_length = src_offsets[i] - src_offset;
memcpySmallAllowReadWriteOverflow15(&dst_data[dst_offset], &src_data[src_offset], src_length);
src_offset = src_offsets[i];
dst_offset += src_length;
if (src_length > 1 && dst_data[dst_offset - 2] != trailing_char_str.front())
{
dst_data[dst_offset - 1] = trailing_char_str.front();
dst_data[dst_offset] = 0;
++dst_offset;
}
dst_offsets[i] = dst_offset;
}
dst_data.resize_assume_reserved(dst_offset);
}
else if (const auto col = typeid_cast<const ColumnConstString *>(&*column))
{
const auto & in_data = col->getData();
block.getByPosition(result).column = std::make_shared<ColumnConstString>(
col->size(),
in_data.size() == 0 ? in_data :
in_data.back() == trailing_char_str.front() ? in_data : in_data + trailing_char_str);
}
else
throw Exception{
"Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN
};
}
void registerFunctionsString(FunctionFactory & factory) void registerFunctionsString(FunctionFactory & factory)
{ {
factory.registerFunction<FunctionEmpty>(); factory.registerFunction<FunctionEmpty>();