Allow GiST distance function to return merely a lower-bound.
authorHeikki Linnakangas <heikki.linnakangas@iki.fi>
Fri, 15 May 2015 11:26:51 +0000 (14:26 +0300)
committerHeikki Linnakangas <heikki.linnakangas@iki.fi>
Fri, 15 May 2015 11:26:51 +0000 (14:26 +0300)
The distance function can now set *recheck = false, like index quals. The
executor will then re-check the ORDER BY expressions, and use a queue to
reorder the results on the fly.

This makes it possible to do kNN-searches on polygons and circles, which
don't store the exact value in the index, but just a bounding box.

Alexander Korotkov and me

19 files changed:
doc/src/sgml/gist.sgml
src/backend/access/gist/gistget.c
src/backend/access/gist/gistproc.c
src/backend/access/gist/gistscan.c
src/backend/executor/nodeIndexscan.c
src/backend/optimizer/plan/createplan.c
src/backend/utils/adt/geo_ops.c
src/include/access/genam.h
src/include/access/relscan.h
src/include/catalog/catversion.h
src/include/catalog/pg_amop.h
src/include/catalog/pg_amproc.h
src/include/catalog/pg_operator.h
src/include/catalog/pg_proc.h
src/include/nodes/execnodes.h
src/include/nodes/plannodes.h
src/include/utils/geo_decls.h
src/test/regress/expected/create_index.out
src/test/regress/sql/create_index.sql

index e7d1ff9d83f4f9b469480dc6dfc22303a2d51c4a..1291f8dd0c8cdb89ea8105f990922b4875ae1acf 100644 (file)
        <literal>~=</>
       </entry>
       <entry>
+       <literal>&lt;-&gt;</>
       </entry>
      </row>
      <row>
        <literal>~=</>
       </entry>
       <entry>
+       <literal>&lt;-&gt;</>
       </entry>
      </row>
      <row>
    </tgroup>
   </table>
 
+ <para>
+  Currently, ordering by the distance operator <literal>&lt;-&gt;</>
+  is supported only with <literal>point</> by the operator classes
+  of the geometric types.
+ </para>
+
  <para>
   For historical reasons, the <literal>inet_ops</> operator class is
   not the default class for types <type>inet</> and <type>cidr</>.
@@ -780,6 +788,7 @@ my_distance(PG_FUNCTION_ARGS)
     data_type  *query = PG_GETARG_DATA_TYPE_P(1);
     StrategyNumber strategy = (StrategyNumber) PG_GETARG_UINT16(2);
     /* Oid subtype = PG_GETARG_OID(3); */
+    /* bool *recheck = (bool *) PG_GETARG_POINTER(4); */
     data_type  *key = DatumGetDataType(entry-&gt;key);
     double      retval;
 
@@ -792,14 +801,24 @@ my_distance(PG_FUNCTION_ARGS)
 </programlisting>
 
        The arguments to the <function>distance</> function are identical to
-       the arguments of the <function>consistent</> function, except that no
-       recheck flag is used.  The distance to a leaf index entry must always
-       be determined exactly, since there is no way to re-order the tuples
-       once they are returned.  Some approximation is allowed when determining
-       the distance to an internal tree node, so long as the result is never
-       greater than any child's actual distance.  Thus, for example, distance
-       to a bounding box is usually sufficient in geometric applications.  The
-       result value can be any finite <type>float8</> value.  (Infinity and
+       the arguments of the <function>consistent</> function.
+      </para>
+
+      <para>
+       Some approximation is allowed when determining the distance, as long as
+       the result is never greater than the entry's actual distance. Thus, for
+       example, distance to a bounding box is usually sufficient in geometric
+       applications.  For an internal tree node, the distance returned must not
+       be greater than the distance to any of the child nodes. If the returned
+       distance is not accurate, the function must set *recheck to false. (This
+       is not necessary for internal tree nodes; for them, the calculation is
+       always assumed to be inaccurate). The executor will calculate the
+       accurate distance after fetching the tuple from the heap, and reorder
+       the tuples if necessary.
+      </para>
+
+      <para>
+       The result value can be any finite <type>float8</> value.  (Infinity and
        minus infinity are used internally to handle cases such as nulls, so it
        is not recommended that <function>distance</> functions return these
        values.)
index e4c00c2c9f5adf8b7e4e65024443b51766afccde..1f791a4f8eeff47591dc0cbf480d97ae45bd6f5f 100644 (file)
  * The index tuple might represent either a heap tuple or a lower index page,
  * depending on whether the containing page is a leaf page or not.
  *
- * On success return for a heap tuple, *recheck_p is set to indicate
- * whether recheck is needed.  We recheck if any of the consistent() functions
- * request it.  recheck is not interesting when examining a non-leaf entry,
- * since we must visit the lower index page if there's any doubt.
+ * On success return for a heap tuple, *recheck_p is set to indicate whether
+ * recheck is needed.  We recheck if any of the consistent() or distance()
+ * functions request it.  recheck is not interesting when examining a non-leaf
+ * entry, since we must visit the lower index page if there's any doubt.
  *
  * If we are doing an ordered scan, so->distances[] is filled with distance
  * data from the distance() functions before returning success.
@@ -176,6 +176,7 @@ gistindex_keytest(IndexScanDesc scan,
        else
        {
            Datum       dist;
+           bool        recheck;
            GISTENTRY   de;
 
            gistdentryinit(giststate, key->sk_attno - 1, &de,
@@ -192,16 +193,21 @@ gistindex_keytest(IndexScanDesc scan,
             * always be zero, but might as well pass it for possible future
             * use.)
             *
-            * Note that Distance functions don't get a recheck argument. We
-            * can't tolerate lossy distance calculations on leaf tuples;
-            * there is no opportunity to re-sort the tuples afterwards.
+            * Distance functions get a recheck argument as well.  In this
+            * case the returned distance is the lower bound of distance
+            * and needs to be rechecked.  We return single recheck flag
+            * which means that both quals and distances are to be
+            * rechecked.
             */
-           dist = FunctionCall4Coll(&key->sk_func,
+           dist = FunctionCall5Coll(&key->sk_func,
                                     key->sk_collation,
                                     PointerGetDatum(&de),
                                     key->sk_argument,
                                     Int32GetDatum(key->sk_strategy),
-                                    ObjectIdGetDatum(key->sk_subtype));
+                                    ObjectIdGetDatum(key->sk_subtype),
+                                    PointerGetDatum(&recheck));
+
+           *recheck_p |= recheck;
 
            *distance_p = DatumGetFloat8(dist);
        }
@@ -434,6 +440,7 @@ getNextNearest(IndexScanDesc scan)
 {
    GISTScanOpaque so = (GISTScanOpaque) scan->opaque;
    bool        res = false;
+   int         i;
 
    if (scan->xs_itup)
    {
@@ -454,6 +461,11 @@ getNextNearest(IndexScanDesc scan)
            /* found a heap item at currently minimal distance */
            scan->xs_ctup.t_self = item->data.heap.heapPtr;
            scan->xs_recheck = item->data.heap.recheck;
+           for (i = 0; i < scan->numberOfOrderBys; i++)
+           {
+               scan->xs_orderbyvals[i] = Float8GetDatum(item->distances[i]);
+               scan->xs_orderbynulls[i] = false;
+           }
 
            /* in an index-only scan, also return the reconstructed tuple. */
            if (scan->xs_want_itup)
index 9d21e3fb947eea13ff7a23b4a3f535679a849d05..9667e397ce470764f36d5353f2428a0bdc173ab2 100644 (file)
@@ -1478,3 +1478,40 @@ gist_point_distance(PG_FUNCTION_ARGS)
 
    PG_RETURN_FLOAT8(distance);
 }
+
+/*
+ * The inexact GiST distance method for geometric types that store bounding
+ * boxes.
+ *
+ * Compute lossy distance from point to index entries.  The result is inexact
+ * because index entries are bounding boxes, not the exact shapes of the
+ * indexed geometric types.  We use distance from point to MBR of index entry.
+ * This is a lower bound estimate of distance from point to indexed geometric
+ * type.
+ */
+Datum
+gist_bbox_distance(PG_FUNCTION_ARGS)
+{
+   GISTENTRY  *entry = (GISTENTRY *) PG_GETARG_POINTER(0);
+   StrategyNumber strategy = (StrategyNumber) PG_GETARG_UINT16(2);
+   bool       *recheck = (bool *) PG_GETARG_POINTER(4);
+   double      distance;
+   StrategyNumber strategyGroup = strategy / GeoStrategyNumberOffset;
+
+   /* Bounding box distance is always inexact. */
+   *recheck = true;
+
+   switch (strategyGroup)
+   {
+       case PointStrategyNumberGroup:
+           distance = computeDistance(false,
+                                      DatumGetBoxP(entry->key),
+                                      PG_GETARG_POINT_P(1));
+           break;
+       default:
+           elog(ERROR, "unknown strategy number: %d", strategy);
+           distance = 0.0;     /* keep compiler quiet */
+   }
+
+   PG_RETURN_FLOAT8(distance);
+}
index 6f6539823026052b2690469c373ee564898fa2a8..099849a606b1e32aeea4f16393eb5868186ead99 100644 (file)
@@ -85,6 +85,11 @@ gistbeginscan(PG_FUNCTION_ARGS)
    /* workspaces with size dependent on numberOfOrderBys: */
    so->distances = palloc(sizeof(double) * scan->numberOfOrderBys);
    so->qual_ok = true;         /* in case there are zero keys */
+   if (scan->numberOfOrderBys > 0)
+   {
+       scan->xs_orderbyvals = palloc(sizeof(Datum) * scan->numberOfOrderBys);
+       scan->xs_orderbynulls = palloc(sizeof(bool) * scan->numberOfOrderBys);
+   }
 
    scan->opaque = so;
 
index 48fa91981ff2a10820e6a764035d20ebb4d857bd..2164da0c84788ff1455404f452ed874328cea2b7 100644 (file)
@@ -16,6 +16,7 @@
  * INTERFACE ROUTINES
  *     ExecIndexScan           scans a relation using an index
  *     IndexNext               retrieve next tuple using index
+ *     IndexNextWithReorder    same, but recheck ORDER BY expressions
  *     ExecInitIndexScan       creates and initializes state info.
  *     ExecReScanIndexScan     rescans the indexed relation.
  *     ExecEndIndexScan        releases all storage.
 #include "access/relscan.h"
 #include "executor/execdebug.h"
 #include "executor/nodeIndexscan.h"
+#include "lib/pairingheap.h"
 #include "optimizer/clauses.h"
 #include "utils/array.h"
+#include "utils/datum.h"
 #include "utils/lsyscache.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
 
+/*
+ * When an ordering operator is used, tuples fetched from the index that
+ * need to be reordered are queued in a pairing heap, as ReorderTuples.
+ */
+typedef struct
+{
+   pairingheap_node ph_node;
+   HeapTuple   htup;
+   Datum      *orderbyvals;
+   bool       *orderbynulls;
+} ReorderTuple;
 
 static TupleTableSlot *IndexNext(IndexScanState *node);
+static TupleTableSlot *IndexNextWithReorder(IndexScanState *node);
+static void EvalOrderByExpressions(IndexScanState *node, ExprContext *econtext);
+static bool IndexRecheck(IndexScanState *node, TupleTableSlot *slot);
+static int cmp_orderbyvals(const Datum *adist, const bool *anulls,
+               const Datum *bdist, const bool *bnulls,
+               IndexScanState *node);
+static int reorderqueue_cmp(const pairingheap_node *a,
+                const pairingheap_node *b, void *arg);
+static void reorderqueue_push(IndexScanState *node, HeapTuple tuple,
+                 Datum *orderbyvals, bool *orderbynulls);
+static HeapTuple reorderqueue_pop(IndexScanState *node);
 
 
 /* ----------------------------------------------------------------
@@ -106,6 +131,169 @@ IndexNext(IndexScanState *node)
        return slot;
    }
 
+   /*
+    * if we get here it means the index scan failed so we are at the end of
+    * the scan..
+    */
+   node->iss_ReachedEnd = true;
+   return ExecClearTuple(slot);
+}
+
+/* ----------------------------------------------------------------
+ *     IndexNextWithReorder
+ *
+ *     Like IndexNext, but his version can also re-check any
+ *     ORDER BY expressions, and reorder the tuples as necessary.
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+IndexNextWithReorder(IndexScanState *node)
+{
+   ExprContext *econtext;
+   IndexScanDesc scandesc;
+   HeapTuple   tuple;
+   TupleTableSlot *slot;
+   ReorderTuple *topmost = NULL;
+   bool        was_exact;
+   Datum      *lastfetched_vals;
+   bool       *lastfetched_nulls;
+   int         cmp;
+
+   /* only forward scan is supported with reordering. */
+   Assert(!ScanDirectionIsBackward(((IndexScan *) node->ss.ps.plan)->indexorderdir));
+   Assert(ScanDirectionIsForward(node->ss.ps.state->es_direction));
+   scandesc = node->iss_ScanDesc;
+   econtext = node->ss.ps.ps_ExprContext;
+   slot = node->ss.ss_ScanTupleSlot;
+
+   for (;;)
+   {
+       /*
+        * Check the reorder queue first.  If the topmost tuple in the queue
+        * has an ORDER BY value smaller than (or equal to) the value last
+        * returned by the index, we can return it now.
+        */
+       if (!pairingheap_is_empty(node->iss_ReorderQueue))
+       {
+           topmost = (ReorderTuple *) pairingheap_first(node->iss_ReorderQueue);
+
+           if (node->iss_ReachedEnd ||
+               cmp_orderbyvals(topmost->orderbyvals,
+                               topmost->orderbynulls,
+                               scandesc->xs_orderbyvals,
+                               scandesc->xs_orderbynulls,
+                               node) <= 0)
+           {
+               tuple = reorderqueue_pop(node);
+
+               /* Pass 'true', as the tuple in the queue is a palloc'd copy */
+               ExecStoreTuple(tuple, slot, InvalidBuffer, true);
+               return slot;
+           }
+       }
+       else if (node->iss_ReachedEnd)
+       {
+           /* Queue is empty, and no more tuples from index.  We're done. */
+           return ExecClearTuple(slot);
+       }
+
+       /*
+        * Fetch next tuple from the index.
+        */
+next_indextuple:
+       tuple = index_getnext(scandesc, ForwardScanDirection);
+       if (!tuple)
+       {
+           /*
+            * No more tuples from the index.  But we still need to drain any
+            * remaining tuples from the queue before we're done.
+            */
+           node->iss_ReachedEnd = true;
+           continue;
+       }
+
+       /*
+        * Store the scanned tuple in the scan tuple slot of the scan state.
+        * Note: we pass 'false' because tuples returned by amgetnext are
+        * pointers onto disk pages and must not be pfree()'d.
+        */
+       ExecStoreTuple(tuple,   /* tuple to store */
+                      slot,    /* slot to store in */
+                      scandesc->xs_cbuf,       /* buffer containing tuple */
+                      false);  /* don't pfree */
+
+       /*
+        * If the index was lossy, we have to recheck the index quals and
+        * ORDER BY expressions using the fetched tuple.
+        */
+       if (scandesc->xs_recheck)
+       {
+           econtext->ecxt_scantuple = slot;
+           ResetExprContext(econtext);
+           if (!ExecQual(node->indexqualorig, econtext, false))
+           {
+               /* Fails recheck, so drop it and loop back for another */
+               InstrCountFiltered2(node, 1);
+               goto next_indextuple;
+           }
+
+           EvalOrderByExpressions(node, econtext);
+
+           /*
+            * Was the ORDER BY value returned by the index accurate?  The
+            * recheck flag means that the index can return inaccurate values,
+            * but then again, the value returned for any particular tuple
+            * could also be exactly correct.  Compare the value returned by
+            * the index with the recalculated value.  (If the value returned
+            * by the index happened to be exact right, we can often avoid
+            * pushing the tuple to the queue, just to pop it back out again.)
+            */
+           cmp = cmp_orderbyvals(node->iss_OrderByValues,
+                                 node->iss_OrderByNulls,
+                                 scandesc->xs_orderbyvals,
+                                 scandesc->xs_orderbynulls,
+                                 node);
+           if (cmp < 0)
+               elog(ERROR, "index returned tuples in wrong order");
+           else if (cmp == 0)
+               was_exact = true;
+           else
+               was_exact = false;
+           lastfetched_vals = node->iss_OrderByValues;
+           lastfetched_nulls = node->iss_OrderByNulls;
+       }
+       else
+       {
+           was_exact = true;
+           lastfetched_vals = scandesc->xs_orderbyvals;
+           lastfetched_nulls = scandesc->xs_orderbynulls;
+       }
+
+       /*
+        * Can we return this tuple immediately, or does it need to be pushed
+        * to the reorder queue?  If the ORDER BY expression values returned
+        * by the index were inaccurate, we can't return it yet, because the
+        * next tuple from the index might need to come before this one.
+        * Also, we can't return it yet if there are any smaller tuples in the
+        * queue already.
+        */
+       if (!was_exact || (topmost && cmp_orderbyvals(lastfetched_vals,
+                                                     lastfetched_nulls,
+                                                     topmost->orderbyvals,
+                                                     topmost->orderbynulls,
+                                                     node) > 0))
+       {
+           /* Put this tuple to the queue */
+           reorderqueue_push(node, tuple, lastfetched_vals, lastfetched_nulls);
+           continue;
+       }
+       else
+       {
+           /* Can return this tuple immediately. */
+           return slot;
+       }
+   }
+
    /*
     * if we get here it means the index scan failed so we are at the end of
     * the scan..
@@ -113,6 +301,33 @@ IndexNext(IndexScanState *node)
    return ExecClearTuple(slot);
 }
 
+/*
+ * Calculate the expressions in the ORDER BY clause, based on the heap tuple.
+ */
+static void
+EvalOrderByExpressions(IndexScanState *node, ExprContext *econtext)
+{
+   int         i;
+   ListCell   *l;
+   MemoryContext oldContext;
+
+   oldContext = MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory);
+
+   i = 0;
+   foreach(l, node->indexorderbyorig)
+   {
+       ExprState  *orderby = (ExprState *) lfirst(l);
+
+       node->iss_OrderByValues[i] = ExecEvalExpr(orderby,
+                                                 econtext,
+                                                 &node->iss_OrderByNulls[i],
+                                                 NULL);
+       i++;
+   }
+
+   MemoryContextSwitchTo(oldContext);
+}
+
 /*
  * IndexRecheck -- access method routine to recheck a tuple in EvalPlanQual
  */
@@ -134,6 +349,109 @@ IndexRecheck(IndexScanState *node, TupleTableSlot *slot)
    return ExecQual(node->indexqualorig, econtext, false);
 }
 
+
+/*
+ * Compare ORDER BY expression values.
+ */
+static int
+cmp_orderbyvals(const Datum *adist, const bool *anulls,
+               const Datum *bdist, const bool *bnulls,
+               IndexScanState *node)
+{
+   int         i;
+   int         result;
+
+   for (i = 0; i < node->iss_NumOrderByKeys; i++)
+   {
+       SortSupport ssup = &node->iss_SortSupport[i];
+
+       /* Handle nulls.  We only support NULLS LAST. */
+       if (anulls[i] && !bnulls[i])
+           return 1;
+       else if (!anulls[i] && bnulls[i])
+           return -1;
+       else if (anulls[i] && bnulls[i])
+           return 0;
+
+       result = ssup->comparator(adist[i], bdist[i], ssup);
+       if (result != 0)
+           return result;
+   }
+
+   return 0;
+}
+
+/*
+ * Pairing heap provides getting topmost (greatest) element while KNN provides
+ * ascending sort.  That's why we inverse the sort order.
+ */
+static int
+reorderqueue_cmp(const pairingheap_node *a, const pairingheap_node *b,
+                void *arg)
+{
+   ReorderTuple *rta = (ReorderTuple *) a;
+   ReorderTuple *rtb = (ReorderTuple *) b;
+   IndexScanState *node = (IndexScanState *) arg;
+
+   return -cmp_orderbyvals(rta->orderbyvals, rta->orderbynulls,
+                           rtb->orderbyvals, rtb->orderbynulls,
+                           node);
+}
+
+/*
+ * Helper function to push a tuple to the reorder queue.
+ */
+static void
+reorderqueue_push(IndexScanState *node, HeapTuple tuple,
+                 Datum *orderbyvals, bool *orderbynulls)
+{
+   IndexScanDesc scandesc = node->iss_ScanDesc;
+   EState     *estate = node->ss.ps.state;
+   MemoryContext oldContext = MemoryContextSwitchTo(estate->es_query_cxt);
+   ReorderTuple *rt;
+   int         i;
+
+   rt = (ReorderTuple *) palloc(sizeof(ReorderTuple));
+   rt->htup = heap_copytuple(tuple);
+   rt->orderbyvals =
+       (Datum *) palloc(sizeof(Datum) * scandesc->numberOfOrderBys);
+   rt->orderbynulls =
+       (bool *) palloc(sizeof(bool) * scandesc->numberOfOrderBys);
+   for (i = 0; i < node->iss_NumOrderByKeys; i++)
+   {
+       if (!orderbynulls[i])
+           rt->orderbyvals[i] = datumCopy(orderbyvals[i],
+                                          node->iss_OrderByTypByVals[i],
+                                          node->iss_OrderByTypLens[i]);
+       else
+           rt->orderbyvals[i] = (Datum) 0;
+       rt->orderbynulls[i] = orderbynulls[i];
+   }
+   pairingheap_add(node->iss_ReorderQueue, &rt->ph_node);
+
+   MemoryContextSwitchTo(oldContext);
+}
+
+/*
+ * Helper function to pop the next tuple from the reorder queue.
+ */
+static HeapTuple
+reorderqueue_pop(IndexScanState *node)
+{
+   HeapTuple   result;
+   ReorderTuple *topmost;
+
+   topmost = (ReorderTuple *) pairingheap_remove_first(node->iss_ReorderQueue);
+
+   result = topmost->htup;
+   pfree(topmost->orderbyvals);
+   pfree(topmost->orderbynulls);
+   pfree(topmost);
+
+   return result;
+}
+
+
 /* ----------------------------------------------------------------
  *     ExecIndexScan(node)
  * ----------------------------------------------------------------
@@ -147,9 +465,14 @@ ExecIndexScan(IndexScanState *node)
    if (node->iss_NumRuntimeKeys != 0 && !node->iss_RuntimeKeysReady)
        ExecReScan((PlanState *) node);
 
-   return ExecScan(&node->ss,
-                   (ExecScanAccessMtd) IndexNext,
-                   (ExecScanRecheckMtd) IndexRecheck);
+   if (node->iss_NumOrderByKeys > 0)
+       return ExecScan(&node->ss,
+                       (ExecScanAccessMtd) IndexNextWithReorder,
+                       (ExecScanRecheckMtd) IndexRecheck);
+   else
+       return ExecScan(&node->ss,
+                       (ExecScanAccessMtd) IndexNext,
+                       (ExecScanRecheckMtd) IndexRecheck);
 }
 
 /* ----------------------------------------------------------------
@@ -465,6 +788,7 @@ ExecInitIndexScan(IndexScan *node, EState *estate, int eflags)
    IndexScanState *indexstate;
    Relation    currentRelation;
    bool        relistarget;
+   int         i;
 
    /*
     * create state structure
@@ -501,6 +825,9 @@ ExecInitIndexScan(IndexScan *node, EState *estate, int eflags)
    indexstate->indexqualorig = (List *)
        ExecInitExpr((Expr *) node->indexqualorig,
                     (PlanState *) indexstate);
+   indexstate->indexorderbyorig = (List *)
+       ExecInitExpr((Expr *) node->indexorderbyorig,
+                    (PlanState *) indexstate);
 
    /*
     * tuple table initialization
@@ -581,6 +908,52 @@ ExecInitIndexScan(IndexScan *node, EState *estate, int eflags)
                           NULL,    /* no ArrayKeys */
                           NULL);
 
+   /* Initialize sort support, if we need to re-check ORDER BY exprs */
+   if (indexstate->iss_NumOrderByKeys > 0)
+   {
+       int         numOrderByKeys = indexstate->iss_NumOrderByKeys;
+
+       /*
+        * Prepare sort support, and look up the distance type for each ORDER
+        * BY expression.
+        */
+       indexstate->iss_SortSupport =
+           palloc0(numOrderByKeys * sizeof(SortSupportData));
+       indexstate->iss_OrderByTypByVals =
+           palloc(numOrderByKeys * sizeof(bool));
+       indexstate->iss_OrderByTypLens =
+           palloc(numOrderByKeys * sizeof(int16));
+       for (i = 0; i < indexstate->iss_NumOrderByKeys; i++)
+       {
+           Oid         orderbyType;
+           Oid         opfamily;
+           int16       strategy;
+
+           PrepareSortSupportFromOrderingOp(node->indexorderbyops[i],
+                                            &indexstate->iss_SortSupport[i]);
+
+           if (!get_ordering_op_properties(node->indexorderbyops[i],
+                                        &opfamily, &orderbyType, &strategy))
+           {
+               elog(LOG, "operator %u is not a valid ordering operator",
+                    node->indexorderbyops[i]);
+           }
+           get_typlenbyval(orderbyType,
+                           &indexstate->iss_OrderByTypLens[i],
+                           &indexstate->iss_OrderByTypByVals[i]);
+       }
+
+       /* allocate arrays to hold the re-calculated distances */
+       indexstate->iss_OrderByValues =
+           palloc(indexstate->iss_NumOrderByKeys * sizeof(Datum));
+       indexstate->iss_OrderByNulls =
+           palloc(indexstate->iss_NumOrderByKeys * sizeof(bool));
+
+       /* and initialize the reorder queue */
+       indexstate->iss_ReorderQueue = pairingheap_allocate(reorderqueue_cmp,
+                                                           indexstate);
+   }
+
    /*
     * If we have runtime keys, we need an ExprContext to evaluate them. The
     * node's standard context won't do because we want to reset that context
index c809237283198ad4d29e6ddce1acd057371cf6bb..783e34b4fbc228f19653b26d4213be88af43863e 100644 (file)
@@ -22,6 +22,7 @@
 #include "access/skey.h"
 #include "access/sysattr.h"
 #include "catalog/pg_class.h"
+#include "catalog/pg_operator.h"
 #include "foreign/fdwapi.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
@@ -102,7 +103,7 @@ static void copy_plan_costsize(Plan *dest, Plan *src);
 static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
 static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
               Oid indexid, List *indexqual, List *indexqualorig,
-              List *indexorderby, List *indexorderbyorig,
+              List *indexorderby, List *indexorderbyorig, Oid *indexorderbyops,
               ScanDirection indexscandir);
 static IndexOnlyScan *make_indexonlyscan(List *qptlist, List *qpqual,
                   Index scanrelid, Oid indexid,
@@ -167,8 +168,8 @@ static Plan *prepare_sort_from_pathkeys(PlannerInfo *root,
                           Oid **p_sortOperators,
                           Oid **p_collations,
                           bool **p_nullsFirst);
-static EquivalenceMember *find_ec_member_for_tle(EquivalenceClass *ec,
-                      TargetEntry *tle,
+static EquivalenceMember *find_ec_member_for_expr(EquivalenceClass *ec,
+                      Expr *tlexpr,
                       Relids relids);
 static Material *make_material(Plan *lefttree);
 
@@ -1158,6 +1159,7 @@ create_indexscan_plan(PlannerInfo *root,
    List       *stripped_indexquals;
    List       *fixed_indexquals;
    List       *fixed_indexorderbys;
+   Oid        *indexorderbyops = NULL;
    ListCell   *l;
 
    /* it should be a base rel... */
@@ -1269,6 +1271,46 @@ create_indexscan_plan(PlannerInfo *root,
            replace_nestloop_params(root, (Node *) indexorderbys);
    }
 
+   /*
+    * If there are ORDER BY expressions, look up the sort operators for
+    * their datatypes.
+    */
+   if (best_path->path.pathkeys && indexorderbys)
+   {
+       int         numOrderBys = list_length(indexorderbys);
+       int         i;
+       ListCell   *pathkeyCell,
+                  *exprCell;
+       PathKey    *pathkey;
+       Expr       *expr;
+       EquivalenceMember *em;
+
+       indexorderbyops = (Oid *) palloc(numOrderBys * sizeof(Oid));
+
+       /*
+        * PathKey contains pointer to the equivalence class, but that's not
+        * enough because we need the expression's datatype to look up the
+        * sort operator in the operator family.  We have to dig the
+        * equivalence member for the datatype.
+        */
+       i = 0;
+       forboth (pathkeyCell, best_path->path.pathkeys, exprCell, indexorderbys)
+       {
+           pathkey = (PathKey *) lfirst(pathkeyCell);
+           expr = (Expr *) lfirst(exprCell);
+
+           /* Find equivalence member for the order by expression */
+           em = find_ec_member_for_expr(pathkey->pk_eclass, expr, NULL);
+
+           /* Get sort operator from opfamily */
+           indexorderbyops[i] = get_opfamily_member(pathkey->pk_opfamily,
+                                                    em->em_datatype,
+                                                    em->em_datatype,
+                                                    pathkey->pk_strategy);
+           i++;
+       }
+   }
+
    /* Finally ready to build the plan node */
    if (indexonly)
        scan_plan = (Scan *) make_indexonlyscan(tlist,
@@ -1288,6 +1330,7 @@ create_indexscan_plan(PlannerInfo *root,
                                            stripped_indexquals,
                                            fixed_indexorderbys,
                                            indexorderbys,
+                                           indexorderbyops,
                                            best_path->indexscandir);
 
    copy_path_costsize(&scan_plan->plan, &best_path->path);
@@ -3344,6 +3387,7 @@ make_indexscan(List *qptlist,
               List *indexqualorig,
               List *indexorderby,
               List *indexorderbyorig,
+              Oid *indexorderbyops,
               ScanDirection indexscandir)
 {
    IndexScan  *node = makeNode(IndexScan);
@@ -3360,6 +3404,7 @@ make_indexscan(List *qptlist,
    node->indexqualorig = indexqualorig;
    node->indexorderby = indexorderby;
    node->indexorderbyorig = indexorderbyorig;
+   node->indexorderbyops = indexorderbyops;
    node->indexorderdir = indexscandir;
 
    return node;
@@ -3990,7 +4035,7 @@ prepare_sort_from_pathkeys(PlannerInfo *root, Plan *lefttree, List *pathkeys,
            tle = get_tle_by_resno(tlist, reqColIdx[numsortkeys]);
            if (tle)
            {
-               em = find_ec_member_for_tle(ec, tle, relids);
+               em = find_ec_member_for_expr(ec, tle->expr, relids);
                if (em)
                {
                    /* found expr at right place in tlist */
@@ -4021,7 +4066,7 @@ prepare_sort_from_pathkeys(PlannerInfo *root, Plan *lefttree, List *pathkeys,
            foreach(j, tlist)
            {
                tle = (TargetEntry *) lfirst(j);
-               em = find_ec_member_for_tle(ec, tle, relids);
+               em = find_ec_member_for_expr(ec, tle->expr, relids);
                if (em)
                {
                    /* found expr already in tlist */
@@ -4142,23 +4187,21 @@ prepare_sort_from_pathkeys(PlannerInfo *root, Plan *lefttree, List *pathkeys,
 }
 
 /*
- * find_ec_member_for_tle
- *     Locate an EquivalenceClass member matching the given TLE, if any
+ * find_ec_member_for_expr
+ *     Locate an EquivalenceClass member matching the given expression, if any
  *
  * Child EC members are ignored unless they match 'relids'.
  */
 static EquivalenceMember *
-find_ec_member_for_tle(EquivalenceClass *ec,
-                      TargetEntry *tle,
-                      Relids relids)
+find_ec_member_for_expr(EquivalenceClass *ec,
+                       Expr *expr,
+                       Relids relids)
 {
-   Expr       *tlexpr;
    ListCell   *lc;
 
    /* We ignore binary-compatible relabeling on both ends */
-   tlexpr = tle->expr;
-   while (tlexpr && IsA(tlexpr, RelabelType))
-       tlexpr = ((RelabelType *) tlexpr)->arg;
+   while (expr && IsA(expr, RelabelType))
+       expr = ((RelabelType *) expr)->arg;
 
    foreach(lc, ec->ec_members)
    {
@@ -4184,7 +4227,7 @@ find_ec_member_for_tle(EquivalenceClass *ec,
        while (emexpr && IsA(emexpr, RelabelType))
            emexpr = ((RelabelType *) emexpr)->arg;
 
-       if (equal(emexpr, tlexpr))
+       if (equal(emexpr, expr))
            return em;
    }
 
index 39a78552410b44c60da235ca28526576c9022eca..a3af08ad6b00e160e7a4528d6e1e8ad34f861082 100644 (file)
@@ -2657,6 +2657,18 @@ dist_ppoly(PG_FUNCTION_ARGS)
    PG_RETURN_FLOAT8(result);
 }
 
+Datum
+dist_polyp(PG_FUNCTION_ARGS)
+{
+   POLYGON    *poly = PG_GETARG_POLYGON_P(0);
+   Point      *point = PG_GETARG_POINT_P(1);
+   float8      result;
+
+   result = dist_ppoly_internal(point, poly);
+
+   PG_RETURN_FLOAT8(result);
+}
+
 static double
 dist_ppoly_internal(Point *pt, POLYGON *poly)
 {
@@ -5112,6 +5124,21 @@ dist_pc(PG_FUNCTION_ARGS)
    PG_RETURN_FLOAT8(result);
 }
 
+/*
+ * Distance from a circle to a point
+ */
+Datum
+dist_cpoint(PG_FUNCTION_ARGS)
+{
+   CIRCLE     *circle = PG_GETARG_CIRCLE_P(0);
+   Point      *point = PG_GETARG_POINT_P(1);
+   float8      result;
+
+   result = point_dt(point, &circle->center) - circle->radius;
+   if (result < 0)
+       result = 0;
+   PG_RETURN_FLOAT8(result);
+}
 
 /*     circle_center   -       returns the center point of the circle.
  */
index d86590ac111e6064c06a85b4dbc7b9979b9192b2..f129c4b58f9f9bf4cd0731a4f0dc3b14c6cb8ac1 100644 (file)
@@ -147,7 +147,10 @@ extern void index_restrpos(IndexScanDesc scan);
 extern ItemPointer index_getnext_tid(IndexScanDesc scan,
                  ScanDirection direction);
 extern HeapTuple index_fetch_heap(IndexScanDesc scan);
+extern bool index_get_heap_values(IndexScanDesc scan, ItemPointer heapPtr,
+                   Datum values[INDEX_MAX_KEYS], bool isnull[INDEX_MAX_KEYS]);
 extern HeapTuple index_getnext(IndexScanDesc scan, ScanDirection direction);
+
 extern int64 index_getbitmap(IndexScanDesc scan, TIDBitmap *bitmap);
 
 extern IndexBulkDeleteResult *index_bulk_delete(IndexVacuumInfo *info,
index 9bb63622fe889eb2adf49105147edacfdad3c352..865d36403a2113a463f27a6e714e521ddc4247e0 100644 (file)
@@ -91,6 +91,15 @@ typedef struct IndexScanDescData
    /* NB: if xs_cbuf is not InvalidBuffer, we hold a pin on that buffer */
    bool        xs_recheck;     /* T means scan keys must be rechecked */
 
+   /*
+    * When fetching with an ordering operator, the values of the ORDER BY
+    * expressions of the last returned tuple, according to the index.  If
+    * xs_recheck is true, these need to be rechecked just like the scan keys,
+    * and the values returned here are a lower-bound on the actual values.
+    */
+   Datum      *xs_orderbyvals;
+   bool       *xs_orderbynulls;
+
    /* state data for traversing HOT chains in index_getnext */
    bool        xs_continue_hot;    /* T if must keep walking HOT chain */
 }  IndexScanDescData;
index 09fdc364e03ba18a4cad6c8ce86088e55dc185a3..b6a6da9a10a78d330de9f6be40fb558f3e4ed5f4 100644 (file)
@@ -53,6 +53,6 @@
  */
 
 /*                         yyyymmddN */
-#define CATALOG_VERSION_NO 201505141
+#define CATALOG_VERSION_NO 201505151
 
 #endif
index 5aab896786fe2a805038b9dda723d8d162350f8f..9b8294fd94c0f464bcfabcef54fab934778be80c 100644 (file)
@@ -650,6 +650,7 @@ DATA(insert (   2594   604 604 11 s 2577 783 0 ));
 DATA(insert (  2594   604 604 12 s 2576 783 0 ));
 DATA(insert (  2594   604 604 13 s 2861 783 0 ));
 DATA(insert (  2594   604 604 14 s 2860 783 0 ));
+DATA(insert (  2594   604 600 15 o 3289 783 1970 ));
 
 /*
  * gist circle_ops
@@ -669,6 +670,7 @@ DATA(insert (   2595   718 718 11 s 1514 783 0 ));
 DATA(insert (  2595   718 718 12 s 2590 783 0 ));
 DATA(insert (  2595   718 718 13 s 2865 783 0 ));
 DATA(insert (  2595   718 718 14 s 2864 783 0 ));
+DATA(insert (  2595   718 600 15 o 3291 783 1970 ));
 
 /*
  * gin array_ops (these anyarray operators are used with all the opclasses
index e3de3b57e79cb4539b735d28f9160f30d36a838e..3111d6f4ad17b484bf9982a0034606701a796600 100644 (file)
@@ -208,6 +208,7 @@ DATA(insert (   2594   604 604 4 2580 ));
 DATA(insert (  2594   604 604 5 2581 ));
 DATA(insert (  2594   604 604 6 2582 ));
 DATA(insert (  2594   604 604 7 2584 ));
+DATA(insert (  2594   604 604 8 3288 ));
 DATA(insert (  2595   718 718 1 2591 ));
 DATA(insert (  2595   718 718 2 2583 ));
 DATA(insert (  2595   718 718 3 2592 ));
@@ -215,6 +216,7 @@ DATA(insert (   2595   718 718 4 2580 ));
 DATA(insert (  2595   718 718 5 2581 ));
 DATA(insert (  2595   718 718 6 2582 ));
 DATA(insert (  2595   718 718 7 2584 ));
+DATA(insert (  2595   718 718 8 3288 ));
 DATA(insert (  3655   3614 3614 1 3654 ));
 DATA(insert (  3655   3614 3614 2 3651 ));
 DATA(insert (  3655   3614 3614 3 3648 ));
index 34ebb50ea5f7cb361831ea00626ef299c004985c..6e260cb304743eaa922ff348a007f7c068f40efd 100644 (file)
@@ -1015,9 +1015,13 @@ DATA(insert OID = 1520 (  "<->"   PGNSP PGUID b f f  718 718  701   1520    0 ci
 DESCR("distance between");
 DATA(insert OID = 1521 (  "#"    PGNSP PGUID l f f  0      604   23      0    0 poly_npoints - - ));
 DESCR("number of points");
-DATA(insert OID = 1522 (  "<->"   PGNSP PGUID b f f  600   718  701      0    0 dist_pc - - ));
+DATA(insert OID = 1522 (  "<->"   PGNSP PGUID b f f  600   718  701   3291    0 dist_pc - - ));
 DESCR("distance between");
-DATA(insert OID = 3276 (  "<->"      PGNSP PGUID b f f  600    604  701      0    0 dist_ppoly - - ));
+DATA(insert OID = 3291 (  "<->"   PGNSP PGUID b f f  718   600  701   1522    0 dist_cpoint - - ));
+DESCR("distance between");
+DATA(insert OID = 3276 (  "<->"      PGNSP PGUID b f f  600    604  701   3289    0 dist_ppoly - - ));
+DESCR("distance between");
+DATA(insert OID = 3289 (  "<->"      PGNSP PGUID b f f  604    600  701   3276    0 dist_polyp - - ));
 DESCR("distance between");
 DATA(insert OID = 1523 (  "<->"   PGNSP PGUID b f f  718   604  701      0    0 dist_cpoly - - ));
 DESCR("distance between");
index d4bc60b7d66a22e1e0bdac3944485619e626d2a0..1c9edbc3b371e46cc1c3eafc02385969669e8db7 100644 (file)
@@ -856,6 +856,8 @@ DATA(insert OID = 727 (  dist_sl           PGNSP PGUID 12 1 0 0 0 f f f f t f i 2 0 70
 DATA(insert OID = 728 (  dist_cpoly           PGNSP PGUID 12 1 0 0 0 f f f f t f i 2 0 701 "718 604" _null_ _null_ _null_ _null_ _null_    dist_cpoly _null_ _null_ _null_ ));
 DATA(insert OID = 729 (  poly_distance    PGNSP PGUID 12 1 0 0 0 f f f f t f i 2 0 701 "604 604" _null_ _null_ _null_ _null_ _null_    poly_distance _null_ _null_ _null_ ));
 DATA(insert OID = 3275 (  dist_ppoly      PGNSP PGUID 12 1 0 0 0 f f f f t f i 2 0 701 "600 604" _null_ _null_ _null_ _null_ _null_    dist_ppoly _null_ _null_ _null_ ));
+DATA(insert OID = 3292 (  dist_polyp      PGNSP PGUID 12 1 0 0 0 f f f f t f i 2 0 701 "604 600" _null_ _null_ _null_ _null_ _null_    dist_polyp _null_ _null_ _null_ ));
+DATA(insert OID = 3290 (  dist_cpoint     PGNSP PGUID 12 1 0 0 0 f f f f t f i 2 0 701 "718 600" _null_ _null_ _null_ _null_ _null_    dist_cpoint _null_ _null_ _null_ ));
 
 DATA(insert OID = 740 (  text_lt          PGNSP PGUID 12 1 0 0 0 f f f f t f i 2 0 16 "25 25" _null_ _null_ _null_ _null_ _null_ text_lt _null_ _null_ _null_ ));
 DATA(insert OID = 741 (  text_le          PGNSP PGUID 12 1 0 0 0 f f f f t f i 2 0 16 "25 25" _null_ _null_ _null_ _null_ _null_ text_le _null_ _null_ _null_ ));
@@ -4165,6 +4167,8 @@ DATA(insert OID = 2179 (  gist_point_consistent PGNSP PGUID 12 1 0 0 0 f f f f t
 DESCR("GiST support");
 DATA(insert OID = 3064 (  gist_point_distance  PGNSP PGUID 12 1 0 0 0 f f f f t f i 4 0 701 "2281 600 23 26" _null_ _null_ _null_ _null_ _null_    gist_point_distance _null_ _null_ _null_ ));
 DESCR("GiST support");
+DATA(insert OID = 3288 (  gist_bbox_distance   PGNSP PGUID 12 1 0 0 0 f f f f t f i 4 0 701 "2281 600 23 26" _null_ _null_ _null_ _null_ _null_    gist_bbox_distance _null_ _null_ _null_ ));
+DESCR("GiST support");
 
 /* GIN */
 DATA(insert OID = 2731 (  gingetbitmap    PGNSP PGUID 12 1 0 0 0 f f f f t f v 2 0 20 "2281 2281" _null_ _null_ _null_ _null_ _null_   gingetbitmap _null_ _null_ _null_ ));
index 5ad2cc235883b973fb8b232feb96af1910842e53..fcfe1107f92cf3b9430b70d4f2ec4a5c0711e9b3 100644 (file)
@@ -17,6 +17,7 @@
 #include "access/genam.h"
 #include "access/heapam.h"
 #include "executor/instrument.h"
+#include "lib/pairingheap.h"
 #include "nodes/params.h"
 #include "nodes/plannodes.h"
 #include "utils/reltrigger.h"
@@ -1262,6 +1263,7 @@ typedef struct
  *  IndexScanState information
  *
  *     indexqualorig      execution state for indexqualorig expressions
+ *     indexorderbyorig   execution state for indexorderbyorig expressions
  *     ScanKeys           Skey structures for index quals
  *     NumScanKeys        number of ScanKeys
  *     OrderByKeys        Skey structures for index ordering operators
@@ -1272,12 +1274,21 @@ typedef struct
  *     RuntimeContext     expr context for evaling runtime Skeys
  *     RelationDesc       index relation descriptor
  *     ScanDesc           index scan descriptor
+ *
+ *     ReorderQueue       tuples that need reordering due to re-check
+ *     ReachedEnd         have we fetched all tuples from index already?
+ *     OrderByValues      values of ORDER BY exprs of last fetched tuple
+ *     OrderByNulls       null flags for OrderByValues
+ *     SortSupport        for reordering ORDER BY exprs
+ *     OrderByTypByVals   is the datatype of order by expression pass-by-value?
+ *     OrderByTypLens     typlens of the datatypes of order by expressions
  * ----------------
  */
 typedef struct IndexScanState
 {
    ScanState   ss;             /* its first field is NodeTag */
    List       *indexqualorig;
+   List       *indexorderbyorig;
    ScanKey     iss_ScanKeys;
    int         iss_NumScanKeys;
    ScanKey     iss_OrderByKeys;
@@ -1288,6 +1299,15 @@ typedef struct IndexScanState
    ExprContext *iss_RuntimeContext;
    Relation    iss_RelationDesc;
    IndexScanDesc iss_ScanDesc;
+
+   /* These are needed for re-checking ORDER BY expr ordering */
+   pairingheap *iss_ReorderQueue;
+   bool        iss_ReachedEnd;
+   Datum      *iss_OrderByValues;
+   bool       *iss_OrderByNulls;
+   SortSupport iss_SortSupport;
+   bool       *iss_OrderByTypByVals;
+   int16      *iss_OrderByTypLens;
 } IndexScanState;
 
 /* ----------------
index 1494b336c22b9c1bbaeb942936ee790e40f8ef6c..65f71d81705ddc7b87587fe468697f4b3922d4fa 100644 (file)
@@ -311,8 +311,13 @@ typedef Scan SeqScan;
  * index column order.  Only the expressions are provided, not the auxiliary
  * sort-order information from the ORDER BY SortGroupClauses; it's assumed
  * that the sort ordering is fully determinable from the top-level operators.
- * indexorderbyorig is unused at run time, but is needed for EXPLAIN.
- * (Note these fields are used for amcanorderbyop cases, not amcanorder cases.)
+ * indexorderbyorig is used at runtime to recheck the ordering, if the index
+ * cannot calculate an accurate ordering.  It is also needed for EXPLAIN.
+ *
+ * indexorderbyops is an array of operators used to sort the ORDER BY
+ * expressions, used together with indexorderbyorig to recheck ordering at run
+ * time.  (Note these fields are used for amcanorderbyop cases, not amcanorder
+ * cases.)
  *
  * indexorderdir specifies the scan ordering, for indexscans on amcanorder
  * indexes (for other indexes it should be "don't care").
@@ -326,6 +331,7 @@ typedef struct IndexScan
    List       *indexqualorig;  /* the same in original form */
    List       *indexorderby;   /* list of index ORDER BY exprs */
    List       *indexorderbyorig;       /* the same in original form */
+   Oid        *indexorderbyops;    /* operators to sort ORDER BY exprs */
    ScanDirection indexorderdir;    /* forward or backward or don't care */
 } IndexScan;
 
index 4377baa645ed26529c1170c9664bfb3b9eb85e40..2311d35dd72c903106871786d43d85ecb929afb4 100644 (file)
@@ -394,8 +394,10 @@ extern Datum circle_diameter(PG_FUNCTION_ARGS);
 extern Datum circle_radius(PG_FUNCTION_ARGS);
 extern Datum circle_distance(PG_FUNCTION_ARGS);
 extern Datum dist_pc(PG_FUNCTION_ARGS);
+extern Datum dist_cpoint(PG_FUNCTION_ARGS);
 extern Datum dist_cpoly(PG_FUNCTION_ARGS);
 extern Datum dist_ppoly(PG_FUNCTION_ARGS);
+extern Datum dist_polyp(PG_FUNCTION_ARGS);
 extern Datum circle_center(PG_FUNCTION_ARGS);
 extern Datum cr_circle(PG_FUNCTION_ARGS);
 extern Datum box_circle(PG_FUNCTION_ARGS);
@@ -420,6 +422,7 @@ extern Datum gist_circle_consistent(PG_FUNCTION_ARGS);
 extern Datum gist_point_compress(PG_FUNCTION_ARGS);
 extern Datum gist_point_consistent(PG_FUNCTION_ARGS);
 extern Datum gist_point_distance(PG_FUNCTION_ARGS);
+extern Datum gist_bbox_distance(PG_FUNCTION_ARGS);
 extern Datum gist_point_fetch(PG_FUNCTION_ARGS);
 
 
index a98b10520d5f212d29b19c88ca5b78119cd9ec5a..71aee7622568f45d6dfe6a5f001dcc3c93c45cd1 100644 (file)
@@ -372,6 +372,36 @@ SELECT count(*) FROM radix_text_tbl WHERE t ~>~  'Worth
     48
 (1 row)
 
+SELECT * FROM gpolygon_tbl ORDER BY f1 <-> '(0,0)'::point LIMIT 10;
+                       f1                        
+-------------------------------------------------
+ ((240,359),(240,455),(337,455),(337,359))
+ ((662,163),(662,187),(759,187),(759,163))
+ ((1000,0),(0,1000))
+ ((0,1000),(1000,1000))
+ ((1346,344),(1346,403),(1444,403),(1444,344))
+ ((278,1409),(278,1457),(369,1457),(369,1409))
+ ((907,1156),(907,1201),(948,1201),(948,1156))
+ ((1517,971),(1517,1043),(1594,1043),(1594,971))
+ ((175,1820),(175,1850),(259,1850),(259,1820))
+ ((2424,81),(2424,160),(2424,160),(2424,81))
+(10 rows)
+
+SELECT * FROM gcircle_tbl ORDER BY f1 <-> '(200,300)'::point LIMIT 10;
+                f1                 
+-----------------------------------
+ <(288.5,407),68.2367203197809>
+ <(710.5,175),49.9624859269432>
+ <(323.5,1433),51.4417145903983>
+ <(927.5,1178.5),30.4384625104489>
+ <(1395,373.5),57.1948424248201>
+ <(1555.5,1007),52.7091073724456>
+ <(217,1835),44.5982062419555>
+ <(489,2421.5),22.3886131772381>
+ <(2424,120.5),39.5>
+ <(751.5,2655),20.4022057631032>
+(10 rows)
+
 -- Now check the results from plain indexscan
 SET enable_seqscan = OFF;
 SET enable_indexscan = ON;
@@ -1152,6 +1182,54 @@ SELECT count(*) FROM radix_text_tbl WHERE t ~>~  'Worth
     48
 (1 row)
 
+EXPLAIN (COSTS OFF)
+SELECT * FROM gpolygon_tbl ORDER BY f1 <-> '(0,0)'::point LIMIT 10;
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Limit
+   ->  Index Scan using ggpolygonind on gpolygon_tbl
+         Order By: (f1 <-> '(0,0)'::point)
+(3 rows)
+
+SELECT * FROM gpolygon_tbl ORDER BY f1 <-> '(0,0)'::point LIMIT 10;
+                       f1                        
+-------------------------------------------------
+ ((240,359),(240,455),(337,455),(337,359))
+ ((662,163),(662,187),(759,187),(759,163))
+ ((1000,0),(0,1000))
+ ((0,1000),(1000,1000))
+ ((1346,344),(1346,403),(1444,403),(1444,344))
+ ((278,1409),(278,1457),(369,1457),(369,1409))
+ ((907,1156),(907,1201),(948,1201),(948,1156))
+ ((1517,971),(1517,1043),(1594,1043),(1594,971))
+ ((175,1820),(175,1850),(259,1850),(259,1820))
+ ((2424,81),(2424,160),(2424,160),(2424,81))
+(10 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT * FROM gcircle_tbl ORDER BY f1 <-> '(200,300)'::point LIMIT 10;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Limit
+   ->  Index Scan using ggcircleind on gcircle_tbl
+         Order By: (f1 <-> '(200,300)'::point)
+(3 rows)
+
+SELECT * FROM gcircle_tbl ORDER BY f1 <-> '(200,300)'::point LIMIT 10;
+                f1                 
+-----------------------------------
+ <(288.5,407),68.2367203197809>
+ <(710.5,175),49.9624859269432>
+ <(323.5,1433),51.4417145903983>
+ <(927.5,1178.5),30.4384625104489>
+ <(1395,373.5),57.1948424248201>
+ <(1555.5,1007),52.7091073724456>
+ <(217,1835),44.5982062419555>
+ <(489,2421.5),22.3886131772381>
+ <(2424,120.5),39.5>
+ <(751.5,2655),20.4022057631032>
+(10 rows)
+
 -- Now check the results from bitmap indexscan
 SET enable_seqscan = OFF;
 SET enable_indexscan = OFF;
index 6333a30bf6d62c774268c4985a7c2ea45647658d..acbfbd51332e354dc67f06c4fe81f02825c89ceb 100644 (file)
@@ -224,6 +224,10 @@ SELECT count(*) FROM radix_text_tbl WHERE t >    'Worth
 
 SELECT count(*) FROM radix_text_tbl WHERE t ~>~  'Worth                         St  ';
 
+SELECT * FROM gpolygon_tbl ORDER BY f1 <-> '(0,0)'::point LIMIT 10;
+
+SELECT * FROM gcircle_tbl ORDER BY f1 <-> '(200,300)'::point LIMIT 10;
+
 -- Now check the results from plain indexscan
 SET enable_seqscan = OFF;
 SET enable_indexscan = ON;
@@ -437,6 +441,14 @@ EXPLAIN (COSTS OFF)
 SELECT count(*) FROM radix_text_tbl WHERE t ~>~  'Worth                         St  ';
 SELECT count(*) FROM radix_text_tbl WHERE t ~>~  'Worth                         St  ';
 
+EXPLAIN (COSTS OFF)
+SELECT * FROM gpolygon_tbl ORDER BY f1 <-> '(0,0)'::point LIMIT 10;
+SELECT * FROM gpolygon_tbl ORDER BY f1 <-> '(0,0)'::point LIMIT 10;
+
+EXPLAIN (COSTS OFF)
+SELECT * FROM gcircle_tbl ORDER BY f1 <-> '(200,300)'::point LIMIT 10;
+SELECT * FROM gcircle_tbl ORDER BY f1 <-> '(200,300)'::point LIMIT 10;
+
 -- Now check the results from bitmap indexscan
 SET enable_seqscan = OFF;
 SET enable_indexscan = OFF;