You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by iv...@apache.org on 2019/10/14 17:45:07 UTC

[lucene-solr] branch branch_8x updated: LUCENE-8746: Refactor EdgeTree (#878)

This is an automated email from the ASF dual-hosted git repository.

ivera pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new d82682c  LUCENE-8746: Refactor EdgeTree  (#878)
d82682c is described below

commit d82682c8450ee192034fff91b88aba638c382012
Author: Ignacio Vera <iv...@apache.org>
AuthorDate: Mon Oct 14 19:36:05 2019 +0200

    LUCENE-8746: Refactor EdgeTree  (#878)
    
    Introduce a Component tree that represents the tree of components (e.g polygons).
     Edge tree is now just a tree of edges.
---
 lucene/CHANGES.txt                                 |   3 +
 .../LatLonDocValuesPointInPolygonQuery.java        |   5 +-
 .../lucene/document/LatLonPointInPolygonQuery.java |  21 +-
 .../java/org/apache/lucene/geo/Component2D.java    |  96 ++++
 .../java/org/apache/lucene/geo/ComponentTree.java  | 206 +++++++++
 .../src/java/org/apache/lucene/geo/EdgeTree.java   | 495 ++++++++-------------
 .../org/apache/lucene/geo/GeoEncodingUtils.java    |  17 +-
 .../src/java/org/apache/lucene/geo/Polygon2D.java  | 219 ++++-----
 .../test/org/apache/lucene/geo/TestPolygon2D.java  |  88 ++--
 .../lucene/document/LatLonShapeLineQuery.java      |   5 +-
 .../lucene/document/LatLonShapePolygonQuery.java   |   5 +-
 .../apache/lucene/document/XYShapeLineQuery.java   |  13 +-
 .../lucene/document/XYShapePolygonQuery.java       |   6 +-
 .../src/java/org/apache/lucene/geo/Line2D.java     | 126 ++++--
 .../java/org/apache/lucene/geo/XYPolygon2D.java    |  10 +-
 .../lucene/document/BaseLatLonShapeTestCase.java   |   5 +-
 .../lucene/document/BaseShapeEncodingTestCase.java |   6 +-
 .../apache/lucene/document/BaseShapeTestCase.java  |  17 +-
 .../lucene/document/BaseXYShapeTestCase.java       |   5 +-
 .../document/TestLatLonLineShapeQueries.java       |  19 +-
 .../document/TestLatLonMultiLineShapeQueries.java  |  22 +-
 .../document/TestLatLonMultiPointShapeQueries.java |  22 +-
 .../TestLatLonMultiPolygonShapeQueries.java        |  22 +-
 .../document/TestLatLonPointShapeQueries.java      |  18 +-
 .../document/TestLatLonPolygonShapeQueries.java    |  18 +-
 .../apache/lucene/document/TestLatLonShape.java    |  11 +-
 .../lucene/document/TestLatLonShapeEncoding.java   |   3 +-
 .../lucene/document/TestXYLineShapeQueries.java    |  19 +-
 .../document/TestXYMultiLineShapeQueries.java      |  22 +-
 .../document/TestXYMultiPointShapeQueries.java     |  22 +-
 .../document/TestXYMultiPolygonShapeQueries.java   |  22 +-
 .../lucene/document/TestXYPointShapeQueries.java   |  18 +-
 .../lucene/document/TestXYPolygonShapeQueries.java |  18 +-
 .../lucene/document/TestXYShapeEncoding.java       |   3 +-
 .../src/test/org/apache/lucene/geo/TestLine2D.java |  18 +-
 35 files changed, 826 insertions(+), 799 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 8a60f77..0a986ab 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -32,6 +32,9 @@ Other
 ---------------------
 (No changes)
 
+* LUCENE-8746: Refactor EdgeTree - Introduce a Component tree that represents the tree of components (e.g polygons).
+  Edge tree is now just a tree of edges. (Ignacio Vera)
+
 Build
 
 * Upgrade forbiddenapis to version 2.7; upgrade Groovy to 2.4.17.  (Uwe Schindler)
diff --git a/lucene/core/src/java/org/apache/lucene/document/LatLonDocValuesPointInPolygonQuery.java b/lucene/core/src/java/org/apache/lucene/document/LatLonDocValuesPointInPolygonQuery.java
index 7b09f84..525b77e 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LatLonDocValuesPointInPolygonQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LatLonDocValuesPointInPolygonQuery.java
@@ -20,6 +20,7 @@ package org.apache.lucene.document;
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.GeoEncodingUtils;
 import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.geo.Polygon2D;
@@ -103,8 +104,8 @@ public class LatLonDocValuesPointInPolygonQuery extends Query {
 
     return new ConstantScoreWeight(this, boost) {
 
-      final Polygon2D tree = Polygon2D.create(polygons);
-      final GeoEncodingUtils.PolygonPredicate polygonPredicate = GeoEncodingUtils.createPolygonPredicate(polygons, tree);
+      final Component2D tree = Polygon2D.create(polygons);
+      final GeoEncodingUtils.PolygonPredicate polygonPredicate = GeoEncodingUtils.createComponentPredicate(tree);
 
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
diff --git a/lucene/core/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java b/lucene/core/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
index 9006f16..82212ae 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
@@ -19,10 +19,10 @@ package org.apache.lucene.document;
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.GeoEncodingUtils;
 import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.geo.Polygon2D;
-import org.apache.lucene.geo.Rectangle;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
@@ -85,7 +85,7 @@ final class LatLonPointInPolygonQuery extends Query {
     }
   }
 
-  private IntersectVisitor getIntersectVisitor(DocIdSetBuilder result, Polygon2D tree, GeoEncodingUtils.PolygonPredicate polygonPredicate,
+  private IntersectVisitor getIntersectVisitor(DocIdSetBuilder result, Component2D tree, GeoEncodingUtils.PolygonPredicate polygonPredicate,
                                                byte[] minLat, byte[] maxLat, byte[] minLon, byte[] maxLon) {
     return new IntersectVisitor() {
           DocIdSetBuilder.BulkAdder adder;
@@ -134,7 +134,7 @@ final class LatLonPointInPolygonQuery extends Query {
             double cellMaxLat = decodeLatitude(maxPackedValue, 0);
             double cellMaxLon = decodeLongitude(maxPackedValue, Integer.BYTES);
 
-            return tree.relate(cellMinLat, cellMaxLat, cellMinLon, cellMaxLon);
+            return tree.relate(cellMinLon, cellMaxLon, cellMinLat, cellMaxLat);
           }
         };
   }
@@ -142,20 +142,17 @@ final class LatLonPointInPolygonQuery extends Query {
   @Override
   public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
 
+    final Component2D tree = Polygon2D.create(polygons);
+    final GeoEncodingUtils.PolygonPredicate polygonPredicate = GeoEncodingUtils.createComponentPredicate(tree);
     // bounding box over all polygons, this can speed up tree intersection/cheaply improve approximation for complex multi-polygons
-    // these are pre-encoded with LatLonPoint's encoding
-    final Rectangle box = Rectangle.fromPolygon(polygons);
     final byte minLat[] = new byte[Integer.BYTES];
     final byte maxLat[] = new byte[Integer.BYTES];
     final byte minLon[] = new byte[Integer.BYTES];
     final byte maxLon[] = new byte[Integer.BYTES];
-    NumericUtils.intToSortableBytes(encodeLatitude(box.minLat), minLat, 0);
-    NumericUtils.intToSortableBytes(encodeLatitude(box.maxLat), maxLat, 0);
-    NumericUtils.intToSortableBytes(encodeLongitude(box.minLon), minLon, 0);
-    NumericUtils.intToSortableBytes(encodeLongitude(box.maxLon), maxLon, 0);
-
-    final Polygon2D tree = Polygon2D.create(polygons);
-    final GeoEncodingUtils.PolygonPredicate polygonPredicate = GeoEncodingUtils.createPolygonPredicate(polygons, tree);
+    NumericUtils.intToSortableBytes(encodeLatitude(tree.getMinY()), minLat, 0);
+    NumericUtils.intToSortableBytes(encodeLatitude(tree.getMaxY()), maxLat, 0);
+    NumericUtils.intToSortableBytes(encodeLongitude(tree.getMinX()), minLon, 0);
+    NumericUtils.intToSortableBytes(encodeLongitude(tree.getMaxX()), maxLon, 0);
 
     return new ConstantScoreWeight(this, boost) {
 
diff --git a/lucene/core/src/java/org/apache/lucene/geo/Component2D.java b/lucene/core/src/java/org/apache/lucene/geo/Component2D.java
new file mode 100644
index 0000000..10ced04
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/geo/Component2D.java
@@ -0,0 +1,96 @@
+/*
+ * 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.lucene.geo;
+
+import org.apache.lucene.index.PointValues;
+
+import static org.apache.lucene.geo.GeoUtils.orient;
+
+/**
+ * 2D Geometry object that supports spatial relationships with bounding boxes,
+ * triangles and points.
+ *
+ * @lucene.internal
+ **/
+public interface Component2D {
+
+  /** min X value for the component **/
+  double getMinX();
+
+  /** max X value for the component **/
+  double getMaxX();
+
+  /** min Y value for the component **/
+  double getMinY();
+
+  /** max Y value for the component **/
+  double getMaxY();
+
+  /** relates this component2D with a point **/
+  boolean contains(double x, double y);
+
+  /** relates this component2D with a bounding box **/
+  PointValues.Relation relate(double minX, double maxX, double minY, double maxY);
+
+  /** relates this component2D with a triangle **/
+  PointValues.Relation relateTriangle(double minX, double maxX, double minY, double maxY,
+                                      double aX, double aY, double bX, double bY, double cX, double cY);
+
+  /** relates this component2D with a triangle **/
+  default PointValues.Relation relateTriangle(double aX, double aY, double bX, double bY, double cX, double cY) {
+    double minY = StrictMath.min(StrictMath.min(aY, bY), cY);
+    double minX = StrictMath.min(StrictMath.min(aX, bX), cX);
+    double maxY = StrictMath.max(StrictMath.max(aY, bY), cY);
+    double maxX = StrictMath.max(StrictMath.max(aX, bX), cX);
+    return relateTriangle(minX, maxX, minY, maxY, aX, aY, bX, bY, cX, cY);
+  }
+
+  /** Compute whether the bounding boxes are disjoint **/
+  static  boolean disjoint(double minX1, double maxX1, double minY1, double maxY1, double minX2, double maxX2, double minY2, double maxY2) {
+    return (maxY1 < minY2 || minY1 > maxY2 || maxX1 < minX2 || minX1 > maxX2);
+  }
+
+  /** Compute whether the first bounding box 1 is within the second bounding box **/
+  static boolean within(double minX1, double maxX1, double minY1, double maxY1, double minX2, double maxX2, double minY2, double maxY2) {
+    return (minY2 <= minY1 && maxY2 >= maxY1 && minX2 <= minX1 && maxX2 >= maxX1);
+  }
+
+  /** returns true if rectangle (defined by minX, maxX, minY, maxY) contains the X Y point */
+  static boolean containsPoint(final double x, final double y, final double minX, final double maxX, final double minY, final double maxY) {
+    return x >= minX && x <= maxX && y >= minY && y <= maxY;
+  }
+
+  /**
+   * Compute whether the given x, y point is in a triangle; uses the winding order method */
+  static boolean pointInTriangle(double minX, double maxX, double minY, double maxY, double x, double y, double aX, double aY, double bX, double bY, double cX, double cY) {
+    //check the bounding box because if the triangle is degenerated, e.g points and lines, we need to filter out
+    //coplanar points that are not part of the triangle.
+    if (x >= minX && x <= maxX && y >= minY && y <= maxY) {
+      int a = orient(x, y, aX, aY, bX, bY);
+      int b = orient(x, y, bX, bY, cX, cY);
+      if (a == 0 || b == 0 || a < 0 == b < 0) {
+        int c = orient(x, y, cX, cY, aX, aY);
+        return c == 0 || (c < 0 == (b < 0 || a < 0));
+      }
+      return false;
+    } else {
+      return false;
+    }
+  }
+
+}
\ No newline at end of file
diff --git a/lucene/core/src/java/org/apache/lucene/geo/ComponentTree.java b/lucene/core/src/java/org/apache/lucene/geo/ComponentTree.java
new file mode 100644
index 0000000..220c53d
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/geo/ComponentTree.java
@@ -0,0 +1,206 @@
+/*
+ * 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.lucene.geo;
+
+import java.util.Comparator;
+
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.util.ArrayUtil;
+
+/**
+ * 2D multi-component geometry implementation represented as an interval tree of components.
+ * <p>
+ * Construction takes {@code O(n log n)} time for sorting and tree construction.
+ *
+ * @lucene.internal
+ */
+final class ComponentTree implements Component2D {
+  /** minimum latitude of this geometry's bounding box area */
+  private double minY;
+  /** maximum latitude of this geometry's bounding box area */
+  private double maxY;
+  /** minimum longitude of this geometry's bounding box area */
+  private double minX;
+  /** maximum longitude of this geometry's bounding box area */
+  private double maxX;
+  // child components, or null. Note internal nodes might mot have
+  // a consistent bounding box. Internal nodes should not be accessed
+  // outside if this class.
+  private Component2D left;
+  private Component2D right;
+  /** which dimension was this node split on */
+  // TODO: its implicit based on level, but boolean keeps code simple
+  final private boolean splitX;
+  /** root node of edge tree */
+  final private Component2D component;
+
+  protected ComponentTree(Component2D component, boolean splitX) {
+    this.minY = component.getMinY();
+    this.maxY = component.getMaxY();
+    this.minX = component.getMinX();
+    this.maxX = component.getMaxX();
+    this.component = component;
+    this.splitX = splitX;
+  }
+
+  @Override
+  public double getMinX() {
+    return minX;
+  }
+
+  @Override
+  public double getMaxX() {
+    return maxX;
+  }
+
+  @Override
+  public double getMinY() {
+    return minY;
+  }
+
+  @Override
+  public double getMaxY() {
+    return maxY;
+  }
+
+  @Override
+  public boolean contains(double x, double y) {
+    if (y <= this.maxY && x <= this.maxX) {
+      if (component.contains(x, y)) {
+        return true;
+      }
+      if (left != null) {
+        if (left.contains(x, y)) {
+          return true;
+        }
+      }
+      if (right != null && ((splitX == false && y >= this.component.getMinY()) || (splitX && x >= this.component.getMinX()))) {
+        if (right.contains(x, y)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  /** Returns relation to the provided triangle */
+  @Override
+  public Relation relateTriangle(double minX, double maxX, double minY, double maxY,
+                                 double ax, double ay, double bx, double by, double cx, double cy) {
+    if (minY <= this.maxY && minX <= this.maxX) {
+      Relation relation = component.relateTriangle(minX, maxX, minY, maxY, ax, ay, bx, by, cx, cy);
+      if (relation != Relation.CELL_OUTSIDE_QUERY) {
+        return relation;
+      }
+      if (left != null) {
+        relation = left.relateTriangle(minX, maxX, minY, maxY, ax, ay, bx, by, cx, cy);
+        if (relation != Relation.CELL_OUTSIDE_QUERY) {
+          return relation;
+        }
+      }
+      if (right != null && ((splitX == false && maxY >= this.component.getMinY()) || (splitX && maxX >= this.component.getMinX()))) {
+        relation = right.relateTriangle(minX, maxX, minY, maxY, ax, ay, bx, by, cx, cy);
+        if (relation != Relation.CELL_OUTSIDE_QUERY) {
+          return relation;
+        }
+      }
+    }
+    return Relation.CELL_OUTSIDE_QUERY;
+  }
+
+  /** Returns relation to the provided rectangle */
+  @Override
+  public Relation relate(double minX, double maxX, double minY, double maxY) {
+    if (minY <= this.maxY && minX <= this.maxX) {
+      Relation relation = component.relate(minX, maxX, minY, maxY);
+      if (relation != Relation.CELL_OUTSIDE_QUERY) {
+        return relation;
+      }
+      if (left != null) {
+        relation = left.relate(minX, maxX, minY, maxY);
+        if (relation != Relation.CELL_OUTSIDE_QUERY) {
+          return relation;
+        }
+      }
+      if (right != null && ((splitX == false && maxY >= this.component.getMinY()) || (splitX && maxX >= this.component.getMinX()))) {
+        relation = right.relate(minX, maxX, minY, maxY);
+        if (relation != Relation.CELL_OUTSIDE_QUERY) {
+          return relation;
+        }
+      }
+    }
+    return Relation.CELL_OUTSIDE_QUERY;
+  }
+
+  /** Creates tree from provided components */
+  public static Component2D create(Component2D[] components) {
+    if (components.length == 1) {
+      return components[0];
+    }
+    ComponentTree root =  createTree(components, 0, components.length - 1, false);
+    // pull up min values for the root node so it contains a consistent bounding box
+    for (Component2D component : components) {
+      root.minY = Math.min(root.minY, component.getMinY());
+      root.minX = Math.min(root.minX, component.getMinX());
+    }
+    return root;
+  }
+
+  /** Creates tree from sorted components (with range low and high inclusive) */
+  private static ComponentTree createTree(Component2D[] components, int low, int high, boolean splitX) {
+    if (low > high) {
+      return null;
+    }
+    final int mid = (low + high) >>> 1;
+    if (low < high) {
+      Comparator<Component2D> comparator;
+      if (splitX) {
+        comparator = (left, right) -> {
+          int ret = Double.compare(left.getMinX(), right.getMinX());
+          if (ret == 0) {
+            ret = Double.compare(left.getMaxX(), right.getMaxX());
+          }
+          return ret;
+        };
+      } else {
+        comparator = (left, right) -> {
+          int ret = Double.compare(left.getMinY(), right.getMinY());
+          if (ret == 0) {
+            ret = Double.compare(left.getMaxY(), right.getMaxY());
+          }
+          return ret;
+        };
+      }
+      ArrayUtil.select(components, low, high + 1, mid, comparator);
+    }
+    ComponentTree newNode = new ComponentTree(components[mid], splitX);
+    // find children
+    newNode.left = createTree(components, low, mid - 1, !splitX);
+    newNode.right = createTree(components, mid + 1, high, !splitX);
+
+    // pull up max values to this node
+    if (newNode.left != null) {
+      newNode.maxX = Math.max(newNode.maxX, newNode.left.getMaxX());
+      newNode.maxY = Math.max(newNode.maxY, newNode.left.getMaxY());
+    }
+    if (newNode.right != null) {
+      newNode.maxX = Math.max(newNode.maxX, newNode.right.getMaxX());
+      newNode.maxY = Math.max(newNode.maxY, newNode.right.getMaxY());
+    }
+    return newNode;
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/geo/EdgeTree.java b/lucene/core/src/java/org/apache/lucene/geo/EdgeTree.java
index 62ed5ed..9b83022 100644
--- a/lucene/core/src/java/org/apache/lucene/geo/EdgeTree.java
+++ b/lucene/core/src/java/org/apache/lucene/geo/EdgeTree.java
@@ -17,10 +17,7 @@
 package org.apache.lucene.geo;
 
 import java.util.Arrays;
-import java.util.Comparator;
-
-import org.apache.lucene.index.PointValues.Relation;
-import org.apache.lucene.util.ArrayUtil;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import static org.apache.lucene.geo.GeoUtils.lineCrossesLine;
 import static org.apache.lucene.geo.GeoUtils.lineCrossesLineWithBoundary;
@@ -34,232 +31,123 @@ import static org.apache.lucene.geo.GeoUtils.orient;
  * practical lines and polygons are much faster than brute force.
  * @lucene.internal
  */
-public abstract class EdgeTree {
-  /** minimum latitude of this geometry's bounding box area */
-  public final double minLat;
-  /** maximum latitude of this geometry's bounding box area */
-  public final double maxLat;
-  /** minimum longitude of this geometry's bounding box area */
-  public final double minLon;
-  /** maximum longitude of this geometry's bounding box area */
-  public final double maxLon;
-
-  // each component is a node in an augmented 2d kd-tree: we alternate splitting between latitude/longitude,
-  // and pull up max values for both dimensions to each parent node (regardless of split).
-
-  /** maximum latitude of this component or any of its children */
-  protected double maxY;
-  /** maximum longitude of this component or any of its children */
-  protected double maxX;
-  /** which dimension was this node split on */
-  // TODO: its implicit based on level, but boolean keeps code simple
-  protected boolean splitX;
-
-  // child components, or null
-  protected EdgeTree left;
-  protected EdgeTree right;
-
-  /** root node of edge tree */
-  protected final Edge tree;
-
-  protected EdgeTree(final double minLat, final double maxLat, final double minLon, final double maxLon, double[] lats, double[] lons) {
-    this.minLat = minLat;
-    this.maxLat = maxLat;
-    this.minLon = minLon;
-    this.maxLon = maxLon;
-    this.maxY = maxLat;
-    this.maxX = maxLon;
-
-    // create interval tree of edges
-    this.tree = createTree(lats, lons);
-  }
+/**
+ * Internal tree node: represents geometry edge from lat1,lon1 to lat2,lon2.
+ * The sort value is {@code low}, which is the minimum latitude of the edge.
+ * {@code max} stores the maximum latitude of this edge or any children.
+ *
+ * @lucene.internal
+ */
+public  class EdgeTree {
+    // lat-lon pair (in original order) of the two vertices
+    final double y1, y2;
+    final double x1, x2;
+    /** min of this edge */
+    final double low;
+    /** max latitude of this edge or any children */
+    double max;
+    /** left child edge, or null */
+    EdgeTree left;
+    /** right child edge, or null */
+    EdgeTree right;
 
-  /** Returns relation to the provided triangle */
-  public Relation relateTriangle(double ax, double ay, double bx, double by, double cx, double cy) {
-    // compute bounding box of triangle
-    double triMinLat = StrictMath.min(StrictMath.min(ay, by), cy);
-    double triMinLon = StrictMath.min(StrictMath.min(ax, bx), cx);
-    if (triMinLat <= maxY && triMinLon <= maxX) {
-      Relation relation = internalComponentRelateTriangle(ax, ay, bx, by, cx, cy);
-      if (relation != Relation.CELL_OUTSIDE_QUERY) {
-        return relation;
-      }
-      if (left != null) {
-        relation = left.relateTriangle(ax, ay, bx, by, cx, cy);
-        if (relation != Relation.CELL_OUTSIDE_QUERY) {
-          return relation;
-        }
-      }
-      double triMaxLat = StrictMath.max(StrictMath.max(ay, by), cy);
-      double triMaxLon = StrictMath.max(StrictMath.max(ax, bx), cx);
-      if (right != null && ((splitX == false && triMaxLat >= this.minLat) || (splitX && triMaxLon >= this.minLon))) {
-        relation = right.relateTriangle(ax, ay, bx, by, cx, cy);
-        if (relation != Relation.CELL_OUTSIDE_QUERY) {
-          return relation;
-        }
-      }
+  EdgeTree(double x1, double y1, double x2, double y2, double low, double max) {
+      this.y1 = y1;
+      this.x1 = x1;
+      this.y2 = y2;
+      this.x2 = x2;
+      this.low = low;
+      this.max = max;
     }
-    return Relation.CELL_OUTSIDE_QUERY;
-  }
 
-  /** Returns relation to the provided rectangle */
-  public Relation relate(double minLat, double maxLat, double minLon, double maxLon) {
-    if (minLat <= maxY && minLon <= maxX) {
-      Relation relation = internalComponentRelate(minLat, maxLat, minLon, maxLon);
-      if (relation != Relation.CELL_OUTSIDE_QUERY) {
-        return relation;
-      }
-      if (left != null) {
-        relation = left.relate(minLat, maxLat, minLon, maxLon);
-        if (relation != Relation.CELL_OUTSIDE_QUERY) {
-          return relation;
+  /**
+   * Returns true if the point crosses this edge subtree an odd number of times
+   * <p>
+   * See <a href="https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html">
+   * https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html</a> for more information.
+   */
+  // ported to java from https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html
+  // original code under the BSD license (https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html#License%20to%20Use)
+  //
+  // Copyright (c) 1970-2003, Wm. Randolph Franklin
+  //
+  // Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+  // documentation files (the "Software"), to deal in the Software without restriction, including without limitation
+  // the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and
+  // to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+  //
+  // 1. Redistributions of source code must retain the above copyright
+  //    notice, this list of conditions and the following disclaimers.
+  // 2. Redistributions in binary form must reproduce the above copyright
+  //    notice in the documentation and/or other materials provided with
+  //    the distribution.
+  // 3. The name of W. Randolph Franklin may not be used to endorse or
+  //    promote products derived from this Software without specific
+  //    prior written permission.
+  //
+  // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED
+  // TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL
+  // THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF
+  // CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+  // IN THE SOFTWARE.
+  protected boolean contains(double x, double y, AtomicBoolean isOnEdge) {
+    boolean res = false;
+    if (isOnEdge.get() == false && y <= this.max) {
+      if (y == this.y1 && y == this.y2 ||
+          (y <= this.y1 && y >= this.y2) != (y >= this.y1 && y <= this.y2)) {
+        if ((x == this.x1 && x == this.x2) ||
+            ((x <= this.x1 && x >= this.x2) != (x >= this.x1 && x <= this.x2) &&
+                GeoUtils.orient(this.x1, this.y1, this.x2, this.y2, x, y) == 0)) {
+          // if its on the boundary return true
+          isOnEdge.set(true);
+          return true;
+        } else if (this.y1 > y != this.y2 > y) {
+          res = x < (this.x2 - this.x1) * (y - this.y1) / (this.y2 - this.y1) + this.x1;
         }
       }
-      if (right != null && ((splitX == false && maxLat >= this.minLat) || (splitX && maxLon >= this.minLon))) {
-        relation = right.relate(minLat, maxLat, minLon, maxLon);
-        if (relation != Relation.CELL_OUTSIDE_QUERY) {
-          return relation;
-        }
+      if (this.left != null) {
+        res ^= left.contains(x, y, isOnEdge);
       }
-    }
-    return Relation.CELL_OUTSIDE_QUERY;
-  }
-
-  /** Returns relation to the provided rectangle for this component */
-  protected abstract Relation componentRelate(double minLat, double maxLat, double minLon, double maxLon);
-
-  /** Returns relation to the provided triangle for this component */
-  protected abstract Relation componentRelateTriangle(double ax, double ay, double bx, double by, double cx, double cy);
-
-
-  private Relation internalComponentRelateTriangle(double ax, double ay, double bx, double by, double cx, double cy) {
-    // compute bounding box of triangle
-    double minLat = StrictMath.min(StrictMath.min(ay, by), cy);
-    double minLon = StrictMath.min(StrictMath.min(ax, bx), cx);
-    double maxLat = StrictMath.max(StrictMath.max(ay, by), cy);
-    double maxLon = StrictMath.max(StrictMath.max(ax, bx), cx);
-    if (maxLon < this.minLon || minLon > this.maxLon || maxLat < this.minLat || minLat > this.maxLat) {
-      return Relation.CELL_OUTSIDE_QUERY;
-    } else if (bx == cx && by == cy) {
-      return componentRelateTriangle(bx, by, ax, ay, cx, cy);
-    } else if (ax == bx && ay == by) {
-      return componentRelateTriangle(bx, by, cx, cy, ax, ay);
-    }
-    return componentRelateTriangle(ax, ay, bx, by, cx, cy);
-  }
-
 
-  /** Returns relation to the provided rectangle for this component */
-  protected Relation internalComponentRelate(double minLat, double maxLat, double minLon, double maxLon) {
-    // if the bounding boxes are disjoint then the shape does not cross
-    if (maxLon < this.minLon || minLon > this.maxLon || maxLat < this.minLat || minLat > this.maxLat) {
-      return Relation.CELL_OUTSIDE_QUERY;
-    }
-    // if the rectangle fully encloses us, we cross.
-    if (minLat <= this.minLat && maxLat >= this.maxLat && minLon <= this.minLon && maxLon >= this.maxLon) {
-      return Relation.CELL_CROSSES_QUERY;
+      if (this.right != null && y >= this.low) {
+        res ^= right.contains(x, y, isOnEdge);
+      }
     }
-    return componentRelate(minLat, maxLat, minLon, maxLon);
+    return isOnEdge.get() || res;
   }
 
-  /** Creates tree from sorted components (with range low and high inclusive) */
-  protected static EdgeTree createTree(EdgeTree components[], int low, int high, boolean splitX) {
-    if (low > high) {
-      return null;
-    }
-    final int mid = (low + high) >>> 1;
-    if (low < high) {
-      Comparator<EdgeTree> comparator;
-      if (splitX) {
-        comparator = (left, right) -> {
-          int ret = Double.compare(left.minLon, right.minLon);
-          if (ret == 0) {
-            ret = Double.compare(left.maxX, right.maxX);
-          }
-          return ret;
-        };
-      } else {
-        comparator = (left, right) -> {
-          int ret = Double.compare(left.minLat, right.minLat);
-          if (ret == 0) {
-            ret = Double.compare(left.maxY, right.maxY);
-          }
-          return ret;
-        };
+  /** returns true if the provided x, y point lies on the line */
+  protected boolean isPointOnLine(double x, double y) {
+    if (y <= max) {
+      if (orient(x1, y1, x2, y2, x, y) == 0) {
+        return true;
+      }
+      if (left != null && left.isPointOnLine(x, y)) {
+        return true;
+      }
+      if (right != null && y >= this.low && right.isPointOnLine(x, y)) {
+        return true;
       }
-      ArrayUtil.select(components, low, high + 1, mid, comparator);
-    }
-    // add midpoint
-    EdgeTree newNode = components[mid];
-    newNode.splitX = splitX;
-    // add children
-    newNode.left = createTree(components, low, mid - 1, !splitX);
-    newNode.right = createTree(components, mid + 1, high, !splitX);
-    // pull up max values to this node
-    if (newNode.left != null) {
-      newNode.maxX = Math.max(newNode.maxX, newNode.left.maxX);
-      newNode.maxY = Math.max(newNode.maxY, newNode.left.maxY);
-    }
-    if (newNode.right != null) {
-      newNode.maxX = Math.max(newNode.maxX, newNode.right.maxX);
-      newNode.maxY = Math.max(newNode.maxY, newNode.right.maxY);
     }
-    return newNode;
+    return false;
   }
 
-  /**
-   * Internal tree node: represents geometry edge from lat1,lon1 to lat2,lon2.
-   * The sort value is {@code low}, which is the minimum latitude of the edge.
-   * {@code max} stores the maximum latitude of this edge or any children.
-   */
-  static class Edge {
-    // lat-lon pair (in original order) of the two vertices
-    final double lat1, lat2;
-    final double lon1, lon2;
-    //edge belongs to the dateline
-    final boolean dateline;
-    /** min of this edge */
-    final double low;
-    /** max latitude of this edge or any children */
-    double max;
-
-    /** left child edge, or null */
-    Edge left;
-    /** right child edge, or null */
-    Edge right;
 
-    Edge(double lat1, double lon1, double lat2, double lon2, double low, double max) {
-      this.lat1 = lat1;
-      this.lon1 = lon1;
-      this.lat2 = lat2;
-      this.lon2 = lon2;
-      this.low = low;
-      this.max = max;
-      dateline = (lon1 == GeoUtils.MIN_LON_INCL && lon2 == GeoUtils.MIN_LON_INCL)
-          || (lon1 == GeoUtils.MAX_LON_INCL && lon2 == GeoUtils.MAX_LON_INCL);
-    }
-
-    /** Returns true if the triangle crosses any edge in this edge subtree */
-    boolean crossesTriangle(double ax, double ay, double bx, double by, double cx, double cy) {
-      // compute min lat of triangle bounding box
-      double triMinLat = StrictMath.min(StrictMath.min(ay, by), cy);
-      if (triMinLat <= max) {
-        double dy = lat1;
-        double ey = lat2;
-        double dx = lon1;
-        double ex = lon2;
-
-        // compute remaining bounding box of triangle
-        double triMinLon = StrictMath.min(StrictMath.min(ax, bx), cx);
-        double triMaxLat = StrictMath.max(StrictMath.max(ay, by), cy);
-        double triMaxLon = StrictMath.max(StrictMath.max(ax, bx), cx);
+  /** Returns true if the triangle crosses any edge in this edge subtree */
+  protected boolean crossesTriangle(double minX, double maxX, double minY, double maxY,
+                          double ax, double ay, double bx, double by, double cx, double cy) {
+      if (minY <= max) {
+        double dy = y1;
+        double ey = y2;
+        double dx = x1;
+        double ex = x2;
 
         // optimization: see if the rectangle is outside of the "bounding box" of the polyline at all
         // if not, don't waste our time trying more complicated stuff
-        boolean outside = (dy < triMinLat && ey < triMinLat) ||
-            (dy > triMaxLat && ey > triMaxLat) ||
-            (dx < triMinLon && ex < triMinLon) ||
-            (dx > triMaxLon && ex > triMaxLon);
+        boolean outside = (dy < minY && ey < minY) ||
+            (dy > maxY && ey > maxY) ||
+            (dx < minX && ex < minX) ||
+            (dx > maxX && ex > maxX);
 
         if (outside == false) {
           if (lineCrossesLine(dx, dy, ex, ey, ax, ay, bx, by) ||
@@ -269,138 +157,109 @@ public abstract class EdgeTree {
           }
         }
 
-        if (left != null && left.crossesTriangle(ax, ay, bx, by, cx, cy)) {
+        if (left != null && left.crossesTriangle(minX, maxX, minY, maxY, ax, ay, bx, by, cx, cy)) {
           return true;
         }
 
-        if (right != null && triMaxLat >= low && right.crossesTriangle(ax, ay, bx, by, cx, cy)) {
+        if (right != null && maxY >= low && right.crossesTriangle(minX, maxX, minY, maxY, ax, ay, bx, by, cx, cy)) {
           return true;
         }
       }
       return false;
     }
 
-    /** Returns true if the box crosses any edge in this edge subtree */
-    boolean crossesBox(double minLat, double maxLat, double minLon, double maxLon, boolean includeBoundary) {
-      // we just have to cross one edge to answer the question, so we descend the tree and return when we do.
-      if (minLat <= max) {
-        // we compute line intersections of every polygon edge with every box line.
-        // if we find one, return true.
-        // for each box line (AB):
-        //   for each poly line (CD):
-        //     intersects = orient(C,D,A) * orient(C,D,B) <= 0 && orient(A,B,C) * orient(A,B,D) <= 0
-        double cy = lat1;
-        double dy = lat2;
-        double cx = lon1;
-        double dx = lon2;
-
-        // optimization: see if either end of the line segment is contained by the rectangle
-        if (Rectangle.containsPoint(cy, cx, minLat, maxLat, minLon, maxLon) ||
-            Rectangle.containsPoint(dy, dx, minLat, maxLat, minLon, maxLon)) {
-          return true;
-        }
-
-        // optimization: see if the rectangle is outside of the "bounding box" of the polyline at all
-        // if not, don't waste our time trying more complicated stuff
-        boolean outside = (cy < minLat && dy < minLat) ||
-            (cy > maxLat && dy > maxLat) ||
-            (cx < minLon && dx < minLon) ||
-            (cx > maxLon && dx > maxLon);
-
-        if (outside == false) {
-          if (includeBoundary == true &&
-              lineCrossesLineWithBoundary(cx, cy, dx, dy, minLon, minLat, maxLon, minLat) ||
-              lineCrossesLineWithBoundary(cx, cy, dx, dy, maxLon, minLat, maxLon, maxLat) ||
-              lineCrossesLineWithBoundary(cx, cy, dx, dy, maxLon, maxLat, maxLon, minLat) ||
-              lineCrossesLineWithBoundary(cx, cy, dx, dy, minLon, maxLat, minLon, minLat)) {
-            // include boundaries: ensures box edges that terminate on the polygon are included
-            return true;
-          } else if (lineCrossesLine(cx, cy, dx, dy, minLon, minLat, maxLon, minLat) ||
-              lineCrossesLine(cx, cy, dx, dy, maxLon, minLat, maxLon, maxLat) ||
-              lineCrossesLine(cx, cy, dx, dy, maxLon, maxLat, maxLon, minLat) ||
-              lineCrossesLine(cx, cy, dx, dy, minLon, maxLat, minLon, minLat)) {
-            return true;
-          }
-        }
+  /** Returns true if the box crosses any edge in this edge subtree */
+  protected boolean crossesBox(double minX, double maxX, double minY, double maxY, boolean includeBoundary) {
+    // we just have to cross one edge to answer the question, so we descend the tree and return when we do.
+    if (minY <= max) {
+      // we compute line intersections of every polygon edge with every box line.
+      // if we find one, return true.
+      // for each box line (AB):
+      //   for each poly line (CD):
+      //     intersects = orient(C,D,A) * orient(C,D,B) <= 0 && orient(A,B,C) * orient(A,B,D) <= 0
+      double cy = y1;
+      double dy = y2;
+      double cx = x1;
+      double dx = x2;
+
+      // optimization: see if either end of the line segment is contained by the rectangle
+      if (Rectangle.containsPoint(cy, cx, minY, maxY, minX, maxX) ||
+          Rectangle.containsPoint(dy, dx, minY, maxY, minX, maxX)) {
+        return true;
+      }
 
-        if (left != null && left.crossesBox(minLat, maxLat, minLon, maxLon, includeBoundary)) {
+      // optimization: see if the rectangle is outside of the "bounding box" of the polyline at all
+      // if not, don't waste our time trying more complicated stuff
+      boolean outside = (cy < minY && dy < minY) ||
+          (cy > maxY && dy > maxY) ||
+          (cx < minX && dx < minX) ||
+          (cx > maxX && dx > maxX);
+
+      if (outside == false) {
+        if (includeBoundary == true &&
+            lineCrossesLineWithBoundary(cx, cy, dx, dy, minX, minY, maxX, minY) ||
+            lineCrossesLineWithBoundary(cx, cy, dx, dy, maxX, minY, maxX, maxY) ||
+            lineCrossesLineWithBoundary(cx, cy, dx, dy, maxX, maxY, minX, maxY) ||
+            lineCrossesLineWithBoundary(cx, cy, dx, dy, minX, maxY, minX, minY)) {
+          // include boundaries: ensures box edges that terminate on the polygon are included
           return true;
-        }
-
-        if (right != null && maxLat >= low && right.crossesBox(minLat, maxLat, minLon, maxLon, includeBoundary)) {
+        } else if (lineCrossesLine(cx, cy, dx, dy, minX, minY, maxX, minY) ||
+            lineCrossesLine(cx, cy, dx, dy, maxX, minY, maxX, maxY) ||
+            lineCrossesLine(cx, cy, dx, dy, maxX, maxY, minX, maxY) ||
+            lineCrossesLine(cx, cy, dx, dy, minX, maxY, minX, minY)) {
           return true;
         }
       }
-      return false;
-    }
 
-    /** Returns true if the line crosses any edge in this edge subtree */
-    boolean crossesLine(double a2x, double a2y, double b2x, double b2y) {
-      double minY = StrictMath.min(a2y, b2y);
-      double maxY = StrictMath.max(a2y, b2y);
-      if (minY <= max) {
-        double a1x = lon1;
-        double a1y = lat1;
-        double b1x = lon2;
-        double b1y = lat2;
-
-        double minX = StrictMath.min(a2x, b2x);
-        double maxX = StrictMath.max(a2x, b2x);
-
-        boolean outside = (a1y < minY && b1y < minY) ||
-            (a1y > maxY && b1y > maxY) ||
-            (a1x < minX && b1x < minX) ||
-            (a1x > maxX && b1x > maxX);
-        if (outside == false && lineCrossesLineWithBoundary(a1x, a1y, b1x, b1y, a2x, a2y, b2x, b2y)) {
-          return true;
-        }
+      if (left != null && left.crossesBox(minX, maxX, minY, maxY, includeBoundary)) {
+        return true;
+      }
 
-        if (left != null && left.crossesLine(a2x, a2y, b2x, b2y)) {
-          return true;
-        }
-        if (right != null && maxY >= low && right.crossesLine(a2x, a2y, b2x, b2y)) {
-          return true;
-        }
+      if (right != null && maxY >= low && right.crossesBox(minX, maxX, minY, maxY, includeBoundary)) {
+        return true;
       }
-      return false;
     }
+    return false;
   }
 
-  //This should be moved when LatLonShape is moved from sandbox!
-  /**
-   * Compute whether the given x, y point is in a triangle; uses the winding order method */
-  protected static boolean pointInTriangle (double x, double y, double ax, double ay, double bx, double by, double cx, double cy) {
-    double minX = StrictMath.min(ax, StrictMath.min(bx, cx));
-    double minY = StrictMath.min(ay, StrictMath.min(by, cy));
-    double maxX = StrictMath.max(ax, StrictMath.max(bx, cx));
-    double maxY = StrictMath.max(ay, StrictMath.max(by, cy));
-    //check the bounding box because if the triangle is degenerated, e.g points and lines, we need to filter out
-    //coplanar points that are not part of the triangle.
-    if (x >= minX && x <= maxX && y >= minY && y <= maxY ) {
-      int a = orient(x, y, ax, ay, bx, by);
-      int b = orient(x, y, bx, by, cx, cy);
-      if (a == 0 || b == 0 || a < 0 == b < 0) {
-        int c = orient(x, y, cx, cy, ax, ay);
-        return c == 0 || (c < 0 == (b < 0 || a < 0));
+  /** Returns true if the line crosses any edge in this edge subtree */
+  protected boolean crossesLine(double minX, double maxX, double minY, double maxY, double a2x, double a2y, double b2x, double b2y) {
+    if (minY <= max) {
+      double a1x = x1;
+      double a1y = y1;
+      double b1x = x2;
+      double b1y = y2;
+
+      boolean outside = (a1y < minY && b1y < minY) ||
+          (a1y > maxY && b1y > maxY) ||
+          (a1x < minX && b1x < minX) ||
+          (a1x > maxX && b1x > maxX);
+      if (outside == false && lineCrossesLineWithBoundary(a1x, a1y, b1x, b1y, a2x, a2y, b2x, b2y)) {
+        return true;
+      }
+
+      if (left != null && left.crossesLine(minX, maxX, minY, maxY, a2x, a2y, b2x, b2y)) {
+        return true;
+      }
+      if (right != null && maxY >= low && right.crossesLine(minX, maxX, minY, maxY, a2x, a2y, b2x, b2y)) {
+        return true;
       }
-      return false;
-    } else {
-      return false;
     }
+    return false;
   }
 
   /**
    * Creates an edge interval tree from a set of geometry vertices.
    * @return root node of the tree.
    */
-  private static Edge createTree(double[] lats, double[] lons) {
-    Edge edges[] = new Edge[lats.length - 1];
-    for (int i = 1; i < lats.length; i++) {
-      double lat1 = lats[i-1];
-      double lon1 = lons[i-1];
-      double lat2 = lats[i];
-      double lon2 = lons[i];
-      edges[i - 1] = new Edge(lat1, lon1, lat2, lon2, Math.min(lat1, lat2), Math.max(lat1, lat2));
+  protected static EdgeTree createTree(double[] x, double[] y) {
+    EdgeTree edges[] = new EdgeTree[x.length - 1];
+    for (int i = 1; i < x.length; i++) {
+      double x1 = x[i-1];
+      double y1 = y[i-1];
+      double x2 = x[i];
+      double y2 = y[i];
+      edges[i - 1] = new EdgeTree(x1, y1, x2, y2, Math.min(y1, y2), Math.max(y1, y2));
     }
     // sort the edges then build a balanced tree from them
     Arrays.sort(edges, (left, right) -> {
@@ -414,13 +273,13 @@ public abstract class EdgeTree {
   }
 
   /** Creates tree from sorted edges (with range low and high inclusive) */
-  private static Edge createTree(Edge edges[], int low, int high) {
+  private static EdgeTree createTree(EdgeTree edges[], int low, int high) {
     if (low > high) {
       return null;
     }
     // add midpoint
     int mid = (low + high) >>> 1;
-    Edge newNode = edges[mid];
+    EdgeTree newNode = edges[mid];
     // add children
     newNode.left = createTree(edges, low, mid - 1);
     newNode.right = createTree(edges, mid + 1, high);
diff --git a/lucene/core/src/java/org/apache/lucene/geo/GeoEncodingUtils.java b/lucene/core/src/java/org/apache/lucene/geo/GeoEncodingUtils.java
index 00b7252..5c7078d 100644
--- a/lucene/core/src/java/org/apache/lucene/geo/GeoEncodingUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/geo/GeoEncodingUtils.java
@@ -181,10 +181,10 @@ public final class GeoEncodingUtils {
   /** Create a predicate that checks whether points are within a polygon.
    *  It works the same way as {@link #createDistancePredicate}.
    *  @lucene.internal */
-  public static PolygonPredicate createPolygonPredicate(Polygon[] polygons, Polygon2D tree) {
-    final Rectangle boundingBox = Rectangle.fromPolygon(polygons);
+  public static PolygonPredicate createComponentPredicate(Component2D tree) {
+    final Rectangle boundingBox = new Rectangle(tree.getMinY(), tree.getMaxY(), tree.getMinX(), tree.getMaxX());
     final Function<Rectangle, Relation> boxToRelation = box -> tree.relate(
-        box.minLat, box.maxLat, box.minLon, box.maxLon);
+        box.minLon, box.maxLon, box.minLat, box.maxLat);
     final Grid subBoxes = createSubBoxes(boundingBox, boxToRelation);
 
     return new PolygonPredicate(
@@ -238,8 +238,9 @@ public final class GeoEncodingUtils {
         final int boxMaxLon = boxMinLon + (1 << lonShift) - 1;
 
         relations[i * maxLonDelta + j] = (byte) boxToRelation.apply(new Rectangle(
-            decodeLatitude(boxMinLat), decodeLatitude(boxMaxLat),
-            decodeLongitude(boxMinLon), decodeLongitude(boxMaxLon))).ordinal();
+                decodeLatitude(boxMinLat), decodeLatitude(boxMaxLat),
+            decodeLongitude(boxMinLon), decodeLongitude(boxMaxLon))
+            ).ordinal();
       }
     }
 
@@ -344,14 +345,14 @@ public final class GeoEncodingUtils {
   /** A predicate that checks whether a given point is within a polygon. */
   public static class PolygonPredicate extends Grid {
 
-    private final Polygon2D tree;
+    private final Component2D tree;
 
     private PolygonPredicate(
         int latShift, int lonShift,
         int latBase, int lonBase,
         int maxLatDelta, int maxLonDelta,
         byte[] relations,
-        Polygon2D tree) {
+        Component2D tree) {
       super(latShift, lonShift, latBase, lonBase, maxLatDelta, maxLonDelta, relations);
       this.tree = tree;
     }
@@ -375,7 +376,7 @@ public final class GeoEncodingUtils {
 
       final int relation = relations[(lat2 - latBase) * maxLonDelta + (lon2 - lonBase)];
       if (relation == Relation.CELL_CROSSES_QUERY.ordinal()) {
-        return tree.contains(decodeLatitude(lat), decodeLongitude(lon));
+        return tree.contains(decodeLongitude(lon), decodeLatitude(lat));
       } else {
         return relation == Relation.CELL_INSIDE_QUERY.ordinal();
       }
diff --git a/lucene/core/src/java/org/apache/lucene/geo/Polygon2D.java b/lucene/core/src/java/org/apache/lucene/geo/Polygon2D.java
index 49dfefa..3b06d21 100644
--- a/lucene/core/src/java/org/apache/lucene/geo/Polygon2D.java
+++ b/lucene/core/src/java/org/apache/lucene/geo/Polygon2D.java
@@ -27,22 +27,54 @@ import org.apache.lucene.index.PointValues.Relation;
  * http://www-ma2.upc.es/geoc/Schirra-pointPolygon.pdf</a>.
  * @lucene.internal
  */
-// Both Polygon.contains() and Polygon.crossesSlowly() loop all edges, and first check that the edge is within a range.
-// we just organize the edges to do the same computations on the same subset of edges more efficiently.
-public class Polygon2D extends EdgeTree {
-  // each component/hole is a node in an augmented 2d kd-tree: we alternate splitting between latitude/longitude,
-  // and pull up max values for both dimensions to each parent node (regardless of split).
+
+public class Polygon2D implements Component2D {
+  /** minimum latitude of this geometry's bounding box area */
+  final private double minY;
+  /** maximum latitude of this geometry's bounding box area */
+  final private double maxY;
+  /** minimum longitude of this geometry's bounding box area */
+  final private double minX;
+  /** maximum longitude of this geometry's bounding box area */
+  final private double maxX;
   /** tree of holes, or null */
-  protected final Polygon2D holes;
-  private final AtomicBoolean containsBoundary = new AtomicBoolean(false);
+  final protected Component2D holes;
+  /** Edges of the polygon represented as a 2-d interval tree.*/
+  final EdgeTree tree;
+  /** helper boolean for points on boundary */
+  final private AtomicBoolean containsBoundary = new AtomicBoolean(false);
 
-  protected Polygon2D(final double minLat, final double maxLat, final double minLon, final double maxLon, double[] lats, double[] lons, Polygon2D holes) {
-    super(minLat, maxLat, minLon, maxLon, lats, lons);
+  protected Polygon2D(final double minX, final double maxX, final double minY, final double maxY, double[] x, double[] y, Component2D holes) {
+    this.minY = minY;
+    this.maxY = maxY;
+    this.minX = minX;
+    this.maxX = maxX;
     this.holes = holes;
+    this.tree = EdgeTree.createTree(x, y);
+  }
+
+  protected Polygon2D(Polygon polygon, Component2D holes) {
+    this(polygon.minLon, polygon.maxLon, polygon.minLat, polygon.maxLat, polygon.getPolyLons(), polygon.getPolyLats(), holes);
+  }
+
+  @Override
+  public double getMinX() {
+    return minX;
+  }
+
+  @Override
+  public double getMaxX() {
+    return maxX;
   }
 
-  protected Polygon2D(Polygon polygon, Polygon2D holes) {
-    this(polygon.minLat, polygon.maxLat, polygon.minLon, polygon.maxLon, polygon.getPolyLats(), polygon.getPolyLons(), holes);
+  @Override
+  public double getMinY() {
+    return minY;
+  }
+
+  @Override
+  public double getMaxY() {
+    return maxY;
   }
 
   /**
@@ -51,34 +83,18 @@ public class Polygon2D extends EdgeTree {
    * See <a href="https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html">
    * https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html</a> for more information.
    */
-  public boolean contains(double latitude, double longitude) {
-    if (latitude <= maxY && longitude <= maxX) {
-      if (componentContains(latitude, longitude)) {
-        return true;
-      }
-      if (left != null) {
-        if (((Polygon2D)left).contains(latitude, longitude)) {
-          return true;
-        }
-      }
-      if (right != null && ((splitX == false && latitude >= minLat) || (splitX && longitude >= minLon))) {
-        if (((Polygon2D)right).contains(latitude, longitude)) {
-          return true;
-        }
-      }
+  @Override
+  public boolean contains(double x, double y) {
+    if (Component2D.containsPoint(x, y, minX, maxX, minY, maxY)) {
+      return internalContains(x, y);
     }
     return false;
   }
 
-  /** Returns true if the point is contained within this polygon component. */
-  private boolean componentContains(double latitude, double longitude) {
-    // check bounding box
-    if (latitude < minLat || latitude > maxLat || longitude < minLon || longitude > maxLon) {
-      return false;
-    }
+  private boolean internalContains(double x, double y) {
     containsBoundary.set(false);
-    if (contains(tree, latitude, longitude, containsBoundary)) {
-      if (holes != null && holes.contains(latitude, longitude)) {
+    if (tree.contains(x, y, containsBoundary)) {
+      if (holes != null && holes.contains(x, y)) {
         return false;
       }
       return true;
@@ -87,10 +103,16 @@ public class Polygon2D extends EdgeTree {
   }
 
   @Override
-  protected Relation componentRelate(double minLat, double maxLat, double minLon, double maxLon) {
+  public Relation relate(double minX, double maxX, double minY, double maxY) {
+    if (Component2D.disjoint(this.minX, this.maxX, this.minY, this.maxY, minX, maxX, minY, maxY)) {
+      return Relation.CELL_OUTSIDE_QUERY;
+    }
+    if (Component2D.within(this.minX, this.maxX, this.minY, this.maxY, minX, maxX, minY, maxY)) {
+      return Relation.CELL_CROSSES_QUERY;
+    }
     // check any holes
     if (holes != null) {
-      Relation holeRelation = holes.relate(minLat, maxLat, minLon, maxLon);
+      Relation holeRelation = holes.relate(minX, maxX, minY, maxY);
       if (holeRelation == Relation.CELL_CROSSES_QUERY) {
         return Relation.CELL_CROSSES_QUERY;
       } else if (holeRelation == Relation.CELL_INSIDE_QUERY) {
@@ -98,17 +120,17 @@ public class Polygon2D extends EdgeTree {
       }
     }
     // check each corner: if < 4 && > 0 are present, its cheaper than crossesSlowly
-    int numCorners = numberOfCorners(minLat, maxLat, minLon, maxLon);
+    int numCorners = numberOfCorners(minX, maxX, minY, maxY);
     if (numCorners == 4) {
-      if (tree.crossesBox(minLat, maxLat, minLon, maxLon, false)) {
+      if (tree.crossesBox(minX, maxX, minY, maxY, false)) {
         return Relation.CELL_CROSSES_QUERY;
       }
       return Relation.CELL_INSIDE_QUERY;
     }  else if (numCorners == 0) {
-      if (minLat >= tree.lat1 && maxLat <= tree.lat1 && minLon >= tree.lon2 && maxLon <= tree.lon2) {
+      if (Component2D.containsPoint(tree.x1, tree.y1, minX, maxX, minY, maxY)) {
         return Relation.CELL_CROSSES_QUERY;
       }
-      if (tree.crossesBox(minLat, maxLat, minLon, maxLon, false)) {
+      if (tree.crossesBox(minX, maxX, minY, maxY, false)) {
         return Relation.CELL_CROSSES_QUERY;
       }
       return Relation.CELL_OUTSIDE_QUERY;
@@ -117,10 +139,14 @@ public class Polygon2D extends EdgeTree {
   }
 
   @Override
-  protected Relation componentRelateTriangle(double ax, double ay, double bx, double by, double cx, double cy) {
+  public Relation relateTriangle(double minX, double maxX, double minY, double maxY,
+                                 double ax, double ay, double bx, double by, double cx, double cy) {
+    if (Component2D.disjoint(this.minX, this.maxX, this.minY, this.maxY, minX, maxX, minY, maxY)) {
+      return Relation.CELL_OUTSIDE_QUERY;
+    }
     // check any holes
     if (holes != null) {
-      Relation holeRelation = holes.relateTriangle(ax, ay, bx, by, cx, cy);
+      Relation holeRelation = holes.relateTriangle(minX, maxX, minY, maxY, ax, ay, bx, by, cx, cy);
       if (holeRelation == Relation.CELL_CROSSES_QUERY) {
         return Relation.CELL_CROSSES_QUERY;
       } else if (holeRelation == Relation.CELL_INSIDE_QUERY) {
@@ -129,33 +155,37 @@ public class Polygon2D extends EdgeTree {
     }
     if (ax == bx && bx == cx && ay == by && by == cy) {
       // indexed "triangle" is a point: shortcut by checking contains
-      return contains(ay, ax) ? Relation.CELL_INSIDE_QUERY : Relation.CELL_OUTSIDE_QUERY;
+      return internalContains(ax, ay) ? Relation.CELL_INSIDE_QUERY : Relation.CELL_OUTSIDE_QUERY;
     } else if ((ax == cx && ay == cy) || (bx == cx && by == cy)) {
       // indexed "triangle" is a line segment: shortcut by calling appropriate method
-      return relateIndexedLineSegment(ax, ay, bx, by);
+      return relateIndexedLineSegment(minX, maxX, minY, maxY, ax, ay, bx, by);
+    } else if ((ax == bx && ay == by)) {
+      // indexed "triangle" is a line segment: shortcut by calling appropriate method
+      return relateIndexedLineSegment(minX, maxX, minY, maxY, ax, ay, cx, cy);
     }
     // indexed "triangle" is a triangle:
-    return relateIndexedTriangle(ax, ay, bx, by, cx, cy);
+    return relateIndexedTriangle(minX, maxX, minY, maxY, ax, ay, bx, by, cx, cy);
   }
 
   /** relates an indexed line segment (a "flat triangle") with the polygon */
-  private Relation relateIndexedLineSegment(double a2x, double a2y, double b2x, double b2y) {
+  private Relation relateIndexedLineSegment(double minX, double maxX, double minY, double maxY,
+                                            double a2x, double a2y, double b2x, double b2y) {
     // check endpoints of the line segment
     int numCorners = 0;
-    if (componentContains(a2y, a2x)) {
+    if (contains(a2x, a2y)) {
       ++numCorners;
     }
-    if (componentContains(b2y, b2x)) {
+    if (contains(b2x, b2y)) {
       ++numCorners;
     }
 
     if (numCorners == 2) {
-      if (tree.crossesLine(a2x, a2y, b2x, b2y)) {
+      if (tree.crossesLine(minX, maxX, minY, maxY, a2x, a2y, b2x, b2y)) {
         return Relation.CELL_CROSSES_QUERY;
       }
       return Relation.CELL_INSIDE_QUERY;
     } else if (numCorners == 0) {
-      if (tree.crossesLine(a2x, a2y, b2x, b2y)) {
+      if (tree.crossesLine(minX, maxX, minY, maxY, a2x, a2y, b2x, b2y)) {
         return Relation.CELL_CROSSES_QUERY;
       }
       return Relation.CELL_OUTSIDE_QUERY;
@@ -164,19 +194,20 @@ public class Polygon2D extends EdgeTree {
   }
 
   /** relates an indexed triangle with the polygon */
-  private Relation relateIndexedTriangle(double ax, double ay, double bx, double by, double cx, double cy) {
+  private Relation relateIndexedTriangle(double minX, double maxX, double minY, double maxY,
+                                         double ax, double ay, double bx, double by, double cx, double cy) {
     // check each corner: if < 3 && > 0 are present, its cheaper than crossesSlowly
     int numCorners = numberOfTriangleCorners(ax, ay, bx, by, cx, cy);
     if (numCorners == 3) {
-      if (tree.crossesTriangle(ax, ay, bx, by, cx, cy)) {
+      if (tree.crossesTriangle(minX, maxX, minY, maxY, ax, ay, bx, by, cx, cy)) {
         return Relation.CELL_CROSSES_QUERY;
       }
       return Relation.CELL_INSIDE_QUERY;
     } else if (numCorners == 0) {
-      if (pointInTriangle(tree.lon1, tree.lat1, ax, ay, bx, by, cx, cy) == true) {
+      if (Component2D.pointInTriangle(minX, maxX, minY, maxY, tree.x1, tree.y1, ax, ay, bx, by, cx, cy) == true) {
         return Relation.CELL_CROSSES_QUERY;
       }
-      if (tree.crossesTriangle(ax, ay, bx, by, cx, cy)) {
+      if (tree.crossesTriangle(minX, maxX, minY, maxY, ax, ay, bx, by, cx, cy)) {
         return Relation.CELL_CROSSES_QUERY;
       }
       return Relation.CELL_OUTSIDE_QUERY;
@@ -186,113 +217,57 @@ public class Polygon2D extends EdgeTree {
 
   private int numberOfTriangleCorners(double ax, double ay, double bx, double by, double cx, double cy) {
     int containsCount = 0;
-    if (componentContains(ay, ax)) {
+    if (contains(ax, ay)) {
       containsCount++;
     }
-    if (componentContains(by, bx)) {
+    if (contains(bx, by)) {
       containsCount++;
     }
     if (containsCount == 1) {
       return containsCount;
     }
-    if (componentContains(cy, cx)) {
+    if (contains(cx, cy)) {
       containsCount++;
     }
     return containsCount;
   }
 
   // returns 0, 4, or something in between
-  private int numberOfCorners(double minLat, double maxLat, double minLon, double maxLon) {
+  private int numberOfCorners(double minX, double maxX, double minY, double maxY) {
     int containsCount = 0;
-    if (componentContains(minLat, minLon)) {
+    if (contains(minX, minY)) {
       containsCount++;
     }
-    if (componentContains(minLat, maxLon)) {
+    if (contains(maxX, minY)) {
       containsCount++;
     }
     if (containsCount == 1) {
       return containsCount;
     }
-    if (componentContains(maxLat, maxLon)) {
+    if (contains(maxX, maxY)) {
       containsCount++;
     }
     if (containsCount == 2) {
       return containsCount;
     }
-    if (componentContains(maxLat, minLon)) {
+    if (contains(minX, maxY)) {
       containsCount++;
     }
     return containsCount;
   }
 
   /** Builds a Polygon2D from multipolygon */
-  public static Polygon2D create(Polygon... polygons) {
-    Polygon2D components[] = new Polygon2D[polygons.length];
+  public static Component2D create(Polygon... polygons) {
+    Component2D components[] = new Component2D[polygons.length];
     for (int i = 0; i < components.length; i++) {
       Polygon gon = polygons[i];
       Polygon gonHoles[] = gon.getHoles();
-      Polygon2D holes = null;
+      Component2D holes = null;
       if (gonHoles.length > 0) {
         holes = create(gonHoles);
       }
       components[i] = new Polygon2D(gon, holes);
     }
-    return (Polygon2D)createTree(components, 0, components.length - 1, false);
-  }
-
-  /**
-   * Returns true if the point crosses this edge subtree an odd number of times
-   * <p>
-   * See <a href="https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html">
-   * https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html</a> for more information.
-   */
-  // ported to java from https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html
-  // original code under the BSD license (https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html#License%20to%20Use)
-  //
-  // Copyright (c) 1970-2003, Wm. Randolph Franklin
-  //
-  // Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
-  // documentation files (the "Software"), to deal in the Software without restriction, including without limitation
-  // the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and
-  // to permit persons to whom the Software is furnished to do so, subject to the following conditions:
-  //
-  // 1. Redistributions of source code must retain the above copyright
-  //    notice, this list of conditions and the following disclaimers.
-  // 2. Redistributions in binary form must reproduce the above copyright
-  //    notice in the documentation and/or other materials provided with
-  //    the distribution.
-  // 3. The name of W. Randolph Franklin may not be used to endorse or
-  //    promote products derived from this Software without specific
-  //    prior written permission.
-  //
-  // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED
-  // TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL
-  // THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF
-  // CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
-  // IN THE SOFTWARE.
-  private static boolean contains(Edge edge, double lat, double lon, AtomicBoolean isOnEdge) {
-    boolean res = false;
-    if (isOnEdge.get() == false && lat <= edge.max) {
-      if (lat == edge.lat1 && lat == edge.lat2 ||
-          (lat <= edge.lat1 && lat >= edge.lat2) != (lat >= edge.lat1 && lat <= edge.lat2)) {
-        if ((lon == edge.lon1 && lon == edge.lon2) ||
-            ((lon <= edge.lon1 && lon >= edge.lon2) != (lon >= edge.lon1 && lon <= edge.lon2) &&
-            GeoUtils.orient(edge.lon1, edge.lat1, edge.lon2, edge.lat2, lon, lat) == 0)) {
-          // if its on the boundary return true
-          isOnEdge.set(true);
-          return true;
-        } else if (edge.lat1 > lat != edge.lat2 > lat) {
-          res = lon < (edge.lon2 - edge.lon1) * (lat - edge.lat1) / (edge.lat2 - edge.lat1) + edge.lon1;
-        }
-      }
-      if (edge.left != null) {
-        res ^= contains(edge.left, lat, lon, isOnEdge);
-      }
-
-      if (edge.right != null && lat >= edge.low) {
-        res ^= contains(edge.right, lat, lon, isOnEdge);
-      }
-    }
-    return isOnEdge.get() || res;
+    return ComponentTree.create(components);
   }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/geo/TestPolygon2D.java b/lucene/core/src/test/org/apache/lucene/geo/TestPolygon2D.java
index c43517b..61cd2b5 100644
--- a/lucene/core/src/test/org/apache/lucene/geo/TestPolygon2D.java
+++ b/lucene/core/src/test/org/apache/lucene/geo/TestPolygon2D.java
@@ -35,7 +35,7 @@ public class TestPolygon2D extends LuceneTestCase {
     Polygon hole = new Polygon(new double[] { -10, -10, 10, 10, -10 }, new double[] { -10, 10, 10, -10, -10 });
     Polygon outer = new Polygon(new double[] { -50, -50, 50, 50, -50 }, new double[] { -50, 50, 50, -50, -50 }, hole);
     Polygon island = new Polygon(new double[] { -5, -5, 5, 5, -5 }, new double[] { -5, 5, 5, -5, -5 } );
-    Polygon2D polygon = Polygon2D.create(outer, island);
+    Component2D polygon = Polygon2D.create(outer, island);
     
     // contains(point)
     assertTrue(polygon.contains(-2, 2)); // on the island
@@ -66,21 +66,21 @@ public class TestPolygon2D extends LuceneTestCase {
     double yMax = 1;//5;
 
     // test cell crossing poly
-    Polygon2D polygon = Polygon2D.create(new Polygon(py, px));
+    Component2D polygon = Polygon2D.create(new Polygon(py, px));
     assertEquals(Relation.CELL_CROSSES_QUERY, polygon.relate(yMin, yMax, xMin, xMax));
   }
   
   public void testBoundingBox() throws Exception {
     for (int i = 0; i < 100; i++) {
-      Polygon2D polygon = Polygon2D.create(nextPolygon());
+      Component2D polygon = Polygon2D.create(nextPolygon());
       
       for (int j = 0; j < 100; j++) {
         double latitude = nextLatitude();
         double longitude = nextLongitude();
         // if the point is within poly, then it should be in our bounding box
-        if (polygon.contains(latitude, longitude)) {
-          assertTrue(latitude >= polygon.minLat && latitude <= polygon.maxLat);
-          assertTrue(longitude >= polygon.minLon && longitude <= polygon.maxLon);
+        if (polygon.contains(longitude, latitude)) {
+          assertTrue(latitude >= polygon.getMinY() && latitude <= polygon.getMaxY());
+          assertTrue(longitude >= polygon.getMinX() && longitude <= polygon.getMaxX());
         }
       }
     }
@@ -90,14 +90,14 @@ public class TestPolygon2D extends LuceneTestCase {
   public void testBoundingBoxEdgeCases() throws Exception {
     for (int i = 0; i < 100; i++) {
       Polygon polygon = nextPolygon();
-      Polygon2D impl = Polygon2D.create(polygon);
+      Component2D impl = Polygon2D.create(polygon);
       
       for (int j = 0; j < 100; j++) {
         double point[] = GeoTestUtil.nextPointNear(polygon);
         double latitude = point[0];
         double longitude = point[1];
         // if the point is within poly, then it should be in our bounding box
-        if (impl.contains(latitude, longitude)) {
+        if (impl.contains(longitude, latitude)) {
           assertTrue(latitude >= polygon.minLat && latitude <= polygon.maxLat);
           assertTrue(longitude >= polygon.minLon && longitude <= polygon.maxLon);
         }
@@ -110,7 +110,7 @@ public class TestPolygon2D extends LuceneTestCase {
     int iters = atLeast(50);
     for (int i = 0; i < iters; i++) {
       Polygon polygon = nextPolygon();
-      Polygon2D impl = Polygon2D.create(polygon);
+      Component2D impl = Polygon2D.create(polygon);
       
       for (int j = 0; j < 100; j++) {
         Rectangle rectangle = GeoTestUtil.nextBoxNear(polygon);
@@ -147,12 +147,12 @@ public class TestPolygon2D extends LuceneTestCase {
   public void testContainsEdgeCases() throws Exception {
     for (int i = 0; i < 1000; i++) {
       Polygon polygon = nextPolygon();
-      Polygon2D impl = Polygon2D.create(polygon);
+      Component2D impl = Polygon2D.create(polygon);
 
       for (int j = 0; j < 10; j++) {
         Rectangle rectangle = GeoTestUtil.nextBoxNear(polygon);
         // allowed to conservatively return false
-        if (impl.relate(rectangle.minLat, rectangle.maxLat, rectangle.minLon, rectangle.maxLon) == Relation.CELL_INSIDE_QUERY) {
+        if (impl.relate(rectangle.minLon, rectangle.maxLon, rectangle.minLat, rectangle.maxLat) == Relation.CELL_INSIDE_QUERY) {
           for (int k = 0; k < 100; k++) {
             // this tests in our range but sometimes outside! so we have to double-check its really in other box
             double point[] = GeoTestUtil.nextPointNear(rectangle);
@@ -160,7 +160,7 @@ public class TestPolygon2D extends LuceneTestCase {
             double longitude = point[1];
             // check for sure its in our box
             if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
-              assertTrue(impl.contains(latitude, longitude));
+              assertTrue(impl.contains(longitude, latitude));
             }
           }
           for (int k = 0; k < 20; k++) {
@@ -170,7 +170,7 @@ public class TestPolygon2D extends LuceneTestCase {
             double longitude = point[1];
             // check for sure its in our box
             if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
-              assertTrue(impl.contains(latitude, longitude));
+              assertTrue(impl.contains(longitude, latitude));
             }
           }
         }
@@ -183,12 +183,12 @@ public class TestPolygon2D extends LuceneTestCase {
     int iters = atLeast(10);
     for (int i = 0; i < iters; i++) {
       Polygon polygon = nextPolygon();
-      Polygon2D impl = Polygon2D.create(polygon);
+      Component2D impl = Polygon2D.create(polygon);
       
       for (int j = 0; j < 100; j++) {
         Rectangle rectangle = GeoTestUtil.nextBoxNear(polygon);
         // allowed to conservatively return true.
-        if (impl.relate(rectangle.minLat, rectangle.maxLat, rectangle.minLon, rectangle.maxLon) == Relation.CELL_OUTSIDE_QUERY) {
+        if (impl.relate(rectangle.minLon, rectangle.maxLon, rectangle.minLat, rectangle.maxLat) == Relation.CELL_OUTSIDE_QUERY) {
           for (int k = 0; k < 1000; k++) {
             double point[] = GeoTestUtil.nextPointNear(rectangle);
             // this tests in our range but sometimes outside! so we have to double-check its really in other box
@@ -196,7 +196,7 @@ public class TestPolygon2D extends LuceneTestCase {
             double longitude = point[1];
             // check for sure its in our box
             if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
-              assertFalse(impl.contains(latitude, longitude));
+              assertFalse(impl.contains(longitude, latitude));
             }
           }
           for (int k = 0; k < 100; k++) {
@@ -206,7 +206,7 @@ public class TestPolygon2D extends LuceneTestCase {
             double longitude = point[1];
             // check for sure its in our box
             if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
-              assertFalse(impl.contains(latitude, longitude));
+              assertFalse(impl.contains(longitude, latitude));
             }
           }
         }
@@ -220,12 +220,12 @@ public class TestPolygon2D extends LuceneTestCase {
   public void testIntersectEdgeCases() {
     for (int i = 0; i < 100; i++) {
       Polygon polygon = nextPolygon();
-      Polygon2D impl = Polygon2D.create(polygon);
+      Component2D impl = Polygon2D.create(polygon);
 
       for (int j = 0; j < 10; j++) {
         Rectangle rectangle = GeoTestUtil.nextBoxNear(polygon);
         // allowed to conservatively return false.
-        if (impl.relate(rectangle.minLat, rectangle.maxLat, rectangle.minLon, rectangle.maxLon) == Relation.CELL_OUTSIDE_QUERY) {
+        if (impl.relate(rectangle.minLon, rectangle.maxLon, rectangle.minLat, rectangle.maxLat) == Relation.CELL_OUTSIDE_QUERY) {
           for (int k = 0; k < 100; k++) {
             // this tests in our range but sometimes outside! so we have to double-check its really in other box
             double point[] = GeoTestUtil.nextPointNear(rectangle);
@@ -233,7 +233,7 @@ public class TestPolygon2D extends LuceneTestCase {
             double longitude = point[1];
             // check for sure its in our box
             if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
-              assertFalse(impl.contains(latitude, longitude));
+              assertFalse(impl.contains(longitude, latitude));
             }
           }
           for (int k = 0; k < 50; k++) {
@@ -243,7 +243,7 @@ public class TestPolygon2D extends LuceneTestCase {
             double longitude = point[1];
             // check for sure its in our box
             if (latitude >= rectangle.minLat && latitude <= rectangle.maxLat && longitude >= rectangle.minLon && longitude <= rectangle.maxLon) {
-              assertFalse(impl.contains(latitude, longitude));
+              assertFalse(impl.contains(longitude, latitude));
             }
           }
         }
@@ -253,23 +253,23 @@ public class TestPolygon2D extends LuceneTestCase {
   
   /** Tests edge case behavior with respect to insideness */
   public void testEdgeInsideness() {
-    Polygon2D poly = Polygon2D.create(new Polygon(new double[] { -2, -2, 2, 2, -2 }, new double[] { -2, 2, 2, -2, -2 }));
+    Component2D poly = Polygon2D.create(new Polygon(new double[] { -2, -2, 2, 2, -2 }, new double[] { -2, 2, 2, -2, -2 }));
     assertTrue(poly.contains(-2, -2)); // bottom left corner: true
-    assertTrue(poly.contains(-2, 2));  // bottom right corner: true
-    assertTrue(poly.contains(2, -2));  // top left corner: true
+    assertTrue(poly.contains(2, -2));  // bottom right corner: true
+    assertTrue(poly.contains(-2, 2));  // top left corner: true
     assertTrue(poly.contains(2,  2));  // top right corner: true
-    assertTrue(poly.contains(-2, -1)); // bottom side: true
-    assertTrue(poly.contains(-2, 0));  // bottom side: true
-    assertTrue(poly.contains(-2, 1));  // bottom side: true
-    assertTrue(poly.contains(2, -1));  // top side: true
-    assertTrue(poly.contains(2, 0));   // top side: true
-    assertTrue(poly.contains(2, 1));   // top side: true
-    assertTrue(poly.contains(-1, 2));  // right side: true
-    assertTrue(poly.contains(0, 2));   // right side: true
-    assertTrue(poly.contains(1, 2));   // right side: true
-    assertTrue(poly.contains(-1, -2)); // left side: true
-    assertTrue(poly.contains(0, -2));  // left side: true
-    assertTrue(poly.contains(1, -2));  // left side: true
+    assertTrue(poly.contains(-1, -2)); // bottom side: true
+    assertTrue(poly.contains(0, -2));  // bottom side: true
+    assertTrue(poly.contains(1, -2));  // bottom side: true
+    assertTrue(poly.contains(-1, 2));  // top side: true
+    assertTrue(poly.contains(0, 2));   // top side: true
+    assertTrue(poly.contains(1, 2));   // top side: true
+    assertTrue(poly.contains(2, -1));  // right side: true
+    assertTrue(poly.contains(2, 0));   // right side: true
+    assertTrue(poly.contains(2, 1));   // right side: true
+    assertTrue(poly.contains(-2, -1)); // left side: true
+    assertTrue(poly.contains(-2, 0));  // left side: true
+    assertTrue(poly.contains(-2, 1));  // left side: true
   }
   
   /** Tests current impl against original algorithm */
@@ -281,14 +281,14 @@ public class TestPolygon2D extends LuceneTestCase {
       while (polygon.getHoles().length > 0) {
         polygon = nextPolygon();
       }
-      Polygon2D impl = Polygon2D.create(polygon);
+      Component2D impl = Polygon2D.create(polygon);
       
       // random lat/lons against polygon
       for (int j = 0; j < 1000; j++) {
         double point[] = GeoTestUtil.nextPointNear(polygon);
         double latitude = point[0];
         double longitude = point[1];
-        boolean expected = GeoTestUtil.containsSlowly(polygon, latitude, longitude);
+        boolean expected = GeoTestUtil.containsSlowly(polygon, longitude, latitude);
         assertEquals(expected, impl.contains(latitude, longitude));
       }
     }
@@ -298,7 +298,7 @@ public class TestPolygon2D extends LuceneTestCase {
   public void testRelateTriangle() {
     for (int i = 0; i < 100; ++i) {
       Polygon polygon = nextPolygon();
-      Polygon2D impl = Polygon2D.create(polygon);
+      Component2D impl = Polygon2D.create(polygon);
 
       for (int j = 0; j < 100; j++) {
         double[] a = nextPointNear(polygon);
@@ -306,7 +306,7 @@ public class TestPolygon2D extends LuceneTestCase {
         double[] c = nextPointNear(polygon);
 
         // if the point is within poly, then triangle should not intersect
-        if (impl.contains(a[0], a[1]) || impl.contains(b[0], b[1]) || impl.contains(c[0], c[1])) {
+        if (impl.contains(a[1], a[0]) || impl.contains(b[1], b[0]) || impl.contains(c[1], c[0])) {
           assertTrue(impl.relateTriangle(a[1], a[0], b[1], b[0], c[1], c[0]) != Relation.CELL_OUTSIDE_QUERY);
         }
       }
@@ -315,7 +315,7 @@ public class TestPolygon2D extends LuceneTestCase {
 
   public void testRelateTriangleContainsPolygon() {
     Polygon polygon = new Polygon(new double[]{0, 0, 1, 1, 0}, new double[]{0, 1, 1, 0, 0});
-    Polygon2D impl = Polygon2D.create(polygon);
+    Component2D impl = Polygon2D.create(polygon);
     assertEquals(Relation.CELL_CROSSES_QUERY, impl.relateTriangle(-10 , -1, 2, -1, 10, 10));
   }
 
@@ -327,7 +327,7 @@ public class TestPolygon2D extends LuceneTestCase {
       // random number of vertices
       int numVertices = RandomNumbers.randomIntBetween(random(), 100, 1000);
       Polygon polygon = createRegularPolygon(0, 0, randomRadius, numVertices);
-      Polygon2D impl = Polygon2D.create(polygon);
+      Component2D impl = Polygon2D.create(polygon);
 
       // create and test a simple tessellation
       for (int j = 1; j < numVertices; ++j) {
@@ -342,7 +342,7 @@ public class TestPolygon2D extends LuceneTestCase {
 
   public void testLineCrossingPolygonPoints() {
     Polygon p = new Polygon(new double[] {0, -1, 0, 1, 0}, new double[] {-1, 0, 1, 0, -1});
-    Polygon2D polygon2D = Polygon2D.create(p);
+    Component2D polygon2D = Polygon2D.create(p);
     Relation rel = polygon2D.relateTriangle(GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(-1.5)),
         GeoEncodingUtils.decodeLatitude(GeoEncodingUtils.encodeLatitude(0)),
         GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(1.5)),
@@ -354,7 +354,7 @@ public class TestPolygon2D extends LuceneTestCase {
 
   public void testRandomLineCrossingPolygon() {
     Polygon p = GeoTestUtil.createRegularPolygon(0, 0, 1000, TestUtil.nextInt(random(), 100, 10000));
-    Polygon2D polygon2D = Polygon2D.create(p);
+    Component2D polygon2D = Polygon2D.create(p);
     for (int i=0; i < 1000; i ++) {
       double longitude = GeoTestUtil.nextLongitude();
       double latitude = GeoTestUtil.nextLatitude();
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeLineQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeLineQuery.java
index b6c300f..04eebcc 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeLineQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeLineQuery.java
@@ -19,6 +19,7 @@ package org.apache.lucene.document;
 import java.util.Arrays;
 
 import org.apache.lucene.document.ShapeField.QueryRelation;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.GeoEncodingUtils;
 import org.apache.lucene.geo.Line;
 import org.apache.lucene.geo.Line2D;
@@ -45,7 +46,7 @@ import org.apache.lucene.util.NumericUtils;
  **/
 final class LatLonShapeLineQuery extends ShapeQuery {
   final Line[] lines;
-  final private Line2D line2D;
+  final private Component2D line2D;
 
   public LatLonShapeLineQuery(String field, QueryRelation queryRelation, Line... lines) {
     super(field, queryRelation);
@@ -80,7 +81,7 @@ final class LatLonShapeLineQuery extends ShapeQuery {
     double maxLon = GeoEncodingUtils.decodeLongitude(NumericUtils.sortableBytesToInt(maxTriangle, maxXOffset));
 
     // check internal node against query
-    return line2D.relate(minLat, maxLat, minLon, maxLon);
+    return line2D.relate(minLon, maxLon, minLat, maxLat);
   }
 
   @Override
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapePolygonQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapePolygonQuery.java
index 5ba47fa..a4d996b 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapePolygonQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapePolygonQuery.java
@@ -19,6 +19,7 @@ package org.apache.lucene.document;
 import java.util.Arrays;
 
 import org.apache.lucene.document.ShapeField.QueryRelation;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.GeoEncodingUtils;
 import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.geo.Polygon2D;
@@ -40,7 +41,7 @@ import org.apache.lucene.util.NumericUtils;
  **/
 final class LatLonShapePolygonQuery extends ShapeQuery {
   final Polygon[] polygons;
-  final private Polygon2D poly2D;
+  final private Component2D poly2D;
 
   /**
    * Creates a query that matches all indexed shapes to the provided polygons
@@ -74,7 +75,7 @@ final class LatLonShapePolygonQuery extends ShapeQuery {
     double maxLon = GeoEncodingUtils.decodeLongitude(NumericUtils.sortableBytesToInt(maxTriangle, maxXOffset));
 
     // check internal node against query
-    return poly2D.relate(minLat, maxLat, minLon, maxLon);
+    return poly2D.relate(minLon, maxLon, minLat, maxLat);
   }
 
   @Override
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/XYShapeLineQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/XYShapeLineQuery.java
index 5f200da..52da2f2 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/XYShapeLineQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/XYShapeLineQuery.java
@@ -19,6 +19,7 @@ package org.apache.lucene.document;
 import java.util.Arrays;
 
 import org.apache.lucene.document.ShapeField.QueryRelation;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.Line2D;
 import org.apache.lucene.geo.XYLine;
 import org.apache.lucene.index.PointValues.Relation;
@@ -45,7 +46,7 @@ import static org.apache.lucene.geo.XYEncodingUtils.decode;
  **/
 final class XYShapeLineQuery extends ShapeQuery {
   final XYLine[] lines;
-  final private Line2D line2D;
+  final private Component2D line2D;
 
   public XYShapeLineQuery(String field, QueryRelation queryRelation, XYLine... lines) {
     super(field, queryRelation);
@@ -76,13 +77,13 @@ final class XYShapeLineQuery extends ShapeQuery {
   @Override
   protected Relation relateRangeBBoxToQuery(int minXOffset, int minYOffset, byte[] minTriangle,
                                             int maxXOffset, int maxYOffset, byte[] maxTriangle) {
-    double minLat = decode(NumericUtils.sortableBytesToInt(minTriangle, minYOffset));
-    double minLon = decode(NumericUtils.sortableBytesToInt(minTriangle, minXOffset));
-    double maxLat = decode(NumericUtils.sortableBytesToInt(maxTriangle, maxYOffset));
-    double maxLon = decode(NumericUtils.sortableBytesToInt(maxTriangle, maxXOffset));
+    double minY = decode(NumericUtils.sortableBytesToInt(minTriangle, minYOffset));
+    double minX = decode(NumericUtils.sortableBytesToInt(minTriangle, minXOffset));
+    double maxY = decode(NumericUtils.sortableBytesToInt(maxTriangle, maxYOffset));
+    double maxX = decode(NumericUtils.sortableBytesToInt(maxTriangle, maxXOffset));
 
     // check internal node against query
-    return line2D.relate(minLat, maxLat, minLon, maxLon);
+    return line2D.relate(minX, maxX, minY, maxY);
   }
 
   @Override
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/XYShapePolygonQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/XYShapePolygonQuery.java
index 49835bd..99acdb7 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/XYShapePolygonQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/XYShapePolygonQuery.java
@@ -19,7 +19,7 @@ package org.apache.lucene.document;
 import java.util.Arrays;
 
 import org.apache.lucene.document.ShapeField.QueryRelation;
-import org.apache.lucene.geo.Polygon2D;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.XYEncodingUtils;
 import org.apache.lucene.geo.XYPolygon;
 import org.apache.lucene.geo.XYPolygon2D;
@@ -38,7 +38,7 @@ import static org.apache.lucene.geo.XYEncodingUtils.decode;
  **/
 final class XYShapePolygonQuery extends ShapeQuery {
   final XYPolygon[] polygons;
-  final private Polygon2D poly2D;
+  final private Component2D poly2D;
 
   /**
    * Creates a query that matches all indexed shapes to the provided polygons
@@ -74,7 +74,7 @@ final class XYShapePolygonQuery extends ShapeQuery {
     double maxLon = XYEncodingUtils.decode(NumericUtils.sortableBytesToInt(maxTriangle, maxXOffset));
 
     // check internal node against query
-    return poly2D.relate(minLat, maxLat, minLon, maxLon);
+    return poly2D.relate(minLon, maxLon, minLat, maxLat);
   }
 
   @Override
diff --git a/lucene/sandbox/src/java/org/apache/lucene/geo/Line2D.java b/lucene/sandbox/src/java/org/apache/lucene/geo/Line2D.java
index 3c7f4d1..15c923e 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/geo/Line2D.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/geo/Line2D.java
@@ -18,8 +18,6 @@ package org.apache.lucene.geo;
 
 import org.apache.lucene.index.PointValues.Relation;
 
-import static org.apache.lucene.geo.GeoUtils.orient;
-
 /**
  * 2D geo line implementation represented as a balanced interval tree of edges.
  * <p>
@@ -27,81 +25,123 @@ import static org.apache.lucene.geo.GeoUtils.orient;
  * {@link #relate relate()} are {@code O(n)}, but for most practical lines are much faster than brute force.
  * @lucene.internal
  */
-public final class Line2D extends EdgeTree {
+public final class Line2D implements Component2D {
+
+  /** minimum latitude of this geometry's bounding box area */
+  final private double minY;
+  /** maximum latitude of this geometry's bounding box area */
+  final private double maxY;
+  /** minimum longitude of this geometry's bounding box area */
+  final private double minX;
+  /** maximum longitude of this geometry's bounding box area */
+  final private double maxX;
+  /** lines represented as a 2-d interval tree.*/
+  final private EdgeTree tree;
 
   private Line2D(Line line) {
-    super(line.minLat, line.maxLat, line.minLon, line.maxLon, line.getLats(), line.getLons());
+    this.minY = line.minLat;
+    this.maxY = line.maxLat;
+    this.minX = line.minLon;
+    this.maxX = line.maxLon;
+    this.tree = EdgeTree.createTree(line.getLons(), line.getLats());
   }
 
   private Line2D(XYLine line) {
-    super(line.minY, line.maxY, line.minX, line.maxX, line.getY(), line.getX());
+    this.minY = line.minY;
+    this.maxY = line.maxY;
+    this.minX = line.minX;
+    this.maxX = line.maxX;
+    this.tree = EdgeTree.createTree(line.getX(), line.getY());
   }
 
-  /** create a Line2D edge tree from provided array of Linestrings */
-  public static Line2D create(Line... lines) {
-    Line2D components[] = new Line2D[lines.length];
-    for (int i = 0; i < components.length; ++i) {
-      components[i] = new Line2D(lines[i]);
-    }
-    return (Line2D)createTree(components, 0, components.length - 1, false);
+  @Override
+  public double getMinX() {
+    return minX;
   }
 
-  /** create a Line2D edge tree from provided array of Linestrings */
-  public static Line2D create(XYLine... lines) {
-    Line2D components[] = new Line2D[lines.length];
-    for (int i = 0; i < components.length; ++i) {
-      components[i] = new Line2D(lines[i]);
+  @Override
+  public double getMaxX() {
+    return maxX;
+  }
+
+  @Override
+  public double getMinY() {
+    return minY;
+  }
+
+  @Override
+  public double getMaxY() {
+    return maxY;
+  }
+
+  @Override
+  public boolean contains(double x, double y) {
+    if (Component2D.containsPoint(x, y, this.minX, this.maxX, this.minY, this.maxY)) {
+      return tree.isPointOnLine(x, y);
     }
-    return (Line2D)createTree(components, 0, components.length - 1, false);
+    return false;
   }
 
   @Override
-  protected Relation componentRelate(double minLat, double maxLat, double minLon, double maxLon) {
-    if (tree.crossesBox(minLat, maxLat, minLon, maxLon, true)) {
+  public Relation relate(double minX, double maxX, double minY, double maxY) {
+    if (Component2D.disjoint(this.minX, this.maxX, this.minY, this.maxY, minX, maxX, minY, maxY)) {
+      return Relation.CELL_OUTSIDE_QUERY;
+    }
+    if (Component2D.within(this.minX, this.maxX, this.minY, this.maxY, minX, maxX, minY, maxY)) {
+      return Relation.CELL_CROSSES_QUERY;
+    }
+    if (tree.crossesBox(minX, maxX, minY, maxY, true)) {
       return Relation.CELL_CROSSES_QUERY;
     }
     return Relation.CELL_OUTSIDE_QUERY;
   }
 
   @Override
-  protected Relation componentRelateTriangle(double ax, double ay, double bx, double by, double cx, double cy) {
+  public Relation relateTriangle(double minX, double maxX, double minY, double maxY,
+                                 double ax, double ay, double bx, double by, double cx, double cy) {
+    if (Component2D.disjoint(this.minX, this.maxX, this.minY, this.maxY, minX, maxX, minY, maxY)) {
+      return Relation.CELL_OUTSIDE_QUERY;
+    }
     if (ax == bx && bx == cx && ay == by && by == cy) {
       // indexed "triangle" is a point: check if point lies on any line segment
-      if (isPointOnLine(tree, ax, ay)) {
+      if (tree.isPointOnLine(ax, ay)) {
         return Relation.CELL_INSIDE_QUERY;
       }
     } else if ((ax == cx && ay == cy) || (bx == cx && by == cy)) {
       // indexed "triangle" is a line:
-      if (tree.crossesLine(ax, ay, bx, by)) {
+      if (tree.crossesLine(minX, maxX, minY, maxY, ax, ay, bx, by)) {
+        return Relation.CELL_CROSSES_QUERY;
+      }
+      return Relation.CELL_OUTSIDE_QUERY;
+    } else if ((ax == bx && ay == by)) {
+      // indexed "triangle" is a line:
+      if (tree.crossesLine(minX, maxX, minY, maxY, ax, ay, cx, cy)) {
         return Relation.CELL_CROSSES_QUERY;
       }
       return Relation.CELL_OUTSIDE_QUERY;
-    } else if (pointInTriangle(tree.lon1, tree.lat1, ax, ay, bx, by, cx, cy) == true ||
-        tree.crossesTriangle(ax, ay, bx, by, cx, cy)) {
+    } else if (Component2D.pointInTriangle(minX, maxX, minY, maxY, tree.x1, tree.y1, ax, ay, bx, by, cx, cy) == true ||
+        tree.crossesTriangle(minX, maxX, minY, maxY, ax, ay, bx, by, cx, cy)) {
       // indexed "triangle" is a triangle:
       return Relation.CELL_CROSSES_QUERY;
     }
     return Relation.CELL_OUTSIDE_QUERY;
   }
 
-  /** returns true if the provided x, y point lies on the line */
-  private boolean isPointOnLine(Edge tree, double x, double y) {
-    if (y <= tree.max) {
-      double minY = StrictMath.min(tree.lat1, tree.lat2);
-      double maxY = StrictMath.max(tree.lat1, tree.lat2);
-      double minX = StrictMath.min(tree.lon1, tree.lon2);
-      double maxX = StrictMath.max(tree.lon1, tree.lon2);
-      if (Rectangle.containsPoint(y, x, minY, maxY, minX, maxX) &&
-          orient(tree.lon1, tree.lat1, tree.lon2, tree.lat2, x, y) == 0) {
-        return true;
-      }
-      if (tree.left != null && isPointOnLine(tree.left, x, y)) {
-        return true;
-      }
-      if (tree.right != null && maxY >= tree.low && isPointOnLine(tree.right, x, y)) {
-        return true;
-      }
+  /** create a Line2D edge tree from provided array of Linestrings */
+  public static Component2D create(Line... lines) {
+    Component2D components[] = new Component2D[lines.length];
+    for (int i = 0; i < components.length; ++i) {
+      components[i] = new Line2D(lines[i]);
     }
-    return false;
+    return ComponentTree.create(components);
+  }
+
+  /** create a Line2D edge tree from provided array of Linestrings */
+  public static Component2D create(XYLine... lines) {
+    Line2D components[] = new Line2D[lines.length];
+    for (int i = 0; i < components.length; ++i) {
+      components[i] = new Line2D(lines[i]);
+    }
+    return ComponentTree.create(components);
   }
 }
\ No newline at end of file
diff --git a/lucene/sandbox/src/java/org/apache/lucene/geo/XYPolygon2D.java b/lucene/sandbox/src/java/org/apache/lucene/geo/XYPolygon2D.java
index 1dba72b..cf5b174 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/geo/XYPolygon2D.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/geo/XYPolygon2D.java
@@ -23,22 +23,22 @@ package org.apache.lucene.geo;
  */
 public class XYPolygon2D extends Polygon2D {
 
-  protected XYPolygon2D(XYPolygon polygon, XYPolygon2D holes) {
-    super(polygon.minY, polygon.maxY, polygon.minX, polygon.maxX, polygon.getPolyY(), polygon.getPolyX(), holes);
+  protected XYPolygon2D(XYPolygon polygon, Component2D holes) {
+    super(polygon.minX, polygon.maxX, polygon.minY, polygon.maxY, polygon.getPolyX(), polygon.getPolyY(), holes);
   }
 
   /** Builds a Polygon2D from multipolygon */
-  public static XYPolygon2D create(XYPolygon... polygons) {
+  public static Component2D create(XYPolygon... polygons) {
     XYPolygon2D components[] = new XYPolygon2D[polygons.length];
     for (int i = 0; i < components.length; i++) {
       XYPolygon gon = polygons[i];
       XYPolygon gonHoles[] = gon.getHoles();
-      XYPolygon2D holes = null;
+      Component2D holes = null;
       if (gonHoles.length > 0) {
         holes = create(gonHoles);
       }
       components[i] = new XYPolygon2D(gon, holes);
     }
-    return (XYPolygon2D)createTree(components, 0, components.length - 1, false);
+    return ComponentTree.create(components);
   }
 }
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java
index 5095bdc..91b0706 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java
@@ -20,6 +20,7 @@ import java.util.Arrays;
 
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import org.apache.lucene.document.ShapeField.QueryRelation;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.GeoTestUtil;
 import org.apache.lucene.geo.Line;
 import org.apache.lucene.geo.Line2D;
@@ -66,12 +67,12 @@ public abstract class BaseLatLonShapeTestCase extends BaseShapeTestCase {
   }
 
   @Override
-  protected Line2D toLine2D(Object... lines) {
+  protected Component2D toLine2D(Object... lines) {
     return Line2D.create(Arrays.stream(lines).toArray(Line[]::new));
   }
 
   @Override
-  protected Polygon2D toPolygon2D(Object... polygons) {
+  protected Component2D toPolygon2D(Object... polygons) {
     return Polygon2D.create(Arrays.stream(polygons).toArray(Polygon[]::new));
   }
 
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/BaseShapeEncodingTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/document/BaseShapeEncodingTestCase.java
index 5d7579f..ee71047 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/BaseShapeEncodingTestCase.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/BaseShapeEncodingTestCase.java
@@ -16,8 +16,8 @@
  */
 package org.apache.lucene.document;
 
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.GeoUtils;
-import org.apache.lucene.geo.Polygon2D;
 import org.apache.lucene.index.PointValues;
 import org.apache.lucene.util.LuceneTestCase;
 
@@ -35,7 +35,7 @@ public abstract class BaseShapeEncodingTestCase extends LuceneTestCase {
   protected abstract double nextY();
 
   protected abstract Object nextPolygon();
-  protected abstract Polygon2D createPolygon2D(Object polygon);
+  protected abstract Component2D createPolygon2D(Object polygon);
 
   //One shared point with MBR -> MinY, MinX
   public void testPolygonEncodingMinLatMinLon() {
@@ -535,7 +535,7 @@ public abstract class BaseShapeEncodingTestCase extends LuceneTestCase {
     }
 
     for (int i =0; i < 100; i ++) {
-      Polygon2D polygon2D = createPolygon2D(nextPolygon());
+      Component2D polygon2D = createPolygon2D(nextPolygon());
       PointValues.Relation originalRelation = polygon2D.relateTriangle(originalQuantize[1], originalQuantize[0], originalQuantize[3], originalQuantize[2], originalQuantize[5], originalQuantize[4]);
       PointValues.Relation encodedRelation = polygon2D.relateTriangle(encodedQuantize[1], encodedQuantize[0], encodedQuantize[3], encodedQuantize[2], encodedQuantize[5], encodedQuantize[4]);
       assertTrue(originalRelation == encodedRelation);
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/BaseShapeTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/document/BaseShapeTestCase.java
index a7f53fe..615d802 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/BaseShapeTestCase.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/BaseShapeTestCase.java
@@ -23,7 +23,7 @@ import java.util.Set;
 
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import org.apache.lucene.document.ShapeField.QueryRelation;
-import org.apache.lucene.geo.Line2D;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
@@ -188,9 +188,9 @@ public abstract class BaseShapeTestCase extends LuceneTestCase {
   /** factory method to create a new polygon query */
   protected abstract Query newPolygonQuery(String field, QueryRelation queryRelation, Object... polygons);
 
-  protected abstract Line2D toLine2D(Object... line);
+  protected abstract Component2D toLine2D(Object... line);
 
-  protected abstract Object toPolygon2D(Object... polygon);
+  protected abstract Component2D toPolygon2D(Object... polygon);
 
   private void verify(Object... shapes) throws Exception {
     IndexWriterConfig iwc = newIndexWriterConfig();
@@ -372,7 +372,7 @@ public abstract class BaseShapeTestCase extends LuceneTestCase {
 
       // line
       Object queryLine = randomQueryLine(shapes);
-      Line2D queryLine2D = toLine2D(queryLine);
+      Component2D queryLine2D = toLine2D(queryLine);
       QueryRelation queryRelation = RandomPicks.randomFrom(random(), POINT_LINE_RELATIONS);
       Query query = newLineQuery(FIELD_NAME, queryRelation, queryLine);
 
@@ -413,7 +413,7 @@ public abstract class BaseShapeTestCase extends LuceneTestCase {
         } else if (shapes[id] == null) {
           expected = false;
         } else {
-          expected = VALIDATOR.setRelation(queryRelation).testLineQuery(queryLine2D, shapes[id]);
+          expected = VALIDATOR.setRelation(queryRelation).testComponentQuery(queryLine2D, shapes[id]);
         }
 
         if (hits.get(docID) != expected) {
@@ -463,7 +463,7 @@ public abstract class BaseShapeTestCase extends LuceneTestCase {
 
       // Polygon
       Object queryPolygon = randomQueryPolygon();
-      Object queryPoly2D = toPolygon2D(queryPolygon);
+      Component2D queryPoly2D = toPolygon2D(queryPolygon);
       QueryRelation queryRelation = RandomPicks.randomFrom(random(), QueryRelation.values());
       Query query = newPolygonQuery(FIELD_NAME, queryRelation, queryPolygon);
 
@@ -504,7 +504,7 @@ public abstract class BaseShapeTestCase extends LuceneTestCase {
         } else if (shapes[id] == null) {
           expected = false;
         } else {
-          expected = VALIDATOR.setRelation(queryRelation).testPolygonQuery(queryPoly2D, shapes[id]);
+          expected = VALIDATOR.setRelation(queryRelation).testComponentQuery(queryPoly2D, shapes[id]);
         }
 
         if (hits.get(docID) != expected) {
@@ -570,8 +570,7 @@ public abstract class BaseShapeTestCase extends LuceneTestCase {
 
     protected QueryRelation queryRelation = QueryRelation.INTERSECTS;
     public abstract boolean testBBoxQuery(double minLat, double maxLat, double minLon, double maxLon, Object shape);
-    public abstract boolean testLineQuery(Line2D line2d, Object shape);
-    public abstract boolean testPolygonQuery(Object poly2d, Object shape);
+    public abstract boolean testComponentQuery(Component2D line2d, Object shape);
 
     public Validator setRelation(QueryRelation relation) {
       this.queryRelation = relation;
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/BaseXYShapeTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/document/BaseXYShapeTestCase.java
index 0ef15c5..bc65426 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/BaseXYShapeTestCase.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/BaseXYShapeTestCase.java
@@ -20,6 +20,7 @@ import java.util.Arrays;
 
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import org.apache.lucene.document.ShapeField.QueryRelation;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.Line2D;
 import org.apache.lucene.geo.ShapeTestUtil;
 import org.apache.lucene.geo.XYLine;
@@ -58,12 +59,12 @@ public abstract class BaseXYShapeTestCase extends BaseShapeTestCase {
   }
 
   @Override
-  protected Line2D toLine2D(Object... lines) {
+  protected Component2D toLine2D(Object... lines) {
     return Line2D.create(Arrays.stream(lines).toArray(XYLine[]::new));
   }
 
   @Override
-  protected XYPolygon2D toPolygon2D(Object... polygons) {
+  protected Component2D toPolygon2D(Object... polygons) {
     return XYPolygon2D.create(Arrays.stream(polygons).toArray(XYPolygon[]::new));
   }
 
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonLineShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonLineShapeQueries.java
index fa31b00..2bc8098 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonLineShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonLineShapeQueries.java
@@ -19,11 +19,9 @@ package org.apache.lucene.document;
 
 import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
 import org.apache.lucene.document.ShapeField.QueryRelation;
-import org.apache.lucene.geo.EdgeTree;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.GeoTestUtil;
 import org.apache.lucene.geo.Line;
-import org.apache.lucene.geo.Line2D;
-import org.apache.lucene.geo.Polygon2D;
 import org.apache.lucene.geo.Rectangle;
 import org.apache.lucene.geo.Rectangle2D;
 import org.apache.lucene.index.PointValues.Relation;
@@ -98,20 +96,11 @@ public class TestLatLonLineShapeQueries extends BaseLatLonShapeTestCase {
     }
 
     @Override
-    public boolean testLineQuery(Line2D line2d, Object shape) {
-      return testLine(line2d, (Line) shape);
-    }
-
-    @Override
-    public boolean testPolygonQuery(Object poly2d, Object shape) {
-      return testLine((Polygon2D)poly2d, (Line) shape);
-    }
-
-    private boolean testLine(EdgeTree queryPoly, Line line) {
-
+    public boolean testComponentQuery(Component2D component2D, Object shape) {
+      Line line = (Line) shape;
       for (int i = 0, j = 1; j < line.numPoints(); ++i, ++j) {
         double[] qTriangle = encoder.quantizeTriangle(line.getLon(i), line.getLat(i), true, line.getLon(j), line.getLat(j), true, line.getLon(i), line.getLat(i), true);
-        Relation r = queryPoly.relateTriangle(qTriangle[1], qTriangle[0], qTriangle[3], qTriangle[2], qTriangle[5], qTriangle[4]);
+        Relation r = component2D.relateTriangle(qTriangle[1], qTriangle[0], qTriangle[3], qTriangle[2], qTriangle[5], qTriangle[4]);
         if (queryRelation == QueryRelation.DISJOINT) {
           if (r != Relation.CELL_OUTSIDE_QUERY) return false;
         } else if (queryRelation == QueryRelation.WITHIN) {
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiLineShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiLineShapeQueries.java
index fc5bc91..4416331 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiLineShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiLineShapeQueries.java
@@ -20,8 +20,8 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.document.ShapeField.QueryRelation;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.Line;
-import org.apache.lucene.geo.Line2D;
 
 /** random bounding box, line, and polygon query tests for random indexed arrays of {@link Line} types */
 public class TestLatLonMultiLineShapeQueries extends BaseLatLonShapeTestCase {
@@ -90,26 +90,10 @@ public class TestLatLonMultiLineShapeQueries extends BaseLatLonShapeTestCase {
     }
 
     @Override
-    public boolean testLineQuery(Line2D query, Object shape) {
+    public boolean testComponentQuery(Component2D query, Object shape) {
       Line[] lines = (Line[])shape;
       for (Line l : lines) {
-        boolean b = LINEVALIDATOR.testLineQuery(query, l);
-        if (b == true && queryRelation == QueryRelation.INTERSECTS) {
-          return true;
-        } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
-          return false;
-        } else if (b == false && queryRelation == QueryRelation.WITHIN) {
-          return false;
-        }
-      }
-      return queryRelation != QueryRelation.INTERSECTS;
-    }
-
-    @Override
-    public boolean testPolygonQuery(Object query, Object shape) {
-      Line[] lines = (Line[])shape;
-      for (Line l : lines) {
-        boolean b = LINEVALIDATOR.testPolygonQuery(query, l);
+        boolean b = LINEVALIDATOR.testComponentQuery(query, l);
         if (b == true && queryRelation == QueryRelation.INTERSECTS) {
           return true;
         } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiPointShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiPointShapeQueries.java
index 536e9c2..8214bf0 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiPointShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiPointShapeQueries.java
@@ -20,7 +20,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.document.ShapeField.QueryRelation;
-import org.apache.lucene.geo.Line2D;
+import org.apache.lucene.geo.Component2D;
 
 /** random bounding box, line, and polygon query tests for random indexed arrays of {@code latitude, longitude} points */
 public class TestLatLonMultiPointShapeQueries extends BaseLatLonShapeTestCase {
@@ -89,26 +89,10 @@ public class TestLatLonMultiPointShapeQueries extends BaseLatLonShapeTestCase {
     }
 
     @Override
-    public boolean testLineQuery(Line2D query, Object shape) {
+    public boolean testComponentQuery(Component2D query, Object shape) {
       Point[] points = (Point[]) shape;
       for (Point p : points) {
-        boolean b = POINTVALIDATOR.testLineQuery(query, p);
-        if (b == true && queryRelation == QueryRelation.INTERSECTS) {
-          return true;
-        } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
-          return false;
-        } else if (b == false && queryRelation == QueryRelation.WITHIN) {
-          return false;
-        }
-      }
-      return queryRelation != QueryRelation.INTERSECTS;
-    }
-
-    @Override
-    public boolean testPolygonQuery(Object query, Object shape) {
-      Point[] points = (Point[]) shape;
-      for (Point p : points) {
-        boolean b = POINTVALIDATOR.testPolygonQuery(query, p);
+        boolean b = POINTVALIDATOR.testComponentQuery(query, p);
         if (b == true && queryRelation == QueryRelation.INTERSECTS) {
           return true;
         } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiPolygonShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiPolygonShapeQueries.java
index bba9f97..7436df8 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiPolygonShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiPolygonShapeQueries.java
@@ -20,7 +20,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.document.ShapeField.QueryRelation;
-import org.apache.lucene.geo.Line2D;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.geo.Tessellator;
 
@@ -102,26 +102,10 @@ public class TestLatLonMultiPolygonShapeQueries extends BaseLatLonShapeTestCase
     }
 
     @Override
-    public boolean testLineQuery(Line2D query, Object shape) {
+    public boolean testComponentQuery(Component2D query, Object shape) {
       Polygon[] polygons = (Polygon[])shape;
       for (Polygon p : polygons) {
-        boolean b = POLYGONVALIDATOR.testLineQuery(query, p);
-        if (b == true && queryRelation == QueryRelation.INTERSECTS) {
-          return true;
-        } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
-          return false;
-        } else if (b == false && queryRelation == QueryRelation.WITHIN) {
-          return false;
-        }
-      }
-      return queryRelation != QueryRelation.INTERSECTS;
-    }
-
-    @Override
-    public boolean testPolygonQuery(Object query, Object shape) {
-      Polygon[] polygons = (Polygon[])shape;
-      for (Polygon p : polygons) {
-        boolean b = POLYGONVALIDATOR.testPolygonQuery(query, p);
+        boolean b = POLYGONVALIDATOR.testComponentQuery(query, p);
         if (b == true && queryRelation == QueryRelation.INTERSECTS) {
           return true;
         } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java
index b3ab59b..4a8540b 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java
@@ -18,11 +18,9 @@ package org.apache.lucene.document;
 
 import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
 import org.apache.lucene.document.ShapeField.QueryRelation;
-import org.apache.lucene.geo.EdgeTree;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.GeoTestUtil;
 import org.apache.lucene.geo.Line;
-import org.apache.lucene.geo.Line2D;
-import org.apache.lucene.geo.Polygon2D;
 import org.apache.lucene.index.PointValues.Relation;
 
 /** random bounding box, line, and polygon query tests for random generated {@code latitude, longitude} points */
@@ -91,20 +89,12 @@ public class TestLatLonPointShapeQueries extends BaseLatLonShapeTestCase {
     }
 
     @Override
-    public boolean testLineQuery(Line2D line2d, Object shape) {
-      return testPoint(line2d, (Point) shape);
-    }
-
-    @Override
-    public boolean testPolygonQuery(Object poly2d, Object shape) {
-      return testPoint((Polygon2D)poly2d, (Point) shape);
-    }
-
-    private boolean testPoint(EdgeTree tree, Point p) {
+    public boolean testComponentQuery(Component2D query, Object shape) {
+      Point p =  (Point) shape;
       double lat = encoder.quantizeY(p.lat);
       double lon = encoder.quantizeX(p.lon);
       // for consistency w/ the query we test the point as a triangle
-      Relation r = tree.relateTriangle(lon, lat, lon, lat, lon, lat);
+      Relation r = query.relateTriangle(lon, lat, lon, lat, lon, lat);
       if (queryRelation == QueryRelation.WITHIN) {
         return r == Relation.CELL_INSIDE_QUERY;
       } else if (queryRelation == QueryRelation.DISJOINT) {
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPolygonShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPolygonShapeQueries.java
index 8fdbf5c..b9e4a3e 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPolygonShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPolygonShapeQueries.java
@@ -19,10 +19,8 @@ package org.apache.lucene.document;
 import java.util.List;
 
 import org.apache.lucene.document.ShapeField.QueryRelation;
-import org.apache.lucene.geo.EdgeTree;
-import org.apache.lucene.geo.Line2D;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.Polygon;
-import org.apache.lucene.geo.Polygon2D;
 import org.apache.lucene.geo.Rectangle;
 import org.apache.lucene.geo.Rectangle2D;
 import org.apache.lucene.geo.Tessellator;
@@ -89,22 +87,14 @@ public class TestLatLonPolygonShapeQueries extends BaseLatLonShapeTestCase {
     }
 
     @Override
-    public boolean testLineQuery(Line2D query, Object shape) {
-      return testPolygon(query, (Polygon) shape);
-    }
-
-    @Override
-    public boolean testPolygonQuery(Object query, Object shape) {
-      return testPolygon((Polygon2D)query, (Polygon) shape);
-    }
-
-    private boolean testPolygon(EdgeTree tree, Polygon shape) {
+    public boolean testComponentQuery(Component2D query, Object o) {
+      Polygon shape = (Polygon) o;
       List<Tessellator.Triangle> tessellation = Tessellator.tessellate(shape);
       for (Tessellator.Triangle t : tessellation) {
         double[] qTriangle = encoder.quantizeTriangle(t.getX(0), t.getY(0), t.isEdgefromPolygon(0),
                                                       t.getX(1), t.getY(1), t.isEdgefromPolygon(1),
                                                       t.getX(2), t.getY(2), t.isEdgefromPolygon(2));
-        Relation r = tree.relateTriangle(qTriangle[1], qTriangle[0], qTriangle[3], qTriangle[2], qTriangle[5], qTriangle[4]);
+        Relation r = query.relateTriangle(qTriangle[1], qTriangle[0], qTriangle[3], qTriangle[2], qTriangle[5], qTriangle[4]);
         if (queryRelation == QueryRelation.DISJOINT) {
           if (r != Relation.CELL_OUTSIDE_QUERY) return false;
         } else if (queryRelation == QueryRelation.WITHIN) {
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShape.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShape.java
index 22ae32c..0102bf7 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShape.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShape.java
@@ -18,6 +18,7 @@ package org.apache.lucene.document;
 
 import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
 import org.apache.lucene.document.ShapeField.QueryRelation;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.GeoTestUtil;
 import org.apache.lucene.geo.Line;
 import org.apache.lucene.geo.Line2D;
@@ -342,7 +343,7 @@ public class TestLatLonShape extends LuceneTestCase {
     double blon = -52.67048754768767;
     Polygon polygon = new Polygon(new double[] {-14.448264200949083, 0, 0, -14.448264200949083, -14.448264200949083},
         new double[] {0.9999999403953552, 0.9999999403953552, 124.50086371762484, 124.50086371762484, 0.9999999403953552});
-    Polygon2D polygon2D = Polygon2D.create(polygon);
+    Component2D polygon2D = Polygon2D.create(polygon);
     PointValues.Relation rel = polygon2D.relateTriangle(
         quantizeLon(alon), quantizeLat(blat),
         quantizeLon(blon), quantizeLat(blat),
@@ -360,7 +361,7 @@ public class TestLatLonShape extends LuceneTestCase {
 
   public void testTriangleTouchingEdges() {
     Polygon p = new Polygon(new double[] {0, 0, 1, 1, 0}, new double[] {0, 1, 1, 0, 0});
-    Polygon2D polygon2D = Polygon2D.create(p);
+    Component2D polygon2D = Polygon2D.create(p);
     //3 shared points
     PointValues.Relation rel = polygon2D.relateTriangle(
         quantizeLon(0.5), quantizeLat(0),
@@ -460,7 +461,7 @@ public class TestLatLonShape extends LuceneTestCase {
 
   public void testTriangleCrossingPolygonVertices() {
     Polygon p = new Polygon(new double[] {0, 0, -5, -10, -5, 0}, new double[] {-1, 1, 5, 0, -5, -1});
-    Polygon2D polygon2D = Polygon2D.create(p);
+    Component2D polygon2D = Polygon2D.create(p);
     PointValues.Relation rel = polygon2D.relateTriangle(
         quantizeLon(-5), quantizeLat(0),
         quantizeLon(10), quantizeLat(0),
@@ -470,7 +471,7 @@ public class TestLatLonShape extends LuceneTestCase {
 
   public void testLineCrossingPolygonVertices() {
     Polygon p = new Polygon(new double[] {0, -1, 0, 1, 0}, new double[] {-1, 0, 1, 0, -1});
-    Polygon2D polygon2D = Polygon2D.create(p);
+    Component2D polygon2D = Polygon2D.create(p);
     PointValues.Relation rel = polygon2D.relateTriangle(
         quantizeLon(-1.5), quantizeLat(0),
         quantizeLon(1.5), quantizeLat(0),
@@ -480,7 +481,7 @@ public class TestLatLonShape extends LuceneTestCase {
 
   public void testLineSharedLine() {
     Line l = new Line(new double[] {0, 0, 0, 0}, new double[] {-2, -1, 0, 1});
-    Line2D l2d = Line2D.create(l);
+    Component2D l2d = Line2D.create(l);
     PointValues.Relation r = l2d.relateTriangle(
         quantizeLon(-5), quantizeLat(0),
         quantizeLon(5), quantizeLat(0),
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShapeEncoding.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShapeEncoding.java
index 22700f5..5dc37ab 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShapeEncoding.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShapeEncoding.java
@@ -16,6 +16,7 @@
  */
 package org.apache.lucene.document;
 
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.GeoEncodingUtils;
 import org.apache.lucene.geo.GeoTestUtil;
 import org.apache.lucene.geo.Polygon;
@@ -60,7 +61,7 @@ public class TestLatLonShapeEncoding extends BaseShapeEncodingTestCase {
   }
 
   @Override
-  protected Polygon2D createPolygon2D(Object polygon) {
+  protected Component2D createPolygon2D(Object polygon) {
     return Polygon2D.create((Polygon)polygon);
   }
 }
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestXYLineShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestXYLineShapeQueries.java
index 5f91175..f560291 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestXYLineShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestXYLineShapeQueries.java
@@ -18,11 +18,9 @@ package org.apache.lucene.document;
 
 import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
 import org.apache.lucene.document.ShapeField.QueryRelation;
-import org.apache.lucene.geo.EdgeTree;
-import org.apache.lucene.geo.Line2D;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.ShapeTestUtil;
 import org.apache.lucene.geo.XYLine;
-import org.apache.lucene.geo.XYPolygon2D;
 import org.apache.lucene.geo.XYRectangle;
 import org.apache.lucene.geo.XYRectangle2D;
 import org.apache.lucene.index.PointValues.Relation;
@@ -96,20 +94,11 @@ public class TestXYLineShapeQueries extends BaseXYShapeTestCase {
     }
 
     @Override
-    public boolean testLineQuery(Line2D line2d, Object shape) {
-      return testLine(line2d, (XYLine) shape);
-    }
-
-    @Override
-    public boolean testPolygonQuery(Object poly2d, Object shape) {
-      return testLine((XYPolygon2D)poly2d, (XYLine) shape);
-    }
-
-    private boolean testLine(EdgeTree queryPoly, XYLine line) {
-
+    public boolean testComponentQuery(Component2D query, Object shape) {
+      XYLine line = (XYLine) shape;
       for (int i = 0, j = 1; j < line.numPoints(); ++i, ++j) {
         double[] qTriangle = encoder.quantizeTriangle(line.getX(i), line.getY(i), true, line.getX(j), line.getY(j), true, line.getX(i), line.getY(i), true);
-        Relation r = queryPoly.relateTriangle(qTriangle[1], qTriangle[0], qTriangle[3], qTriangle[2], qTriangle[5], qTriangle[4]);
+        Relation r = query.relateTriangle(qTriangle[1], qTriangle[0], qTriangle[3], qTriangle[2], qTriangle[5], qTriangle[4]);
         if (queryRelation == QueryRelation.DISJOINT) {
           if (r != Relation.CELL_OUTSIDE_QUERY) return false;
         } else if (queryRelation == QueryRelation.WITHIN) {
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestXYMultiLineShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestXYMultiLineShapeQueries.java
index b2978e5..8139669 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestXYMultiLineShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestXYMultiLineShapeQueries.java
@@ -20,7 +20,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.document.ShapeField.QueryRelation;
-import org.apache.lucene.geo.Line2D;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.XYLine;
 
 /** random cartesian bounding box, line, and polygon query tests for random indexed arrays of cartesian {@link XYLine} types */
@@ -89,26 +89,10 @@ public class TestXYMultiLineShapeQueries extends BaseXYShapeTestCase {
     }
 
     @Override
-    public boolean testLineQuery(Line2D query, Object shape) {
+    public boolean testComponentQuery(Component2D query, Object shape) {
       XYLine[] lines = (XYLine[])shape;
       for (XYLine l : lines) {
-        boolean b = LINEVALIDATOR.testLineQuery(query, l);
-        if (b == true && queryRelation == ShapeField.QueryRelation.INTERSECTS) {
-          return true;
-        } else if (b == false && queryRelation == ShapeField.QueryRelation.DISJOINT) {
-          return false;
-        } else if (b == false && queryRelation == ShapeField.QueryRelation.WITHIN) {
-          return false;
-        }
-      }
-      return queryRelation != ShapeField.QueryRelation.INTERSECTS;
-    }
-
-    @Override
-    public boolean testPolygonQuery(Object query, Object shape) {
-      XYLine[] lines = (XYLine[])shape;
-      for (XYLine l : lines) {
-        boolean b = LINEVALIDATOR.testPolygonQuery(query, l);
+        boolean b = LINEVALIDATOR.testComponentQuery(query, l);
         if (b == true && queryRelation == ShapeField.QueryRelation.INTERSECTS) {
           return true;
         } else if (b == false && queryRelation == ShapeField.QueryRelation.DISJOINT) {
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestXYMultiPointShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestXYMultiPointShapeQueries.java
index 54de8ab..da65ffc 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestXYMultiPointShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestXYMultiPointShapeQueries.java
@@ -20,7 +20,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.document.ShapeField.QueryRelation;
-import org.apache.lucene.geo.Line2D;
+import org.apache.lucene.geo.Component2D;
 
 /** random cartesian bounding box, line, and polygon query tests for random indexed arrays of {@code x, y} points */
 public class TestXYMultiPointShapeQueries extends BaseXYShapeTestCase {
@@ -88,26 +88,10 @@ public class TestXYMultiPointShapeQueries extends BaseXYShapeTestCase {
     }
 
     @Override
-    public boolean testLineQuery(Line2D query, Object shape) {
+    public boolean testComponentQuery(Component2D query, Object shape) {
       Point[] points = (Point[]) shape;
       for (Point p : points) {
-        boolean b = POINTVALIDATOR.testLineQuery(query, p);
-        if (b == true && queryRelation == QueryRelation.INTERSECTS) {
-          return true;
-        } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
-          return false;
-        } else if (b == false && queryRelation == QueryRelation.WITHIN) {
-          return false;
-        }
-      }
-      return queryRelation != QueryRelation.INTERSECTS;
-    }
-
-    @Override
-    public boolean testPolygonQuery(Object query, Object shape) {
-      Point[] points = (Point[]) shape;
-      for (Point p : points) {
-        boolean b = POINTVALIDATOR.testPolygonQuery(query, p);
+        boolean b = POINTVALIDATOR.testComponentQuery(query, p);
         if (b == true && queryRelation == QueryRelation.INTERSECTS) {
           return true;
         } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestXYMultiPolygonShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestXYMultiPolygonShapeQueries.java
index 5519196..ed62364 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestXYMultiPolygonShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestXYMultiPolygonShapeQueries.java
@@ -20,7 +20,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.document.ShapeField.QueryRelation;
-import org.apache.lucene.geo.Line2D;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.Tessellator;
 import org.apache.lucene.geo.XYPolygon;
 
@@ -101,26 +101,10 @@ public class TestXYMultiPolygonShapeQueries extends BaseXYShapeTestCase {
     }
 
     @Override
-    public boolean testLineQuery(Line2D query, Object shape) {
+    public boolean testComponentQuery(Component2D query, Object shape) {
       XYPolygon[] polygons = (XYPolygon[])shape;
       for (XYPolygon p : polygons) {
-        boolean b = POLYGONVALIDATOR.testLineQuery(query, p);
-        if (b == true && queryRelation == QueryRelation.INTERSECTS) {
-          return true;
-        } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
-          return false;
-        } else if (b == false && queryRelation == QueryRelation.WITHIN) {
-          return false;
-        }
-      }
-      return queryRelation != QueryRelation.INTERSECTS;
-    }
-
-    @Override
-    public boolean testPolygonQuery(Object query, Object shape) {
-      XYPolygon[] polygons = (XYPolygon[])shape;
-      for (XYPolygon p : polygons) {
-        boolean b = POLYGONVALIDATOR.testPolygonQuery(query, p);
+        boolean b = POLYGONVALIDATOR.testComponentQuery(query, p);
         if (b == true && queryRelation == QueryRelation.INTERSECTS) {
           return true;
         } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestXYPointShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestXYPointShapeQueries.java
index edeefa5..5a401c4 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestXYPointShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestXYPointShapeQueries.java
@@ -18,11 +18,9 @@ package org.apache.lucene.document;
 
 import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
 import org.apache.lucene.document.ShapeField.QueryRelation;
-import org.apache.lucene.geo.EdgeTree;
-import org.apache.lucene.geo.Line2D;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.ShapeTestUtil;
 import org.apache.lucene.geo.XYLine;
-import org.apache.lucene.geo.XYPolygon2D;
 import org.apache.lucene.index.PointValues.Relation;
 
 /** random cartesian bounding box, line, and polygon query tests for random generated {@code x, y} points */
@@ -91,20 +89,12 @@ public class TestXYPointShapeQueries extends BaseXYShapeTestCase {
     }
 
     @Override
-    public boolean testLineQuery(Line2D line2d, Object shape) {
-      return testPoint(line2d, (Point) shape);
-    }
-
-    @Override
-    public boolean testPolygonQuery(Object poly2d, Object shape) {
-      return testPoint((XYPolygon2D)poly2d, (Point) shape);
-    }
-
-    private boolean testPoint(EdgeTree tree, Point p) {
+    public boolean testComponentQuery(Component2D query, Object shape) {
+      Point p = (Point) shape;
       double lat = encoder.quantizeY(p.y);
       double lon = encoder.quantizeX(p.x);
       // for consistency w/ the query we test the point as a triangle
-      Relation r = tree.relateTriangle(lon, lat, lon, lat, lon, lat);
+      Relation r = query.relateTriangle(lon, lat, lon, lat, lon, lat);
       if (queryRelation == QueryRelation.WITHIN) {
         return r == Relation.CELL_INSIDE_QUERY;
       } else if (queryRelation == QueryRelation.DISJOINT) {
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestXYPolygonShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestXYPolygonShapeQueries.java
index bbcc554..a261d9e 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestXYPolygonShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestXYPolygonShapeQueries.java
@@ -19,11 +19,9 @@ package org.apache.lucene.document;
 import java.util.List;
 
 import org.apache.lucene.document.ShapeField.QueryRelation;
-import org.apache.lucene.geo.EdgeTree;
-import org.apache.lucene.geo.Line2D;
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.Tessellator;
 import org.apache.lucene.geo.XYPolygon;
-import org.apache.lucene.geo.XYPolygon2D;
 import org.apache.lucene.geo.XYRectangle;
 import org.apache.lucene.geo.XYRectangle2D;
 import org.apache.lucene.index.PointValues.Relation;
@@ -89,22 +87,14 @@ public class TestXYPolygonShapeQueries extends BaseXYShapeTestCase {
     }
 
     @Override
-    public boolean testLineQuery(Line2D query, Object shape) {
-      return testPolygon(query, (XYPolygon) shape);
-    }
-
-    @Override
-    public boolean testPolygonQuery(Object query, Object shape) {
-      return testPolygon((XYPolygon2D)query, (XYPolygon) shape);
-    }
-
-    private boolean testPolygon(EdgeTree tree, XYPolygon shape) {
+    public boolean testComponentQuery(Component2D query, Object o) {
+      XYPolygon shape = (XYPolygon) o;
       List<Tessellator.Triangle> tessellation = Tessellator.tessellate(shape);
       for (Tessellator.Triangle t : tessellation) {
         double[] qTriangle = encoder.quantizeTriangle(t.getX(0), t.getY(0), t.isEdgefromPolygon(0),
                                                       t.getX(1), t.getY(1), t.isEdgefromPolygon(1),
                                                       t.getX(2), t.getY(2), t.isEdgefromPolygon(2));
-        Relation r = tree.relateTriangle(qTriangle[1], qTriangle[0], qTriangle[3], qTriangle[2], qTriangle[5], qTriangle[4]);
+        Relation r = query.relateTriangle(qTriangle[1], qTriangle[0], qTriangle[3], qTriangle[2], qTriangle[5], qTriangle[4]);
         if (queryRelation == QueryRelation.DISJOINT) {
           if (r != Relation.CELL_OUTSIDE_QUERY) return false;
         } else if (queryRelation == QueryRelation.WITHIN) {
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestXYShapeEncoding.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestXYShapeEncoding.java
index 62d5392..01d0b66 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestXYShapeEncoding.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestXYShapeEncoding.java
@@ -16,6 +16,7 @@
  */
 package org.apache.lucene.document;
 
+import org.apache.lucene.geo.Component2D;
 import org.apache.lucene.geo.ShapeTestUtil;
 import org.apache.lucene.geo.XYEncodingUtils;
 import org.apache.lucene.geo.XYPolygon;
@@ -59,7 +60,7 @@ public class TestXYShapeEncoding extends BaseShapeEncodingTestCase {
   }
 
   @Override
-  protected XYPolygon2D createPolygon2D(Object polygon) {
+  protected Component2D createPolygon2D(Object polygon) {
     return XYPolygon2D.create((XYPolygon)polygon);
   }
 }
diff --git a/lucene/sandbox/src/test/org/apache/lucene/geo/TestLine2D.java b/lucene/sandbox/src/test/org/apache/lucene/geo/TestLine2D.java
index 3596b6a..6a701b1 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/geo/TestLine2D.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/geo/TestLine2D.java
@@ -25,43 +25,43 @@ public class TestLine2D extends LuceneTestCase {
 
   public void testTriangleDisjoint() {
     Line line = new Line(new double[] {0, 1, 2, 3}, new double[] {0, 0, 2, 2});
-    Line2D line2D = Line2D.create(line);
+    Component2D line2D = Line2D.create(line);
     int ax = GeoEncodingUtils.encodeLongitude(4);
     int ay = GeoEncodingUtils.encodeLatitude(4);
     int bx = GeoEncodingUtils.encodeLongitude(5);
     int by = GeoEncodingUtils.encodeLatitude(5);
     int cx = GeoEncodingUtils.encodeLongitude(5);
     int cy = GeoEncodingUtils.encodeLatitude(4);
-    assertEquals(Relation.CELL_OUTSIDE_QUERY, line2D.componentRelateTriangle(ax, ay, bx, by , cx, cy));;
+    assertEquals(Relation.CELL_OUTSIDE_QUERY, line2D.relateTriangle(ax, ay, bx, by , cx, cy));;
   }
 
   public void testTriangleIntersects() {
     Line line = new Line(new double[] {0.5, 0, 1, 2, 3}, new double[] {0.5, 0, 0, 2, 2});
-    Line2D line2D = Line2D.create(line);
+    Component2D line2D = Line2D.create(line);
     int ax = GeoEncodingUtils.encodeLongitude(0.0);
     int ay = GeoEncodingUtils.encodeLatitude(0.0);
     int bx = GeoEncodingUtils.encodeLongitude(1);
     int by = GeoEncodingUtils.encodeLatitude(0);
     int cx = GeoEncodingUtils.encodeLongitude(0);
     int cy = GeoEncodingUtils.encodeLatitude(1);
-    assertEquals(Relation.CELL_CROSSES_QUERY, line2D.componentRelateTriangle(ax, ay, bx, by , cx, cy));
+    assertEquals(Relation.CELL_CROSSES_QUERY, line2D.relateTriangle(ax, ay, bx, by , cx, cy));
   }
 
   public void testTriangleContains() {
     Line line = new Line(new double[] {0.5, 0, 1, 2, 3}, new double[] {0.5, 0, 0, 2, 2});
-    Line2D line2D = Line2D.create(line);
+    Component2D line2D = Line2D.create(line);
     int ax = GeoEncodingUtils.encodeLongitude(-10);
     int ay = GeoEncodingUtils.encodeLatitude(-10);
     int bx = GeoEncodingUtils.encodeLongitude(4);
     int by = GeoEncodingUtils.encodeLatitude(-10);
     int cx = GeoEncodingUtils.encodeLongitude(4);
     int cy = GeoEncodingUtils.encodeLatitude(30);
-    assertEquals(Relation.CELL_CROSSES_QUERY, line2D.componentRelateTriangle(ax, ay, bx, by , cx, cy));
+    assertEquals(Relation.CELL_CROSSES_QUERY, line2D.relateTriangle(ax, ay, bx, by , cx, cy));
   }
 
   public void testRandomTriangles() {
     Line line = TestLatLonLineShapeQueries.getNextLine();
-    Line2D line2D = Line2D.create(line);
+    Component2D line2D = Line2D.create(line);
 
     for (int i =0; i < 100; i++) {
       double ax = GeoTestUtil.nextLongitude();
@@ -76,9 +76,9 @@ public class TestLine2D extends LuceneTestCase {
       double tMinY = StrictMath.min(StrictMath.min(ay, by), cy);
       double tMaxY = StrictMath.max(StrictMath.max(ay, by), cy);
 
-      Relation r = line2D.relate(tMinY, tMaxY, tMinX, tMaxX);
+      Relation r = line2D.relate(tMinX, tMaxX, tMinY, tMaxY);
       if (r == Relation.CELL_OUTSIDE_QUERY) {
-        assertEquals(Relation.CELL_OUTSIDE_QUERY, line2D.componentRelateTriangle(ax, ay, bx, by, cx, cy));
+        assertEquals(Relation.CELL_OUTSIDE_QUERY, line2D.relateTriangle(ax, ay, bx, by, cx, cy));
       }
     }
   }