diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c index 9b03579e6e0..9a83ebf3231 100644 --- a/src/backend/catalog/heap.c +++ b/src/backend/catalog/heap.c @@ -1799,57 +1799,57 @@ heap_drop_with_catalog(Oid relid) rel = relation_open(relid, AccessExclusiveLock); /* * There can no longer be anyone *else* touching the relation, but we * might still have open queries or cursors, or pending trigger events, in * our own session. */ CheckTableNotInUse(rel, "DROP TABLE"); /* * This effectively deletes all rows in the table, and may be done in a * serializable transaction. In that case we must record a rw-conflict in * to this transaction from each transaction holding a predicate lock on * the table. */ CheckTableForSerializableConflictIn(rel); /* * Delete pg_foreign_table tuple first. */ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE) { - Relation rel; - HeapTuple tuple; + Relation pg_foreign_table; + HeapTuple foreigntuple; - rel = table_open(ForeignTableRelationId, RowExclusiveLock); + pg_foreign_table = table_open(ForeignTableRelationId, RowExclusiveLock); - tuple = SearchSysCache1(FOREIGNTABLEREL, ObjectIdGetDatum(relid)); - if (!HeapTupleIsValid(tuple)) + foreigntuple = SearchSysCache1(FOREIGNTABLEREL, ObjectIdGetDatum(relid)); + if (!HeapTupleIsValid(foreigntuple)) elog(ERROR, "cache lookup failed for foreign table %u", relid); - CatalogTupleDelete(rel, &tuple->t_self); + CatalogTupleDelete(pg_foreign_table, &foreigntuple->t_self); - ReleaseSysCache(tuple); - table_close(rel, RowExclusiveLock); + ReleaseSysCache(foreigntuple); + table_close(pg_foreign_table, RowExclusiveLock); } /* * If a partitioned table, delete the pg_partitioned_table tuple. */ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE) RemovePartitionKeyByRelId(relid); /* * If the relation being dropped is the default partition itself, * invalidate its entry in pg_partitioned_table. */ if (relid == defaultPartOid) update_default_partition_oid(parentOid, InvalidOid); /* * Schedule unlinking of the relation's physical files at commit. */ if (RELKIND_HAS_STORAGE(rel->rd_rel->relkind)) RelationDropStorage(rel); /* ensure that stats are dropped if transaction commits */ diff --git a/src/backend/commands/publicationcmds.c b/src/backend/commands/publicationcmds.c index 8b574b86c47..f9366f588fb 100644 --- a/src/backend/commands/publicationcmds.c +++ b/src/backend/commands/publicationcmds.c @@ -87,70 +87,70 @@ parse_publication_options(ParseState *pstate, { ListCell *lc; *publish_given = false; *publish_via_partition_root_given = false; /* defaults */ pubactions->pubinsert = true; pubactions->pubupdate = true; pubactions->pubdelete = true; pubactions->pubtruncate = true; *publish_via_partition_root = false; /* Parse options */ foreach(lc, options) { DefElem *defel = (DefElem *) lfirst(lc); if (strcmp(defel->defname, "publish") == 0) { char *publish; List *publish_list; - ListCell *lc; + ListCell *lc2; if (*publish_given) errorConflictingDefElem(defel, pstate); /* * If publish option was given only the explicitly listed actions * should be published. */ pubactions->pubinsert = false; pubactions->pubupdate = false; pubactions->pubdelete = false; pubactions->pubtruncate = false; *publish_given = true; publish = defGetString(defel); if (!SplitIdentifierString(publish, ',', &publish_list)) ereport(ERROR, (errcode(ERRCODE_SYNTAX_ERROR), errmsg("invalid list syntax for \"publish\" option"))); /* Process the option list. */ - foreach(lc, publish_list) + foreach(lc2, publish_list) { - char *publish_opt = (char *) lfirst(lc); + char *publish_opt = (char *) lfirst(lc2); if (strcmp(publish_opt, "insert") == 0) pubactions->pubinsert = true; else if (strcmp(publish_opt, "update") == 0) pubactions->pubupdate = true; else if (strcmp(publish_opt, "delete") == 0) pubactions->pubdelete = true; else if (strcmp(publish_opt, "truncate") == 0) pubactions->pubtruncate = true; else ereport(ERROR, (errcode(ERRCODE_SYNTAX_ERROR), errmsg("unrecognized \"publish\" value: \"%s\"", publish_opt))); } } else if (strcmp(defel->defname, "publish_via_partition_root") == 0) { if (*publish_via_partition_root_given) errorConflictingDefElem(defel, pstate); *publish_via_partition_root_given = true; *publish_via_partition_root = defGetBoolean(defel); } diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c index dacc989d855..7535b86bcae 100644 --- a/src/backend/commands/tablecmds.c +++ b/src/backend/commands/tablecmds.c @@ -10204,45 +10204,45 @@ CloneFkReferencing(List **wqueue, Relation parentRel, Relation partRel) foreach(cell, clone) { Oid parentConstrOid = lfirst_oid(cell); Form_pg_constraint constrForm; Relation pkrel; HeapTuple tuple; int numfks; AttrNumber conkey[INDEX_MAX_KEYS]; AttrNumber mapped_conkey[INDEX_MAX_KEYS]; AttrNumber confkey[INDEX_MAX_KEYS]; Oid conpfeqop[INDEX_MAX_KEYS]; Oid conppeqop[INDEX_MAX_KEYS]; Oid conffeqop[INDEX_MAX_KEYS]; int numfkdelsetcols; AttrNumber confdelsetcols[INDEX_MAX_KEYS]; Constraint *fkconstraint; bool attached; Oid indexOid; Oid constrOid; ObjectAddress address, referenced; - ListCell *cell; + ListCell *lc; Oid insertTriggerOid, updateTriggerOid; tuple = SearchSysCache1(CONSTROID, parentConstrOid); if (!HeapTupleIsValid(tuple)) elog(ERROR, "cache lookup failed for constraint %u", parentConstrOid); constrForm = (Form_pg_constraint) GETSTRUCT(tuple); /* Don't clone constraints whose parents are being cloned */ if (list_member_oid(clone, constrForm->conparentid)) { ReleaseSysCache(tuple); continue; } /* * Need to prevent concurrent deletions. If pkrel is a partitioned * relation, that means to lock all partitions. */ pkrel = table_open(constrForm->confrelid, ShareRowExclusiveLock); if (pkrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE) @@ -10257,47 +10257,47 @@ CloneFkReferencing(List **wqueue, Relation parentRel, Relation partRel) /* * Get the "check" triggers belonging to the constraint to pass as * parent OIDs for similar triggers that will be created on the * partition in addFkRecurseReferencing(). They are also passed to * tryAttachPartitionForeignKey() below to simply assign as parents to * the partition's existing "check" triggers, that is, if the * corresponding constraints is deemed attachable to the parent * constraint. */ GetForeignKeyCheckTriggers(trigrel, constrForm->oid, constrForm->confrelid, constrForm->conrelid, &insertTriggerOid, &updateTriggerOid); /* * Before creating a new constraint, see whether any existing FKs are * fit for the purpose. If one is, attach the parent constraint to * it, and don't clone anything. This way we avoid the expensive * verification step and don't end up with a duplicate FK, and we * don't need to recurse to partitions for this constraint. */ attached = false; - foreach(cell, partFKs) + foreach(lc, partFKs) { - ForeignKeyCacheInfo *fk = lfirst_node(ForeignKeyCacheInfo, cell); + ForeignKeyCacheInfo *fk = lfirst_node(ForeignKeyCacheInfo, lc); if (tryAttachPartitionForeignKey(fk, RelationGetRelid(partRel), parentConstrOid, numfks, mapped_conkey, confkey, conpfeqop, insertTriggerOid, updateTriggerOid, trigrel)) { attached = true; table_close(pkrel, NoLock); break; } } if (attached) { ReleaseSysCache(tuple); continue; } diff --git a/src/backend/commands/trigger.c b/src/backend/commands/trigger.c index 62a09fb131b..f1801a160ed 100644 --- a/src/backend/commands/trigger.c +++ b/src/backend/commands/trigger.c @@ -1130,77 +1130,77 @@ CreateTriggerFiringOn(CreateTrigStmt *stmt, const char *queryString, } /* * If it has a WHEN clause, add dependencies on objects mentioned in the * expression (eg, functions, as well as any columns used). */ if (whenRtable != NIL) recordDependencyOnExpr(&myself, whenClause, whenRtable, DEPENDENCY_NORMAL); /* Post creation hook for new trigger */ InvokeObjectPostCreateHookArg(TriggerRelationId, trigoid, 0, isInternal); /* * Lastly, create the trigger on child relations, if needed. */ if (partition_recurse) { PartitionDesc partdesc = RelationGetPartitionDesc(rel, true); List *idxs = NIL; List *childTbls = NIL; - ListCell *l; int i; MemoryContext oldcxt, perChildCxt; perChildCxt = AllocSetContextCreate(CurrentMemoryContext, "part trig clone", ALLOCSET_SMALL_SIZES); /* * When a trigger is being created associated with an index, we'll * need to associate the trigger in each child partition with the * corresponding index on it. */ if (OidIsValid(indexOid)) { ListCell *l; List *idxs = NIL; idxs = find_inheritance_children(indexOid, ShareRowExclusiveLock); foreach(l, idxs) childTbls = lappend_oid(childTbls, IndexGetRelation(lfirst_oid(l), false)); } oldcxt = MemoryContextSwitchTo(perChildCxt); /* Iterate to create the trigger on each existing partition */ for (i = 0; i < partdesc->nparts; i++) { Oid indexOnChild = InvalidOid; - ListCell *l2; + ListCell *l, + *l2; CreateTrigStmt *childStmt; Relation childTbl; Node *qual; childTbl = table_open(partdesc->oids[i], ShareRowExclusiveLock); /* Find which of the child indexes is the one on this partition */ if (OidIsValid(indexOid)) { forboth(l, idxs, l2, childTbls) { if (lfirst_oid(l2) == partdesc->oids[i]) { indexOnChild = lfirst_oid(l); break; } } if (!OidIsValid(indexOnChild)) elog(ERROR, "failed to find index matching index \"%s\" in partition \"%s\"", get_rel_name(indexOid), get_rel_name(partdesc->oids[i])); } @@ -1707,47 +1707,47 @@ renametrig_partition(Relation tgrel, Oid partitionId, Oid parentTriggerOid, NULL, 1, &key); while (HeapTupleIsValid(tuple = systable_getnext(tgscan))) { Form_pg_trigger tgform = (Form_pg_trigger) GETSTRUCT(tuple); Relation partitionRel; if (tgform->tgparentid != parentTriggerOid) continue; /* not our trigger */ partitionRel = table_open(partitionId, NoLock); /* Rename the trigger on this partition */ renametrig_internal(tgrel, partitionRel, tuple, newname, expected_name); /* And if this relation is partitioned, recurse to its partitions */ if (partitionRel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE) { PartitionDesc partdesc = RelationGetPartitionDesc(partitionRel, true); for (int i = 0; i < partdesc->nparts; i++) { - Oid partitionId = partdesc->oids[i]; + Oid partid = partdesc->oids[i]; - renametrig_partition(tgrel, partitionId, tgform->oid, newname, + renametrig_partition(tgrel, partid, tgform->oid, newname, NameStr(tgform->tgname)); } } table_close(partitionRel, NoLock); /* There should be at most one matching tuple */ break; } systable_endscan(tgscan); } /* * EnableDisableTrigger() * * Called by ALTER TABLE ENABLE/DISABLE [ REPLICA | ALWAYS ] TRIGGER * to change 'tgenabled' field for the specified trigger(s) * * rel: relation to process (caller must hold suitable lock on it) * tgname: trigger to process, or NULL to scan all triggers * fires_when: new value for tgenabled field. In addition to generic * enablement/disablement, this also defines when the trigger * should be fired in session replication roles. diff --git a/src/backend/executor/nodeAgg.c b/src/backend/executor/nodeAgg.c index 933c3049016..736082c8fb3 100644 --- a/src/backend/executor/nodeAgg.c +++ b/src/backend/executor/nodeAgg.c @@ -3168,45 +3168,44 @@ hashagg_reset_spill_state(AggState *aggstate) AggState * ExecInitAgg(Agg *node, EState *estate, int eflags) { AggState *aggstate; AggStatePerAgg peraggs; AggStatePerTrans pertransstates; AggStatePerGroup *pergroups; Plan *outerPlan; ExprContext *econtext; TupleDesc scanDesc; int max_aggno; int max_transno; int numaggrefs; int numaggs; int numtrans; int phase; int phaseidx; ListCell *l; Bitmapset *all_grouped_cols = NULL; int numGroupingSets = 1; int numPhases; int numHashes; - int i = 0; int j = 0; bool use_hashing = (node->aggstrategy == AGG_HASHED || node->aggstrategy == AGG_MIXED); /* check for unsupported flags */ Assert(!(eflags & (EXEC_FLAG_BACKWARD | EXEC_FLAG_MARK))); /* * create state structure */ aggstate = makeNode(AggState); aggstate->ss.ps.plan = (Plan *) node; aggstate->ss.ps.state = estate; aggstate->ss.ps.ExecProcNode = ExecAgg; aggstate->aggs = NIL; aggstate->numaggs = 0; aggstate->numtrans = 0; aggstate->aggstrategy = node->aggstrategy; aggstate->aggsplit = node->aggsplit; aggstate->maxsets = 0; aggstate->projected_set = -1; @@ -3259,45 +3258,45 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) aggstate->numphases = numPhases; aggstate->aggcontexts = (ExprContext **) palloc0(sizeof(ExprContext *) * numGroupingSets); /* * Create expression contexts. We need three or more, one for * per-input-tuple processing, one for per-output-tuple processing, one * for all the hashtables, and one for each grouping set. The per-tuple * memory context of the per-grouping-set ExprContexts (aggcontexts) * replaces the standalone memory context formerly used to hold transition * values. We cheat a little by using ExecAssignExprContext() to build * all of them. * * NOTE: the details of what is stored in aggcontexts and what is stored * in the regular per-query memory context are driven by a simple * decision: we want to reset the aggcontext at group boundaries (if not * hashing) and in ExecReScanAgg to recover no-longer-wanted space. */ ExecAssignExprContext(estate, &aggstate->ss.ps); aggstate->tmpcontext = aggstate->ss.ps.ps_ExprContext; - for (i = 0; i < numGroupingSets; ++i) + for (int i = 0; i < numGroupingSets; ++i) { ExecAssignExprContext(estate, &aggstate->ss.ps); aggstate->aggcontexts[i] = aggstate->ss.ps.ps_ExprContext; } if (use_hashing) aggstate->hashcontext = CreateWorkExprContext(estate); ExecAssignExprContext(estate, &aggstate->ss.ps); /* * Initialize child nodes. * * If we are doing a hashed aggregation then the child plan does not need * to handle REWIND efficiently; see ExecReScanAgg. */ if (node->aggstrategy == AGG_HASHED) eflags &= ~EXEC_FLAG_REWIND; outerPlan = outerPlan(node); outerPlanState(aggstate) = ExecInitNode(outerPlan, estate, eflags); /* @@ -3399,75 +3398,76 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) Agg *aggnode; Sort *sortnode; if (phaseidx > 0) { aggnode = list_nth_node(Agg, node->chain, phaseidx - 1); sortnode = castNode(Sort, outerPlan(aggnode)); } else { aggnode = node; sortnode = NULL; } Assert(phase <= 1 || sortnode); if (aggnode->aggstrategy == AGG_HASHED || aggnode->aggstrategy == AGG_MIXED) { AggStatePerPhase phasedata = &aggstate->phases[0]; AggStatePerHash perhash; Bitmapset *cols = NULL; + int setno = phasedata->numsets++; Assert(phase == 0); - i = phasedata->numsets++; - perhash = &aggstate->perhash[i]; + perhash = &aggstate->perhash[setno]; /* phase 0 always points to the "real" Agg in the hash case */ phasedata->aggnode = node; phasedata->aggstrategy = node->aggstrategy; /* but the actual Agg node representing this hash is saved here */ perhash->aggnode = aggnode; - phasedata->gset_lengths[i] = perhash->numCols = aggnode->numCols; + phasedata->gset_lengths[setno] = perhash->numCols = aggnode->numCols; for (j = 0; j < aggnode->numCols; ++j) cols = bms_add_member(cols, aggnode->grpColIdx[j]); - phasedata->grouped_cols[i] = cols; + phasedata->grouped_cols[setno] = cols; all_grouped_cols = bms_add_members(all_grouped_cols, cols); continue; } else { AggStatePerPhase phasedata = &aggstate->phases[++phase]; int num_sets; phasedata->numsets = num_sets = list_length(aggnode->groupingSets); if (num_sets) { + int i; phasedata->gset_lengths = palloc(num_sets * sizeof(int)); phasedata->grouped_cols = palloc(num_sets * sizeof(Bitmapset *)); i = 0; foreach(l, aggnode->groupingSets) { int current_length = list_length(lfirst(l)); Bitmapset *cols = NULL; /* planner forces this to be correct */ for (j = 0; j < current_length; ++j) cols = bms_add_member(cols, aggnode->grpColIdx[j]); phasedata->grouped_cols[i] = cols; phasedata->gset_lengths[i] = current_length; ++i; } all_grouped_cols = bms_add_members(all_grouped_cols, phasedata->grouped_cols[0]); } @@ -3515,71 +3515,73 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) /* and for all grouped columns, unless already computed */ if (phasedata->eqfunctions[aggnode->numCols - 1] == NULL) { phasedata->eqfunctions[aggnode->numCols - 1] = execTuplesMatchPrepare(scanDesc, aggnode->numCols, aggnode->grpColIdx, aggnode->grpOperators, aggnode->grpCollations, (PlanState *) aggstate); } } phasedata->aggnode = aggnode; phasedata->aggstrategy = aggnode->aggstrategy; phasedata->sortnode = sortnode; } } /* * Convert all_grouped_cols to a descending-order list. */ - i = -1; - while ((i = bms_next_member(all_grouped_cols, i)) >= 0) - aggstate->all_grouped_cols = lcons_int(i, aggstate->all_grouped_cols); + { + int i = -1; + while ((i = bms_next_member(all_grouped_cols, i)) >= 0) + aggstate->all_grouped_cols = lcons_int(i, aggstate->all_grouped_cols); + } /* * Set up aggregate-result storage in the output expr context, and also * allocate my private per-agg working storage */ econtext = aggstate->ss.ps.ps_ExprContext; econtext->ecxt_aggvalues = (Datum *) palloc0(sizeof(Datum) * numaggs); econtext->ecxt_aggnulls = (bool *) palloc0(sizeof(bool) * numaggs); peraggs = (AggStatePerAgg) palloc0(sizeof(AggStatePerAggData) * numaggs); pertransstates = (AggStatePerTrans) palloc0(sizeof(AggStatePerTransData) * numtrans); aggstate->peragg = peraggs; aggstate->pertrans = pertransstates; aggstate->all_pergroups = (AggStatePerGroup *) palloc0(sizeof(AggStatePerGroup) * (numGroupingSets + numHashes)); pergroups = aggstate->all_pergroups; if (node->aggstrategy != AGG_HASHED) { - for (i = 0; i < numGroupingSets; i++) + for (int i = 0; i < numGroupingSets; i++) { pergroups[i] = (AggStatePerGroup) palloc0(sizeof(AggStatePerGroupData) * numaggs); } aggstate->pergroups = pergroups; pergroups += numGroupingSets; } /* * Hashing can only appear in the initial phase. */ if (use_hashing) { Plan *outerplan = outerPlan(node); uint64 totalGroups = 0; int i; aggstate->hash_metacxt = AllocSetContextCreate(aggstate->ss.ps.state->es_query_cxt, "HashAgg meta context", ALLOCSET_DEFAULT_SIZES); aggstate->hash_spill_rslot = ExecInitExtraTupleSlot(estate, scanDesc, diff --git a/src/backend/libpq/auth.c b/src/backend/libpq/auth.c index 1545ff9f161..f9d40fa1a0d 100644 --- a/src/backend/libpq/auth.c +++ b/src/backend/libpq/auth.c @@ -1631,54 +1631,54 @@ interpret_ident_response(const char *ident_response, while (pg_isblank(*cursor)) cursor++; /* skip blanks */ if (strcmp(response_type, "USERID") != 0) return false; else { /* * It's a USERID response. Good. "cursor" should be pointing * to the colon that precedes the operating system type. */ if (*cursor != ':') return false; else { cursor++; /* Go over colon */ /* Skip over operating system field. */ while (*cursor != ':' && *cursor != '\r') cursor++; if (*cursor != ':') return false; else { - int i; /* Index into *ident_user */ + int j; /* Index into *ident_user */ cursor++; /* Go over colon */ while (pg_isblank(*cursor)) cursor++; /* skip blanks */ /* Rest of line is user name. Copy it over. */ - i = 0; + j = 0; while (*cursor != '\r' && i < IDENT_USERNAME_MAX) - ident_user[i++] = *cursor++; - ident_user[i] = '\0'; + ident_user[j++] = *cursor++; + ident_user[j] = '\0'; return true; } } } } } } /* * Talk to the ident server on "remote_addr" and find out who * owns the tcp connection to "local_addr" * If the username is successfully retrieved, check the usermap. * * XXX: Using WaitLatchOrSocket() and doing a CHECK_FOR_INTERRUPTS() if the * latch was set would improve the responsiveness to timeouts/cancellations. */ static int ident_inet(hbaPort *port) { const SockAddr remote_addr = port->raddr; const SockAddr local_addr = port->laddr; diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c index 75acea149c7..74adc4f3946 100644 --- a/src/backend/optimizer/path/costsize.c +++ b/src/backend/optimizer/path/costsize.c @@ -2526,48 +2526,48 @@ cost_append(AppendPath *apath, PlannerInfo *root) apath->path.rows = 0; if (apath->subpaths == NIL) return; if (!apath->path.parallel_aware) { List *pathkeys = apath->path.pathkeys; if (pathkeys == NIL) { Path *subpath = (Path *) linitial(apath->subpaths); /* * For an unordered, non-parallel-aware Append we take the startup * cost as the startup cost of the first subpath. */ apath->path.startup_cost = subpath->startup_cost; /* Compute rows and costs as sums of subplan rows and costs. */ foreach(l, apath->subpaths) { - Path *subpath = (Path *) lfirst(l); + Path *sub = (Path *) lfirst(l); - apath->path.rows += subpath->rows; - apath->path.total_cost += subpath->total_cost; + apath->path.rows += sub->rows; + apath->path.total_cost += sub->total_cost; } } else { /* * For an ordered, non-parallel-aware Append we take the startup * cost as the sum of the subpath startup costs. This ensures * that we don't underestimate the startup cost when a query's * LIMIT is such that several of the children have to be run to * satisfy it. This might be overkill --- another plausible hack * would be to take the Append's startup cost as the maximum of * the child startup costs. But we don't want to risk believing * that an ORDER BY LIMIT query can be satisfied at small cost * when the first child has small startup cost but later ones * don't. (If we had the ability to deal with nonlinear cost * interpolation for partial retrievals, we would not need to be * so conservative about this.) * * This case is also different from the above in that we have to * account for possibly injecting sorts into subpaths that aren't * natively ordered. */ diff --git a/src/backend/optimizer/path/tidpath.c b/src/backend/optimizer/path/tidpath.c index 279ca1f5b44..23194d6e007 100644 --- a/src/backend/optimizer/path/tidpath.c +++ b/src/backend/optimizer/path/tidpath.c @@ -286,48 +286,48 @@ TidQualFromRestrictInfoList(PlannerInfo *root, List *rlist, RelOptInfo *rel) { ListCell *j; /* * We must be able to extract a CTID condition from every * sub-clause of an OR, or we can't use it. */ foreach(j, ((BoolExpr *) rinfo->orclause)->args) { Node *orarg = (Node *) lfirst(j); List *sublist; /* OR arguments should be ANDs or sub-RestrictInfos */ if (is_andclause(orarg)) { List *andargs = ((BoolExpr *) orarg)->args; /* Recurse in case there are sub-ORs */ sublist = TidQualFromRestrictInfoList(root, andargs, rel); } else { - RestrictInfo *rinfo = castNode(RestrictInfo, orarg); + RestrictInfo *list = castNode(RestrictInfo, orarg); - Assert(!restriction_is_or_clause(rinfo)); - sublist = TidQualFromRestrictInfo(root, rinfo, rel); + Assert(!restriction_is_or_clause(list)); + sublist = TidQualFromRestrictInfo(root, list, rel); } /* * If nothing found in this arm, we can't do anything with * this OR clause. */ if (sublist == NIL) { rlst = NIL; /* forget anything we had */ break; /* out of loop over OR args */ } /* * OK, continue constructing implicitly-OR'ed result list. */ rlst = list_concat(rlst, sublist); } } else { /* Not an OR clause, so handle base cases */ rlst = TidQualFromRestrictInfo(root, rinfo, rel); diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c index cf9e0a74dbf..e969f2be3fe 100644 --- a/src/backend/optimizer/plan/planner.c +++ b/src/backend/optimizer/plan/planner.c @@ -1975,46 +1975,44 @@ grouping_planner(PlannerInfo *root, double tuple_fraction) * of rollups, and preparing annotations which will later be filled in with * size estimates. */ static grouping_sets_data * preprocess_grouping_sets(PlannerInfo *root) { Query *parse = root->parse; List *sets; int maxref = 0; ListCell *lc; ListCell *lc_set; grouping_sets_data *gd = palloc0(sizeof(grouping_sets_data)); parse->groupingSets = expand_grouping_sets(parse->groupingSets, parse->groupDistinct, -1); gd->any_hashable = false; gd->unhashable_refs = NULL; gd->unsortable_refs = NULL; gd->unsortable_sets = NIL; if (parse->groupClause) { - ListCell *lc; - foreach(lc, parse->groupClause) { SortGroupClause *gc = lfirst_node(SortGroupClause, lc); Index ref = gc->tleSortGroupRef; if (ref > maxref) maxref = ref; if (!gc->hashable) gd->unhashable_refs = bms_add_member(gd->unhashable_refs, ref); if (!OidIsValid(gc->sortop)) gd->unsortable_refs = bms_add_member(gd->unsortable_refs, ref); } } /* Allocate workspace array for remapping */ gd->tleref_to_colnum_map = (int *) palloc((maxref + 1) * sizeof(int)); /* * If we have any unsortable sets, we must extract them before trying to * prepare rollups. Unsortable sets don't go through @@ -3439,72 +3437,70 @@ get_number_of_groups(PlannerInfo *root, List *target_list) { Query *parse = root->parse; double dNumGroups; if (parse->groupClause) { List *groupExprs; if (parse->groupingSets) { /* Add up the estimates for each grouping set */ ListCell *lc; ListCell *lc2; Assert(gd); /* keep Coverity happy */ dNumGroups = 0; foreach(lc, gd->rollups) { RollupData *rollup = lfirst_node(RollupData, lc); - ListCell *lc; + ListCell *lc3; groupExprs = get_sortgrouplist_exprs(rollup->groupClause, target_list); rollup->numGroups = 0.0; - forboth(lc, rollup->gsets, lc2, rollup->gsets_data) + forboth(lc3, rollup->gsets, lc2, rollup->gsets_data) { - List *gset = (List *) lfirst(lc); + List *gset = (List *) lfirst(lc3); GroupingSetData *gs = lfirst_node(GroupingSetData, lc2); double numGroups = estimate_num_groups(root, groupExprs, path_rows, &gset, NULL); gs->numGroups = numGroups; rollup->numGroups += numGroups; } dNumGroups += rollup->numGroups; } if (gd->hash_sets_idx) { - ListCell *lc; - gd->dNumHashGroups = 0; groupExprs = get_sortgrouplist_exprs(parse->groupClause, target_list); forboth(lc, gd->hash_sets_idx, lc2, gd->unsortable_sets) { List *gset = (List *) lfirst(lc); GroupingSetData *gs = lfirst_node(GroupingSetData, lc2); double numGroups = estimate_num_groups(root, groupExprs, path_rows, &gset, NULL); gs->numGroups = numGroups; gd->dNumHashGroups += numGroups; } dNumGroups += gd->dNumHashGroups; } } @@ -5015,49 +5011,49 @@ create_ordered_paths(PlannerInfo *root, path, path->pathtarget, root->sort_pathkeys, NULL, &total_groups); /* Add projection step if needed */ if (path->pathtarget != target) path = apply_projection_to_path(root, ordered_rel, path, target); add_path(ordered_rel, path); } /* * Consider incremental sort with a gather merge on partial paths. * * We can also skip the entire loop when we only have a single-item * sort_pathkeys because then we can't possibly have a presorted * prefix of the list without having the list be fully sorted. */ if (enable_incremental_sort && list_length(root->sort_pathkeys) > 1) { - ListCell *lc; + ListCell *lc2; - foreach(lc, input_rel->partial_pathlist) + foreach(lc2, input_rel->partial_pathlist) { - Path *input_path = (Path *) lfirst(lc); + Path *input_path = (Path *) lfirst(lc2); Path *sorted_path; bool is_sorted; int presorted_keys; double total_groups; /* * We don't care if this is the cheapest partial path - we * can't simply skip it, because it may be partially sorted in * which case we want to consider adding incremental sort * (instead of full sort, which is what happens above). */ is_sorted = pathkeys_count_contained_in(root->sort_pathkeys, input_path->pathkeys, &presorted_keys); /* No point in adding incremental sort on fully sorted paths. */ if (is_sorted) continue; if (presorted_keys == 0) continue; @@ -7588,58 +7584,58 @@ apply_scanjoin_target_to_paths(PlannerInfo *root, rel->reltarget = llast_node(PathTarget, scanjoin_targets); /* * If the relation is partitioned, recursively apply the scan/join target * to all partitions, and generate brand-new Append paths in which the * scan/join target is computed below the Append rather than above it. * Since Append is not projection-capable, that might save a separate * Result node, and it also is important for partitionwise aggregate. */ if (rel_is_partitioned) { List *live_children = NIL; int i; /* Adjust each partition. */ i = -1; while ((i = bms_next_member(rel->live_parts, i)) >= 0) { RelOptInfo *child_rel = rel->part_rels[i]; AppendRelInfo **appinfos; int nappinfos; List *child_scanjoin_targets = NIL; - ListCell *lc; + ListCell *lc2; Assert(child_rel != NULL); /* Dummy children can be ignored. */ if (IS_DUMMY_REL(child_rel)) continue; /* Translate scan/join targets for this child. */ appinfos = find_appinfos_by_relids(root, child_rel->relids, &nappinfos); - foreach(lc, scanjoin_targets) + foreach(lc2, scanjoin_targets) { - PathTarget *target = lfirst_node(PathTarget, lc); + PathTarget *target = lfirst_node(PathTarget, lc2); target = copy_pathtarget(target); target->exprs = (List *) adjust_appendrel_attrs(root, (Node *) target->exprs, nappinfos, appinfos); child_scanjoin_targets = lappend(child_scanjoin_targets, target); } pfree(appinfos); /* Recursion does the real work. */ apply_scanjoin_target_to_paths(root, child_rel, child_scanjoin_targets, scanjoin_targets_contain_srfs, scanjoin_target_parallel_safe, tlist_same_exprs); /* Save non-dummy children for Append paths. */ if (!IS_DUMMY_REL(child_rel)) live_children = lappend(live_children, child_rel); } diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c index 71052c841d7..f97c2f5256c 100644 --- a/src/backend/optimizer/prep/prepunion.c +++ b/src/backend/optimizer/prep/prepunion.c @@ -639,47 +639,47 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root, add_path(result_rel, path); /* * Estimate number of groups. For now we just assume the output is unique * --- this is certainly true for the UNION case, and we want worst-case * estimates anyway. */ result_rel->rows = path->rows; /* * Now consider doing the same thing using the partial paths plus Append * plus Gather. */ if (partial_paths_valid) { Path *ppath; int parallel_workers = 0; /* Find the highest number of workers requested for any subpath. */ foreach(lc, partial_pathlist) { - Path *path = lfirst(lc); + Path *partial_path = lfirst(lc); - parallel_workers = Max(parallel_workers, path->parallel_workers); + parallel_workers = Max(parallel_workers, partial_path->parallel_workers); } Assert(parallel_workers > 0); /* * If the use of parallel append is permitted, always request at least * log2(# of children) paths. We assume it can be useful to have * extra workers in this case because they will be spread out across * the children. The precise formula is just a guess; see * add_paths_to_append_rel. */ if (enable_parallel_append) { parallel_workers = Max(parallel_workers, pg_leftmost_one_pos32(list_length(partial_pathlist)) + 1); parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather); } Assert(parallel_workers > 0); ppath = (Path *) create_append_path(root, result_rel, NIL, partial_pathlist, NIL, NULL, diff --git a/src/backend/optimizer/util/paramassign.c b/src/backend/optimizer/util/paramassign.c index 8e2d4bf5158..933460989b3 100644 --- a/src/backend/optimizer/util/paramassign.c +++ b/src/backend/optimizer/util/paramassign.c @@ -418,93 +418,93 @@ replace_nestloop_param_placeholdervar(PlannerInfo *root, PlaceHolderVar *phv) * while planning the subquery. So we need not modify the subplan or the * PlannerParamItems here. What we do need to do is add entries to * root->curOuterParams to signal the parent nestloop plan node that it must * provide these values. This differs from replace_nestloop_param_var in * that the PARAM_EXEC slots to use have already been determined. * * Note that we also use root->curOuterRels as an implicit parameter for * sanity checks. */ void process_subquery_nestloop_params(PlannerInfo *root, List *subplan_params) { ListCell *lc; foreach(lc, subplan_params) { PlannerParamItem *pitem = lfirst_node(PlannerParamItem, lc); if (IsA(pitem->item, Var)) { Var *var = (Var *) pitem->item; NestLoopParam *nlp; - ListCell *lc; + ListCell *lc2; /* If not from a nestloop outer rel, complain */ if (!bms_is_member(var->varno, root->curOuterRels)) elog(ERROR, "non-LATERAL parameter required by subquery"); /* Is this param already listed in root->curOuterParams? */ - foreach(lc, root->curOuterParams) + foreach(lc2, root->curOuterParams) { - nlp = (NestLoopParam *) lfirst(lc); + nlp = (NestLoopParam *) lfirst(lc2); if (nlp->paramno == pitem->paramId) { Assert(equal(var, nlp->paramval)); /* Present, so nothing to do */ break; } } - if (lc == NULL) + if (lc2 == NULL) { /* No, so add it */ nlp = makeNode(NestLoopParam); nlp->paramno = pitem->paramId; nlp->paramval = copyObject(var); root->curOuterParams = lappend(root->curOuterParams, nlp); } } else if (IsA(pitem->item, PlaceHolderVar)) { PlaceHolderVar *phv = (PlaceHolderVar *) pitem->item; NestLoopParam *nlp; - ListCell *lc; + ListCell *lc2; /* If not from a nestloop outer rel, complain */ if (!bms_is_subset(find_placeholder_info(root, phv)->ph_eval_at, root->curOuterRels)) elog(ERROR, "non-LATERAL parameter required by subquery"); /* Is this param already listed in root->curOuterParams? */ - foreach(lc, root->curOuterParams) + foreach(lc2, root->curOuterParams) { - nlp = (NestLoopParam *) lfirst(lc); + nlp = (NestLoopParam *) lfirst(lc2); if (nlp->paramno == pitem->paramId) { Assert(equal(phv, nlp->paramval)); /* Present, so nothing to do */ break; } } - if (lc == NULL) + if (lc2 == NULL) { /* No, so add it */ nlp = makeNode(NestLoopParam); nlp->paramno = pitem->paramId; nlp->paramval = (Var *) copyObject(phv); root->curOuterParams = lappend(root->curOuterParams, nlp); } } else elog(ERROR, "unexpected type of subquery parameter"); } } /* * Identify any NestLoopParams that should be supplied by a NestLoop plan * node with the specified lefthand rels. Remove them from the active * root->curOuterParams list and return them as the result list. */ List * identify_current_nestloop_params(PlannerInfo *root, Relids leftrelids) { List *result; diff --git a/src/backend/parser/parse_clause.c b/src/backend/parser/parse_clause.c index b85fbebd00e..53a17ac3f6a 100644 --- a/src/backend/parser/parse_clause.c +++ b/src/backend/parser/parse_clause.c @@ -520,49 +520,49 @@ transformRangeFunction(ParseState *pstate, RangeFunction *r) * likely expecting an un-tweaked function call. * * Note: the transformation changes a non-schema-qualified unnest() * function name into schema-qualified pg_catalog.unnest(). This * choice is also a bit debatable, but it seems reasonable to force * use of built-in unnest() when we make this transformation. */ if (IsA(fexpr, FuncCall)) { FuncCall *fc = (FuncCall *) fexpr; if (list_length(fc->funcname) == 1 && strcmp(strVal(linitial(fc->funcname)), "unnest") == 0 && list_length(fc->args) > 1 && fc->agg_order == NIL && fc->agg_filter == NULL && fc->over == NULL && !fc->agg_star && !fc->agg_distinct && !fc->func_variadic && coldeflist == NIL) { - ListCell *lc; + ListCell *lc2; - foreach(lc, fc->args) + foreach(lc2, fc->args) { - Node *arg = (Node *) lfirst(lc); + Node *arg = (Node *) lfirst(lc2); FuncCall *newfc; last_srf = pstate->p_last_srf; newfc = makeFuncCall(SystemFuncName("unnest"), list_make1(arg), COERCE_EXPLICIT_CALL, fc->location); newfexpr = transformExpr(pstate, (Node *) newfc, EXPR_KIND_FROM_FUNCTION); /* nodeFunctionscan.c requires SRFs to be at top level */ if (pstate->p_last_srf != last_srf && pstate->p_last_srf != newfexpr) ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("set-returning functions must appear at top level of FROM"), parser_errposition(pstate, exprLocation(pstate->p_last_srf)))); funcexprs = lappend(funcexprs, newfexpr); diff --git a/src/backend/statistics/dependencies.c b/src/backend/statistics/dependencies.c index bf698c1fc3f..744bc512b65 100644 --- a/src/backend/statistics/dependencies.c +++ b/src/backend/statistics/dependencies.c @@ -1673,45 +1673,44 @@ dependencies_clauselist_selectivity(PlannerInfo *root, * * XXX We have to do this even when there are no expressions in * clauses, otherwise find_strongest_dependency may fail for stats * with expressions (due to lookup of negative value in bitmap). So we * need to at least filter out those dependencies. Maybe we could do * it in a cheaper way (if there are no expr clauses, we can just * discard all negative attnums without any lookups). */ if (unique_exprs_cnt > 0 || stat->exprs != NIL) { int ndeps = 0; for (i = 0; i < deps->ndeps; i++) { bool skip = false; MVDependency *dep = deps->deps[i]; int j; for (j = 0; j < dep->nattributes; j++) { int idx; Node *expr; - int k; AttrNumber unique_attnum = InvalidAttrNumber; AttrNumber attnum; /* undo the per-statistics offset */ attnum = dep->attributes[j]; /* * For regular attributes we can simply check if it * matches any clause. If there's no matching clause, we * can just ignore it. We need to offset the attnum * though. */ if (AttrNumberIsForUserDefinedAttr(attnum)) { dep->attributes[j] = attnum + attnum_offset; if (!bms_is_member(dep->attributes[j], clauses_attnums)) { skip = true; break; } @@ -1721,53 +1720,53 @@ dependencies_clauselist_selectivity(PlannerInfo *root, /* * the attnum should be a valid system attnum (-1, -2, * ...) */ Assert(AttributeNumberIsValid(attnum)); /* * For expressions, we need to do two translations. First * we have to translate the negative attnum to index in * the list of expressions (in the statistics object). * Then we need to see if there's a matching clause. The * index of the unique expression determines the attnum * (and we offset it). */ idx = -(1 + attnum); /* Is the expression index is valid? */ Assert((idx >= 0) && (idx < list_length(stat->exprs))); expr = (Node *) list_nth(stat->exprs, idx); /* try to find the expression in the unique list */ - for (k = 0; k < unique_exprs_cnt; k++) + for (int m = 0; m < unique_exprs_cnt; m++) { /* * found a matching unique expression, use the attnum * (derived from index of the unique expression) */ - if (equal(unique_exprs[k], expr)) + if (equal(unique_exprs[m], expr)) { - unique_attnum = -(k + 1) + attnum_offset; + unique_attnum = -(m + 1) + attnum_offset; break; } } /* * Found no matching expression, so we can simply skip * this dependency, because there's no chance it will be * fully covered. */ if (unique_attnum == InvalidAttrNumber) { skip = true; break; } /* otherwise remap it to the new attnum */ dep->attributes[j] = unique_attnum; } /* if found a matching dependency, keep it */ if (!skip) { diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c index 6b0a8652622..ba9a568389f 100644 --- a/src/backend/tcop/utility.c +++ b/src/backend/tcop/utility.c @@ -1068,44 +1068,61 @@ standard_ProcessUtility(PlannedStmt *pstmt, ExecSecLabelStmt(stmt); break; } default: /* All other statement types have event trigger support */ ProcessUtilitySlow(pstate, pstmt, queryString, context, params, queryEnv, dest, qc); break; } free_parsestate(pstate); /* * Make effects of commands visible, for instance so that * PreCommit_on_commit_actions() can see them (see for example bug * #15631). */ CommandCounterIncrement(); } +static ObjectAddress +TryExecRefreshMatView(RefreshMatViewStmt *stmt, const char *queryString, + ParamListInfo params, QueryCompletion *qc) +{ + ObjectAddress address; + PG_TRY(); + { + address = ExecRefreshMatView(stmt, queryString, params, qc); + } + PG_FINALLY(); + { + EventTriggerUndoInhibitCommandCollection(); + } + PG_END_TRY(); + return address; +} + /* * The "Slow" variant of ProcessUtility should only receive statements * supported by the event triggers facility. Therefore, we always * perform the trigger support calls if the context allows it. */ static void ProcessUtilitySlow(ParseState *pstate, PlannedStmt *pstmt, const char *queryString, ProcessUtilityContext context, ParamListInfo params, QueryEnvironment *queryEnv, DestReceiver *dest, QueryCompletion *qc) { Node *parsetree = pstmt->utilityStmt; bool isTopLevel = (context == PROCESS_UTILITY_TOPLEVEL); bool isCompleteQuery = (context != PROCESS_UTILITY_SUBCOMMAND); bool needCleanup; bool commandCollected = false; ObjectAddress address; ObjectAddress secondaryObject = InvalidObjectAddress; @@ -1659,54 +1676,48 @@ ProcessUtilitySlow(ParseState *pstate, case T_CreateSeqStmt: address = DefineSequence(pstate, (CreateSeqStmt *) parsetree); break; case T_AlterSeqStmt: address = AlterSequence(pstate, (AlterSeqStmt *) parsetree); break; case T_CreateTableAsStmt: address = ExecCreateTableAs(pstate, (CreateTableAsStmt *) parsetree, params, queryEnv, qc); break; case T_RefreshMatViewStmt: /* * REFRESH CONCURRENTLY executes some DDL commands internally. * Inhibit DDL command collection here to avoid those commands * from showing up in the deparsed command queue. The refresh * command itself is queued, which is enough. */ EventTriggerInhibitCommandCollection(); - PG_TRY(); - { - address = ExecRefreshMatView((RefreshMatViewStmt *) parsetree, - queryString, params, qc); - } - PG_FINALLY(); - { - EventTriggerUndoInhibitCommandCollection(); - } - PG_END_TRY(); + + address = TryExecRefreshMatView((RefreshMatViewStmt *) parsetree, + queryString, params, qc); + break; case T_CreateTrigStmt: address = CreateTrigger((CreateTrigStmt *) parsetree, queryString, InvalidOid, InvalidOid, InvalidOid, InvalidOid, InvalidOid, InvalidOid, NULL, false, false); break; case T_CreatePLangStmt: address = CreateProceduralLanguage((CreatePLangStmt *) parsetree); break; case T_CreateDomainStmt: address = DefineDomain((CreateDomainStmt *) parsetree); break; case T_CreateConversionStmt: address = CreateConversionCommand((CreateConversionStmt *) parsetree); break; case T_CreateCastStmt: diff --git a/src/backend/utils/adt/levenshtein.c b/src/backend/utils/adt/levenshtein.c index 3026cc24311..2e67a90e516 100644 --- a/src/backend/utils/adt/levenshtein.c +++ b/src/backend/utils/adt/levenshtein.c @@ -174,54 +174,54 @@ varstr_levenshtein(const char *source, int slen, * total cost increases by ins_c + del_c for each move right. */ int slack_d = max_d - min_theo_d; int best_column = net_inserts < 0 ? -net_inserts : 0; stop_column = best_column + (slack_d / (ins_c + del_c)) + 1; if (stop_column > m) stop_column = m + 1; } } #endif /* * In order to avoid calling pg_mblen() repeatedly on each character in s, * we cache all the lengths before starting the main loop -- but if all * the characters in both strings are single byte, then we skip this and * use a fast-path in the main loop. If only one string contains * multi-byte characters, we still build the array, so that the fast-path * needn't deal with the case where the array hasn't been initialized. */ if (m != slen || n != tlen) { - int i; + int k; const char *cp = source; s_char_len = (int *) palloc((m + 1) * sizeof(int)); - for (i = 0; i < m; ++i) + for (k = 0; k < m; ++k) { - s_char_len[i] = pg_mblen(cp); - cp += s_char_len[i]; + s_char_len[k] = pg_mblen(cp); + cp += s_char_len[k]; } - s_char_len[i] = 0; + s_char_len[k] = 0; } /* One more cell for initialization column and row. */ ++m; ++n; /* Previous and current rows of notional array. */ prev = (int *) palloc(2 * m * sizeof(int)); curr = prev + m; /* * To transform the first i characters of s into the first 0 characters of * t, we must perform i deletions. */ for (i = START_COLUMN; i < STOP_COLUMN; i++) prev[i] = i * del_c; /* Loop through rows of the notional array */ for (y = target, j = 1; j < n; j++) { int *temp; const char *x = source; diff --git a/src/pl/plpgsql/src/pl_funcs.c b/src/pl/plpgsql/src/pl_funcs.c index 93d9cef06ba..8d7b6b58c05 100644 --- a/src/pl/plpgsql/src/pl_funcs.c +++ b/src/pl/plpgsql/src/pl_funcs.c @@ -1628,51 +1628,50 @@ plpgsql_dumptree(PLpgSQL_function *func) { printf(" DEFAULT "); dump_expr(var->default_val); printf("\n"); } if (var->cursor_explicit_expr != NULL) { if (var->cursor_explicit_argrow >= 0) printf(" CURSOR argument row %d\n", var->cursor_explicit_argrow); printf(" CURSOR IS "); dump_expr(var->cursor_explicit_expr); printf("\n"); } if (var->promise != PLPGSQL_PROMISE_NONE) printf(" PROMISE %d\n", (int) var->promise); } break; case PLPGSQL_DTYPE_ROW: { PLpgSQL_row *row = (PLpgSQL_row *) d; - int i; printf("ROW %-16s fields", row->refname); - for (i = 0; i < row->nfields; i++) + for (int j = 0; j < row->nfields; j++) { - printf(" %s=var %d", row->fieldnames[i], - row->varnos[i]); + printf(" %s=var %d", row->fieldnames[j], + row->varnos[j]); } printf("\n"); } break; case PLPGSQL_DTYPE_REC: printf("REC %-16s typoid %u\n", ((PLpgSQL_rec *) d)->refname, ((PLpgSQL_rec *) d)->rectypeid); if (((PLpgSQL_rec *) d)->isconst) printf(" CONSTANT\n"); if (((PLpgSQL_rec *) d)->notnull) printf(" NOT NULL\n"); if (((PLpgSQL_rec *) d)->default_val != NULL) { printf(" DEFAULT "); dump_expr(((PLpgSQL_rec *) d)->default_val); printf("\n"); } break; case PLPGSQL_DTYPE_RECFIELD: printf("RECFIELD %-16s of REC %d\n", ((PLpgSQL_recfield *) d)->fieldname,