PolygonDictionary support read method

This commit is contained in:
Maksim Kita 2021-10-13 19:31:59 +03:00
parent 3ac2766c13
commit 5802213722
6 changed files with 142 additions and 63 deletions

View File

@ -7,7 +7,8 @@
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Processors/Pipe.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <Dictionaries/DictionaryFactory.h>
#include <Dictionaries/DictionarySource.h>
@ -28,14 +29,12 @@ IPolygonDictionary::IPolygonDictionary(
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_)
Configuration configuration_)
: IDictionary(dict_id_)
, dict_struct(dict_struct_)
, source_ptr(std::move(source_ptr_))
, dict_lifetime(dict_lifetime_)
, input_type(input_type_)
, point_type(point_type_)
, configuration(configuration_)
{
setup();
loadData();
@ -55,7 +54,7 @@ ColumnPtr IPolygonDictionary::getColumn(
DefaultValueProvider default_value_provider(attribute.null_value, default_values_column);
size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second;
const auto & attribute_values_column = attributes[attribute_index];
const auto & attribute_values_column = attributes_columns[attribute_index];
auto result = attribute_values_column->cloneEmpty();
result->reserve(requested_key_points.size());
@ -119,21 +118,52 @@ ColumnPtr IPolygonDictionary::getColumn(
return result;
}
Pipe IPolygonDictionary::read(const Names &, size_t) const
Pipe IPolygonDictionary::read(const Names & column_names, size_t) const
{
// TODO: In order for this to work one would first have to support retrieving arrays from dictionaries.
// I believe this is a separate task done by some other people.
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Reading the dictionary is not allowed");
if (!configuration.store_polygon_key_column)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Set `store_polygon_key_column` setting in dictionary configuration to true to support reading from PolygonDictionary.");
const auto & dictionary_structure_keys = *dict_struct.key;
const auto & dictionary_key_attribute = dictionary_structure_keys[0];
ColumnsWithTypeAndName result_columns;
result_columns.reserve(column_names.size());
for (const auto & column_name : column_names) {
ColumnWithTypeAndName column_with_type;
if (column_name == dictionary_key_attribute.name) {
column_with_type.column = key_attribute_column;
column_with_type.type = dictionary_key_attribute.type;
} else {
const auto & dictionary_attribute = dict_struct.getAttribute(column_name);
size_t attribute_index = dict_struct.attribute_name_to_index.find(dictionary_attribute.name)->second;
column_with_type.column = attributes_columns[attribute_index];
column_with_type.type = dictionary_attribute.type;
}
column_with_type.name = column_name;
result_columns.emplace_back(column_with_type);
}
auto source = std::make_shared<SourceFromSingleChunk>(Block(result_columns));
return Pipe(std::move(source));
}
void IPolygonDictionary::setup()
{
attributes.reserve(dict_struct.attributes.size());
const auto & dictionary_structure_keys = *dict_struct.key;
key_attribute_column = dictionary_structure_keys[0].type->createColumn();
attributes_columns.reserve(dict_struct.attributes.size());
for (const auto & attribute : dict_struct.attributes)
{
auto column = attribute.type->createColumn();
attributes.emplace_back(std::move(column));
attributes_columns.emplace_back(std::move(column));
if (attribute.hierarchical)
throw Exception(ErrorCodes::TYPE_MISMATCH,
@ -147,20 +177,24 @@ void IPolygonDictionary::blockToAttributes(const DB::Block & block)
const auto rows = block.rows();
size_t skip_key_column_offset = 1;
for (size_t i = 0; i < attributes.size(); ++i)
for (size_t i = 0; i < attributes_columns.size(); ++i)
{
const auto & block_column = block.safeGetByPosition(i + skip_key_column_offset);
const auto & column = block_column.column;
attributes[i]->assumeMutable()->insertRangeFrom(*column, 0, column->size());
attributes_columns[i]->assumeMutable()->insertRangeFrom(*column, 0, column->size());
}
/** Multi-polygons could cause bigger sizes, but this is better than nothing. */
polygons.reserve(polygons.size() + rows);
polygon_index_to_attribute_value_index.reserve(polygon_index_to_attribute_value_index.size() + rows);
const auto & key = block.safeGetByPosition(0).column;
extractPolygons(key);
const auto & key_column = block.safeGetByPosition(0).column;
if (configuration.store_polygon_key_column)
key_attribute_column->assumeMutable()->insertRangeFrom(*key_column, 0, key_column->size());
extractPolygons(key_column);
}
void IPolygonDictionary::loadData()
@ -210,7 +244,7 @@ void IPolygonDictionary::calculateBytesAllocated()
{
/// Index allocated by subclass not counted because it take a small part in relation to attributes and polygons
for (const auto & column : attributes)
for (const auto & column : attributes_columns)
bytes_allocated += column->allocatedBytes();
for (auto & polygon : polygons)
@ -487,7 +521,7 @@ void IPolygonDictionary::extractPolygons(const ColumnPtr & column)
Offset offset;
const IColumn * points_collection = nullptr;
switch (input_type)
switch (configuration.input_type)
{
case InputType::MultiPolygon:
points_collection = unrollMultiPolygons(column, offset);
@ -504,7 +538,7 @@ void IPolygonDictionary::extractPolygons(const ColumnPtr & column)
/** Adding the first empty polygon */
data.addPolygon(true);
switch (point_type)
switch (configuration.point_type)
{
case PointType::Array:
handlePointsReprByArrays(points_collection, data, offset);

View File

@ -50,13 +50,22 @@ public:
Tuple,
};
struct Configuration
{
/// Store polygon key column. That will allow to read columns from polygon dictionary.
bool store_polygon_key_column = false;
InputType input_type = InputType::MultiPolygon;
PointType point_type = PointType::Array;
};
IPolygonDictionary(
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_);
Configuration configuration_);
std::string getTypeName() const override { return "Polygon"; }
@ -74,7 +83,7 @@ public:
double getHitRate() const override { return 1.0; }
size_t getElementCount() const override { return attributes.empty() ? 0 : attributes.front()->size(); }
size_t getElementCount() const override { return attributes_columns.empty() ? 0 : attributes_columns.front()->size(); }
double getLoadFactor() const override { return 1.0; }
@ -120,9 +129,7 @@ protected:
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
const InputType input_type;
const PointType point_type;
const Configuration configuration;
private:
/** Helper functions for loading the data from the configuration.
@ -146,7 +153,9 @@ private:
ValueSetter && set_value,
DefaultValueExtractor & default_value_extractor) const;
Columns attributes;
ColumnPtr key_attribute_column;
Columns attributes_columns;
size_t bytes_allocated = 0;
mutable std::atomic<size_t> query_count{0};

View File

@ -22,9 +22,8 @@ PolygonDictionarySimple::PolygonDictionarySimple(
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_):
IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_)
Configuration configuration_)
: IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, configuration_)
{
}
@ -35,8 +34,7 @@ std::shared_ptr<const IExternalLoadable> PolygonDictionarySimple::clone() const
this->dict_struct,
this->source_ptr->clone(),
this->dict_lifetime,
this->input_type,
this->point_type);
this->configuration);
}
bool PolygonDictionarySimple::find(const Point & point, size_t & polygon_index) const
@ -59,11 +57,10 @@ PolygonDictionaryIndexEach::PolygonDictionaryIndexEach(
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_,
Configuration configuration_,
int min_intersections_,
int max_depth_)
: IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_),
: IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, configuration_),
grid(min_intersections_, max_depth_, polygons),
min_intersections(min_intersections_),
max_depth(max_depth_)
@ -84,8 +81,7 @@ std::shared_ptr<const IExternalLoadable> PolygonDictionaryIndexEach::clone() con
this->dict_struct,
this->source_ptr->clone(),
this->dict_lifetime,
this->input_type,
this->point_type,
this->configuration,
this->min_intersections,
this->max_depth);
}
@ -118,11 +114,10 @@ PolygonDictionaryIndexCell::PolygonDictionaryIndexCell(
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_,
Configuration configuration_,
size_t min_intersections_,
size_t max_depth_)
: IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_),
: IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, configuration_),
index(min_intersections_, max_depth_, polygons),
min_intersections(min_intersections_),
max_depth(max_depth_)
@ -136,8 +131,7 @@ std::shared_ptr<const IExternalLoadable> PolygonDictionaryIndexCell::clone() con
this->dict_struct,
this->source_ptr->clone(),
this->dict_lifetime,
this->input_type,
this->point_type,
this->configuration,
this->min_intersections,
this->max_depth);
}
@ -179,35 +173,37 @@ DictionaryPtr createLayout(const std::string & ,
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"The 'key' should consist of a single attribute for a polygon dictionary");
IPolygonDictionary::InputType input_type;
IPolygonDictionary::PointType point_type;
IPolygonDictionary::Configuration configuration;
const auto key_type = (*dict_struct.key)[0].type;
const auto f64 = std::make_shared<DataTypeFloat64>();
const auto multi_polygon_array = DataTypeArray(std::make_shared<DataTypeArray>(std::make_shared<DataTypeArray>(std::make_shared<DataTypeArray>(f64))));
const auto multi_polygon_tuple = DataTypeArray(std::make_shared<DataTypeArray>(std::make_shared<DataTypeArray>(std::make_shared<DataTypeTuple>(std::vector<DataTypePtr>{f64, f64}))));
const auto simple_polygon_array = DataTypeArray(std::make_shared<DataTypeArray>(f64));
const auto simple_polygon_tuple = DataTypeArray(std::make_shared<DataTypeTuple>(std::vector<DataTypePtr>{f64, f64}));
if (key_type->equals(multi_polygon_array))
{
input_type = IPolygonDictionary::InputType::MultiPolygon;
point_type = IPolygonDictionary::PointType::Array;
configuration.input_type = IPolygonDictionary::InputType::MultiPolygon;
configuration.point_type = IPolygonDictionary::PointType::Array;
}
else if (key_type->equals(multi_polygon_tuple))
{
input_type = IPolygonDictionary::InputType::MultiPolygon;
point_type = IPolygonDictionary::PointType::Tuple;
configuration.input_type = IPolygonDictionary::InputType::MultiPolygon;
configuration.point_type = IPolygonDictionary::PointType::Tuple;
}
else if (key_type->equals(simple_polygon_array))
{
input_type = IPolygonDictionary::InputType::SimplePolygon;
point_type = IPolygonDictionary::PointType::Array;
configuration.input_type = IPolygonDictionary::InputType::SimplePolygon;
configuration.point_type = IPolygonDictionary::PointType::Array;
}
else if (key_type->equals(simple_polygon_tuple))
{
input_type = IPolygonDictionary::InputType::SimplePolygon;
point_type = IPolygonDictionary::PointType::Tuple;
configuration.input_type = IPolygonDictionary::InputType::SimplePolygon;
configuration.point_type = IPolygonDictionary::PointType::Tuple;
}
else
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"The key type {} is not one of the following allowed types for a polygon dictionary: {} {} {} {} ",
key_type->getName(),
@ -215,6 +211,14 @@ DictionaryPtr createLayout(const std::string & ,
multi_polygon_tuple.getName(),
simple_polygon_array.getName(),
simple_polygon_tuple.getName());
}
const auto & layout_prefix = config_prefix + ".layout";
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(layout_prefix, keys);
const auto & dict_prefix = layout_prefix + "." + keys.front();
configuration.store_polygon_key_column = config.getUInt(dict_prefix + ".store_polygon_key_column", false);
if (dict_struct.range_min || dict_struct.range_max)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
@ -228,16 +232,12 @@ DictionaryPtr createLayout(const std::string & ,
if constexpr (std::is_same_v<PolygonDictionary, PolygonDictionaryIndexEach> || std::is_same_v<PolygonDictionary, PolygonDictionaryIndexCell>)
{
const auto & layout_prefix = config_prefix + ".layout";
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(layout_prefix, keys);
const auto & dict_prefix = layout_prefix + "." + keys.front();
size_t max_depth = config.getUInt(dict_prefix + ".max_depth", PolygonDictionary::kMaxDepthDefault);
size_t min_intersections = config.getUInt(dict_prefix + ".min_intersections", PolygonDictionary::kMinIntersectionsDefault);
return std::make_unique<PolygonDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, input_type, point_type, min_intersections, max_depth);
return std::make_unique<PolygonDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, configuration, min_intersections, max_depth);
}
else
return std::make_unique<PolygonDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, input_type, point_type);
return std::make_unique<PolygonDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, configuration);
}
void registerDictionaryPolygon(DictionaryFactory & factory)

View File

@ -21,8 +21,7 @@ public:
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_);
Configuration configuration_);
std::shared_ptr<const IExternalLoadable> clone() const override;
@ -44,8 +43,7 @@ public:
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_,
Configuration configuration_,
int min_intersections_,
int max_depth_);
@ -73,8 +71,7 @@ public:
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_,
Configuration configuration_,
size_t min_intersections_,
size_t max_depth_);

View File

@ -0,0 +1 @@
[[[(3,1),(0,1),(0,-1),(3,-1)]]] Value

View File

@ -0,0 +1,38 @@
DROP TABLE IF EXISTS polygons_test_table;
CREATE TABLE polygons_test_table
(
key Array(Array(Array(Tuple(Float64, Float64)))),
name String
) ENGINE = TinyLog;
INSERT INTO polygons_test_table VALUES ([[[(3, 1), (0, 1), (0, -1), (3, -1)]]], 'Value');
DROP DICTIONARY IF EXISTS polygons_test_dictionary_no_option;
CREATE DICTIONARY polygons_test_dictionary_no_option
(
key Array(Array(Array(Tuple(Float64, Float64)))),
name String
)
PRIMARY KEY key
SOURCE(CLICKHOUSE(TABLE 'polygons_test_table'))
LAYOUT(POLYGON())
LIFETIME(0);
SELECT * FROM polygons_test_dictionary_no_option; -- {serverError 1}
DROP DICTIONARY IF EXISTS polygons_test_dictionary;
CREATE DICTIONARY polygons_test_dictionary
(
key Array(Array(Array(Tuple(Float64, Float64)))),
name String
)
PRIMARY KEY key
SOURCE(CLICKHOUSE(TABLE 'polygons_test_table'))
LAYOUT(POLYGON(STORE_POLYGON_KEY_COLUMN 1))
LIFETIME(0);
SELECT * FROM polygons_test_dictionary;
DROP DICTIONARY polygons_test_dictionary_no_option;
DROP DICTIONARY polygons_test_dictionary;
DROP TABLE polygons_test_table;