Skip to content

Commit

Permalink
format
Browse files Browse the repository at this point in the history
  • Loading branch information
Maxxen committed Nov 1, 2024
1 parent e079db5 commit 00b55e1
Show file tree
Hide file tree
Showing 17 changed files with 100 additions and 103 deletions.
4 changes: 2 additions & 2 deletions spatial/include/spatial/core/index/rtree/rtree_index.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@
#include "spatial/core/index/rtree/rtree.hpp"

namespace duckdb {
class PhysicalOperator;
class PhysicalOperator;
}

namespace spatial {
Expand All @@ -34,7 +34,7 @@ class RTreeIndex final : public BoundIndex {

static unique_ptr<BoundIndex> Create(CreateIndexInput &input) {
auto res = make_uniq<RTreeIndex>(input.name, input.constraint_type, input.column_ids, input.table_io_manager,
input.unbound_expressions, input.db, input.options, input.storage_info);
input.unbound_expressions, input.db, input.options, input.storage_info);
return std::move(res);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,8 @@ class LogicalCreateRTreeIndex final : public LogicalExtensionOperator {
LogicalExtensionOperator::Serialize(writer);
writer.WritePropertyWithDefault(300, "operator_type", string("logical_rtree_create_index"));
writer.WritePropertyWithDefault<unique_ptr<CreateIndexInfo>>(400, "info", info);
writer.WritePropertyWithDefault<vector<unique_ptr<Expression>>>(401, "unbound_expressions", unbound_expressions);
writer.WritePropertyWithDefault<vector<unique_ptr<Expression>>>(401, "unbound_expressions",
unbound_expressions);
}

string GetExtensionName() const override {
Expand All @@ -59,10 +60,12 @@ class LogicalCreateRTreeIndexOperatorExtension final : public OperatorExtension
const auto operator_type = reader.ReadPropertyWithDefault<string>(300, "operator_type");
// We only have one extension operator type right now
if (operator_type != "logical_rtree_create_index") {
throw SerializationException("This version of the spatial extension does not support operator type '%s!", operator_type);
throw SerializationException("This version of the spatial extension does not support operator type '%s!",
operator_type);
}
auto create_info = reader.ReadPropertyWithDefault<unique_ptr<CreateInfo>>(400, "info");
auto unbound_expressions = reader.ReadPropertyWithDefault<vector<unique_ptr<Expression>>>(401, "unbound_expressions");
auto unbound_expressions =
reader.ReadPropertyWithDefault<vector<unique_ptr<Expression>>>(401, "unbound_expressions");

auto info = unique_ptr_cast<CreateInfo, CreateIndexInfo>(std::move(create_info));

Expand All @@ -78,7 +81,6 @@ class LogicalCreateRTreeIndexOperatorExtension final : public OperatorExtension
}
};


} // namespace core

} // namespace spatial
6 changes: 4 additions & 2 deletions spatial/include/spatial/core/util/managed_collection.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,8 @@ class ManagedCollectionBlock {
idx_t item_count;

explicit ManagedCollectionBlock(idx_t item_capacity)
: handle(nullptr), item_capacity(item_capacity), item_count(0) {}
: handle(nullptr), item_capacity(item_capacity), item_count(0) {
}
ManagedCollectionBlock(shared_ptr<BlockHandle> handle, idx_t item_capacity)
: handle(std::move(handle)), item_capacity(item_capacity), item_count(0) {
}
Expand Down Expand Up @@ -83,7 +84,8 @@ void ManagedCollection<T>::InitializeAppend(ManagedCollectionAppendState &state,

if (initial_smaller_capacity < block_capacity) {
// Allocate a new small block
blocks.emplace_back(manager.RegisterSmallMemory(initial_smaller_capacity * sizeof(T)), initial_smaller_capacity);
blocks.emplace_back(manager.RegisterSmallMemory(initial_smaller_capacity * sizeof(T)),
initial_smaller_capacity);
state.block = &blocks.back();
state.block->item_count = 0;
state.block->item_capacity = initial_smaller_capacity;
Expand Down
3 changes: 2 additions & 1 deletion spatial/include/spatial/doc_util.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,8 @@ struct DocUtil {
AddDocumentation(db, function_name, description, example, tag_map);
}

static void AddFunctionParameterNames(duckdb::DatabaseInstance &db, const char *function_name, duckdb::vector<duckdb::string> names);
static void AddFunctionParameterNames(duckdb::DatabaseInstance &db, const char *function_name,
duckdb::vector<duckdb::string> names);
};

} // namespace spatial
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ static constexpr const char *DOC_EXAMPLE = R"(
-- POLYGON ((1 1, 1 5, 5 5, 5 1, 1 1))
)";

static constexpr const char* DOC_ALIAS_DESCRIPTION = R"(
static constexpr const char *DOC_ALIAS_DESCRIPTION = R"(
Alias for [ST_Extent_Agg](#st_extent_agg).
Computes the minimal-bounding-box polygon containing the set of input geometries.
Expand All @@ -109,7 +109,7 @@ static constexpr const char* DOC_ALIAS_DESCRIPTION = R"(
void CoreAggregateFunctions::RegisterStExtentAgg(DatabaseInstance &db) {

auto function = AggregateFunction::UnaryAggregate<ExtentAggState, geometry_t, geometry_t, ExtentAggFunction>(
GeoTypes::GEOMETRY(), GeoTypes::GEOMETRY());
GeoTypes::GEOMETRY(), GeoTypes::GEOMETRY());

// Register the function
function.name = "ST_Extent_Agg";
Expand All @@ -120,7 +120,6 @@ void CoreAggregateFunctions::RegisterStExtentAgg(DatabaseInstance &db) {
function.name = "ST_Envelope_Agg";
ExtensionUtil::RegisterFunction(db, function);
DocUtil::AddDocumentation(db, "ST_Envelope_Agg", DOC_ALIAS_DESCRIPTION, DOC_EXAMPLE, DOC_TAGS);

}

} // namespace core
Expand Down
61 changes: 30 additions & 31 deletions spatial/src/spatial/core/functions/scalar/st_hilbert.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -77,14 +77,13 @@ inline uint32_t HilbertEncode(uint32_t n, uint32_t x, uint32_t y) {
return ((Interleave(i1) << 1) | Interleave(i0)) >> (32 - 2 * n);
}

static uint32_t FloatToUint32(float f)
{
static uint32_t FloatToUint32(float f) {
if (std::isnan(f)) {
return 0xFFFFFFFF;
}
uint32_t res;
memcpy(&res, &f, sizeof(res));
if((res & 0x80000000) != 0) {
if ((res & 0x80000000) != 0) {
res ^= 0xFFFFFFFF;
} else {
res |= 0x80000000;
Expand Down Expand Up @@ -140,13 +139,14 @@ static void HilbertEncodeBoundsFunction(DataChunk &args, ExpressionState &state,
input_vec, bounds_vec, result, count, [&](const GEOM_TYPE &geom_type, const BOX_TYPE &bounds) {
const auto geom = geom_type.val;

Box2D<double> geom_bounds;
if(!geom.TryGetCachedBounds(geom_bounds)) {
throw InvalidInputException("ST_Hilbert(geom, bounds) requires that all geometries have a bounding box");
}
Box2D<double> geom_bounds;
if (!geom.TryGetCachedBounds(geom_bounds)) {
throw InvalidInputException(
"ST_Hilbert(geom, bounds) requires that all geometries have a bounding box");
}

const auto dx = geom_bounds.min.x + (geom_bounds.max.x - geom_bounds.min.x) / 2;
const auto dy = geom_bounds.min.y + (geom_bounds.max.y - geom_bounds.min.y) / 2;
const auto dx = geom_bounds.min.x + (geom_bounds.max.x - geom_bounds.min.x) / 2;
const auto dy = geom_bounds.min.y + (geom_bounds.max.y - geom_bounds.min.y) / 2;

const auto hilbert_width = max_hilbert / (bounds.c_val - bounds.a_val);
const auto hilbert_height = max_hilbert / (bounds.d_val - bounds.b_val);
Expand All @@ -167,30 +167,29 @@ static void HilbertEncodeGeometryFunction(DataChunk &args, ExpressionState &stat
auto &input_vec = args.data[0];

UnaryExecutor::ExecuteWithNulls<geometry_t, uint32_t>(
input_vec, result, count, [&](const geometry_t &geom, ValidityMask &mask, idx_t out_idx) -> uint32_t {
Box2D<double> bounds;
if(!geom.TryGetCachedBounds(bounds)) {
mask.SetInvalid(out_idx);
return 0;
}

Box2D<float> bounds_f;
bounds_f.min.x = MathUtil::DoubleToFloatDown(bounds.min.x);
bounds_f.min.y = MathUtil::DoubleToFloatDown(bounds.min.y);
bounds_f.max.x = MathUtil::DoubleToFloatUp(bounds.max.x);
bounds_f.max.y = MathUtil::DoubleToFloatUp(bounds.max.y);

const auto dx = bounds_f.min.x + (bounds_f.max.x - bounds_f.min.x) / 2;
const auto dy = bounds_f.min.y + (bounds_f.max.y - bounds_f.min.y) / 2;

const auto hx = FloatToUint32(dx);
const auto hy = FloatToUint32(dy);

return HilbertEncode(16, hx, hy);
});
input_vec, result, count, [&](const geometry_t &geom, ValidityMask &mask, idx_t out_idx) -> uint32_t {
Box2D<double> bounds;
if (!geom.TryGetCachedBounds(bounds)) {
mask.SetInvalid(out_idx);
return 0;
}

Box2D<float> bounds_f;
bounds_f.min.x = MathUtil::DoubleToFloatDown(bounds.min.x);
bounds_f.min.y = MathUtil::DoubleToFloatDown(bounds.min.y);
bounds_f.max.x = MathUtil::DoubleToFloatUp(bounds.max.x);
bounds_f.max.y = MathUtil::DoubleToFloatUp(bounds.max.y);

const auto dx = bounds_f.min.x + (bounds_f.max.x - bounds_f.min.x) / 2;
const auto dy = bounds_f.min.y + (bounds_f.max.y - bounds_f.min.y) / 2;

const auto hx = FloatToUint32(dx);
const auto hy = FloatToUint32(dy);

return HilbertEncode(16, hx, hy);
});
}


//------------------------------------------------------------------------------
// BOX_2D/BOX_2DF
//------------------------------------------------------------------------------
Expand Down
5 changes: 2 additions & 3 deletions spatial/src/spatial/core/functions/scalar/st_multi.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,10 @@ static void GeometryMultiFunction(DataChunk &args, ExpressionState &state, Vecto
auto &input = args.data[0];

UnaryExecutor::Execute<geometry_t, geometry_t>(input, result, args.size(), [&](const geometry_t &geom_blob) {

const bool has_z = geom_blob.GetProperties().HasZ();
const bool has_m = geom_blob.GetProperties().HasM();

switch(geom_blob.GetType()) {
switch (geom_blob.GetType()) {
case GeometryType::POINT: {
auto mpoint = MultiPoint::Create(arena, 1, has_z, has_m);
MultiPoint::Part(mpoint, 0) = Geometry::Deserialize(arena, geom_blob);
Expand Down Expand Up @@ -72,7 +71,7 @@ static constexpr DocTag DOC_TAGS[] = {{"ext", "spatial"}, {"category", "construc
// Register functions
//------------------------------------------------------------------------------
void CoreScalarFunctions::RegisterStMulti(DatabaseInstance &db) {
ScalarFunction function("ST_Multi",{GeoTypes::GEOMETRY()}, GeoTypes::GEOMETRY(), GeometryMultiFunction);
ScalarFunction function("ST_Multi", {GeoTypes::GEOMETRY()}, GeoTypes::GEOMETRY(), GeometryMultiFunction);
function.init_local_state = GeometryFunctionLocalState::Init;
ExtensionUtil::RegisterFunction(db, function);
DocUtil::AddDocumentation(db, "ST_Multi", DOC_DESCRIPTION, DOC_EXAMPLE, DOC_TAGS);
Expand Down
26 changes: 12 additions & 14 deletions spatial/src/spatial/core/index/rtree/rtree.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -416,7 +416,7 @@ void RTree::RootInsert(RTreeEntry &root_entry, const RTreeEntry &new_entry) {
// Delete
//------------------------------------------------------------------------------
DeleteResult RTree::NodeDelete(RTreeEntry &entry, const RTreeEntry &target, vector<RTreeEntry> &orphans) {
if(!entry.bounds.Intersects(target.bounds)) {
if (!entry.bounds.Intersects(target.bounds)) {
return {false, false, false};
}
const auto is_leaf = entry.pointer.IsLeafPage();
Expand Down Expand Up @@ -493,8 +493,9 @@ DeleteResult RTree::LeafDelete(RTreeEntry &entry, const RTreeEntry &target, vect

// Do a binary search with std::lower_bound to find the matching rowid
// This is faster than a linear search
const auto it = std::lower_bound(node.begin(), node.end(), target.pointer.GetRowId(),
[](const RTreeEntry &item, const row_t &row) { return item.pointer.GetRowId() < row; });
const auto it =
std::lower_bound(node.begin(), node.end(), target.pointer.GetRowId(),
[](const RTreeEntry &item, const row_t &row) { return item.pointer.GetRowId() < row; });
if (it == node.end()) {
// Not found in this leaf
return {false, false, false};
Expand All @@ -503,7 +504,7 @@ DeleteResult RTree::LeafDelete(RTreeEntry &entry, const RTreeEntry &target, vect
const auto &child = *it;

// Ok, did the binary search actually find the rowid?
if(child.pointer.GetRowId() != target.pointer.GetRowId()) {
if (child.pointer.GetRowId() != target.pointer.GetRowId()) {
return {false, false, false};
}

Expand Down Expand Up @@ -608,16 +609,16 @@ string RTree::ToString() const {

stack.emplace_back(root.pointer);

while(!stack.empty()) {
while (!stack.empty()) {
auto &frame = stack.back();
const auto &node = Ref(frame.pointer);
const auto count = node.GetCount();

if(frame.pointer.IsLeafPage()) {
while(frame.entry_idx < count) {
if (frame.pointer.IsLeafPage()) {
while (frame.entry_idx < count) {
auto &entry = node[frame.entry_idx];
// TODO: Print entry
for(idx_t i = 0; i < level; i++) {
for (idx_t i = 0; i < level; i++) {
result += " ";
}

Expand All @@ -627,14 +628,13 @@ string RTree::ToString() const {
}
stack.pop_back();
level--;
}
else {
} else {
D_ASSERT(frame.pointer.IsBranchPage());
if(frame.entry_idx < count) {
if (frame.entry_idx < count) {
auto &entry = node[frame.entry_idx];

// TODO: Print entry
for(idx_t i = 0; i < level; i++) {
for (idx_t i = 0; i < level; i++) {
result += " ";
}

Expand All @@ -650,15 +650,13 @@ string RTree::ToString() const {
}
}


return result;
}

void RTree::Print() const {
Printer::Print(ToString());
}


} // namespace core

} // namespace spatial
Original file line number Diff line number Diff line change
Expand Up @@ -42,8 +42,8 @@ void LogicalCreateRTreeIndex::ResolveColumnBindings(ColumnBindingResolver &res,
[&](unique_ptr<Expression> *child) { res.VisitExpression(child); });
}

static unique_ptr<PhysicalOperator> CreateNullFilter(const LogicalOperator &op,
const vector<LogicalType> &types, ClientContext &context) {
static unique_ptr<PhysicalOperator> CreateNullFilter(const LogicalOperator &op, const vector<LogicalType> &types,
ClientContext &context) {
vector<unique_ptr<Expression>> filter_select_list;

// Filter NOT NULL on the GEOMETRY column
Expand Down Expand Up @@ -75,9 +75,8 @@ static unique_ptr<PhysicalOperator> CreateNullFilter(const LogicalOperator &op,
return make_uniq<PhysicalFilter>(types, std::move(filter_select_list), op.estimated_cardinality);
}

static unique_ptr<PhysicalOperator> CreateBoundingBoxProjection(const LogicalOperator &op,
const vector<LogicalType> &types,
ClientContext &context) {
static unique_ptr<PhysicalOperator>
CreateBoundingBoxProjection(const LogicalOperator &op, const vector<LogicalType> &types, ClientContext &context) {
auto &catalog = Catalog::GetSystemCatalog(context);

// Get the bounding box function
Expand All @@ -103,8 +102,8 @@ static unique_ptr<PhysicalOperator> CreateBoundingBoxProjection(const LogicalOpe
return make_uniq<PhysicalProjection>(types, std::move(select_list), op.estimated_cardinality);
}

static unique_ptr<PhysicalOperator> CreateOrderByMinX(const LogicalOperator &op,
const vector<LogicalType> &types, ClientContext &context) {
static unique_ptr<PhysicalOperator> CreateOrderByMinX(const LogicalOperator &op, const vector<LogicalType> &types,
ClientContext &context) {
auto &catalog = Catalog::GetSystemCatalog(context);

// Get the centroid value function
Expand Down Expand Up @@ -137,7 +136,6 @@ static unique_ptr<PhysicalOperator> CreateOrderByMinX(const LogicalOperator &op,
return make_uniq<PhysicalOrder>(types, std::move(orders), projections, op.estimated_cardinality);
}


unique_ptr<PhysicalOperator> RTreeIndex::CreatePlan(PlanIndexInput &input) {

auto &op = input.op;
Expand Down Expand Up @@ -204,7 +202,6 @@ unique_ptr<PhysicalOperator> RTreeIndex::CreatePlan(PlanIndexInput &input) {
physical_create_index->children.push_back(std::move(physical_order));

return std::move(physical_create_index);

}

// TODO: Remove this
Expand Down
Loading

0 comments on commit 00b55e1

Please sign in to comment.