Merge pull request #20541 from kitaisreal/replace-null-fields-in-tuple-during-parsing-with-default-values

Replace null fields in tuple during parsing with default values
This commit is contained in:
Maksim Kita 2021-02-24 12:00:48 +03:00 committed by GitHub
commit 7bbc4e013a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 284 additions and 22 deletions

View File

@ -229,7 +229,21 @@ void ColumnMap::protect()
void ColumnMap::getExtremes(Field & min, Field & max) const
{
nested->getExtremes(min, max);
Field nested_min;
Field nested_max;
nested->getExtremes(nested_min, nested_max);
/// Convert result Array fields to Map fields because client expect min and max field to have type Map
Array nested_min_value = nested_min.get<Array>();
Array nested_max_value = nested_max.get<Array>();
Map map_min_value(nested_min_value.begin(), nested_min_value.end());
Map map_max_value(nested_max_value.begin(), nested_max_value.end());
min = std::move(map_min_value);
max = std::move(map_max_value);
}
void ColumnMap::forEachSubcolumn(ColumnCallback callback)

View File

@ -138,7 +138,7 @@ void DataTypeTuple::serializeBinary(const IColumn & column, size_t row_num, Writ
idx_elem.second->serializeBinary(extractElementColumn(column, idx_elem.first), row_num, ostr);
}
/// Function must atomically insert values into tuple column
template <typename F>
static void addElementSafe(const DataTypes & elems, IColumn & column, F && impl)
{
@ -151,7 +151,8 @@ static void addElementSafe(const DataTypes & elems, IColumn & column, F && impl)
// Check that all columns now have the same size.
size_t new_size = column.size();
for (auto i : ext::range(1, ext::size(elems)))
for (auto i : ext::range(0, ext::size(elems)))
{
const auto & element_column = extractElementColumn(column, i);
if (element_column.size() != new_size)
@ -168,6 +169,7 @@ static void addElementSafe(const DataTypes & elems, IColumn & column, F && impl)
for (const auto & i : ext::range(0, ext::size(elems)))
{
auto & element_column = extractElementColumn(column, i);
if (element_column.size() > old_size)
element_column.popBack(1);
}
@ -215,17 +217,18 @@ void DataTypeTuple::deserializeText(IColumn & column, ReadBuffer & istr, const F
}
elems[i]->deserializeAsTextQuoted(extractElementColumn(column, i), istr, settings);
}
});
// Special format for one element tuple (1,)
if (1 == elems.size())
{
// Special format for one element tuple (1,)
if (1 == elems.size())
{
skipWhitespaceIfAny(istr);
// Allow both (1) and (1,)
checkChar(',', istr);
}
skipWhitespaceIfAny(istr);
// Allow both (1) and (1,)
checkChar(',', istr);
}
skipWhitespaceIfAny(istr);
assertChar(')', istr);
assertChar(')', istr);
});
}
void DataTypeTuple::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -290,10 +293,10 @@ void DataTypeTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr, con
auto & element_column = extractElementColumn(column, element_pos);
elems[element_pos]->deserializeAsTextJSON(element_column, istr, settings);
}
});
skipWhitespaceIfAny(istr);
assertChar('}', istr);
skipWhitespaceIfAny(istr);
assertChar('}', istr);
});
}
else
{
@ -312,10 +315,10 @@ void DataTypeTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr, con
}
elems[i]->deserializeAsTextJSON(extractElementColumn(column, i), istr, settings);
}
});
skipWhitespaceIfAny(istr);
assertChar(']', istr);
skipWhitespaceIfAny(istr);
assertChar(']', istr);
});
}
}

View File

@ -276,6 +276,44 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
return have_unconvertible_element ? Field(Null()) : Field(res);
}
}
else if (const DataTypeMap * type_map = typeid_cast<const DataTypeMap *>(&type))
{
if (src.getType() == Field::Types::Map)
{
const auto & key_type = *type_map->getKeyType();
const auto & value_type = *type_map->getValueType();
const auto & map = src.get<Map>();
size_t map_size = map.size();
Map res(map_size);
bool have_unconvertible_element = false;
for (size_t i = 0; i < map_size; ++i)
{
const auto & map_entry = map[i].get<Tuple>();
const auto & key = map_entry[0];
const auto & value = map_entry[1];
Tuple updated_entry(2);
updated_entry[0] = convertFieldToType(key, key_type);
if (updated_entry[0].isNull() && !key_type.isNullable())
have_unconvertible_element = true;
updated_entry[1] = convertFieldToType(value, value_type);
if (updated_entry[1].isNull() && !value_type.isNullable())
have_unconvertible_element = true;
res[i] = updated_entry;
}
return have_unconvertible_element ? Field(Null()) : Field(res);
}
}
else if (const DataTypeAggregateFunction * agg_func_type = typeid_cast<const DataTypeAggregateFunction *>(&type))
{
if (src.getType() != Field::Types::AggregateFunctionState)

View File

@ -209,6 +209,14 @@ private:
if (map.size() % 2)
return false;
}
else if (literal->value.getType() == Field::Types::Tuple)
{
const Tuple & tuple = literal->value.get<Tuple>();
for (const auto & value : tuple)
if (value.isNull())
return true;
}
String column_name = "_dummy_" + std::to_string(replaced_literals.size());
replaced_literals.emplace_back(literal, column_name, force_nullable);

View File

@ -8,10 +8,14 @@
#include <Formats/FormatFactory.h>
#include <Common/FieldVisitors.h>
#include <Core/Block.h>
#include <Common/typeid_cast.h>
#include <common/find_symbols.h>
#include <Common/typeid_cast.h>
#include <Common/checkStackSize.h>
#include <Parsers/ASTLiteral.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
namespace DB
@ -181,6 +185,87 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx)
}
}
namespace
{
void tryToReplaceNullFieldsInComplexTypesWithDefaultValues(Field & value, const IDataType & data_type)
{
checkStackSize();
WhichDataType type(data_type);
if (type.isTuple() && value.getType() == Field::Types::Tuple)
{
const DataTypeTuple & type_tuple = static_cast<const DataTypeTuple &>(data_type);
Tuple & tuple_value = value.get<Tuple>();
size_t src_tuple_size = tuple_value.size();
size_t dst_tuple_size = type_tuple.getElements().size();
if (src_tuple_size != dst_tuple_size)
throw Exception(fmt::format("Bad size of tuple. Expected size: {}, actual size: {}.",
std::to_string(src_tuple_size), std::to_string(dst_tuple_size)), ErrorCodes::TYPE_MISMATCH);
for (size_t i = 0; i < src_tuple_size; ++i)
{
const auto & element_type = *(type_tuple.getElements()[i]);
if (tuple_value[i].isNull() && !element_type.isNullable())
tuple_value[i] = element_type.getDefault();
tryToReplaceNullFieldsInComplexTypesWithDefaultValues(tuple_value[i], element_type);
}
}
else if (type.isArray() && value.getType() == Field::Types::Array)
{
const DataTypeArray & type_aray = static_cast<const DataTypeArray &>(data_type);
const auto & element_type = *(type_aray.getNestedType());
if (element_type.isNullable())
return;
Array & array_value = value.get<Array>();
size_t array_value_size = array_value.size();
for (size_t i = 0; i < array_value_size; ++i)
{
if (array_value[i].isNull())
array_value[i] = element_type.getDefault();
tryToReplaceNullFieldsInComplexTypesWithDefaultValues(array_value[i], element_type);
}
}
else if (type.isMap() && value.getType() == Field::Types::Map)
{
const DataTypeMap & type_map = static_cast<const DataTypeMap &>(data_type);
const auto & key_type = *type_map.getKeyType();
const auto & value_type = *type_map.getValueType();
auto & map = value.get<Map>();
size_t map_size = map.size();
for (size_t i = 0; i < map_size; ++i)
{
auto & map_entry = map[i].get<Tuple>();
auto & entry_key = map_entry[0];
auto & entry_value = map_entry[1];
if (entry_key.isNull() && !key_type.isNullable())
entry_key = key_type.getDefault();
tryToReplaceNullFieldsInComplexTypesWithDefaultValues(entry_key, key_type);
if (entry_value.isNull() && !value_type.isNullable())
entry_value = value_type.getDefault();
tryToReplaceNullFieldsInComplexTypesWithDefaultValues(entry_value, value_type);
}
}
}
}
bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx)
{
const Block & header = getPort().getHeader();
@ -298,7 +383,13 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx
buf.position() = const_cast<char *>(token_iterator->begin);
std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(ast, *context);
Field value = convertFieldToType(value_raw.first, type, value_raw.second.get());
Field & expression_value = value_raw.first;
if (format_settings.null_as_default)
tryToReplaceNullFieldsInComplexTypesWithDefaultValues(expression_value, type);
Field value = convertFieldToType(expression_value, type, value_raw.second.get());
/// Check that we are indeed allowed to insert a NULL.
if (value.isNull() && !type.isNullable())

View File

@ -48,7 +48,7 @@ private:
SingleExpressionEvaluation
};
typedef std::vector<std::optional<ConstantExpressionTemplate>> ConstantExpressionTemplates;
using ConstantExpressionTemplates = std::vector<std::optional<ConstantExpressionTemplate>>;
Chunk generate() override;

View File

@ -6,6 +6,8 @@ lisi
female
zhangsan
gender
2020-01-01 {1:0,2:1}
2020-01-01 {1:0,2:-1}
1116
1117
1118

View File

@ -22,7 +22,21 @@ select a[b] from table_map;
select b from table_map where a = map('name','lisi', 'gender', 'female');
drop table if exists table_map;
-- Int type
-- Big Integer type
create table table_map (d DATE, m Map(Int8, UInt256)) ENGINE = MergeTree() order by d;
insert into table_map values ('2020-01-01', map(1, 0, 2, 1));
select * from table_map;
drop table table_map;
-- Integer type
create table table_map (d DATE, m Map(Int8, Int8)) ENGINE = MergeTree() order by d;
insert into table_map values ('2020-01-01', map(1, 0, 2, -1));
select * from table_map;
drop table table_map;
-- Unsigned Int type
drop table if exists table_map;
create table table_map(a Map(UInt8, UInt64), b UInt8) Engine = MergeTree() order by b;
insert into table_map select map(number, number+5), number from numbers(1111,4);

View File

@ -0,0 +1,15 @@
Tuple
(0,1)
(0,1)
Tuple nested in Array
[(0,2),(3,0),(0,4)]
[(0,2),(3,0),(0,4)]
Tuple nested in Array nested in Tuple
(0,[(0,2),(3,0),(0,4)])
(0,[(0,2),(3,0),(0,4)])
Tuple nested in Map
{'test':(0,1)}
{'test':(0,1)}
Tuple nested in Map nested in Tuple
(0,{'test':(0,1)})
(0,{'test':(0,1)})

View File

@ -0,0 +1,77 @@
SELECT 'Tuple';
DROP TABLE IF EXISTS test_tuple;
CREATE TABLE test_tuple (value Tuple(UInt8, UInt8)) ENGINE=TinyLog;
SET input_format_null_as_default = 1;
INSERT INTO test_tuple VALUES ((NULL, 1));
SELECT * FROM test_tuple;
SET input_format_null_as_default = 0;
INSERT INTO test_tuple VALUES ((NULL, 2)); -- { clientError 53 }
SELECT * FROM test_tuple;
DROP TABLE test_tuple;
SELECT 'Tuple nested in Array';
DROP TABLE IF EXISTS test_tuple_nested_in_array;
CREATE TABLE test_tuple_nested_in_array (value Array(Tuple(UInt8, UInt8))) ENGINE=TinyLog;
SET input_format_null_as_default = 1;
INSERT INTO test_tuple_nested_in_array VALUES ([(NULL, 2), (3, NULL), (NULL, 4)]);
SELECT * FROM test_tuple_nested_in_array;
SET input_format_null_as_default = 0;
INSERT INTO test_tuple_nested_in_array VALUES ([(NULL, 1)]); -- { clientError 53 }
SELECT * FROM test_tuple_nested_in_array;
DROP TABLE test_tuple_nested_in_array;
SELECT 'Tuple nested in Array nested in Tuple';
DROP TABLE IF EXISTS test_tuple_nested_in_array_nested_in_tuple;
CREATE TABLE test_tuple_nested_in_array_nested_in_tuple (value Tuple(UInt8, Array(Tuple(UInt8, UInt8)))) ENGINE=TinyLog;
SET input_format_null_as_default = 1;
INSERT INTO test_tuple_nested_in_array_nested_in_tuple VALUES ( (NULL, [(NULL, 2), (3, NULL), (NULL, 4)]) );
SELECT * FROM test_tuple_nested_in_array_nested_in_tuple;
SET input_format_null_as_default = 0;
INSERT INTO test_tuple_nested_in_array_nested_in_tuple VALUES ( (NULL, [(NULL, 1)]) ); -- { clientError 53 }
SELECT * FROM test_tuple_nested_in_array_nested_in_tuple;
DROP TABLE test_tuple_nested_in_array_nested_in_tuple;
SELECT 'Tuple nested in Map';
SET allow_experimental_map_type = 1;
DROP TABLE IF EXISTS test_tuple_nested_in_map;
CREATE TABLE test_tuple_nested_in_map (value Map(String, Tuple(UInt8, UInt8))) ENGINE=TinyLog;
SET input_format_null_as_default = 1;
INSERT INTO test_tuple_nested_in_map VALUES (map('test', (NULL, 1)));
SELECT * FROM test_tuple_nested_in_map;
SET input_format_null_as_default = 0;
INSERT INTO test_tuple_nested_in_map VALUES (map('test', (NULL, 1))); -- { clientError 53 }
SELECT * FROM test_tuple_nested_in_map;
DROP TABLE test_tuple_nested_in_map;
SELECT 'Tuple nested in Map nested in Tuple';
DROP TABLE IF EXISTS test_tuple_nested_in_map_nested_in_tuple;
CREATE TABLE test_tuple_nested_in_map_nested_in_tuple (value Tuple(UInt8, Map(String, Tuple(UInt8, UInt8)))) ENGINE=TinyLog;
SET input_format_null_as_default = 1;
INSERT INTO test_tuple_nested_in_map_nested_in_tuple VALUES ( (NULL, map('test', (NULL, 1))) );
SELECT * FROM test_tuple_nested_in_map_nested_in_tuple;
SET input_format_null_as_default = 0;
INSERT INTO test_tuple_nested_in_map_nested_in_tuple VALUES ( (NULL, map('test', (NULL, 1))) ); -- { clientError 53 }
SELECT * FROM test_tuple_nested_in_map_nested_in_tuple;
DROP TABLE test_tuple_nested_in_map_nested_in_tuple;