fix deducing of nested types

This commit is contained in:
Anton Popov 2022-02-25 18:07:30 +03:00
parent fcdebea925
commit 7df8b38f6a
4 changed files with 42 additions and 35 deletions

View File

@ -4,6 +4,7 @@
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <IO/Operators.h>
namespace DB
@ -35,49 +36,41 @@ SerializationPtr DataTypeObject::doGetDefaultSerialization() const
return default_serialization;
}
static constexpr auto NAME_DEFAULT = "Default";
static constexpr auto NAME_NULL = "Null";
String DataTypeObject::doGetName() const
{
WriteBufferFromOwnString out;
out << "Object(" << quote << schema_format;
if (is_nullable)
out << ", " << quote << NAME_NULL;
out << ")";
out << "Object(Nullable(" << quote << schema_format << "))";
else
out << "Object(" << quote << schema_format << ")";
return out.str();
}
static DataTypePtr create(const ASTPtr & arguments)
{
if (!arguments || arguments->children.empty() || arguments->children.size() > 2)
throw Exception("Object data type family must have one or two arguments -"
" name of schema format and type of default value",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const auto * literal = arguments->children[0]->as<ASTLiteral>();
if (!literal || literal->value.getType() != Field::Types::String)
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE,
"Object data type family must have a const string as its first argument");
if (!arguments || arguments->children.size() != 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Object data type family must have one argument - name of schema format");
ASTPtr schema_argument = arguments->children[0];
bool is_nullable = false;
if (arguments->children.size() == 2)
{
const auto * default_literal = arguments->children[1]->as<ASTLiteral>();
if (!default_literal || default_literal->value.getType() != Field::Types::String)
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE,
"Object data type family must have a const string as its second argument");
const auto & default_kind = default_literal->value.get<const String &>();
if (default_kind == NAME_NULL)
is_nullable = true;
else if (default_kind != NAME_DEFAULT)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Unexpected type of default value '{}'. Should be {} or {}",
default_kind, NAME_DEFAULT, NAME_NULL);
if (const auto * func = schema_argument->as<ASTFunction>())
{
if (func->name != "Nullable" || func->arguments->children.size() != 1)
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE,
"Expected 'Nullable(<schema_name>)' as parameter for type Object", func->name);
schema_argument = func->arguments->children[0];
is_nullable = true;
}
return std::make_shared<DataTypeObject>(literal->value.get<const String>(), is_nullable);
const auto * literal = schema_argument->as<ASTLiteral>();
if (!literal || literal->value.getType() != Field::Types::String)
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE,
"Object data type family must have a const string as its schema name parameter");
return std::make_shared<DataTypeObject>(literal->value.get<const String &>(), is_nullable);
}
void registerDataTypeObject(DataTypeFactory & factory)

View File

@ -426,6 +426,9 @@ ColumnPtr reduceNumberOfDimensions(ColumnPtr column, size_t dimensions_to_reduce
return column;
}
/// We save intermidiate column, type and number of array
/// dimensions for each intermidiate node in path in subcolumns tree.
struct ColumnWithTypeAndDimensions
{
ColumnPtr column;
@ -433,8 +436,8 @@ struct ColumnWithTypeAndDimensions
size_t array_dimensions;
};
using SubcolumnsTreeWithTypes = SubcolumnsTree<ColumnWithTypeAndDimensions>;
using Node = SubcolumnsTreeWithTypes::Node;
using SubcolumnsTreeWithColumns = SubcolumnsTree<ColumnWithTypeAndDimensions>;
using Node = SubcolumnsTreeWithColumns::Node;
/// Creates data type and column from tree of subcolumns.
ColumnWithTypeAndDimensions createTypeFromNode(const Node * node)
@ -479,17 +482,19 @@ ColumnWithTypeAndDimensions createTypeFromNode(const Node * node)
const auto & current_array = assert_cast<const ColumnArray &>(*node->data.column);
offsets_columns.push_back(current_array.getOffsetsPtr());
auto first_column = tuple_columns[0].column;
for (size_t i = 0; i < tuple_columns[0].array_dimensions; ++i)
{
const auto & column_array = assert_cast<const ColumnArray &>(*tuple_columns[0].column);
const auto & column_array = assert_cast<const ColumnArray &>(*first_column);
offsets_columns.push_back(column_array.getOffsetsPtr());
tuple_columns[0].column = column_array.getDataPtr();
first_column = column_array.getDataPtr();
}
size_t num_elements = tuple_columns.size();
Columns tuple_elements_columns(num_elements);
DataTypes tuple_elements_types(num_elements);
/// Reduce extra array dimesions to get columns and types of Nested elements.
for (size_t i = 0; i < num_elements; ++i)
{
assert(tuple_columns[i].array_dimensions == tuple_columns[0].array_dimensions);
@ -500,6 +505,7 @@ ColumnWithTypeAndDimensions createTypeFromNode(const Node * node)
auto result_column = ColumnArray::create(ColumnTuple::create(tuple_elements_columns), offsets_columns.back());
auto result_type = createNested(tuple_elements_types, tuple_names);
/// Recreate result Array type and Array column.
for (auto it = offsets_columns.rbegin() + 1; it != offsets_columns.rend(); ++it)
{
result_column = ColumnArray::create(result_column, *it);
@ -575,7 +581,7 @@ std::pair<ColumnPtr, DataTypePtr> unflattenTuple(
/// We add all paths to the subcolumn tree and then create a type from it.
/// The tree stores column, type and number of array dimensions
/// for each intermediate node.
SubcolumnsTreeWithTypes tree;
SubcolumnsTreeWithColumns tree;
for (size_t i = 0; i < paths.size(); ++i)
{
@ -596,6 +602,7 @@ std::pair<ColumnPtr, DataTypePtr> unflattenTuple(
size_t array_dimensions = kind == Node::NESTED ? 1 : parts[pos].anonymous_array_level;
ColumnWithTypeAndDimensions current_column{column, type, array_dimensions};
/// Get type and column for next node.
if (array_dimensions)
{
type = reduceNumberOfDimensions(type, array_dimensions);
@ -633,16 +640,21 @@ static void addConstantToWithClause(const ASTPtr & query, const String & column_
select.with()->children.push_back(std::move(node));
}
/// @expected_columns and @available_columns contain descriptions
/// of extended Object columns.
void replaceMissedSubcolumnsByConstants(
const ColumnsDescription & expected_columns,
const ColumnsDescription & available_columns,
ASTPtr query)
{
NamesAndTypes missed_names_types;
/// Find all subcolumns that are in @expected_columns, but not in @available_columns.
for (const auto & column : available_columns)
{
auto expected_column = expected_columns.getColumn(GetColumnsOptions::All, column.name);
/// Extract all paths from both descriptions to easily check existence of subcolumns.
auto [available_paths, available_types] = flattenTuple(column.type);
auto [expected_paths, expected_types] = flattenTuple(expected_column.type);
@ -675,6 +687,8 @@ void replaceMissedSubcolumnsByConstants(
IdentifierNameSet identifiers;
query->collectIdentifierNames(identifiers);
/// Replace missed subcolumns to default literals of theirs type.
for (const auto & [name, type] : missed_names_types)
if (identifiers.count(name))
addConstantToWithClause(query, name, type);

0
tests/queries/0_stateless/01825_type_json_8.sh Normal file → Executable file
View File

View File

@ -2,7 +2,7 @@
DROP TABLE IF EXISTS t_json_null;
CREATE TABLE t_json_null(id UInt64, data Object('JSON', 'Null'))
CREATE TABLE t_json_null(id UInt64, data Object(Nullable('JSON')))
ENGINE = MergeTree ORDER BY tuple();
INSERT INTO t_json_null FORMAT JSONEachRow {"id": 1, "data": {"k1": 1, "k2" : 2}} {"id": 2, "data": {"k2": 3, "k3" : 4}};