aboutsummaryrefslogtreecommitdiff
path: root/src/backend/utils/activity/pgstat_relation.c
diff options
context:
space:
mode:
Diffstat (limited to 'src/backend/utils/activity/pgstat_relation.c')
-rw-r--r--src/backend/utils/activity/pgstat_relation.c592
1 files changed, 302 insertions, 290 deletions
diff --git a/src/backend/utils/activity/pgstat_relation.c b/src/backend/utils/activity/pgstat_relation.c
index 51a87b66739..bec190c5897 100644
--- a/src/backend/utils/activity/pgstat_relation.c
+++ b/src/backend/utils/activity/pgstat_relation.c
@@ -19,6 +19,7 @@
#include "access/twophase_rmgr.h"
#include "access/xact.h"
+#include "catalog/partition.h"
#include "postmaster/autovacuum.h"
#include "utils/memutils.h"
#include "utils/pgstat_internal.h"
@@ -26,38 +27,6 @@
#include "utils/timestamp.h"
-/*
- * Structures in which backends store per-table info that's waiting to be
- * sent to the collector.
- *
- * NOTE: once allocated, TabStatusArray structures are never moved or deleted
- * for the life of the backend. Also, we zero out the t_id fields of the
- * contained PgStat_TableStatus structs whenever they are not actively in use.
- * This allows relcache pgstat_info pointers to be treated as long-lived data,
- * avoiding repeated searches in pgstat_init_relation() when a relation is
- * repeatedly opened during a transaction.
- */
-#define TABSTAT_QUANTUM 100 /* we alloc this many at a time */
-
-
-typedef struct TabStatusArray
-{
- struct TabStatusArray *tsa_next; /* link to next array, if any */
- int tsa_used; /* # entries currently used */
- PgStat_TableStatus tsa_entries[TABSTAT_QUANTUM]; /* per-table data */
-} TabStatusArray;
-
-static TabStatusArray *pgStatTabList = NULL;
-
-/*
- * pgStatTabHash entry: map from relation OID to PgStat_TableStatus pointer
- */
-typedef struct TabStatHashEntry
-{
- Oid t_id;
- PgStat_TableStatus *tsa_entry;
-} TabStatHashEntry;
-
/* Record that's written to 2PC state file when pgstat state is persisted */
typedef struct TwoPhasePgStatRecord
{
@@ -74,8 +43,7 @@ typedef struct TwoPhasePgStatRecord
} TwoPhasePgStatRecord;
-static PgStat_TableStatus *get_tabstat_entry(Oid rel_id, bool isshared);
-static void pgstat_send_tabstat(PgStat_MsgTabstat *tsmsg, TimestampTz now);
+static PgStat_TableStatus *pgstat_prep_relation_pending(Oid rel_id, bool isshared);
static void add_tabstat_xact_level(PgStat_TableStatus *pgstat_info, int nest_level);
static void ensure_tabstat_xact_level(PgStat_TableStatus *pgstat_info);
static void save_truncdrop_counters(PgStat_TableXactStatus *trans, bool is_drop);
@@ -83,19 +51,6 @@ static void restore_truncdrop_counters(PgStat_TableXactStatus *trans);
/*
- * Indicates if backend has some relation stats that it hasn't yet
- * sent to the collector.
- */
-bool have_relation_stats;
-
-
-/*
- * Hash table for O(1) t_id -> tsa_entry lookup
- */
-static HTAB *pgStatTabHash = NULL;
-
-
-/*
* Copy stats between relations. This is used for things like REINDEX
* CONCURRENTLY.
*/
@@ -103,43 +58,39 @@ void
pgstat_copy_relation_stats(Relation dst, Relation src)
{
PgStat_StatTabEntry *srcstats;
+ PgStatShared_Relation *dstshstats;
+ PgStat_EntryRef *dst_ref;
- srcstats = pgstat_fetch_stat_tabentry(RelationGetRelid(src));
-
+ srcstats = pgstat_fetch_stat_tabentry_ext(src->rd_rel->relisshared,
+ RelationGetRelid(src));
if (!srcstats)
return;
- if (pgstat_should_count_relation(dst))
- {
- /*
- * XXX: temporarily this does not actually quite do what the name
- * says, and just copy index related fields. A subsequent commit will
- * do more.
- */
-
- dst->pgstat_info->t_counts.t_numscans = srcstats->numscans;
- dst->pgstat_info->t_counts.t_tuples_returned = srcstats->tuples_returned;
- dst->pgstat_info->t_counts.t_tuples_fetched = srcstats->tuples_fetched;
- dst->pgstat_info->t_counts.t_blocks_fetched = srcstats->blocks_fetched;
- dst->pgstat_info->t_counts.t_blocks_hit = srcstats->blocks_hit;
-
- /* the data will be sent by the next pgstat_report_stat() call */
- }
+ dst_ref = pgstat_get_entry_ref_locked(PGSTAT_KIND_RELATION,
+ dst->rd_rel->relisshared ? InvalidOid : MyDatabaseId,
+ RelationGetRelid(dst),
+ false);
+
+ dstshstats = (PgStatShared_Relation *) dst_ref->shared_stats;
+ dstshstats->stats = *srcstats;
+
+ pgstat_unlock_entry(dst_ref);
}
/*
- * Initialize a relcache entry to count access statistics.
- * Called whenever a relation is opened.
+ * Initialize a relcache entry to count access statistics. Called whenever a
+ * relation is opened.
*
- * We assume that a relcache entry's pgstat_info field is zeroed by
- * relcache.c when the relcache entry is made; thereafter it is long-lived
- * data. We can avoid repeated searches of the TabStatus arrays when the
- * same relation is touched repeatedly within a transaction.
+ * We assume that a relcache entry's pgstat_info field is zeroed by relcache.c
+ * when the relcache entry is made; thereafter it is long-lived data.
+ *
+ * This does not create a reference to a stats entry in shared memory, nor
+ * allocate memory for the pending stats. That happens in
+ * pgstat_assoc_relation().
*/
void
pgstat_init_relation(Relation rel)
{
- Oid rel_id = rel->rd_id;
char relkind = rel->rd_rel->relkind;
/*
@@ -147,27 +98,68 @@ pgstat_init_relation(Relation rel)
*/
if (!RELKIND_HAS_STORAGE(relkind) && relkind != RELKIND_PARTITIONED_TABLE)
{
+ rel->pgstat_enabled = false;
rel->pgstat_info = NULL;
return;
}
- if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+ if (!pgstat_track_counts)
{
+ if (rel->pgstat_info)
+ pgstat_unlink_relation(rel);
+
/* We're not counting at all */
+ rel->pgstat_enabled = false;
rel->pgstat_info = NULL;
return;
}
- /*
- * If we already set up this relation in the current transaction, nothing
- * to do.
- */
- if (rel->pgstat_info != NULL &&
- rel->pgstat_info->t_id == rel_id)
- return;
+ rel->pgstat_enabled = true;
+}
+
+/*
+ * Prepare for statistics for this relation to be collected.
+ *
+ * This ensures we have a reference to the stats entry before stats can be
+ * generated. That is important because a relation drop in another connection
+ * could otherwise lead to the stats entry being dropped, which then later
+ * would get recreated when flushing stats.
+ *
+ * This is separate from pgstat_init_relation() as it is not uncommon for
+ * relcache entries to be opened without ever getting stats reported.
+ */
+void
+pgstat_assoc_relation(Relation rel)
+{
+ Assert(rel->pgstat_enabled);
+ Assert(rel->pgstat_info == NULL);
/* Else find or make the PgStat_TableStatus entry, and update link */
- rel->pgstat_info = get_tabstat_entry(rel_id, rel->rd_rel->relisshared);
+ rel->pgstat_info = pgstat_prep_relation_pending(RelationGetRelid(rel),
+ rel->rd_rel->relisshared);
+
+ /* don't allow link a stats to multiple relcache entries */
+ Assert(rel->pgstat_info->relation == NULL);
+
+ /* mark this relation as the owner */
+ rel->pgstat_info->relation = rel;
+}
+
+/*
+ * Break the mutual link between a relcache entry and pending stats entry.
+ * This must be called whenever one end of the link is removed.
+ */
+void
+pgstat_unlink_relation(Relation rel)
+{
+ /* remove the link to stats info if any */
+ if (rel->pgstat_info == NULL)
+ return;
+
+ /* link sanity check */
+ Assert(rel->pgstat_info->relation == rel);
+ rel->pgstat_info->relation = NULL;
+ rel->pgstat_info = NULL;
}
/*
@@ -187,9 +179,26 @@ pgstat_create_relation(Relation rel)
void
pgstat_drop_relation(Relation rel)
{
+ int nest_level = GetCurrentTransactionNestLevel();
+ PgStat_TableStatus *pgstat_info = rel->pgstat_info;
+
pgstat_drop_transactional(PGSTAT_KIND_RELATION,
rel->rd_rel->relisshared ? InvalidOid : MyDatabaseId,
RelationGetRelid(rel));
+
+ /*
+ * Transactionally set counters to 0. That ensures that accesses to
+ * pg_stat_xact_all_tables inside the transaction show 0.
+ */
+ if (pgstat_info &&
+ pgstat_info->trans != NULL &&
+ pgstat_info->trans->nest_level == nest_level)
+ {
+ save_truncdrop_counters(pgstat_info->trans, true);
+ pgstat_info->trans->tuples_inserted = 0;
+ pgstat_info->trans->tuples_updated = 0;
+ pgstat_info->trans->tuples_deleted = 0;
+ }
}
/*
@@ -199,19 +208,52 @@ void
pgstat_report_vacuum(Oid tableoid, bool shared,
PgStat_Counter livetuples, PgStat_Counter deadtuples)
{
- PgStat_MsgVacuum msg;
+ PgStat_EntryRef *entry_ref;
+ PgStatShared_Relation *shtabentry;
+ PgStat_StatTabEntry *tabentry;
+ Oid dboid = (shared ? InvalidOid : MyDatabaseId);
+ TimestampTz ts;
- if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+ if (!pgstat_track_counts)
return;
- pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_VACUUM);
- msg.m_databaseid = shared ? InvalidOid : MyDatabaseId;
- msg.m_tableoid = tableoid;
- msg.m_autovacuum = IsAutoVacuumWorkerProcess();
- msg.m_vacuumtime = GetCurrentTimestamp();
- msg.m_live_tuples = livetuples;
- msg.m_dead_tuples = deadtuples;
- pgstat_send(&msg, sizeof(msg));
+ /* Store the data in the table's hash table entry. */
+ ts = GetCurrentTimestamp();
+
+ /* block acquiring lock for the same reason as pgstat_report_autovac() */
+ entry_ref = pgstat_get_entry_ref_locked(PGSTAT_KIND_RELATION,
+ dboid, tableoid, false);
+
+ shtabentry = (PgStatShared_Relation *) entry_ref->shared_stats;
+ tabentry = &shtabentry->stats;
+
+ tabentry->n_live_tuples = livetuples;
+ tabentry->n_dead_tuples = deadtuples;
+
+ /*
+ * It is quite possible that a non-aggressive VACUUM ended up skipping
+ * various pages, however, we'll zero the insert counter here regardless.
+ * It's currently used only to track when we need to perform an "insert"
+ * autovacuum, which are mainly intended to freeze newly inserted tuples.
+ * Zeroing this may just mean we'll not try to vacuum the table again
+ * until enough tuples have been inserted to trigger another insert
+ * autovacuum. An anti-wraparound autovacuum will catch any persistent
+ * stragglers.
+ */
+ tabentry->inserts_since_vacuum = 0;
+
+ if (IsAutoVacuumWorkerProcess())
+ {
+ tabentry->autovac_vacuum_timestamp = ts;
+ tabentry->autovac_vacuum_count++;
+ }
+ else
+ {
+ tabentry->vacuum_timestamp = ts;
+ tabentry->vacuum_count++;
+ }
+
+ pgstat_unlock_entry(entry_ref);
}
/*
@@ -225,9 +267,12 @@ pgstat_report_analyze(Relation rel,
PgStat_Counter livetuples, PgStat_Counter deadtuples,
bool resetcounter)
{
- PgStat_MsgAnalyze msg;
+ PgStat_EntryRef *entry_ref;
+ PgStatShared_Relation *shtabentry;
+ PgStat_StatTabEntry *tabentry;
+ Oid dboid = (rel->rd_rel->relisshared ? InvalidOid : MyDatabaseId);
- if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+ if (!pgstat_track_counts)
return;
/*
@@ -259,15 +304,39 @@ pgstat_report_analyze(Relation rel,
deadtuples = Max(deadtuples, 0);
}
- pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_ANALYZE);
- msg.m_databaseid = rel->rd_rel->relisshared ? InvalidOid : MyDatabaseId;
- msg.m_tableoid = RelationGetRelid(rel);
- msg.m_autovacuum = IsAutoVacuumWorkerProcess();
- msg.m_resetcounter = resetcounter;
- msg.m_analyzetime = GetCurrentTimestamp();
- msg.m_live_tuples = livetuples;
- msg.m_dead_tuples = deadtuples;
- pgstat_send(&msg, sizeof(msg));
+ /* block acquiring lock for the same reason as pgstat_report_autovac() */
+ entry_ref = pgstat_get_entry_ref_locked(PGSTAT_KIND_RELATION, dboid,
+ RelationGetRelid(rel),
+ false);
+ /* can't get dropped while accessed */
+ Assert(entry_ref != NULL && entry_ref->shared_stats != NULL);
+
+ shtabentry = (PgStatShared_Relation *) entry_ref->shared_stats;
+ tabentry = &shtabentry->stats;
+
+ tabentry->n_live_tuples = livetuples;
+ tabentry->n_dead_tuples = deadtuples;
+
+ /*
+ * If commanded, reset changes_since_analyze to zero. This forgets any
+ * changes that were committed while the ANALYZE was in progress, but we
+ * have no good way to estimate how many of those there were.
+ */
+ if (resetcounter)
+ tabentry->changes_since_analyze = 0;
+
+ if (IsAutoVacuumWorkerProcess())
+ {
+ tabentry->autovac_analyze_timestamp = GetCurrentTimestamp();
+ tabentry->autovac_analyze_count++;
+ }
+ else
+ {
+ tabentry->analyze_timestamp = GetCurrentTimestamp();
+ tabentry->analyze_count++;
+ }
+
+ pgstat_unlock_entry(entry_ref);
}
/*
@@ -357,29 +426,60 @@ pgstat_update_heap_dead_tuples(Relation rel, int delta)
}
/*
+ * Support function for the SQL-callable pgstat* functions. Returns
+ * the collected statistics for one table or NULL. NULL doesn't mean
+ * that the table doesn't exist, just that there are no statistics, so the
+ * caller is better off to report ZERO instead.
+ */
+PgStat_StatTabEntry *
+pgstat_fetch_stat_tabentry(Oid relid)
+{
+ PgStat_StatTabEntry *tabentry;
+
+ tabentry = pgstat_fetch_stat_tabentry_ext(false, relid);
+ if (tabentry != NULL)
+ return tabentry;
+
+ /*
+ * If we didn't find it, maybe it's a shared table.
+ */
+ tabentry = pgstat_fetch_stat_tabentry_ext(true, relid);
+ return tabentry;
+}
+
+/*
+ * More efficient version of pgstat_fetch_stat_tabentry(), allowing to specify
+ * whether the to-be-accessed table is a shared relation or not.
+ */
+PgStat_StatTabEntry *
+pgstat_fetch_stat_tabentry_ext(bool shared, Oid reloid)
+{
+ Oid dboid = (shared ? InvalidOid : MyDatabaseId);
+
+ return (PgStat_StatTabEntry *)
+ pgstat_fetch_entry(PGSTAT_KIND_RELATION, dboid, reloid);
+}
+
+/*
* find any existing PgStat_TableStatus entry for rel
*
- * If no entry, return NULL, don't create a new one
+ * Find any existing PgStat_TableStatus entry for rel_id in the current
+ * database. If not found, try finding from shared tables.
*
- * Note: if we got an error in the most recent execution of pgstat_report_stat,
- * it's possible that an entry exists but there's no hashtable entry for it.
- * That's okay, we'll treat this case as "doesn't exist".
+ * If no entry found, return NULL, don't create a new one
*/
PgStat_TableStatus *
find_tabstat_entry(Oid rel_id)
{
- TabStatHashEntry *hash_entry;
+ PgStat_EntryRef *entry_ref;
- /* If hashtable doesn't exist, there are no entries at all */
- if (!pgStatTabHash)
- return NULL;
+ entry_ref = pgstat_fetch_pending_entry(PGSTAT_KIND_RELATION, MyDatabaseId, rel_id);
+ if (!entry_ref)
+ entry_ref = pgstat_fetch_pending_entry(PGSTAT_KIND_RELATION, InvalidOid, rel_id);
- hash_entry = hash_search(pgStatTabHash, &rel_id, HASH_FIND, NULL);
- if (!hash_entry)
- return NULL;
-
- /* Note that this step could also return NULL, but that's correct */
- return hash_entry->tsa_entry;
+ if (entry_ref)
+ return entry_ref->pending;
+ return NULL;
}
/*
@@ -536,7 +636,7 @@ AtPrepare_PgStat_Relations(PgStat_SubXactStatus *xact_state)
for (trans = xact_state->first; trans != NULL; trans = trans->next)
{
- PgStat_TableStatus *tabstat;
+ PgStat_TableStatus *tabstat PG_USED_FOR_ASSERTS_ONLY;
TwoPhasePgStatRecord record;
Assert(trans->nest_level == 1);
@@ -594,7 +694,7 @@ pgstat_twophase_postcommit(TransactionId xid, uint16 info,
PgStat_TableStatus *pgstat_info;
/* Find or create a tabstat entry for the rel */
- pgstat_info = get_tabstat_entry(rec->t_id, rec->t_shared);
+ pgstat_info = pgstat_prep_relation_pending(rec->t_id, rec->t_shared);
/* Same math as in AtEOXact_PgStat, commit case */
pgstat_info->t_counts.t_tuples_inserted += rec->tuples_inserted;
@@ -630,7 +730,7 @@ pgstat_twophase_postabort(TransactionId xid, uint16 info,
PgStat_TableStatus *pgstat_info;
/* Find or create a tabstat entry for the rel */
- pgstat_info = get_tabstat_entry(rec->t_id, rec->t_shared);
+ pgstat_info = pgstat_prep_relation_pending(rec->t_id, rec->t_shared);
/* Same math as in AtEOXact_PgStat, abort case */
if (rec->t_truncdropped)
@@ -647,204 +747,116 @@ pgstat_twophase_postabort(TransactionId xid, uint16 info,
}
/*
- * Subroutine for pgstat_report_stat: Send relation statistics
+ * Flush out pending stats for the entry
+ *
+ * If nowait is true, this function returns false if lock could not
+ * immediately acquired, otherwise true is returned.
+ *
+ * Some of the stats are copied to the corresponding pending database stats
+ * entry when successfully flushing.
*/
-void
-pgstat_send_tabstats(TimestampTz now, bool disconnect)
+bool
+pgstat_relation_flush_cb(PgStat_EntryRef *entry_ref, bool nowait)
{
- /* we assume this inits to all zeroes: */
static const PgStat_TableCounts all_zeroes;
- PgStat_MsgTabstat regular_msg;
- PgStat_MsgTabstat shared_msg;
- TabStatusArray *tsa;
- int i;
+ Oid dboid;
+ PgStat_TableStatus *lstats; /* pending stats entry */
+ PgStatShared_Relation *shtabstats;
+ PgStat_StatTabEntry *tabentry; /* table entry of shared stats */
+ PgStat_StatDBEntry *dbentry; /* pending database entry */
- /*
- * Destroy pgStatTabHash before we start invalidating PgStat_TableEntry
- * entries it points to. (Should we fail partway through the loop below,
- * it's okay to have removed the hashtable already --- the only
- * consequence is we'd get multiple entries for the same table in the
- * pgStatTabList, and that's safe.)
- */
- if (pgStatTabHash)
- hash_destroy(pgStatTabHash);
- pgStatTabHash = NULL;
+ dboid = entry_ref->shared_entry->key.dboid;
+ lstats = (PgStat_TableStatus *) entry_ref->pending;
+ shtabstats = (PgStatShared_Relation *) entry_ref->shared_stats;
/*
- * Scan through the TabStatusArray struct(s) to find tables that actually
- * have counts, and build messages to send. We have to separate shared
- * relations from regular ones because the databaseid field in the message
- * header has to depend on that.
+ * Ignore entries that didn't accumulate any actual counts, such as
+ * indexes that were opened by the planner but not used.
*/
- regular_msg.m_databaseid = MyDatabaseId;
- shared_msg.m_databaseid = InvalidOid;
- regular_msg.m_nentries = 0;
- shared_msg.m_nentries = 0;
-
- for (tsa = pgStatTabList; tsa != NULL; tsa = tsa->tsa_next)
+ if (memcmp(&lstats->t_counts, &all_zeroes,
+ sizeof(PgStat_TableCounts)) == 0)
{
- for (i = 0; i < tsa->tsa_used; i++)
- {
- PgStat_TableStatus *entry = &tsa->tsa_entries[i];
- PgStat_MsgTabstat *this_msg;
- PgStat_TableEntry *this_ent;
+ return true;
+ }
- /* Shouldn't have any pending transaction-dependent counts */
- Assert(entry->trans == NULL);
+ if (!pgstat_lock_entry(entry_ref, nowait))
+ return false;
- /*
- * Ignore entries that didn't accumulate any actual counts, such
- * as indexes that were opened by the planner but not used.
- */
- if (memcmp(&entry->t_counts, &all_zeroes,
- sizeof(PgStat_TableCounts)) == 0)
- continue;
+ /* add the values to the shared entry. */
+ tabentry = &shtabstats->stats;
- /*
- * OK, insert data into the appropriate message, and send if full.
- */
- this_msg = entry->t_shared ? &shared_msg : &regular_msg;
- this_ent = &this_msg->m_entry[this_msg->m_nentries];
- this_ent->t_id = entry->t_id;
- memcpy(&this_ent->t_counts, &entry->t_counts,
- sizeof(PgStat_TableCounts));
- if (++this_msg->m_nentries >= PGSTAT_NUM_TABENTRIES)
- {
- pgstat_send_tabstat(this_msg, now);
- this_msg->m_nentries = 0;
- }
- }
- /* zero out PgStat_TableStatus structs after use */
- MemSet(tsa->tsa_entries, 0,
- tsa->tsa_used * sizeof(PgStat_TableStatus));
- tsa->tsa_used = 0;
- }
+ tabentry->numscans += lstats->t_counts.t_numscans;
+ tabentry->tuples_returned += lstats->t_counts.t_tuples_returned;
+ tabentry->tuples_fetched += lstats->t_counts.t_tuples_fetched;
+ tabentry->tuples_inserted += lstats->t_counts.t_tuples_inserted;
+ tabentry->tuples_updated += lstats->t_counts.t_tuples_updated;
+ tabentry->tuples_deleted += lstats->t_counts.t_tuples_deleted;
+ tabentry->tuples_hot_updated += lstats->t_counts.t_tuples_hot_updated;
/*
- * Send partial messages. Make sure that any pending xact commit/abort
- * and connection stats get counted, even if there are no table stats to
- * send.
+ * If table was truncated/dropped, first reset the live/dead counters.
*/
- if (regular_msg.m_nentries > 0 ||
- pgStatXactCommit > 0 || pgStatXactRollback > 0 || disconnect)
- pgstat_send_tabstat(&regular_msg, now);
- if (shared_msg.m_nentries > 0)
- pgstat_send_tabstat(&shared_msg, now);
+ if (lstats->t_counts.t_truncdropped)
+ {
+ tabentry->n_live_tuples = 0;
+ tabentry->n_dead_tuples = 0;
+ tabentry->inserts_since_vacuum = 0;
+ }
- have_relation_stats = false;
+ tabentry->n_live_tuples += lstats->t_counts.t_delta_live_tuples;
+ tabentry->n_dead_tuples += lstats->t_counts.t_delta_dead_tuples;
+ tabentry->changes_since_analyze += lstats->t_counts.t_changed_tuples;
+ tabentry->inserts_since_vacuum += lstats->t_counts.t_tuples_inserted;
+ tabentry->blocks_fetched += lstats->t_counts.t_blocks_fetched;
+ tabentry->blocks_hit += lstats->t_counts.t_blocks_hit;
+
+ /* Clamp n_live_tuples in case of negative delta_live_tuples */
+ tabentry->n_live_tuples = Max(tabentry->n_live_tuples, 0);
+ /* Likewise for n_dead_tuples */
+ tabentry->n_dead_tuples = Max(tabentry->n_dead_tuples, 0);
+
+ pgstat_unlock_entry(entry_ref);
+
+ /* The entry was successfully flushed, add the same to database stats */
+ dbentry = pgstat_prep_database_pending(dboid);
+ dbentry->n_tuples_returned += lstats->t_counts.t_tuples_returned;
+ dbentry->n_tuples_fetched += lstats->t_counts.t_tuples_fetched;
+ dbentry->n_tuples_inserted += lstats->t_counts.t_tuples_inserted;
+ dbentry->n_tuples_updated += lstats->t_counts.t_tuples_updated;
+ dbentry->n_tuples_deleted += lstats->t_counts.t_tuples_deleted;
+ dbentry->n_blocks_fetched += lstats->t_counts.t_blocks_fetched;
+ dbentry->n_blocks_hit += lstats->t_counts.t_blocks_hit;
+
+ return true;
}
-/*
- * Subroutine for pgstat_send_tabstats: finish and send one tabstat message
- */
-static void
-pgstat_send_tabstat(PgStat_MsgTabstat *tsmsg, TimestampTz now)
+void
+pgstat_relation_delete_pending_cb(PgStat_EntryRef *entry_ref)
{
- int n;
- int len;
-
- /* It's unlikely we'd get here with no socket, but maybe not impossible */
- if (pgStatSock == PGINVALID_SOCKET)
- return;
+ PgStat_TableStatus *pending = (PgStat_TableStatus *) entry_ref->pending;
- /*
- * Report and reset accumulated xact commit/rollback and I/O timings
- * whenever we send a normal tabstat message
- */
- pgstat_update_dbstats(tsmsg, now);
-
- n = tsmsg->m_nentries;
- len = offsetof(PgStat_MsgTabstat, m_entry[0]) +
- n * sizeof(PgStat_TableEntry);
-
- pgstat_setheader(&tsmsg->m_hdr, PGSTAT_MTYPE_TABSTAT);
- pgstat_send(tsmsg, len);
+ if (pending->relation)
+ pgstat_unlink_relation(pending->relation);
}
/*
- * find or create a PgStat_TableStatus entry for rel
+ * Find or create a PgStat_TableStatus entry for rel. New entry is created and
+ * initialized if not exists.
*/
static PgStat_TableStatus *
-get_tabstat_entry(Oid rel_id, bool isshared)
+pgstat_prep_relation_pending(Oid rel_id, bool isshared)
{
- TabStatHashEntry *hash_entry;
- PgStat_TableStatus *entry;
- TabStatusArray *tsa;
- bool found;
-
- pgstat_assert_is_up();
+ PgStat_EntryRef *entry_ref;
+ PgStat_TableStatus *pending;
- have_relation_stats = true;
-
- /*
- * Create hash table if we don't have it already.
- */
- if (pgStatTabHash == NULL)
- {
- HASHCTL ctl;
-
- ctl.keysize = sizeof(Oid);
- ctl.entrysize = sizeof(TabStatHashEntry);
-
- pgStatTabHash = hash_create("pgstat TabStatusArray lookup hash table",
- TABSTAT_QUANTUM,
- &ctl,
- HASH_ELEM | HASH_BLOBS);
- }
-
- /*
- * Find an entry or create a new one.
- */
- hash_entry = hash_search(pgStatTabHash, &rel_id, HASH_ENTER, &found);
- if (!found)
- {
- /* initialize new entry with null pointer */
- hash_entry->tsa_entry = NULL;
- }
-
- /*
- * If entry is already valid, we're done.
- */
- if (hash_entry->tsa_entry)
- return hash_entry->tsa_entry;
-
- /*
- * Locate the first pgStatTabList entry with free space, making a new list
- * entry if needed. Note that we could get an OOM failure here, but if so
- * we have left the hashtable and the list in a consistent state.
- */
- if (pgStatTabList == NULL)
- {
- /* Set up first pgStatTabList entry */
- pgStatTabList = (TabStatusArray *)
- MemoryContextAllocZero(TopMemoryContext,
- sizeof(TabStatusArray));
- }
-
- tsa = pgStatTabList;
- while (tsa->tsa_used >= TABSTAT_QUANTUM)
- {
- if (tsa->tsa_next == NULL)
- tsa->tsa_next = (TabStatusArray *)
- MemoryContextAllocZero(TopMemoryContext,
- sizeof(TabStatusArray));
- tsa = tsa->tsa_next;
- }
-
- /*
- * Allocate a PgStat_TableStatus entry within this list entry. We assume
- * the entry was already zeroed, either at creation or after last use.
- */
- entry = &tsa->tsa_entries[tsa->tsa_used++];
- entry->t_id = rel_id;
- entry->t_shared = isshared;
-
- /*
- * Now we can fill the entry in pgStatTabHash.
- */
- hash_entry->tsa_entry = entry;
+ entry_ref = pgstat_prep_pending_entry(PGSTAT_KIND_RELATION,
+ isshared ? InvalidOid : MyDatabaseId,
+ rel_id, NULL);
+ pending = entry_ref->pending;
+ pending->t_id = rel_id;
+ pending->t_shared = isshared;
- return entry;
+ return pending;
}
/*