Merge pull request #35690 from CurtizJ/flatten-tuple

Add function `flattenTuple`
This commit is contained in:
Anton Popov 2022-03-30 00:24:36 +02:00 committed by GitHub
commit a842a81aba
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 98 additions and 0 deletions

View 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>();
}
}

View File

@ -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);

View 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

View File

@ -0,0 +1,24 @@
-- Tags: no-fasttest
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 ORDER BY data.id;
DROP TABLE IF EXISTS t_flatten_tuple;
DROP TABLE IF EXISTS t_flatten_object;