mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
add new function mapContainsKeyLike
fixed error because of inconsistent offsets
This commit is contained in:
parent
12b6110495
commit
d4d97d7b9a
@ -419,4 +419,9 @@ struct MatchImpl
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
struct NameLike
|
||||||
|
{
|
||||||
|
static constexpr auto name = "like";
|
||||||
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -5,19 +5,10 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
namespace
|
|
||||||
{
|
|
||||||
|
|
||||||
struct NameLike
|
|
||||||
{
|
|
||||||
static constexpr auto name = "like";
|
|
||||||
};
|
|
||||||
|
|
||||||
using LikeImpl = MatchImpl<NameLike, /*SQL LIKE */ true, /*revert*/false>;
|
using LikeImpl = MatchImpl<NameLike, /*SQL LIKE */ true, /*revert*/false>;
|
||||||
using FunctionLike = FunctionsStringSearch<LikeImpl>;
|
using FunctionLike = FunctionsStringSearch<LikeImpl>;
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
void registerFunctionLike(FunctionFactory & factory)
|
void registerFunctionLike(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
factory.registerFunction<FunctionLike>();
|
factory.registerFunction<FunctionLike>();
|
||||||
|
@ -4,6 +4,7 @@
|
|||||||
#include <DataTypes/DataTypeMap.h>
|
#include <DataTypes/DataTypeMap.h>
|
||||||
#include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
#include <DataTypes/DataTypeFixedString.h>
|
||||||
#include <Columns/ColumnMap.h>
|
#include <Columns/ColumnMap.h>
|
||||||
#include <Columns/ColumnArray.h>
|
#include <Columns/ColumnArray.h>
|
||||||
#include <Columns/ColumnString.h>
|
#include <Columns/ColumnString.h>
|
||||||
@ -15,6 +16,8 @@
|
|||||||
#include <Common/assert_cast.h>
|
#include <Common/assert_cast.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include "array/arrayIndex.h"
|
#include "array/arrayIndex.h"
|
||||||
|
#include "Functions/MatchImpl.h"
|
||||||
|
#include "Functions/FunctionsStringSearch.h"
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -28,6 +31,9 @@ namespace ErrorCodes
|
|||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
|
using LikeImpl = MatchImpl<NameLike, /*SQL LIKE */ true, /*revert*/false>;
|
||||||
|
using FunctionLike = FunctionsStringSearch<LikeImpl>;
|
||||||
|
|
||||||
// map(x, y, ...) is a function that allows you to make key-value pair
|
// map(x, y, ...) is a function that allows you to make key-value pair
|
||||||
class FunctionMap : public IFunction
|
class FunctionMap : public IFunction
|
||||||
{
|
{
|
||||||
@ -274,6 +280,121 @@ public:
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
class FunctionMapContainsKeyLike : public IFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "mapContainsKeyLike";
|
||||||
|
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMapContainsKeyLike>(); }
|
||||||
|
String getName() const override { return name; }
|
||||||
|
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*info*/) const override { return true; }
|
||||||
|
|
||||||
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||||
|
{
|
||||||
|
bool is_const = isColumnConst(*arguments[0].column);
|
||||||
|
const ColumnMap * col_map = is_const ? checkAndGetColumnConstData<ColumnMap>(arguments[0].column.get())
|
||||||
|
: checkAndGetColumn<ColumnMap>(arguments[0].column.get());
|
||||||
|
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
|
||||||
|
if (!col_map || !map_type)
|
||||||
|
throw Exception{"First argument for function " + getName() + " must be a map", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||||
|
|
||||||
|
auto col_res = ColumnVector<UInt8>::create();
|
||||||
|
typename ColumnVector<UInt8>::Container & vec_res = col_res->getData();
|
||||||
|
|
||||||
|
if (input_rows_count == 0)
|
||||||
|
return col_res;
|
||||||
|
|
||||||
|
vec_res.resize(input_rows_count);
|
||||||
|
|
||||||
|
const auto & column_array = typeid_cast<const ColumnArray &>(col_map->getNestedColumn());
|
||||||
|
const auto & column_tuple = typeid_cast<const ColumnTuple &>(column_array.getData());
|
||||||
|
|
||||||
|
const ColumnString * column_string = checkAndGetColumn<ColumnString>(column_tuple.getColumn(0));
|
||||||
|
const ColumnFixedString * column_fixed_string = checkAndGetColumn<ColumnFixedString>(column_tuple.getColumn(0));
|
||||||
|
|
||||||
|
FunctionLike func_like;
|
||||||
|
|
||||||
|
for (size_t row = 0; row < input_rows_count; row++)
|
||||||
|
{
|
||||||
|
size_t element_start_row = row != 0 ? column_array.getOffsets()[row-1] : 0;
|
||||||
|
size_t elem_size = column_array.getOffsets()[row]- element_start_row;
|
||||||
|
|
||||||
|
ColumnPtr sub_map_column;
|
||||||
|
DataTypePtr data_type;
|
||||||
|
|
||||||
|
//The keys of one row map will be processed as a single ColumnString
|
||||||
|
if (column_string)
|
||||||
|
{
|
||||||
|
sub_map_column = column_string->cut(element_start_row, elem_size);
|
||||||
|
data_type = std::make_shared<DataTypeString>();
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
sub_map_column = column_fixed_string->cut(element_start_row, elem_size);
|
||||||
|
data_type = std::make_shared<DataTypeFixedString>(checkAndGetColumn<ColumnFixedString>(sub_map_column.get())->getN());
|
||||||
|
}
|
||||||
|
|
||||||
|
size_t col_key_size = sub_map_column->size();
|
||||||
|
auto column = is_const? ColumnConst::create(std::move(sub_map_column), std::move(col_key_size)) : std::move(sub_map_column);
|
||||||
|
|
||||||
|
ColumnsWithTypeAndName new_arguments =
|
||||||
|
{
|
||||||
|
{
|
||||||
|
column,
|
||||||
|
data_type,
|
||||||
|
""
|
||||||
|
},
|
||||||
|
arguments[1]
|
||||||
|
};
|
||||||
|
|
||||||
|
auto res = func_like.executeImpl(new_arguments, result_type, input_rows_count);
|
||||||
|
|
||||||
|
const auto & container = checkAndGetColumn<ColumnUInt8>(res.get())->getData();
|
||||||
|
|
||||||
|
bool exist = 0;
|
||||||
|
for (auto iter = container.begin(); iter != container.end(); iter++)
|
||||||
|
{
|
||||||
|
if (*iter == 1)
|
||||||
|
{
|
||||||
|
exist = 1;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
vec_res[row] = exist;
|
||||||
|
}
|
||||||
|
|
||||||
|
return col_res;
|
||||||
|
}
|
||||||
|
|
||||||
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & 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);
|
||||||
|
|
||||||
|
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
|
||||||
|
const DataTypeString * pattern_type = checkAndGetDataType<DataTypeString>(arguments[1].type.get());
|
||||||
|
|
||||||
|
if (!map_type)
|
||||||
|
throw Exception{"First argument for function " + getName() + " must be a Map",
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||||
|
if (!pattern_type)
|
||||||
|
throw Exception{"Second argument for function " + getName() + " must be String",
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||||
|
|
||||||
|
if (!isStringOrFixedString(map_type->getKeyType()))
|
||||||
|
throw Exception{"Key type of map for function " + getName() + " must be `String` or `FixedString`",
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||||
|
|
||||||
|
return std::make_shared<DataTypeUInt8>();
|
||||||
|
}
|
||||||
|
|
||||||
|
size_t getNumberOfArguments() const override { return 2; }
|
||||||
|
|
||||||
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void registerFunctionsMap(FunctionFactory & factory)
|
void registerFunctionsMap(FunctionFactory & factory)
|
||||||
@ -282,6 +403,7 @@ void registerFunctionsMap(FunctionFactory & factory)
|
|||||||
factory.registerFunction<FunctionMapContains>();
|
factory.registerFunction<FunctionMapContains>();
|
||||||
factory.registerFunction<FunctionMapKeys>();
|
factory.registerFunction<FunctionMapKeys>();
|
||||||
factory.registerFunction<FunctionMapValues>();
|
factory.registerFunction<FunctionMapValues>();
|
||||||
|
factory.registerFunction<FunctionMapContainsKeyLike>();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,6 @@
|
|||||||
|
1 {'1-K1':'1-V1','1-K2':'1-V2'}
|
||||||
|
1 {'1-K1':'1-V1','1-K2':'1-V2'}
|
||||||
|
2 {'2-K1':'2-V1','2-K2':'2-V2'}
|
||||||
|
4 {'4-K1':'4-V1','4-K2':'4-V2'}
|
||||||
|
5 {'5-K1':'5-V1','5-K2':'5-V2'}
|
||||||
|
6 {'6-K1':'6-V1','6-K2':'6-V2'}
|
@ -0,0 +1,12 @@
|
|||||||
|
DROP TABLE IF EXISTS map_containsKeyLike_test;
|
||||||
|
|
||||||
|
CREATE TABLE map_containsKeyLike_test (id UInt32, map Map(String, String)) Engine=MergeTree() ORDER BY id settings index_granularity=2;
|
||||||
|
|
||||||
|
INSERT INTO map_containsKeyLike_test VALUES (1, {'1-K1':'1-V1','1-K2':'1-V2'}),(2,{'2-K1':'2-V1','2-K2':'2-V2'});
|
||||||
|
INSERT INTO map_containsKeyLike_test VALUES (3, {'3-K1':'3-V1','3-K2':'3-V2'}),(4, {'4-K1':'4-V1','4-K2':'4-V2'});
|
||||||
|
INSERT INTO map_containsKeyLike_test VALUES (5, {'5-K1':'5-V1','5-K2':'5-V2'}),(6, {'6-K1':'6-V1','6-K2':'6-V2'});
|
||||||
|
|
||||||
|
SELECT id, map FROM map_containsKeyLike_test WHERE mapContainsKeyLike(map, '1-%') = 1;
|
||||||
|
SELECT id, map FROM map_containsKeyLike_test WHERE mapContainsKeyLike(map, '3-%') = 0 order by id;
|
||||||
|
|
||||||
|
DROP TABLE map_containsKeyLike_test;
|
Loading…
Reference in New Issue
Block a user