aboutsummaryrefslogtreecommitdiff
path: root/src/backend/executor/nodeWindowAgg.c
diff options
context:
space:
mode:
Diffstat (limited to 'src/backend/executor/nodeWindowAgg.c')
-rw-r--r--src/backend/executor/nodeWindowAgg.c436
1 files changed, 249 insertions, 187 deletions
diff --git a/src/backend/executor/nodeWindowAgg.c b/src/backend/executor/nodeWindowAgg.c
index 37ef9a5e830..181e66a8964 100644
--- a/src/backend/executor/nodeWindowAgg.c
+++ b/src/backend/executor/nodeWindowAgg.c
@@ -27,7 +27,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
- * $PostgreSQL: pgsql/src/backend/executor/nodeWindowAgg.c,v 1.1 2008/12/28 18:53:55 tgl Exp $
+ * $PostgreSQL: pgsql/src/backend/executor/nodeWindowAgg.c,v 1.2 2008/12/31 00:08:35 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -118,11 +118,10 @@ typedef struct WindowStatePerAggData
bool initValueIsNull;
/*
- * cached value for non-moving frame
+ * cached value for current frame boundaries
*/
Datum resultValue;
bool resultValueIsNull;
- bool hasResult;
/*
* We need the len and byval info for the agg's input, result, and
@@ -164,6 +163,10 @@ static void begin_partition(WindowAggState *winstate);
static void spool_tuples(WindowAggState *winstate, int64 pos);
static void release_partition(WindowAggState *winstate);
+static bool row_is_in_frame(WindowAggState *winstate, int64 pos,
+ TupleTableSlot *slot);
+static void update_frametailpos(WindowObject winobj, TupleTableSlot *slot);
+
static WindowStatePerAggData *initialize_peragg(WindowAggState *winstate,
WindowFunc *wfunc,
WindowStatePerAgg peraggstate);
@@ -198,6 +201,7 @@ initialize_windowaggregate(WindowAggState *winstate,
}
peraggstate->transValueIsNull = peraggstate->initValueIsNull;
peraggstate->noTransValue = peraggstate->initValueIsNull;
+ peraggstate->resultValueIsNull = true;
}
/*
@@ -385,9 +389,7 @@ eval_windowaggregates(WindowAggState *winstate)
int i;
MemoryContext oldContext;
ExprContext *econtext;
- TupleTableSlot *first_peer_slot = winstate->first_peer_slot;
- TupleTableSlot *slot;
- bool first;
+ TupleTableSlot *agg_row_slot;
numaggs = winstate->numaggs;
if (numaggs == 0)
@@ -397,35 +399,45 @@ eval_windowaggregates(WindowAggState *winstate)
econtext = winstate->ss.ps.ps_ExprContext;
/*
- * We don't currently support explicitly-specified window frames. That
- * means that the window frame always includes all the rows in the
- * partition preceding and including the current row, and all its
- * peers. As a special case, if there's no ORDER BY, all rows are peers,
- * so the window frame includes all rows in the partition.
+ * Currently, we support only a subset of the SQL-standard window framing
+ * rules. In all the supported cases, the window frame always consists
+ * of a contiguous group of rows extending forward from the start of the
+ * partition, and rows only enter the frame, never exit it, as the
+ * current row advances forward. This makes it possible to use an
+ * incremental strategy for evaluating aggregates: we run the transition
+ * function for each row added to the frame, and run the final function
+ * whenever we need the current aggregate value. This is considerably
+ * more efficient than the naive approach of re-running the entire
+ * aggregate calculation for each current row. It does assume that the
+ * final function doesn't damage the running transition value. (Some
+ * C-coded aggregates do that for efficiency's sake --- but they are
+ * supposed to do so only when their fcinfo->context is an AggState, not
+ * a WindowAggState.)
*
- * When there's peer rows, all rows in a peer group will have the same
- * aggregate values. The values will be calculated when current position
- * reaches the first peer row, and on all the following peer rows we will
- * just return the saved results.
+ * In many common cases, multiple rows share the same frame and hence
+ * the same aggregate value. (In particular, if there's no ORDER BY in
+ * a RANGE window, then all rows are peers and so they all have window
+ * frame equal to the whole partition.) We optimize such cases by
+ * calculating the aggregate value once when we reach the first row of a
+ * peer group, and then returning the saved value for all subsequent rows.
*
- * 'aggregatedupto' keeps track of the last row that has already been
- * accumulated for the aggregates. When the current row has no peers,
- * aggregatedupto will be the same as the current row after this
- * function. If there are peer rows, all peers will be accumulated in one
- * call of this function, and aggregatedupto will be ahead of the current
- * position. If there's no ORDER BY, and thus all rows are peers, the
- * first call will aggregate all rows in the partition.
+ * 'aggregatedupto' keeps track of the first row that has not yet been
+ * accumulated into the aggregate transition values. Whenever we start a
+ * new peer group, we accumulate forward to the end of the peer group.
*
- * TODO: In the future, we could implement sliding frames by recalculating
- * the aggregate whenever a row exits the frame. That would be pretty
- * slow, though. For aggregates like SUM and COUNT we could implement a
- * "negative transition function" that would be called for all the rows
- * that exit the frame.
+ * TODO: In the future, we should implement the full SQL-standard set
+ * of framing rules. We could implement the other cases by recalculating
+ * the aggregates whenever a row exits the frame. That would be pretty
+ * slow, though. For aggregates like SUM and COUNT we could implement a
+ * "negative transition function" that would be called for each row as it
+ * exits the frame. We'd have to think about avoiding recalculation of
+ * volatile arguments of aggregate functions, too.
*/
/*
* If we've already aggregated up through current row, reuse the
- * saved result values
+ * saved result values. NOTE: this test works for the currently
+ * supported framing rules, but will need fixing when more are added.
*/
if (winstate->aggregatedupto > winstate->currentpos)
{
@@ -440,76 +452,61 @@ eval_windowaggregates(WindowAggState *winstate)
}
/* Initialize aggregates on first call for partition */
- for (i = 0; i < numaggs; i++)
+ if (winstate->currentpos == 0)
{
- peraggstate = &winstate->peragg[i];
- wfuncno = peraggstate->wfuncno;
- if (!peraggstate->hasResult)
+ for (i = 0; i < numaggs; i++)
+ {
+ peraggstate = &winstate->peragg[i];
+ wfuncno = peraggstate->wfuncno;
initialize_windowaggregate(winstate,
&winstate->perfunc[wfuncno],
- &winstate->peragg[i]);
- }
-
- /*
- * If this is the first call for this partition, fetch the first row
- * for comparing peer rows. On subsequent calls, we'll always read
- * ahead until we reach the first non-peer row, and store that row in
- * first_peer_slot, for use in the next call.
- */
- if (TupIsNull(first_peer_slot))
- {
- spool_tuples(winstate, winstate->aggregatedupto);
- tuplestore_select_read_pointer(winstate->buffer, winstate->agg_ptr);
- if (!tuplestore_gettupleslot(winstate->buffer, true, first_peer_slot))
- elog(ERROR, "unexpected end of tuplestore");
+ peraggstate);
+ }
}
/*
- * Advance until we reach the next non-peer row
+ * Advance until we reach a row not in frame (or end of partition).
+ *
+ * Note the loop invariant: agg_row_slot is either empty or holds the
+ * row at position aggregatedupto. The agg_ptr read pointer must always
+ * point to the next row to read into agg_row_slot.
*/
- first = true;
+ agg_row_slot = winstate->agg_row_slot;
for (;;)
{
- if (!first)
+ /* Fetch next row if we didn't already */
+ if (TupIsNull(agg_row_slot))
{
- /* Fetch the next row, and see if it's a peer */
spool_tuples(winstate, winstate->aggregatedupto);
tuplestore_select_read_pointer(winstate->buffer,
winstate->agg_ptr);
- slot = winstate->temp_slot_1;
- if (!tuplestore_gettupleslot(winstate->buffer, true, slot))
- break;
- if (!are_peers(winstate, first_peer_slot, slot))
- {
- ExecCopySlot(first_peer_slot, slot);
- break;
- }
- }
- else
- {
- /*
- * On first iteration, just accumulate the tuple saved from
- * last call
- */
- slot = first_peer_slot;
- first = false;
+ if (!tuplestore_gettupleslot(winstate->buffer, true, agg_row_slot))
+ break; /* must be end of partition */
}
- /* set tuple context for evaluation of aggregate arguments */
- winstate->tmpcontext->ecxt_outertuple = slot;
+ /* Exit loop (for now) if not in frame */
+ if (!row_is_in_frame(winstate, winstate->aggregatedupto, agg_row_slot))
+ break;
+ /* Set tuple context for evaluation of aggregate arguments */
+ winstate->tmpcontext->ecxt_outertuple = agg_row_slot;
+
+ /* Accumulate row into the aggregates */
for (i = 0; i < numaggs; i++)
{
- wfuncno = winstate->peragg[i].wfuncno;
-
+ peraggstate = &winstate->peragg[i];
+ wfuncno = peraggstate->wfuncno;
advance_windowaggregate(winstate,
&winstate->perfunc[wfuncno],
- &winstate->peragg[i]);
-
+ peraggstate);
}
+
/* Reset per-input-tuple context after each tuple */
ResetExprContext(winstate->tmpcontext);
+
+ /* And advance the aggregated-row state */
winstate->aggregatedupto++;
+ ExecClearTuple(agg_row_slot);
}
/*
@@ -526,37 +523,44 @@ eval_windowaggregates(WindowAggState *winstate)
isnull = &econtext->ecxt_aggnulls[wfuncno];
finalize_windowaggregate(winstate,
&winstate->perfunc[wfuncno],
- peraggstate, result, isnull);
+ peraggstate,
+ result, isnull);
/*
- * save the result for the next (non-shrinking frame) call.
+ * save the result in case next row shares the same frame.
+ *
+ * XXX in some framing modes, eg ROWS/END_CURRENT_ROW, we can know
+ * in advance that the next row can't possibly share the same frame.
+ * Is it worth detecting that and skipping this code?
*/
- if (!peraggstate->resulttypeByVal && !*isnull)
+ if (!peraggstate->resulttypeByVal)
{
/*
- * clear old resultValue in order not to leak memory.
+ * clear old resultValue in order not to leak memory. (Note:
+ * the new result can't possibly be the same datum as old
+ * resultValue, because we never passed it to the trans function.)
*/
- if (peraggstate->hasResult &&
- (DatumGetPointer(peraggstate->resultValue) !=
- DatumGetPointer(*result)) &&
- !peraggstate->resultValueIsNull)
+ if (!peraggstate->resultValueIsNull)
pfree(DatumGetPointer(peraggstate->resultValue));
/*
* If pass-by-ref, copy it into our global context.
*/
- oldContext = MemoryContextSwitchTo(winstate->wincontext);
- peraggstate->resultValue = datumCopy(*result,
- peraggstate->resulttypeByVal,
- peraggstate->resulttypeLen);
- MemoryContextSwitchTo(oldContext);
+ if (!*isnull)
+ {
+ oldContext = MemoryContextSwitchTo(winstate->wincontext);
+ peraggstate->resultValue =
+ datumCopy(*result,
+ peraggstate->resulttypeByVal,
+ peraggstate->resulttypeLen);
+ MemoryContextSwitchTo(oldContext);
+ }
}
else
{
peraggstate->resultValue = *result;
}
peraggstate->resultValueIsNull = *isnull;
- peraggstate->hasResult = true;
}
}
@@ -620,14 +624,16 @@ begin_partition(WindowAggState *winstate)
int i;
winstate->partition_spooled = false;
+ winstate->frametail_valid = false;
winstate->spooled_rows = 0;
winstate->currentpos = 0;
winstate->frametailpos = -1;
winstate->aggregatedupto = 0;
+ ExecClearTuple(winstate->agg_row_slot);
/*
* If this is the very first partition, we need to fetch the first
- * input row to store in it.
+ * input row to store in first_part_slot.
*/
if (TupIsNull(winstate->first_part_slot))
{
@@ -775,35 +781,138 @@ release_partition(WindowAggState *winstate)
/* Release any partition-local state of this window function */
if (perfuncstate->winobj)
perfuncstate->winobj->localmem = NULL;
-
- /* Reset agg result cache */
- if (perfuncstate->plain_agg)
- {
- int aggno = perfuncstate->aggno;
- WindowStatePerAggData *peraggstate = &winstate->peragg[aggno];
-
- peraggstate->resultValueIsNull = true;
- peraggstate->hasResult = false;
- }
}
/*
* Release all partition-local memory (in particular, any partition-local
- * state or aggregate temp data that we might have trashed our pointers
- * to in the above loop). We don't rely on retail pfree because some
+ * state that we might have trashed our pointers to in the above loop, and
+ * any aggregate temp data). We don't rely on retail pfree because some
* aggregates might have allocated data we don't have direct pointers to.
*/
MemoryContextResetAndDeleteChildren(winstate->wincontext);
- /* Ensure eval_windowaggregates will see next call as partition start */
- ExecClearTuple(winstate->first_peer_slot);
-
if (winstate->buffer)
tuplestore_end(winstate->buffer);
winstate->buffer = NULL;
winstate->partition_spooled = false;
}
+/*
+ * row_is_in_frame
+ * Determine whether a row is in the current row's window frame according
+ * to our window framing rule
+ *
+ * The caller must have already determined that the row is in the partition
+ * and fetched it into a slot. This function just encapsulates the framing
+ * rules.
+ */
+static bool
+row_is_in_frame(WindowAggState *winstate, int64 pos, TupleTableSlot *slot)
+{
+ WindowAgg *node = (WindowAgg *) winstate->ss.ps.plan;
+ int frameOptions = node->frameOptions;
+
+ Assert(pos >= 0); /* else caller error */
+
+ /* We only support frame start mode UNBOUNDED PRECEDING for now */
+ Assert(frameOptions & FRAMEOPTION_START_UNBOUNDED_PRECEDING);
+
+ /* In UNBOUNDED FOLLOWING mode, all partition rows are in frame */
+ if (frameOptions & FRAMEOPTION_END_UNBOUNDED_FOLLOWING)
+ return true;
+
+ /* Else frame tail mode must be CURRENT ROW */
+ Assert(frameOptions & FRAMEOPTION_END_CURRENT_ROW);
+
+ /* if row is current row or a predecessor, it must be in frame */
+ if (pos <= winstate->currentpos)
+ return true;
+
+ /* In ROWS mode, *only* such rows are in frame */
+ if (frameOptions & FRAMEOPTION_ROWS)
+ return false;
+
+ /* Else must be RANGE mode */
+ Assert(frameOptions & FRAMEOPTION_RANGE);
+
+ /* In frame iff it's a peer of current row */
+ return are_peers(winstate, slot, winstate->ss.ss_ScanTupleSlot);
+}
+
+/*
+ * update_frametailpos
+ * make frametailpos valid for the current row
+ *
+ * Uses the winobj's read pointer for any required fetches; the winobj's
+ * mark must not be past the currently known frame tail. Also uses the
+ * specified slot for any required fetches.
+ */
+static void
+update_frametailpos(WindowObject winobj, TupleTableSlot *slot)
+{
+ WindowAggState *winstate = winobj->winstate;
+ WindowAgg *node = (WindowAgg *) winstate->ss.ps.plan;
+ int frameOptions = node->frameOptions;
+ int64 ftnext;
+
+ if (winstate->frametail_valid)
+ return; /* already known for current row */
+
+ /* We only support frame start mode UNBOUNDED PRECEDING for now */
+ Assert(frameOptions & FRAMEOPTION_START_UNBOUNDED_PRECEDING);
+
+ /* In UNBOUNDED FOLLOWING mode, all partition rows are in frame */
+ if (frameOptions & FRAMEOPTION_END_UNBOUNDED_FOLLOWING)
+ {
+ spool_tuples(winstate, -1);
+ winstate->frametailpos = winstate->spooled_rows - 1;
+ winstate->frametail_valid = true;
+ return;
+ }
+
+ /* Else frame tail mode must be CURRENT ROW */
+ Assert(frameOptions & FRAMEOPTION_END_CURRENT_ROW);
+
+ /* In ROWS mode, exactly the rows up to current are in frame */
+ if (frameOptions & FRAMEOPTION_ROWS)
+ {
+ winstate->frametailpos = winstate->currentpos;
+ winstate->frametail_valid = true;
+ return;
+ }
+
+ /* Else must be RANGE mode */
+ Assert(frameOptions & FRAMEOPTION_RANGE);
+
+ /* If no ORDER BY, all rows are peers with each other */
+ if (node->ordNumCols == 0)
+ {
+ spool_tuples(winstate, -1);
+ winstate->frametailpos = winstate->spooled_rows - 1;
+ winstate->frametail_valid = true;
+ return;
+ }
+
+ /*
+ * Else we have to search for the first non-peer of the current row.
+ * We assume the current value of frametailpos is a lower bound on the
+ * possible frame tail location, ie, frame tail never goes backward, and
+ * that currentpos is also a lower bound, ie, current row is always in
+ * frame.
+ */
+ ftnext = Max(winstate->frametailpos, winstate->currentpos) + 1;
+ for (;;)
+ {
+ if (!window_gettupleslot(winobj, ftnext, slot))
+ break; /* end of partition */
+ if (!are_peers(winstate, slot, winstate->ss.ss_ScanTupleSlot))
+ break; /* not peer of current row */
+ ftnext++;
+ }
+ winstate->frametailpos = ftnext - 1;
+ winstate->frametail_valid = true;
+}
+
/* -----------------
* ExecWindowAgg
@@ -849,11 +958,14 @@ restart:
{
/* Initialize for first partition and set current row = 0 */
begin_partition(winstate);
+ /* If there are no input rows, we'll detect that and exit below */
}
else
{
/* Advance current row within partition */
winstate->currentpos++;
+ /* This might mean that the frame tail moves, too */
+ winstate->frametail_valid = false;
}
/*
@@ -887,8 +999,7 @@ restart:
ResetExprContext(econtext);
/*
- * Read the current row from the tuplestore, and save in ScanTupleSlot
- * for possible use by WinGetFuncArgCurrent or the final projection step.
+ * Read the current row from the tuplestore, and save in ScanTupleSlot.
* (We can't rely on the outerplan's output slot because we may have to
* read beyond the current row.)
*
@@ -1002,7 +1113,7 @@ ExecInitWindowAgg(WindowAgg *node, EState *estate, int eflags)
ExecInitScanTupleSlot(estate, &winstate->ss);
ExecInitResultTupleSlot(estate, &winstate->ss.ps);
winstate->first_part_slot = ExecInitExtraTupleSlot(estate);
- winstate->first_peer_slot = ExecInitExtraTupleSlot(estate);
+ winstate->agg_row_slot = ExecInitExtraTupleSlot(estate);
winstate->temp_slot_1 = ExecInitExtraTupleSlot(estate);
winstate->temp_slot_2 = ExecInitExtraTupleSlot(estate);
@@ -1031,7 +1142,7 @@ ExecInitWindowAgg(WindowAgg *node, EState *estate, int eflags)
ExecSetSlotDescriptor(winstate->first_part_slot,
winstate->ss.ss_ScanTupleSlot->tts_tupleDescriptor);
- ExecSetSlotDescriptor(winstate->first_peer_slot,
+ ExecSetSlotDescriptor(winstate->agg_row_slot,
winstate->ss.ss_ScanTupleSlot->tts_tupleDescriptor);
ExecSetSlotDescriptor(winstate->temp_slot_1,
winstate->ss.ss_ScanTupleSlot->tts_tupleDescriptor);
@@ -1081,6 +1192,10 @@ ExecInitWindowAgg(WindowAgg *node, EState *estate, int eflags)
AclResult aclresult;
int i;
+ if (wfunc->winref != node->winref) /* planner screwed up? */
+ elog(ERROR, "WindowFunc with winref %u assigned to WindowAgg with winref %u",
+ wfunc->winref, node->winref);
+
/* Look for a previous duplicate window function */
for (i = 0; i <= wfuncno; i++)
{
@@ -1183,7 +1298,7 @@ ExecEndWindowAgg(WindowAggState *node)
ExecClearTuple(node->ss.ss_ScanTupleSlot);
ExecClearTuple(node->first_part_slot);
- ExecClearTuple(node->first_peer_slot);
+ ExecClearTuple(node->agg_row_slot);
ExecClearTuple(node->temp_slot_1);
ExecClearTuple(node->temp_slot_2);
@@ -1219,7 +1334,7 @@ ExecReScanWindowAgg(WindowAggState *node, ExprContext *exprCtxt)
/* release all temp tuples, but especially first_part_slot */
ExecClearTuple(node->ss.ss_ScanTupleSlot);
ExecClearTuple(node->first_part_slot);
- ExecClearTuple(node->first_peer_slot);
+ ExecClearTuple(node->agg_row_slot);
ExecClearTuple(node->temp_slot_1);
ExecClearTuple(node->temp_slot_2);
@@ -1411,6 +1526,8 @@ GetAggInitVal(Datum textInitVal, Oid transtype)
/*
* are_peers
* compare two rows to see if they are equal according to the ORDER BY clause
+ *
+ * NB: this does not consider the window frame mode.
*/
static bool
are_peers(WindowAggState *winstate, TupleTableSlot *slot1,
@@ -1430,7 +1547,8 @@ are_peers(WindowAggState *winstate, TupleTableSlot *slot1,
/*
* window_gettupleslot
- * Fetch the pos'th tuple of the current partition into the slot
+ * Fetch the pos'th tuple of the current partition into the slot,
+ * using the winobj's read pointer
*
* Returns true if successful, false if no such row
*/
@@ -1582,6 +1700,8 @@ WinSetMarkPosition(WindowObject winobj, int64 markpos)
* WinRowsArePeers
* Compare two rows (specified by absolute position in window) to see
* if they are equal according to the ORDER BY clause.
+ *
+ * NB: this does not consider the window frame mode.
*/
bool
WinRowsArePeers(WindowObject winobj, int64 pos1, int64 pos2)
@@ -1593,7 +1713,6 @@ WinRowsArePeers(WindowObject winobj, int64 pos1, int64 pos2)
bool res;
Assert(WindowObjectIsValid(winobj));
-
winstate = winobj->winstate;
node = (WindowAgg *) winstate->ss.ps.plan;
@@ -1642,27 +1761,28 @@ WinGetFuncArgInPartition(WindowObject winobj, int argno,
int relpos, int seektype, bool set_mark,
bool *isnull, bool *isout)
{
+ WindowAggState *winstate;
ExprContext *econtext;
TupleTableSlot *slot;
bool gottuple;
int64 abs_pos;
Assert(WindowObjectIsValid(winobj));
-
- econtext = winobj->winstate->ss.ps.ps_ExprContext;
- slot = winobj->winstate->temp_slot_1;
+ winstate = winobj->winstate;
+ econtext = winstate->ss.ps.ps_ExprContext;
+ slot = winstate->temp_slot_1;
switch (seektype)
{
case WINDOW_SEEK_CURRENT:
- abs_pos = winobj->winstate->currentpos + relpos;
+ abs_pos = winstate->currentpos + relpos;
break;
case WINDOW_SEEK_HEAD:
abs_pos = relpos;
break;
case WINDOW_SEEK_TAIL:
- spool_tuples(winobj->winstate, -1);
- abs_pos = winobj->winstate->spooled_rows - 1 + relpos;
+ spool_tuples(winstate, -1);
+ abs_pos = winstate->spooled_rows - 1 + relpos;
break;
default:
elog(ERROR, "unrecognized window seek type: %d", seektype);
@@ -1670,10 +1790,7 @@ WinGetFuncArgInPartition(WindowObject winobj, int argno,
break;
}
- if (abs_pos >= 0)
- gottuple = window_gettupleslot(winobj, abs_pos, slot);
- else
- gottuple = false;
+ gottuple = window_gettupleslot(winobj, abs_pos, slot);
if (!gottuple)
{
@@ -1717,34 +1834,28 @@ WinGetFuncArgInFrame(WindowObject winobj, int argno,
int relpos, int seektype, bool set_mark,
bool *isnull, bool *isout)
{
+ WindowAggState *winstate;
ExprContext *econtext;
TupleTableSlot *slot;
bool gottuple;
int64 abs_pos;
- int64 frametailpos;
Assert(WindowObjectIsValid(winobj));
-
- /* if no ordering columns, partition and frame are the same thing */
- if (((WindowAgg *) winobj->winstate->ss.ps.plan)->ordNumCols == 0)
- return WinGetFuncArgInPartition(winobj, argno, relpos, seektype,
- set_mark, isnull, isout);
-
- econtext = winobj->winstate->ss.ps.ps_ExprContext;
- slot = winobj->winstate->temp_slot_1;
- frametailpos = winobj->winstate->frametailpos;
+ winstate = winobj->winstate;
+ econtext = winstate->ss.ps.ps_ExprContext;
+ slot = winstate->temp_slot_1;
switch (seektype)
{
case WINDOW_SEEK_CURRENT:
- abs_pos = winobj->winstate->currentpos + relpos;
+ abs_pos = winstate->currentpos + relpos;
break;
case WINDOW_SEEK_HEAD:
abs_pos = relpos;
break;
case WINDOW_SEEK_TAIL:
- /* abs_pos is calculated later */
- abs_pos = 0; /* keep compiler quiet */
+ update_frametailpos(winobj, slot);
+ abs_pos = winstate->frametailpos + relpos;
break;
default:
elog(ERROR, "unrecognized window seek type: %d", seektype);
@@ -1752,58 +1863,9 @@ WinGetFuncArgInFrame(WindowObject winobj, int argno,
break;
}
- /*
- * Seek for frame tail. If the tail position is before current,
- * always check if the tail is after the current or not.
- */
- if (frametailpos <= winobj->winstate->currentpos)
- {
- int64 add = 1;
-
- for (;;)
- {
- spool_tuples(winobj->winstate, winobj->winstate->currentpos + add);
- if (winobj->winstate->spooled_rows > winobj->winstate->currentpos + add)
- {
- /*
- * When seektype is not TAIL, we may optimize not to
- * spool unnecessary tuples. In TAIL mode, we need to search
- * until we find a row that's definitely not a peer.
- */
- if (!WinRowsArePeers(winobj, winobj->winstate->currentpos,
- winobj->winstate->currentpos + add) ||
- (seektype != WINDOW_SEEK_TAIL &&
- winobj->winstate->currentpos + add < abs_pos))
- break;
- add++;
- }
- else
- {
- /*
- * If hit the partition end, the last row is the frame tail.
- */
- break;
- }
- }
- frametailpos = winobj->winstate->currentpos + add - 1;
- winobj->winstate->frametailpos = frametailpos;
- }
-
- if (seektype == WINDOW_SEEK_TAIL)
- {
- abs_pos = frametailpos + relpos;
- }
-
- /*
- * If there is an ORDER BY (we don't support other window frame
- * specifications yet), the frame runs from first row of the partition
- * to the last peer of the current row. Otherwise the frame is the
- * whole partition.
- */
- if (abs_pos < 0 || abs_pos > frametailpos)
- gottuple = false;
- else
- gottuple = window_gettupleslot(winobj, abs_pos, slot);
+ gottuple = window_gettupleslot(winobj, abs_pos, slot);
+ if (gottuple)
+ gottuple = row_is_in_frame(winstate, abs_pos, slot);
if (!gottuple)
{