summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--contrib/file_fdw/file_fdw.c248
-rw-r--r--doc/src/sgml/fdwhandler.sgml42
-rw-r--r--doc/src/sgml/maintenance.sgml10
-rw-r--r--doc/src/sgml/ref/alter_foreign_table.sgml28
-rw-r--r--doc/src/sgml/ref/analyze.sgml10
-rw-r--r--src/backend/commands/analyze.c146
-rw-r--r--src/backend/commands/tablecmds.c7
-rw-r--r--src/bin/psql/describe.c6
-rw-r--r--src/bin/psql/tab-complete.c17
-rw-r--r--src/include/commands/vacuum.h3
-rw-r--r--src/include/foreign/fdwapi.h24
-rw-r--r--src/test/regress/expected/foreign_data.out39
-rw-r--r--src/test/regress/sql/foreign_data.sql3
13 files changed, 484 insertions, 99 deletions
diff --git a/contrib/file_fdw/file_fdw.c b/contrib/file_fdw/file_fdw.c
index e8907709bd9..30ed9fbad14 100644
--- a/contrib/file_fdw/file_fdw.c
+++ b/contrib/file_fdw/file_fdw.c
@@ -20,6 +20,7 @@
#include "commands/copy.h"
#include "commands/defrem.h"
#include "commands/explain.h"
+#include "commands/vacuum.h"
#include "foreign/fdwapi.h"
#include "foreign/foreign.h"
#include "miscadmin.h"
@@ -28,6 +29,7 @@
#include "optimizer/pathnode.h"
#include "optimizer/planmain.h"
#include "optimizer/restrictinfo.h"
+#include "utils/memutils.h"
#include "utils/rel.h"
PG_MODULE_MAGIC;
@@ -123,6 +125,7 @@ static void fileBeginForeignScan(ForeignScanState *node, int eflags);
static TupleTableSlot *fileIterateForeignScan(ForeignScanState *node);
static void fileReScanForeignScan(ForeignScanState *node);
static void fileEndForeignScan(ForeignScanState *node);
+static AcquireSampleRowsFunc fileAnalyzeForeignTable(Relation relation);
/*
* Helper functions
@@ -136,6 +139,10 @@ static void estimate_size(PlannerInfo *root, RelOptInfo *baserel,
static void estimate_costs(PlannerInfo *root, RelOptInfo *baserel,
FileFdwPlanState *fdw_private,
Cost *startup_cost, Cost *total_cost);
+static int file_acquire_sample_rows(Relation onerel, int elevel,
+ HeapTuple *rows, int targrows,
+ double *totalrows, double *totaldeadrows,
+ BlockNumber *totalpages);
/*
@@ -155,6 +162,7 @@ file_fdw_handler(PG_FUNCTION_ARGS)
fdwroutine->IterateForeignScan = fileIterateForeignScan;
fdwroutine->ReScanForeignScan = fileReScanForeignScan;
fdwroutine->EndForeignScan = fileEndForeignScan;
+ fdwroutine->AnalyzeForeignTable = fileAnalyzeForeignTable;
PG_RETURN_POINTER(fdwroutine);
}
@@ -614,6 +622,23 @@ fileIterateForeignScan(ForeignScanState *node)
}
/*
+ * fileReScanForeignScan
+ * Rescan table, possibly with new parameters
+ */
+static void
+fileReScanForeignScan(ForeignScanState *node)
+{
+ FileFdwExecutionState *festate = (FileFdwExecutionState *) node->fdw_state;
+
+ EndCopyFrom(festate->cstate);
+
+ festate->cstate = BeginCopyFrom(node->ss.ss_currentRelation,
+ festate->filename,
+ NIL,
+ festate->options);
+}
+
+/*
* fileEndForeignScan
* Finish scanning foreign table and dispose objects used for this scan
*/
@@ -628,20 +653,13 @@ fileEndForeignScan(ForeignScanState *node)
}
/*
- * fileReScanForeignScan
- * Rescan table, possibly with new parameters
+ * fileAnalyzeForeignTable
+ * Test whether analyzing this foreign table is supported
*/
-static void
-fileReScanForeignScan(ForeignScanState *node)
+static AcquireSampleRowsFunc
+fileAnalyzeForeignTable(Relation relation)
{
- FileFdwExecutionState *festate = (FileFdwExecutionState *) node->fdw_state;
-
- EndCopyFrom(festate->cstate);
-
- festate->cstate = BeginCopyFrom(node->ss.ss_currentRelation,
- festate->filename,
- NIL,
- festate->options);
+ return file_acquire_sample_rows;
}
/*
@@ -657,7 +675,6 @@ estimate_size(PlannerInfo *root, RelOptInfo *baserel,
{
struct stat stat_buf;
BlockNumber pages;
- int tuple_width;
double ntuples;
double nrows;
@@ -674,26 +691,45 @@ estimate_size(PlannerInfo *root, RelOptInfo *baserel,
pages = (stat_buf.st_size + (BLCKSZ - 1)) / BLCKSZ;
if (pages < 1)
pages = 1;
-
fdw_private->pages = pages;
/*
- * Estimate the number of tuples in the file. We back into this estimate
- * using the planner's idea of the relation width; which is bogus if not
- * all columns are being read, not to mention that the text representation
- * of a row probably isn't the same size as its internal representation.
- * FIXME later.
+ * Estimate the number of tuples in the file.
*/
- tuple_width = MAXALIGN(baserel->width) + MAXALIGN(sizeof(HeapTupleHeaderData));
+ if (baserel->pages > 0)
+ {
+ /*
+ * We have # of pages and # of tuples from pg_class (that is, from a
+ * previous ANALYZE), so compute a tuples-per-page estimate and scale
+ * that by the current file size.
+ */
+ double density;
- ntuples = clamp_row_est((double) stat_buf.st_size / (double) tuple_width);
+ density = baserel->tuples / (double) baserel->pages;
+ ntuples = clamp_row_est(density * (double) pages);
+ }
+ else
+ {
+ /*
+ * Otherwise we have to fake it. We back into this estimate using the
+ * planner's idea of the relation width; which is bogus if not all
+ * columns are being read, not to mention that the text representation
+ * of a row probably isn't the same size as its internal
+ * representation. Possibly we could do something better, but the
+ * real answer to anyone who complains is "ANALYZE" ...
+ */
+ int tuple_width;
+ tuple_width = MAXALIGN(baserel->width) +
+ MAXALIGN(sizeof(HeapTupleHeaderData));
+ ntuples = clamp_row_est((double) stat_buf.st_size /
+ (double) tuple_width);
+ }
fdw_private->ntuples = ntuples;
/*
* Now estimate the number of rows returned by the scan after applying the
- * baserestrictinfo quals. This is pretty bogus too, since the planner
- * will have no stats about the relation, but it's better than nothing.
+ * baserestrictinfo quals.
*/
nrows = ntuples *
clauselist_selectivity(root,
@@ -736,3 +772,169 @@ estimate_costs(PlannerInfo *root, RelOptInfo *baserel,
run_cost += cpu_per_tuple * ntuples;
*total_cost = *startup_cost + run_cost;
}
+
+/*
+ * file_acquire_sample_rows -- acquire a random sample of rows from the table
+ *
+ * Selected rows are returned in the caller-allocated array rows[],
+ * which must have at least targrows entries.
+ * The actual number of rows selected is returned as the function result.
+ * We also count the total number of rows in the file and return it into
+ * *totalrows, and return the file's physical size in *totalpages.
+ * Note that *totaldeadrows is always set to 0.
+ *
+ * Note that the returned list of rows is not always in order by physical
+ * position in the file. Therefore, correlation estimates derived later
+ * may be meaningless, but it's OK because we don't use the estimates
+ * currently (the planner only pays attention to correlation for indexscans).
+ */
+static int
+file_acquire_sample_rows(Relation onerel, int elevel,
+ HeapTuple *rows, int targrows,
+ double *totalrows, double *totaldeadrows,
+ BlockNumber *totalpages)
+{
+ int numrows = 0;
+ double rowstoskip = -1; /* -1 means not set yet */
+ double rstate;
+ TupleDesc tupDesc;
+ Datum *values;
+ bool *nulls;
+ bool found;
+ char *filename;
+ struct stat stat_buf;
+ List *options;
+ CopyState cstate;
+ ErrorContextCallback errcontext;
+ MemoryContext oldcontext = CurrentMemoryContext;
+ MemoryContext tupcontext;
+
+ Assert(onerel);
+ Assert(targrows > 0);
+
+ tupDesc = RelationGetDescr(onerel);
+ values = (Datum *) palloc(tupDesc->natts * sizeof(Datum));
+ nulls = (bool *) palloc(tupDesc->natts * sizeof(bool));
+
+ /* Fetch options of foreign table */
+ fileGetOptions(RelationGetRelid(onerel), &filename, &options);
+
+ /*
+ * Get size of the file.
+ */
+ if (stat(filename, &stat_buf) < 0)
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not stat file \"%s\": %m",
+ filename)));
+
+ /*
+ * Convert size to pages for use in I/O cost estimate.
+ */
+ *totalpages = (stat_buf.st_size + (BLCKSZ - 1)) / BLCKSZ;
+ if (*totalpages < 1)
+ *totalpages = 1;
+
+ /*
+ * Create CopyState from FDW options.
+ */
+ cstate = BeginCopyFrom(onerel, filename, NIL, options);
+
+ /*
+ * Use per-tuple memory context to prevent leak of memory used to read rows
+ * from the file with Copy routines.
+ */
+ tupcontext = AllocSetContextCreate(CurrentMemoryContext,
+ "file_fdw temporary context",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /* Prepare for sampling rows */
+ rstate = anl_init_selection_state(targrows);
+
+ /* Set up callback to identify error line number. */
+ errcontext.callback = CopyFromErrorCallback;
+ errcontext.arg = (void *) cstate;
+ errcontext.previous = error_context_stack;
+ error_context_stack = &errcontext;
+
+ *totalrows = 0;
+ *totaldeadrows = 0;
+ for (;;)
+ {
+ /* Check for user-requested abort or sleep */
+ vacuum_delay_point();
+
+ /* Fetch next row */
+ MemoryContextReset(tupcontext);
+ MemoryContextSwitchTo(tupcontext);
+
+ found = NextCopyFrom(cstate, NULL, values, nulls, NULL);
+
+ MemoryContextSwitchTo(oldcontext);
+
+ if (!found)
+ break;
+
+ /*
+ * The first targrows sample rows are simply copied into the
+ * reservoir. Then we start replacing tuples in the sample until we
+ * reach the end of the relation. This algorithm is from Jeff Vitter's
+ * paper (see more info in commands/analyze.c).
+ */
+ if (numrows < targrows)
+ {
+ rows[numrows++] = heap_form_tuple(tupDesc, values, nulls);
+ }
+ else
+ {
+ /*
+ * t in Vitter's paper is the number of records already processed.
+ * If we need to compute a new S value, we must use the
+ * not-yet-incremented value of totalrows as t.
+ */
+ if (rowstoskip < 0)
+ rowstoskip = anl_get_next_S(*totalrows, targrows, &rstate);
+
+ if (rowstoskip <= 0)
+ {
+ /*
+ * Found a suitable tuple, so save it, replacing one
+ * old tuple at random
+ */
+ int k = (int) (targrows * anl_random_fract());
+
+ Assert(k >= 0 && k < targrows);
+ heap_freetuple(rows[k]);
+ rows[k] = heap_form_tuple(tupDesc, values, nulls);
+ }
+
+ rowstoskip -= 1;
+ }
+
+ *totalrows += 1;
+ }
+
+ /* Remove error callback. */
+ error_context_stack = errcontext.previous;
+
+ /* Clean up. */
+ MemoryContextDelete(tupcontext);
+
+ EndCopyFrom(cstate);
+
+ pfree(values);
+ pfree(nulls);
+
+ /*
+ * Emit some interesting relation info
+ */
+ ereport(elevel,
+ (errmsg("\"%s\": scanned %u pages containing %.0f rows; "
+ "%d rows in sample",
+ RelationGetRelationName(onerel),
+ *totalpages, *totalrows, numrows)));
+
+ return numrows;
+}
diff --git a/doc/src/sgml/fdwhandler.sgml b/doc/src/sgml/fdwhandler.sgml
index f7bf3d8a395..8e7525ab596 100644
--- a/doc/src/sgml/fdwhandler.sgml
+++ b/doc/src/sgml/fdwhandler.sgml
@@ -10,8 +10,8 @@
<para>
All operations on a foreign table are handled through its foreign data
- wrapper, which consists of a set of functions that the planner and
- executor call. The foreign data wrapper is responsible for fetching
+ wrapper, which consists of a set of functions that the core server
+ calls. The foreign data wrapper is responsible for fetching
data from the remote data source and returning it to the
<productname>PostgreSQL</productname> executor. This chapter outlines how
to write a new foreign data wrapper.
@@ -47,7 +47,8 @@
<para>
The handler function simply returns a struct of function pointers to
- callback functions that will be called by the planner and executor.
+ callback functions that will be called by the planner, executor, and
+ various maintenance commands.
Most of the effort in writing an FDW is in implementing these callback
functions.
The handler function must be registered with
@@ -277,6 +278,41 @@ EndForeignScan (ForeignScanState *node);
</para>
<para>
+<programlisting>
+AcquireSampleRowsFunc
+AnalyzeForeignTable (Relation relation);
+</programlisting>
+
+ This function is called when <xref linkend="sql-analyze"> is executed on
+ a foreign table. If the FDW supports collecting statistics for this
+ foreign table, it should return a pointer to a function that will collect
+ sample rows from the table. Otherwise, return <literal>NULL</>. If the
+ FDW does not support collecting statistics for any tables, the
+ <function>AnalyzeForeignTable</> pointer can be set to <literal>NULL</>.
+ </para>
+
+ <para>
+ If provided, the sample collection function must have the signature
+<programlisting>
+int
+AcquireSampleRowsFunc (Relation relation, int elevel,
+ HeapTuple *rows, int targrows,
+ double *totalrows,
+ double *totaldeadrows,
+ BlockNumber *totalpages);
+</programlisting>
+
+ A random sample of up to <parameter>targrows</> rows should be collected
+ from the table and stored into the caller-provided <parameter>rows</>
+ array. The actual number of rows collected must be returned. In
+ addition, store estimates of the total numbers of live rows, dead rows,
+ and pages in the table into the output parameters
+ <parameter>totalrows</>, <parameter>totaldeadrows</>, and
+ <parameter>totalpages</>. These numbers will be recorded in the table's
+ <structname>pg_class</> entry for future use.
+ </para>
+
+ <para>
The <structname>FdwRoutine</> struct type is declared in
<filename>src/include/foreign/fdwapi.h</>, which see for additional
details.
diff --git a/doc/src/sgml/maintenance.sgml b/doc/src/sgml/maintenance.sgml
index 93c3ff5f2bb..cc33e5f76b0 100644
--- a/doc/src/sgml/maintenance.sgml
+++ b/doc/src/sgml/maintenance.sgml
@@ -332,6 +332,16 @@
plans that use the expression index.
</para>
</tip>
+
+ <tip>
+ <para>
+ The autovacuum daemon does not issue <command>ANALYZE</> commands for
+ foreign tables, since it has no means of determining how often that
+ might be useful. If your queries require statistics on foreign tables
+ for proper planning, it's a good idea to run manually-managed
+ <command>ANALYZE</> commands on those tables on a suitable schedule.
+ </para>
+ </tip>
</sect2>
<sect2 id="vacuum-for-visibility-map">
diff --git a/doc/src/sgml/ref/alter_foreign_table.sgml b/doc/src/sgml/ref/alter_foreign_table.sgml
index c4cdaa8289d..c3f33a50aec 100644
--- a/doc/src/sgml/ref/alter_foreign_table.sgml
+++ b/doc/src/sgml/ref/alter_foreign_table.sgml
@@ -36,6 +36,9 @@ ALTER FOREIGN TABLE [ IF EXISTS ] <replaceable class="PARAMETER">name</replaceab
DROP [ COLUMN ] [ IF EXISTS ] <replaceable class="PARAMETER">column</replaceable> [ RESTRICT | CASCADE ]
ALTER [ COLUMN ] <replaceable class="PARAMETER">column</replaceable> [ SET DATA ] TYPE <replaceable class="PARAMETER">type</replaceable>
ALTER [ COLUMN ] <replaceable class="PARAMETER">column</replaceable> { SET | DROP } NOT NULL
+ ALTER [ COLUMN ] <replaceable class="PARAMETER">column</replaceable> SET STATISTICS <replaceable class="PARAMETER">integer</replaceable>
+ ALTER [ COLUMN ] <replaceable class="PARAMETER">column</replaceable> SET ( <replaceable class="PARAMETER">attribute_option</replaceable> = <replaceable class="PARAMETER">value</replaceable> [, ... ] )
+ ALTER [ COLUMN ] <replaceable class="PARAMETER">column</replaceable> RESET ( <replaceable class="PARAMETER">attribute_option</replaceable> [, ... ] )
ALTER [ COLUMN ] <replaceable class="PARAMETER">column</replaceable> OPTIONS ( [ ADD | SET | DROP ] <replaceable class="PARAMETER">option</replaceable> ['<replaceable class="PARAMETER">value</replaceable>'] [, ... ])
OWNER TO <replaceable class="PARAMETER">new_owner</replaceable>
OPTIONS ( [ ADD | SET | DROP ] <replaceable class="PARAMETER">option</replaceable> ['<replaceable class="PARAMETER">value</replaceable>'] [, ... ])
@@ -104,6 +107,31 @@ ALTER FOREIGN TABLE [ IF EXISTS ] <replaceable class="PARAMETER">name</replaceab
</varlistentry>
<varlistentry>
+ <term><literal>SET STATISTICS</literal></term>
+ <listitem>
+ <para>
+ This form
+ sets the per-column statistics-gathering target for subsequent
+ <xref linkend="sql-analyze"> operations.
+ See the similar form of <xref linkend="sql-altertable">
+ for more details.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term><literal>SET ( <replaceable class="PARAMETER">attribute_option</replaceable> = <replaceable class="PARAMETER">value</replaceable> [, ... ] )</literal></term>
+ <term><literal>RESET ( <replaceable class="PARAMETER">attribute_option</replaceable> [, ... ] )</literal></term>
+ <listitem>
+ <para>
+ This form sets or resets per-attribute options.
+ See the similar form of <xref linkend="sql-altertable">
+ for more details.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
<term><literal>OWNER</literal></term>
<listitem>
<para>
diff --git a/doc/src/sgml/ref/analyze.sgml b/doc/src/sgml/ref/analyze.sgml
index 8c9057beb91..52b2114889d 100644
--- a/doc/src/sgml/ref/analyze.sgml
+++ b/doc/src/sgml/ref/analyze.sgml
@@ -63,7 +63,8 @@ ANALYZE [ VERBOSE ] [ <replaceable class="PARAMETER">table</replaceable> [ ( <re
<listitem>
<para>
The name (possibly schema-qualified) of a specific table to
- analyze. Defaults to all tables in the current database.
+ analyze. If omitted, all regular tables (but not foreign tables)
+ in the current database are analyzed.
</para>
</listitem>
</varlistentry>
@@ -93,6 +94,13 @@ ANALYZE [ VERBOSE ] [ <replaceable class="PARAMETER">table</replaceable> [ ( <re
<title>Notes</title>
<para>
+ Foreign tables are analyzed only when explicitly selected. Not all
+ foreign data wrappers support <command>ANALYZE</>. If the table's
+ wrapper does not support <command>ANALYZE</>, the command prints a
+ warning and does nothing.
+ </para>
+
+ <para>
In the default <productname>PostgreSQL</productname> configuration,
the autovacuum daemon (see <xref linkend="autovacuum">)
takes care of automatic analyzing of tables when they are first loaded
diff --git a/src/backend/commands/analyze.c b/src/backend/commands/analyze.c
index 9cd6e672ced..17abe48f25e 100644
--- a/src/backend/commands/analyze.c
+++ b/src/backend/commands/analyze.c
@@ -30,6 +30,7 @@
#include "commands/tablecmds.h"
#include "commands/vacuum.h"
#include "executor/executor.h"
+#include "foreign/fdwapi.h"
#include "miscadmin.h"
#include "nodes/nodeFuncs.h"
#include "parser/parse_oper.h"
@@ -78,14 +79,12 @@ typedef struct AnlIndexData
int default_statistics_target = 100;
/* A few variables that don't seem worth passing around as parameters */
-static int elevel = -1;
-
static MemoryContext anl_context = NULL;
-
static BufferAccessStrategy vac_strategy;
-static void do_analyze_rel(Relation onerel, VacuumStmt *vacstmt, bool inh);
+static void do_analyze_rel(Relation onerel, VacuumStmt *vacstmt,
+ AcquireSampleRowsFunc acquirefunc, bool inh, int elevel);
static void BlockSampler_Init(BlockSampler bs, BlockNumber nblocks,
int samplesize);
static bool BlockSampler_HasMore(BlockSampler bs);
@@ -96,13 +95,12 @@ static void compute_index_stats(Relation onerel, double totalrows,
MemoryContext col_context);
static VacAttrStats *examine_attribute(Relation onerel, int attnum,
Node *index_expr);
-static int acquire_sample_rows(Relation onerel, HeapTuple *rows,
- int targrows, double *totalrows, double *totaldeadrows);
-static double random_fract(void);
-static double init_selection_state(int n);
-static double get_next_S(double t, int n, double *stateptr);
+static int acquire_sample_rows(Relation onerel, int elevel,
+ HeapTuple *rows, int targrows,
+ double *totalrows, double *totaldeadrows,
+ BlockNumber *totalpages);
static int compare_rows(const void *a, const void *b);
-static int acquire_inherited_sample_rows(Relation onerel,
+static int acquire_inherited_sample_rows(Relation onerel, int elevel,
HeapTuple *rows, int targrows,
double *totalrows, double *totaldeadrows);
static void update_attstats(Oid relid, bool inh,
@@ -118,13 +116,16 @@ void
analyze_rel(Oid relid, VacuumStmt *vacstmt, BufferAccessStrategy bstrategy)
{
Relation onerel;
+ int elevel;
+ AcquireSampleRowsFunc acquirefunc;
- /* Set up static variables */
+ /* Select logging level */
if (vacstmt->options & VACOPT_VERBOSE)
elevel = INFO;
else
elevel = DEBUG2;
+ /* Set up static variables */
vac_strategy = bstrategy;
/*
@@ -182,10 +183,40 @@ analyze_rel(Oid relid, VacuumStmt *vacstmt, BufferAccessStrategy bstrategy)
}
/*
- * Check that it's a plain table; we used to do this in get_rel_oids() but
- * seems safer to check after we've locked the relation.
+ * Check that it's a plain table or foreign table; we used to do this
+ * in get_rel_oids() but seems safer to check after we've locked the
+ * relation.
*/
- if (onerel->rd_rel->relkind != RELKIND_RELATION)
+ if (onerel->rd_rel->relkind == RELKIND_RELATION)
+ {
+ /* Regular table, so we'll use the regular row acquisition function */
+ acquirefunc = acquire_sample_rows;
+ }
+ else if (onerel->rd_rel->relkind == RELKIND_FOREIGN_TABLE)
+ {
+ /*
+ * For a foreign table, call the FDW's hook function to see whether it
+ * supports analysis.
+ */
+ FdwRoutine *fdwroutine;
+
+ fdwroutine = GetFdwRoutineByRelId(RelationGetRelid(onerel));
+
+ if (fdwroutine->AnalyzeForeignTable != NULL)
+ acquirefunc = fdwroutine->AnalyzeForeignTable(onerel);
+ else
+ acquirefunc = NULL;
+
+ if (acquirefunc == NULL)
+ {
+ ereport(WARNING,
+ (errmsg("skipping \"%s\" --- cannot analyze this foreign table",
+ RelationGetRelationName(onerel))));
+ relation_close(onerel, ShareUpdateExclusiveLock);
+ return;
+ }
+ }
+ else
{
/* No need for a WARNING if we already complained during VACUUM */
if (!(vacstmt->options & VACOPT_VACUUM))
@@ -227,13 +258,13 @@ analyze_rel(Oid relid, VacuumStmt *vacstmt, BufferAccessStrategy bstrategy)
/*
* Do the normal non-recursive ANALYZE.
*/
- do_analyze_rel(onerel, vacstmt, false);
+ do_analyze_rel(onerel, vacstmt, acquirefunc, false, elevel);
/*
* If there are child tables, do recursive ANALYZE.
*/
if (onerel->rd_rel->relhassubclass)
- do_analyze_rel(onerel, vacstmt, true);
+ do_analyze_rel(onerel, vacstmt, acquirefunc, true, elevel);
/*
* Close source relation now, but keep lock so that no one deletes it
@@ -254,9 +285,15 @@ analyze_rel(Oid relid, VacuumStmt *vacstmt, BufferAccessStrategy bstrategy)
/*
* do_analyze_rel() -- analyze one relation, recursively or not
+ *
+ * Note that "acquirefunc" is only relevant for the non-inherited case.
+ * If we supported foreign tables in inheritance trees,
+ * acquire_inherited_sample_rows would need to determine the appropriate
+ * acquirefunc for each child table.
*/
static void
-do_analyze_rel(Relation onerel, VacuumStmt *vacstmt, bool inh)
+do_analyze_rel(Relation onerel, VacuumStmt *vacstmt,
+ AcquireSampleRowsFunc acquirefunc, bool inh, int elevel)
{
int attr_cnt,
tcnt,
@@ -271,6 +308,7 @@ do_analyze_rel(Relation onerel, VacuumStmt *vacstmt, bool inh)
numrows;
double totalrows,
totaldeadrows;
+ BlockNumber totalpages;
HeapTuple *rows;
PGRUsage ru0;
TimestampTz starttime = 0;
@@ -447,11 +485,17 @@ do_analyze_rel(Relation onerel, VacuumStmt *vacstmt, bool inh)
*/
rows = (HeapTuple *) palloc(targrows * sizeof(HeapTuple));
if (inh)
- numrows = acquire_inherited_sample_rows(onerel, rows, targrows,
+ {
+ numrows = acquire_inherited_sample_rows(onerel, elevel,
+ rows, targrows,
&totalrows, &totaldeadrows);
+ totalpages = 0; /* not needed in this path */
+ }
else
- numrows = acquire_sample_rows(onerel, rows, targrows,
- &totalrows, &totaldeadrows);
+ numrows = (*acquirefunc) (onerel, elevel,
+ rows, targrows,
+ &totalrows, &totaldeadrows,
+ &totalpages);
/*
* Compute the statistics. Temporary results during the calculations for
@@ -532,7 +576,7 @@ do_analyze_rel(Relation onerel, VacuumStmt *vacstmt, bool inh)
*/
if (!inh)
vac_update_relstats(onerel,
- RelationGetNumberOfBlocks(onerel),
+ totalpages,
totalrows,
visibilitymap_count(onerel),
hasindex,
@@ -947,8 +991,8 @@ BlockSampler_Next(BlockSampler bs)
* Knuth says to skip the current block with probability 1 - k/K.
* If we are to skip, we should advance t (hence decrease K), and
* repeat the same probabilistic test for the next block. The naive
- * implementation thus requires a random_fract() call for each block
- * number. But we can reduce this to one random_fract() call per
+ * implementation thus requires an anl_random_fract() call for each block
+ * number. But we can reduce this to one anl_random_fract() call per
* selected block, by noting that each time the while-test succeeds,
* we can reinterpret V as a uniform random number in the range 0 to p.
* Therefore, instead of choosing a new V, we just adjust p to be
@@ -963,7 +1007,7 @@ BlockSampler_Next(BlockSampler bs)
* less than k, which means that we cannot fail to select enough blocks.
*----------
*/
- V = random_fract();
+ V = anl_random_fract();
p = 1.0 - (double) k / (double) K;
while (V < p)
{
@@ -988,6 +1032,7 @@ BlockSampler_Next(BlockSampler bs)
* The actual number of rows selected is returned as the function result.
* We also estimate the total numbers of live and dead rows in the table,
* and return them into *totalrows and *totaldeadrows, respectively.
+ * Also, the number of pages in the relation is returned into *totalpages.
*
* The returned list of tuples is in order by physical position in the table.
* (We will rely on this later to derive correlation estimates.)
@@ -1014,8 +1059,10 @@ BlockSampler_Next(BlockSampler bs)
* density near the start of the table.
*/
static int
-acquire_sample_rows(Relation onerel, HeapTuple *rows, int targrows,
- double *totalrows, double *totaldeadrows)
+acquire_sample_rows(Relation onerel, int elevel,
+ HeapTuple *rows, int targrows,
+ double *totalrows, double *totaldeadrows,
+ BlockNumber *totalpages)
{
int numrows = 0; /* # rows now in reservoir */
double samplerows = 0; /* total # rows collected */
@@ -1030,6 +1077,7 @@ acquire_sample_rows(Relation onerel, HeapTuple *rows, int targrows,
Assert(targrows > 0);
totalblocks = RelationGetNumberOfBlocks(onerel);
+ *totalpages = totalblocks;
/* Need a cutoff xmin for HeapTupleSatisfiesVacuum */
OldestXmin = GetOldestXmin(onerel->rd_rel->relisshared, true);
@@ -1037,7 +1085,7 @@ acquire_sample_rows(Relation onerel, HeapTuple *rows, int targrows,
/* Prepare for sampling block numbers */
BlockSampler_Init(&bs, totalblocks, targrows);
/* Prepare for sampling rows */
- rstate = init_selection_state(targrows);
+ rstate = anl_init_selection_state(targrows);
/* Outer loop over blocks to sample */
while (BlockSampler_HasMore(&bs))
@@ -1184,7 +1232,8 @@ acquire_sample_rows(Relation onerel, HeapTuple *rows, int targrows,
* t.
*/
if (rowstoskip < 0)
- rowstoskip = get_next_S(samplerows, targrows, &rstate);
+ rowstoskip = anl_get_next_S(samplerows, targrows,
+ &rstate);
if (rowstoskip <= 0)
{
@@ -1192,7 +1241,7 @@ acquire_sample_rows(Relation onerel, HeapTuple *rows, int targrows,
* Found a suitable tuple, so save it, replacing one
* old tuple at random
*/
- int k = (int) (targrows * random_fract());
+ int k = (int) (targrows * anl_random_fract());
Assert(k >= 0 && k < targrows);
heap_freetuple(rows[k]);
@@ -1252,8 +1301,8 @@ acquire_sample_rows(Relation onerel, HeapTuple *rows, int targrows,
}
/* Select a random value R uniformly distributed in (0 - 1) */
-static double
-random_fract(void)
+double
+anl_random_fract(void)
{
return ((double) random() + 1) / ((double) MAX_RANDOM_VALUE + 2);
}
@@ -1266,21 +1315,21 @@ random_fract(void)
* It is computed primarily based on t, the number of records already read.
* The only extra state needed between calls is W, a random state variable.
*
- * init_selection_state computes the initial W value.
+ * anl_init_selection_state computes the initial W value.
*
- * Given that we've already read t records (t >= n), get_next_S
+ * Given that we've already read t records (t >= n), anl_get_next_S
* determines the number of records to skip before the next record is
* processed.
*/
-static double
-init_selection_state(int n)
+double
+anl_init_selection_state(int n)
{
/* Initial value of W (for use when Algorithm Z is first applied) */
- return exp(-log(random_fract()) / n);
+ return exp(-log(anl_random_fract()) / n);
}
-static double
-get_next_S(double t, int n, double *stateptr)
+double
+anl_get_next_S(double t, int n, double *stateptr)
{
double S;
@@ -1291,7 +1340,7 @@ get_next_S(double t, int n, double *stateptr)
double V,
quot;
- V = random_fract(); /* Generate V */
+ V = anl_random_fract(); /* Generate V */
S = 0;
t += 1;
/* Note: "num" in Vitter's code is always equal to t - n */
@@ -1323,7 +1372,7 @@ get_next_S(double t, int n, double *stateptr)
tmp;
/* Generate U and X */
- U = random_fract();
+ U = anl_random_fract();
X = t * (W - 1.0);
S = floor(X); /* S is tentatively set to floor(X) */
/* Test if U <= h(S)/cg(X) in the manner of (6.3) */
@@ -1352,7 +1401,7 @@ get_next_S(double t, int n, double *stateptr)
y *= numer / denom;
denom -= 1;
}
- W = exp(-log(random_fract()) / n); /* Generate W in advance */
+ W = exp(-log(anl_random_fract()) / n); /* Generate W in advance */
if (exp(log(y) / n) <= (t + X) / t)
break;
}
@@ -1389,12 +1438,13 @@ compare_rows(const void *a, const void *b)
/*
* acquire_inherited_sample_rows -- acquire sample rows from inheritance tree
*
- * This has the same API as acquire_sample_rows, except that rows are
+ * This has largely the same API as acquire_sample_rows, except that rows are
* collected from all inheritance children as well as the specified table.
* We fail and return zero if there are no inheritance children.
*/
static int
-acquire_inherited_sample_rows(Relation onerel, HeapTuple *rows, int targrows,
+acquire_inherited_sample_rows(Relation onerel, int elevel,
+ HeapTuple *rows, int targrows,
double *totalrows, double *totaldeadrows)
{
List *tableOIDs;
@@ -1431,6 +1481,11 @@ acquire_inherited_sample_rows(Relation onerel, HeapTuple *rows, int targrows,
/*
* Count the blocks in all the relations. The result could overflow
* BlockNumber, so we use double arithmetic.
+ *
+ * XXX eventually we will probably want to allow child tables that are
+ * foreign tables. Since we can't do RelationGetNumberOfBlocks on a
+ * foreign table, it's not very clear what fraction of the total to assign
+ * to it here.
*/
rels = (Relation *) palloc(list_length(tableOIDs) * sizeof(Relation));
relblocks = (double *) palloc(list_length(tableOIDs) * sizeof(double));
@@ -1485,13 +1540,16 @@ acquire_inherited_sample_rows(Relation onerel, HeapTuple *rows, int targrows,
int childrows;
double trows,
tdrows;
+ BlockNumber tpages;
/* Fetch a random sample of the child's rows */
childrows = acquire_sample_rows(childrel,
+ elevel,
rows + numrows,
childtargrows,
&trows,
- &tdrows);
+ &tdrows,
+ &tpages);
/* We may need to convert from child's rowtype to parent's */
if (childrows > 0 &&
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index a35e338cc87..a4a36045df1 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -3054,7 +3054,7 @@ ATPrepCmd(List **wqueue, Relation rel, AlterTableCmd *cmd,
break;
case AT_SetOptions: /* ALTER COLUMN SET ( options ) */
case AT_ResetOptions: /* ALTER COLUMN RESET ( options ) */
- ATSimplePermissions(rel, ATT_TABLE | ATT_INDEX);
+ ATSimplePermissions(rel, ATT_TABLE | ATT_INDEX | ATT_FOREIGN_TABLE);
/* This command never recurses */
pass = AT_PASS_MISC;
break;
@@ -5032,10 +5032,11 @@ ATPrepSetStatistics(Relation rel, const char *colName, Node *newValue, LOCKMODE
* allowSystemTableMods to be turned on.
*/
if (rel->rd_rel->relkind != RELKIND_RELATION &&
- rel->rd_rel->relkind != RELKIND_INDEX)
+ rel->rd_rel->relkind != RELKIND_INDEX &&
+ rel->rd_rel->relkind != RELKIND_FOREIGN_TABLE)
ereport(ERROR,
(errcode(ERRCODE_WRONG_OBJECT_TYPE),
- errmsg("\"%s\" is not a table or index",
+ errmsg("\"%s\" is not a table, index, or foreign table",
RelationGetRelationName(rel))));
/* Permissions checks */
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index dc2248bc242..844a2912bb5 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -1104,7 +1104,7 @@ describeOneTableDetails(const char *schemaname,
bool printTableInitialized = false;
int i;
char *view_def = NULL;
- char *headers[6];
+ char *headers[9];
char **seq_values = NULL;
char **modifiers = NULL;
char **ptr;
@@ -1395,7 +1395,7 @@ describeOneTableDetails(const char *schemaname,
if (verbose)
{
headers[cols++] = gettext_noop("Storage");
- if (tableinfo.relkind == 'r')
+ if (tableinfo.relkind == 'r' || tableinfo.relkind == 'f')
headers[cols++] = gettext_noop("Stats target");
/* Column comments, if the relkind supports this feature. */
if (tableinfo.relkind == 'r' || tableinfo.relkind == 'v' ||
@@ -1498,7 +1498,7 @@ describeOneTableDetails(const char *schemaname,
false, false);
/* Statistics target, if the relkind supports this feature */
- if (tableinfo.relkind == 'r')
+ if (tableinfo.relkind == 'r' || tableinfo.relkind == 'f')
{
printTableAddCell(&cont, PQgetvalue(res, i, firstvcol + 1),
false, false);
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 975d65584b8..d113adff033 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -409,6 +409,21 @@ static const SchemaQuery Query_for_list_of_tsvf = {
NULL
};
+static const SchemaQuery Query_for_list_of_tf = {
+ /* catname */
+ "pg_catalog.pg_class c",
+ /* selcondition */
+ "c.relkind IN ('r', 'f')",
+ /* viscondition */
+ "pg_catalog.pg_table_is_visible(c.oid)",
+ /* namespace */
+ "c.relnamespace",
+ /* result */
+ "pg_catalog.quote_ident(c.relname)",
+ /* qualresult */
+ NULL
+};
+
static const SchemaQuery Query_for_list_of_views = {
/* catname */
"pg_catalog.pg_class c",
@@ -2833,7 +2848,7 @@ psql_completion(char *text, int start, int end)
/* ANALYZE */
/* If the previous word is ANALYZE, produce list of tables */
else if (pg_strcasecmp(prev_wd, "ANALYZE") == 0)
- COMPLETE_WITH_SCHEMA_QUERY(Query_for_list_of_tables, NULL);
+ COMPLETE_WITH_SCHEMA_QUERY(Query_for_list_of_tf, NULL);
/* WHERE */
/* Simple case of the word before the where being the table name */
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 3deee66b413..3c95dec3bc6 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -170,5 +170,8 @@ extern void lazy_vacuum_rel(Relation onerel, VacuumStmt *vacstmt,
extern void analyze_rel(Oid relid, VacuumStmt *vacstmt,
BufferAccessStrategy bstrategy);
extern bool std_typanalyze(VacAttrStats *stats);
+extern double anl_random_fract(void);
+extern double anl_init_selection_state(int n);
+extern double anl_get_next_S(double t, int n, double *stateptr);
#endif /* VACUUM_H */
diff --git a/src/include/foreign/fdwapi.h b/src/include/foreign/fdwapi.h
index 854f17755c4..6bf3a5e2306 100644
--- a/src/include/foreign/fdwapi.h
+++ b/src/include/foreign/fdwapi.h
@@ -50,20 +50,32 @@ typedef void (*ReScanForeignScan_function) (ForeignScanState *node);
typedef void (*EndForeignScan_function) (ForeignScanState *node);
+typedef int (*AcquireSampleRowsFunc) (Relation relation, int elevel,
+ HeapTuple *rows, int targrows,
+ double *totalrows,
+ double *totaldeadrows,
+ BlockNumber *totalpages);
+
+typedef AcquireSampleRowsFunc (*AnalyzeForeignTable_function) (Relation relation);
+
/*
* FdwRoutine is the struct returned by a foreign-data wrapper's handler
* function. It provides pointers to the callback functions needed by the
* planner and executor.
*
- * Currently, all functions must be supplied. Later there may be optional
- * additions. It's recommended that the handler initialize the struct with
- * makeNode(FdwRoutine) so that all fields are set to zero.
+ * More function pointers are likely to be added in the future. Therefore
+ * it's recommended that the handler initialize the struct with
+ * makeNode(FdwRoutine) so that all fields are set to NULL. This will
+ * ensure that no fields are accidentally left undefined.
*/
typedef struct FdwRoutine
{
NodeTag type;
+ /*
+ * These functions are required.
+ */
GetForeignRelSize_function GetForeignRelSize;
GetForeignPaths_function GetForeignPaths;
GetForeignPlan_function GetForeignPlan;
@@ -72,6 +84,12 @@ typedef struct FdwRoutine
IterateForeignScan_function IterateForeignScan;
ReScanForeignScan_function ReScanForeignScan;
EndForeignScan_function EndForeignScan;
+
+ /*
+ * These functions are optional. Set the pointer to NULL for any
+ * that are not provided.
+ */
+ AnalyzeForeignTable_function AnalyzeForeignTable;
} FdwRoutine;
diff --git a/src/test/regress/expected/foreign_data.out b/src/test/regress/expected/foreign_data.out
index ba86883b867..b3165d59014 100644
--- a/src/test/regress/expected/foreign_data.out
+++ b/src/test/regress/expected/foreign_data.out
@@ -679,12 +679,12 @@ CREATE FOREIGN TABLE ft1 (
COMMENT ON FOREIGN TABLE ft1 IS 'ft1';
COMMENT ON COLUMN ft1.c1 IS 'ft1.c1';
\d+ ft1
- Foreign table "public.ft1"
- Column | Type | Modifiers | FDW Options | Storage | Description
---------+---------+-----------+--------------------------------+----------+-------------
- c1 | integer | not null | ("param 1" 'val1') | plain | ft1.c1
- c2 | text | | (param2 'val2', param3 'val3') | extended |
- c3 | date | | | plain |
+ Foreign table "public.ft1"
+ Column | Type | Modifiers | FDW Options | Storage | Stats target | Description
+--------+---------+-----------+--------------------------------+----------+--------------+-------------
+ c1 | integer | not null | ("param 1" 'val1') | plain | | ft1.c1
+ c2 | text | | (param2 'val2', param3 'val3') | extended | |
+ c3 | date | | | plain | |
Server: s0
FDW Options: (delimiter ',', quote '"', "be quoted" 'value')
Has OIDs: no
@@ -730,19 +730,22 @@ ERROR: cannot alter system column "xmin"
ALTER FOREIGN TABLE ft1 ALTER COLUMN c7 OPTIONS (ADD p1 'v1', ADD p2 'v2'),
ALTER COLUMN c8 OPTIONS (ADD p1 'v1', ADD p2 'v2');
ALTER FOREIGN TABLE ft1 ALTER COLUMN c8 OPTIONS (SET p2 'V2', DROP p1);
+ALTER FOREIGN TABLE ft1 ALTER COLUMN c1 SET STATISTICS 10000;
+ALTER FOREIGN TABLE ft1 ALTER COLUMN c1 SET (n_distinct = 100);
+ALTER FOREIGN TABLE ft1 ALTER COLUMN c8 SET STATISTICS -1;
\d+ ft1
- Foreign table "public.ft1"
- Column | Type | Modifiers | FDW Options | Storage | Description
---------+---------+-----------+--------------------------------+----------+-------------
- c1 | integer | not null | ("param 1" 'val1') | plain |
- c2 | text | | (param2 'val2', param3 'val3') | extended |
- c3 | date | | | plain |
- c4 | integer | | | plain |
- c6 | integer | not null | | plain |
- c7 | integer | | (p1 'v1', p2 'v2') | plain |
- c8 | text | | (p2 'V2') | extended |
- c9 | integer | | | plain |
- c10 | integer | | (p1 'v1') | plain |
+ Foreign table "public.ft1"
+ Column | Type | Modifiers | FDW Options | Storage | Stats target | Description
+--------+---------+-----------+--------------------------------+----------+--------------+-------------
+ c1 | integer | not null | ("param 1" 'val1') | plain | 10000 |
+ c2 | text | | (param2 'val2', param3 'val3') | extended | |
+ c3 | date | | | plain | |
+ c4 | integer | | | plain | |
+ c6 | integer | not null | | plain | |
+ c7 | integer | | (p1 'v1', p2 'v2') | plain | |
+ c8 | text | | (p2 'V2') | extended | |
+ c9 | integer | | | plain | |
+ c10 | integer | | (p1 'v1') | plain | |
Server: s0
FDW Options: (delimiter ',', quote '"', "be quoted" 'value')
Has OIDs: no
diff --git a/src/test/regress/sql/foreign_data.sql b/src/test/regress/sql/foreign_data.sql
index 0c956727deb..4137969e7de 100644
--- a/src/test/regress/sql/foreign_data.sql
+++ b/src/test/regress/sql/foreign_data.sql
@@ -307,6 +307,9 @@ ALTER FOREIGN TABLE ft1 ALTER COLUMN xmin OPTIONS (ADD p1 'v1'); -- ERROR
ALTER FOREIGN TABLE ft1 ALTER COLUMN c7 OPTIONS (ADD p1 'v1', ADD p2 'v2'),
ALTER COLUMN c8 OPTIONS (ADD p1 'v1', ADD p2 'v2');
ALTER FOREIGN TABLE ft1 ALTER COLUMN c8 OPTIONS (SET p2 'V2', DROP p1);
+ALTER FOREIGN TABLE ft1 ALTER COLUMN c1 SET STATISTICS 10000;
+ALTER FOREIGN TABLE ft1 ALTER COLUMN c1 SET (n_distinct = 100);
+ALTER FOREIGN TABLE ft1 ALTER COLUMN c8 SET STATISTICS -1;
\d+ ft1
-- can't change the column type if it's used elsewhere
CREATE TABLE use_ft1_column_type (x ft1);