mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge pull request #54351 from Avogar/null-as-default-nested
Support NULL as default for nested types Array/Tuple/Map for input formats
This commit is contained in:
commit
3e242f8ba3
@ -493,7 +493,10 @@ void SerializationArray::deserializeText(IColumn & column, ReadBuffer & istr, co
|
||||
deserializeTextImpl(column, istr,
|
||||
[&](IColumn & nested_column)
|
||||
{
|
||||
nested->deserializeTextQuoted(nested_column, istr, settings);
|
||||
if (settings.null_as_default)
|
||||
SerializationNullable::deserializeTextQuotedImpl(nested_column, istr, settings, nested);
|
||||
else
|
||||
nested->deserializeTextQuoted(nested_column, istr, settings);
|
||||
}, false);
|
||||
|
||||
if (whole && !istr.eof())
|
||||
@ -604,7 +607,10 @@ void SerializationArray::deserializeTextCSV(IColumn & column, ReadBuffer & istr,
|
||||
deserializeTextImpl(column, rb,
|
||||
[&](IColumn & nested_column)
|
||||
{
|
||||
nested->deserializeTextCSV(nested_column, rb, settings);
|
||||
if (settings.null_as_default)
|
||||
SerializationNullable::deserializeTextCSVImpl(nested_column, rb, settings, nested);
|
||||
else
|
||||
nested->deserializeTextCSV(nested_column, rb, settings);
|
||||
}, true);
|
||||
}
|
||||
else
|
||||
@ -612,7 +618,10 @@ void SerializationArray::deserializeTextCSV(IColumn & column, ReadBuffer & istr,
|
||||
deserializeTextImpl(column, rb,
|
||||
[&](IColumn & nested_column)
|
||||
{
|
||||
nested->deserializeTextQuoted(nested_column, rb, settings);
|
||||
if (settings.null_as_default)
|
||||
SerializationNullable::deserializeTextQuotedImpl(nested_column, rb, settings, nested);
|
||||
else
|
||||
nested->deserializeTextQuoted(nested_column, rb, settings);
|
||||
}, true);
|
||||
}
|
||||
}
|
||||
|
@ -192,7 +192,10 @@ void SerializationMap::deserializeText(IColumn & column, ReadBuffer & istr, cons
|
||||
deserializeTextImpl(column, istr,
|
||||
[&settings](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn)
|
||||
{
|
||||
subcolumn_serialization->deserializeTextQuoted(subcolumn, buf, settings);
|
||||
if (settings.null_as_default)
|
||||
SerializationNullable::deserializeTextQuotedImpl(subcolumn, buf, settings, subcolumn_serialization);
|
||||
else
|
||||
subcolumn_serialization->deserializeTextQuoted(subcolumn, buf, settings);
|
||||
});
|
||||
|
||||
if (whole && !istr.eof())
|
||||
|
@ -135,7 +135,10 @@ void SerializationTuple::deserializeText(IColumn & column, ReadBuffer & istr, co
|
||||
assertChar(',', istr);
|
||||
skipWhitespaceIfAny(istr);
|
||||
}
|
||||
elems[i]->deserializeTextQuoted(extractElementColumn(column, i), istr, settings);
|
||||
if (settings.null_as_default)
|
||||
SerializationNullable::deserializeTextQuotedImpl(extractElementColumn(column, i), istr, settings, elems[i]);
|
||||
else
|
||||
elems[i]->deserializeTextQuoted(extractElementColumn(column, i), istr, settings);
|
||||
}
|
||||
|
||||
// Special format for one element tuple (1,)
|
||||
@ -366,7 +369,10 @@ void SerializationTuple::deserializeTextCSV(IColumn & column, ReadBuffer & istr,
|
||||
assertChar(settings.csv.tuple_delimiter, istr);
|
||||
skipWhitespaceIfAny(istr);
|
||||
}
|
||||
elems[i]->deserializeTextCSV(extractElementColumn(column, i), istr, settings);
|
||||
if (settings.null_as_default)
|
||||
SerializationNullable::deserializeTextCSVImpl(extractElementColumn(column, i), istr, settings, elems[i]);
|
||||
else
|
||||
elems[i]->deserializeTextCSV(extractElementColumn(column, i), istr, settings);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -1,16 +1,96 @@
|
||||
#include <Formats/insertNullAsDefaultIfNeeded.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const ColumnWithTypeAndName & header_column, size_t column_i, BlockMissingValues * block_missing_values)
|
||||
bool insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const ColumnWithTypeAndName & header_column, size_t column_i, BlockMissingValues * block_missing_values)
|
||||
{
|
||||
if (isArray(input_column.type) && isArray(header_column.type))
|
||||
{
|
||||
ColumnWithTypeAndName nested_input_column;
|
||||
const auto * array_input_column = checkAndGetColumn<ColumnArray>(input_column.column.get());
|
||||
nested_input_column.column = array_input_column->getDataPtr();
|
||||
nested_input_column.type = checkAndGetDataType<DataTypeArray>(input_column.type.get())->getNestedType();
|
||||
|
||||
ColumnWithTypeAndName nested_header_column;
|
||||
nested_header_column.column = checkAndGetColumn<ColumnArray>(header_column.column.get())->getDataPtr();
|
||||
nested_header_column.type = checkAndGetDataType<DataTypeArray>(header_column.type.get())->getNestedType();
|
||||
|
||||
if (!insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr))
|
||||
return false;
|
||||
|
||||
input_column.column = ColumnArray::create(nested_input_column.column, array_input_column->getOffsetsPtr());
|
||||
input_column.type = std::make_shared<DataTypeArray>(std::move(nested_input_column.type));
|
||||
return true;
|
||||
}
|
||||
|
||||
if (isTuple(input_column.type) && isTuple(header_column.type))
|
||||
{
|
||||
const auto * tuple_input_column = checkAndGetColumn<ColumnTuple>(input_column.column.get());
|
||||
const auto * tuple_input_type = checkAndGetDataType<DataTypeTuple>(input_column.type.get());
|
||||
const auto * tuple_header_column = checkAndGetColumn<ColumnTuple>(header_column.column.get());
|
||||
const auto * tuple_header_type = checkAndGetDataType<DataTypeTuple>(header_column.type.get());
|
||||
|
||||
if (tuple_input_type->getElements().size() != tuple_header_type->getElements().size())
|
||||
return false;
|
||||
|
||||
Columns nested_input_columns;
|
||||
nested_input_columns.reserve(tuple_input_type->getElements().size());
|
||||
DataTypes nested_input_types;
|
||||
nested_input_types.reserve(tuple_input_type->getElements().size());
|
||||
bool changed = false;
|
||||
for (size_t i = 0; i != tuple_input_type->getElements().size(); ++i)
|
||||
{
|
||||
ColumnWithTypeAndName nested_input_column;
|
||||
nested_input_column.column = tuple_input_column->getColumnPtr(i);
|
||||
nested_input_column.type = tuple_input_type->getElement(i);
|
||||
ColumnWithTypeAndName nested_header_column;
|
||||
nested_header_column.column = tuple_header_column->getColumnPtr(i);
|
||||
nested_header_column.type = tuple_header_type->getElement(i);
|
||||
changed |= insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr);
|
||||
nested_input_columns.push_back(std::move(nested_input_column.column));
|
||||
nested_input_types.push_back(std::move(nested_input_column.type));
|
||||
}
|
||||
|
||||
if (!changed)
|
||||
return false;
|
||||
|
||||
input_column.column = ColumnTuple::create(std::move(nested_input_columns));
|
||||
input_column.type = std::make_shared<DataTypeTuple>(std::move(nested_input_types));
|
||||
return true;
|
||||
}
|
||||
|
||||
if (isMap(input_column.type) && isMap(header_column.type))
|
||||
{
|
||||
ColumnWithTypeAndName nested_input_column;
|
||||
nested_input_column.column = checkAndGetColumn<ColumnMap>(input_column.column.get())->getNestedColumnPtr();
|
||||
nested_input_column.type = checkAndGetDataType<DataTypeMap>(input_column.type.get())->getNestedType();
|
||||
|
||||
ColumnWithTypeAndName nested_header_column;
|
||||
nested_header_column.column = checkAndGetColumn<ColumnMap>(header_column.column.get())->getNestedColumnPtr();
|
||||
nested_header_column.type = checkAndGetDataType<DataTypeMap>(header_column.type.get())->getNestedType();
|
||||
|
||||
if (!insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr))
|
||||
return false;
|
||||
|
||||
input_column.column = ColumnMap::create(std::move(nested_input_column.column));
|
||||
input_column.type = std::make_shared<DataTypeMap>(std::move(nested_input_column.type));
|
||||
return true;
|
||||
}
|
||||
|
||||
if (!isNullableOrLowCardinalityNullable(input_column.type) || isNullableOrLowCardinalityNullable(header_column.type))
|
||||
return;
|
||||
return false;
|
||||
|
||||
if (block_missing_values)
|
||||
{
|
||||
@ -32,6 +112,8 @@ void insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const Col
|
||||
const auto * lc_type = assert_cast<const DataTypeLowCardinality *>(input_column.type.get());
|
||||
input_column.type = std::make_shared<DataTypeLowCardinality>(removeNullable(lc_type->getDictionaryType()));
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -5,6 +5,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const ColumnWithTypeAndName & header_column, size_t column_i, BlockMissingValues * block_missing_values);
|
||||
bool insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const ColumnWithTypeAndName & header_column, size_t column_i, BlockMissingValues * block_missing_values);
|
||||
|
||||
}
|
||||
|
@ -850,14 +850,6 @@ static ColumnWithTypeAndName readColumnFromORCColumn(
|
||||
if (skipped)
|
||||
return {};
|
||||
|
||||
if (value_type_hint && !value_type_hint->equals(*value_column.type))
|
||||
{
|
||||
/// Cast value column to target type, because it can happen
|
||||
/// that parsed type cannot be ClickHouse Map value type.
|
||||
value_column.column = castColumn(value_column, value_type_hint);
|
||||
value_column.type = value_type_hint;
|
||||
}
|
||||
|
||||
auto offsets_column = readOffsetsFromORCListColumn(orc_map_column);
|
||||
auto map_column = ColumnMap::create(key_column.column, value_column.column, offsets_column);
|
||||
auto map_type = std::make_shared<DataTypeMap>(key_column.type, value_column.type);
|
||||
|
@ -1,6 +1,7 @@
|
||||
DROP TABLE IF EXISTS arraytest;
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
set input_format_null_as_default=0;
|
||||
CREATE TABLE arraytest ( created_date Date DEFAULT toDate(created_at), created_at DateTime DEFAULT now(), strings Array(String) DEFAULT emptyArrayString()) ENGINE = MergeTree(created_date, cityHash64(created_at), (created_date, cityHash64(created_at)), 8192);
|
||||
|
||||
INSERT INTO arraytest (created_at, strings) VALUES (now(), ['aaaaa', 'bbbbb', 'ccccc']);
|
||||
|
@ -0,0 +1,66 @@
|
||||
Native
|
||||
0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0)
|
||||
42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1)
|
||||
2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2)
|
||||
42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3)
|
||||
4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4)
|
||||
Parquet
|
||||
0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0)
|
||||
42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1)
|
||||
2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2)
|
||||
42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3)
|
||||
4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4)
|
||||
ORC
|
||||
0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0)
|
||||
42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1)
|
||||
2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2)
|
||||
42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3)
|
||||
4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4)
|
||||
Arrow
|
||||
0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0)
|
||||
42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1)
|
||||
2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2)
|
||||
42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3)
|
||||
4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4)
|
||||
Avro
|
||||
0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0)
|
||||
42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1)
|
||||
2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2)
|
||||
42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3)
|
||||
4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4)
|
||||
BSONEachRow
|
||||
0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0)
|
||||
42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1)
|
||||
2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2)
|
||||
42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3)
|
||||
4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4)
|
||||
MsgPack
|
||||
0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0)
|
||||
42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1)
|
||||
2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2)
|
||||
42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3)
|
||||
4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4)
|
||||
JSONEachRow
|
||||
0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0)
|
||||
42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1)
|
||||
2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2)
|
||||
42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3)
|
||||
4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4)
|
||||
CSV
|
||||
0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0)
|
||||
42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1)
|
||||
2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2)
|
||||
42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3)
|
||||
4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4)
|
||||
TSV
|
||||
0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0)
|
||||
42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1)
|
||||
2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2)
|
||||
42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3)
|
||||
4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4)
|
||||
Values
|
||||
0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0)
|
||||
42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1)
|
||||
2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2)
|
||||
42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3)
|
||||
4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4)
|
15
tests/queries/0_stateless/02872_null_as_default_nested.sh
Executable file
15
tests/queries/0_stateless/02872_null_as_default_nested.sh
Executable file
@ -0,0 +1,15 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
for format in Native Parquet ORC Arrow Avro BSONEachRow MsgPack JSONEachRow CSV TSV Values
|
||||
do
|
||||
echo $format
|
||||
$CLICKHOUSE_LOCAL -q "select number % 2 ? NULL : number as n, [number, number % 2 ? NULL : number] as arr1, [[[number, number % 2 ? NULL : number], [NULL]]] as arr2, tuple('hello', number % 2 ? NULL : number, number) as tup1, tuple('Hello', tuple('Hello', tuple(number, number % 2 ? NULL : number)), number) as tup2, map(number, number % 2 ? NULL : number) as map1, map(number, map(number, number % 2 ? null : number)) as map2, tuple('Hello', [tuple('Hello', map(number, number % 2 ? NULL : number), [number % 2 ? NULL : number])], number) as nested from numbers(5) format $format" > $CLICKHOUSE_TEST_UNIQUE_NAME.$format
|
||||
$CLICKHOUSE_LOCAL -q "select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.$format', auto, 'n UInt64 default 42, arr1 Array(UInt64), arr2 Array(Array(Array(UInt64))), tup1 Tuple(String, UInt64, UInt64), tup2 Tuple(String, Tuple(String, Tuple(UInt64, UInt64)), UInt64), map1 Map(UInt64, UInt64), map2 Map(UInt64, Map(UInt64, UInt64)), nested Tuple(String, Array(Tuple(String, Map(UInt64, UInt64), Array(UInt64))), UInt64)') settings input_format_null_as_default=1"
|
||||
rm $CLICKHOUSE_TEST_UNIQUE_NAME.$format
|
||||
done
|
||||
|
Loading…
Reference in New Issue
Block a user