Some fixes

This commit is contained in:
Pavel Kruglov 2021-05-12 22:02:56 +03:00
parent 8ed6ad7c55
commit 86f061976e
7 changed files with 37 additions and 34 deletions

View File

@ -274,9 +274,11 @@ namespace DB
// TODO: check if a column is const?
if (!is_nullable && !isColumnArray(internal_column) && arrow_column->null_count())
{
throw Exception{
"Can not insert NULL data into non-nullable column \"" + column_name + "\"",
ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN};
throw Exception
{
"Can not insert NULL data into non-nullable column \"" + column_name + "\"",
ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN
};
}
switch (arrow_column->type()->id())
@ -373,10 +375,11 @@ namespace DB
{
return DataTypeFactory::instance().get(internal_type_it->second);
}
throw Exception{
"The type \"" + arrow_type->name() + "\" of an input column \"" + column_name + "\" is not supported for conversion from a "
+ format_name + " data format",
ErrorCodes::CANNOT_CONVERT_TYPE};
throw Exception
{
"The type \"" + arrow_type->name() + "\" of an input column \"" + column_name + "\" is not supported for conversion from a " + format_name + " data format",
ErrorCodes::CANNOT_CONVERT_TYPE
};
}
void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr<arrow::Table> & table,

View File

@ -218,7 +218,7 @@ namespace DB
const ColumnNullable * column_nullable = checkAndGetColumn<ColumnNullable>(column.get());
ColumnPtr nested_column = column_nullable->getNestedColumnPtr();
DataTypePtr nested_type = typeid_cast<const DataTypeNullable *>(column_type.get())->getNestedType();
ColumnPtr null_column = column_nullable->getNullMapColumnPtr();
const ColumnPtr & null_column = column_nullable->getNullMapColumnPtr();
const PaddedPODArray<UInt8> & bytemap = assert_cast<const ColumnVector<UInt8> &>(*null_column).getData();
fillArrowArray(column_name, nested_column, nested_type, &bytemap, array_builder, format_name, start, end);
}
@ -269,9 +269,11 @@ namespace DB
#undef DISPATCH
else
{
throw Exception{"Internal type \"" + column_type_name + "\" of a column \"" + column_name + "\""
" is not supported for conversion into a " + format_name + " data format",
ErrorCodes::UNKNOWN_TYPE};
throw Exception
{
"Internal type \"" + column_type_name + "\" of a column \"" + column_name + "\" is not supported for conversion into a " + format_name + " data format",
ErrorCodes::UNKNOWN_TYPE
};
}
}
@ -349,12 +351,10 @@ namespace DB
{
return arrow_type_it->second;
}
throw Exception{
"The type \"" + type_name + "\" of a column \"" + column_name
+ "\""
" is not supported for conversion into a "
+ format_name + " data format",
ErrorCodes::UNKNOWN_TYPE};
throw Exception{"The type \"" + column_name + "\" of a column \"" + column_name + "\""
" is not supported for conversion into a " + format_name + " data format",
ErrorCodes::UNKNOWN_TYPE};
}
void CHColumnToArrowColumn::chChunkToArrowTable(

View File

@ -7,9 +7,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
DATA_FILE=$CUR_DIR/data_orc/array_test.orc
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(Datetime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) ENGINE=Memory()"
cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC"
timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < $DATA_FILE
${CLICKHOUSE_CLIENT} --query="select * from orc_load"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_arrays (a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(Datetime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) ENGINE=Memory()"
cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_arrays format ORC"
timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_arrays format ORC" < $DATA_FILE
${CLICKHOUSE_CLIENT} --query="select * from orc_arrays"
${CLICKHOUSE_CLIENT} --query="drop table orc_load"
${CLICKHOUSE_CLIENT} --query="drop table orc_arrays"

View File

@ -6,10 +6,10 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
DATA_FILE=$CUR_DIR/data_orc/nested_array_test.orc
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory()"
cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC"
timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < $DATA_FILE
${CLICKHOUSE_CLIENT} --query="select * from orc_load"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_nested_arrays"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nested_arrays (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory()"
cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_nested_arrays format ORC"
timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_nested_arrays format ORC" < $DATA_FILE
${CLICKHOUSE_CLIENT} --query="select * from orc_nested_arrays"
${CLICKHOUSE_CLIENT} --query="drop table orc_load"
${CLICKHOUSE_CLIENT} --query="drop table orc_nested_arrays"

View File

@ -6,10 +6,10 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
DATA_FILE=$CUR_DIR/data_orc/nullable_array_test.orc
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) ENGINE=Memory()"
cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC"
timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < $DATA_FILE
${CLICKHOUSE_CLIENT} --query="select * from orc_load"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_nullable_arrays"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nullable_arrays (a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) ENGINE=Memory()"
cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_nullable_arrays format ORC"
timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_nullable_arrays format ORC" < $DATA_FILE
${CLICKHOUSE_CLIENT} --query="select * from orc_nullable_arrays"
${CLICKHOUSE_CLIENT} --query="drop table orc_load"
${CLICKHOUSE_CLIENT} --query="drop table orc_nullable_arrays"