mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #40173 from Avogar/arrow-dict
Improve and fix dictionaries in Arrow format
This commit is contained in:
commit
b67cb9e378
@ -70,6 +70,7 @@ namespace ErrorCodes
|
||||
extern const int THERE_IS_NO_COLUMN;
|
||||
extern const int UNKNOWN_EXCEPTION;
|
||||
extern const int INCORRECT_NUMBER_OF_COLUMNS;
|
||||
extern const int INCORRECT_DATA;
|
||||
}
|
||||
|
||||
/// Inserts numeric data right into internal column data to reduce an overhead
|
||||
@ -266,6 +267,9 @@ static ColumnWithTypeAndName readColumnWithDecimalData(std::shared_ptr<arrow::Ch
|
||||
/// Creates a null bytemap from arrow's null bitmap
|
||||
static ColumnPtr readByteMapFromArrowColumn(std::shared_ptr<arrow::ChunkedArray> & arrow_column)
|
||||
{
|
||||
if (!arrow_column->null_count())
|
||||
return ColumnUInt8::create(arrow_column->length(), 0);
|
||||
|
||||
auto nullmap_column = ColumnUInt8::create();
|
||||
PaddedPODArray<UInt8> & bytemap_data = assert_cast<ColumnVector<UInt8> &>(*nullmap_column).getData();
|
||||
bytemap_data.reserve(arrow_column->length());
|
||||
@ -298,14 +302,121 @@ static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr<arrow::ChunkedAr
|
||||
return offsets_column;
|
||||
}
|
||||
|
||||
static ColumnPtr readColumnWithIndexesData(std::shared_ptr<arrow::ChunkedArray> & arrow_column)
|
||||
/*
|
||||
* Arrow Dictionary and ClickHouse LowCardinality types are a bit different.
|
||||
* Dictionary(Nullable(X)) in ArrowColumn format is composed of a nullmap, dictionary and an index.
|
||||
* It doesn't have the concept of null or default values.
|
||||
* An empty string is just a regular value appended at any position of the dictionary.
|
||||
* Null values have an index of 0, but it should be ignored since the nullmap will return null.
|
||||
* In ClickHouse LowCardinality, it's different. The dictionary contains null (if dictionary type is Nullable)
|
||||
* and default values at the beginning. [default, ...] when default values have index of 0 or [null, default, ...]
|
||||
* when null values have an index of 0 and default values have an index of 1.
|
||||
* So, we should remap indexes while converting Arrow Dictionary to ClickHouse LowCardinality
|
||||
* */
|
||||
template <typename NumericType, typename VectorType = ColumnVector<NumericType>>
|
||||
static ColumnWithTypeAndName readColumnWithIndexesDataImpl(std::shared_ptr<arrow::ChunkedArray> & arrow_column, const String & column_name, Int64 default_value_index, NumericType dict_size, bool is_nullable)
|
||||
{
|
||||
auto internal_type = std::make_shared<DataTypeNumber<NumericType>>();
|
||||
auto internal_column = internal_type->createColumn();
|
||||
auto & column_data = static_cast<VectorType &>(*internal_column).getData();
|
||||
column_data.reserve(arrow_column->length());
|
||||
NumericType shift = is_nullable ? 2 : 1;
|
||||
|
||||
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i)
|
||||
{
|
||||
std::shared_ptr<arrow::Array> chunk = arrow_column->chunk(chunk_i);
|
||||
if (chunk->length() == 0)
|
||||
continue;
|
||||
|
||||
/// buffers[0] is a null bitmap and buffers[1] are actual values
|
||||
std::shared_ptr<arrow::Buffer> buffer = chunk->data()->buffers[1];
|
||||
const auto * data = reinterpret_cast<const NumericType *>(buffer->data());
|
||||
|
||||
/// Check that indexes are correct (protection against corrupted files)
|
||||
for (int64_t i = 0; i != chunk->length(); ++i)
|
||||
{
|
||||
if (data[i] < 0 || data[i] >= dict_size)
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Index {} in Dictionary column is out of bounds, dictionary size is {}", Int64(data[i]), UInt64(dict_size));
|
||||
}
|
||||
|
||||
/// If dictionary type is not nullable and arrow dictionary contains default type
|
||||
/// at 0 index, we don't need to remap anything (it's the case when this data
|
||||
/// was generated by ClickHouse)
|
||||
if (!is_nullable && default_value_index == 0)
|
||||
{
|
||||
column_data.insert_assume_reserved(data, data + chunk->length());
|
||||
}
|
||||
/// If dictionary don't contain default value, we should move all indexes
|
||||
/// to the right one or two (if dictionary is Nullable) positions
|
||||
/// Example:
|
||||
/// Dictionary:
|
||||
/// dict: ["one", "two"]
|
||||
/// indexes: [0, 1, 0]
|
||||
/// LowCardinality:
|
||||
/// dict: ["", "one", "two"]
|
||||
/// indexes: [1, 2, 1]
|
||||
/// LowCardinality(Nullable):
|
||||
/// dict: [null, "", "one", "two"]
|
||||
/// indexes: [2, 3, 2]
|
||||
else if (default_value_index == -1)
|
||||
{
|
||||
for (int64_t i = 0; i != chunk->length(); ++i)
|
||||
{
|
||||
if (chunk->IsNull(i))
|
||||
column_data.push_back(0);
|
||||
else
|
||||
column_data.push_back(data[i] + shift);
|
||||
}
|
||||
}
|
||||
/// If dictionary contains default value, we change all indexes of it to
|
||||
/// 0 or 1 (if dictionary type is Nullable) and move all indexes
|
||||
/// that are less then default value index to the right one or two
|
||||
/// (if dictionary is Nullable) position and all indexes that are
|
||||
/// greater then default value index zero or one (if dictionary is Nullable)
|
||||
/// positions.
|
||||
/// Example:
|
||||
/// Dictionary:
|
||||
/// dict: ["one", "two", "", "three"]
|
||||
/// indexes: [0, 1, 2, 3, 0]
|
||||
/// LowCardinality :
|
||||
/// dict: ["", "one", "two", "three"]
|
||||
/// indexes: [1, 2, 0, 3, 1]
|
||||
/// LowCardinality(Nullable):
|
||||
/// dict: [null, "", "one", "two", "three"]
|
||||
/// indexes: [2, 3, 1, 4, 2]
|
||||
else
|
||||
{
|
||||
NumericType new_default_index = is_nullable ? 1 : 0;
|
||||
NumericType default_index = NumericType(default_value_index);
|
||||
for (int64_t i = 0; i != chunk->length(); ++i)
|
||||
{
|
||||
if (chunk->IsNull(i))
|
||||
column_data.push_back(0);
|
||||
else
|
||||
{
|
||||
NumericType value = data[i];
|
||||
if (value == default_index)
|
||||
value = new_default_index;
|
||||
else if (value < default_index)
|
||||
value += shift;
|
||||
else
|
||||
value += shift - 1;
|
||||
column_data.push_back(value);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return {std::move(internal_column), std::move(internal_type), column_name};
|
||||
}
|
||||
|
||||
static ColumnPtr readColumnWithIndexesData(std::shared_ptr<arrow::ChunkedArray> & arrow_column, Int64 default_value_index, UInt64 dict_size, bool is_nullable)
|
||||
{
|
||||
switch (arrow_column->type()->id())
|
||||
{
|
||||
# define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \
|
||||
case ARROW_NUMERIC_TYPE: \
|
||||
{ \
|
||||
return readColumnWithNumericData<CPP_NUMERIC_TYPE>(arrow_column, "").column; \
|
||||
return readColumnWithIndexesDataImpl<CPP_NUMERIC_TYPE>(arrow_column, "", default_value_index, dict_size, is_nullable).column; \
|
||||
}
|
||||
FOR_ARROW_INDEXES_TYPES(DISPATCH)
|
||||
# undef DISPATCH
|
||||
@ -327,85 +438,25 @@ static std::shared_ptr<arrow::ChunkedArray> getNestedArrowColumn(std::shared_ptr
|
||||
return std::make_shared<arrow::ChunkedArray>(array_vector);
|
||||
}
|
||||
|
||||
static ColumnWithTypeAndName createLCColumnFromArrowDictionaryValues(
|
||||
const std::shared_ptr<ColumnWithTypeAndName> & dict_values,
|
||||
const ColumnPtr & indexes_column,
|
||||
const String & column_name
|
||||
)
|
||||
{
|
||||
auto lc_type = std::make_shared<DataTypeLowCardinality>(dict_values->type);
|
||||
|
||||
auto lc_column = lc_type->createColumn();
|
||||
|
||||
for (auto i = 0u; i < indexes_column->size(); i++)
|
||||
{
|
||||
Field f;
|
||||
dict_values->column->get(indexes_column->getUInt(i), f);
|
||||
lc_column->insert(f);
|
||||
}
|
||||
|
||||
return {std::move(lc_column), std::move(lc_type), column_name};
|
||||
}
|
||||
|
||||
/*
|
||||
* Dictionary(Nullable(X)) in ArrowColumn format is composed of a nullmap, dictionary and an index.
|
||||
* It doesn't have the concept of null or default values.
|
||||
* An empty string is just a regular value appended at any position of the dictionary.
|
||||
* Null values have an index of 0, but it should be ignored since the nullmap will return null.
|
||||
* In ClickHouse LowCardinality, it's different. The dictionary contains null and default values at the beginning.
|
||||
* [null, default, ...]. Therefore, null values have an index of 0 and default values have an index of 1.
|
||||
* No nullmap is used.
|
||||
* */
|
||||
static ColumnWithTypeAndName createLCOfNullableColumnFromArrowDictionaryValues(
|
||||
const std::shared_ptr<ColumnWithTypeAndName> & dict_values,
|
||||
const ColumnPtr & indexes_column,
|
||||
const ColumnPtr & nullmap_column,
|
||||
const String & column_name
|
||||
)
|
||||
{
|
||||
/*
|
||||
* ArrowColumn format handles nulls by maintaining a nullmap column, there is no nullable type.
|
||||
* Therefore, dict_values->type is the actual data type/ non-nullable. It needs to be transformed into nullable
|
||||
* so LC column is created from nullable type and a null value at the beginning of the collection
|
||||
* is automatically added.
|
||||
* */
|
||||
auto lc_type = std::make_shared<DataTypeLowCardinality>(makeNullable(dict_values->type));
|
||||
|
||||
auto lc_column = lc_type->createColumn();
|
||||
|
||||
for (auto i = 0u; i < indexes_column->size(); i++)
|
||||
{
|
||||
if (nullmap_column && nullmap_column->getBool(i))
|
||||
{
|
||||
lc_column->insertDefault();
|
||||
}
|
||||
else
|
||||
{
|
||||
Field f;
|
||||
dict_values->column->get(indexes_column->getUInt(i), f);
|
||||
lc_column->insert(f);
|
||||
}
|
||||
}
|
||||
|
||||
return {std::move(lc_column), std::move(lc_type), column_name};
|
||||
}
|
||||
|
||||
static ColumnWithTypeAndName readColumnFromArrowColumn(
|
||||
std::shared_ptr<arrow::ChunkedArray> & arrow_column,
|
||||
const std::string & column_name,
|
||||
const std::string & format_name,
|
||||
bool is_nullable,
|
||||
std::unordered_map<String, std::shared_ptr<ColumnWithTypeAndName>> & dictionary_values,
|
||||
bool read_ints_as_dates,
|
||||
std::unordered_map<String, ArrowColumnToCHColumn::DictionaryInfo> & dictionary_infos,
|
||||
bool allow_null_type,
|
||||
bool skip_columns_with_unsupported_types,
|
||||
bool & skipped)
|
||||
bool & skipped,
|
||||
DataTypePtr type_hint = nullptr)
|
||||
{
|
||||
if (!is_nullable && arrow_column->null_count() && arrow_column->type()->id() != arrow::Type::LIST
|
||||
if (!is_nullable && (arrow_column->null_count() || (type_hint && type_hint->isNullable())) && arrow_column->type()->id() != arrow::Type::LIST
|
||||
&& arrow_column->type()->id() != arrow::Type::MAP && arrow_column->type()->id() != arrow::Type::STRUCT &&
|
||||
arrow_column->type()->id() != arrow::Type::DICTIONARY)
|
||||
{
|
||||
auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped);
|
||||
DataTypePtr nested_type_hint;
|
||||
if (type_hint)
|
||||
nested_type_hint = removeNullable(type_hint);
|
||||
auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, nested_type_hint);
|
||||
if (skipped)
|
||||
return {};
|
||||
auto nullmap_column = readByteMapFromArrowColumn(arrow_column);
|
||||
@ -435,14 +486,14 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
|
||||
case arrow::Type::UINT16:
|
||||
{
|
||||
auto column = readColumnWithNumericData<UInt16>(arrow_column, column_name);
|
||||
if (read_ints_as_dates)
|
||||
if (type_hint && (isDateOrDate32(type_hint) || isDateTime(type_hint) || isDateTime64(type_hint)))
|
||||
column.type = std::make_shared<DataTypeDate>();
|
||||
return column;
|
||||
}
|
||||
case arrow::Type::UINT32:
|
||||
{
|
||||
auto column = readColumnWithNumericData<UInt32>(arrow_column, column_name);
|
||||
if (read_ints_as_dates)
|
||||
if (type_hint && (isDateOrDate32(type_hint) || isDateTime(type_hint) || isDateTime64(type_hint)))
|
||||
column.type = std::make_shared<DataTypeDateTime>();
|
||||
return column;
|
||||
}
|
||||
@ -454,8 +505,15 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
|
||||
return readColumnWithDecimalData<arrow::Decimal256Array>(arrow_column, column_name);
|
||||
case arrow::Type::MAP:
|
||||
{
|
||||
DataTypePtr nested_type_hint;
|
||||
if (type_hint)
|
||||
{
|
||||
const auto * map_type_hint = typeid_cast<const DataTypeMap *>(type_hint.get());
|
||||
if (map_type_hint)
|
||||
nested_type_hint = assert_cast<const DataTypeArray *>(map_type_hint->getNestedType().get())->getNestedType();
|
||||
}
|
||||
auto arrow_nested_column = getNestedArrowColumn(arrow_column);
|
||||
auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped);
|
||||
auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, nested_type_hint);
|
||||
if (skipped)
|
||||
return {};
|
||||
|
||||
@ -469,8 +527,15 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
|
||||
}
|
||||
case arrow::Type::LIST:
|
||||
{
|
||||
DataTypePtr nested_type_hint;
|
||||
if (type_hint)
|
||||
{
|
||||
const auto * array_type_hint = typeid_cast<const DataTypeArray *>(type_hint.get());
|
||||
if (array_type_hint)
|
||||
nested_type_hint = array_type_hint->getNestedType();
|
||||
}
|
||||
auto arrow_nested_column = getNestedArrowColumn(arrow_column);
|
||||
auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped);
|
||||
auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, nested_type_hint);
|
||||
if (skipped)
|
||||
return {};
|
||||
auto offsets_column = readOffsetsFromArrowListColumn(arrow_column);
|
||||
@ -493,11 +558,25 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
|
||||
Columns tuple_elements;
|
||||
DataTypes tuple_types;
|
||||
std::vector<String> tuple_names;
|
||||
const auto * tuple_type_hint = type_hint ? typeid_cast<const DataTypeTuple *>(type_hint.get()) : nullptr;
|
||||
|
||||
for (int i = 0; i != arrow_struct_type->num_fields(); ++i)
|
||||
{
|
||||
auto field_name = arrow_struct_type->field(i)->name();
|
||||
DataTypePtr nested_type_hint;
|
||||
if (tuple_type_hint)
|
||||
{
|
||||
if (tuple_type_hint->haveExplicitNames())
|
||||
{
|
||||
auto pos = tuple_type_hint->tryGetPositionByName(field_name);
|
||||
if (pos)
|
||||
nested_type_hint = tuple_type_hint->getElement(*pos);
|
||||
}
|
||||
else if (size_t(i) < tuple_type_hint->getElements().size())
|
||||
nested_type_hint = tuple_type_hint->getElement(i);
|
||||
}
|
||||
auto nested_arrow_column = std::make_shared<arrow::ChunkedArray>(nested_arrow_columns[i]);
|
||||
auto element = readColumnFromArrowColumn(nested_arrow_column, arrow_struct_type->field(i)->name(), format_name, false, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped);
|
||||
auto element = readColumnFromArrowColumn(nested_arrow_column, field_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, nested_type_hint);
|
||||
if (skipped)
|
||||
return {};
|
||||
tuple_elements.emplace_back(std::move(element.column));
|
||||
@ -511,9 +590,11 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
|
||||
}
|
||||
case arrow::Type::DICTIONARY:
|
||||
{
|
||||
auto & dict_values = dictionary_values[column_name];
|
||||
auto & dict_info = dictionary_infos[column_name];
|
||||
const auto is_lc_nullable = arrow_column->null_count() > 0 || (type_hint && type_hint->isLowCardinalityNullable());
|
||||
|
||||
/// Load dictionary values only once and reuse it.
|
||||
if (!dict_values)
|
||||
if (!dict_info.values)
|
||||
{
|
||||
arrow::ArrayVector dict_array;
|
||||
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i)
|
||||
@ -522,8 +603,22 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
|
||||
dict_array.emplace_back(dict_chunk.dictionary());
|
||||
}
|
||||
auto arrow_dict_column = std::make_shared<arrow::ChunkedArray>(dict_array);
|
||||
auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped);
|
||||
dict_values = std::make_shared<ColumnWithTypeAndName>(std::move(dict_column));
|
||||
auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped);
|
||||
for (size_t i = 0; i != dict_column.column->size(); ++i)
|
||||
{
|
||||
if (dict_column.column->isDefaultAt(i))
|
||||
{
|
||||
dict_info.default_value_index = i;
|
||||
break;
|
||||
}
|
||||
}
|
||||
auto lc_type = std::make_shared<DataTypeLowCardinality>(is_lc_nullable ? makeNullable(dict_column.type) : dict_column.type);
|
||||
auto tmp_lc_column = lc_type->createColumn();
|
||||
auto tmp_dict_column = IColumn::mutate(assert_cast<ColumnLowCardinality *>(tmp_lc_column.get())->getDictionaryPtr());
|
||||
dynamic_cast<IColumnUnique *>(tmp_dict_column.get())->uniqueInsertRangeFrom(*dict_column.column, 0, dict_column.column->size());
|
||||
dict_column.column = std::move(tmp_dict_column);
|
||||
dict_info.values = std::make_shared<ColumnWithTypeAndName>(std::move(dict_column));
|
||||
dict_info.dictionary_size = arrow_dict_column->length();
|
||||
}
|
||||
|
||||
arrow::ArrayVector indexes_array;
|
||||
@ -534,20 +629,10 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
|
||||
}
|
||||
|
||||
auto arrow_indexes_column = std::make_shared<arrow::ChunkedArray>(indexes_array);
|
||||
auto indexes_column = readColumnWithIndexesData(arrow_indexes_column);
|
||||
|
||||
const auto contains_null = arrow_column->null_count() > 0;
|
||||
|
||||
if (contains_null)
|
||||
{
|
||||
auto nullmap_column = readByteMapFromArrowColumn(arrow_column);
|
||||
|
||||
return createLCOfNullableColumnFromArrowDictionaryValues(dict_values, indexes_column, nullmap_column, column_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
return createLCColumnFromArrowDictionaryValues(dict_values, indexes_column, column_name);
|
||||
}
|
||||
auto indexes_column = readColumnWithIndexesData(arrow_indexes_column, dict_info.default_value_index, dict_info.dictionary_size, is_lc_nullable);
|
||||
auto lc_column = ColumnLowCardinality::create(dict_info.values->column, indexes_column);
|
||||
auto lc_type = std::make_shared<DataTypeLowCardinality>(is_lc_nullable ? makeNullable(dict_info.values->type) : dict_info.values->type);
|
||||
return {std::move(lc_column), std::move(lc_type), column_name};
|
||||
}
|
||||
# define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \
|
||||
case ARROW_NUMERIC_TYPE: \
|
||||
@ -623,13 +708,13 @@ Block ArrowColumnToCHColumn::arrowSchemaToCHHeader(
|
||||
|
||||
arrow::ArrayVector array_vector = {arrow_array};
|
||||
auto arrow_column = std::make_shared<arrow::ChunkedArray>(array_vector);
|
||||
std::unordered_map<std::string, std::shared_ptr<ColumnWithTypeAndName>> dict_values;
|
||||
std::unordered_map<std::string, DictionaryInfo> dict_infos;
|
||||
bool skipped = false;
|
||||
bool allow_null_type = false;
|
||||
if (hint_header && hint_header->has(field->name()) && hint_header->getByName(field->name()).type->isNullable())
|
||||
allow_null_type = true;
|
||||
ColumnWithTypeAndName sample_column = readColumnFromArrowColumn(
|
||||
arrow_column, field->name(), format_name, false, dict_values, false, allow_null_type, skip_columns_with_unsupported_types, skipped);
|
||||
arrow_column, field->name(), format_name, false, dict_infos, allow_null_type, skip_columns_with_unsupported_types, skipped);
|
||||
if (!skipped)
|
||||
sample_columns.emplace_back(std::move(sample_column));
|
||||
}
|
||||
@ -700,9 +785,17 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr &
|
||||
{
|
||||
if (!nested_tables.contains(search_nested_table_name))
|
||||
{
|
||||
NamesAndTypesList nested_columns;
|
||||
for (const auto & name_and_type : header.getNamesAndTypesList())
|
||||
{
|
||||
if (name_and_type.name.starts_with(nested_table_name + "."))
|
||||
nested_columns.push_back(name_and_type);
|
||||
}
|
||||
auto nested_table_type = Nested::collect(nested_columns).front().type;
|
||||
|
||||
std::shared_ptr<arrow::ChunkedArray> arrow_column = name_to_column_ptr[search_nested_table_name];
|
||||
ColumnsWithTypeAndName cols = {readColumnFromArrowColumn(
|
||||
arrow_column, nested_table_name, format_name, false, dictionary_values, true, true, false, skipped)};
|
||||
arrow_column, nested_table_name, format_name, false, dictionary_infos, true, false, skipped, nested_table_type)};
|
||||
BlockPtr block_ptr = std::make_shared<Block>(cols);
|
||||
auto column_extractor = std::make_shared<NestedColumnExtractHelper>(*block_ptr, case_insensitive_matching);
|
||||
nested_tables[search_nested_table_name] = {block_ptr, column_extractor};
|
||||
@ -735,7 +828,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr &
|
||||
{
|
||||
auto arrow_column = name_to_column_ptr[search_column_name];
|
||||
column = readColumnFromArrowColumn(
|
||||
arrow_column, header_column.name, format_name, false, dictionary_values, true, true, false, skipped);
|
||||
arrow_column, header_column.name, format_name, false, dictionary_infos, true, false, skipped, header_column.type);
|
||||
}
|
||||
|
||||
try
|
||||
|
@ -44,6 +44,14 @@ public:
|
||||
const Block * hint_header = nullptr,
|
||||
bool ignore_case = false);
|
||||
|
||||
struct DictionaryInfo
|
||||
{
|
||||
std::shared_ptr<ColumnWithTypeAndName> values;
|
||||
Int64 default_value_index = -1;
|
||||
UInt64 dictionary_size;
|
||||
};
|
||||
|
||||
|
||||
private:
|
||||
const Block & header;
|
||||
const std::string format_name;
|
||||
@ -55,7 +63,7 @@ private:
|
||||
/// Map {column name : dictionary column}.
|
||||
/// To avoid converting dictionary from Arrow Dictionary
|
||||
/// to LowCardinality every chunk we save it and reuse.
|
||||
std::unordered_map<std::string, std::shared_ptr<ColumnWithTypeAndName>> dictionary_values;
|
||||
std::unordered_map<std::string, DictionaryInfo> dictionary_infos;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -235,27 +235,30 @@ namespace DB
|
||||
}
|
||||
|
||||
template<typename T>
|
||||
static PaddedPODArray<Int64> extractIndexesImpl(ColumnPtr column, size_t start, size_t end)
|
||||
static PaddedPODArray<Int64> extractIndexesImpl(ColumnPtr column, size_t start, size_t end, bool shift)
|
||||
{
|
||||
const PaddedPODArray<T> & data = assert_cast<const ColumnVector<T> *>(column.get())->getData();
|
||||
PaddedPODArray<Int64> result;
|
||||
result.reserve(end - start);
|
||||
std::transform(data.begin() + start, data.begin() + end, std::back_inserter(result), [](T value) { return Int64(value); });
|
||||
if (shift)
|
||||
std::transform(data.begin() + start, data.begin() + end, std::back_inserter(result), [](T value) { return Int64(value) - 1; });
|
||||
else
|
||||
std::transform(data.begin() + start, data.begin() + end, std::back_inserter(result), [](T value) { return Int64(value); });
|
||||
return result;
|
||||
}
|
||||
|
||||
static PaddedPODArray<Int64> extractIndexesImpl(ColumnPtr column, size_t start, size_t end)
|
||||
static PaddedPODArray<Int64> extractIndexesImpl(ColumnPtr column, size_t start, size_t end, bool shift)
|
||||
{
|
||||
switch (column->getDataType())
|
||||
{
|
||||
case TypeIndex::UInt8:
|
||||
return extractIndexesImpl<UInt8>(column, start, end);
|
||||
return extractIndexesImpl<UInt8>(column, start, end, shift);
|
||||
case TypeIndex::UInt16:
|
||||
return extractIndexesImpl<UInt16>(column, start, end);
|
||||
return extractIndexesImpl<UInt16>(column, start, end, shift);
|
||||
case TypeIndex::UInt32:
|
||||
return extractIndexesImpl<UInt32>(column, start, end);
|
||||
return extractIndexesImpl<UInt32>(column, start, end, shift);
|
||||
case TypeIndex::UInt64:
|
||||
return extractIndexesImpl<UInt64>(column, start, end);
|
||||
return extractIndexesImpl<UInt64>(column, start, end, shift);
|
||||
default:
|
||||
throw Exception(fmt::format("Indexes column must be ColumnUInt, got {}.", column->getName()),
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
@ -267,7 +270,7 @@ namespace DB
|
||||
const String & column_name,
|
||||
ColumnPtr & column,
|
||||
const std::shared_ptr<const IDataType> & column_type,
|
||||
const PaddedPODArray<UInt8> * null_bytemap,
|
||||
const PaddedPODArray<UInt8> *,
|
||||
arrow::ArrayBuilder * array_builder,
|
||||
String format_name,
|
||||
size_t start,
|
||||
@ -278,6 +281,7 @@ namespace DB
|
||||
const auto * column_lc = assert_cast<const ColumnLowCardinality *>(column.get());
|
||||
arrow::DictionaryBuilder<ValueType> * builder = assert_cast<arrow::DictionaryBuilder<ValueType> *>(array_builder);
|
||||
auto & dict_values = dictionary_values[column_name];
|
||||
bool is_nullable = column_type->isLowCardinalityNullable();
|
||||
|
||||
/// Convert dictionary from LowCardinality to Arrow dictionary only once and then reuse it.
|
||||
if (!dict_values)
|
||||
@ -288,9 +292,9 @@ namespace DB
|
||||
arrow::Status status = MakeBuilder(pool, value_type, &values_builder);
|
||||
checkStatus(status, column->getName(), format_name);
|
||||
|
||||
auto dict_column = column_lc->getDictionary().getNestedColumn();
|
||||
const auto & dict_type = assert_cast<const DataTypeLowCardinality *>(column_type.get())->getDictionaryType();
|
||||
fillArrowArray(column_name, dict_column, dict_type, nullptr, values_builder.get(), format_name, 0, dict_column->size(), output_string_as_string, dictionary_values);
|
||||
auto dict_column = column_lc->getDictionary().getNestedNotNullableColumn();
|
||||
const auto & dict_type = removeNullable(assert_cast<const DataTypeLowCardinality *>(column_type.get())->getDictionaryType());
|
||||
fillArrowArray(column_name, dict_column, dict_type, nullptr, values_builder.get(), format_name, is_nullable, dict_column->size(), output_string_as_string, dictionary_values);
|
||||
status = values_builder->Finish(&dict_values);
|
||||
checkStatus(status, column->getName(), format_name);
|
||||
}
|
||||
@ -300,15 +304,14 @@ namespace DB
|
||||
|
||||
/// AppendIndices in DictionaryBuilder works only with int64_t data, so we cannot use
|
||||
/// fillArrowArray here and should copy all indexes to int64_t container.
|
||||
auto indexes = extractIndexesImpl(column_lc->getIndexesPtr(), start, end);
|
||||
auto indexes = extractIndexesImpl(column_lc->getIndexesPtr(), start, end, is_nullable);
|
||||
const uint8_t * arrow_null_bytemap_raw_ptr = nullptr;
|
||||
PaddedPODArray<uint8_t> arrow_null_bytemap;
|
||||
if (null_bytemap)
|
||||
if (column_type->isLowCardinalityNullable())
|
||||
{
|
||||
/// Invert values since Arrow interprets 1 as a non-null value, while CH as a null
|
||||
arrow_null_bytemap.reserve(end - start);
|
||||
for (size_t i = start; i < end; ++i)
|
||||
arrow_null_bytemap.emplace_back(!(*null_bytemap)[i]);
|
||||
arrow_null_bytemap.emplace_back(!column_lc->isNullAt(i));
|
||||
|
||||
arrow_null_bytemap_raw_ptr = arrow_null_bytemap.data();
|
||||
}
|
||||
@ -680,7 +683,7 @@ namespace DB
|
||||
{
|
||||
auto nested_type = assert_cast<const DataTypeLowCardinality *>(column_type.get())->getDictionaryType();
|
||||
const auto * lc_column = assert_cast<const ColumnLowCardinality *>(column.get());
|
||||
const auto & nested_column = lc_column->getDictionaryPtr();
|
||||
const auto & nested_column = lc_column->getDictionary().getNestedColumn();
|
||||
const auto & indexes_column = lc_column->getIndexesPtr();
|
||||
return arrow::dictionary(
|
||||
getArrowTypeForLowCardinalityIndexes(indexes_column),
|
||||
|
10
tests/performance/arrow_format.xml
Normal file
10
tests/performance/arrow_format.xml
Normal file
@ -0,0 +1,10 @@
|
||||
<test>
|
||||
<create_query>CREATE TABLE test (uint32 UInt32, n_uint32 Nullable(UInt32), lc LowCardinality(String)) ENGINE=File(Arrow) SETTINGS output_format_arrow_low_cardinality_as_dictionary=1</create_query>
|
||||
<fill_query>insert into test select number, number, toString(number % 10000) from numbers(10000000)</fill_query>
|
||||
|
||||
<query>SELECT uint32 from test format Null</query>
|
||||
<query>SELECT n_uint32 from test format Null</query>
|
||||
<query>SELECT lc from test format Null</query>
|
||||
|
||||
<drop_query>DROP TABLE IF EXISTS test</drop_query>
|
||||
</test>
|
@ -0,0 +1,32 @@
|
||||
dict LowCardinality(Nullable(String))
|
||||
one
|
||||
two
|
||||
three
|
||||
one
|
||||
two
|
||||
dict LowCardinality(Nullable(String))
|
||||
|
||||
one
|
||||
two
|
||||
three
|
||||
|
||||
one
|
||||
three
|
||||
dict LowCardinality(Nullable(String))
|
||||
one
|
||||
two
|
||||
|
||||
three
|
||||
one
|
||||
two
|
||||
three
|
||||
lc LowCardinality(Nullable(String))
|
||||
OK
|
||||
dict LowCardinality(Nullable(String))
|
||||
one
|
||||
two
|
||||
|
||||
three
|
||||
one
|
||||
\N
|
||||
three
|
29
tests/queries/0_stateless/02383_arrow_dict_special_cases.sh
Executable file
29
tests/queries/0_stateless/02383_arrow_dict_special_cases.sh
Executable file
@ -0,0 +1,29 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
|
||||
|
||||
mkdir -p $USER_FILES_PATH/test_02383
|
||||
cp $CURDIR/data_arrow/dictionary*.arrow $USER_FILES_PATH/test_02383/
|
||||
cp $CURDIR/data_arrow/corrupted.arrow $USER_FILES_PATH/test_02383/
|
||||
cp $CURDIR/data_arrow/dict_with_nulls.arrow $USER_FILES_PATH/test_02383/
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary1.arrow')"
|
||||
$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary1.arrow')"
|
||||
$CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary2.arrow')"
|
||||
$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary2.arrow')"
|
||||
$CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary3.arrow')"
|
||||
$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary3.arrow')"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "desc file('test_02383/corrupted.arrow')"
|
||||
$CLICKHOUSE_CLIENT -q "select * from file('test_02383/corrupted.arrow')" 2>&1 | grep -F -q "INCORRECT_DATA" && echo OK || echo FAIL
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "desc file('test_02383/dict_with_nulls.arrow')"
|
||||
$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dict_with_nulls.arrow')"
|
||||
|
||||
|
||||
rm -rf $USER_FILES_PATH/test_02383
|
@ -0,0 +1,4 @@
|
||||
lc LowCardinality(Nullable(String))
|
||||
abc
|
||||
lc LowCardinality(Nullable(String))
|
||||
abc
|
@ -0,0 +1,8 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
insert into function file(02384_data.arrow) select toLowCardinality(toNullable('abc')) as lc settings output_format_arrow_low_cardinality_as_dictionary=1, output_format_arrow_string_as_string=0, engine_file_truncate_on_insert=1;
|
||||
desc file(02384_data.arrow);
|
||||
select * from file(02384_data.arrow);
|
||||
insert into function file(02384_data.arrow) select toLowCardinality(toNullable('abc')) as lc settings output_format_arrow_low_cardinality_as_dictionary=1, output_format_arrow_string_as_string=1, engine_file_truncate_on_insert=1;
|
||||
desc file(02384_data.arrow);
|
||||
select * from file(02384_data.arrow);
|
BIN
tests/queries/0_stateless/data_arrow/corrupted.arrow
Normal file
BIN
tests/queries/0_stateless/data_arrow/corrupted.arrow
Normal file
Binary file not shown.
BIN
tests/queries/0_stateless/data_arrow/dict_with_nulls.arrow
Normal file
BIN
tests/queries/0_stateless/data_arrow/dict_with_nulls.arrow
Normal file
Binary file not shown.
BIN
tests/queries/0_stateless/data_arrow/dictionary1.arrow
Normal file
BIN
tests/queries/0_stateless/data_arrow/dictionary1.arrow
Normal file
Binary file not shown.
BIN
tests/queries/0_stateless/data_arrow/dictionary2.arrow
Normal file
BIN
tests/queries/0_stateless/data_arrow/dictionary2.arrow
Normal file
Binary file not shown.
BIN
tests/queries/0_stateless/data_arrow/dictionary3.arrow
Normal file
BIN
tests/queries/0_stateless/data_arrow/dictionary3.arrow
Normal file
Binary file not shown.
Loading…
Reference in New Issue
Block a user