2021-01-23 13:18:24 +00:00
|
|
|
|
#pragma once
|
|
|
|
|
|
2021-04-02 20:16:04 +00:00
|
|
|
|
#include <Common/HashTable/HashMap.h>
|
2021-01-23 13:18:24 +00:00
|
|
|
|
#include <Columns/IColumn.h>
|
|
|
|
|
#include <Columns/ColumnDecimal.h>
|
|
|
|
|
#include <Columns/ColumnString.h>
|
|
|
|
|
#include <Columns/ColumnVector.h>
|
2021-06-09 10:43:40 +00:00
|
|
|
|
#include <Columns/ColumnArray.h>
|
2021-06-12 10:53:03 +00:00
|
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
|
#include <Columns/ColumnNullable.h>
|
2021-04-17 02:25:22 +00:00
|
|
|
|
#include <Columns/ColumnSparse.h>
|
2021-01-23 13:18:24 +00:00
|
|
|
|
#include <DataTypes/DataTypesDecimal.h>
|
2021-06-09 10:43:40 +00:00
|
|
|
|
#include <DataTypes/DataTypeArray.h>
|
2021-06-09 20:51:08 +00:00
|
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
2021-04-02 20:16:04 +00:00
|
|
|
|
#include <Core/Block.h>
|
2021-02-16 21:33:02 +00:00
|
|
|
|
#include <Dictionaries/IDictionary.h>
|
|
|
|
|
#include <Dictionaries/DictionaryStructure.h>
|
2021-08-04 17:58:18 +00:00
|
|
|
|
#include <Processors/Executors/PullingPipelineExecutor.h>
|
2021-10-16 14:03:50 +00:00
|
|
|
|
#include <QueryPipeline/QueryPipelineBuilder.h>
|
2021-07-17 18:06:46 +00:00
|
|
|
|
|
2021-01-23 13:18:24 +00:00
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
2021-01-23 16:47:33 +00:00
|
|
|
|
namespace ErrorCodes
|
|
|
|
|
{
|
|
|
|
|
extern const int TYPE_MISMATCH;
|
2021-02-26 16:04:43 +00:00
|
|
|
|
extern const int BAD_ARGUMENTS;
|
2021-01-23 16:47:33 +00:00
|
|
|
|
}
|
|
|
|
|
|
2023-04-12 13:36:23 +00:00
|
|
|
|
class Arena;
|
|
|
|
|
|
2021-03-04 14:34:39 +00:00
|
|
|
|
/** Simple helper for getting default.
|
|
|
|
|
* Initialized with default value and default values column.
|
|
|
|
|
* If default values column is not null default value is taken from column.
|
|
|
|
|
* If default value is null default value is taken from initializer.
|
|
|
|
|
*/
|
2021-03-03 18:58:43 +00:00
|
|
|
|
class DefaultValueProvider final
|
|
|
|
|
{
|
|
|
|
|
public:
|
|
|
|
|
explicit DefaultValueProvider(Field default_value_, ColumnPtr default_values_column_ = nullptr)
|
|
|
|
|
: default_value(std::move(default_value_))
|
|
|
|
|
, default_values_column(default_values_column_)
|
|
|
|
|
{
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
inline bool isConstant() const { return default_values_column == nullptr; }
|
|
|
|
|
|
|
|
|
|
Field getDefaultValue(size_t row) const
|
|
|
|
|
{
|
|
|
|
|
if (default_values_column)
|
|
|
|
|
return (*default_values_column)[row];
|
|
|
|
|
|
|
|
|
|
return default_value;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private:
|
|
|
|
|
Field default_value;
|
|
|
|
|
ColumnPtr default_values_column;
|
|
|
|
|
};
|
|
|
|
|
|
2021-02-17 11:48:06 +00:00
|
|
|
|
/** Support class for dictionary storages.
|
|
|
|
|
|
|
|
|
|
The main idea is that during fetch we create all columns, but fill only columns that client requested.
|
|
|
|
|
|
|
|
|
|
We need to create other columns during fetch, because in case of serialized storage we can skip
|
|
|
|
|
unnecessary columns serialized in cache with skipSerializedInArena method.
|
|
|
|
|
|
|
|
|
|
When result is fetched from the storage client of storage can filterOnlyNecessaryColumns
|
|
|
|
|
and get only columns that match attributes_names_to_fetch.
|
|
|
|
|
*/
|
|
|
|
|
class DictionaryStorageFetchRequest
|
|
|
|
|
{
|
|
|
|
|
public:
|
2023-01-24 15:32:45 +00:00
|
|
|
|
DictionaryStorageFetchRequest(const DictionaryStructure & structure,
|
|
|
|
|
const Strings & attributes_to_fetch_names,
|
|
|
|
|
const DataTypes & attributes_to_fetch_types,
|
|
|
|
|
const Columns & attributes_to_fetch_default_values_columns)
|
|
|
|
|
: attributes_to_fetch_filter(structure.attributes.size(), false)
|
2021-02-17 11:48:06 +00:00
|
|
|
|
{
|
2023-01-24 15:32:45 +00:00
|
|
|
|
size_t attributes_to_fetch_size = attributes_to_fetch_names.size();
|
2021-03-03 18:58:43 +00:00
|
|
|
|
|
2023-01-24 15:32:45 +00:00
|
|
|
|
assert(attributes_to_fetch_size == attributes_to_fetch_types.size());
|
|
|
|
|
assert(attributes_to_fetch_size == attributes_to_fetch_default_values_columns.size());
|
|
|
|
|
|
|
|
|
|
for (size_t i = 0; i < attributes_to_fetch_size; ++i)
|
|
|
|
|
attributes_to_fetch_name_to_index.emplace(attributes_to_fetch_names[i], i);
|
|
|
|
|
|
|
|
|
|
if (attributes_to_fetch_name_to_index.size() != attributes_to_fetch_name_to_index.size())
|
2021-02-26 15:56:41 +00:00
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Attribute names to fetch should be unique");
|
|
|
|
|
|
2021-02-17 11:48:06 +00:00
|
|
|
|
size_t attributes_size = structure.attributes.size();
|
2023-01-24 15:32:45 +00:00
|
|
|
|
dictionary_attributes_names_and_types.reserve(attributes_size);
|
|
|
|
|
attributes_default_value_providers.reserve(attributes_size);
|
2021-02-17 11:48:06 +00:00
|
|
|
|
|
2023-01-24 15:32:45 +00:00
|
|
|
|
for (size_t attribute_index = 0; attribute_index < attributes_size; ++attribute_index)
|
2021-02-17 11:48:06 +00:00
|
|
|
|
{
|
2023-01-24 15:32:45 +00:00
|
|
|
|
const auto & dictionary_attribute = structure.attributes[attribute_index];
|
|
|
|
|
dictionary_attributes_names_and_types.emplace_back(dictionary_attribute.name, dictionary_attribute.type);
|
2021-02-17 11:48:06 +00:00
|
|
|
|
|
2023-01-24 15:32:45 +00:00
|
|
|
|
auto attribute_to_fetch_index_it = attributes_to_fetch_name_to_index.find(dictionary_attribute.name);
|
|
|
|
|
if (attribute_to_fetch_index_it == attributes_to_fetch_name_to_index.end())
|
2021-02-17 11:48:06 +00:00
|
|
|
|
{
|
2023-01-24 15:32:45 +00:00
|
|
|
|
attributes_default_value_providers.emplace_back(dictionary_attribute.null_value);
|
|
|
|
|
continue;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
attributes_to_fetch_filter[attribute_index] = true;
|
|
|
|
|
|
|
|
|
|
size_t attributes_to_fetch_index = attribute_to_fetch_index_it->second;
|
|
|
|
|
const auto & attribute_to_fetch_result_type = attributes_to_fetch_types[attributes_to_fetch_index];
|
|
|
|
|
const auto & attribute_to_fetch_default_value_column = attributes_to_fetch_default_values_columns[attributes_to_fetch_index];
|
2021-04-04 22:26:26 +00:00
|
|
|
|
|
2023-01-24 15:32:45 +00:00
|
|
|
|
if (!attribute_to_fetch_result_type->equals(*dictionary_attribute.type))
|
|
|
|
|
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
|
|
|
|
"Attribute {} type does not match, expected {}, found {}",
|
|
|
|
|
dictionary_attribute.name,
|
2021-04-04 22:26:26 +00:00
|
|
|
|
attribute_to_fetch_result_type->getName(),
|
2023-01-24 15:32:45 +00:00
|
|
|
|
dictionary_attribute.type->getName());
|
2021-04-04 22:26:26 +00:00
|
|
|
|
|
2023-01-24 15:32:45 +00:00
|
|
|
|
attributes_default_value_providers.emplace_back(dictionary_attribute.null_value, attribute_to_fetch_default_value_column);
|
2021-02-17 11:48:06 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
DictionaryStorageFetchRequest() = default;
|
|
|
|
|
|
|
|
|
|
/// Check requested attributes size
|
2021-02-26 15:56:41 +00:00
|
|
|
|
ALWAYS_INLINE size_t attributesSize() const
|
2021-02-17 11:48:06 +00:00
|
|
|
|
{
|
2023-01-24 15:32:45 +00:00
|
|
|
|
return dictionary_attributes_names_and_types.size();
|
2021-02-17 11:48:06 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Check if attribute with attribute_name was requested to fetch
|
2021-02-26 15:56:41 +00:00
|
|
|
|
ALWAYS_INLINE bool containsAttribute(const String & attribute_name) const
|
2021-02-17 11:48:06 +00:00
|
|
|
|
{
|
2023-01-24 15:32:45 +00:00
|
|
|
|
return attributes_to_fetch_name_to_index.contains(attribute_name);
|
2021-02-17 11:48:06 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Check if attribute with attribute_index should be filled during fetch
|
2021-02-26 15:56:41 +00:00
|
|
|
|
ALWAYS_INLINE bool shouldFillResultColumnWithIndex(size_t attribute_index) const
|
2021-02-17 11:48:06 +00:00
|
|
|
|
{
|
|
|
|
|
return attributes_to_fetch_filter[attribute_index];
|
|
|
|
|
}
|
|
|
|
|
|
2021-02-26 15:56:41 +00:00
|
|
|
|
const DataTypePtr & dataTypeAtIndex(size_t attribute_index) const
|
|
|
|
|
{
|
2023-01-24 15:32:45 +00:00
|
|
|
|
return dictionary_attributes_names_and_types[attribute_index].type;
|
2021-02-26 15:56:41 +00:00
|
|
|
|
}
|
|
|
|
|
|
2021-03-03 18:58:43 +00:00
|
|
|
|
const DefaultValueProvider & defaultValueProviderAtIndex(size_t attribute_index) const
|
|
|
|
|
{
|
|
|
|
|
return attributes_default_value_providers[attribute_index];
|
|
|
|
|
}
|
|
|
|
|
|
2021-02-17 11:48:06 +00:00
|
|
|
|
/// Create columns for each of dictionary attributes
|
|
|
|
|
MutableColumns makeAttributesResultColumns() const
|
|
|
|
|
{
|
|
|
|
|
MutableColumns result;
|
2023-01-24 15:32:45 +00:00
|
|
|
|
result.reserve(dictionary_attributes_names_and_types.size());
|
2021-02-17 11:48:06 +00:00
|
|
|
|
|
2023-01-24 15:32:45 +00:00
|
|
|
|
for (const auto & name_and_type : dictionary_attributes_names_and_types)
|
|
|
|
|
result.emplace_back(name_and_type.type->createColumn());
|
2021-02-17 11:48:06 +00:00
|
|
|
|
|
|
|
|
|
return result;
|
|
|
|
|
}
|
|
|
|
|
|
2021-03-03 18:58:43 +00:00
|
|
|
|
Columns makeAttributesResultColumnsNonMutable() const
|
|
|
|
|
{
|
|
|
|
|
Columns result;
|
2023-01-24 15:32:45 +00:00
|
|
|
|
result.reserve(dictionary_attributes_names_and_types.size());
|
2021-03-03 18:58:43 +00:00
|
|
|
|
|
2023-01-24 15:32:45 +00:00
|
|
|
|
for (const auto & name_and_type : dictionary_attributes_names_and_types)
|
|
|
|
|
result.emplace_back(name_and_type.type->createColumn());
|
2021-03-03 18:58:43 +00:00
|
|
|
|
|
|
|
|
|
return result;
|
|
|
|
|
}
|
|
|
|
|
|
2021-02-17 21:42:51 +00:00
|
|
|
|
/// Filter only requested columns
|
2021-02-17 11:48:06 +00:00
|
|
|
|
Columns filterRequestedColumns(MutableColumns & fetched_mutable_columns) const
|
|
|
|
|
{
|
2023-01-24 15:32:45 +00:00
|
|
|
|
Columns result(attributes_to_fetch_name_to_index.size());
|
|
|
|
|
size_t dictionary_attributes_size = dictionary_attributes_names_and_types.size();
|
2021-02-17 11:48:06 +00:00
|
|
|
|
|
2023-01-24 15:32:45 +00:00
|
|
|
|
for (size_t attribute_index = 0; attribute_index < dictionary_attributes_size; ++attribute_index)
|
|
|
|
|
{
|
|
|
|
|
if (!shouldFillResultColumnWithIndex(attribute_index))
|
|
|
|
|
continue;
|
|
|
|
|
|
|
|
|
|
const auto & dictionary_attribute_name = dictionary_attributes_names_and_types[attribute_index].name;
|
|
|
|
|
size_t fetch_attribute_index = attributes_to_fetch_name_to_index.find(dictionary_attribute_name)->second;
|
|
|
|
|
result[fetch_attribute_index] = std::move(fetched_mutable_columns[attribute_index]);
|
|
|
|
|
}
|
2021-02-17 11:48:06 +00:00
|
|
|
|
|
|
|
|
|
return result;
|
|
|
|
|
}
|
|
|
|
|
private:
|
2023-01-24 15:32:45 +00:00
|
|
|
|
NamesAndTypes dictionary_attributes_names_and_types;
|
|
|
|
|
std::unordered_map<String, size_t> attributes_to_fetch_name_to_index;
|
2021-02-17 11:48:06 +00:00
|
|
|
|
std::vector<bool> attributes_to_fetch_filter;
|
2021-03-03 18:58:43 +00:00
|
|
|
|
std::vector<DefaultValueProvider> attributes_default_value_providers;
|
2021-02-17 11:48:06 +00:00
|
|
|
|
};
|
|
|
|
|
|
2022-03-12 17:46:38 +00:00
|
|
|
|
static inline void insertDefaultValuesIntoColumns( /// NOLINT
|
2021-03-03 18:58:43 +00:00
|
|
|
|
MutableColumns & columns,
|
|
|
|
|
const DictionaryStorageFetchRequest & fetch_request,
|
|
|
|
|
size_t row_index)
|
|
|
|
|
{
|
|
|
|
|
size_t columns_size = columns.size();
|
|
|
|
|
|
|
|
|
|
for (size_t column_index = 0; column_index < columns_size; ++column_index)
|
|
|
|
|
{
|
|
|
|
|
const auto & column = columns[column_index];
|
|
|
|
|
const auto & default_value_provider = fetch_request.defaultValueProviderAtIndex(column_index);
|
|
|
|
|
|
|
|
|
|
if (fetch_request.shouldFillResultColumnWithIndex(column_index))
|
|
|
|
|
column->insert(default_value_provider.getDefaultValue(row_index));
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2021-02-17 18:19:04 +00:00
|
|
|
|
/// Deserialize column value and insert it in columns.
|
|
|
|
|
/// Skip unnecessary columns that were not requested from deserialization.
|
2022-03-12 17:46:38 +00:00
|
|
|
|
static inline void deserializeAndInsertIntoColumns( /// NOLINT
|
2021-02-17 18:19:04 +00:00
|
|
|
|
MutableColumns & columns,
|
|
|
|
|
const DictionaryStorageFetchRequest & fetch_request,
|
|
|
|
|
const char * place_for_serialized_columns)
|
|
|
|
|
{
|
2021-02-27 16:04:32 +00:00
|
|
|
|
size_t columns_size = columns.size();
|
|
|
|
|
|
|
|
|
|
for (size_t column_index = 0; column_index < columns_size; ++column_index)
|
2021-02-17 18:19:04 +00:00
|
|
|
|
{
|
|
|
|
|
const auto & column = columns[column_index];
|
|
|
|
|
|
|
|
|
|
if (fetch_request.shouldFillResultColumnWithIndex(column_index))
|
|
|
|
|
place_for_serialized_columns = column->deserializeAndInsertFromArena(place_for_serialized_columns);
|
|
|
|
|
else
|
|
|
|
|
place_for_serialized_columns = column->skipSerializedInArena(place_for_serialized_columns);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2021-01-23 13:18:24 +00:00
|
|
|
|
/**
|
|
|
|
|
* In Dictionaries implementation String attribute is stored in arena and StringRefs are pointing to it.
|
|
|
|
|
*/
|
|
|
|
|
template <typename DictionaryAttributeType>
|
2021-01-23 16:47:33 +00:00
|
|
|
|
using DictionaryValueType =
|
2021-01-23 13:18:24 +00:00
|
|
|
|
std::conditional_t<std::is_same_v<DictionaryAttributeType, String>, StringRef, DictionaryAttributeType>;
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Used to create column with right type for DictionaryAttributeType.
|
|
|
|
|
*/
|
|
|
|
|
template <typename DictionaryAttributeType>
|
|
|
|
|
class DictionaryAttributeColumnProvider
|
|
|
|
|
{
|
|
|
|
|
public:
|
2021-01-23 16:47:33 +00:00
|
|
|
|
using ColumnType =
|
2021-06-09 10:43:40 +00:00
|
|
|
|
std::conditional_t<std::is_same_v<DictionaryAttributeType, Array>, ColumnArray,
|
|
|
|
|
std::conditional_t<std::is_same_v<DictionaryAttributeType, String>, ColumnString,
|
2021-09-10 11:49:22 +00:00
|
|
|
|
ColumnVectorOrDecimal<DictionaryAttributeType>>>;
|
2021-01-23 13:18:24 +00:00
|
|
|
|
|
|
|
|
|
using ColumnPtr = typename ColumnType::MutablePtr;
|
|
|
|
|
|
|
|
|
|
static ColumnPtr getColumn(const DictionaryAttribute & dictionary_attribute, size_t size)
|
|
|
|
|
{
|
2021-06-09 10:43:40 +00:00
|
|
|
|
if constexpr (std::is_same_v<DictionaryAttributeType, Array>)
|
|
|
|
|
{
|
2021-06-09 20:51:08 +00:00
|
|
|
|
if (const auto * array_type = typeid_cast<const DataTypeArray *>(dictionary_attribute.type.get()))
|
2021-06-09 10:43:40 +00:00
|
|
|
|
{
|
|
|
|
|
auto nested_column = array_type->getNestedType()->createColumn();
|
|
|
|
|
return ColumnArray::create(std::move(nested_column));
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
throw Exception(ErrorCodes::TYPE_MISMATCH, "Unsupported attribute type.");
|
|
|
|
|
}
|
|
|
|
|
}
|
2021-01-23 13:18:24 +00:00
|
|
|
|
if constexpr (std::is_same_v<DictionaryAttributeType, String>)
|
|
|
|
|
{
|
|
|
|
|
return ColumnType::create();
|
|
|
|
|
}
|
2021-05-09 21:02:08 +00:00
|
|
|
|
else if constexpr (std::is_same_v<DictionaryAttributeType, UUID>)
|
|
|
|
|
{
|
|
|
|
|
return ColumnType::create(size);
|
|
|
|
|
}
|
2022-11-14 14:17:17 +00:00
|
|
|
|
else if constexpr (std::is_same_v<DictionaryAttributeType, IPv4>)
|
|
|
|
|
{
|
|
|
|
|
return ColumnType::create(size);
|
|
|
|
|
}
|
|
|
|
|
else if constexpr (std::is_same_v<DictionaryAttributeType, IPv6>)
|
|
|
|
|
{
|
|
|
|
|
return ColumnType::create(size);
|
|
|
|
|
}
|
2021-09-10 11:49:22 +00:00
|
|
|
|
else if constexpr (is_decimal<DictionaryAttributeType>)
|
2021-01-23 13:18:24 +00:00
|
|
|
|
{
|
2021-06-09 20:51:08 +00:00
|
|
|
|
auto nested_type = removeNullable(dictionary_attribute.type);
|
|
|
|
|
auto scale = getDecimalScale(*nested_type);
|
2021-01-23 13:18:24 +00:00
|
|
|
|
return ColumnType::create(size, scale);
|
|
|
|
|
}
|
2021-05-09 21:26:34 +00:00
|
|
|
|
else if constexpr (is_arithmetic_v<DictionaryAttributeType>)
|
2021-05-09 21:02:08 +00:00
|
|
|
|
{
|
2021-01-23 13:18:24 +00:00
|
|
|
|
return ColumnType::create(size);
|
2021-05-09 21:02:08 +00:00
|
|
|
|
}
|
2021-01-23 13:18:24 +00:00
|
|
|
|
else
|
2021-04-10 18:48:36 +00:00
|
|
|
|
throw Exception(ErrorCodes::TYPE_MISMATCH, "Unsupported attribute type.");
|
2021-01-23 13:18:24 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
/**
|
2021-01-23 16:47:33 +00:00
|
|
|
|
* DictionaryDefaultValueExtractor used to simplify getting default value for IDictionary function `getColumn`.
|
|
|
|
|
* Provides interface for getting default value with operator[];
|
|
|
|
|
*
|
|
|
|
|
* If default_values_column is null then attribute_default_value will be used.
|
|
|
|
|
* If default_values_column is not null in constructor than this column values will be used as default values.
|
2021-06-12 10:53:03 +00:00
|
|
|
|
*
|
|
|
|
|
* For nullable dictionary attribute isNullAt method is provided.
|
2021-01-23 13:18:24 +00:00
|
|
|
|
*/
|
2021-01-23 16:47:33 +00:00
|
|
|
|
template <typename DictionaryAttributeType>
|
2021-01-23 13:18:24 +00:00
|
|
|
|
class DictionaryDefaultValueExtractor
|
|
|
|
|
{
|
2021-01-23 16:47:33 +00:00
|
|
|
|
using DefaultColumnType = typename DictionaryAttributeColumnProvider<DictionaryAttributeType>::ColumnType;
|
2021-01-23 13:18:24 +00:00
|
|
|
|
|
|
|
|
|
public:
|
2021-01-25 12:58:26 +00:00
|
|
|
|
using DefaultValueType = DictionaryValueType<DictionaryAttributeType>;
|
|
|
|
|
|
2021-06-12 10:53:03 +00:00
|
|
|
|
explicit DictionaryDefaultValueExtractor(
|
|
|
|
|
Field attribute_default_value,
|
|
|
|
|
ColumnPtr default_values_column_)
|
2021-01-23 13:18:24 +00:00
|
|
|
|
{
|
2021-06-12 10:53:03 +00:00
|
|
|
|
if (default_values_column_ != nullptr &&
|
|
|
|
|
isColumnConst(*default_values_column_))
|
|
|
|
|
{
|
|
|
|
|
attribute_default_value = (*default_values_column_)[0];
|
|
|
|
|
default_values_column_ = nullptr;
|
|
|
|
|
}
|
|
|
|
|
|
2021-01-25 08:18:07 +00:00
|
|
|
|
if (default_values_column_ == nullptr)
|
2021-06-12 10:53:03 +00:00
|
|
|
|
{
|
2021-06-09 10:43:40 +00:00
|
|
|
|
use_attribute_default_value = true;
|
2021-06-12 10:53:03 +00:00
|
|
|
|
|
|
|
|
|
if (attribute_default_value.isNull())
|
|
|
|
|
default_value_is_null = true;
|
|
|
|
|
else
|
2022-09-11 03:38:59 +00:00
|
|
|
|
default_value = static_cast<DictionaryAttributeType>(attribute_default_value.get<DictionaryAttributeType>());
|
2021-06-12 10:53:03 +00:00
|
|
|
|
}
|
2021-01-23 16:47:33 +00:00
|
|
|
|
else
|
2021-01-23 13:18:24 +00:00
|
|
|
|
{
|
2021-06-12 10:53:03 +00:00
|
|
|
|
const IColumn * default_values_column_ptr = default_values_column_.get();
|
|
|
|
|
|
|
|
|
|
if (const ColumnNullable * nullable_column = typeid_cast<const ColumnNullable *>(default_values_column_.get()))
|
2021-01-23 13:18:24 +00:00
|
|
|
|
{
|
2021-06-12 10:53:03 +00:00
|
|
|
|
default_values_column_ptr = nullable_column->getNestedColumnPtr().get();
|
|
|
|
|
is_null_map = &nullable_column->getNullMapColumn();
|
2021-01-23 13:18:24 +00:00
|
|
|
|
}
|
2021-06-12 10:53:03 +00:00
|
|
|
|
|
|
|
|
|
if (const auto * const default_col = checkAndGetColumn<DefaultColumnType>(default_values_column_ptr))
|
2021-01-23 13:18:24 +00:00
|
|
|
|
{
|
2021-06-12 10:53:03 +00:00
|
|
|
|
default_values_column = default_col;
|
|
|
|
|
use_attribute_default_value = false;
|
2021-01-23 13:18:24 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
2021-04-10 18:48:36 +00:00
|
|
|
|
throw Exception(ErrorCodes::TYPE_MISMATCH, "Type of default column is not the same as dictionary attribute type.");
|
2021-01-23 13:18:24 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2021-01-25 12:58:26 +00:00
|
|
|
|
DefaultValueType operator[](size_t row)
|
2021-01-23 13:18:24 +00:00
|
|
|
|
{
|
2021-06-09 10:43:40 +00:00
|
|
|
|
if (use_attribute_default_value)
|
2021-01-26 10:47:12 +00:00
|
|
|
|
return static_cast<DefaultValueType>(default_value);
|
|
|
|
|
|
|
|
|
|
assert(default_values_column != nullptr);
|
2021-01-23 16:47:33 +00:00
|
|
|
|
|
2021-06-09 10:43:40 +00:00
|
|
|
|
if constexpr (std::is_same_v<DefaultColumnType, ColumnArray>)
|
|
|
|
|
{
|
|
|
|
|
Field field = (*default_values_column)[row];
|
|
|
|
|
return field.get<Array>();
|
|
|
|
|
}
|
|
|
|
|
else if constexpr (std::is_same_v<DefaultColumnType, ColumnString>)
|
2021-01-23 13:18:24 +00:00
|
|
|
|
return default_values_column->getDataAt(row);
|
|
|
|
|
else
|
|
|
|
|
return default_values_column->getData()[row];
|
|
|
|
|
}
|
2021-06-12 10:53:03 +00:00
|
|
|
|
|
|
|
|
|
bool isNullAt(size_t row)
|
|
|
|
|
{
|
|
|
|
|
if (default_value_is_null)
|
|
|
|
|
return true;
|
|
|
|
|
|
|
|
|
|
if (is_null_map)
|
|
|
|
|
return is_null_map->getData()[row];
|
|
|
|
|
|
|
|
|
|
return false;
|
|
|
|
|
}
|
|
|
|
|
|
2021-01-23 13:18:24 +00:00
|
|
|
|
private:
|
2021-06-12 10:53:03 +00:00
|
|
|
|
DictionaryAttributeType default_value {};
|
2021-01-23 16:47:33 +00:00
|
|
|
|
const DefaultColumnType * default_values_column = nullptr;
|
2021-06-12 10:53:03 +00:00
|
|
|
|
const ColumnUInt8 * is_null_map = nullptr;
|
2021-06-09 10:43:40 +00:00
|
|
|
|
bool use_attribute_default_value = false;
|
2021-06-12 10:53:03 +00:00
|
|
|
|
bool default_value_is_null = false;
|
2021-01-23 13:18:24 +00:00
|
|
|
|
};
|
|
|
|
|
|
2021-03-31 21:12:21 +00:00
|
|
|
|
template <DictionaryKeyType key_type>
|
|
|
|
|
class DictionaryKeysArenaHolder;
|
|
|
|
|
|
|
|
|
|
template <>
|
2021-08-17 17:35:43 +00:00
|
|
|
|
class DictionaryKeysArenaHolder<DictionaryKeyType::Simple>
|
2021-03-31 21:12:21 +00:00
|
|
|
|
{
|
|
|
|
|
public:
|
|
|
|
|
static Arena * getComplexKeyArena() { return nullptr; }
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
template <>
|
2021-08-17 17:35:43 +00:00
|
|
|
|
class DictionaryKeysArenaHolder<DictionaryKeyType::Complex>
|
2021-03-31 21:12:21 +00:00
|
|
|
|
{
|
|
|
|
|
public:
|
|
|
|
|
|
|
|
|
|
Arena * getComplexKeyArena() { return &complex_key_arena; }
|
|
|
|
|
|
|
|
|
|
private:
|
|
|
|
|
Arena complex_key_arena;
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2021-02-16 21:33:02 +00:00
|
|
|
|
template <DictionaryKeyType key_type>
|
|
|
|
|
class DictionaryKeysExtractor
|
|
|
|
|
{
|
|
|
|
|
public:
|
2021-08-17 17:35:43 +00:00
|
|
|
|
using KeyType = std::conditional_t<key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
|
2021-02-16 21:33:02 +00:00
|
|
|
|
|
2021-03-31 10:21:30 +00:00
|
|
|
|
explicit DictionaryKeysExtractor(const Columns & key_columns_, Arena * complex_key_arena_)
|
|
|
|
|
: key_columns(key_columns_)
|
|
|
|
|
, complex_key_arena(complex_key_arena_)
|
2021-02-16 21:33:02 +00:00
|
|
|
|
{
|
|
|
|
|
assert(!key_columns.empty());
|
|
|
|
|
|
2021-08-17 17:35:43 +00:00
|
|
|
|
if constexpr (key_type == DictionaryKeyType::Simple)
|
2021-03-25 17:38:30 +00:00
|
|
|
|
{
|
2021-04-17 02:25:22 +00:00
|
|
|
|
key_columns[0] = recursiveRemoveSparse(key_columns[0]->convertToFullColumnIfConst());
|
2021-03-25 17:38:30 +00:00
|
|
|
|
|
2021-03-31 10:21:30 +00:00
|
|
|
|
const auto * vector_col = checkAndGetColumn<ColumnVector<UInt64>>(key_columns[0].get());
|
2021-03-25 17:38:30 +00:00
|
|
|
|
if (!vector_col)
|
2021-03-31 10:21:30 +00:00
|
|
|
|
throw Exception(ErrorCodes::TYPE_MISMATCH, "Column type mismatch for simple key expected UInt64");
|
2021-03-25 17:38:30 +00:00
|
|
|
|
}
|
2021-03-31 10:21:30 +00:00
|
|
|
|
|
|
|
|
|
keys_size = key_columns.front()->size();
|
2021-02-16 21:33:02 +00:00
|
|
|
|
}
|
|
|
|
|
|
2021-03-31 10:21:30 +00:00
|
|
|
|
inline size_t getKeysSize() const
|
2021-02-16 21:33:02 +00:00
|
|
|
|
{
|
2021-03-31 10:21:30 +00:00
|
|
|
|
return keys_size;
|
2021-02-16 21:33:02 +00:00
|
|
|
|
}
|
|
|
|
|
|
2021-03-31 10:21:30 +00:00
|
|
|
|
inline size_t getCurrentKeyIndex() const
|
2021-02-16 21:33:02 +00:00
|
|
|
|
{
|
2021-03-31 10:21:30 +00:00
|
|
|
|
return current_key_index;
|
|
|
|
|
}
|
2021-02-16 21:33:02 +00:00
|
|
|
|
|
2021-03-31 10:21:30 +00:00
|
|
|
|
inline KeyType extractCurrentKey()
|
|
|
|
|
{
|
|
|
|
|
assert(current_key_index < keys_size);
|
2021-02-16 21:33:02 +00:00
|
|
|
|
|
2021-08-17 17:35:43 +00:00
|
|
|
|
if constexpr (key_type == DictionaryKeyType::Simple)
|
2021-03-31 10:21:30 +00:00
|
|
|
|
{
|
|
|
|
|
const auto & column_vector = static_cast<const ColumnVector<UInt64> &>(*key_columns[0]);
|
|
|
|
|
const auto & data = column_vector.getData();
|
2021-02-16 21:33:02 +00:00
|
|
|
|
|
2021-03-31 10:21:30 +00:00
|
|
|
|
auto key = data[current_key_index];
|
|
|
|
|
++current_key_index;
|
|
|
|
|
return key;
|
|
|
|
|
}
|
|
|
|
|
else
|
2021-02-16 21:33:02 +00:00
|
|
|
|
{
|
|
|
|
|
size_t allocated_size_for_columns = 0;
|
|
|
|
|
const char * block_start = nullptr;
|
|
|
|
|
|
2021-03-31 10:21:30 +00:00
|
|
|
|
for (const auto & column : key_columns)
|
2021-02-16 21:33:02 +00:00
|
|
|
|
{
|
2021-03-31 10:21:30 +00:00
|
|
|
|
StringRef serialized_data = column->serializeValueIntoArena(current_key_index, *complex_key_arena, block_start);
|
|
|
|
|
allocated_size_for_columns += serialized_data.size;
|
2021-02-16 21:33:02 +00:00
|
|
|
|
}
|
|
|
|
|
|
2021-03-31 10:21:30 +00:00
|
|
|
|
++current_key_index;
|
|
|
|
|
current_complex_key = StringRef{block_start, allocated_size_for_columns};
|
|
|
|
|
return current_complex_key;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void rollbackCurrentKey() const
|
|
|
|
|
{
|
2021-08-17 17:35:43 +00:00
|
|
|
|
if constexpr (key_type == DictionaryKeyType::Complex)
|
2021-03-31 10:21:30 +00:00
|
|
|
|
complex_key_arena->rollback(current_complex_key.size);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
PaddedPODArray<KeyType> extractAllKeys()
|
|
|
|
|
{
|
|
|
|
|
PaddedPODArray<KeyType> result;
|
|
|
|
|
result.reserve(keys_size - current_key_index);
|
|
|
|
|
|
|
|
|
|
for (; current_key_index < keys_size;)
|
|
|
|
|
{
|
|
|
|
|
auto value = extractCurrentKey();
|
|
|
|
|
result.emplace_back(value);
|
2021-02-16 21:33:02 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return result;
|
|
|
|
|
}
|
|
|
|
|
|
2021-03-31 10:21:30 +00:00
|
|
|
|
void reset()
|
|
|
|
|
{
|
|
|
|
|
current_key_index = 0;
|
|
|
|
|
}
|
|
|
|
|
private:
|
|
|
|
|
Columns key_columns;
|
|
|
|
|
|
|
|
|
|
size_t keys_size = 0;
|
|
|
|
|
size_t current_key_index = 0;
|
2021-02-27 20:39:34 +00:00
|
|
|
|
|
2021-03-31 10:21:30 +00:00
|
|
|
|
KeyType current_complex_key {};
|
|
|
|
|
Arena * complex_key_arena;
|
2021-02-16 21:33:02 +00:00
|
|
|
|
};
|
|
|
|
|
|
2021-08-12 21:39:20 +00:00
|
|
|
|
/// Deserialize columns from keys array using dictionary structure
|
|
|
|
|
MutableColumns deserializeColumnsFromKeys(
|
|
|
|
|
const DictionaryStructure & dictionary_structure,
|
|
|
|
|
const PaddedPODArray<StringRef> & keys,
|
|
|
|
|
size_t start,
|
|
|
|
|
size_t end);
|
|
|
|
|
|
|
|
|
|
/// Deserialize columns with type and name from keys array using dictionary structure
|
|
|
|
|
ColumnsWithTypeAndName deserializeColumnsWithTypeAndNameFromKeys(
|
|
|
|
|
const DictionaryStructure & dictionary_structure,
|
|
|
|
|
const PaddedPODArray<StringRef> & keys,
|
|
|
|
|
size_t start,
|
|
|
|
|
size_t end);
|
|
|
|
|
|
2021-04-02 20:16:04 +00:00
|
|
|
|
/** Merge block with blocks from stream. If there are duplicate keys in block they are filtered out.
|
2021-04-04 13:30:01 +00:00
|
|
|
|
* In result block_to_update will be merged with blocks from stream.
|
2021-04-02 20:16:04 +00:00
|
|
|
|
* Note: readPrefix readImpl readSuffix will be called on stream object during function execution.
|
|
|
|
|
*/
|
|
|
|
|
template <DictionaryKeyType dictionary_key_type>
|
2021-08-04 17:58:18 +00:00
|
|
|
|
void mergeBlockWithPipe(
|
2021-05-01 10:58:24 +00:00
|
|
|
|
size_t key_columns_size,
|
|
|
|
|
Block & block_to_update,
|
2022-05-20 19:49:31 +00:00
|
|
|
|
QueryPipeline pipeline)
|
2021-04-02 20:16:04 +00:00
|
|
|
|
{
|
2021-08-17 17:35:43 +00:00
|
|
|
|
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
|
2021-04-02 20:16:04 +00:00
|
|
|
|
|
|
|
|
|
Columns saved_block_key_columns;
|
2021-05-01 10:58:24 +00:00
|
|
|
|
saved_block_key_columns.reserve(key_columns_size);
|
2021-04-02 20:16:04 +00:00
|
|
|
|
|
|
|
|
|
/// Split into keys columns and attribute columns
|
2021-05-01 10:58:24 +00:00
|
|
|
|
for (size_t i = 0; i < key_columns_size; ++i)
|
2021-04-02 20:16:04 +00:00
|
|
|
|
saved_block_key_columns.emplace_back(block_to_update.safeGetByPosition(i).column);
|
|
|
|
|
|
|
|
|
|
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
|
|
|
|
|
DictionaryKeysExtractor<dictionary_key_type> saved_keys_extractor(saved_block_key_columns, arena_holder.getComplexKeyArena());
|
|
|
|
|
auto saved_keys_extracted_from_block = saved_keys_extractor.extractAllKeys();
|
|
|
|
|
|
2021-05-01 10:58:24 +00:00
|
|
|
|
/**
|
|
|
|
|
* We create filter with our block to update size, because we want to filter out values that have duplicate keys
|
|
|
|
|
* if they appear in blocks that we fetch from stream.
|
|
|
|
|
* But first we try to filter out duplicate keys from existing block.
|
|
|
|
|
* For example if we have block with keys 1, 2, 2, 2, 3, 3
|
|
|
|
|
* Our filter will have [1, 0, 0, 1, 0, 1] after first stage.
|
|
|
|
|
* We also update saved_key_to_index hash map for keys to point to their latest indexes.
|
|
|
|
|
* For example if in blocks from stream we will get keys [4, 2, 3]
|
|
|
|
|
* Our filter will be [1, 0, 0, 0, 0, 0].
|
|
|
|
|
* After reading all blocks from stream we filter our duplicate keys from block_to_update and insert loaded columns.
|
|
|
|
|
*/
|
|
|
|
|
|
2021-04-02 20:16:04 +00:00
|
|
|
|
IColumn::Filter filter(saved_keys_extracted_from_block.size(), true);
|
|
|
|
|
|
|
|
|
|
HashMap<KeyType, size_t> saved_key_to_index;
|
|
|
|
|
saved_key_to_index.reserve(saved_keys_extracted_from_block.size());
|
|
|
|
|
|
|
|
|
|
size_t indexes_to_remove_count = 0;
|
|
|
|
|
|
|
|
|
|
for (size_t i = 0; i < saved_keys_extracted_from_block.size(); ++i)
|
|
|
|
|
{
|
|
|
|
|
auto saved_key = saved_keys_extracted_from_block[i];
|
|
|
|
|
auto [it, was_inserted] = saved_key_to_index.insert(makePairNoInit(saved_key, i));
|
|
|
|
|
|
|
|
|
|
if (!was_inserted)
|
|
|
|
|
{
|
|
|
|
|
size_t index_to_remove = it->getMapped();
|
|
|
|
|
filter[index_to_remove] = false;
|
|
|
|
|
it->getMapped() = i;
|
|
|
|
|
++indexes_to_remove_count;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
auto result_fetched_columns = block_to_update.cloneEmptyColumns();
|
|
|
|
|
|
2021-08-04 17:58:18 +00:00
|
|
|
|
PullingPipelineExecutor executor(pipeline);
|
|
|
|
|
Block block;
|
2021-04-02 20:16:04 +00:00
|
|
|
|
|
2021-08-04 17:58:18 +00:00
|
|
|
|
while (executor.pull(block))
|
2021-04-02 20:16:04 +00:00
|
|
|
|
{
|
2021-05-21 00:57:11 +00:00
|
|
|
|
convertToFullIfSparse(block);
|
2021-04-21 00:23:02 +00:00
|
|
|
|
|
2021-04-02 20:16:04 +00:00
|
|
|
|
Columns block_key_columns;
|
2021-05-01 10:58:24 +00:00
|
|
|
|
block_key_columns.reserve(key_columns_size);
|
2021-04-02 20:16:04 +00:00
|
|
|
|
|
|
|
|
|
/// Split into keys columns and attribute columns
|
2021-05-01 10:58:24 +00:00
|
|
|
|
for (size_t i = 0; i < key_columns_size; ++i)
|
2021-04-02 20:16:04 +00:00
|
|
|
|
block_key_columns.emplace_back(block.safeGetByPosition(i).column);
|
|
|
|
|
|
|
|
|
|
DictionaryKeysExtractor<dictionary_key_type> update_keys_extractor(block_key_columns, arena_holder.getComplexKeyArena());
|
|
|
|
|
PaddedPODArray<KeyType> update_keys = update_keys_extractor.extractAllKeys();
|
|
|
|
|
|
|
|
|
|
for (auto update_key : update_keys)
|
|
|
|
|
{
|
|
|
|
|
const auto * it = saved_key_to_index.find(update_key);
|
|
|
|
|
if (it != nullptr)
|
|
|
|
|
{
|
|
|
|
|
size_t index_to_filter = it->getMapped();
|
|
|
|
|
filter[index_to_filter] = false;
|
|
|
|
|
++indexes_to_remove_count;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
size_t rows = block.rows();
|
|
|
|
|
|
|
|
|
|
for (size_t column_index = 0; column_index < block.columns(); ++column_index)
|
|
|
|
|
{
|
|
|
|
|
const auto update_column = block.safeGetByPosition(column_index).column;
|
|
|
|
|
MutableColumnPtr & result_fetched_column = result_fetched_columns[column_index];
|
|
|
|
|
|
|
|
|
|
result_fetched_column->insertRangeFrom(*update_column, 0, rows);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
size_t result_fetched_rows = result_fetched_columns.front()->size();
|
|
|
|
|
size_t filter_hint = filter.size() - indexes_to_remove_count;
|
|
|
|
|
|
|
|
|
|
for (size_t column_index = 0; column_index < block_to_update.columns(); ++column_index)
|
|
|
|
|
{
|
|
|
|
|
auto & column = block_to_update.getByPosition(column_index).column;
|
|
|
|
|
column = column->filter(filter, filter_hint);
|
|
|
|
|
|
|
|
|
|
MutableColumnPtr mutable_column = column->assumeMutable();
|
|
|
|
|
const IColumn & fetched_column = *result_fetched_columns[column_index];
|
|
|
|
|
mutable_column->insertRangeFrom(fetched_column, 0, result_fetched_rows);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2021-01-23 13:18:24 +00:00
|
|
|
|
/**
|
2021-01-23 16:47:33 +00:00
|
|
|
|
* Returns ColumnVector data as PaddedPodArray.
|
|
|
|
|
|
2021-01-23 13:18:24 +00:00
|
|
|
|
* If column is constant parameter backup_storage is used to store values.
|
|
|
|
|
*/
|
2021-03-24 16:31:00 +00:00
|
|
|
|
/// TODO: Remove
|
2021-01-23 13:18:24 +00:00
|
|
|
|
template <typename T>
|
|
|
|
|
static const PaddedPODArray<T> & getColumnVectorData(
|
2021-03-24 16:31:00 +00:00
|
|
|
|
const IDictionary * dictionary,
|
2021-01-23 13:18:24 +00:00
|
|
|
|
const ColumnPtr column,
|
|
|
|
|
PaddedPODArray<T> & backup_storage)
|
|
|
|
|
{
|
|
|
|
|
bool is_const_column = isColumnConst(*column);
|
2021-04-17 02:25:22 +00:00
|
|
|
|
auto full_column = recursiveRemoveSparse(column->convertToFullColumnIfConst());
|
2021-01-23 13:18:24 +00:00
|
|
|
|
auto vector_col = checkAndGetColumn<ColumnVector<T>>(full_column.get());
|
|
|
|
|
|
|
|
|
|
if (!vector_col)
|
|
|
|
|
{
|
2021-04-10 18:48:36 +00:00
|
|
|
|
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
2021-01-23 13:18:24 +00:00
|
|
|
|
"{}: type mismatch: column has wrong type expected {}",
|
|
|
|
|
dictionary->getDictionaryID().getNameForLogs(),
|
2021-05-04 17:26:09 +00:00
|
|
|
|
TypeName<T>);
|
2021-01-23 13:18:24 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (is_const_column)
|
|
|
|
|
{
|
|
|
|
|
// With type conversion and const columns we need to use backup storage here
|
|
|
|
|
auto & data = vector_col->getData();
|
|
|
|
|
backup_storage.assign(data);
|
|
|
|
|
|
|
|
|
|
return backup_storage;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
return vector_col->getData();
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2021-08-11 12:32:01 +00:00
|
|
|
|
template <typename T>
|
|
|
|
|
static ColumnPtr getColumnFromPODArray(const PaddedPODArray<T> & array)
|
|
|
|
|
{
|
|
|
|
|
auto column_vector = ColumnVector<T>::create();
|
|
|
|
|
column_vector->getData().reserve(array.size());
|
|
|
|
|
column_vector->getData().insert(array.begin(), array.end());
|
|
|
|
|
|
|
|
|
|
return column_vector;
|
|
|
|
|
}
|
|
|
|
|
|
2022-01-19 16:08:56 +00:00
|
|
|
|
template <typename T>
|
|
|
|
|
static ColumnPtr getColumnFromPODArray(PaddedPODArray<T> && array)
|
|
|
|
|
{
|
|
|
|
|
auto column_vector = ColumnVector<T>::create();
|
|
|
|
|
column_vector->getData() = std::move(array);
|
|
|
|
|
|
|
|
|
|
return column_vector;
|
|
|
|
|
}
|
|
|
|
|
|
2021-10-21 14:17:53 +00:00
|
|
|
|
template <typename T>
|
|
|
|
|
static ColumnPtr getColumnFromPODArray(const PaddedPODArray<T> & array, size_t start, size_t length)
|
|
|
|
|
{
|
|
|
|
|
auto column_vector = ColumnVector<T>::create();
|
|
|
|
|
column_vector->getData().reserve(length);
|
|
|
|
|
column_vector->getData().insert(array.begin() + start, array.begin() + start + length);
|
|
|
|
|
|
|
|
|
|
return column_vector;
|
|
|
|
|
}
|
|
|
|
|
|
2021-01-23 13:18:24 +00:00
|
|
|
|
}
|