objClass) {
+ Node s = (Node)kryo.readClassAndObject(input);
+ Node p = (Node)kryo.readClassAndObject(input);
+ Node o = (Node)kryo.readClassAndObject(input);
+ Triple result = Triple.create(s, p, o);
+ return result;
+ }
+}
diff --git a/jena-geosparql/src/main/java/org/apache/jena/geosparql/spatial/index/v2/serde/sedona/CustomGeometrySerde.java b/jena-geosparql/src/main/java/org/apache/jena/geosparql/spatial/index/v2/serde/sedona/CustomGeometrySerde.java
new file mode 100644
index 00000000000..5f7f09de530
--- /dev/null
+++ b/jena-geosparql/src/main/java/org/apache/jena/geosparql/spatial/index/v2/serde/sedona/CustomGeometrySerde.java
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.geosparql.spatial.index.v2.serde.sedona;
+
+import java.io.Serializable;
+
+import org.apache.jena.geosparql.spatial.index.v2.serde.GeometrySerdeAdapter;
+import org.apache.sedona.common.geometryObjects.Circle;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryCollection;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Registration;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+
+/* This file is an adapted copy of apache-sedona org.apache.sedona.common.geometrySerde.GeometrySerde */
+/* The change is, that geometry read/write is delegated to a GeometrySerdeAdapter instance. */
+
+/**
+ * Provides methods to efficiently serialize and deserialize geometry types.
+ *
+ * Supports Point, LineString, Polygon, MultiPoint, MultiLineString, MultiPolygon,
+ * GeometryCollection, Circle and Envelope types.
+ *
+ * First byte contains {@link Type#id}. Then go type-specific bytes, followed
+ * by user-data attached to the geometry.
+ */
+public class CustomGeometrySerde
+ extends Serializer implements Serializable
+{
+ private static final long serialVersionUID = 1L;
+
+ protected GeometrySerdeAdapter geometrySerdeAdapter;
+
+ public CustomGeometrySerde(GeometrySerdeAdapter geometrySerdeAdapter) {
+ super();
+ this.geometrySerdeAdapter = geometrySerdeAdapter;
+ }
+
+ private void writeGeometry(Kryo kryo, Output out, Geometry geometry)
+ {
+ geometrySerdeAdapter.write(kryo, out, geometry);
+ writeUserData(kryo, out, geometry);
+ }
+
+ private Geometry readGeometry(Kryo kryo, Input input)
+ {
+ Geometry geometry;
+ try {
+ geometry = geometrySerdeAdapter.read(kryo, input);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+
+ geometry.setUserData(readUserData(kryo, input));
+ return geometry;
+ }
+
+ @Override
+ public void write(Kryo kryo, Output out, Object object) {
+ if (object instanceof Circle) {
+ Circle circle = (Circle) object;
+ writeType(out, Type.CIRCLE);
+ out.writeDouble(circle.getRadius());
+ writeGeometry(kryo, out, circle.getCenterGeometry());
+ writeUserData(kryo, out, circle);
+ } else if (object instanceof Point
+ || object instanceof LineString
+ || object instanceof Polygon
+ || object instanceof GeometryCollection) {
+ writeType(out, Type.SHAPE);
+ writeGeometry(kryo, out, (Geometry) object);
+ } else if (object instanceof Envelope) {
+ Envelope envelope = (Envelope) object;
+ writeType(out, Type.ENVELOPE);
+ out.writeDouble(envelope.getMinX());
+ out.writeDouble(envelope.getMaxX());
+ out.writeDouble(envelope.getMinY());
+ out.writeDouble(envelope.getMaxY());
+ } else {
+ writeType(out, Type.OTHER);
+ kryo.writeClassAndObject(out, object);
+ }
+ }
+
+ private void writeType(Output out, Type type) {
+ out.writeByte((byte) type.id);
+ }
+
+// private void writeGeometry(Kryo kryo, Output out, Geometry geometry) {
+// byte[] data = GeometrySerializer.serialize(geometry);
+// out.writeInt(data.length);
+// out.write(data, 0, data.length);
+// writeUserData(kryo, out, geometry);
+// }
+
+ private void writeUserData(Kryo kryo, Output out, Geometry geometry) {
+ out.writeBoolean(geometry.getUserData() != null);
+ if (geometry.getUserData() != null) {
+ kryo.writeClass(out, geometry.getUserData().getClass());
+ kryo.writeObject(out, geometry.getUserData());
+ }
+ }
+
+ @Override
+ public Object read(Kryo kryo, Input input, Class aClass) {
+ byte typeId = input.readByte();
+ Type geometryType = Type.fromId(typeId);
+
+ switch (geometryType) {
+ case SHAPE:
+ return readGeometry(kryo, input);
+ case CIRCLE:
+ {
+ double radius = input.readDouble();
+ Geometry centerGeometry = readGeometry(kryo, input);
+ Object userData = readUserData(kryo, input);
+
+ Circle circle = new Circle(centerGeometry, radius);
+ circle.setUserData(userData);
+ return circle;
+ }
+ case ENVELOPE:
+ {
+ double xMin = input.readDouble();
+ double xMax = input.readDouble();
+ double yMin = input.readDouble();
+ double yMax = input.readDouble();
+ if (xMin <= xMax) {
+ return new Envelope(xMin, xMax, yMin, yMax);
+ } else {
+ // Null envelope cannot be constructed using Envelope(xMin, xMax, yMin, yMax)
+ return new Envelope();
+ }
+ }
+ case OTHER:
+ {
+ return kryo.readClassAndObject(input);
+ }
+ default:
+ throw new UnsupportedOperationException(
+ "Cannot deserialize object of type " + geometryType);
+ }
+ }
+
+ private Object readUserData(Kryo kryo, Input input) {
+ Object userData = null;
+ if (input.readBoolean()) {
+ Registration clazz = kryo.readClass(input);
+ userData = kryo.readObject(input, clazz.getType());
+ }
+ return userData;
+ }
+
+// private Geometry readGeometry(Kryo kryo, Input input) {
+// int length = input.readInt();
+// byte[] bytes = new byte[length];
+// input.readBytes(bytes);
+// Geometry geometry = GeometrySerializer.deserialize(bytes);
+// geometry.setUserData(readUserData(kryo, input));
+// return geometry;
+// }
+
+ private enum Type {
+ SHAPE(0),
+ CIRCLE(1),
+ ENVELOPE(2),
+
+ OTHER(3)
+ ;
+
+ private final int id;
+
+ Type(int id) {
+ this.id = id;
+ }
+
+ public static Type fromId(int id) {
+ for (Type type : values()) {
+ if (type.id == id) {
+ return type;
+ }
+ }
+
+ return null;
+ }
+ }
+}
diff --git a/jena-geosparql/src/main/java/org/apache/jena/geosparql/spatial/index/v2/serde/sedona/CustomSpatialIndexSerde.java b/jena-geosparql/src/main/java/org/apache/jena/geosparql/spatial/index/v2/serde/sedona/CustomSpatialIndexSerde.java
new file mode 100644
index 00000000000..93af643855c
--- /dev/null
+++ b/jena-geosparql/src/main/java/org/apache/jena/geosparql/spatial/index/v2/serde/sedona/CustomSpatialIndexSerde.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.geosparql.spatial.index.v2.serde.sedona;
+
+import java.io.Serializable;
+
+import org.locationtech.jts.index.quadtree.IndexSerde;
+import org.locationtech.jts.index.quadtree.Quadtree;
+import org.locationtech.jts.index.strtree.STRtree;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+
+/* This file is an adapter copy of apache-sedona org.apache.sedona.common.geometrySerde.SpatialIndexSerde */
+/* The change is, that geometry read/write is delegated to a GeometrySerdeAdapter instance. */
+
+/**
+ * Provides methods to efficiently serialize and deserialize spatialIndex types.
+ *
+ *
Support Quadtree, STRtree types
+ *
+ *
trees are serialized recursively.
+ */
+public class CustomSpatialIndexSerde extends Serializer implements Serializable {
+
+ private final CustomGeometrySerde geometrySerde;
+
+ public CustomSpatialIndexSerde(CustomGeometrySerde geometrySerde) {
+ super();
+ this.geometrySerde = geometrySerde;
+ }
+
+ @Override
+ public void write(Kryo kryo, Output output, Object o) {
+ if (o instanceof Quadtree) {
+ // serialize quadtree index
+ writeType(output, Type.QUADTREE);
+ Quadtree tree = (Quadtree) o;
+ IndexSerde indexSerde = new IndexSerde();
+ indexSerde.write(kryo, output, tree);
+ } else if (o instanceof STRtree) {
+ // serialize rtree index
+ writeType(output, Type.RTREE);
+ STRtree tree = (STRtree) o;
+ org.locationtech.jts.index.strtree.CustomIndexSerdeV2 indexSerde =
+ new org.locationtech.jts.index.strtree.CustomIndexSerdeV2(geometrySerde);
+ indexSerde.write(kryo, output, tree);
+ } else {
+ throw new UnsupportedOperationException(" index type not supported ");
+ }
+ }
+
+ @Override
+ public Object read(Kryo kryo, Input input, Class aClass) {
+ byte typeID = input.readByte();
+ Type indexType = Type.fromId(typeID);
+ switch (indexType) {
+ case QUADTREE:
+ {
+ IndexSerde indexSerde = new IndexSerde();
+ return indexSerde.read(kryo, input);
+ }
+ case RTREE:
+ {
+ org.locationtech.jts.index.strtree.CustomIndexSerdeV2 indexSerde =
+ new org.locationtech.jts.index.strtree.CustomIndexSerdeV2(geometrySerde);
+ return indexSerde.read(kryo, input);
+ }
+ default:
+ {
+ throw new UnsupportedOperationException(
+ "can't deserialize spatial index of type" + indexType);
+ }
+ }
+ }
+
+ private void writeType(Output output, Type type) {
+ output.writeByte((byte) type.id);
+ }
+
+ private enum Type {
+ QUADTREE(0),
+ RTREE(1);
+
+ private final int id;
+
+ Type(int id) {
+ this.id = id;
+ }
+
+ public static Type fromId(int id) {
+ for (Type type : values()) {
+ if (type.id == id) {
+ return type;
+ }
+ }
+
+ return null;
+ }
+ }
+}
+
diff --git a/jena-geosparql/src/main/java/org/apache/jena/geosparql/spatial/property_functions/GenericSpatialPropertyFunction.java b/jena-geosparql/src/main/java/org/apache/jena/geosparql/spatial/property_functions/GenericSpatialPropertyFunction.java
index aa1b2e3427e..855665d142f 100644
--- a/jena-geosparql/src/main/java/org/apache/jena/geosparql/spatial/property_functions/GenericSpatialPropertyFunction.java
+++ b/jena-geosparql/src/main/java/org/apache/jena/geosparql/spatial/property_functions/GenericSpatialPropertyFunction.java
@@ -18,7 +18,7 @@
package org.apache.jena.geosparql.spatial.property_functions;
import java.util.Arrays;
-import java.util.HashSet;
+import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.stream.Stream;
@@ -31,12 +31,12 @@
import org.apache.jena.geosparql.implementation.vocabulary.SpatialExtension;
import org.apache.jena.geosparql.spatial.ConvertLatLon;
import org.apache.jena.geosparql.spatial.SearchEnvelope;
-import org.apache.jena.geosparql.spatial.SpatialIndex;
import org.apache.jena.geosparql.spatial.SpatialIndexException;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndex;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndexUtils;
import org.apache.jena.graph.Graph;
import org.apache.jena.graph.Node;
import org.apache.jena.graph.Triple;
-import org.apache.jena.rdf.model.Resource;
import org.apache.jena.sparql.core.Var;
import org.apache.jena.sparql.engine.ExecutionContext;
import org.apache.jena.sparql.engine.QueryIterator;
@@ -66,7 +66,7 @@ public abstract class GenericSpatialPropertyFunction extends PFuncSimpleAndList
@Override
public final QueryIterator execEvaluated(Binding binding, Node subject, Node predicate, PropFuncArg object, ExecutionContext execCxt) {
try {
- spatialIndex = SpatialIndex.retrieve(execCxt);
+ spatialIndex = SpatialIndexUtils.retrieve(execCxt);
spatialArguments = extractObjectArguments(predicate, object, spatialIndex.getSrsInfo());
return search(binding, execCxt, subject, spatialArguments.limit);
} catch (SpatialIndexException ex) {
@@ -184,15 +184,35 @@ private QueryIterator checkUnbound(Binding binding, ExecutionContext execCxt, No
//Find all Features in the spatial index which are within the rough search envelope.
SearchEnvelope searchEnvelope = spatialArguments.searchEnvelope;
- HashSet features = searchEnvelope.check(spatialIndex);
+ Graph activeGraph = execCxt.getActiveGraph();
+
+ Node graphName = SpatialIndexUtils.unwrapGraphName(activeGraph);
+ Collection features = searchEnvelope.check(spatialIndex, graphName);
+// Collection features;
+//
+//// FIXME: Confirm that we no longer need to consider symSpatialIndexPerGraph. We just pass the graph to the index and if the index
+//// supports named graphs than it will restrict matches to that graph - otherwise we'll just get more results with more post-processing work.
+// if (!execCxt.getDataset().getContext().get(SpatialIndexUtils.symSpatialIndexPerGraph, false)) {
+// // no index per graph activated, thus, fallback to query the default graph spatial index tree only
+// // which is the default behaviour
+// features = searchEnvelope.check(spatialIndex, null);
+// } else {
+// // check if context is a named graph, if so use to query only the corresponding spatial index tree
+// // otherwise, query only the default graph spatial index tree
+// Node graphName = SpatialIndexUtils.unwrapGraphName(activeGraph);
+// features = searchEnvelope.check(spatialIndex, graphName);
+//// features = graphName != null
+//// ? searchEnvelope.check(spatialIndex, graphName)
+//// : searchEnvelope.check(spatialIndex);
+// }
Var subjectVar = Var.alloc(subject.getName());
- Stream stream = features.stream();
+ Stream stream = features.stream();
if (requireSecondFilter()) {
- stream = stream.filter(feature -> checkBound(execCxt, feature.asNode()));
+ stream = stream.filter(feature -> checkBound(execCxt, feature));
}
- Iterator iterator = stream.map(feature -> BindingFactory.binding(binding, subjectVar, feature.asNode()))
+ Iterator iterator = stream.map(feature -> BindingFactory.binding(binding, subjectVar, feature))
.limit(limit)
.iterator();
return QueryIterPlainWrapper.create(iterator, execCxt);
diff --git a/jena-geosparql/src/main/java/org/locationtech/jts/index/strtree/CustomIndexSerdeV2.java b/jena-geosparql/src/main/java/org/locationtech/jts/index/strtree/CustomIndexSerdeV2.java
new file mode 100644
index 00000000000..bcca5ae2cc8
--- /dev/null
+++ b/jena-geosparql/src/main/java/org/locationtech/jts/index/strtree/CustomIndexSerdeV2.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.locationtech.jts.index.strtree;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.geosparql.spatial.index.v2.serde.sedona.CustomGeometrySerde;
+import org.locationtech.jts.geom.Envelope;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+
+/* This file is an adapted copy of org.locationtech.jts.index.strtree.IndexSerde. */
+/* The change is, that geometry read/write is delegated to a GeometrySerdeAdapter instance. */
+
+/**
+ * Provides methods to efficiently serialize and deserialize the index. trees are serialized
+ * recursively.
+ */
+public class CustomIndexSerdeV2 {
+ protected CustomGeometrySerde geometrySerde;
+
+ public CustomIndexSerdeV2(CustomGeometrySerde geometrySerde) {
+ this.geometrySerde = geometrySerde;
+ }
+
+ public Object read(Kryo kryo, Input input) {
+ int nodeCapacity = input.readInt();
+ boolean notEmpty = (input.readByte() & 0x01) == 1;
+ if (notEmpty) {
+ boolean built = (input.readByte() & 0x01) == 1;
+ if (built) {
+ // if built, root is not null, set itemBoundables to null
+ STRtree index = new STRtree(nodeCapacity, readSTRtreeNode(kryo, input));
+ return index;
+ } else {
+ // if not built, just read itemBoundables
+ ArrayList itemBoundables = new ArrayList();
+ int itemSize = input.readInt();
+ for (int i = 0; i < itemSize; ++i) {
+ itemBoundables.add(readItemBoundable(kryo, input));
+ }
+ STRtree index = new STRtree(nodeCapacity, itemBoundables);
+ return index;
+ }
+ } else {
+ return new STRtree(nodeCapacity);
+ }
+ }
+
+ public void write(Kryo kryo, Output output, STRtree tree) {
+ output.writeInt(tree.getNodeCapacity());
+ if (tree.isEmpty()) {
+ output.writeByte(0);
+ } else {
+ output.writeByte(1);
+ // write head
+ boolean isBuilt = tree.getItemBoundables() == null;
+ output.writeByte(isBuilt ? 1 : 0);
+ if (!isBuilt) {
+ // if not built, itemBoundables will not be null, record it
+ ArrayList itemBoundables = tree.getItemBoundables();
+ output.writeInt(itemBoundables.size());
+ for (Object obj : itemBoundables) {
+ if (!(obj instanceof ItemBoundable)) {
+ throw new UnsupportedOperationException(
+ " itemBoundables should only contain ItemBoundable objects ");
+ }
+ ItemBoundable itemBoundable = (ItemBoundable) obj;
+ // write envelope
+ writeItemBoundable(kryo, output, itemBoundable);
+ }
+ } else {
+ // if built, write from root
+ writeSTRTreeNode(kryo, output, tree.getRoot());
+ }
+ }
+ }
+
+ private void writeSTRTreeNode(Kryo kryo, Output output, AbstractNode node) {
+ // write head
+ output.writeInt(node.getLevel());
+ // write children
+ List children = node.getChildBoundables();
+ int childrenSize = children.size();
+ output.writeInt(childrenSize);
+ // if children not empty, write children
+ if (childrenSize > 0) {
+ if (children.get(0) instanceof AbstractNode) {
+ // write type as 0, non-leaf node
+ output.writeByte(0);
+ for (Object obj : children) {
+ AbstractNode child = (AbstractNode) obj;
+ writeSTRTreeNode(kryo, output, child);
+ }
+ } else if (children.get(0) instanceof ItemBoundable) {
+ // write type as 1, leaf node
+ output.writeByte(1);
+ // for leaf node, write items
+ for (Object obj : children) {
+ writeItemBoundable(kryo, output, (ItemBoundable) obj);
+ }
+ } else {
+ throw new UnsupportedOperationException("wrong node type of STRtree");
+ }
+ }
+ }
+
+ private STRtree.STRtreeNode readSTRtreeNode(Kryo kryo, Input input) {
+ int level = input.readInt();
+ STRtree.STRtreeNode node = new STRtree.STRtreeNode(level);
+ int childrenSize = input.readInt();
+ boolean isLeaf = (input.readByte() & 0x01) == 1;
+ ArrayList children = new ArrayList();
+ if (isLeaf) {
+ for (int i = 0; i < childrenSize; ++i) {
+ children.add(readItemBoundable(kryo, input));
+ }
+ } else {
+ for (int i = 0; i < childrenSize; ++i) {
+ children.add(readSTRtreeNode(kryo, input));
+ }
+ }
+ node.setChildBoundables(children);
+ return node;
+ }
+
+ private void writeItemBoundable(Kryo kryo, Output output, ItemBoundable itemBoundable) {
+ geometrySerde.write(kryo, output, itemBoundable.getBounds());
+ geometrySerde.write(kryo, output, itemBoundable.getItem());
+ }
+
+ private ItemBoundable readItemBoundable(Kryo kryo, Input input) {
+ return new ItemBoundable(
+ geometrySerde.read(kryo, input, Envelope.class),
+ geometrySerde.read(kryo, input, Object.class));
+ }
+}
diff --git a/jena-geosparql/src/test/java/org/apache/jena/geosparql/geo/topological/CancelQueryTest.java b/jena-geosparql/src/test/java/org/apache/jena/geosparql/geo/topological/CancelQueryTest.java
index 7b54ebb9049..c3da3ee0919 100644
--- a/jena-geosparql/src/test/java/org/apache/jena/geosparql/geo/topological/CancelQueryTest.java
+++ b/jena-geosparql/src/test/java/org/apache/jena/geosparql/geo/topological/CancelQueryTest.java
@@ -31,8 +31,9 @@
import org.apache.jena.geosparql.implementation.datatype.WKTDatatype;
import org.apache.jena.geosparql.implementation.index.IndexConfiguration;
import org.apache.jena.geosparql.implementation.vocabulary.Geo;
-import org.apache.jena.geosparql.spatial.SpatialIndex;
import org.apache.jena.geosparql.spatial.SpatialIndexException;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndex;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndexUtils;
import org.apache.jena.graph.Graph;
import org.apache.jena.graph.NodeFactory;
import org.apache.jena.query.Dataset;
@@ -106,8 +107,8 @@ public void test_cancel_spatial_property_function1() {
// create spatial index
if (useIndex){
try {
- SpatialIndex index = SpatialIndex.buildSpatialIndex(ds);
- SpatialIndex.setSpatialIndex(ds, index);
+ SpatialIndex index = SpatialIndexUtils.buildSpatialIndex(ds.asDatasetGraph());
+ SpatialIndexUtils.setSpatialIndex(ds, index);
} catch (SpatialIndexException e) {
throw new RuntimeException(e);
}
diff --git a/jena-geosparql/src/test/java/org/apache/jena/geosparql/geo/topological/GenericPropertyFunctionTest.java b/jena-geosparql/src/test/java/org/apache/jena/geosparql/geo/topological/GenericPropertyFunctionTest.java
index 5a99718258f..376a3dd714b 100644
--- a/jena-geosparql/src/test/java/org/apache/jena/geosparql/geo/topological/GenericPropertyFunctionTest.java
+++ b/jena-geosparql/src/test/java/org/apache/jena/geosparql/geo/topological/GenericPropertyFunctionTest.java
@@ -17,7 +17,17 @@
*/
package org.apache.jena.geosparql.geo.topological;
-import static org.apache.jena.geosparql.geo.topological.QueryRewriteTestData.*;
+import static org.apache.jena.geosparql.geo.topological.QueryRewriteTestData.FEATURE_A;
+import static org.apache.jena.geosparql.geo.topological.QueryRewriteTestData.FEATURE_B;
+import static org.apache.jena.geosparql.geo.topological.QueryRewriteTestData.FEATURE_D;
+import static org.apache.jena.geosparql.geo.topological.QueryRewriteTestData.GEOMETRY_A;
+import static org.apache.jena.geosparql.geo.topological.QueryRewriteTestData.GEOMETRY_B;
+import static org.apache.jena.geosparql.geo.topological.QueryRewriteTestData.GEOMETRY_C_BLANK;
+import static org.apache.jena.geosparql.geo.topological.QueryRewriteTestData.GEOMETRY_D;
+import static org.apache.jena.geosparql.geo.topological.QueryRewriteTestData.GEOMETRY_F;
+import static org.apache.jena.geosparql.geo.topological.QueryRewriteTestData.GEO_FEATURE_Y;
+import static org.apache.jena.geosparql.geo.topological.QueryRewriteTestData.GEO_FEATURE_Z;
+import static org.apache.jena.geosparql.geo.topological.QueryRewriteTestData.TEST_SRS_URI;
import static org.junit.Assert.assertEquals;
import java.util.ArrayList;
@@ -30,16 +40,24 @@
import org.apache.jena.geosparql.implementation.index.IndexConfiguration.IndexOption;
import org.apache.jena.geosparql.implementation.index.QueryRewriteIndex;
import org.apache.jena.geosparql.implementation.vocabulary.Geo;
-import org.apache.jena.geosparql.spatial.SpatialIndex;
import org.apache.jena.geosparql.spatial.SpatialIndexException;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndexUtils;
import org.apache.jena.graph.Graph;
import org.apache.jena.graph.Node;
import org.apache.jena.graph.NodeFactory;
-import org.apache.jena.query.*;
+import org.apache.jena.query.Dataset;
+import org.apache.jena.query.QueryExecution;
+import org.apache.jena.query.QueryExecutionFactory;
+import org.apache.jena.query.QuerySolution;
+import org.apache.jena.query.ResultSet;
import org.apache.jena.rdf.model.Model;
import org.apache.jena.rdf.model.Resource;
import org.apache.jena.vocabulary.RDF;
-import org.junit.*;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
/**
*
@@ -57,7 +75,7 @@ public GenericPropertyFunctionTest() {
public static void setUpClass() throws SpatialIndexException {
GeoSPARQLConfig.setup(IndexOption.MEMORY, Boolean.TRUE);
model = QueryRewriteTestData.createTestData();
- dataset = SpatialIndex.wrapModel(model, TEST_SRS_URI);
+ dataset = SpatialIndexUtils.wrapModel(model, TEST_SRS_URI);
}
@AfterClass
diff --git a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/AbstractSpatialIndexGraphLookpTest.java b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/AbstractSpatialIndexGraphLookpTest.java
new file mode 100644
index 00000000000..b34658549b7
--- /dev/null
+++ b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/AbstractSpatialIndexGraphLookpTest.java
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.geosparql.spatial;
+
+import org.apache.jena.geosparql.implementation.vocabulary.SRS_URI;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndex;
+import org.apache.jena.query.ResultSet;
+import org.apache.jena.query.ResultSetFormatter;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFParserBuilder;
+import org.apache.jena.sparql.algebra.Table;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.exec.QueryExec;
+import org.junit.Assert;
+import org.junit.Test;
+
+public abstract class AbstractSpatialIndexGraphLookpTest {
+ protected abstract SpatialIndex buildSpatialIndex(DatasetGraph dsg, String srsUri) throws SpatialIndexException;
+
+ private static boolean enableDebugPrint = false;
+
+ private static void debugPrint(Table table) {
+ if (enableDebugPrint) {
+ System.err.println(ResultSetFormatter.asText(ResultSet.adapt(table.toRowSet())));
+ }
+ }
+
+ // SpatialIndexUtils.buildSpatialIndex(dsg, SRS_URI.DEFAULT_WKT_CRS84);
+ @Test
+ public void mustNotMatchDefaultGraph1() throws SpatialIndexException {
+ DatasetGraph dsg = RDFParserBuilder.create().fromString( """
+ PREFIX eg:
+ PREFIX geo:
+
+ eg:graph1 {
+ eg:feature1 geo:hasGeometry eg:geometry1 .
+ eg:geometry1 geo:asWKT "POINT (0.3 0.3)"^^geo:wktLiteral .
+ }
+
+ eg:graph2 {
+ eg:feature1 geo:hasGeometry eg:geometry1 .
+ eg:geometry1 geo:asWKT "POINT (0.7 0.7)"^^geo:wktLiteral .
+ }
+ """).lang(Lang.TRIG).toDatasetGraph();
+
+ String queryStr = """
+ PREFIX eg:
+ PREFIX spatial:
+ PREFIX geo:
+
+ SELECT * {
+ VALUES ?search {
+ "POLYGON ((0 0, 1 0, 1 1, 0 1, 0 0))"^^geo:wktLiteral
+ }
+ LATERAL {
+ # GRAPH eg:graph1 { ?feature spatial:intersectBoxGeom(?search) . }
+ ?feature spatial:intersectBoxGeom(?search) .
+ }
+ }
+ """;
+
+ buildSpatialIndex(dsg, SRS_URI.DEFAULT_WKT_CRS84);
+ Table table = QueryExec.dataset(dsg).query(queryStr).table();
+ debugPrint(table);
+ Assert.assertTrue(table.isEmpty());
+ }
+
+ @Test
+ public void mustNotMatchDefaultGraph2() throws SpatialIndexException {
+ DatasetGraph dsg = RDFParserBuilder.create().fromString( """
+ PREFIX eg:
+ PREFIX geo:
+
+ # Feature in default graph is outside of query polygon
+ eg:feature1 geo:hasGeometry eg:geometry1 .
+ eg:geometry1 geo:asWKT "POINT (-10 -10)"^^geo:wktLiteral .
+
+ eg:graph1 {
+ eg:feature1 geo:hasGeometry eg:geometry1 .
+ eg:geometry1 geo:asWKT "POINT (0.3 0.3)"^^geo:wktLiteral .
+ }
+
+ eg:graph2 {
+ eg:feature1 geo:hasGeometry eg:geometry1 .
+ eg:geometry1 geo:asWKT "POINT (0.7 0.7)"^^geo:wktLiteral .
+ }
+ """).lang(Lang.TRIG).toDatasetGraph();
+
+ String queryStr = """
+ PREFIX eg:
+ PREFIX spatial:
+ PREFIX geo:
+
+ SELECT * {
+ VALUES ?search {
+ "POLYGON ((0 0, 1 0, 1 1, 0 1, 0 0))"^^geo:wktLiteral
+ }
+ LATERAL {
+ ?feature spatial:intersectBoxGeom(?search) .
+ }
+ }
+ """;
+
+ buildSpatialIndex(dsg, SRS_URI.DEFAULT_WKT_CRS84);
+ Table table = QueryExec.dataset(dsg).query(queryStr).table();
+ debugPrint(table);
+ Assert.assertTrue(table.isEmpty());
+ }
+
+ @Test
+ public void mustNotMatchNamedGraph() throws SpatialIndexException {
+ DatasetGraph dsg = RDFParserBuilder.create().fromString( """
+ PREFIX eg:
+ PREFIX geo:
+
+ eg:graph1 {
+ eg:feature1 geo:hasGeometry eg:geometry1 .
+ eg:geometry1 geo:asWKT "POINT (-10 -10)"^^geo:wktLiteral .
+ }
+
+ eg:graph2 {
+ eg:feature1 geo:hasGeometry eg:geometry1 .
+ eg:geometry1 geo:asWKT "POINT (0.7 0.7)"^^geo:wktLiteral .
+ }
+ """).lang(Lang.TRIG).toDatasetGraph();
+
+ String queryStr = """
+ PREFIX eg:
+ PREFIX spatial:
+ PREFIX geo:
+
+ SELECT * {
+ VALUES ?search {
+ "POLYGON ((0 0, 1 0, 1 1, 0 1, 0 0))"^^geo:wktLiteral
+ }
+ LATERAL {
+ GRAPH eg:graph1 { ?feature spatial:intersectBoxGeom(?search) . }
+ }
+ }
+ """;
+
+ buildSpatialIndex(dsg, SRS_URI.DEFAULT_WKT_CRS84);
+ Table table = QueryExec.dataset(dsg).query(queryStr).table();
+ debugPrint(table);
+ Assert.assertTrue(table.isEmpty());
+ }
+
+ @Test
+ public void mustMatchNamedGraph() throws SpatialIndexException {
+ DatasetGraph dsg = RDFParserBuilder.create().fromString( """
+ PREFIX eg:
+ PREFIX geo:
+
+ eg:graph1 {
+ eg:feature1 geo:hasGeometry eg:geometry1 .
+ eg:geometry1 geo:asWKT "POINT (-10 -10)"^^geo:wktLiteral .
+ }
+
+ eg:graph2 {
+ eg:feature1 geo:hasGeometry eg:geometry1 .
+ eg:geometry1 geo:asWKT "POINT (0.7 0.7)"^^geo:wktLiteral .
+ }
+ """).lang(Lang.TRIG).toDatasetGraph();
+
+ String queryStr = """
+ PREFIX eg:
+ PREFIX spatial:
+ PREFIX geo:
+
+ SELECT * {
+ VALUES ?search {
+ "POLYGON ((0 0, 1 0, 1 1, 0 1, 0 0))"^^geo:wktLiteral
+ }
+ LATERAL {
+ GRAPH eg:graph2 { ?feature spatial:intersectBoxGeom(?search) . }
+ }
+ }
+ """;
+
+ buildSpatialIndex(dsg, SRS_URI.DEFAULT_WKT_CRS84);
+ Table table = QueryExec.dataset(dsg).query(queryStr).table();
+ debugPrint(table);
+ Assert.assertFalse(table.isEmpty());
+ }
+}
diff --git a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/SearchEnvelopeTest.java b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/SearchEnvelopeTest.java
index 371f18f1ad0..8fcf28d586a 100644
--- a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/SearchEnvelopeTest.java
+++ b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/SearchEnvelopeTest.java
@@ -17,15 +17,18 @@
*/
package org.apache.jena.geosparql.spatial;
-import java.util.Arrays;
-import java.util.HashSet;
+import static org.junit.Assert.assertEquals;
+
+import java.util.Collection;
+import java.util.Set;
+
import org.apache.jena.geosparql.implementation.GeometryWrapper;
import org.apache.jena.geosparql.implementation.datatype.WKTDatatype;
import org.apache.jena.geosparql.implementation.vocabulary.Unit_URI;
-import org.apache.jena.rdf.model.Resource;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndex;
+import org.apache.jena.graph.Node;
import org.junit.After;
import org.junit.AfterClass;
-import static org.junit.Assert.assertEquals;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -364,8 +367,8 @@ public void testCheck() {
SearchEnvelope instance = SearchEnvelope.build(geometryWrapper, SpatialIndexTestData.WGS_84_SRS_INFO, radius, unitsURI);
//Function Test
- HashSet expResult = new HashSet<>(Arrays.asList(SpatialIndexTestData.LONDON_FEATURE));
- HashSet result = instance.check(spatialIndex);
+ Set expResult = Set.of(SpatialIndexTestData.LONDON_FEATURE.asNode());
+ Collection result = instance.check(spatialIndex);
assertEquals(expResult, result);
}
@@ -384,8 +387,8 @@ public void testCheck_empty() {
SearchEnvelope instance = SearchEnvelope.build(geometryWrapper, SpatialIndexTestData.WGS_84_SRS_INFO, radius, unitsURI);
//Function Test
- HashSet expResult = new HashSet<>();
- HashSet result = instance.check(spatialIndex);
+ Collection expResult = Set.of();
+ Collection result = instance.check(spatialIndex);
assertEquals(expResult, result);
}
diff --git a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/SpatialIndexTest.java b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/SpatialIndexTest.java
new file mode 100644
index 00000000000..e3af8f2784b
--- /dev/null
+++ b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/SpatialIndexTest.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.geosparql.spatial;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.jena.geosparql.implementation.GeometryWrapperFactory;
+import org.apache.jena.geosparql.implementation.SRSInfo;
+import org.apache.jena.geosparql.implementation.datatype.WKTDatatype;
+import org.apache.jena.geosparql.implementation.vocabulary.SRS_URI;
+import org.apache.jena.geosparql.spatial.index.compat.SpatialIndexIo;
+import org.apache.jena.geosparql.spatial.index.v1.SpatialIndexV1;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndex;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndexIoKryo;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndexPerGraph;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndexUtils;
+import org.apache.jena.graph.Node;
+import org.apache.jena.query.ResultSet;
+import org.apache.jena.query.ResultSetFormatter;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFParserBuilder;
+import org.apache.jena.sparql.algebra.Table;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.exec.QueryExec;
+import org.junit.Assert;
+import org.junit.Test;
+import org.locationtech.jts.geom.Envelope;
+
+public class SpatialIndexTest {
+
+ @Test
+ public void testLegacyLoading() throws IOException, SpatialIndexException {
+ Path file = Files.createTempFile("jena", "spatial.index");
+ try {
+ List items = SpatialIndexTestData.getTestItems();
+
+ SpatialIndexV1.save(file.toFile(), items, SRS_URI.DEFAULT_WKT_CRS84);
+
+ SpatialIndex index = SpatialIndexIo.load(file, true);
+ Envelope envelope = new Envelope(-90, 0, 0, 90);
+ Collection actual = index.query(envelope, null);
+ Set expected = Set.of(SpatialIndexTestData.LONDON_FEATURE.asNode(), SpatialIndexTestData.NEW_YORK_FEATURE.asNode());
+ Assert.assertEquals(expected, actual);
+ } finally {
+ Files.delete(file);
+ }
+ }
+
+ @Test
+ public void testSerdeSpatialIndex() throws IOException, SpatialIndexException {
+ // create spatial index
+ SpatialIndexPerGraph index1 = SpatialIndexTestData.createTestIndex();
+
+ // query index 1
+ SRSInfo srsInfo1 = index1.getSrsInfo();
+ SearchEnvelope searchEnvelope1 = SearchEnvelope.build(GeometryWrapperFactory.createPolygon(srsInfo1.getDomainEnvelope(), WKTDatatype.URI), srsInfo1);
+ Collection res1 = searchEnvelope1.check(index1);
+
+ // save to tmp file
+ // File file = new File("/tmp/test-spatial.index"); //File.createTempFile( "jena", "spatial.index");
+ Path file = Files.createTempFile("jena", "spatial.index");
+ try {
+ SpatialIndexIoKryo.save(file, index1);
+
+ // load from tmp file as new index 2
+ SpatialIndex index2 = SpatialIndexIo.load(file);
+
+ // query index 2
+ SRSInfo srsInfo2 = index2.getSrsInfo();
+ SearchEnvelope searchEnvelope2 = SearchEnvelope.build(GeometryWrapperFactory.createPolygon(srsInfo2.getDomainEnvelope(), WKTDatatype.URI), srsInfo2);
+ Collection res2 = searchEnvelope2.check(index2);
+
+ assertEquals(srsInfo1, srsInfo2);
+ assertEquals(res1, res2);
+ } finally {
+ Files.deleteIfExists(file);
+ }
+ }
+
+}
diff --git a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/SpatialIndexTestData.java b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/SpatialIndexTestData.java
index b3ec8949988..074607deb8f 100644
--- a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/SpatialIndexTestData.java
+++ b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/SpatialIndexTestData.java
@@ -17,17 +17,29 @@
*/
package org.apache.jena.geosparql.spatial;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
import org.apache.jena.geosparql.implementation.GeometryWrapper;
import org.apache.jena.geosparql.implementation.SRSInfo;
import org.apache.jena.geosparql.implementation.datatype.WKTDatatype;
import org.apache.jena.geosparql.implementation.vocabulary.Geo;
import org.apache.jena.geosparql.implementation.vocabulary.SRS_URI;
+import org.apache.jena.geosparql.spatial.index.v2.STRtreePerGraph;
+import org.apache.jena.geosparql.spatial.index.v2.STRtreeUtils;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndex;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndexPerGraph;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndexUtils;
+import org.apache.jena.graph.Node;
import org.apache.jena.query.Dataset;
import org.apache.jena.query.DatasetFactory;
import org.apache.jena.rdf.model.Model;
import org.apache.jena.rdf.model.ModelFactory;
import org.apache.jena.rdf.model.Resource;
import org.apache.jena.rdf.model.ResourceFactory;
+import org.locationtech.jts.index.strtree.STRtree;
/**
*
@@ -60,21 +72,28 @@ public class SpatialIndexTestData {
public static final SRSInfo WGS_84_SRS_INFO = new SRSInfo(SRS_URI.WGS84_CRS);
public static final SRSInfo OSGB_SRS_INFO = new SRSInfo(SRS_URI.OSGB36_CRS);
- private static SpatialIndex TEST_SPATIAL_INDEX = null;
+ private static SpatialIndexPerGraph TEST_SPATIAL_INDEX = null;
private static Dataset TEST_DATASET = null;
- public static final SpatialIndex createTestIndex() {
+ public static final List getTestItems() {
+ List items = List.of(
+ SpatialIndexItem.of(LONDON_GEOMETRY_WRAPPER.getEnvelope(), LONDON_FEATURE),
+ SpatialIndexItem.of(NEW_YORK_GEOMETRY_WRAPPER.getEnvelope(), NEW_YORK_FEATURE),
+ SpatialIndexItem.of(HONOLULU_GEOMETRY_WRAPPER.getEnvelope(), HONOLULU_FEATURE),
+ SpatialIndexItem.of(PERTH_GEOMETRY_WRAPPER.getEnvelope(), PERTH_FEATURE),
+ SpatialIndexItem.of(AUCKLAND_GEOMETRY_WRAPPER.getEnvelope(), AUCKLAND_FEATURE));
+ return items;
+ }
+
+ public static final SpatialIndexPerGraph createTestIndex() {
if (TEST_SPATIAL_INDEX == null) {
try {
- SpatialIndex spatialIndex = new SpatialIndex(100, SRS_URI.WGS84_CRS);
- spatialIndex.insertItem(LONDON_GEOMETRY_WRAPPER.getEnvelope(), LONDON_FEATURE);
- spatialIndex.insertItem(NEW_YORK_GEOMETRY_WRAPPER.getEnvelope(), NEW_YORK_FEATURE);
- spatialIndex.insertItem(HONOLULU_GEOMETRY_WRAPPER.getEnvelope(), HONOLULU_FEATURE);
- spatialIndex.insertItem(PERTH_GEOMETRY_WRAPPER.getEnvelope(), PERTH_FEATURE);
- spatialIndex.insertItem(AUCKLAND_GEOMETRY_WRAPPER.getEnvelope(), AUCKLAND_FEATURE);
-
- spatialIndex.build();
+ // SpatialIndexPerGraph spatialIndex = new SpatialIndexPerGraph(100, SRS_URI.WGS84_CRS);
+ List items = getTestItems();
+ STRtree tree = STRtreeUtils.buildSpatialIndexTree(items);
+ STRtreePerGraph index = new STRtreePerGraph(tree);
+ SpatialIndexPerGraph spatialIndex = new SpatialIndexPerGraph(index);
TEST_SPATIAL_INDEX = spatialIndex;
} catch (SpatialIndexException ex) {
@@ -102,11 +121,15 @@ public static final Dataset createTestDataset() {
dataset.setDefaultModel(model);
SpatialIndex spatialIndex = createTestIndex();
- SpatialIndex.setSpatialIndex(dataset, spatialIndex);
+ SpatialIndexUtils.setSpatialIndex(dataset, spatialIndex);
TEST_DATASET = dataset;
}
return TEST_DATASET;
}
+ public static Set asNodes(Collection resources) {
+ return resources.stream().map(Resource::asNode).collect(Collectors.toSet());
+ }
+
}
diff --git a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/TestSpatialIndexGraphLookupV1.java b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/TestSpatialIndexGraphLookupV1.java
new file mode 100644
index 00000000000..2f9bf97504b
--- /dev/null
+++ b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/TestSpatialIndexGraphLookupV1.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.geosparql.spatial;
+
+import org.apache.jena.geosparql.spatial.index.v1.SpatialIndexAdapterV1;
+import org.apache.jena.geosparql.spatial.index.v1.SpatialIndexV1;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndex;
+import org.apache.jena.query.DatasetFactory;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.junit.Ignore;
+
+@Ignore
+public class TestSpatialIndexGraphLookupV1
+ extends AbstractSpatialIndexGraphLookpTest
+{
+ @Override
+ protected SpatialIndex buildSpatialIndex(DatasetGraph dsg, String srsUri) throws SpatialIndexException {
+ SpatialIndexV1 v1 = SpatialIndexV1.buildSpatialIndex(DatasetFactory.wrap(dsg), srsUri);
+ return new SpatialIndexAdapterV1(v1);
+ }
+}
diff --git a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/TestSpatialIndexGraphLookupV2.java b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/TestSpatialIndexGraphLookupV2.java
new file mode 100644
index 00000000000..32a1eb94013
--- /dev/null
+++ b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/TestSpatialIndexGraphLookupV2.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.geosparql.spatial;
+
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndex;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndexUtils;
+import org.apache.jena.sparql.core.DatasetGraph;
+
+public class TestSpatialIndexGraphLookupV2
+ extends AbstractSpatialIndexGraphLookpTest
+{
+ @Override
+ protected SpatialIndex buildSpatialIndex(DatasetGraph dsg, String srsUri) throws SpatialIndexException {
+ return SpatialIndexUtils.buildSpatialIndex(dsg, srsUri);
+ }
+}
diff --git a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/GenericSpatialPropertyFunctionTest.java b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/GenericSpatialPropertyFunctionTest.java
index 724f4d2dacc..0b2ff3e6e88 100644
--- a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/GenericSpatialPropertyFunctionTest.java
+++ b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/GenericSpatialPropertyFunctionTest.java
@@ -17,15 +17,18 @@
*/
package org.apache.jena.geosparql.spatial.property_functions;
+import static org.junit.Assert.assertEquals;
+
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
+
import org.apache.jena.datatypes.xsd.XSDDatatype;
import org.apache.jena.geosparql.configuration.GeoSPARQLConfig;
import org.apache.jena.geosparql.implementation.vocabulary.SpatialExtension;
-import org.apache.jena.geosparql.spatial.SpatialIndex;
import org.apache.jena.geosparql.spatial.SpatialIndexException;
import org.apache.jena.geosparql.spatial.SpatialIndexTestData;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndexUtils;
import org.apache.jena.query.Dataset;
import org.apache.jena.query.QueryExecution;
import org.apache.jena.query.QueryExecutionFactory;
@@ -37,7 +40,6 @@
import org.apache.jena.rdf.model.ResourceFactory;
import org.junit.After;
import org.junit.AfterClass;
-import static org.junit.Assert.*;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -201,7 +203,7 @@ public void testExecEvaluated_Nearby_geo() throws SpatialIndexException {
Resource geoFeature = ResourceFactory.createResource("http://example.org/GeoFeatureX");
model.add(geoFeature, SpatialExtension.GEO_LAT_PROP, ResourceFactory.createTypedLiteral("0.0", XSDDatatype.XSDfloat));
model.add(geoFeature, SpatialExtension.GEO_LON_PROP, ResourceFactory.createTypedLiteral("0.0", XSDDatatype.XSDfloat));
- Dataset dataset = SpatialIndex.wrapModel(model);
+ Dataset dataset = SpatialIndexUtils.wrapModel(model);
String query = "PREFIX spatial: \n"
+ "\n"
diff --git a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/EastGeomPFTest.java b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/EastGeomPFTest.java
index 91e6cb00e73..436aad91879 100644
--- a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/EastGeomPFTest.java
+++ b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/EastGeomPFTest.java
@@ -17,16 +17,21 @@
*/
package org.apache.jena.geosparql.spatial.property_functions.cardinal;
+import static org.junit.Assert.assertEquals;
+
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.HashSet;
+import java.util.Collection;
import java.util.List;
+import java.util.Set;
+
import org.apache.jena.geosparql.configuration.GeoSPARQLConfig;
import org.apache.jena.geosparql.implementation.GeometryWrapper;
import org.apache.jena.geosparql.spatial.CardinalDirection;
import org.apache.jena.geosparql.spatial.SearchEnvelope;
-import org.apache.jena.geosparql.spatial.SpatialIndex;
import org.apache.jena.geosparql.spatial.SpatialIndexTestData;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndex;
+import org.apache.jena.graph.Node;
import org.apache.jena.query.Dataset;
import org.apache.jena.query.QueryExecution;
import org.apache.jena.query.QueryExecutionFactory;
@@ -35,7 +40,6 @@
import org.apache.jena.rdf.model.Resource;
import org.junit.After;
import org.junit.AfterClass;
-import static org.junit.Assert.*;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -91,8 +95,10 @@ public void testCheckSearchEnvelope_no_wrap() {
GeometryWrapper geometryWrapper = SpatialIndexTestData.HONOLULU_GEOMETRY_WRAPPER;
EastGeomPF instance = new EastGeomPF();
SearchEnvelope searchEnvelope = instance.buildSearchEnvelope(geometryWrapper, SpatialIndexTestData.WGS_84_SRS_INFO); //Needed to initialise the search.
- HashSet expResult = new HashSet<>(Arrays.asList(SpatialIndexTestData.LONDON_FEATURE, SpatialIndexTestData.HONOLULU_FEATURE, SpatialIndexTestData.NEW_YORK_FEATURE));
- HashSet result = searchEnvelope.check(spatialIndex);
+ Set expResult = SpatialIndexTestData.asNodes(
+ List.of(SpatialIndexTestData.LONDON_FEATURE, SpatialIndexTestData.HONOLULU_FEATURE, SpatialIndexTestData.NEW_YORK_FEATURE));
+ Collection result = searchEnvelope.check(spatialIndex);
+
assertEquals(expResult, result);
}
@@ -108,8 +114,9 @@ public void testCheckSearchEnvelope_wrap() {
GeometryWrapper geometryWrapper = SpatialIndexTestData.PERTH_GEOMETRY_WRAPPER;
EastGeomPF instance = new EastGeomPF();
SearchEnvelope searchEnvelope = instance.buildSearchEnvelope(geometryWrapper, SpatialIndexTestData.WGS_84_SRS_INFO); //Needed to initialise the search.
- HashSet expResult = new HashSet<>(Arrays.asList(SpatialIndexTestData.AUCKLAND_FEATURE, SpatialIndexTestData.PERTH_FEATURE, SpatialIndexTestData.HONOLULU_FEATURE, SpatialIndexTestData.NEW_YORK_FEATURE));
- HashSet result = searchEnvelope.check(spatialIndex);
+ Set expResult = SpatialIndexTestData.asNodes(
+ List.of(SpatialIndexTestData.AUCKLAND_FEATURE, SpatialIndexTestData.PERTH_FEATURE, SpatialIndexTestData.HONOLULU_FEATURE, SpatialIndexTestData.NEW_YORK_FEATURE));
+ Collection result = searchEnvelope.check(spatialIndex);
assertEquals(expResult, result);
}
diff --git a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/EastPFTest.java b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/EastPFTest.java
index 38896523c06..4e2f5f8c6b1 100644
--- a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/EastPFTest.java
+++ b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/EastPFTest.java
@@ -17,16 +17,15 @@
*/
package org.apache.jena.geosparql.spatial.property_functions.cardinal;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
+import java.util.*;
+
import org.apache.jena.geosparql.configuration.GeoSPARQLConfig;
import org.apache.jena.geosparql.implementation.GeometryWrapper;
import org.apache.jena.geosparql.spatial.CardinalDirection;
import org.apache.jena.geosparql.spatial.SearchEnvelope;
-import org.apache.jena.geosparql.spatial.SpatialIndex;
import org.apache.jena.geosparql.spatial.SpatialIndexTestData;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndex;
+import org.apache.jena.graph.Node;
import org.apache.jena.query.Dataset;
import org.apache.jena.query.QueryExecution;
import org.apache.jena.query.QueryExecutionFactory;
@@ -91,8 +90,8 @@ public void testCheckSearchEnvelope_no_wrap() {
GeometryWrapper geometryWrapper = SpatialIndexTestData.HONOLULU_GEOMETRY_WRAPPER;
EastPF instance = new EastPF();
SearchEnvelope searchEnvelope = instance.buildSearchEnvelope(geometryWrapper, SpatialIndexTestData.WGS_84_SRS_INFO); //Needed to initialise the search.
- HashSet expResult = new HashSet<>(Arrays.asList(SpatialIndexTestData.LONDON_FEATURE, SpatialIndexTestData.HONOLULU_FEATURE, SpatialIndexTestData.NEW_YORK_FEATURE));
- HashSet result = searchEnvelope.check(spatialIndex);
+ Set expResult = SpatialIndexTestData.asNodes(List.of(SpatialIndexTestData.LONDON_FEATURE, SpatialIndexTestData.HONOLULU_FEATURE, SpatialIndexTestData.NEW_YORK_FEATURE));
+ Collection result = searchEnvelope.check(spatialIndex);
assertEquals(expResult, result);
}
@@ -108,8 +107,8 @@ public void testCheckSearchEnvelope_wrap() {
GeometryWrapper geometryWrapper = SpatialIndexTestData.PERTH_GEOMETRY_WRAPPER;
EastPF instance = new EastPF();
SearchEnvelope searchEnvelope = instance.buildSearchEnvelope(geometryWrapper, SpatialIndexTestData.WGS_84_SRS_INFO); //Needed to initialise the search.
- HashSet expResult = new HashSet<>(Arrays.asList(SpatialIndexTestData.AUCKLAND_FEATURE, SpatialIndexTestData.PERTH_FEATURE, SpatialIndexTestData.HONOLULU_FEATURE, SpatialIndexTestData.NEW_YORK_FEATURE));
- HashSet result = searchEnvelope.check(spatialIndex);
+ Set expResult = SpatialIndexTestData.asNodes(List.of(SpatialIndexTestData.AUCKLAND_FEATURE, SpatialIndexTestData.PERTH_FEATURE, SpatialIndexTestData.HONOLULU_FEATURE, SpatialIndexTestData.NEW_YORK_FEATURE));
+ Collection result = searchEnvelope.check(spatialIndex);
assertEquals(expResult, result);
}
diff --git a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/WestGeomPFTest.java b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/WestGeomPFTest.java
index abda7afebf2..790f0e05e7b 100644
--- a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/WestGeomPFTest.java
+++ b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/WestGeomPFTest.java
@@ -17,16 +17,21 @@
*/
package org.apache.jena.geosparql.spatial.property_functions.cardinal;
+import static org.junit.Assert.assertEquals;
+
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.HashSet;
+import java.util.Collection;
import java.util.List;
+import java.util.Set;
+
import org.apache.jena.geosparql.configuration.GeoSPARQLConfig;
import org.apache.jena.geosparql.implementation.GeometryWrapper;
import org.apache.jena.geosparql.spatial.CardinalDirection;
import org.apache.jena.geosparql.spatial.SearchEnvelope;
-import org.apache.jena.geosparql.spatial.SpatialIndex;
import org.apache.jena.geosparql.spatial.SpatialIndexTestData;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndex;
+import org.apache.jena.graph.Node;
import org.apache.jena.query.Dataset;
import org.apache.jena.query.QueryExecution;
import org.apache.jena.query.QueryExecutionFactory;
@@ -35,7 +40,6 @@
import org.apache.jena.rdf.model.Resource;
import org.junit.After;
import org.junit.AfterClass;
-import static org.junit.Assert.*;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -91,8 +95,8 @@ public void testCheckSearchEnvelope_no_wrap() {
GeometryWrapper geometryWrapper = SpatialIndexTestData.PERTH_GEOMETRY_WRAPPER;
WestGeomPF instance = new WestGeomPF();
SearchEnvelope searchEnvelope = instance.buildSearchEnvelope(geometryWrapper, SpatialIndexTestData.WGS_84_SRS_INFO); //Needed to initialise the search.
- HashSet expResult = new HashSet<>(Arrays.asList(SpatialIndexTestData.LONDON_FEATURE, SpatialIndexTestData.PERTH_FEATURE));
- HashSet result = searchEnvelope.check(spatialIndex);
+ Set expResult = SpatialIndexTestData.asNodes(List.of(SpatialIndexTestData.LONDON_FEATURE, SpatialIndexTestData.PERTH_FEATURE));
+ Collection result = searchEnvelope.check(spatialIndex);
assertEquals(expResult, result);
}
@@ -108,8 +112,8 @@ public void testCheckSearchEnvelope_wrap() {
GeometryWrapper geometryWrapper = SpatialIndexTestData.HONOLULU_GEOMETRY_WRAPPER;
WestGeomPF instance = new WestGeomPF();
SearchEnvelope searchEnvelope = instance.buildSearchEnvelope(geometryWrapper, SpatialIndexTestData.WGS_84_SRS_INFO); //Needed to initialise the search.
- HashSet expResult = new HashSet<>(Arrays.asList(SpatialIndexTestData.AUCKLAND_FEATURE, SpatialIndexTestData.PERTH_FEATURE, SpatialIndexTestData.HONOLULU_FEATURE));
- HashSet result = searchEnvelope.check(spatialIndex);
+ Set expResult = SpatialIndexTestData.asNodes(List.of(SpatialIndexTestData.AUCKLAND_FEATURE, SpatialIndexTestData.PERTH_FEATURE, SpatialIndexTestData.HONOLULU_FEATURE));
+ Collection result = searchEnvelope.check(spatialIndex);
assertEquals(expResult, result);
}
diff --git a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/WestPFTest.java b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/WestPFTest.java
index 4aea6a59698..41b0ae2e44a 100644
--- a/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/WestPFTest.java
+++ b/jena-geosparql/src/test/java/org/apache/jena/geosparql/spatial/property_functions/cardinal/WestPFTest.java
@@ -17,16 +17,15 @@
*/
package org.apache.jena.geosparql.spatial.property_functions.cardinal;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
+import java.util.*;
+
import org.apache.jena.geosparql.configuration.GeoSPARQLConfig;
import org.apache.jena.geosparql.implementation.GeometryWrapper;
import org.apache.jena.geosparql.spatial.CardinalDirection;
import org.apache.jena.geosparql.spatial.SearchEnvelope;
-import org.apache.jena.geosparql.spatial.SpatialIndex;
import org.apache.jena.geosparql.spatial.SpatialIndexTestData;
+import org.apache.jena.geosparql.spatial.index.v2.SpatialIndex;
+import org.apache.jena.graph.Node;
import org.apache.jena.query.Dataset;
import org.apache.jena.query.QueryExecution;
import org.apache.jena.query.QueryExecutionFactory;
@@ -91,8 +90,8 @@ public void testCheckSearchEnvelope_no_wrap() {
GeometryWrapper geometryWrapper = SpatialIndexTestData.PERTH_GEOMETRY_WRAPPER;
WestPF instance = new WestPF();
SearchEnvelope searchEnvelope = instance.buildSearchEnvelope(geometryWrapper, SpatialIndexTestData.WGS_84_SRS_INFO); //Needed to initialise the search.
- HashSet expResult = new HashSet<>(Arrays.asList(SpatialIndexTestData.LONDON_FEATURE, SpatialIndexTestData.PERTH_FEATURE));
- HashSet result = searchEnvelope.check(spatialIndex);
+ Set expResult = SpatialIndexTestData.asNodes(List.of(SpatialIndexTestData.LONDON_FEATURE, SpatialIndexTestData.PERTH_FEATURE));
+ Collection result = searchEnvelope.check(spatialIndex);
assertEquals(expResult, result);
}
@@ -108,8 +107,8 @@ public void testCheckSearchEnvelope_wrap() {
GeometryWrapper geometryWrapper = SpatialIndexTestData.HONOLULU_GEOMETRY_WRAPPER;
WestPF instance = new WestPF();
SearchEnvelope searchEnvelope = instance.buildSearchEnvelope(geometryWrapper, SpatialIndexTestData.WGS_84_SRS_INFO); //Needed to initialise the search.
- HashSet expResult = new HashSet<>(Arrays.asList(SpatialIndexTestData.AUCKLAND_FEATURE, SpatialIndexTestData.PERTH_FEATURE, SpatialIndexTestData.HONOLULU_FEATURE));
- HashSet result = searchEnvelope.check(spatialIndex);
+ Set expResult = SpatialIndexTestData.asNodes(List.of(SpatialIndexTestData.AUCKLAND_FEATURE, SpatialIndexTestData.PERTH_FEATURE, SpatialIndexTestData.HONOLULU_FEATURE));
+ Collection result = searchEnvelope.check(spatialIndex);
assertEquals(expResult, result);
}