mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
add function 'flattenTuple'
This commit is contained in:
parent
8de655bf1f
commit
24c0cf86d4
68
src/Functions/flattenTuple.cpp
Normal file
68
src/Functions/flattenTuple.cpp
Normal file
@ -0,0 +1,68 @@
|
|||||||
|
#include <Functions/IFunction.h>
|
||||||
|
#include <Functions/FunctionFactory.h>
|
||||||
|
#include <Functions/FunctionHelpers.h>
|
||||||
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
|
#include <DataTypes/ObjectUtils.h>
|
||||||
|
#include <Columns/ColumnTuple.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int ILLEGAL_COLUMN;
|
||||||
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
class FunctionFlattenTuple : public IFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "flattenTuple";
|
||||||
|
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionFlattenTuple>(); }
|
||||||
|
|
||||||
|
String getName() const override { return name; }
|
||||||
|
size_t getNumberOfArguments() const override { return 1; }
|
||||||
|
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return true; }
|
||||||
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
|
{
|
||||||
|
const auto & type = arguments[0];
|
||||||
|
const auto * type_tuple = checkAndGetDataType<DataTypeTuple>(type.get());
|
||||||
|
if (!type_tuple || !type_tuple->haveExplicitNames())
|
||||||
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||||
|
"Argument for function '{}' must be Named Tuple. Got '{}'",
|
||||||
|
getName(), type->getName());
|
||||||
|
|
||||||
|
auto [paths, types] = flattenTuple(type);
|
||||||
|
Names names;
|
||||||
|
names.reserve(paths.size());
|
||||||
|
for (const auto & path : paths)
|
||||||
|
names.push_back(path.getPath());
|
||||||
|
|
||||||
|
return std::make_shared<DataTypeTuple>(types, names);
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||||
|
{
|
||||||
|
auto column = arguments.at(0).column;
|
||||||
|
if (!checkAndGetColumn<ColumnTuple>(column.get()))
|
||||||
|
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
|
||||||
|
"Illegal column {} of first argument of function {}. Expected ColumnTuple",
|
||||||
|
column->getName(), getName());
|
||||||
|
|
||||||
|
return flattenTuple(column);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerFunctionFlattenTuple(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionFlattenTuple>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -80,6 +80,7 @@ void registerFunctionInitialQueryID(FunctionFactory & factory);
|
|||||||
void registerFunctionServerUUID(FunctionFactory &);
|
void registerFunctionServerUUID(FunctionFactory &);
|
||||||
void registerFunctionZooKeeperSessionUptime(FunctionFactory &);
|
void registerFunctionZooKeeperSessionUptime(FunctionFactory &);
|
||||||
void registerFunctionGetOSKernelVersion(FunctionFactory &);
|
void registerFunctionGetOSKernelVersion(FunctionFactory &);
|
||||||
|
void registerFunctionFlattenTuple(FunctionFactory &);
|
||||||
|
|
||||||
#if USE_ICU
|
#if USE_ICU
|
||||||
void registerFunctionConvertCharset(FunctionFactory &);
|
void registerFunctionConvertCharset(FunctionFactory &);
|
||||||
@ -166,6 +167,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
|||||||
registerFunctionServerUUID(factory);
|
registerFunctionServerUUID(factory);
|
||||||
registerFunctionZooKeeperSessionUptime(factory);
|
registerFunctionZooKeeperSessionUptime(factory);
|
||||||
registerFunctionGetOSKernelVersion(factory);
|
registerFunctionGetOSKernelVersion(factory);
|
||||||
|
registerFunctionFlattenTuple(factory);
|
||||||
|
|
||||||
#if USE_ICU
|
#if USE_ICU
|
||||||
registerFunctionConvertCharset(factory);
|
registerFunctionConvertCharset(factory);
|
||||||
|
4
tests/queries/0_stateless/02246_flatten_tuple.reference
Normal file
4
tests/queries/0_stateless/02246_flatten_tuple.reference
Normal file
@ -0,0 +1,4 @@
|
|||||||
|
([1,2],['a','b'],3,'c',4) Tuple(`t1.a` Array(UInt32), `t1.s` Array(String), b UInt32, `t2.k` String, `t2.v` UInt32)
|
||||||
|
Tuple(id Int8, obj Tuple(k1 Int8, k2 Tuple(k3 String, k4 Nested(k5 Int8, k6 Int8)), some Int8), s String) Tuple(id Int8, `obj.k1` Int8, `obj.k2.k3` String, `obj.k2.k4.k5` Array(Int8), `obj.k2.k4.k6` Array(Int8), `obj.some` Int8, s String)
|
||||||
|
1 1 2 [3,4] [0,0] 0 foo
|
||||||
|
2 0 str [0] [55] 42 bar
|
22
tests/queries/0_stateless/02246_flatten_tuple.sql
Normal file
22
tests/queries/0_stateless/02246_flatten_tuple.sql
Normal file
@ -0,0 +1,22 @@
|
|||||||
|
DROP TABLE IF EXISTS t_flatten_tuple;
|
||||||
|
DROP TABLE IF EXISTS t_flatten_object;
|
||||||
|
|
||||||
|
SET flatten_nested = 0;
|
||||||
|
|
||||||
|
CREATE TABLE t_flatten_tuple(t Tuple(t1 Nested(a UInt32, s String), b UInt32, t2 Tuple(k String, v UInt32))) ENGINE = Memory;
|
||||||
|
|
||||||
|
INSERT INTO t_flatten_tuple VALUES (([(1, 'a'), (2, 'b')], 3, ('c', 4)));
|
||||||
|
|
||||||
|
SELECT flattenTuple(t) AS ft, toTypeName(ft) FROM t_flatten_tuple;
|
||||||
|
|
||||||
|
SET allow_experimental_object_type = 1;
|
||||||
|
CREATE TABLE t_flatten_object(data JSON) ENGINE = Memory;
|
||||||
|
|
||||||
|
INSERT INTO t_flatten_object VALUES ('{"id": 1, "obj": {"k1": 1, "k2": {"k3": 2, "k4": [{"k5": 3}, {"k5": 4}]}}, "s": "foo"}');
|
||||||
|
INSERT INTO t_flatten_object VALUES ('{"id": 2, "obj": {"k2": {"k3": "str", "k4": [{"k6": 55}]}, "some": 42}, "s": "bar"}');
|
||||||
|
|
||||||
|
SELECT toTypeName(data), toTypeName(flattenTuple(data)) FROM t_flatten_object LIMIT 1;
|
||||||
|
SELECT untuple(flattenTuple(data)) FROM t_flatten_object;
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS t_flatten_tuple;
|
||||||
|
DROP TABLE IF EXISTS t_flatten_object;
|
Loading…
Reference in New Issue
Block a user