Re: why partition pruning doesn't work?

Поиск
Список
Период
Сортировка
От Tom Lane
Тема Re: why partition pruning doesn't work?
Дата
Msg-id 637.1528863347@sss.pgh.pa.us
обсуждение исходный текст
Ответ на Re: why partition pruning doesn't work?  (Robert Haas <robertmhaas@gmail.com>)
Ответы Re: why partition pruning doesn't work?  (David Rowley <david.rowley@2ndquadrant.com>)
Re: why partition pruning doesn't work?  (Robert Haas <robertmhaas@gmail.com>)
Список pgsql-hackers
Robert Haas <robertmhaas@gmail.com> writes:
> On Tue, Jun 12, 2018 at 12:54 PM, Tom Lane <tgl@sss.pgh.pa.us> wrote:
>> While I've not looked into the exact reasons for that, my first guess
>> is that the partitioned table is not held open because it's not one
>> of the ones to be scanned.  Are you prepared to change something like
>> that at this stage of the release cycle?

> The partition key is immutable, so it should NOT be able to disappear
> out from under us.

Hm.  That could be better documented.

> As for whether to change it at this point in the release cycle, I
> guess that, to me, depends on how invasive the fix is.

It seems not to be that bad: we just need a shutdown call for the
PartitionPruneState, and then we can remember the open relation there.
The attached is based on David's patch from yesterday.

I'm still a bit annoyed at the fmgr_info_copy calls in this.  It'd be
better to use the FmgrInfos in the relcache when applicable.  However,
mixing those with the cross-type ones would seem to require that we change
the API for get_matching_hash_bounds et al from taking "FmgrInfo *" to
taking "FmgrInfo **", which looks rather invasive.

            regards, tom lane

diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 33513ff..4eeee7c 100644
*** a/src/backend/executor/execPartition.c
--- b/src/backend/executor/execPartition.c
*************** adjust_partition_tlist(List *tlist, Tupl
*** 1357,1367 ****
   *
   * Functions:
   *
!  * ExecSetupPartitionPruneState:
   *        Creates the PartitionPruneState required by each of the two pruning
   *        functions.  Details stored include how to map the partition index
   *        returned by the partition pruning code into subplan indexes.
   *
   * ExecFindInitialMatchingSubPlans:
   *        Returns indexes of matching subplans.  Partition pruning is attempted
   *        without any evaluation of expressions containing PARAM_EXEC Params.
--- 1357,1370 ----
   *
   * Functions:
   *
!  * ExecCreatePartitionPruneState:
   *        Creates the PartitionPruneState required by each of the two pruning
   *        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.
*************** adjust_partition_tlist(List *tlist, Tupl
*** 1382,1389 ****
   */

  /*
!  * ExecSetupPartitionPruneState
!  *        Set up the data structure required for calling
   *        ExecFindInitialMatchingSubPlans and ExecFindMatchingSubPlans.
   *
   * 'planstate' is the parent plan node's execution state.
--- 1385,1392 ----
   */

  /*
!  * ExecCreatePartitionPruneState
!  *        Build the data structure required for calling
   *        ExecFindInitialMatchingSubPlans and ExecFindMatchingSubPlans.
   *
   * 'planstate' is the parent plan node's execution state.
*************** adjust_partition_tlist(List *tlist, Tupl
*** 1395,1401 ****
   * in each PartitionPruneInfo.
   */
  PartitionPruneState *
! ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
  {
      PartitionPruneState *prunestate;
      PartitionPruningData *prunedata;
--- 1398,1404 ----
   * in each PartitionPruneInfo.
   */
  PartitionPruneState *
! ExecCreatePartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
  {
      PartitionPruneState *prunestate;
      PartitionPruningData *prunedata;
*************** ExecSetupPartitionPruneState(PlanState *
*** 1435,1445 ****
          PartitionPruningData *pprune = &prunedata[i];
          PartitionPruneContext *context = &pprune->context;
          PartitionDesc partdesc;
-         Relation    rel;
          PartitionKey partkey;
-         ListCell   *lc2;
          int            partnatts;
          int            n_steps;

          /*
           * We must copy the subplan_map rather than pointing directly to the
--- 1438,1447 ----
          PartitionPruningData *pprune = &prunedata[i];
          PartitionPruneContext *context = &pprune->context;
          PartitionDesc partdesc;
          PartitionKey partkey;
          int            partnatts;
          int            n_steps;
+         ListCell   *lc2;

          /*
           * We must copy the subplan_map rather than pointing directly to the
*************** ExecSetupPartitionPruneState(PlanState *
*** 1456,1481 ****
          pprune->present_parts = bms_copy(pinfo->present_parts);

          /*
!          * Grab some info from the table's relcache; lock was already obtained
!          * by ExecLockNonLeafAppendTables.
           */
!         rel = relation_open(pinfo->reloid, NoLock);

!         partkey = RelationGetPartitionKey(rel);
!         partdesc = RelationGetPartitionDesc(rel);

          context->strategy = partkey->strategy;
          context->partnatts = partnatts = partkey->partnatts;
!         context->partopfamily = partkey->partopfamily;
!         context->partopcintype = partkey->partopcintype;
          context->partcollation = partkey->partcollation;
          context->partsupfunc = partkey->partsupfunc;
!         context->nparts = pinfo->nparts;
!         context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
          context->planstate = planstate;

          /* Initialize expression state for each expression we need */
-         n_steps = list_length(pinfo->pruning_steps);
          context->exprstates = (ExprState **)
              palloc0(sizeof(ExprState *) * n_steps * partnatts);
          foreach(lc2, pinfo->pruning_steps)
--- 1458,1490 ----
          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.
           */
!         context->partrel = relation_open(pinfo->reloid, NoLock);

!         partkey = RelationGetPartitionKey(context->partrel);
!         partdesc = RelationGetPartitionDesc(context->partrel);
!         n_steps = list_length(pinfo->pruning_steps);

          context->strategy = partkey->strategy;
          context->partnatts = partnatts = partkey->partnatts;
!         context->nparts = pinfo->nparts;
!         context->boundinfo = partdesc->boundinfo;
          context->partcollation = partkey->partcollation;
          context->partsupfunc = partkey->partsupfunc;
!
!         /* We'll look up type-specific support functions as needed */
!         context->stepcmpfuncs = (FmgrInfo *)
!             palloc0(sizeof(FmgrInfo) * n_steps * partnatts);
!
!         context->ppccontext = CurrentMemoryContext;
          context->planstate = planstate;

          /* Initialize expression state for each expression we need */
          context->exprstates = (ExprState **)
              palloc0(sizeof(ExprState *) * n_steps * partnatts);
          foreach(lc2, pinfo->pruning_steps)
*************** ExecSetupPartitionPruneState(PlanState *
*** 1527,1534 ****
          prunestate->execparamids = bms_add_members(prunestate->execparamids,
                                                     pinfo->execparamids);

-         relation_close(rel, NoLock);
-
          i++;
      }

--- 1536,1541 ----
*************** ExecSetupPartitionPruneState(PlanState *
*** 1536,1541 ****
--- 1543,1568 ----
  }

  /*
+  * 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)
+ {
+     int            i;
+
+     for (i = 0; i < prunestate->num_partprunedata; i++)
+     {
+         PartitionPruningData *pprune = &prunestate->partprunedata[i];
+
+         relation_close(pprune->context.partrel, NoLock);
+     }
+ }
+
+ /*
   * ExecFindInitialMatchingSubPlans
   *        Identify the set of subplans that cannot be eliminated by initial
   *        pruning (disregarding any pruning constraints involving PARAM_EXEC
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 6dd53e9..5ce4fb4 100644
*** a/src/backend/executor/nodeAppend.c
--- b/src/backend/executor/nodeAppend.c
*************** ExecInitAppend(Append *node, EState *est
*** 136,143 ****
          /* We may need an expression context to evaluate partition exprs */
          ExecAssignExprContext(estate, &appendstate->ps);

!         prunestate = ExecSetupPartitionPruneState(&appendstate->ps,
!                                                   node->part_prune_infos);

          /* Perform an initial partition prune, if required. */
          if (prunestate->do_initial_prune)
--- 136,145 ----
          /* We may need an expression context to evaluate partition exprs */
          ExecAssignExprContext(estate, &appendstate->ps);

!         /* Create the working data structure for pruning. */
!         prunestate = ExecCreatePartitionPruneState(&appendstate->ps,
!                                                    node->part_prune_infos);
!         appendstate->as_prune_state = prunestate;

          /* Perform an initial partition prune, if required. */
          if (prunestate->do_initial_prune)
*************** ExecInitAppend(Append *node, EState *est
*** 178,185 ****
           */
          if (!prunestate->do_exec_prune)
              appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
-
-         appendstate->as_prune_state = prunestate;
      }
      else
      {
--- 180,185 ----
*************** ExecEndAppend(AppendState *node)
*** 330,335 ****
--- 330,341 ----
       */
      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
diff --git a/src/backend/partitioning/partprune.c b/src/backend/partitioning/partprune.c
index 856bdd3..ee5826e 100644
*** a/src/backend/partitioning/partprune.c
--- b/src/backend/partitioning/partprune.c
*************** prune_append_rel_partitions(RelOptInfo *
*** 436,449 ****
      if (contradictory)
          return NULL;

      context.strategy = rel->part_scheme->strategy;
      context.partnatts = rel->part_scheme->partnatts;
-     context.partopfamily = rel->part_scheme->partopfamily;
-     context.partopcintype = rel->part_scheme->partopcintype;
-     context.partcollation = rel->part_scheme->partcollation;
-     context.partsupfunc = rel->part_scheme->partsupfunc;
      context.nparts = rel->nparts;
      context.boundinfo = rel->boundinfo;

      /* These are not valid when being called from the planner */
      context.planstate = NULL;
--- 436,453 ----
      if (contradictory)
          return NULL;

+     /* Set up PartitionPruneContext */
+     context.partrel = NULL;
      context.strategy = rel->part_scheme->strategy;
      context.partnatts = rel->part_scheme->partnatts;
      context.nparts = rel->nparts;
      context.boundinfo = rel->boundinfo;
+     context.partcollation = rel->part_scheme->partcollation;
+     context.partsupfunc = rel->part_scheme->partsupfunc;
+     context.stepcmpfuncs = (FmgrInfo *) palloc0(sizeof(FmgrInfo) *
+                                                 context.partnatts *
+                                                 list_length(pruning_steps));
+     context.ppccontext = CurrentMemoryContext;

      /* These are not valid when being called from the planner */
      context.planstate = NULL;
*************** perform_pruning_base_step(PartitionPrune
*** 2809,2815 ****
      int            keyno,
                  nvalues;
      Datum        values[PARTITION_MAX_KEYS];
!     FmgrInfo    partsupfunc[PARTITION_MAX_KEYS];

      /*
       * There better be the same number of expressions and compare functions.
--- 2813,2820 ----
      int            keyno,
                  nvalues;
      Datum        values[PARTITION_MAX_KEYS];
!     FmgrInfo   *partsupfunc;
!     int            stateidx;

      /*
       * There better be the same number of expressions and compare functions.
*************** perform_pruning_base_step(PartitionPrune
*** 2844,2850 ****
          if (lc1 != NULL)
          {
              Expr       *expr;
-             int            stateidx;
              Datum        datum;
              bool        isnull;

--- 2849,2854 ----
*************** perform_pruning_base_step(PartitionPrune
*** 2873,2891 ****
                      return result;
                  }

!                 /*
!                  * If we're going to need a different comparison function than
!                  * the one cached in the PartitionKey, we'll need to look up
!                  * the FmgrInfo.
!                  */
                  cmpfn = lfirst_oid(lc2);
                  Assert(OidIsValid(cmpfn));
!                 if (cmpfn != context->partsupfunc[keyno].fn_oid)
!                     fmgr_info(cmpfn, &partsupfunc[keyno]);
!                 else
!                     fmgr_info_copy(&partsupfunc[keyno],
!                                    &context->partsupfunc[keyno],
!                                    CurrentMemoryContext);

                  values[keyno] = datum;
                  nvalues++;
--- 2877,2901 ----
                      return result;
                  }

!                 /* Set up the stepcmpfuncs entry, unless we already did */
                  cmpfn = lfirst_oid(lc2);
                  Assert(OidIsValid(cmpfn));
!                 if (cmpfn != context->stepcmpfuncs[stateidx].fn_oid)
!                 {
!                     /*
!                      * If the needed support function is the same one cached
!                      * in the relation's partition key, copy the cached
!                      * FmgrInfo.  Otherwise (i.e., when we have a cross-type
!                      * comparison), an actual lookup is required.
!                      */
!                     if (cmpfn == context->partsupfunc[keyno].fn_oid)
!                         fmgr_info_copy(&context->stepcmpfuncs[stateidx],
!                                        &context->partsupfunc[keyno],
!                                        context->ppccontext);
!                     else
!                         fmgr_info_cxt(cmpfn, &context->stepcmpfuncs[stateidx],
!                                       context->ppccontext);
!                 }

                  values[keyno] = datum;
                  nvalues++;
*************** perform_pruning_base_step(PartitionPrune
*** 2896,2901 ****
--- 2906,2918 ----
          }
      }

+     /*
+      * Point partsupfunc to the entry for the 0th key of this step; the
+      * additional support functions, if any, follow consecutively.
+      */
+     stateidx = PruneCxtStateIdx(context->partnatts, opstep->step.step_id, 0);
+     partsupfunc = &context->stepcmpfuncs[stateidx];
+
      switch (context->strategy)
      {
          case PARTITION_STRATEGY_HASH:
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 3dfb1b8..dc69280 100644
*** a/src/backend/utils/cache/relcache.c
--- b/src/backend/utils/cache/relcache.c
*************** RelationClearRelation(Relation relation,
*** 2401,2406 ****
--- 2401,2407 ----
          keep_tupdesc = equalTupleDescs(relation->rd_att, newrel->rd_att);
          keep_rules = equalRuleLocks(relation->rd_rules, newrel->rd_rules);
          keep_policies = equalRSDesc(relation->rd_rsdesc, newrel->rd_rsdesc);
+         /* partkey is immutable once set up, so we can always keep it */
          keep_partkey = (relation->rd_partkey != NULL);
          keep_partdesc = equalPartitionDescs(relation->rd_partkey,
                                              relation->rd_partdesc,
*************** RelationClearRelation(Relation relation,
*** 2445,2451 ****
          SWAPFIELD(Form_pg_class, rd_rel);
          /* ... but actually, we don't have to update newrel->rd_rel */
          memcpy(relation->rd_rel, newrel->rd_rel, CLASS_TUPLE_SIZE);
!         /* preserve old tupledesc and rules if no logical change */
          if (keep_tupdesc)
              SWAPFIELD(TupleDesc, rd_att);
          if (keep_rules)
--- 2446,2452 ----
          SWAPFIELD(Form_pg_class, rd_rel);
          /* ... but actually, we don't have to update newrel->rd_rel */
          memcpy(relation->rd_rel, newrel->rd_rel, CLASS_TUPLE_SIZE);
!         /* preserve old tupledesc, rules, policies if no logical change */
          if (keep_tupdesc)
              SWAPFIELD(TupleDesc, rd_att);
          if (keep_rules)
*************** RelationClearRelation(Relation relation,
*** 2459,2471 ****
          SWAPFIELD(Oid, rd_toastoid);
          /* pgstat_info must be preserved */
          SWAPFIELD(struct PgStat_TableStatus *, pgstat_info);
!         /* partition key must be preserved, if we have one */
          if (keep_partkey)
          {
              SWAPFIELD(PartitionKey, rd_partkey);
              SWAPFIELD(MemoryContext, rd_partkeycxt);
          }
-         /* preserve old partdesc if no logical change */
          if (keep_partdesc)
          {
              SWAPFIELD(PartitionDesc, rd_partdesc);
--- 2460,2471 ----
          SWAPFIELD(Oid, rd_toastoid);
          /* pgstat_info must be preserved */
          SWAPFIELD(struct PgStat_TableStatus *, pgstat_info);
!         /* preserve old partitioning info if no logical change */
          if (keep_partkey)
          {
              SWAPFIELD(PartitionKey, rd_partkey);
              SWAPFIELD(MemoryContext, rd_partkeycxt);
          }
          if (keep_partdesc)
          {
              SWAPFIELD(PartitionDesc, rd_partdesc);
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 71d639f..862bf65 100644
*** a/src/include/executor/execPartition.h
--- b/src/include/executor/execPartition.h
*************** extern HeapTuple ConvertPartitionTupleSl
*** 208,215 ****
                            TupleTableSlot **p_my_slot);
  extern void ExecCleanupTupleRouting(ModifyTableState *mtstate,
                          PartitionTupleRouting *proute);
! extern PartitionPruneState *ExecSetupPartitionPruneState(PlanState *planstate,
!                              List *partitionpruneinfo);
  extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruneState *prunestate);
  extern Bitmapset *ExecFindInitialMatchingSubPlans(PartitionPruneState *prunestate,
                                  int nsubplans);
--- 208,216 ----
                            TupleTableSlot **p_my_slot);
  extern void ExecCleanupTupleRouting(ModifyTableState *mtstate,
                          PartitionTupleRouting *proute);
! extern PartitionPruneState *ExecCreatePartitionPruneState(PlanState *planstate,
!                               List *partitionpruneinfo);
! extern void ExecDestroyPartitionPruneState(PartitionPruneState *prunestate);
  extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruneState *prunestate);
  extern Bitmapset *ExecFindInitialMatchingSubPlans(PartitionPruneState *prunestate,
                                  int nsubplans);
diff --git a/src/include/partitioning/partprune.h b/src/include/partitioning/partprune.h
index e3b3bfb..09147b5 100644
*** a/src/include/partitioning/partprune.h
--- b/src/include/partitioning/partprune.h
***************
*** 20,68 ****

  /*
   * PartitionPruneContext
   *
!  * Information about a partitioned table needed to perform partition pruning.
   */
  typedef struct PartitionPruneContext
  {
!     /* Partition key information */
      char        strategy;
      int            partnatts;
-     Oid           *partopfamily;
-     Oid           *partopcintype;
-     Oid           *partcollation;
-     FmgrInfo   *partsupfunc;
-
-     /* Number of partitions */
      int            nparts;
-
-     /* Partition boundary info */
      PartitionBoundInfo boundinfo;
!
!     /*
!      * This will be set when the context is used from the executor, to allow
!      * Params to be evaluated.
!      */
      PlanState  *planstate;
-
-     /*
-      * Array of ExprStates, indexed as per PruneCtxStateIdx; one for each
-      * partkey in each pruning step.  Allocated if planstate is non-NULL,
-      * otherwise NULL.
-      */
      ExprState **exprstates;
-
-     /*
-      * Similar array of flags, each true if corresponding 'exprstate'
-      * expression contains any PARAM_EXEC Params.  (Can be NULL if planstate
-      * is NULL.)
-      */
      bool       *exprhasexecparam;
-
-     /* true if it's safe to evaluate PARAM_EXEC Params */
      bool        evalexecparams;
  } PartitionPruneContext;

  #define PruneCxtStateIdx(partnatts, step_id, keyno) \
      ((partnatts) * (step_id) + (keyno))

--- 20,76 ----

  /*
   * PartitionPruneContext
+  *        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.
!  * boundinfo        Partition boundary info for the partitioned table.
!  * partcollation    Array of partnatts elements, storing the collations of the
!  *                    partition key columns.
!  * partsupfunc        Array of FmgrInfos for the comparison or hashing functions
!  *                    associated with the partition keys (partnatts elements).
!  *                    (This points into the partrel's partition key, typically.)
!  * stepcmpfuncs        Array of FmgrInfos for the comparison or hashing function
!  *                    for each pruning step and partition key.
!  * ppccontext        Memory context holding this PartitionPruneContext's
!  *                    subsidiary data, such as the FmgrInfos.
!  * planstate        Points to the parent plan node's PlanState when called
!  *                    during execution; NULL when called from the planner.
!  * exprstates        Array of ExprStates, indexed as per PruneCtxStateIdx; one
!  *                    for each partition key in each pruning step.  Allocated if
!  *                    planstate is non-NULL, otherwise NULL.
!  * exprhasexecparam    Array of bools, each true if corresponding 'exprstate'
!  *                    expression contains any PARAM_EXEC Params.  (Can be NULL
!  *                    if planstate is NULL.)
!  * evalexecparams    True if it's safe to evaluate PARAM_EXEC Params.
   */
  typedef struct PartitionPruneContext
  {
!     Relation    partrel;
      char        strategy;
      int            partnatts;
      int            nparts;
      PartitionBoundInfo boundinfo;
!     Oid           *partcollation;
!     FmgrInfo   *partsupfunc;
!     FmgrInfo   *stepcmpfuncs;
!     MemoryContext ppccontext;
      PlanState  *planstate;
      ExprState **exprstates;
      bool       *exprhasexecparam;
      bool        evalexecparams;
  } PartitionPruneContext;

+ /*
+  * PruneCxtStateIdx() computes the correct index into the stepcmpfuncs[],
+  * exprstates[] and exprhasexecparam[] arrays for step step_id and
+  * partition key column keyno.  (Note: there is code that assumes the
+  * entries for a given step are sequential, so this is not chosen freely.)
+  */
  #define PruneCxtStateIdx(partnatts, step_id, keyno) \
      ((partnatts) * (step_id) + (keyno))


В списке pgsql-hackers по дате отправления:

Предыдущее
От: Michael Paquier
Дата:
Сообщение: Re: Fix some error handling for read() and errno
Следующее
От: "Kato, Sho"
Дата:
Сообщение: Add function to release an allocated SQLDA