mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
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:
commit
7bbc4e013a
@ -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)
|
||||
|
@ -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);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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);
|
||||
|
@ -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())
|
||||
|
@ -48,7 +48,7 @@ private:
|
||||
SingleExpressionEvaluation
|
||||
};
|
||||
|
||||
typedef std::vector<std::optional<ConstantExpressionTemplate>> ConstantExpressionTemplates;
|
||||
using ConstantExpressionTemplates = std::vector<std::optional<ConstantExpressionTemplate>>;
|
||||
|
||||
Chunk generate() override;
|
||||
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
|
@ -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)})
|
@ -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;
|
Loading…
Reference in New Issue
Block a user