Add startsWith and endsWith

This commit is contained in:
Vadim 2018-05-28 01:43:31 +03:00
parent d4b861df89
commit 79791ead1e

View File

@ -521,6 +521,39 @@ struct SubstringUTF8Impl
};
struct StartsEndsWithImpl
{
static void vector(StringSource str, const String & str_template, PaddedPODArray<UInt8> & res_data, bool side)
{
size_t ind = 0;
size_t start_pos;
while (!str.isEnd()) {
std::cout << str.getElementSize() << str_template.size() << "!!!!!" << std::endl;
if (str.getElementSize() <= str_template.size()) {
res_data[ind] = false;
} else {
if (!side) {
start_pos = 0;
} else {
start_pos = str.getElementSize() - str_template.size() - 1;
}
for (size_t i = 0; i < str_template.size(); ++i) {
if (str.getWhole().data[i + start_pos] != str_template[i]) {
res_data[ind] = false;
break;
}
res_data[ind] = true;
}
}
str.next();
ind++;
}
}
};
template <typename Impl, typename Name, typename ResultType>
class FunctionStringOrArrayToT : public IFunction
{
@ -1082,6 +1115,126 @@ private:
ErrorCodes::ILLEGAL_COLUMN};
}
};
class FunctionStartsWith: public IFunction
{
public:
static constexpr auto name = "startsWith";
static FunctionPtr create(const Context &)
{
return std::make_shared<FunctionStartsWith>();
}
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override
{
return 2;
}
bool useDefaultImplementationForConstants() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!arguments[0]->isString())
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!arguments[1]->isString())
throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeNumber<UInt8>>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnPtr column_ptr = block.getByPosition(arguments[0]).column;
Field field_template = (*block.getByPosition(arguments[1]).column)[0];
const String string_template = field_template.get<String>();
if (string_template.size() == 0) {
throw Exception("Empty string in function " + getName());
}
if (const ColumnString * column = checkAndGetColumn<ColumnString>(column_ptr.get())) {
auto col_res = ColumnVector<UInt8>::create();
typename ColumnVector<UInt8>::Container &vec_res = col_res->getData();
vec_res.resize(column->size());
StartsEndsWithImpl::vector(StringSource(*column), string_template, vec_res, false);
auto res = ColumnString::create();
block.getByPosition(result).column = std::move(col_res);
}
}
};
class FunctionEndsWith: public IFunction
{
public:
static constexpr auto name = "endsWith";
static FunctionPtr create(const Context &)
{
return std::make_shared<FunctionEndsWith>();
}
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override
{
return 2;
}
bool useDefaultImplementationForConstants() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!arguments[0]->isString())
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!arguments[1]->isString())
throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeNumber<UInt8>>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnPtr column_ptr = block.getByPosition(arguments[0]).column;
Field field_template = (*block.getByPosition(arguments[1]).column)[0];
const String string_template = field_template.get<String>();
if (string_template.size() == 0) {
throw Exception("Empty string in function " + getName());
}
if (const ColumnString * column = checkAndGetColumn<ColumnString>(column_ptr.get())) {
auto col_res = ColumnVector<UInt8>::create();
typename ColumnVector<UInt8>::Container &vec_res = col_res->getData();
vec_res.resize(column->size());
StartsEndsWithImpl::vector(StringSource(*column), string_template, vec_res, true);
auto res = ColumnString::create();
block.getByPosition(result).column = std::move(col_res);
}
}
};
struct NameEmpty
@ -1149,5 +1302,7 @@ void registerFunctionsString(FunctionFactory & factory)
factory.registerFunction<FunctionSubstring>();
factory.registerFunction<FunctionSubstringUTF8>();
factory.registerFunction<FunctionAppendTrailingCharIfAbsent>();
factory.registerFunction<FunctionStartsWith>();
factory.registerFunction<FunctionEndsWith>();
}
}