mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #29471 from lingtaolf/feature/fucntion-mapContainsKeyLike
New function `mapContainsKeyLike`
This commit is contained in:
commit
fba0e11cff
@ -352,4 +352,43 @@ Result:
|
||||
└──────────────────┘
|
||||
```
|
||||
|
||||
## mapContainsKeyLike {#mapContainsKeyLike}
|
||||
|
||||
**Syntax**
|
||||
|
||||
```sql
|
||||
mapContainsKeyLike(map, pattern)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
|
||||
- `map` — Map. [Map](../../sql-reference/data-types/map.md).
|
||||
- `pattern` - String pattern to match.
|
||||
|
||||
**Returned value**
|
||||
|
||||
- `1` if `map` contains `key` like specified pattern, `0` if not.
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
```sql
|
||||
CREATE TABLE test (a Map(String,String)) ENGINE = Memory;
|
||||
|
||||
INSERT INTO test VALUES ({'abc':'abc','def':'def'}), ({'hij':'hij','klm':'klm'});
|
||||
|
||||
SELECT mapContainsKeyLike(a, 'a%') FROM test;
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```text
|
||||
┌─mapContainsKeyLike(a, 'a%')─┐
|
||||
│ 1 │
|
||||
│ 0 │
|
||||
└─────────────────────────────┘
|
||||
```
|
||||
|
||||
|
||||
[Original article](https://clickhouse.com/docs/en/sql-reference/functions/tuple-map-functions/) <!--hide-->
|
||||
|
@ -1,22 +1,10 @@
|
||||
#include "FunctionsStringSearch.h"
|
||||
#include "FunctionFactory.h"
|
||||
#include "MatchImpl.h"
|
||||
#include "like.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace
|
||||
{
|
||||
|
||||
struct NameLike
|
||||
{
|
||||
static constexpr auto name = "like";
|
||||
};
|
||||
|
||||
using LikeImpl = MatchImpl<NameLike, /*SQL LIKE */ true, /*revert*/false>;
|
||||
using FunctionLike = FunctionsStringSearch<LikeImpl>;
|
||||
|
||||
}
|
||||
|
||||
void registerFunctionLike(FunctionFactory & factory)
|
||||
{
|
||||
|
17
src/Functions/like.h
Normal file
17
src/Functions/like.h
Normal file
@ -0,0 +1,17 @@
|
||||
#pragma once
|
||||
|
||||
#include "MatchImpl.h"
|
||||
#include "FunctionsStringSearch.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct NameLike
|
||||
{
|
||||
static constexpr auto name = "like";
|
||||
};
|
||||
|
||||
using LikeImpl = MatchImpl<NameLike, /*SQL LIKE */ true, /*revert*/false>;
|
||||
using FunctionLike = FunctionsStringSearch<LikeImpl>;
|
||||
|
||||
}
|
@ -4,6 +4,7 @@
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
@ -15,6 +16,8 @@
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include "array/arrayIndex.h"
|
||||
#include "Functions/like.h"
|
||||
#include "Functions/FunctionsStringSearch.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -274,6 +277,111 @@ 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();
|
||||
|
||||
const auto it = std::find_if(container.begin(), container.end(), [](int element){ return element == 1; }); // NOLINT
|
||||
vec_res[row] = it == container.end() ? 0 : 1;
|
||||
}
|
||||
|
||||
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)
|
||||
@ -282,6 +390,7 @@ void registerFunctionsMap(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionMapContains>();
|
||||
factory.registerFunction<FunctionMapKeys>();
|
||||
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