#include "access/htup_details.h"
#include "catalog/pg_type.h"
#include "funcapi.h"
+#include "port/pg_numa.h"
#include "storage/buf_internals.h"
#include "storage/bufmgr.h"
#define NUM_BUFFERCACHE_SUMMARY_ELEM 5
#define NUM_BUFFERCACHE_USAGE_COUNTS_ELEM 4
+#define NUM_BUFFERCACHE_NUMA_ELEM 3
+
PG_MODULE_MAGIC_EXT(
.name = "pg_buffercache",
.version = PG_VERSION
BufferCachePagesRec *record;
} BufferCachePagesContext;
+/*
+ * Record structure holding the to be exposed cache data.
+ */
+typedef struct
+{
+ uint32 bufferid;
+ int32 page_num;
+ int32 numa_node;
+} BufferCacheNumaRec;
+
+/*
+ * Function context for data persisting over repeated calls.
+ */
+typedef struct
+{
+ TupleDesc tupdesc;
+ int buffers_per_page;
+ int pages_per_buffer;
+ int os_page_size;
+ BufferCacheNumaRec *record;
+} BufferCacheNumaContext;
+
/*
* Function returning data from the shared buffer cache - buffer number,
* relation node/tablespace/database/blocknum and dirty indicator.
*/
PG_FUNCTION_INFO_V1(pg_buffercache_pages);
+PG_FUNCTION_INFO_V1(pg_buffercache_numa_pages);
PG_FUNCTION_INFO_V1(pg_buffercache_summary);
PG_FUNCTION_INFO_V1(pg_buffercache_usage_counts);
PG_FUNCTION_INFO_V1(pg_buffercache_evict);
+
+/* Only need to touch memory once per backend process lifetime */
+static bool firstNumaTouch = true;
+
+
Datum
pg_buffercache_pages(PG_FUNCTION_ARGS)
{
SRF_RETURN_DONE(funcctx);
}
+/*
+ * Inquire about NUMA memory mappings for shared buffers.
+ *
+ * Returns NUMA node ID for each memory page used by the buffer. Buffers may
+ * be smaller or larger than OS memory pages. For each buffer we return one
+ * entry for each memory page used by the buffer (it fhe buffer is smaller,
+ * it only uses a part of one memory page).
+ *
+ * We expect both sizes (for buffers and memory pages) to be a power-of-2, so
+ * one is always a multiple of the other.
+ *
+ * In order to get reliable results we also need to touch memory pages, so
+ * that the inquiry about NUMA memory node doesn't return -2 (which indicates
+ * unmapped/unallocated pages).
+ */
+Datum
+pg_buffercache_numa_pages(PG_FUNCTION_ARGS)
+{
+ FuncCallContext *funcctx;
+ MemoryContext oldcontext;
+ BufferCacheNumaContext *fctx; /* User function context. */
+ TupleDesc tupledesc;
+ TupleDesc expected_tupledesc;
+ HeapTuple tuple;
+ Datum result;
+
+ if (SRF_IS_FIRSTCALL())
+ {
+ int i,
+ idx;
+ Size os_page_size;
+ void **os_page_ptrs;
+ int *os_page_status;
+ uint64 os_page_count;
+ int pages_per_buffer;
+ int max_entries;
+ volatile uint64 touch pg_attribute_unused();
+ char *startptr,
+ *endptr;
+
+ if (pg_numa_init() == -1)
+ elog(ERROR, "libnuma initialization failed or NUMA is not supported on this platform");
+
+ /*
+ * The database block size and OS memory page size are unlikely to be
+ * the same. The block size is 1-32KB, the memory page size depends on
+ * platform. On x86 it's usually 4KB, on ARM it's 4KB or 64KB, but
+ * there are also features like THP etc. Moreover, we don't quite know
+ * how the pages and buffers "align" in memory - the buffers may be
+ * shifted in some way, using more memory pages than necessary.
+ *
+ * So we need to be careful about mappping buffers to memory pages. We
+ * calculate the maximum number of pages a buffer might use, so that
+ * we allocate enough space for the entries. And then we count the
+ * actual number of entries as we scan the buffers.
+ *
+ * This information is needed before calling move_pages() for NUMA
+ * node id inquiry.
+ */
+ os_page_size = pg_numa_get_pagesize();
+
+ /*
+ * The pages and block size is expected to be 2^k, so one divides the
+ * other (we don't know in which direction). This does not say
+ * anything about relative alignment of pages/buffers.
+ */
+ Assert((os_page_size % BLCKSZ == 0) || (BLCKSZ % os_page_size == 0));
+
+ /*
+ * How many addresses we are going to query? Simply get the page for
+ * the first buffer, and first page after the last buffer, and count
+ * the pages from that.
+ */
+ startptr = (char *) TYPEALIGN_DOWN(os_page_size,
+ BufferGetBlock(1));
+ endptr = (char *) TYPEALIGN(os_page_size,
+ (char *) BufferGetBlock(NBuffers) + BLCKSZ);
+ os_page_count = (endptr - startptr) / os_page_size;
+
+ /* Used to determine the NUMA node for all OS pages at once */
+ os_page_ptrs = palloc0(sizeof(void *) * os_page_count);
+ os_page_status = palloc(sizeof(uint64) * os_page_count);
+
+ /* Fill pointers for all the memory pages. */
+ idx = 0;
+ for (char *ptr = startptr; ptr < endptr; ptr += os_page_size)
+ {
+ os_page_ptrs[idx++] = ptr;
+
+ /* Only need to touch memory once per backend process lifetime */
+ if (firstNumaTouch)
+ pg_numa_touch_mem_if_required(touch, ptr);
+ }
+
+ Assert(idx == os_page_count);
+
+ elog(DEBUG1, "NUMA: NBuffers=%d os_page_count=" UINT64_FORMAT " "
+ "os_page_size=%zu", NBuffers, os_page_count, os_page_size);
+
+ /*
+ * If we ever get 0xff back from kernel inquiry, then we probably have
+ * bug in our buffers to OS page mapping code here.
+ */
+ memset(os_page_status, 0xff, sizeof(int) * os_page_count);
+
+ /* Query NUMA status for all the pointers */
+ if (pg_numa_query_pages(0, os_page_count, os_page_ptrs, os_page_status) == -1)
+ elog(ERROR, "failed NUMA pages inquiry: %m");
+
+ /* Initialize the multi-call context, load entries about buffers */
+
+ funcctx = SRF_FIRSTCALL_INIT();
+
+ /* Switch context when allocating stuff to be used in later calls */
+ oldcontext = MemoryContextSwitchTo(funcctx->multi_call_memory_ctx);
+
+ /* Create a user function context for cross-call persistence */
+ fctx = (BufferCacheNumaContext *) palloc(sizeof(BufferCacheNumaContext));
+
+ if (get_call_result_type(fcinfo, NULL, &expected_tupledesc) != TYPEFUNC_COMPOSITE)
+ elog(ERROR, "return type must be a row type");
+
+ if (expected_tupledesc->natts != NUM_BUFFERCACHE_NUMA_ELEM)
+ elog(ERROR, "incorrect number of output arguments");
+
+ /* Construct a tuple descriptor for the result rows. */
+ tupledesc = CreateTemplateTupleDesc(expected_tupledesc->natts);
+ TupleDescInitEntry(tupledesc, (AttrNumber) 1, "bufferid",
+ INT4OID, -1, 0);
+ TupleDescInitEntry(tupledesc, (AttrNumber) 2, "os_page_num",
+ INT4OID, -1, 0);
+ TupleDescInitEntry(tupledesc, (AttrNumber) 3, "numa_node",
+ INT4OID, -1, 0);
+
+ fctx->tupdesc = BlessTupleDesc(tupledesc);
+
+ /*
+ * Each buffer needs at least one entry, but it might be offset in
+ * some way, and use one extra entry. So we allocate space for the
+ * maximum number of entries we might need, and then count the exact
+ * number as we're walking buffers. That way we can do it in one pass,
+ * without reallocating memory.
+ */
+ pages_per_buffer = Max(1, BLCKSZ / os_page_size) + 1;
+ max_entries = NBuffers * pages_per_buffer;
+
+ /* Allocate entries for BufferCachePagesRec records. */
+ fctx->record = (BufferCacheNumaRec *)
+ MemoryContextAllocHuge(CurrentMemoryContext,
+ sizeof(BufferCacheNumaRec) * max_entries);
+
+ /* Return to original context when allocating transient memory */
+ MemoryContextSwitchTo(oldcontext);
+
+ if (firstNumaTouch)
+ elog(DEBUG1, "NUMA: page-faulting the buffercache for proper NUMA readouts");
+
+ /*
+ * Scan through all the buffers, saving the relevant fields in the
+ * fctx->record structure.
+ *
+ * We don't hold the partition locks, so we don't get a consistent
+ * snapshot across all buffers, but we do grab the buffer header
+ * locks, so the information of each buffer is self-consistent.
+ *
+ * This loop touches and stores addresses into os_page_ptrs[] as input
+ * to one big big move_pages(2) inquiry system call. Basically we ask
+ * for all memory pages for NBuffers.
+ */
+ startptr = (char *) TYPEALIGN_DOWN(os_page_size, (char *) BufferGetBlock(1));
+ idx = 0;
+ for (i = 0; i < NBuffers; i++)
+ {
+ char *buffptr = (char *) BufferGetBlock(i + 1);
+ BufferDesc *bufHdr;
+ uint32 buf_state;
+ uint32 bufferid;
+ int32 page_num;
+ char *startptr_buff,
+ *endptr_buff;
+
+ CHECK_FOR_INTERRUPTS();
+
+ bufHdr = GetBufferDescriptor(i);
+
+ /* Lock each buffer header before inspecting. */
+ buf_state = LockBufHdr(bufHdr);
+ bufferid = BufferDescriptorGetBuffer(bufHdr);
+ UnlockBufHdr(bufHdr, buf_state);
+
+ /* start of the first page of this buffer */
+ startptr_buff = (char *) TYPEALIGN_DOWN(os_page_size, buffptr);
+
+ /* end of the buffer (no need to align to memory page) */
+ endptr_buff = buffptr + BLCKSZ;
+
+ Assert(startptr_buff < endptr_buff);
+
+ /* calculate ID of the first page for this buffer */
+ page_num = (startptr_buff - startptr) / os_page_size;
+
+ /* Add an entry for each OS page overlapping with this buffer. */
+ for (char *ptr = startptr_buff; ptr < endptr_buff; ptr += os_page_size)
+ {
+ fctx->record[idx].bufferid = bufferid;
+ fctx->record[idx].page_num = page_num;
+ fctx->record[idx].numa_node = os_page_status[page_num];
+
+ /* advance to the next entry/page */
+ ++idx;
+ ++page_num;
+ }
+ }
+
+ Assert((idx >= os_page_count) && (idx <= max_entries));
+
+ /* Set max calls and remember the user function context. */
+ funcctx->max_calls = idx;
+ funcctx->user_fctx = fctx;
+
+ /* Remember this backend touched the pages */
+ firstNumaTouch = false;
+ }
+
+ funcctx = SRF_PERCALL_SETUP();
+
+ /* Get the saved state */
+ fctx = funcctx->user_fctx;
+
+ if (funcctx->call_cntr < funcctx->max_calls)
+ {
+ uint32 i = funcctx->call_cntr;
+ Datum values[NUM_BUFFERCACHE_NUMA_ELEM];
+ bool nulls[NUM_BUFFERCACHE_NUMA_ELEM];
+
+ values[0] = Int32GetDatum(fctx->record[i].bufferid);
+ nulls[0] = false;
+
+ values[1] = Int32GetDatum(fctx->record[i].page_num);
+ nulls[1] = false;
+
+ values[2] = Int32GetDatum(fctx->record[i].numa_node);
+ nulls[2] = false;
+
+ /* Build and return the tuple. */
+ tuple = heap_form_tuple(fctx->tupdesc, values, nulls);
+ result = HeapTupleGetDatum(tuple);
+
+ SRF_RETURN_NEXT(funcctx, result);
+ }
+ else
+ SRF_RETURN_DONE(funcctx);
+}
+
Datum
pg_buffercache_summary(PG_FUNCTION_ARGS)
{
<para>
This module provides the <function>pg_buffercache_pages()</function>
function (wrapped in the <structname>pg_buffercache</structname> view),
- the <function>pg_buffercache_summary()</function> function, the
+ <function>pg_buffercache_numa_pages()</function> function (wrapped in the
+ <structname>pg_buffercache_numa</structname> view), the
+ <function>pg_buffercache_summary()</function> function, the
<function>pg_buffercache_usage_counts()</function> function and
the <function>pg_buffercache_evict()</function> function.
</para>
convenient use.
</para>
+ <para>
+ The <function>pg_buffercache_numa_pages()</function> provides
+ <acronym>NUMA</acronym> node mappings for shared buffer entries. This
+ information is not part of <function>pg_buffercache_pages()</function>
+ itself, as it is much slower to retrieve.
+ The <structname>pg_buffercache_numa</structname> view wraps the function for
+ convenient use.
+ </para>
+
<para>
The <function>pg_buffercache_summary()</function> function returns a single
row summarizing the state of the shared buffer cache.
</para>
</sect2>
+ <sect2 id="pgbuffercache-pg-buffercache-numa">
+ <title>The <structname>pg_buffercache_numa</structname> View</title>
+
+ <para>
+ The definitions of the columns exposed by the view are shown in <xref linkend="pgbuffercache-numa-columns"/>.
+ </para>
+
+ <table id="pgbuffercache-numa-columns">
+ <title><structname>pg_buffercache_numa</structname> Columns</title>
+ <tgroup cols="1">
+ <thead>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ Column Type
+ </para>
+ <para>
+ Description
+ </para></entry>
+ </row>
+ </thead>
+
+ <tbody>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>bufferid</structfield> <type>integer</type>
+ </para>
+ <para>
+ ID, in the range 1..<varname>shared_buffers</varname>
+ </para></entry>
+ </row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>os_page_num</structfield> <type>int</type>
+ </para>
+ <para>
+ number of OS memory page for this buffer
+ </para></entry>
+ </row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>numa_node</structfield> <type>int</type>
+ </para>
+ <para>
+ ID of <acronym>NUMA</acronym> node
+ </para></entry>
+ </row>
+
+ </tbody>
+ </tgroup>
+ </table>
+
+ <para>
+ As <acronym>NUMA</acronym> node ID inquiry for each page requires memory pages
+ to be paged-in, the first execution of this function can take a noticeable
+ amount of time. In all the cases (first execution or not), retrieving this
+ information is costly and querying the view at a high frequency is not recommended.
+ </para>
+
+ <warning>
+ <para>
+ When determining the <acronym>NUMA</acronym> node, the view touches
+ all memory pages for the shared memory segment. This will force
+ allocation of the shared memory, if it wasn't allocated already,
+ and the memory may get allocated in a single <acronym>NUMA</acronym>
+ node (depending on system configuration).
+ </para>
+ </warning>
+
+ </sect2>
+
<sect2 id="pgbuffercache-summary">
<title>The <function>pg_buffercache_summary()</function> Function</title>