mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Support Map type, fix and add tests
This commit is contained in:
parent
c8b37977da
commit
a4decd0848
@ -117,7 +117,7 @@ class IColumn;
|
||||
M(Bool, skip_unavailable_shards, false, "If 1, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \
|
||||
\
|
||||
M(UInt64, parallel_distributed_insert_select, 0, "Process distributed INSERT SELECT query in the same cluster on local tables on every shard, if 1 SELECT is executed on each shard, if 2 SELECT and INSERT is executed on each shard", 0) \
|
||||
M(UInt64, distributed_group_by_no_merge, 0, "If 1, Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards. If 2 - same as 1 but also apply ORDER BY and LIMIT stages", 0) \
|
||||
M(UInt64, distributed_group_by_no_merge, 0, "If 1, Do not merge aggregation states from different servers for distributed queries (shards will process query up to the Complete stage, initiator just proxies the data from the shards). If 2 the initiator will apply ORDER BY and LIMIT stages (it is not in case when shard process query up to the Complete stage)", 0) \
|
||||
M(Bool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avoiding costly aggregation on the initiator server).", 0) \
|
||||
M(UInt64, optimize_skip_unused_shards_limit, 1000, "Limit for number of sharding key values, turns off optimize_skip_unused_shards if the limit is reached", 0) \
|
||||
M(Bool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <common/DateLUTImpl.h>
|
||||
#include <common/types.h>
|
||||
#include <Core/Block.h>
|
||||
@ -18,6 +19,7 @@
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <Columns/ColumnUnique.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <algorithm>
|
||||
|
||||
@ -293,14 +295,15 @@ namespace DB
|
||||
{
|
||||
if (internal_column.isNullable())
|
||||
{
|
||||
ColumnNullable & column_nullable = typeid_cast<ColumnNullable &>(internal_column);
|
||||
ColumnNullable & column_nullable = assert_cast<ColumnNullable &>(internal_column);
|
||||
readColumnFromArrowColumn(arrow_column, column_nullable.getNestedColumn(), column_name, format_name, true, dictionary_values);
|
||||
fillByteMapFromArrowColumn(arrow_column, column_nullable.getNullMapColumn());
|
||||
return;
|
||||
}
|
||||
|
||||
// TODO: check if a column is const?
|
||||
if (!is_nullable && !checkColumn<ColumnArray>(internal_column) && arrow_column->null_count())
|
||||
/// TODO: check if a column is const?
|
||||
if (!is_nullable && arrow_column->null_count() && arrow_column->type()->id() != arrow::Type::LIST
|
||||
&& arrow_column->type()->id() != arrow::Type::MAP && arrow_column->type()->id() != arrow::Type::STRUCT)
|
||||
{
|
||||
throw Exception
|
||||
{
|
||||
@ -332,6 +335,7 @@ namespace DB
|
||||
//fillColumnWithNumericData<Decimal128, ColumnDecimal<Decimal128>>(arrow_column, read_column); // Have problems with trash values under NULL, but faster
|
||||
fillColumnWithDecimalData(arrow_column, internal_column /*, internal_nested_type*/);
|
||||
break;
|
||||
case arrow::Type::MAP: [[fallthrough]];
|
||||
case arrow::Type::LIST:
|
||||
{
|
||||
arrow::ArrayVector array_vector;
|
||||
@ -344,14 +348,17 @@ namespace DB
|
||||
}
|
||||
auto arrow_nested_column = std::make_shared<arrow::ChunkedArray>(array_vector);
|
||||
|
||||
ColumnArray & column_array = typeid_cast<ColumnArray &>(internal_column);
|
||||
ColumnArray & column_array = arrow_column->type()->id() == arrow::Type::MAP
|
||||
? assert_cast<ColumnMap &>(internal_column).getNestedColumn()
|
||||
: assert_cast<ColumnArray &>(internal_column);
|
||||
|
||||
readColumnFromArrowColumn(arrow_nested_column, column_array.getData(), column_name, format_name, false, dictionary_values);
|
||||
fillOffsetsFromArrowListColumn(arrow_column, column_array.getOffsetsColumn());
|
||||
break;
|
||||
}
|
||||
case arrow::Type::STRUCT:
|
||||
{
|
||||
ColumnTuple & column_tuple = typeid_cast<ColumnTuple &>(internal_column);
|
||||
ColumnTuple & column_tuple = assert_cast<ColumnTuple &>(internal_column);
|
||||
int fields_count = column_tuple.tupleSize();
|
||||
std::vector<arrow::ArrayVector> nested_arrow_columns(fields_count);
|
||||
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i)
|
||||
@ -370,7 +377,7 @@ namespace DB
|
||||
}
|
||||
case arrow::Type::DICTIONARY:
|
||||
{
|
||||
ColumnLowCardinality & column_lc = typeid_cast<ColumnLowCardinality &>(internal_column);
|
||||
ColumnLowCardinality & column_lc = assert_cast<ColumnLowCardinality &>(internal_column);
|
||||
auto & dict_values = dictionary_values[column_name];
|
||||
/// Load dictionary values only once and reuse it.
|
||||
if (!dict_values)
|
||||
@ -430,7 +437,7 @@ namespace DB
|
||||
{
|
||||
if (column_type->isNullable())
|
||||
{
|
||||
DataTypePtr nested_type = typeid_cast<const DataTypeNullable *>(column_type.get())->getNestedType();
|
||||
DataTypePtr nested_type = assert_cast<const DataTypeNullable *>(column_type.get())->getNestedType();
|
||||
return makeNullable(getInternalType(arrow_type, nested_type, column_name, format_name));
|
||||
}
|
||||
|
||||
@ -447,7 +454,7 @@ namespace DB
|
||||
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(column_type.get());
|
||||
if (!array_type)
|
||||
throw Exception{"Cannot convert arrow LIST type to a not Array ClickHouse type " + column_type->getName(), ErrorCodes::CANNOT_CONVERT_TYPE};
|
||||
throw Exception{"Cannot convert arrow LIST type to a not Array/Map ClickHouse type " + column_type->getName(), ErrorCodes::CANNOT_CONVERT_TYPE};
|
||||
|
||||
return std::make_shared<DataTypeArray>(getInternalType(list_nested_type, array_type->getNestedType(), column_name, format_name));
|
||||
}
|
||||
@ -478,7 +485,23 @@ namespace DB
|
||||
if (arrow_type->id() == arrow::Type::DICTIONARY)
|
||||
{
|
||||
const auto * arrow_dict_type = static_cast<arrow::DictionaryType *>(arrow_type.get());
|
||||
return std::make_shared<DataTypeLowCardinality>(getInternalType(arrow_dict_type->value_type(), column_type, column_name, format_name));
|
||||
const auto * lc_type = typeid_cast<const DataTypeLowCardinality *>(column_type.get());
|
||||
/// We allow to insert arrow dictionary into a non-LowCardinality column.
|
||||
const auto & dict_type = lc_type ? lc_type->getDictionaryType() : column_type;
|
||||
return std::make_shared<DataTypeLowCardinality>(getInternalType(arrow_dict_type->value_type(), dict_type, column_name, format_name));
|
||||
}
|
||||
|
||||
if (arrow_type->id() == arrow::Type::MAP)
|
||||
{
|
||||
const auto * arrow_map_type = typeid_cast<arrow::MapType *>(arrow_type.get());
|
||||
const auto * map_type = typeid_cast<const DataTypeMap *>(column_type.get());
|
||||
if (!map_type)
|
||||
throw Exception{"Cannot convert arrow MAP type to a not Map ClickHouse type " + column_type->getName(), ErrorCodes::CANNOT_CONVERT_TYPE};
|
||||
|
||||
return std::make_shared<DataTypeMap>(
|
||||
getInternalType(arrow_map_type->key_type(), map_type->getKeyType(), column_name, format_name),
|
||||
getInternalType(arrow_map_type->item_type(), map_type->getValueType(), column_name, format_name)
|
||||
);
|
||||
}
|
||||
|
||||
if (const auto * internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(),
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Core/callOnTypeIndex.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
@ -15,6 +16,7 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <Processors/Formats/IOutputFormat.h>
|
||||
#include <arrow/api.h>
|
||||
#include <arrow/builder.h>
|
||||
@ -107,6 +109,7 @@ namespace DB
|
||||
size_t end,
|
||||
std::unordered_map<String, std::shared_ptr<arrow::Array>> & dictionary_values);
|
||||
|
||||
template <typename Builder>
|
||||
static void fillArrowArrayWithArrayColumnData(
|
||||
const String & column_name,
|
||||
ColumnPtr & column,
|
||||
@ -118,12 +121,12 @@ namespace DB
|
||||
size_t end,
|
||||
std::unordered_map<String, std::shared_ptr<arrow::Array>> & dictionary_values)
|
||||
{
|
||||
const auto * column_array = typeid_cast<const ColumnArray *>(column.get());
|
||||
const auto * column_array = assert_cast<const ColumnArray *>(column.get());
|
||||
ColumnPtr nested_column = column_array->getDataPtr();
|
||||
DataTypePtr nested_type = typeid_cast<const DataTypeArray *>(column_type.get())->getNestedType();
|
||||
DataTypePtr nested_type = assert_cast<const DataTypeArray *>(column_type.get())->getNestedType();
|
||||
const auto & offsets = column_array->getOffsets();
|
||||
|
||||
arrow::ListBuilder & builder = assert_cast<arrow::ListBuilder &>(*array_builder);
|
||||
Builder & builder = assert_cast<Builder &>(*array_builder);
|
||||
arrow::ArrayBuilder * value_builder = builder.value_builder();
|
||||
arrow::Status components_status;
|
||||
|
||||
@ -147,8 +150,8 @@ namespace DB
|
||||
size_t end,
|
||||
std::unordered_map<String, std::shared_ptr<arrow::Array>> & dictionary_values)
|
||||
{
|
||||
const auto * column_tuple = typeid_cast<const ColumnTuple *>(column.get());
|
||||
const auto & nested_types = typeid_cast<const DataTypeTuple *>(column_type.get())->getElements();
|
||||
const auto * column_tuple = assert_cast<const ColumnTuple *>(column.get());
|
||||
const auto & nested_types = assert_cast<const DataTypeTuple *>(column_type.get())->getElements();
|
||||
|
||||
arrow::StructBuilder & builder = assert_cast<arrow::StructBuilder &>(*array_builder);
|
||||
|
||||
@ -168,7 +171,7 @@ namespace DB
|
||||
template<typename T>
|
||||
static PaddedPODArray<Int64> extractIndexesImpl(ColumnPtr column, size_t start, size_t end)
|
||||
{
|
||||
const PaddedPODArray<T> & data = checkAndGetColumn<ColumnVector<T>>(*column)->getData();
|
||||
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); });
|
||||
@ -205,21 +208,21 @@ namespace DB
|
||||
size_t end,
|
||||
std::unordered_map<String, std::shared_ptr<arrow::Array>> & dictionary_values)
|
||||
{
|
||||
const auto * column_lc = typeid_cast<const ColumnLowCardinality *>(column.get());
|
||||
arrow::DictionaryBuilder<ValueType> * builder = typeid_cast<arrow::DictionaryBuilder<ValueType> *>(array_builder);
|
||||
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];
|
||||
|
||||
/// Convert dictionary from LowCardinality to Arrow dictionary only once and then reuse it.
|
||||
if (!dict_values)
|
||||
{
|
||||
auto value_type = typeid_cast<arrow::DictionaryType *>(builder->type().get())->value_type();
|
||||
const auto & value_type = assert_cast<arrow::DictionaryType *>(builder->type().get())->value_type();
|
||||
std::unique_ptr<arrow::ArrayBuilder> values_builder;
|
||||
arrow::MemoryPool* pool = arrow::default_memory_pool();
|
||||
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 = typeid_cast<const DataTypeLowCardinality *>(column_type.get())->getDictionaryType();
|
||||
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(), dictionary_values);
|
||||
status = values_builder->Finish(&dict_values);
|
||||
checkStatus(status, column->getName(), format_name);
|
||||
@ -259,8 +262,7 @@ namespace DB
|
||||
size_t end,
|
||||
std::unordered_map<String, std::shared_ptr<arrow::Array>> & dictionary_values)
|
||||
{
|
||||
auto * dict_type = typeid_cast<arrow::DictionaryType *>(array_builder->type().get());
|
||||
auto value_type = dict_type->value_type();
|
||||
auto value_type = assert_cast<arrow::DictionaryType *>(array_builder->type().get())->value_type();
|
||||
|
||||
#define DISPATCH(ARROW_TYPE_ID, ARROW_TYPE) \
|
||||
if (arrow::Type::ARROW_TYPE_ID == value_type->id()) \
|
||||
@ -367,9 +369,9 @@ namespace DB
|
||||
|
||||
if ("Nullable" == column_type_name)
|
||||
{
|
||||
const ColumnNullable * column_nullable = checkAndGetColumn<ColumnNullable>(column.get());
|
||||
const ColumnNullable * column_nullable = assert_cast<const ColumnNullable *>(column.get());
|
||||
ColumnPtr nested_column = column_nullable->getNestedColumnPtr();
|
||||
DataTypePtr nested_type = typeid_cast<const DataTypeNullable *>(column_type.get())->getNestedType();
|
||||
DataTypePtr nested_type = assert_cast<const DataTypeNullable *>(column_type.get())->getNestedType();
|
||||
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, dictionary_values);
|
||||
@ -392,7 +394,7 @@ namespace DB
|
||||
}
|
||||
else if ("Array" == column_type_name)
|
||||
{
|
||||
fillArrowArrayWithArrayColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values);
|
||||
fillArrowArrayWithArrayColumnData<arrow::ListBuilder>(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values);
|
||||
}
|
||||
else if ("Tuple" == column_type_name)
|
||||
{
|
||||
@ -402,6 +404,12 @@ namespace DB
|
||||
{
|
||||
fillArrowArrayWithLowCardinalityColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values);
|
||||
}
|
||||
else if ("Map" == column_type_name)
|
||||
{
|
||||
ColumnPtr column_array = assert_cast<const ColumnMap *>(column.get())->getNestedColumnPtr();
|
||||
DataTypePtr array_type = assert_cast<const DataTypeMap *>(column_type.get())->getNestedType();
|
||||
fillArrowArrayWithArrayColumnData<arrow::MapBuilder>(column_name, column_array, array_type, null_bytemap, array_builder, format_name, start, end, dictionary_values);
|
||||
}
|
||||
else if (isDecimal(column_type))
|
||||
{
|
||||
auto fill_decimal = [&](const auto & types) -> bool
|
||||
@ -446,7 +454,7 @@ namespace DB
|
||||
size_t start,
|
||||
size_t end)
|
||||
{
|
||||
const auto & column = static_cast<const typename DataType::ColumnType &>(*write_column);
|
||||
const auto & column = assert_cast<const typename DataType::ColumnType &>(*write_column);
|
||||
arrow::DecimalBuilder & builder = assert_cast<arrow::DecimalBuilder &>(*array_builder);
|
||||
arrow::Status status;
|
||||
|
||||
@ -487,8 +495,8 @@ namespace DB
|
||||
{
|
||||
if (column_type->isNullable())
|
||||
{
|
||||
DataTypePtr nested_type = typeid_cast<const DataTypeNullable *>(column_type.get())->getNestedType();
|
||||
ColumnPtr nested_column = checkAndGetColumn<ColumnNullable>(*column)->getNestedColumnPtr();
|
||||
DataTypePtr nested_type = assert_cast<const DataTypeNullable *>(column_type.get())->getNestedType();
|
||||
ColumnPtr nested_column = assert_cast<const ColumnNullable *>(column.get())->getNestedColumnPtr();
|
||||
auto arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, is_column_nullable);
|
||||
*is_column_nullable = true;
|
||||
return arrow_type;
|
||||
@ -506,7 +514,7 @@ namespace DB
|
||||
|| std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>>
|
||||
|| std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>>)
|
||||
{
|
||||
const auto & decimal_type = static_cast<const ToDataType *>(column_type.get());
|
||||
const auto & decimal_type = assert_cast<const ToDataType *>(column_type.get());
|
||||
arrow_type = arrow::decimal(decimal_type->getPrecision(), decimal_type->getScale());
|
||||
}
|
||||
|
||||
@ -518,16 +526,16 @@ namespace DB
|
||||
|
||||
if (isArray(column_type))
|
||||
{
|
||||
auto nested_type = typeid_cast<const DataTypeArray *>(column_type.get())->getNestedType();
|
||||
auto nested_column = checkAndGetColumn<ColumnArray>(*column)->getDataPtr();
|
||||
auto nested_type = assert_cast<const DataTypeArray *>(column_type.get())->getNestedType();
|
||||
auto nested_column = assert_cast<const ColumnArray *>(column.get())->getDataPtr();
|
||||
auto nested_arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, is_column_nullable);
|
||||
return arrow::list(nested_arrow_type);
|
||||
}
|
||||
|
||||
if (isTuple(column_type))
|
||||
{
|
||||
const auto & nested_types = typeid_cast<const DataTypeTuple *>(column_type.get())->getElements();
|
||||
const auto * tuple_column = checkAndGetColumn<ColumnTuple>(*column);
|
||||
const auto & nested_types = assert_cast<const DataTypeTuple *>(column_type.get())->getElements();
|
||||
const auto * tuple_column = assert_cast<const ColumnTuple *>(column.get());
|
||||
std::vector<std::shared_ptr<arrow::Field>> nested_fields;
|
||||
for (size_t i = 0; i != nested_types.size(); ++i)
|
||||
{
|
||||
@ -540,15 +548,28 @@ namespace DB
|
||||
|
||||
if (column_type->lowCardinality())
|
||||
{
|
||||
auto nested_type = typeid_cast<const DataTypeLowCardinality *>(column_type.get())->getDictionaryType();
|
||||
const auto * lc_column = checkAndGetColumn<ColumnLowCardinality>(*column);
|
||||
auto nested_column = lc_column->getDictionaryPtr();
|
||||
auto indexes_column = lc_column->getIndexesPtr();
|
||||
auto nested_type = assert_cast<const DataTypeLowCardinality *>(column_type.get())->getDictionaryType();
|
||||
const auto * lc_column = assert_cast<const ColumnLowCardinality *>(column.get());
|
||||
ColumnPtr nested_column = lc_column->getDictionaryPtr();
|
||||
ColumnPtr indexes_column = lc_column->getIndexesPtr();
|
||||
return arrow::dictionary(
|
||||
getArrowTypeForLowCardinalityIndexes(indexes_column),
|
||||
getArrowType(nested_type, nested_column, column_name, format_name, is_column_nullable));
|
||||
}
|
||||
|
||||
if (isMap(column_type))
|
||||
{
|
||||
const auto * map_type = assert_cast<const DataTypeMap *>(column_type.get());
|
||||
const auto & key_type = map_type->getKeyType();
|
||||
const auto & val_type = map_type->getValueType();
|
||||
|
||||
const auto & columns = assert_cast<const ColumnMap *>(column.get())->getNestedData().getColumns();
|
||||
return arrow::map(
|
||||
getArrowType(key_type, columns[0], column_name, format_name, is_column_nullable),
|
||||
getArrowType(val_type, columns[1], column_name, format_name, is_column_nullable)
|
||||
);
|
||||
}
|
||||
|
||||
const std::string type_name = column_type->getFamilyName();
|
||||
if (const auto * arrow_type_it = std::find_if(
|
||||
internal_type_to_arrow_type.begin(),
|
||||
@ -559,7 +580,7 @@ namespace DB
|
||||
return arrow_type_it->second;
|
||||
}
|
||||
|
||||
throw Exception{"The type \"" + column_name + "\" of a column \"" + column_name + "\""
|
||||
throw Exception{"The type \"" + column_type->getName() + "\" of a column \"" + column_name + "\""
|
||||
" is not supported for conversion into a " + format_name + " data format",
|
||||
ErrorCodes::UNKNOWN_TYPE};
|
||||
}
|
||||
|
@ -72,15 +72,25 @@ static size_t countIndicesForType(std::shared_ptr<arrow::DataType> type)
|
||||
if (type->id() == arrow::Type::LIST)
|
||||
return countIndicesForType(static_cast<arrow::ListType *>(type.get())->value_type()) + 1;
|
||||
|
||||
int indices = 1;
|
||||
if (type->id() == arrow::Type::STRUCT)
|
||||
{
|
||||
int indices = 1;
|
||||
auto * struct_type = static_cast<arrow::StructType *>(type.get());
|
||||
for (int i = 0; i != struct_type->num_fields(); ++i)
|
||||
indices += countIndicesForType(struct_type->field(i)->type());
|
||||
return indices;
|
||||
}
|
||||
|
||||
return indices;
|
||||
if (type->id() == arrow::Type::MAP)
|
||||
{
|
||||
int indices = 0;
|
||||
auto * map_type = static_cast<arrow::MapType *>(type.get());
|
||||
indices += countIndicesForType(map_type->key_type());
|
||||
indices += countIndicesForType(map_type->item_type());
|
||||
return indices;
|
||||
}
|
||||
|
||||
return 1;
|
||||
}
|
||||
|
||||
void ORCBlockInputFormat::prepareReader()
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
@ -18,6 +19,7 @@
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -102,27 +104,27 @@ ORC_UNIQUE_PTR<orc::Type> ORCBlockOutputFormat::getORCType(const DataTypePtr & t
|
||||
}
|
||||
case TypeIndex::Array:
|
||||
{
|
||||
const auto * array_type = typeid_cast<const DataTypeArray *>(type.get());
|
||||
const auto * array_type = assert_cast<const DataTypeArray *>(type.get());
|
||||
return orc::createListType(getORCType(array_type->getNestedType(), column_name));
|
||||
}
|
||||
case TypeIndex::Decimal32:
|
||||
{
|
||||
const auto * decimal_type = typeid_cast<const DataTypeDecimal<Decimal32> *>(type.get());
|
||||
const auto * decimal_type = assert_cast<const DataTypeDecimal<Decimal32> *>(type.get());
|
||||
return orc::createDecimalType(decimal_type->getPrecision(), decimal_type->getScale());
|
||||
}
|
||||
case TypeIndex::Decimal64:
|
||||
{
|
||||
const auto * decimal_type = typeid_cast<const DataTypeDecimal<Decimal64> *>(type.get());
|
||||
const auto * decimal_type = assert_cast<const DataTypeDecimal<Decimal64> *>(type.get());
|
||||
return orc::createDecimalType(decimal_type->getPrecision(), decimal_type->getScale());
|
||||
}
|
||||
case TypeIndex::Decimal128:
|
||||
{
|
||||
const auto * decimal_type = typeid_cast<const DataTypeDecimal<Decimal128> *>(type.get());
|
||||
const auto * decimal_type = assert_cast<const DataTypeDecimal<Decimal128> *>(type.get());
|
||||
return orc::createDecimalType(decimal_type->getPrecision(), decimal_type->getScale());
|
||||
}
|
||||
case TypeIndex::Tuple:
|
||||
{
|
||||
const auto * tuple_type = typeid_cast<const DataTypeTuple *>(type.get());
|
||||
const auto * tuple_type = assert_cast<const DataTypeTuple *>(type.get());
|
||||
const auto & nested_types = tuple_type->getElements();
|
||||
auto struct_type = orc::createStructType();
|
||||
for (size_t i = 0; i < nested_types.size(); ++i)
|
||||
@ -132,6 +134,14 @@ ORC_UNIQUE_PTR<orc::Type> ORCBlockOutputFormat::getORCType(const DataTypePtr & t
|
||||
}
|
||||
return struct_type;
|
||||
}
|
||||
case TypeIndex::Map:
|
||||
{
|
||||
const auto * map_type = assert_cast<const DataTypeMap *>(type.get());
|
||||
return orc::createMapType(
|
||||
getORCType(map_type->getKeyType(), column_name),
|
||||
getORCType(map_type->getValueType(), column_name)
|
||||
);
|
||||
}
|
||||
default:
|
||||
{
|
||||
throw Exception("Type " + type->getName() + " is not supported for ORC output format", ErrorCodes::ILLEGAL_COLUMN);
|
||||
@ -174,7 +184,7 @@ void ORCBlockOutputFormat::writeDecimals(
|
||||
{
|
||||
DecimalVectorBatch & decimal_orc_column = dynamic_cast<DecimalVectorBatch &>(orc_column);
|
||||
const auto & decimal_column = assert_cast<const ColumnDecimal<Decimal> &>(column);
|
||||
const auto * decimal_type = typeid_cast<const DataTypeDecimal<Decimal> *>(type.get());
|
||||
const auto * decimal_type = assert_cast<const DataTypeDecimal<Decimal> *>(type.get());
|
||||
decimal_orc_column.precision = decimal_type->getPrecision();
|
||||
decimal_orc_column.scale = decimal_type->getScale();
|
||||
decimal_orc_column.resize(decimal_column.size());
|
||||
@ -400,13 +410,40 @@ void ORCBlockOutputFormat::writeColumn(
|
||||
{
|
||||
orc::StructVectorBatch & struct_orc_column = dynamic_cast<orc::StructVectorBatch &>(orc_column);
|
||||
const auto & tuple_column = assert_cast<const ColumnTuple &>(column);
|
||||
auto nested_types = typeid_cast<const DataTypeTuple *>(type.get())->getElements();
|
||||
auto nested_types = assert_cast<const DataTypeTuple *>(type.get())->getElements();
|
||||
for (size_t i = 0; i != tuple_column.size(); ++i)
|
||||
struct_orc_column.notNull[i] = 1;
|
||||
for (size_t i = 0; i != tuple_column.tupleSize(); ++i)
|
||||
writeColumn(*struct_orc_column.fields[i], tuple_column.getColumn(i), nested_types[i], null_bytemap);
|
||||
break;
|
||||
}
|
||||
case TypeIndex::Map:
|
||||
{
|
||||
orc::MapVectorBatch & map_orc_column = dynamic_cast<orc::MapVectorBatch &>(orc_column);
|
||||
const auto & list_column = assert_cast<const ColumnMap &>(column).getNestedColumn();
|
||||
const auto & map_type = assert_cast<const DataTypeMap &>(*type);
|
||||
const ColumnArray::Offsets & offsets = list_column.getOffsets();
|
||||
|
||||
map_orc_column.resize(list_column.size());
|
||||
/// The length of list i in ListVectorBatch is offsets[i+1] - offsets[i].
|
||||
map_orc_column.offsets[0] = 0;
|
||||
for (size_t i = 0; i != list_column.size(); ++i)
|
||||
{
|
||||
map_orc_column.offsets[i + 1] = offsets[i];
|
||||
map_orc_column.notNull[i] = 1;
|
||||
}
|
||||
const auto nested_columns = assert_cast<const ColumnTuple *>(list_column.getDataPtr().get())->getColumns();
|
||||
|
||||
orc::ColumnVectorBatch & keys_orc_column = *map_orc_column.keys;
|
||||
auto key_type = map_type.getKeyType();
|
||||
writeColumn(keys_orc_column, *nested_columns[0], key_type, null_bytemap);
|
||||
|
||||
orc::ColumnVectorBatch & values_orc_column = *map_orc_column.elements;
|
||||
auto value_type = map_type.getValueType();
|
||||
writeColumn(values_orc_column, *nested_columns[1], value_type, null_bytemap);
|
||||
|
||||
map_orc_column.numElements = list_column.size();
|
||||
break;
|
||||
}
|
||||
default:
|
||||
throw Exception("Type " + type->getName() + " is not supported for ORC output format", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
@ -79,6 +79,12 @@ static size_t countIndicesForType(std::shared_ptr<arrow::DataType> type)
|
||||
for (int i = 0; i != struct_type->num_fields(); ++i)
|
||||
indices += countIndicesForType(struct_type->field(i)->type());
|
||||
}
|
||||
else if (type->id() == arrow::Type::MAP)
|
||||
{
|
||||
auto * map_type = static_cast<arrow::MapType *>(type.get());
|
||||
indices += countIndicesForType(map_type->key_type());
|
||||
indices += countIndicesForType(map_type->item_type());
|
||||
}
|
||||
else
|
||||
indices = 1;
|
||||
|
||||
|
@ -298,24 +298,40 @@ Code: 33. DB::ParsingEx---tion: Error while reading Parquet data: IOError: Not y
|
||||
[[['a','b'],['c','d']],[[],['e']]] 1
|
||||
[[['a','b'],['c','d'],['e']],[[],['f']]] 1
|
||||
=== Try load data from nested_maps.snappy.parquet
|
||||
Code: 70. DB::Ex---tion: The type "map" of an input column "a" is not supported for conversion from a Parquet data format: data for INSERT was parsed from stdin
|
||||
|
||||
{'a':{1:1,2:0}} 1 1
|
||||
{'b':{1:1}} 1 1
|
||||
{'c':{}} 1 1
|
||||
{'d':{}} 1 1
|
||||
{'e':{1:1}} 1 1
|
||||
{'f':{3:1,4:0,5:1}} 1 1
|
||||
=== Try load data from non_hadoop_lz4_compressed.parquet
|
||||
1593604800 abc 42
|
||||
1593604800 def 7.7
|
||||
1593604801 abc 42.125
|
||||
1593604801 def 7.7
|
||||
=== Try load data from nonnullable.impala.parquet
|
||||
../contrib/arrow/cpp/src/arrow/array/array_nested.cc:192: Check failed: (self->list_type_->value_type()->id()) == (data->child_data[0]->type->id())
|
||||
8 [-1] [[-1,-2],[]] {'k1':-1} [{},{'k1':1},{},{}] (-1,[-1],([[(-1)]]),{})
|
||||
=== Try load data from nullable.impala.parquet
|
||||
../contrib/arrow/cpp/src/arrow/array/array_nested.cc:192: Check failed: (self->list_type_->value_type()->id()) == (data->child_data[0]->type->id())
|
||||
1 [1,2,3] [[1,2],[3,4]] {'k1':1,'k2':100} [{'k1':1}] (1,[1],([[(10),(-10)],[(11)]]),{'foo':(([1.1]))})
|
||||
2 [NULL,1,2,NULL,3,NULL] [[NULL,1,2,NULL],[3,NULL,4],[],[]] {'k1':2,'k2':NULL} [{'k3':NULL,'k1':1},{},{}] (NULL,[NULL],([[(NULL),(10),(NULL),(-10),(NULL)],[(11),(NULL)],[],[]]),{'g1':(([2.2,NULL])),'g2':(([])),'g3':(([])),'g4':(([])),'g5':(([]))})
|
||||
3 [] [[]] {} [{},{}] (NULL,[],([]),{})
|
||||
4 [] [] {} [] (NULL,[],([]),{})
|
||||
5 [] [] {} [] (NULL,[],([]),{'foo':(([2.2,3.3]))})
|
||||
6 [] [] {} [] (NULL,[],([]),{})
|
||||
7 [] [[],[5,6]] {'k1':NULL,'k3':NULL} [] (7,[2,3,NULL],([[],[(NULL)],[]]),{})
|
||||
=== Try load data from nullable_list.parquet
|
||||
[1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42]
|
||||
[NULL] [NULL] [NULL]
|
||||
[] [] []
|
||||
=== Try load data from nulls.snappy.parquet
|
||||
Code: 70. DB::Ex---tion: The type "struct" of an input column "b_struct" is not supported for conversion from a Parquet data format: data for INSERT was parsed from stdin
|
||||
|
||||
(NULL)
|
||||
(NULL)
|
||||
(NULL)
|
||||
(NULL)
|
||||
(NULL)
|
||||
(NULL)
|
||||
(NULL)
|
||||
(NULL)
|
||||
=== Try load data from single_nan.parquet
|
||||
\N
|
||||
=== Try load data from userdata1.parquet
|
||||
|
@ -55,7 +55,10 @@ for NAME in $(find "$DATA_DIR"/*.parquet -print0 | xargs -0 -n 1 basename | LC_A
|
||||
COLUMNS=$(cat "$COLUMNS_FILE") || continue
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_load ($COLUMNS) ENGINE = Memory"
|
||||
$CLICKHOUSE_CLIENT --multiquery <<EOF
|
||||
SET allow_experimental_map_type = 1;
|
||||
CREATE TABLE parquet_load ($COLUMNS) ENGINE = Memory;
|
||||
EOF
|
||||
|
||||
# Some files contain unsupported data structures, exception is ok.
|
||||
cat "$DATA_DIR"/"$NAME" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_load FORMAT Parquet" 2>&1 | sed 's/Exception/Ex---tion/'
|
||||
|
@ -0,0 +1,6 @@
|
||||
Arrow
|
||||
{1:2,2:3} {'1':'a','2':'b'} {1:(1,2),2:(3,4)} {1:[1,2],2:[3,4]} [{1:2,2:3},{3:4,4:5}] ({1:2,2:3},{'a':'b','c':'d'}) [{1:[({1:2},(1)),({2:3},(2))]},{2:[({3:4},(3)),({4:5},(4))]}]
|
||||
Parquet
|
||||
{1:2,2:3} {'1':'a','2':'b'} {1:(1,2),2:(3,4)} {1:[1,2],2:[3,4]} [{1:2,2:3},{3:4,4:5}] ({1:2,2:3},{'a':'b','c':'d'}) [{1:[({1:2},(1)),({2:3},(2))]},{2:[({3:4},(3)),({4:5},(4))]}]
|
||||
ORC
|
||||
{1:2,2:3} {'1':'a','2':'b'} {1:(1,2),2:(3,4)} {1:[1,2],2:[3,4]} [{1:2,2:3},{3:4,4:5}] ({1:2,2:3},{'a':'b','c':'d'}) [{1:[({1:2},(1)),({2:3},(2))]},{2:[({3:4},(3)),({4:5},(4))]}]
|
28
tests/queries/0_stateless/00900_orc_arrow_parquet_maps.sh
Executable file
28
tests/queries/0_stateless/00900_orc_arrow_parquet_maps.sh
Executable file
@ -0,0 +1,28 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS maps"
|
||||
${CLICKHOUSE_CLIENT} --multiquery <<EOF
|
||||
SET allow_experimental_map_type = 1;
|
||||
CREATE TABLE maps (m1 Map(UInt32, UInt32), m2 Map(String, String), m3 Map(UInt32, Tuple(UInt32, UInt32)), m4 Map(UInt32, Array(UInt32)), m5 Array(Map(UInt32, UInt32)), m6 Tuple(Map(UInt32, UInt32), Map(String, String)), m7 Array(Map(UInt32, Array(Tuple(Map(UInt32, UInt32), Tuple(UInt32)))))) ENGINE=Memory();
|
||||
EOF
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO maps VALUES ({1 : 2, 2 : 3}, {'1' : 'a', '2' : 'b'}, {1 : (1, 2), 2 : (3, 4)}, {1 : [1, 2], 2 : [3, 4]}, [{1 : 2, 2 : 3}, {3 : 4, 4 : 5}], ({1 : 2, 2 : 3}, {'a' : 'b', 'c' : 'd'}), [{1 : [({1 : 2}, (1)), ({2 : 3}, (2))]}, {2 : [({3 : 4}, (3)), ({4 : 5}, (4))]}])"
|
||||
|
||||
|
||||
formats="Arrow Parquet ORC";
|
||||
|
||||
|
||||
for format in ${formats}; do
|
||||
echo $format
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM maps FORMAT Parquet" > "${CLICKHOUSE_TMP}"/maps
|
||||
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE maps"
|
||||
cat "${CLICKHOUSE_TMP}"/maps | ${CLICKHOUSE_CLIENT} -q "INSERT INTO maps FORMAT Parquet"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM maps"
|
||||
done
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE maps"
|
@ -1,2 +1,6 @@
|
||||
Arrow
|
||||
(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))]
|
||||
Parquet
|
||||
(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))]
|
||||
ORC
|
||||
(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))]
|
24
tests/queries/0_stateless/00900_orc_arrow_parquet_tuples.sh
Executable file
24
tests/queries/0_stateless/00900_orc_arrow_parquet_tuples.sh
Executable file
@ -0,0 +1,24 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS tuples";
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE tuples (t1 Tuple(UInt32, UInt32), t2 Tuple(String, String), t3 Tuple(Tuple(UInt32, String), UInt32), t4 Tuple(Tuple(UInt32, UInt32), Tuple(String, String)), t5 Tuple(Array(UInt32), UInt32), t6 Tuple(Tuple(Array(UInt32), Array(UInt32)), Tuple(Array(Array(UInt32)), UInt32)), t7 Array(Tuple(Array(Array(UInt32)), Tuple(Array(Tuple(UInt32, UInt32)), UInt32)))) ENGINE=Memory()"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO tuples VALUES ((1, 2), ('1', '2'), ((1, '1'), 1), ((1, 2), ('1', '2')), ([1,2,3], 1), (([1,2,3], [1,2,3]), ([[1,2,3], [1,2,3]], 1)), [([[1,2,3], [1,2,3]], ([(1, 2), (1, 2)], 1))])"
|
||||
|
||||
formats="Arrow Parquet ORC";
|
||||
|
||||
for format in ${formats}; do
|
||||
echo $format
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM tuples FORMAT $format" > "${CLICKHOUSE_TMP}"/tuples
|
||||
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE tuples"
|
||||
cat "${CLICKHOUSE_TMP}"/tuples | ${CLICKHOUSE_CLIENT} -q "INSERT INTO tuples FORMAT $format"
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM tuples"
|
||||
done
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE tuples"
|
20
tests/queries/0_stateless/00900_orc_map_load.sh
Normal file
20
tests/queries/0_stateless/00900_orc_map_load.sh
Normal file
@ -0,0 +1,20 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_maps"
|
||||
${CLICKHOUSE_CLIENT} --multiquery <<EOF
|
||||
SET allow_experimental_map_type = 1;
|
||||
CREATE TABLE parquet_maps (m1 Map(UInt32, UInt32), m2 Map(String, String), m3 Map(UInt32, Tuple(UInt32, UInt32)), m4 Map(UInt32, Array(UInt32)), m5 Array(Map(UInt32, UInt32)), m6 Tuple(Map(UInt32, UInt32), Map(String, String)), m7 Array(Map(UInt32, Array(Tuple(Map(UInt32, UInt32), Tuple(UInt32)))))) ENGINE=Memory();
|
||||
EOF
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_maps VALUES ({1 : 2, 2 : 3}, {'1' : 'a', '2' : 'b'}, {1 : (1, 2), 2 : (3, 4)}, {1 : [1, 2], 2 : [3, 4]}, [{1 : 2, 2 : 3}, {3 : 4, 4 : 5}], ({1 : 2, 2 : 3}, {'a' : 'b', 'c' : 'd'}), [{1 : [({1 : 2}, (1)), ({2 : 3}, (2))]}, {2 : [({3 : 4}, (3)), ({4 : 5}, (4))]}])"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_maps FORMAT Parquet" > "${CLICKHOUSE_TMP}"/maps.parquet
|
||||
|
||||
cat "${CLICKHOUSE_TMP}"/maps.parquet | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_maps FORMAT Parquet"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_maps"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_maps"
|
@ -1,17 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_tuples"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_tuples (t1 Tuple(UInt32, UInt32), t2 Tuple(String, String), t3 Tuple(Tuple(UInt32, String), UInt32), t4 Tuple(Tuple(UInt32, UInt32), Tuple(String, String)), t5 Tuple(Array(UInt32), UInt32), t6 Tuple(Tuple(Array(UInt32), Array(UInt32)), Tuple(Array(Array(UInt32)), UInt32)), t7 Array(Tuple(Array(Array(UInt32)), Tuple(Array(Tuple(UInt32, UInt32)), UInt32)))) ENGINE=Memory()"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_tuples VALUES ((1, 2), ('1', '2'), ((1, '1'), 1), ((1, 2), ('1', '2')), ([1,2,3], 1), (([1,2,3], [1,2,3]), ([[1,2,3], [1,2,3]], 1)), [([[1,2,3], [1,2,3]], ([(1, 2), (1, 2)], 1))])"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_tuples FORMAT ORC" > "${CLICKHOUSE_TMP}"/tuples.orc
|
||||
|
||||
cat "${CLICKHOUSE_TMP}"/tuples.orc | ${CLICKHOUSE_CLIENT} -q "INSERT INTO orc_tuples FORMAT ORC"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_tuples"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE orc_tuples"
|
@ -1,2 +0,0 @@
|
||||
(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))]
|
||||
(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))]
|
@ -1,17 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_tuples"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_tuples (t1 Tuple(UInt32, UInt32), t2 Tuple(String, String), t3 Tuple(Tuple(UInt32, String), UInt32), t4 Tuple(Tuple(UInt32, UInt32), Tuple(String, String)), t5 Tuple(Array(UInt32), UInt32), t6 Tuple(Tuple(Array(UInt32), Array(UInt32)), Tuple(Array(Array(UInt32)), UInt32)), t7 Array(Tuple(Array(Array(UInt32)), Tuple(Array(Tuple(UInt32, UInt32)), UInt32)))) ENGINE=Memory()"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_tuples VALUES ((1, 2), ('1', '2'), ((1, '1'), 1), ((1, 2), ('1', '2')), ([1,2,3], 1), (([1,2,3], [1,2,3]), ([[1,2,3], [1,2,3]], 1)), [([[1,2,3], [1,2,3]], ([(1, 2), (1, 2)], 1))])"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_tuples FORMAT Parquet" > "${CLICKHOUSE_TMP}"/tuples.parquet
|
||||
|
||||
cat "${CLICKHOUSE_TMP}"/tuples.parquet | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_tuples FORMAT Parquet"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_tuples"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_tuples"
|
@ -1,2 +0,0 @@
|
||||
(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))]
|
||||
(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))]
|
@ -1,17 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_tuples"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_tuples (t1 Tuple(UInt32, UInt32), t2 Tuple(String, String), t3 Tuple(Tuple(UInt32, String), UInt32), t4 Tuple(Tuple(UInt32, UInt32), Tuple(String, String)), t5 Tuple(Array(UInt32), UInt32), t6 Tuple(Tuple(Array(UInt32), Array(UInt32)), Tuple(Array(Array(UInt32)), UInt32)), t7 Array(Tuple(Array(Array(UInt32)), Tuple(Array(Tuple(UInt32, UInt32)), UInt32)))) ENGINE=Memory()"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_tuples VALUES ((1, 2), ('1', '2'), ((1, '1'), 1), ((1, 2), ('1', '2')), ([1,2,3], 1), (([1,2,3], [1,2,3]), ([[1,2,3], [1,2,3]], 1)), [([[1,2,3], [1,2,3]], ([(1, 2), (1, 2)], 1))])"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_tuples FORMAT Arrow" > "${CLICKHOUSE_TMP}"/tuples.arrow
|
||||
|
||||
cat "${CLICKHOUSE_TMP}"/tuples.arrow | ${CLICKHOUSE_CLIENT} -q "INSERT INTO arrow_tuples FORMAT Arrow"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_tuples"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_tuples"
|
@ -1 +1 @@
|
||||
`a` Tuple(Nullable(String), Nullable(Int32), Nullable(UInt8)), `b` Nullable(Int32), `c` Nullable(Float64)
|
||||
`a` Map(String, Map(Int32, Nullable(UInt8))), `b` Nullable(Int32), `c` Nullable(Float64)
|
||||
|
@ -1 +1 @@
|
||||
`ID` Nullable(Int64), `Int_Array` Nullable(Int32), `int_array_array` Nullable(Int32), `Int_Map` Tuple(Nullable(String), Nullable(Int32)), `int_map_array` Tuple(Nullable(String), Nullable(Int32)), `nested_Struct` Tuple(Nullable(Int32), Nullable(Int32), Nullable(Int32), Nullable(String), Nullable(String), Nullable(Float64))
|
||||
`ID` Nullable(Int64), `Int_Array` Array(Nullable(Int32)), `int_array_array` Array(Array(Nullable(Int32))), `Int_Map` Map(String, Nullable(Int32)), `int_map_array` Array(Map(String, Nullable(Int32))), `nested_Struct` Tuple(Nullable(Int32), Array(Nullable(Int32)), Tuple(Array(Array(Tuple(Nullable(Int32))))), Map(String, Tuple(Tuple(Array(Nullable(Float64))))))
|
||||
|
@ -1 +1 @@
|
||||
`id` Nullable(Int64), `int_array` Nullable(Int32), `int_array_Array` Nullable(Int32), `int_map` Tuple(Nullable(String), Nullable(Int32)), `int_Map_Array` Tuple(Nullable(String), Nullable(Int32)), `nested_struct` Tuple(Nullable(Int32), Nullable(Int32), Nullable(Int32), Nullable(String), Nullable(String), Nullable(Float64))
|
||||
`id` Nullable(Int64), `int_array` Array(Nullable(Int32)), `int_array_Array` Array(Array(Nullable(Int32))), `int_map` Map(String, Nullable(Int32)), `int_Map_Array` Array(Map(String, Nullable(Int32))), `nested_struct` Tuple(Nullable(Int32), Array(Nullable(Int32)), Tuple(Array(Array(Tuple(Nullable(Int32))))), Map(String, Tuple(Tuple(Array(Nullable(Float64))))))
|
||||
|
@ -1 +1 @@
|
||||
`b_struct` Nullable(Int32)
|
||||
`b_struct` Tuple(Nullable(Int32))
|
||||
|
Loading…
Reference in New Issue
Block a user