Centralize executor's opening/closing of Relations for rangetable entries.
authorTom Lane <tgl@sss.pgh.pa.us>
Thu, 4 Oct 2018 18:03:37 +0000 (14:03 -0400)
committerTom Lane <tgl@sss.pgh.pa.us>
Thu, 4 Oct 2018 18:03:42 +0000 (14:03 -0400)
Create an array estate->es_relations[] paralleling the es_range_table,
and store references to Relations (relcache entries) there, so that any
given RT entry is opened and closed just once per executor run.  Scan
nodes typically still call ExecOpenScanRelation, but ExecCloseScanRelation
is no more; relation closing is now done centrally in ExecEndPlan.

This is slightly more complex than one would expect because of the
interactions with relcache references held in ResultRelInfo nodes.
The general convention is now that ResultRelInfo->ri_RelationDesc does
not represent a separate relcache reference and so does not need to be
explicitly closed; but there is an exception for ResultRelInfos in the
es_trig_target_relations list, which are manufactured by
ExecGetTriggerResultRel and have to be cleaned up by
ExecCleanUpTriggerState.  (That much was true all along, but these
ResultRelInfos are now more different from others than they used to be.)

To allow the partition pruning logic to make use of es_relations[] rather
than having its own relcache references, adjust PartitionedRelPruneInfo
to store an RT index rather than a relation OID.

Amit Langote, reviewed by David Rowley and Jesper Pedersen,
some mods by me

Discussion: https://postgr.es/m/468c85d9-540e-66a2-1dde-fec2b741e688@lab.ntt.co.jp

27 files changed:
contrib/postgres_fdw/postgres_fdw.c
src/backend/catalog/indexing.c
src/backend/commands/copy.c
src/backend/executor/execMain.c
src/backend/executor/execPartition.c
src/backend/executor/execUtils.c
src/backend/executor/nodeAppend.c
src/backend/executor/nodeBitmapHeapscan.c
src/backend/executor/nodeCustom.c
src/backend/executor/nodeForeignscan.c
src/backend/executor/nodeIndexonlyscan.c
src/backend/executor/nodeIndexscan.c
src/backend/executor/nodeMergeAppend.c
src/backend/executor/nodeSamplescan.c
src/backend/executor/nodeSeqscan.c
src/backend/executor/nodeTidscan.c
src/backend/nodes/copyfuncs.c
src/backend/nodes/outfuncs.c
src/backend/nodes/readfuncs.c
src/backend/optimizer/plan/setrefs.c
src/backend/partitioning/partprune.c
src/backend/replication/logical/worker.c
src/include/executor/execPartition.h
src/include/executor/executor.h
src/include/nodes/execnodes.h
src/include/nodes/plannodes.h
src/include/partitioning/partprune.h

index 6cbba97c22390297dba25135e7d649aa28550a48..c02287a55cade1b2635a983ca78ed3cbf2e6225b 100644 (file)
@@ -2546,10 +2546,6 @@ postgresEndDirectModify(ForeignScanState *node)
    ReleaseConnection(dmstate->conn);
    dmstate->conn = NULL;
 
-   /* close the target relation. */
-   if (dmstate->resultRel)
-       ExecCloseScanRelation(dmstate->resultRel);
-
    /* MemoryContext will be deleted automatically. */
 }
 
index 4674beaf3e77413d8179a64764e5fe3e2f2e1c57..daf7ae2eb2b07cc4648862fce0fd8228ecdff3d8 100644 (file)
@@ -42,7 +42,7 @@ CatalogOpenIndexes(Relation heapRel)
    ResultRelInfo *resultRelInfo;
 
    resultRelInfo = makeNode(ResultRelInfo);
-   resultRelInfo->ri_RangeTableIndex = 1;  /* dummy */
+   resultRelInfo->ri_RangeTableIndex = 0;  /* dummy */
    resultRelInfo->ri_RelationDesc = heapRel;
    resultRelInfo->ri_TrigDesc = NULL;  /* we don't fire triggers */
 
index 32706fad90f6459f55b39542a9a66dfa8f814094..df98e4ac626dea9d8786b2302ca0b590703385d8 100644 (file)
@@ -2471,7 +2471,7 @@ CopyFrom(CopyState cstate)
    resultRelInfo = makeNode(ResultRelInfo);
    InitResultRelInfo(resultRelInfo,
                      cstate->rel,
-                     1,        /* dummy rangetable index */
+                     1,        /* must match rel's position in range_table */
                      NULL,
                      0);
    target_resultRelInfo = resultRelInfo;
@@ -2485,6 +2485,8 @@ CopyFrom(CopyState cstate)
    estate->es_num_result_relations = 1;
    estate->es_result_relation_info = resultRelInfo;
    estate->es_range_table = cstate->range_table;
+   estate->es_relations = (Relation *) palloc0(list_length(cstate->range_table) *
+                                               sizeof(Relation));
 
    /* Set up a tuple slot too */
    myslot = ExecInitExtraTupleSlot(estate, tupDesc);
index 9569d2fa42300f85d6b07a0f1c66f9e3381ff66b..8cbd75ed7aa36a3610ac6c44d78c69f12f9614f5 100644 (file)
@@ -824,6 +824,15 @@ InitPlan(QueryDesc *queryDesc, int eflags)
     * initialize the node's execution state
     */
    estate->es_range_table = rangeTable;
+
+   /*
+    * Allocate an array to store an open Relation corresponding to each
+    * rangeTable item, and initialize entries to NULL.  Relations are opened
+    * and stored here as needed.
+    */
+   estate->es_relations = (Relation *) palloc0(list_length(rangeTable) *
+                                               sizeof(Relation));
+
    estate->es_plannedstmt = plannedstmt;
 
    /*
@@ -845,13 +854,10 @@ InitPlan(QueryDesc *queryDesc, int eflags)
        foreach(l, resultRelations)
        {
            Index       resultRelationIndex = lfirst_int(l);
-           Oid         resultRelationOid;
            Relation    resultRelation;
 
-           resultRelationOid = getrelid(resultRelationIndex, rangeTable);
-           resultRelation = heap_open(resultRelationOid, NoLock);
-           Assert(CheckRelationLockedByMe(resultRelation, RowExclusiveLock, true));
-
+           resultRelation = ExecGetRangeTableRelation(estate,
+                                                      resultRelationIndex);
            InitResultRelInfo(resultRelInfo,
                              resultRelation,
                              resultRelationIndex,
@@ -886,15 +892,13 @@ InitPlan(QueryDesc *queryDesc, int eflags)
            foreach(l, plannedstmt->rootResultRelations)
            {
                Index       resultRelIndex = lfirst_int(l);
-               Oid         resultRelOid;
                Relation    resultRelDesc;
 
-               resultRelOid = getrelid(resultRelIndex, rangeTable);
-               resultRelDesc = heap_open(resultRelOid, NoLock);
-               Assert(CheckRelationLockedByMe(resultRelDesc, RowExclusiveLock, true));
+               resultRelDesc = ExecGetRangeTableRelation(estate,
+                                                         resultRelIndex);
                InitResultRelInfo(resultRelInfo,
                                  resultRelDesc,
-                                 lfirst_int(l),
+                                 resultRelIndex,
                                  NULL,
                                  estate->es_instrument);
                resultRelInfo++;
@@ -967,10 +971,7 @@ InitPlan(QueryDesc *queryDesc, int eflags)
            case ROW_MARK_SHARE:
            case ROW_MARK_KEYSHARE:
            case ROW_MARK_REFERENCE:
-               relation = heap_open(relid, NoLock);
-               Assert(CheckRelationLockedByMe(relation,
-                                              rt_fetch(rc->rti, rangeTable)->rellockmode,
-                                              true));
+               relation = ExecGetRangeTableRelation(estate, rc->rti);
                break;
            case ROW_MARK_COPY:
                /* no physical table access is required */
@@ -1483,8 +1484,19 @@ ExecCleanUpTriggerState(EState *estate)
    {
        ResultRelInfo *resultRelInfo = (ResultRelInfo *) lfirst(l);
 
-       /* Close indices and then the relation itself */
-       ExecCloseIndices(resultRelInfo);
+       /*
+        * Assert this is a "dummy" ResultRelInfo, see above.  Otherwise we
+        * might be issuing a duplicate close against a Relation opened by
+        * ExecGetRangeTableRelation.
+        */
+       Assert(resultRelInfo->ri_RangeTableIndex == 0);
+
+       /*
+        * Since ExecGetTriggerResultRel doesn't call ExecOpenIndices for
+        * these rels, we needn't call ExecCloseIndices either.
+        */
+       Assert(resultRelInfo->ri_NumIndices == 0);
+
        heap_close(resultRelInfo->ri_RelationDesc, NoLock);
    }
 }
@@ -1607,6 +1619,7 @@ static void
 ExecEndPlan(PlanState *planstate, EState *estate)
 {
    ResultRelInfo *resultRelInfo;
+   int         num_relations;
    int         i;
    ListCell   *l;
 
@@ -1634,39 +1647,29 @@ ExecEndPlan(PlanState *planstate, EState *estate)
    ExecResetTupleTable(estate->es_tupleTable, false);
 
    /*
-    * close the result relation(s) if any, but hold locks until xact commit.
+    * close indexes of result relation(s) if any.  (Rels themselves get
+    * closed next.)
     */
    resultRelInfo = estate->es_result_relations;
    for (i = estate->es_num_result_relations; i > 0; i--)
    {
-       /* Close indices and then the relation itself */
        ExecCloseIndices(resultRelInfo);
-       heap_close(resultRelInfo->ri_RelationDesc, NoLock);
        resultRelInfo++;
    }
 
-   /* Close the root target relation(s). */
-   resultRelInfo = estate->es_root_result_relations;
-   for (i = estate->es_num_root_result_relations; i > 0; i--)
+   /*
+    * close whatever rangetable Relations have been opened.  We did not
+    * acquire locks in ExecGetRangeTableRelation, so don't release 'em here.
+    */
+   num_relations = list_length(estate->es_range_table);
+   for (i = 0; i < num_relations; i++)
    {
-       heap_close(resultRelInfo->ri_RelationDesc, NoLock);
-       resultRelInfo++;
+       if (estate->es_relations[i])
+           heap_close(estate->es_relations[i], NoLock);
    }
 
    /* likewise close any trigger target relations */
    ExecCleanUpTriggerState(estate);
-
-   /*
-    * close any relations selected FOR [KEY] UPDATE/SHARE, again keeping
-    * locks
-    */
-   foreach(l, estate->es_rowMarks)
-   {
-       ExecRowMark *erm = (ExecRowMark *) lfirst(l);
-
-       if (erm->relation)
-           heap_close(erm->relation, NoLock);
-   }
 }
 
 /* ----------------------------------------------------------------
@@ -3161,6 +3164,7 @@ EvalPlanQualStart(EPQState *epqstate, EState *parentestate, Plan *planTree)
    estate->es_snapshot = parentestate->es_snapshot;
    estate->es_crosscheck_snapshot = parentestate->es_crosscheck_snapshot;
    estate->es_range_table = parentestate->es_range_table;
+   estate->es_relations = parentestate->es_relations;
    estate->es_plannedstmt = parentestate->es_plannedstmt;
    estate->es_junkFilter = parentestate->es_junkFilter;
    estate->es_output_cid = parentestate->es_output_cid;
index 832c79b41eaf217a36d06e10e7f39cc3614181e5..c7dc9bf1362334b4e60859ada09f6d6c44418d33 100644 (file)
@@ -219,7 +219,7 @@ ExecFindPartition(ResultRelInfo *resultRelInfo, PartitionDispatch *pd,
    ExprContext *ecxt = GetPerTupleExprContext(estate);
    TupleTableSlot *ecxt_scantuple_old = ecxt->ecxt_scantuple;
    TupleTableSlot *myslot = NULL;
-   MemoryContext   oldcxt;
+   MemoryContext oldcxt;
 
    /* use per-tuple context here to avoid leaking memory */
    oldcxt = MemoryContextSwitchTo(GetPerTupleMemoryContext(estate));
@@ -1389,9 +1389,6 @@ adjust_partition_tlist(List *tlist, TupleConversionMap *map)
  *     functions.  Details stored include how to map the partition index
  *     returned by the partition pruning code into subplan indexes.
  *
- * ExecDestroyPartitionPruneState:
- *     Deletes a PartitionPruneState. Must be called during executor shutdown.
- *
  * ExecFindInitialMatchingSubPlans:
  *     Returns indexes of matching subplans.  Partition pruning is attempted
  *     without any evaluation of expressions containing PARAM_EXEC Params.
@@ -1433,6 +1430,7 @@ PartitionPruneState *
 ExecCreatePartitionPruneState(PlanState *planstate,
                              PartitionPruneInfo *partitionpruneinfo)
 {
+   EState     *estate = planstate->state;
    PartitionPruneState *prunestate;
    int         n_part_hierarchies;
    ListCell   *lc;
@@ -1487,6 +1485,7 @@ ExecCreatePartitionPruneState(PlanState *planstate,
            PartitionedRelPruneInfo *pinfo = lfirst_node(PartitionedRelPruneInfo, lc2);
            PartitionedRelPruningData *pprune = &prunedata->partrelprunedata[j];
            PartitionPruneContext *context = &pprune->context;
+           Relation    partrel;
            PartitionDesc partdesc;
            PartitionKey partkey;
            int         partnatts;
@@ -1509,16 +1508,15 @@ ExecCreatePartitionPruneState(PlanState *planstate,
            pprune->present_parts = bms_copy(pinfo->present_parts);
 
            /*
-            * We need to hold a pin on the partitioned table's relcache entry
-            * so that we can rely on its copies of the table's partition key
-            * and partition descriptor.  We need not get a lock though; one
-            * should have been acquired already by InitPlan or
-            * ExecLockNonLeafAppendTables.
+            * We can rely on the copies of the partitioned table's partition
+            * key and partition descriptor appearing in its relcache entry,
+            * because that entry will be held open and locked for the
+            * duration of this executor run.
             */
-           context->partrel = relation_open(pinfo->reloid, NoLock);
+           partrel = ExecGetRangeTableRelation(estate, pinfo->rtindex);
+           partkey = RelationGetPartitionKey(partrel);
+           partdesc = RelationGetPartitionDesc(partrel);
 
-           partkey = RelationGetPartitionKey(context->partrel);
-           partdesc = RelationGetPartitionDesc(context->partrel);
            n_steps = list_length(pinfo->pruning_steps);
 
            context->strategy = partkey->strategy;
@@ -1595,30 +1593,6 @@ ExecCreatePartitionPruneState(PlanState *planstate,
    return prunestate;
 }
 
-/*
- * ExecDestroyPartitionPruneState
- *     Release resources at plan shutdown.
- *
- * We don't bother to free any memory here, since the whole executor context
- * will be going away shortly.  We do need to release our relcache pins.
- */
-void
-ExecDestroyPartitionPruneState(PartitionPruneState *prunestate)
-{
-   PartitionPruningData **partprunedata = prunestate->partprunedata;
-   int         i;
-
-   for (i = 0; i < prunestate->num_partprunedata; i++)
-   {
-       PartitionPruningData *prunedata = partprunedata[i];
-       PartitionedRelPruningData *pprune = prunedata->partrelprunedata;
-       int         j;
-
-       for (j = 0; j < prunedata->num_partrelprunedata; j++)
-           relation_close(pprune[j].context.partrel, NoLock);
-   }
-}
-
 /*
  * ExecFindInitialMatchingSubPlans
  *     Identify the set of subplans that cannot be eliminated by initial
index ba93b401046fa3f90e08a638004783a2b094f291..a89ef2a482f93aca7db50b7075019fd0b4581e7e 100644 (file)
@@ -25,7 +25,8 @@
  *     etc
  *
  *     ExecOpenScanRelation    Common code for scan node init routines.
- *     ExecCloseScanRelation
+ *
+ *     ExecGetRangeTableRelation       Fetch Relation for a rangetable entry.
  *
  *     executor_errposition    Report syntactic position of an error.
  *
@@ -107,6 +108,7 @@ CreateExecutorState(void)
    estate->es_snapshot = InvalidSnapshot;  /* caller must initialize this */
    estate->es_crosscheck_snapshot = InvalidSnapshot;   /* no crosscheck */
    estate->es_range_table = NIL;
+   estate->es_relations = NULL;
    estate->es_plannedstmt = NULL;
 
    estate->es_junkFilter = NULL;
@@ -648,15 +650,9 @@ Relation
 ExecOpenScanRelation(EState *estate, Index scanrelid, int eflags)
 {
    Relation    rel;
-   Oid         reloid;
 
-   /* Open the relation and verify lock was obtained upstream */
-   reloid = getrelid(scanrelid, estate->es_range_table);
-   rel = heap_open(reloid, NoLock);
-   Assert(IsParallelWorker() ||
-          CheckRelationLockedByMe(rel,
-                                  rt_fetch(scanrelid, estate->es_range_table)->rellockmode,
-                                  true));
+   /* Open the relation. */
+   rel = ExecGetRangeTableRelation(estate, scanrelid);
 
    /*
     * Complain if we're attempting a scan of an unscannable relation, except
@@ -674,24 +670,40 @@ ExecOpenScanRelation(EState *estate, Index scanrelid, int eflags)
    return rel;
 }
 
-/* ----------------------------------------------------------------
- *     ExecCloseScanRelation
- *
- *     Close the heap relation scanned by a base-level scan plan node.
- *     This should be called during the node's ExecEnd routine.
- *
- * Currently, we do not release the lock acquired by ExecOpenScanRelation.
- * This lock should be held till end of transaction.  (There is a faction
- * that considers this too much locking, however.)
+/*
+ * ExecGetRangeTableRelation
+ *     Open the Relation for a range table entry, if not already done
  *
- * If we did want to release the lock, we'd have to repeat the logic in
- * ExecOpenScanRelation in order to figure out what to release.
- * ----------------------------------------------------------------
+ * The Relations will be closed again in ExecEndPlan().
  */
-void
-ExecCloseScanRelation(Relation scanrel)
+Relation
+ExecGetRangeTableRelation(EState *estate, Index rti)
 {
-   heap_close(scanrel, NoLock);
+   Relation    rel;
+
+   Assert(rti > 0 && rti <= list_length(estate->es_range_table));
+
+   rel = estate->es_relations[rti - 1];
+   if (rel == NULL)
+   {
+       /* First time through, so open the relation */
+       RangeTblEntry *rte = rt_fetch(rti, estate->es_range_table);
+
+       Assert(rte->rtekind == RTE_RELATION);
+
+       rel = estate->es_relations[rti - 1] = heap_open(rte->relid, NoLock);
+
+       /*
+        * Verify that appropriate lock was obtained before execution.
+        *
+        * In the case of parallel query, only the leader would've obtained
+        * the lock (that needs to be fixed, though).
+        */
+       Assert(IsParallelWorker() ||
+              CheckRelationLockedByMe(rel, rte->rellockmode, false));
+   }
+
+   return rel;
 }
 
 /*
index f08dfcbcf01510364ba9ec2c2d312f525dddef98..d44befd44e35f7a49bd32794c482f7236757bd04 100644 (file)
@@ -335,12 +335,6 @@ ExecEndAppend(AppendState *node)
     */
    for (i = 0; i < nplans; i++)
        ExecEndNode(appendplans[i]);
-
-   /*
-    * release any resources associated with run-time pruning
-    */
-   if (node->as_prune_state)
-       ExecDestroyPartitionPruneState(node->as_prune_state);
 }
 
 void
index baffae27e3e7fc6a87b8450e361563c9a641c742..5307cd1b87033c2f7fe516ae107727ece21e5caf 100644 (file)
@@ -785,13 +785,11 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 void
 ExecEndBitmapHeapScan(BitmapHeapScanState *node)
 {
-   Relation    relation;
    HeapScanDesc scanDesc;
 
    /*
     * extract information from the node
     */
-   relation = node->ss.ss_currentRelation;
    scanDesc = node->ss.ss_currentScanDesc;
 
    /*
@@ -832,11 +830,6 @@ ExecEndBitmapHeapScan(BitmapHeapScanState *node)
     * close heap scan
     */
    heap_endscan(scanDesc);
-
-   /*
-    * close the heap relation.
-    */
-   ExecCloseScanRelation(relation);
 }
 
 /* ----------------------------------------------------------------
index b816e0b31db6930cfbd33b11a3df392866d89108..9a33eda688797a9c004eb6bb765a0d3782fdb7d6 100644 (file)
@@ -126,10 +126,6 @@ ExecEndCustomScan(CustomScanState *node)
    /* Clean out the tuple table */
    ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
    ExecClearTuple(node->ss.ss_ScanTupleSlot);
-
-   /* Close the heap relation */
-   if (node->ss.ss_currentRelation)
-       ExecCloseScanRelation(node->ss.ss_currentRelation);
 }
 
 void
index a2a28b7ec261957b20c7fbf188e004a22da369dd..cf7df72d8c2e93b11cc2d593dd436dd094b8ee43 100644 (file)
@@ -258,10 +258,6 @@ ExecEndForeignScan(ForeignScanState *node)
    /* clean out the tuple table */
    ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
    ExecClearTuple(node->ss.ss_ScanTupleSlot);
-
-   /* close the relation. */
-   if (node->ss.ss_currentRelation)
-       ExecCloseScanRelation(node->ss.ss_currentRelation);
 }
 
 /* ----------------------------------------------------------------
index 4b6d531810c6f714934a1dcf3eba5da7b16fb0ce..1b530cea405cc4546bf46bace66385fcc77e6243 100644 (file)
@@ -373,14 +373,12 @@ ExecEndIndexOnlyScan(IndexOnlyScanState *node)
 {
    Relation    indexRelationDesc;
    IndexScanDesc indexScanDesc;
-   Relation    relation;
 
    /*
     * extract information from the node
     */
    indexRelationDesc = node->ioss_RelationDesc;
    indexScanDesc = node->ioss_ScanDesc;
-   relation = node->ss.ss_currentRelation;
 
    /* Release VM buffer pin, if any. */
    if (node->ioss_VMBuffer != InvalidBuffer)
@@ -411,11 +409,6 @@ ExecEndIndexOnlyScan(IndexOnlyScanState *node)
        index_endscan(indexScanDesc);
    if (indexRelationDesc)
        index_close(indexRelationDesc, NoLock);
-
-   /*
-    * close the heap relation.
-    */
-   ExecCloseScanRelation(relation);
 }
 
 /* ----------------------------------------------------------------
index 6285a2114e8f29fae9c7b1a848802af7803a7819..d74499fd7c1f762c48b5b0d51cfd4f0caf12edf1 100644 (file)
@@ -140,7 +140,7 @@ IndexNext(IndexScanState *node)
         * Note: we pass 'false' because tuples returned by amgetnext are
         * pointers onto disk pages and must not be pfree()'d.
         */
-       ExecStoreBufferHeapTuple(tuple, /* tuple to store */
+       ExecStoreBufferHeapTuple(tuple, /* tuple to store */
                                 slot,  /* slot to store in */
                                 scandesc->xs_cbuf);    /* buffer containing
                                                         * tuple */
@@ -285,7 +285,7 @@ next_indextuple:
        /*
         * Store the scanned tuple in the scan tuple slot of the scan state.
         */
-       ExecStoreBufferHeapTuple(tuple, /* tuple to store */
+       ExecStoreBufferHeapTuple(tuple, /* tuple to store */
                                 slot,  /* slot to store in */
                                 scandesc->xs_cbuf);    /* buffer containing
                                                         * tuple */
@@ -802,14 +802,12 @@ ExecEndIndexScan(IndexScanState *node)
 {
    Relation    indexRelationDesc;
    IndexScanDesc indexScanDesc;
-   Relation    relation;
 
    /*
     * extract information from the node
     */
    indexRelationDesc = node->iss_RelationDesc;
    indexScanDesc = node->iss_ScanDesc;
-   relation = node->ss.ss_currentRelation;
 
    /*
     * Free the exprcontext(s) ... now dead code, see ExecFreeExprContext
@@ -833,11 +831,6 @@ ExecEndIndexScan(IndexScanState *node)
        index_endscan(indexScanDesc);
    if (indexRelationDesc)
        index_close(indexRelationDesc, NoLock);
-
-   /*
-    * close the heap relation.
-    */
-   ExecCloseScanRelation(relation);
 }
 
 /* ----------------------------------------------------------------
index 9a72d3a0ace6a62eb2116f191f25ff29938588c1..6daf60a454d57f6fe8c35d202c1cd9ca2aeb57ab 100644 (file)
@@ -369,12 +369,6 @@ ExecEndMergeAppend(MergeAppendState *node)
     */
    for (i = 0; i < nplans; i++)
        ExecEndNode(mergeplans[i]);
-
-   /*
-    * release any resources associated with run-time pruning
-    */
-   if (node->ms_prune_state)
-       ExecDestroyPartitionPruneState(node->ms_prune_state);
 }
 
 void
index 99528be84a685573a0638297801a6d428ddf2dc0..70ae1bc7e462c9027856c3ea0b169ef0a121b466 100644 (file)
@@ -63,9 +63,9 @@ SampleNext(SampleScanState *node)
    slot = node->ss.ss_ScanTupleSlot;
 
    if (tuple)
-       ExecStoreBufferHeapTuple(tuple, /* tuple to store */
+       ExecStoreBufferHeapTuple(tuple, /* tuple to store */
                                 slot,  /* slot to store in */
-                                node->ss.ss_currentScanDesc->rs_cbuf); /* tuple's buffer */
+                                node->ss.ss_currentScanDesc->rs_cbuf); /* tuple's buffer */
    else
        ExecClearTuple(slot);
 
@@ -222,11 +222,6 @@ ExecEndSampleScan(SampleScanState *node)
     */
    if (node->ss.ss_currentScanDesc)
        heap_endscan(node->ss.ss_currentScanDesc);
-
-   /*
-    * close the heap relation.
-    */
-   ExecCloseScanRelation(node->ss.ss_currentRelation);
 }
 
 /* ----------------------------------------------------------------
index cd53491be0a6a71530e4069d2e3eee5dbd84742b..5dede816c6a8275cabecadbabb6755d2ee5409ac 100644 (file)
@@ -88,7 +88,7 @@ SeqNext(SeqScanState *node)
     * refcount will not be dropped until the tuple table slot is cleared.
     */
    if (tuple)
-       ExecStoreBufferHeapTuple(tuple, /* tuple to store */
+       ExecStoreBufferHeapTuple(tuple, /* tuple to store */
                                 slot,  /* slot to store in */
                                 scandesc->rs_cbuf);    /* buffer associated
                                                         * with this tuple */
@@ -201,13 +201,11 @@ ExecInitSeqScan(SeqScan *node, EState *estate, int eflags)
 void
 ExecEndSeqScan(SeqScanState *node)
 {
-   Relation    relation;
    HeapScanDesc scanDesc;
 
    /*
     * get information from node
     */
-   relation = node->ss.ss_currentRelation;
    scanDesc = node->ss.ss_currentScanDesc;
 
    /*
@@ -226,11 +224,6 @@ ExecEndSeqScan(SeqScanState *node)
     */
    if (scanDesc != NULL)
        heap_endscan(scanDesc);
-
-   /*
-    * close the heap relation.
-    */
-   ExecCloseScanRelation(relation);
 }
 
 /* ----------------------------------------------------------------
index 0cb1946a3e3e4d6a327860714e7bda0ce4f9c80f..593185f726ba65a28cbbe9650f5a53eba4464eff 100644 (file)
@@ -380,7 +380,7 @@ TidNext(TidScanState *node)
             * Store the scanned tuple in the scan tuple slot of the scan
             * state.  Eventually we will only do this and not return a tuple.
             */
-           ExecStoreBufferHeapTuple(tuple, /* tuple to store */
+           ExecStoreBufferHeapTuple(tuple, /* tuple to store */
                                     slot,  /* slot to store in */
                                     buffer);   /* buffer associated with
                                                 * tuple */
@@ -489,11 +489,6 @@ ExecEndTidScan(TidScanState *node)
     */
    ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
    ExecClearTuple(node->ss.ss_ScanTupleSlot);
-
-   /*
-    * close the heap relation.
-    */
-   ExecCloseScanRelation(node->ss.ss_currentRelation);
 }
 
 /* ----------------------------------------------------------------
index 925cb8f3800a7dcc357c11df69c6872da00d1e0c..3b690b55b38e73b25b0249693b2b13e8075b6afa 100644 (file)
@@ -1193,7 +1193,7 @@ _copyPartitionedRelPruneInfo(const PartitionedRelPruneInfo *from)
 {
    PartitionedRelPruneInfo *newnode = makeNode(PartitionedRelPruneInfo);
 
-   COPY_SCALAR_FIELD(reloid);
+   COPY_SCALAR_FIELD(rtindex);
    COPY_NODE_FIELD(pruning_steps);
    COPY_BITMAPSET_FIELD(present_parts);
    COPY_SCALAR_FIELD(nparts);
index 22dbae15d3bd2ce01bfde7bd8497076b09c02081..863d29cc57db76083a90165d092e380238dbfcb1 100644 (file)
@@ -1032,7 +1032,7 @@ _outPartitionedRelPruneInfo(StringInfo str, const PartitionedRelPruneInfo *node)
 
    WRITE_NODE_TYPE("PARTITIONEDRELPRUNEINFO");
 
-   WRITE_OID_FIELD(reloid);
+   WRITE_UINT_FIELD(rtindex);
    WRITE_NODE_FIELD(pruning_steps);
    WRITE_BITMAPSET_FIELD(present_parts);
    WRITE_INT_FIELD(nparts);
index ce556580a5f06774d6304c646ed38871b048f90d..73ffa9714c60f72fc273af28712327f5b741ee5f 100644 (file)
@@ -2376,7 +2376,7 @@ _readPartitionedRelPruneInfo(void)
 {
    READ_LOCALS(PartitionedRelPruneInfo);
 
-   READ_OID_FIELD(reloid);
+   READ_UINT_FIELD(rtindex);
    READ_NODE_FIELD(pruning_steps);
    READ_BITMAPSET_FIELD(present_parts);
    READ_INT_FIELD(nparts);
index f66f39d8c685e5306eb9def2b9554e374ccda701..54293e25aff8ffa01bebbe97089923f8be2543b3 100644 (file)
@@ -925,6 +925,21 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
                                              (Plan *) lfirst(l),
                                              rtoffset);
                }
+               if (splan->part_prune_info)
+               {
+                   foreach(l, splan->part_prune_info->prune_infos)
+                   {
+                       List       *prune_infos = lfirst(l);
+                       ListCell   *l2;
+
+                       foreach(l2, prune_infos)
+                       {
+                           PartitionedRelPruneInfo *pinfo = lfirst(l2);
+
+                           pinfo->rtindex += rtoffset;
+                       }
+                   }
+               }
            }
            break;
        case T_MergeAppend:
@@ -947,6 +962,21 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
                                              (Plan *) lfirst(l),
                                              rtoffset);
                }
+               if (splan->part_prune_info)
+               {
+                   foreach(l, splan->part_prune_info->prune_infos)
+                   {
+                       List       *prune_infos = lfirst(l);
+                       ListCell   *l2;
+
+                       foreach(l2, prune_infos)
+                       {
+                           PartitionedRelPruneInfo *pinfo = lfirst(l2);
+
+                           pinfo->rtindex += rtoffset;
+                       }
+                   }
+               }
            }
            break;
        case T_RecursiveUnion:
index e1ce8b4ddcb544fb1d1fa4c0bd83afdbbff91da0..d6ca03de4a32acae94495ced4125bdb4ff3669af 100644 (file)
@@ -357,7 +357,6 @@ make_partitionedrel_pruneinfo(PlannerInfo *root, RelOptInfo *parentrel,
        Index       rti = lfirst_int(lc);
        RelOptInfo *subpart = find_base_rel(root, rti);
        PartitionedRelPruneInfo *pinfo;
-       RangeTblEntry *rte;
        Bitmapset  *present_parts;
        int         nparts = subpart->nparts;
        int         partnatts = subpart->part_scheme->partnatts;
@@ -459,10 +458,8 @@ make_partitionedrel_pruneinfo(PlannerInfo *root, RelOptInfo *parentrel,
                present_parts = bms_add_member(present_parts, i);
        }
 
-       rte = root->simple_rte_array[subpart->relid];
-
        pinfo = makeNode(PartitionedRelPruneInfo);
-       pinfo->reloid = rte->relid;
+       pinfo->rtindex = rti;
        pinfo->pruning_steps = pruning_steps;
        pinfo->present_parts = present_parts;
        pinfo->nparts = nparts;
@@ -587,7 +584,6 @@ prune_append_rel_partitions(RelOptInfo *rel)
    context.ppccontext = CurrentMemoryContext;
 
    /* These are not valid when being called from the planner */
-   context.partrel = NULL;
    context.planstate = NULL;
    context.exprstates = NULL;
    context.exprhasexecparam = NULL;
index e247539d7b158d5a05f0c7d886038c8f59624ec1..c3443c5a2ce031715cde78552cc706838a4b0cdb 100644 (file)
@@ -201,6 +201,7 @@ create_estate_for_relation(LogicalRepRelMapEntry *rel)
    rte->relkind = rel->localrel->rd_rel->relkind;
    rte->rellockmode = AccessShareLock;
    estate->es_range_table = list_make1(rte);
+   estate->es_relations = (Relation *) palloc0(1 * sizeof(Relation));
 
    resultRelInfo = makeNode(ResultRelInfo);
    InitResultRelInfo(resultRelInfo, rel->localrel, 1, NULL, 0);
index 8b4a9ca04479bc0e247af6a87b4ce1715ac8f146..3e08104ea4297e0b7ece1d9822d4e7fbe0f9ed74 100644 (file)
@@ -195,7 +195,6 @@ extern void ExecCleanupTupleRouting(ModifyTableState *mtstate,
                        PartitionTupleRouting *proute);
 extern PartitionPruneState *ExecCreatePartitionPruneState(PlanState *planstate,
                              PartitionPruneInfo *partitionpruneinfo);
-extern void ExecDestroyPartitionPruneState(PartitionPruneState *prunestate);
 extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruneState *prunestate);
 extern Bitmapset *ExecFindInitialMatchingSubPlans(PartitionPruneState *prunestate,
                                int nsubplans);
index f82b51667f453f56278f7c8f9f8ea181927a97e8..83ce989e3a34c08dc869fa08f25f4863612fe198 100644 (file)
@@ -513,7 +513,8 @@ extern void ExecCreateScanSlotFromOuterPlan(EState *estate, ScanState *scanstate
 extern bool ExecRelationIsTargetRelation(EState *estate, Index scanrelid);
 
 extern Relation ExecOpenScanRelation(EState *estate, Index scanrelid, int eflags);
-extern void ExecCloseScanRelation(Relation scanrel);
+
+extern Relation ExecGetRangeTableRelation(EState *estate, Index rti);
 
 extern int executor_errposition(EState *estate, int location);
 
index 020ecdcd404139afa37989cf1f3052aec36121a8..35646231a4c8207e562642676646e4f41124cda0 100644 (file)
@@ -386,12 +386,19 @@ typedef struct OnConflictSetState
  * Whenever we update an existing relation, we have to update indexes on the
  * relation, and perhaps also fire triggers.  ResultRelInfo holds all the
  * information needed about a result relation, including indexes.
+ *
+ * Normally, a ResultRelInfo refers to a table that is in the query's
+ * range table; then ri_RangeTableIndex is the RT index and ri_RelationDesc
+ * is just a copy of the relevant es_relations[] entry.  But sometimes,
+ * in ResultRelInfos used only for triggers, ri_RangeTableIndex is zero
+ * and ri_RelationDesc is a separately-opened relcache pointer that needs
+ * to be separately closed.  See ExecGetTriggerResultRel.
  */
 typedef struct ResultRelInfo
 {
    NodeTag     type;
 
-   /* result relation's range table index */
+   /* result relation's range table index, or 0 if not in range table */
    Index       ri_RangeTableIndex;
 
    /* relation descriptor for result relation */
@@ -479,6 +486,8 @@ typedef struct EState
    Snapshot    es_snapshot;    /* time qual to use */
    Snapshot    es_crosscheck_snapshot; /* crosscheck time qual for RI */
    List       *es_range_table; /* List of RangeTblEntry */
+   Relation   *es_relations;   /* Array of per-es_range_table-entry Relation
+                                * pointers, or NULL if not yet opened */
    PlannedStmt *es_plannedstmt;    /* link to top of plan tree */
    const char *es_sourceText;  /* Source text from QueryDesc */
 
index 7c2abbd03a681d680775ec1f335c0927e36bb903..8b3743705769224cafa51aca55d672cc2310642c 100644 (file)
@@ -1105,7 +1105,7 @@ typedef struct PartitionPruneInfo
 typedef struct PartitionedRelPruneInfo
 {
    NodeTag     type;
-   Oid         reloid;         /* OID of partition rel for this level */
+   Index       rtindex;        /* RT index of partition rel for this level */
    List       *pruning_steps;  /* List of PartitionPruneStep, see below */
    Bitmapset  *present_parts;  /* Indexes of all partitions which subplans or
                                 * subparts are present for. */
index b95c346bab06b45030819dd394bbdf1875a3a7cc..e07aaaf798b27cc0ecf9e4514ed14dd3d89ae19d 100644 (file)
@@ -23,8 +23,6 @@
  *     Stores information needed at runtime for pruning computations
  *     related to a single partitioned table.
  *
- * partrel         Relcache pointer for the partitioned table,
- *                 if we have it open (else NULL).
  * strategy            Partition strategy, e.g. LIST, RANGE, HASH.
  * partnatts       Number of columns in the partition key.
  * nparts          Number of partitions in this partitioned table.
@@ -50,7 +48,6 @@
  */
 typedef struct PartitionPruneContext
 {
-   Relation    partrel;
    char        strategy;
    int         partnatts;
    int         nparts;