diff --git a/presto-geospatial-toolkit/src/main/java/com/facebook/presto/geospatial/rtree/Flatbush.java b/presto-geospatial-toolkit/src/main/java/com/facebook/presto/geospatial/rtree/Flatbush.java
new file mode 100644
index 000000000000..99f518a5e14a
--- /dev/null
+++ b/presto-geospatial-toolkit/src/main/java/com/facebook/presto/geospatial/rtree/Flatbush.java
@@ -0,0 +1,306 @@
+/*
+ * Licensed 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 com.facebook.presto.geospatial.rtree;
+
+import com.facebook.presto.array.DoubleBigArray;
+import com.facebook.presto.geospatial.Rectangle;
+import com.google.common.annotations.VisibleForTesting;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import org.openjdk.jol.info.ClassLayout;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.function.Consumer;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.airlift.slice.SizeOf.sizeOf;
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * A fast, low memory footprint static RTree.
+ *
+ * Packed Hilbert RTrees -- aka Flatbushes -- create very few objects, instead
+ * storing the tree in a flat array. They support the standard RTree queries,
+ * but cannot be modified once built. It is quite possible to semi-efficiently
+ * remove objects.
+ *
+ * Consider an RTree with branching factor `b`. Each non-leaf node has two
+ * pieces of information:
+ * 1. The minimum bounding box of all descendants, and
+ * 2. Pointers to its children.
+ *
+ * The former is simply four doubles. The latter can be derived from the
+ * node's level and which sibling it is. This means we can actually flatten
+ * the tree into a single array of doubles, four per node. We can
+ * programmatically find the indices of a node's children (it will be faster if
+ * we pre-compute level offsets), and do envelope checks with just float
+ * operations. "padded" empty nodes will have NaN entries, which will naturally
+ * return false for all comparison operators, thus being automatically not
+ * selected.
+ *
+ * A critical choice in RTree implementation is how to group leaf nodes as
+ * children (and recursively, their parents). One method that is very efficient
+ * to construct and comparable to best lookup performance is sorting by an
+ * object's Hilbert curve index. Hilbert curves are naturally hierarchical, so
+ * successively grouping children and their parents will give a naturally nested
+ * structure. This means we only need to sort the items once.
+ *
+ * If sort time is a problem, we actually just need to "partition" into groups
+ * of `degree`, since the order of the children of a single parent doesn't
+ * matter. This could be done with quicksort, stopping once all items at index
+ * `n * degree` are correctly placed.
+ *
+ * Original implementation in Javascript: https://github.com/mourner/flatbush
+ */
+public class Flatbush
+{
+ // Number of coordinates to define an envelope
+ @VisibleForTesting
+ static final int ENVELOPE_SIZE = 4;
+ private static final int INSTANCE_SIZE = ClassLayout.parseClass(Flatbush.class).instanceSize();
+ private static final int DEFAULT_DEGREE = 16;
+ // Number of children per node
+ private final int degree;
+ // Offsets in tree for each level
+ private final int[] levelOffsets;
+ // Each node has four doubles: xMin, yMin, xMax, yMax
+ private final DoubleBigArray tree;
+ private final T[] items;
+
+ /**
+ * Build Flatbush RTree for `items`.
+ *
+ * @param items Items to index.
+ * @param degree Number of children for each intermediate node.
+ */
+ public Flatbush(T[] items, int degree)
+ {
+ checkArgument(degree > 0, "degree must be positive");
+ this.degree = degree;
+ this.items = requireNonNull(items, "items is null");
+ this.levelOffsets = calculateLevelOffsets(items.length, degree);
+ this.tree = buildTree();
+ }
+
+ /**
+ * Build Flatbush RTree for `items` with default number of children per node.
+ *
+ * This will use the default degree.
+ *
+ * @param items Items to index.
+ */
+ public Flatbush(T[] items)
+ {
+ this(items, DEFAULT_DEGREE);
+ }
+
+ /**
+ * Calculate the indices for each level.
+ *
+ * A given level contains a certain number of items. We give it a capacity
+ * equal to the next multiple of `degree`, so that each parent will have
+ * an equal number (`degree`) of children. This means the next level will
+ * have `capacity/degree` nodes, which yields a capacity equal to the next
+ * multiple, and so on, until the number of items is 1.
+ *
+ * Since we are storing each node as 4 doubles, we will actually multiply
+ * every index by 4.
+ */
+ private static int[] calculateLevelOffsets(int numItems, int degree)
+ {
+ List offsets = new ArrayList<>();
+ // Leaf nodes start at 0, root is the last element.
+ offsets.add(0);
+ int level = 0;
+ while (numItems > 1) {
+ // The number of children will be the smallest multiple of degree >= numItems
+ int numChildren = (int) Math.ceil(1.0 * numItems / degree) * degree;
+ offsets.add(offsets.get(level) + ENVELOPE_SIZE * numChildren);
+ numItems = numChildren / degree;
+ level += 1;
+ }
+ return offsets.stream().mapToInt(Integer::intValue).toArray();
+ }
+
+ private DoubleBigArray buildTree()
+ {
+ // We initialize it to NaN, because all comparisons with NaN are false.
+ // Thus the normal intersection logic will not select uninitialized
+ // nodes.
+ DoubleBigArray tree = new DoubleBigArray(Double.NaN);
+ tree.ensureCapacity(levelOffsets[levelOffsets.length - 1] + ENVELOPE_SIZE);
+
+ if (items.length > degree) {
+ sortByHilbertIndex(items);
+ }
+
+ int writeOffset = 0;
+ for (T item : items) {
+ tree.set(writeOffset++, item.getExtent().getXMin());
+ tree.set(writeOffset++, item.getExtent().getYMin());
+ tree.set(writeOffset++, item.getExtent().getXMax());
+ tree.set(writeOffset++, item.getExtent().getYMax());
+ }
+
+ int numChildren = items.length;
+ for (int level = 0; level < levelOffsets.length - 1; level++) {
+ int readOffset = levelOffsets[level];
+ writeOffset = levelOffsets[level + 1];
+ int numParents = 0;
+ double xMin = Double.POSITIVE_INFINITY;
+ double yMin = Double.POSITIVE_INFINITY;
+ double xMax = Double.NEGATIVE_INFINITY;
+ double yMax = Double.NEGATIVE_INFINITY;
+ int child = 0;
+ for (; child < numChildren; child++) {
+ xMin = min(xMin, tree.get(readOffset++));
+ yMin = min(yMin, tree.get(readOffset++));
+ xMax = max(xMax, tree.get(readOffset++));
+ yMax = max(yMax, tree.get(readOffset++));
+
+ if ((child + 1) % degree == 0) {
+ numParents++;
+ tree.set(writeOffset++, xMin);
+ tree.set(writeOffset++, yMin);
+ tree.set(writeOffset++, xMax);
+ tree.set(writeOffset++, yMax);
+ xMin = Double.POSITIVE_INFINITY;
+ yMin = Double.POSITIVE_INFINITY;
+ xMax = Double.NEGATIVE_INFINITY;
+ yMax = Double.NEGATIVE_INFINITY;
+ }
+ }
+
+ if (child % degree != 0) {
+ numParents++;
+ tree.set(writeOffset++, xMin);
+ tree.set(writeOffset++, yMin);
+ tree.set(writeOffset++, xMax);
+ tree.set(writeOffset++, yMax);
+ }
+ numChildren = numParents;
+ }
+
+ return tree;
+ }
+
+ /**
+ * Find intersection candidates for `query` rectangle.
+ *
+ * This will feed to `consumer` each object in the rtree whose bounding
+ * rectangle intersects the query rectangle. The actual intersection
+ * check will need to be performed by the caller.
+ *
+ * @param query Rectangle for which to search for intersection.
+ * @param consumer Function to call for each intersection candidate.
+ */
+ public void findIntersections(Rectangle query, Consumer consumer)
+ {
+ IntArrayList todoNodes = new IntArrayList(levelOffsets.length * degree);
+ IntArrayList todoLevels = new IntArrayList(levelOffsets.length * degree);
+
+ int rootLevel = levelOffsets.length - 1;
+ int rootIndex = levelOffsets[rootLevel];
+ if (doesIntersect(query, rootIndex)) {
+ todoNodes.push(rootIndex);
+ todoLevels.push(rootLevel);
+ }
+
+ while (!todoNodes.isEmpty()) {
+ int nodeIndex = todoNodes.popInt();
+ int level = todoLevels.popInt();
+
+ if (level == 0) {
+ // This is a leaf node
+ consumer.accept(items[nodeIndex / ENVELOPE_SIZE]);
+ }
+ else {
+ int childrenOffset = getChildrenOffset(nodeIndex, level);
+ for (int i = 0; i < degree; i++) {
+ int childIndex = childrenOffset + ENVELOPE_SIZE * i;
+ if (doesIntersect(query, childIndex)) {
+ todoNodes.push(childIndex);
+ todoLevels.push(level - 1);
+ }
+ }
+ }
+ }
+ }
+
+ private boolean doesIntersect(Rectangle query, int nodeIndex)
+ {
+ return query.getXMax() >= tree.get(nodeIndex) // xMin
+ && query.getYMax() >= tree.get(nodeIndex + 1) // yMin
+ && query.getXMin() <= tree.get(nodeIndex + 2) // xMax
+ && query.getYMin() <= tree.get(nodeIndex + 3); // yMax
+ }
+
+ /**
+ * Get the offset of the first child for the node.
+ *
+ * @param nodeIndex Index in tree of first entry for node
+ * @param level Level of node
+ */
+ @VisibleForTesting
+ int getChildrenOffset(int nodeIndex, int level)
+ {
+ int indexInLevel = nodeIndex - levelOffsets[level];
+ return levelOffsets[level - 1] + degree * indexInLevel;
+ }
+
+ @VisibleForTesting
+ int getHeight()
+ {
+ return levelOffsets.length;
+ }
+
+ /*
+ * Sorts items in-place by the Hilbert index of the envelope center.
+ */
+ private void sortByHilbertIndex(T[] items)
+ {
+ if (items == null || items.length < 2) {
+ return;
+ }
+
+ Rectangle totalExtent = items[0].getExtent();
+ for (int i = 1; i < items.length; i++) {
+ totalExtent = totalExtent.merge(items[i].getExtent());
+ }
+
+ HilbertIndex hilbert = new HilbertIndex(totalExtent);
+ Arrays.parallelSort(items, Comparator.comparing(item ->
+ hilbert.indexOf(
+ (item.getExtent().getXMin() + item.getExtent().getXMax()) / 2,
+ (item.getExtent().getYMin() + item.getExtent().getYMax()) / 2)));
+ }
+
+ public boolean isEmpty()
+ {
+ return items.length == 0;
+ }
+
+ public long getEstimatedSizeInBytes()
+ {
+ long result = INSTANCE_SIZE + sizeOf(levelOffsets) + tree.sizeOf();
+ for (T item : items) {
+ result += item.getEstimatedSizeInBytes();
+ }
+ return result;
+ }
+}
diff --git a/presto-geospatial-toolkit/src/main/java/com/facebook/presto/geospatial/rtree/HilbertIndex.java b/presto-geospatial-toolkit/src/main/java/com/facebook/presto/geospatial/rtree/HilbertIndex.java
new file mode 100644
index 000000000000..95aff025819a
--- /dev/null
+++ b/presto-geospatial-toolkit/src/main/java/com/facebook/presto/geospatial/rtree/HilbertIndex.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed 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 com.facebook.presto.geospatial.rtree;
+
+import com.facebook.presto.geospatial.Rectangle;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * A class that can calculate Hilbert indices.
+ *
+ * A Hilbert index is the index on a Hilbert curve; a Hilbert curve is a
+ * space-filling curve in a rectangle. This class is instantiated with the
+ * rectangle within which it can calculate the index.
+ *
+ * The (fast) index algorithm is adapted from the C++ from
+ * https://github.com/rawrunprotected/hilbert_curves ,
+ * original algorithm by http://threadlocalmutex.com/?p=126 .
+ */
+public class HilbertIndex
+{
+ private static final int HILBERT_BITS = 16;
+ private static final double HILBERT_MAX = (1 << HILBERT_BITS) - 1;
+
+ private final Rectangle rectangle;
+ private final double xScale;
+ private final double yScale;
+
+ /**
+ * @param rectangle Rectangle defining bounds of Hilbert curve
+ */
+ public HilbertIndex(Rectangle rectangle)
+ {
+ this.rectangle = requireNonNull(rectangle, "rectangle is null");
+ if (rectangle.getXMax() == rectangle.getXMin()) {
+ this.xScale = 0;
+ }
+ else {
+ this.xScale = HILBERT_MAX / (rectangle.getXMax() - rectangle.getXMin());
+ }
+ if (rectangle.getYMax() == rectangle.getYMin()) {
+ this.yScale = 0;
+ }
+ else {
+ this.yScale = HILBERT_MAX / (rectangle.getYMax() - rectangle.getYMin());
+ }
+ }
+
+ /**
+ * Calculate Hilbert index of coordinates in rectangle.
+ *
+ * This gives a reasonable index for coordinates contained in the bounding
+ * rectangle; coordinates not in the box will return `Long.MAX_VALUE`.
+ *
+ * @param x
+ * @param y
+ * @return Hilbert curve index, relative to rectangle
+ */
+ public long indexOf(double x, double y)
+ {
+ if (!rectangle.contains(x, y)) {
+ // Put things outside the box at the end
+ // This will also handle infinities and NaNs
+ return Long.MAX_VALUE;
+ }
+
+ int xInt = (int) (xScale * (x - rectangle.getXMin()));
+ int yInt = (int) (yScale * (y - rectangle.getYMin()));
+ return discreteIndexOf(xInt, yInt);
+ }
+
+ /**
+ * Calculate the Hilbert index of a discrete coordinate.
+ *
+ * Since Java doesn't have unsigned ints, we put incoming ints into the
+ * lower 32 bits of a long and do the calculations there.
+ *
+ * @param x discrete positive x coordinate
+ * @param y discrete positive y coordinate
+ * @return Hilbert curve index
+ */
+ private long discreteIndexOf(int x, int y)
+ {
+ int a = x ^ y;
+ int b = 0x0000FFFF ^ a;
+ int c = 0x0000FFFF ^ (x | y);
+ int d = x & (y ^ 0x0000FFFF);
+
+ int e = a | (b >>> 1);
+ int f = (a >>> 1) ^ a;
+ int g = ((c >>> 1) ^ (b & (d >>> 1))) ^ c;
+ int h = ((a & (c >>> 1)) ^ (d >>> 1)) ^ d;
+
+ a = e;
+ b = f;
+ c = g;
+ d = h;
+ e = (a & (a >>> 2)) ^ (b & (b >>> 2));
+ f = (a & (b >>> 2)) ^ (b & ((a ^ b) >>> 2));
+ g ^= (a & (c >>> 2)) ^ (b & (d >>> 2));
+ h ^= (b & (c >>> 2)) ^ ((a ^ b) & (d >>> 2));
+
+ a = e;
+ b = f;
+ c = g;
+ d = h;
+ e = (a & (a >>> 4)) ^ (b & (b >>> 4));
+ f = (a & (b >>> 4)) ^ (b & ((a ^ b) >>> 4));
+ g ^= (a & (c >>> 4)) ^ (b & (d >>> 4));
+ h ^= (b & (c >>> 4)) ^ ((a ^ b) & (d >>> 4));
+
+ a = e;
+ b = f;
+ c = g;
+ d = h;
+ g ^= (a & (c >>> 8)) ^ (b & (d >>> 8));
+ h ^= (b & (c >>> 8)) ^ ((a ^ b) & (d >>> 8));
+
+ a = (g ^ (g >>> 1));
+ b = (h ^ (h >>> 1));
+
+ int i0 = (x ^ y);
+ int i1 = (b | (0x0000FFFF ^ (i0 | a)));
+
+ i0 = (i0 | (i0 << 8)) & 0x00FF00FF;
+ i0 = (i0 | (i0 << 4)) & 0x0F0F0F0F;
+ i0 = (i0 | (i0 << 2)) & 0x33333333;
+ i0 = (i0 | (i0 << 1)) & 0x55555555;
+
+ i1 = (i1 | (i1 << 8)) & 0x00FF00FF;
+ i1 = (i1 | (i1 << 4)) & 0x0F0F0F0F;
+ i1 = (i1 | (i1 << 2)) & 0x33333333;
+ i1 = (i1 | (i1 << 1)) & 0x55555555;
+
+ return (((long) ((i1 << 1) | i0)) << 32) >>> 32;
+ }
+}
diff --git a/presto-geospatial-toolkit/src/test/java/com/facebook/presto/geospatial/rtree/RtreeTestUtils.java b/presto-geospatial-toolkit/src/test/java/com/facebook/presto/geospatial/rtree/RtreeTestUtils.java
new file mode 100644
index 000000000000..e527511602d3
--- /dev/null
+++ b/presto-geospatial-toolkit/src/test/java/com/facebook/presto/geospatial/rtree/RtreeTestUtils.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed 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 com.facebook.presto.geospatial.rtree;
+
+import com.facebook.presto.geospatial.Rectangle;
+
+import java.util.List;
+import java.util.Random;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public final class RtreeTestUtils
+{
+ private RtreeTestUtils() {}
+
+ public static List makeRectangles(Random random, int numRectangles)
+ {
+ return IntStream.range(0, numRectangles)
+ .mapToObj(i -> makeRectangle(random))
+ .collect(Collectors.toList());
+ }
+
+ /*
+ * Make a random rectangle at a random origin of size < 10.
+ */
+ private static Rectangle makeRectangle(Random random)
+ {
+ double minX = randomDouble(random, -100, 100);
+ double minY = randomDouble(random, -100, 100);
+ double sizeX = randomDouble(random, 0.0, 10);
+ double sizeY = randomDouble(random, 0.0, 10);
+ return new Rectangle(minX, minY, minX + sizeX, minY + sizeY);
+ }
+
+ private static double randomDouble(Random random, double min, double max)
+ {
+ return min + random.nextDouble() * (max - min);
+ }
+}
diff --git a/presto-geospatial-toolkit/src/test/java/com/facebook/presto/geospatial/rtree/TestFlatbush.java b/presto-geospatial-toolkit/src/test/java/com/facebook/presto/geospatial/rtree/TestFlatbush.java
new file mode 100644
index 000000000000..8ae8687057dc
--- /dev/null
+++ b/presto-geospatial-toolkit/src/test/java/com/facebook/presto/geospatial/rtree/TestFlatbush.java
@@ -0,0 +1,279 @@
+/*
+ * Licensed 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 com.facebook.presto.geospatial.rtree;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import com.facebook.presto.geospatial.GeometryUtils;
+import com.facebook.presto.geospatial.Rectangle;
+import com.google.common.collect.ImmutableList;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Random;
+
+import static com.facebook.presto.geospatial.rtree.Flatbush.ENVELOPE_SIZE;
+import static com.facebook.presto.geospatial.rtree.RtreeTestUtils.makeRectangles;
+import static com.google.common.collect.ImmutableList.toImmutableList;
+import static java.lang.Double.NEGATIVE_INFINITY;
+import static java.lang.Double.POSITIVE_INFINITY;
+import static java.util.stream.Collectors.toList;
+import static org.testng.Assert.assertEquals;
+
+public class TestFlatbush
+{
+ private static final Rectangle EVERYTHING = new Rectangle(NEGATIVE_INFINITY, NEGATIVE_INFINITY, POSITIVE_INFINITY, POSITIVE_INFINITY);
+
+ private static final Comparator RECTANGLE_COMPARATOR = Comparator
+ .comparing(Rectangle::getXMin)
+ .thenComparing(Rectangle::getYMin)
+ .thenComparing(Rectangle::getXMax)
+ .thenComparing(Rectangle::getYMax);
+
+ // 2 intersecting polygons: A and B
+ private static final OGCGeometry POLYGON_A = OGCGeometry.fromText("POLYGON ((0 0, -0.5 2.5, 0 5, 2.5 5.5, 5 5, 5.5 2.5, 5 0, 2.5 -0.5, 0 0))");
+ private static final OGCGeometry POLYGON_B = OGCGeometry.fromText("POLYGON ((4 4, 3.5 7, 4 10, 7 10.5, 10 10, 10.5 7, 10 4, 7 3.5, 4 4))");
+
+ // A set of points: X in A, Y in A and B, Z in B, W outside of A and B
+ private static final OGCGeometry POINT_X = new OGCPoint(new Point(1.0, 1.0), null);
+ private static final OGCGeometry POINT_Y = new OGCPoint(new Point(4.5, 4.5), null);
+ private static final OGCGeometry POINT_Z = new OGCPoint(new Point(6.0, 6.0), null);
+ private static final OGCGeometry POINT_W = new OGCPoint(new Point(20.0, 20.0), null);
+
+ @Test
+ public void testEmptyFlatbush()
+ {
+ Flatbush rtree = new Flatbush<>(new Rectangle[] {});
+ assertEquals(findIntersections(rtree, EVERYTHING), ImmutableList.of());
+ }
+
+ @Test
+ public void testSingletonFlatbush()
+ {
+ List items = ImmutableList.of(new Rectangle(0, 0, 1, 1));
+ Flatbush rtree = new Flatbush<>(items.toArray(new Rectangle[] {}));
+
+ assertEquals(findIntersections(rtree, EVERYTHING), items);
+ // hit
+ assertEquals(findIntersections(rtree, new Rectangle(1, 1, 2, 2)), items);
+ // miss
+ assertEquals(findIntersections(rtree, new Rectangle(-1, -1, -0.1, -0.1)), ImmutableList.of());
+ }
+
+ @Test
+ public void testSingletonFlatbushXY()
+ {
+ // Because mixing up x and y is easy to do...
+ List items = ImmutableList.of(new Rectangle(0, 10, 1, 11));
+ Flatbush rtree = new Flatbush<>(items.toArray(new Rectangle[] {}));
+
+ // hit
+ assertEquals(findIntersections(rtree, new Rectangle(1, 11, 2, 12)), items);
+ // miss
+ assertEquals(findIntersections(rtree, new Rectangle(11, 1, 12, 2)), ImmutableList.of());
+ }
+
+ @Test
+ public void testDoubletonFlatbush()
+ {
+ // This is the smallest Rtree with height > 1
+ // Also test for some degeneracies
+ Rectangle rect0 = new Rectangle(1, 1, 1, 1);
+ Rectangle rect1 = new Rectangle(-1, -2, -1, -1);
+ List items = ImmutableList.of(rect0, rect1);
+
+ Flatbush rtree = new Flatbush<>(items.toArray(new Rectangle[] {}));
+
+ List allResults = findIntersections(rtree, EVERYTHING);
+ assertEqualsSorted(allResults, items, RECTANGLE_COMPARATOR);
+
+ assertEquals(findIntersections(rtree, new Rectangle(1, 1, 2, 2)), ImmutableList.of(rect0));
+ assertEquals(findIntersections(rtree, new Rectangle(-2, -2, -1, -2)), ImmutableList.of(rect1));
+ // This should test missing at the root level
+ assertEquals(findIntersections(rtree, new Rectangle(10, 10, 12, 12)), ImmutableList.of());
+ // This should test missing at the leaf level
+ assertEquals(findIntersections(rtree, new Rectangle(0, 0, 0, 0)), ImmutableList.of());
+ }
+
+ @Test
+ public void testTwoLevelFlatbush()
+ {
+ // This is the smallest Rtree with height > 2
+ // Also test for NaN behavior
+ Rectangle rect0 = new Rectangle(1, 1, 1, 1);
+ Rectangle rect1 = new Rectangle(-1, -1, -1, -1);
+ Rectangle rect2 = new Rectangle(1, -1, 1, -1);
+ List items = ImmutableList.of(rect0, rect1, rect2);
+
+ Flatbush rtree = new Flatbush<>(items.toArray(new Rectangle[] {}), 2);
+
+ List allResults = findIntersections(rtree, EVERYTHING);
+ assertEqualsSorted(allResults, items, RECTANGLE_COMPARATOR);
+
+ assertEquals(findIntersections(rtree, new Rectangle(1, 1, 1, 1)), ImmutableList.of(rect0));
+ assertEquals(findIntersections(rtree, new Rectangle(-1, -1, -1, -1)), ImmutableList.of(rect1));
+ assertEquals(findIntersections(rtree, new Rectangle(1, -1, 1, -1)), ImmutableList.of(rect2));
+ // Test hitting across parent nodes
+ List results12 = findIntersections(rtree, new Rectangle(-1, -1, 1, -1));
+ assertEqualsSorted(results12, ImmutableList.of(rect1, rect2), RECTANGLE_COMPARATOR);
+
+ // This should test missing at the root level
+ assertEquals(findIntersections(rtree, new Rectangle(10, 10, 12, 12)), ImmutableList.of());
+ // This should test missing at the leaf level
+ assertEquals(findIntersections(rtree, new Rectangle(0, 0, 0, 0)), ImmutableList.of());
+ }
+
+ @Test
+ public void testOctagonQuery()
+ {
+ OGCGeometryWrapper octagonA = new OGCGeometryWrapper(POLYGON_A);
+ OGCGeometryWrapper octagonB = new OGCGeometryWrapper(POLYGON_B);
+
+ OGCGeometryWrapper pointX = new OGCGeometryWrapper(POINT_X);
+ OGCGeometryWrapper pointY = new OGCGeometryWrapper(POINT_Y);
+ OGCGeometryWrapper pointZ = new OGCGeometryWrapper(POINT_Z);
+ OGCGeometryWrapper pointW = new OGCGeometryWrapper(POINT_W);
+
+ Flatbush rtree = new Flatbush<>(new OGCGeometryWrapper[] {pointX, pointY, pointZ, pointW});
+
+ List resultsA = findIntersections(rtree, octagonA.getExtent());
+ assertEqualsSorted(resultsA, ImmutableList.of(pointX, pointY), Comparator.naturalOrder());
+
+ List resultsB = findIntersections(rtree, octagonB.getExtent());
+ assertEqualsSorted(resultsB, ImmutableList.of(pointY, pointZ), Comparator.naturalOrder());
+ }
+
+ @Test
+ public void testOctagonTree()
+ {
+ OGCGeometryWrapper octagonA = new OGCGeometryWrapper(POLYGON_A);
+ OGCGeometryWrapper octagonB = new OGCGeometryWrapper(POLYGON_B);
+
+ OGCGeometryWrapper pointX = new OGCGeometryWrapper(POINT_X);
+ OGCGeometryWrapper pointY = new OGCGeometryWrapper(POINT_Y);
+ OGCGeometryWrapper pointZ = new OGCGeometryWrapper(POINT_Z);
+ OGCGeometryWrapper pointW = new OGCGeometryWrapper(POINT_W);
+
+ Flatbush rtree = new Flatbush<>(new OGCGeometryWrapper[] {octagonA, octagonB});
+
+ assertEquals(findIntersections(rtree, pointX.getExtent()), ImmutableList.of(octagonA));
+
+ List results = findIntersections(rtree, pointY.getExtent());
+ assertEqualsSorted(results, ImmutableList.of(octagonA, octagonB), Comparator.naturalOrder());
+
+ assertEquals(findIntersections(rtree, pointZ.getExtent()), ImmutableList.of(octagonB));
+
+ assertEquals(findIntersections(rtree, pointW.getExtent()), ImmutableList.of());
+ }
+
+ @DataProvider(name = "rectangle-counts")
+ private Object[][] rectangleCounts()
+ {
+ return new Object[][] {{100, 1000, 42}, {1000, 10_000, 123}, {5000, 50_000, 321}};
+ }
+
+ @Test(dataProvider = "rectangle-counts")
+ public void testRectangleCollection(int numBuildRectangles, int numProbeRectangles, int seed)
+ {
+ Random random = new Random(seed);
+ List buildRectangles = makeRectangles(random, numBuildRectangles);
+ List probeRectangles = makeRectangles(random, numProbeRectangles);
+
+ Flatbush rtree = new Flatbush<>(buildRectangles.toArray(new Rectangle[] {}));
+ for (Rectangle query : probeRectangles) {
+ List actual = findIntersections(rtree, query);
+ List expected = buildRectangles.stream()
+ .filter(rect -> rect.intersects(query))
+ .collect(toList());
+ assertEqualsSorted(actual, expected, RECTANGLE_COMPARATOR);
+ }
+ }
+
+ @Test
+ public void testChildrenOffsets()
+ {
+ int numRectangles = 10;
+ int degree = 8;
+ Random random = new Random(122);
+
+ int firstParentIndex = 2 * degree * ENVELOPE_SIZE;
+ int secondParentIndex = firstParentIndex + ENVELOPE_SIZE;
+ int grandparentIndex = 3 * degree * ENVELOPE_SIZE;
+
+ List rectangles = makeRectangles(random, numRectangles);
+ Flatbush rtree = new Flatbush<>(rectangles.toArray(new Rectangle[] {}), degree);
+ assertEquals(rtree.getHeight(), 3);
+ assertEquals(rtree.getChildrenOffset(firstParentIndex, 1), 0);
+ assertEquals(rtree.getChildrenOffset(secondParentIndex, 1), degree * ENVELOPE_SIZE);
+ assertEquals(rtree.getChildrenOffset(grandparentIndex, 2), 2 * degree * ENVELOPE_SIZE);
+ }
+
+ private static List findIntersections(Flatbush rtree, Rectangle rectangle)
+ {
+ List results = new ArrayList<>();
+ rtree.findIntersections(rectangle, results::add);
+ return results;
+ }
+
+ /*
+ * Asserts the two lists of Rectangles are equal after sorting.
+ */
+ private static void assertEqualsSorted(List actual, List expected, Comparator comparator)
+ {
+ List actualSorted = actual.stream().sorted(comparator).collect(toImmutableList());
+ List expectedSorted = expected.stream().sorted(comparator).collect(toImmutableList());
+
+ assertEquals(actualSorted, expectedSorted);
+ }
+
+ private static final class OGCGeometryWrapper
+ implements HasExtent, Comparable
+ {
+ private final OGCGeometry geometry;
+ private final Rectangle extent;
+
+ public OGCGeometryWrapper(OGCGeometry geometry)
+ {
+ this.geometry = geometry;
+ this.extent = GeometryUtils.getExtent(geometry);
+ }
+
+ public OGCGeometry getGeometry()
+ {
+ return geometry;
+ }
+
+ @Override
+ public Rectangle getExtent()
+ {
+ return extent;
+ }
+
+ @Override
+ public long getEstimatedSizeInBytes()
+ {
+ return geometry.estimateMemorySize();
+ }
+
+ @Override
+ public int compareTo(OGCGeometryWrapper other)
+ {
+ return RECTANGLE_COMPARATOR.compare(this.getExtent(), other.getExtent());
+ }
+ }
+}
diff --git a/presto-geospatial-toolkit/src/test/java/com/facebook/presto/geospatial/rtree/TestHilbertIndex.java b/presto-geospatial-toolkit/src/test/java/com/facebook/presto/geospatial/rtree/TestHilbertIndex.java
new file mode 100644
index 000000000000..7ef288b403c5
--- /dev/null
+++ b/presto-geospatial-toolkit/src/test/java/com/facebook/presto/geospatial/rtree/TestHilbertIndex.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed 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 com.facebook.presto.geospatial.rtree;
+
+import com.facebook.presto.geospatial.Rectangle;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+public class TestHilbertIndex
+{
+ @Test
+ public void testOrder()
+ {
+ HilbertIndex hilbert = new HilbertIndex(new Rectangle(0, 0, 4, 4));
+ long h0 = hilbert.indexOf(0., 0.);
+ long h1 = hilbert.indexOf(1., 1.);
+ long h2 = hilbert.indexOf(1., 3.);
+ long h3 = hilbert.indexOf(3., 3.);
+ long h4 = hilbert.indexOf(3., 1.);
+ assertTrue(h0 < h1);
+ assertTrue(h1 < h2);
+ assertTrue(h2 < h3);
+ assertTrue(h3 < h4);
+ }
+
+ @Test
+ public void testOutOfBounds()
+ {
+ HilbertIndex hilbert = new HilbertIndex(new Rectangle(0, 0, 1, 1));
+ assertEquals(hilbert.indexOf(2., 2.), Long.MAX_VALUE);
+ }
+
+ @Test
+ public void testDegenerateRectangle()
+ {
+ HilbertIndex hilbert = new HilbertIndex(new Rectangle(0, 0, 0, 0));
+ assertEquals(hilbert.indexOf(0., 0.), 0);
+ assertEquals(hilbert.indexOf(2., 2.), Long.MAX_VALUE);
+ }
+
+ @Test
+ public void testDegenerateHorizontalRectangle()
+ {
+ HilbertIndex hilbert = new HilbertIndex(new Rectangle(0, 0, 4, 0));
+ assertEquals(hilbert.indexOf(0., 0.), 0);
+ assertTrue(hilbert.indexOf(1., 0.) < hilbert.indexOf(2., 0.));
+ assertEquals(hilbert.indexOf(0., 2.), Long.MAX_VALUE);
+ assertEquals(hilbert.indexOf(2., 2.), Long.MAX_VALUE);
+ }
+
+ @Test
+ public void testDegenerateVerticalRectangle()
+ {
+ HilbertIndex hilbert = new HilbertIndex(new Rectangle(0, 0, 0, 4));
+ assertEquals(hilbert.indexOf(0., 0.), 0);
+ assertTrue(hilbert.indexOf(0., 1.) < hilbert.indexOf(0., 2.));
+ assertEquals(hilbert.indexOf(2., 0.), Long.MAX_VALUE);
+ assertEquals(hilbert.indexOf(2., 2.), Long.MAX_VALUE);
+ }
+}