baserel->baserestrictinfo,
0,
JOIN_INNER,
+ NULL,
NULL);
nrows = clamp_row_est(nrows);
fpinfo->local_conds,
baserel->relid,
JOIN_INNER,
+ NULL,
NULL);
cost_qual_eval(&fpinfo->local_conds_cost, fpinfo->local_conds, root);
local_param_join_conds,
foreignrel->relid,
JOIN_INNER,
+ NULL,
NULL);
local_sel *= fpinfo->local_conds_sel;
fpinfo->local_conds,
0,
JOIN_INNER,
+ NULL,
NULL);
cost_qual_eval(&fpinfo->local_conds_cost, fpinfo->local_conds, root);
if (!fpinfo->use_remote_estimate)
fpinfo->joinclause_sel = clauselist_selectivity(root, fpinfo->joinclauses,
0, fpinfo->jointype,
- extra->sjinfo);
+ extra->sjinfo, NULL);
/* Estimate costs for bare join relation */
estimate_path_cost_size(root, joinrel, NIL, NIL, &rows,
</entry>
</row>
+ <row>
+ <entry><structfield>stadependencies</structfield></entry>
+ <entry><type>pg_dependencies</type></entry>
+ <entry></entry>
+ <entry>
+ Functional dependencies, serialized as <structname>pg_dependencies</> type.
+ </entry>
+ </row>
+
</tbody>
</tgroup>
</table>
in <filename>src/backend/utils/adt/selfuncs.c</filename>.
</para>
+ <sect2 id="functional-dependencies">
+ <title>Functional Dependencies</title>
+
+ <para>
+ The simplest type of extended statistics are functional dependencies,
+ used in definitions of database normal forms. When simplified, saying that
+ <literal>b</> is functionally dependent on <literal>a</> means that
+ knowledge of value of <literal>a</> is sufficient to determine value of
+ <literal>b</>.
+ </para>
+
+ <para>
+ In normalized databases, only functional dependencies on primary keys
+ and superkeys are allowed. However, in practice, many data sets are not
+ fully normalized, for example, due to intentional denormalization for
+ performance reasons.
+ </para>
+
+ <para>
+ Functional dependencies directly affect accuracy of the estimates, as
+ conditions on the dependent column(s) do not restrict the result set,
+ resulting in underestimates.
+ </para>
+
+ <para>
+ To inform the planner about the functional dependencies, we collect
+ measurements of dependency during <command>ANALYZE</>. Assessing
+ dependency between all sets of columns would be prohibitively
+ expensive, so we limit our search to potential dependencies defined
+ using the <command>CREATE STATISTICS</> command.
+
+<programlisting>
+CREATE TABLE t (a INT, b INT);
+INSERT INTO t SELECT i/100, i/100 FROM generate_series(1,10000) s(i);
+CREATE STATISTICS s1 WITH (dependencies) ON (a, b) FROM t;
+ANALYZE t;
+EXPLAIN ANALYZE SELECT * FROM t WHERE a = 1 AND b = 1;
+ QUERY PLAN
+-------------------------------------------------------------------------------------------------
+ Seq Scan on t (cost=0.00..195.00 rows=100 width=8) (actual time=0.095..3.118 rows=100 loops=1)
+ Filter: ((a = 1) AND (b = 1))
+ Rows Removed by Filter: 9900
+ Planning time: 0.367 ms
+ Execution time: 3.380 ms
+(5 rows)
+</programlisting>
+
+ The planner is now aware of the functional dependencies and considers
+ them when computing the selectivity of the second condition. Running
+ the query without the statistics would lead to quite different estimates.
+
+<programlisting>
+DROP STATISTICS s1;
+EXPLAIN ANALYZE SELECT * FROM t WHERE a = 1 AND b = 1;
+ QUERY PLAN
+-----------------------------------------------------------------------------------------------
+ Seq Scan on t (cost=0.00..195.00 rows=1 width=8) (actual time=0.000..6.379 rows=100 loops=1)
+ Filter: ((a = 1) AND (b = 1))
+ Rows Removed by Filter: 9900
+ Planning time: 0.000 ms
+ Execution time: 6.379 ms
+(5 rows)
+</programlisting>
+ </para>
+
+ <para>
+ If no dependency exists, the collected statistics do not influence the
+ query plan. The only effect is to slow down <command>ANALYZE</>. Should
+ partial dependencies exist these will also be stored and applied
+ during planning.
+ </para>
+
+ <para>
+ Similarly to per-column statistics, extended statistics are stored in
+ a system catalog called <structname>pg_statistic_ext</structname>, but
+ there is also a more convenient view <structname>pg_stats_ext</structname>.
+ To inspect the statistics <literal>s1</literal> defined above,
+ you may do this:
+
+<programlisting>
+SELECT tablename, staname, attnums, depsbytes
+ FROM pg_stats_ext WHERE staname = 's1';
+ tablename | staname | attnums | depsbytes
+-----------+---------+---------+-----------
+ t | s1 | 1 2 | 40
+(1 row)
+</programlisting>
+
+ This shows that the statistics are defined on table <structname>t</>,
+ <structfield>attnums</structfield> lists attribute numbers of columns
+ (references <structname>pg_attribute</structname>). It also shows
+ the length in bytes of the functional dependencies, as found by
+ <command>ANALYZE</> when serialized into a <literal>bytea</> column.
+ </para>
+
+ <para>
+ When computing the selectivity, the planner inspects all conditions and
+ attempts to identify which conditions are already implied by other
+ conditions. The selectivity estimates from any redundant conditions are
+ ignored from a selectivity point of view. In the example query above,
+ the selectivity estimates for either of the conditions may be eliminated,
+ thus improving the overall estimate.
+ </para>
+
+ <sect3 id="functional-dependencies-limitations">
+ <title>Limitations of functional dependencies</title>
+
+ <para>
+ Functional dependencies are a very simple type of statistics, and
+ as such have several limitations. The first limitation is that they
+ only work with simple equality conditions, comparing columns and constant
+ values. It's not possible to use them to eliminate equality conditions
+ comparing two columns or a column to an expression, range clauses,
+ <literal>LIKE</> or any other type of conditions.
+ </para>
+
+ <para>
+ When eliminating the implied conditions, the planner assumes that the
+ conditions are compatible. Consider the following example, violating
+ this assumption:
+
+<programlisting>
+EXPLAIN ANALYZE SELECT * FROM t WHERE a = 1 AND b = 10;
+ QUERY PLAN
+-----------------------------------------------------------------------------------------------
+ Seq Scan on t (cost=0.00..195.00 rows=100 width=8) (actual time=2.992..2.992 rows=0 loops=1)
+ Filter: ((a = 1) AND (b = 10))
+ Rows Removed by Filter: 10000
+ Planning time: 0.232 ms
+ Execution time: 3.033 ms
+(5 rows)
+</programlisting>
+
+ While there are no rows with such combination of values, the planner
+ is unable to verify whether the values match - it only knows that
+ the columns are functionally dependent.
+ </para>
+
+ <para>
+ This assumption is more about queries executed on the database - in many
+ cases, it's actually satisfied (e.g. when the GUI only allows selecting
+ compatible values). But if that's not the case, functional dependencies
+ may not be a viable option.
+ </para>
+
+ <para>
+ For additional information about functional dependencies, see
+ <filename>src/backend/statistics/README.dependencies</>.
+ </para>
+
+ </sect3>
+
+ </sect2>
+
</sect1>
</chapter>
<refsynopsisdiv>
<synopsis>
-CREATE STATISTICS [ IF NOT EXISTS ] <replaceable class="PARAMETER">statistics_name</replaceable> ON (
- <replaceable class="PARAMETER">column_name</replaceable>, <replaceable class="PARAMETER">column_name</replaceable> [, ...])
+CREATE STATISTICS [ IF NOT EXISTS ] <replaceable class="PARAMETER">statistics_name</replaceable>
+ WITH ( <replaceable class="PARAMETER">option</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] )
+ ON ( <replaceable class="PARAMETER">column_name</replaceable>, <replaceable class="PARAMETER">column_name</replaceable> [, ...])
FROM <replaceable class="PARAMETER">table_name</replaceable>
</synopsis>
</variablelist>
+ <refsect2 id="SQL-CREATESTATISTICS-parameters">
+ <title id="SQL-CREATESTATISTICS-parameters-title">Parameters</title>
+
+ <indexterm zone="sql-createstatistics-parameters">
+ <primary>statistics parameters</primary>
+ </indexterm>
+
+ <para>
+ The <literal>WITH</> clause can specify <firstterm>options</>
+ for the statistics. Available options are listed below.
+ </para>
+
+ <variablelist>
+
+ <varlistentry>
+ <term><literal>dependencies</> (<type>boolean</>)</term>
+ <listitem>
+ <para>
+ Enables functional dependencies for the statistics.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term><literal>ndistinct</> (<type>boolean</>)</term>
+ <listitem>
+ <para>
+ Enables ndistinct coefficients for the statistics.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ </variablelist>
+
+ </refsect2>
</refsect1>
<refsect1>
INSERT INTO t1 SELECT i/100, i/500
FROM generate_series(1,1000000) s(i);
-CREATE STATISTICS s1 ON (a, b) FROM t1;
+CREATE STATISTICS s1 WITH (dependencies) ON (a, b) FROM t1;
ANALYZE t1;
C.relname AS tablename,
S.staname AS staname,
S.stakeys AS attnums,
- length(s.standistinct) AS ndistbytes
+ length(s.standistinct::bytea) AS ndistbytes,
+ length(S.stadependencies::bytea) AS depsbytes
FROM (pg_statistic_ext S JOIN pg_class C ON (C.oid = S.starelid))
LEFT JOIN pg_namespace N ON (N.oid = C.relnamespace);
Oid relid;
ObjectAddress parentobject,
childobject;
- Datum types[1]; /* only ndistinct defined now */
+ Datum types[2]; /* one for each possible type of statistics */
int ntypes;
ArrayType *staenabled;
bool build_ndistinct;
+ bool build_dependencies;
bool requested_type = false;
Assert(IsA(stmt, CreateStatsStmt));
errmsg("statistics require at least 2 columns")));
/*
- * Sort the attnums, which makes detecting duplicies somewhat easier, and
+ * Sort the attnums, which makes detecting duplicities somewhat easier, and
* it does not hurt (it does not affect the efficiency, unlike for
* indexes, for example).
*/
* recognized.
*/
build_ndistinct = false;
+ build_dependencies = false;
foreach(l, stmt->options)
{
DefElem *opt = (DefElem *) lfirst(l);
build_ndistinct = defGetBoolean(opt);
requested_type = true;
}
+ else if (strcmp(opt->defname, "dependencies") == 0)
+ {
+ build_dependencies = defGetBoolean(opt);
+ requested_type = true;
+ }
else
ereport(ERROR,
(errcode(ERRCODE_SYNTAX_ERROR),
}
/* If no statistic type was specified, build them all. */
if (!requested_type)
+ {
build_ndistinct = true;
+ build_dependencies = true;
+ }
/* construct the char array of enabled statistic types */
ntypes = 0;
if (build_ndistinct)
types[ntypes++] = CharGetDatum(STATS_EXT_NDISTINCT);
+ if (build_dependencies)
+ types[ntypes++] = CharGetDatum(STATS_EXT_DEPENDENCIES);
Assert(ntypes > 0);
staenabled = construct_array(types, ntypes, CHAROID, 1, true, 'c');
/* no statistics build yet */
nulls[Anum_pg_statistic_ext_standistinct - 1] = true;
+ nulls[Anum_pg_statistic_ext_stadependencies - 1] = true;
/* insert it into pg_statistic_ext */
statrel = heap_open(StatisticExtRelationId, RowExclusiveLock);
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/selfuncs.h"
+#include "statistics/statistics.h"
/*
* subclauses. However, that's only right if the subclauses have independent
* probabilities, and in reality they are often NOT independent. So,
* we want to be smarter where we can.
-
- * Currently, the only extra smarts we have is to recognize "range queries",
- * such as "x > 34 AND x < 42". Clauses are recognized as possible range
- * query components if they are restriction opclauses whose operators have
- * scalarltsel() or scalargtsel() as their restriction selectivity estimator.
- * We pair up clauses of this form that refer to the same variable. An
- * unpairable clause of this kind is simply multiplied into the selectivity
- * product in the normal way. But when we find a pair, we know that the
- * selectivities represent the relative positions of the low and high bounds
- * within the column's range, so instead of figuring the selectivity as
- * hisel * losel, we can figure it as hisel + losel - 1. (To visualize this,
- * see that hisel is the fraction of the range below the high bound, while
- * losel is the fraction above the low bound; so hisel can be interpreted
- * directly as a 0..1 value but we need to convert losel to 1-losel before
- * interpreting it as a value. Then the available range is 1-losel to hisel.
- * However, this calculation double-excludes nulls, so really we need
- * hisel + losel + null_frac - 1.)
+ *
+ * When 'rel' is not null and rtekind = RTE_RELATION, we'll try to apply
+ * selectivity estimates using any extended statistcs on 'rel'.
+ *
+ * If we identify such extended statistics exist, we try to apply them.
+ * Currently we only have (soft) functional dependencies, so apply these in as
+ * many cases as possible, and fall back on normal estimates for remaining
+ * clauses.
+ *
+ * We also recognize "range queries", such as "x > 34 AND x < 42". Clauses
+ * are recognized as possible range query components if they are restriction
+ * opclauses whose operators have scalarltsel() or scalargtsel() as their
+ * restriction selectivity estimator. We pair up clauses of this form that
+ * refer to the same variable. An unpairable clause of this kind is simply
+ * multiplied into the selectivity product in the normal way. But when we
+ * find a pair, we know that the selectivities represent the relative
+ * positions of the low and high bounds within the column's range, so instead
+ * of figuring the selectivity as hisel * losel, we can figure it as hisel +
+ * losel - 1. (To visualize this, see that hisel is the fraction of the range
+ * below the high bound, while losel is the fraction above the low bound; so
+ * hisel can be interpreted directly as a 0..1 value but we need to convert
+ * losel to 1-losel before interpreting it as a value. Then the available
+ * range is 1-losel to hisel. However, this calculation double-excludes
+ * nulls, so really we need hisel + losel + null_frac - 1.)
*
* If either selectivity is exactly DEFAULT_INEQ_SEL, we forget this equation
* and instead use DEFAULT_RANGE_INEQ_SEL. The same applies if the equation
List *clauses,
int varRelid,
JoinType jointype,
- SpecialJoinInfo *sjinfo)
+ SpecialJoinInfo *sjinfo,
+ RelOptInfo *rel)
{
Selectivity s1 = 1.0;
RangeQueryClause *rqlist = NULL;
ListCell *l;
+ Bitmapset *estimatedclauses = NULL;
+ int listidx;
/*
- * If there's exactly one clause, then no use in trying to match up pairs,
- * so just go directly to clause_selectivity().
+ * If there's exactly one clause, then extended statistics is futile at
+ * this level (we might be able to apply them later if it's AND/OR
+ * clause). So just go directly to clause_selectivity().
*/
if (list_length(clauses) == 1)
return clause_selectivity(root, (Node *) linitial(clauses),
- varRelid, jointype, sjinfo);
+ varRelid, jointype, sjinfo, rel);
+
+ /*
+ * When a relation of RTE_RELATION is given as 'rel', we'll try to
+ * perform selectivity estimation using extended statistics.
+ */
+ if (rel && rel->rtekind == RTE_RELATION && rel->statlist != NIL)
+ {
+ /*
+ * Perform selectivity estimations on any clauses found applicable by
+ * dependencies_clauselist_selectivity. The 0-based list position of
+ * estimated clauses will be populated in 'estimatedclauses'.
+ */
+ s1 *= dependencies_clauselist_selectivity(root, clauses, varRelid,
+ jointype, sjinfo, rel, &estimatedclauses);
+
+ /*
+ * This would be the place to apply any other types of extended
+ * statistics selectivity estimations for remaining clauses.
+ */
+ }
/*
- * Initial scan over clauses. Anything that doesn't look like a potential
- * rangequery clause gets multiplied into s1 and forgotten. Anything that
- * does gets inserted into an rqlist entry.
+ * Apply normal selectivity estimates for remaining clauses. We'll be
+ * careful to skip any clauses which were already estimated above.
+ *
+ * Anything that doesn't look like a potential rangequery clause gets
+ * multiplied into s1 and forgotten. Anything that does gets inserted into
+ * an rqlist entry.
*/
+ listidx = -1;
foreach(l, clauses)
{
Node *clause = (Node *) lfirst(l);
RestrictInfo *rinfo;
Selectivity s2;
+ listidx++;
+
+ /*
+ * Skip this clause if it's already been estimated by some other
+ * statistics above.
+ */
+ if (bms_is_member(listidx, estimatedclauses))
+ continue;
+
/* Always compute the selectivity using clause_selectivity */
- s2 = clause_selectivity(root, clause, varRelid, jointype, sjinfo);
+ s2 = clause_selectivity(root, clause, varRelid, jointype, sjinfo, rel);
/*
* Check for being passed a RestrictInfo.
Node *clause,
int varRelid,
JoinType jointype,
- SpecialJoinInfo *sjinfo)
+ SpecialJoinInfo *sjinfo,
+ RelOptInfo *rel)
{
Selectivity s1 = 0.5; /* default for any unhandled clause type */
RestrictInfo *rinfo = NULL;
(Node *) get_notclausearg((Expr *) clause),
varRelid,
jointype,
- sjinfo);
+ sjinfo,
+ rel);
}
else if (and_clause(clause))
{
((BoolExpr *) clause)->args,
varRelid,
jointype,
- sjinfo);
+ sjinfo,
+ rel);
}
else if (or_clause(clause))
{
(Node *) lfirst(arg),
varRelid,
jointype,
- sjinfo);
+ sjinfo,
+ rel);
s1 = s1 + s2 - s1 * s2;
}
(Node *) ((RelabelType *) clause)->arg,
varRelid,
jointype,
- sjinfo);
+ sjinfo,
+ rel);
}
else if (IsA(clause, CoerceToDomain))
{
(Node *) ((CoerceToDomain *) clause)->arg,
varRelid,
jointype,
- sjinfo);
+ sjinfo,
+ rel);
}
else
{
joinquals,
0,
jointype,
- sjinfo);
+ sjinfo,
+ NULL);
/*
* Also get the normal inner-join selectivity of the join clauses.
joinquals,
0,
JOIN_INNER,
- &norm_sjinfo);
+ &norm_sjinfo,
+ NULL);
/* Avoid leaking a lot of ListCells */
if (jointype == JOIN_ANTI)
Node *qual = (Node *) lfirst(l);
/* Note that clause_selectivity will be able to cache its result */
- selec *= clause_selectivity(root, qual, 0, JOIN_INNER, &sjinfo);
+ selec *= clause_selectivity(root, qual, 0, JOIN_INNER, &sjinfo, NULL);
}
/* Apply it to the input relation sizes */
rel->baserestrictinfo,
0,
JOIN_INNER,
- NULL);
+ NULL,
+ rel);
rel->rows = clamp_row_est(nrows);
allclauses,
rel->relid, /* do not use 0! */
JOIN_INNER,
- NULL);
+ NULL,
+ rel);
nrows = clamp_row_est(nrows);
/* For safety, make sure result is not more than the base estimate */
if (nrows > rel->rows)
joinquals,
0,
jointype,
- sjinfo);
+ sjinfo,
+ NULL);
pselec = clauselist_selectivity(root,
pushedquals,
0,
jointype,
- sjinfo);
+ sjinfo,
+ NULL);
/* Avoid leaking a lot of ListCells */
list_free(joinquals);
restrictlist,
0,
jointype,
- sjinfo);
+ sjinfo,
+ NULL);
pselec = 0.0; /* not used, keep compiler quiet */
}
Selectivity csel;
csel = clause_selectivity(root, (Node *) rinfo,
- 0, jointype, sjinfo);
+ 0, jointype, sjinfo, NULL);
thisfksel = Min(thisfksel, csel);
}
fkselec *= thisfksel;
* saving work later.)
*/
or_selec = clause_selectivity(root, (Node *) or_rinfo,
- 0, JOIN_INNER, NULL);
+ 0, JOIN_INNER, NULL, rel);
/*
* The clause is only worth adding to the query if it rejects a useful
/* Compute inner-join size */
orig_selec = clause_selectivity(root, (Node *) join_or_rinfo,
- 0, JOIN_INNER, &sjinfo);
+ 0, JOIN_INNER, &sjinfo, NULL);
/* And hack cached selectivity so join size remains the same */
join_or_rinfo->norm_selec = orig_selec / or_selec;
stainfos = lcons(info, stainfos);
}
+ if (statext_is_kind_built(htup, STATS_EXT_DEPENDENCIES))
+ {
+ StatisticExtInfo *info = makeNode(StatisticExtInfo);
+
+ info->statOid = statOid;
+ info->rel = rel;
+ info->kind = STATS_EXT_DEPENDENCIES;
+ info->keys = bms_copy(keys);
+
+ stainfos = lcons(info, stainfos);
+ }
+
ReleaseSysCache(htup);
bms_free(keys);
}
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = extended_stats.o mvdistinct.o
+OBJS = extended_stats.o dependencies.o mvdistinct.o
include $(top_srcdir)/src/backend/common.mk
Extended statistics track different types of dependencies between the columns,
hopefully improving the estimates and producing better plans.
-Currently we only have one type of extended statistics - ndistinct
-coefficients, and we use it to improve estimates of grouping queries. See
-README.ndistinct for details.
+Types of statistics
+-------------------
+
+There are two kinds of extended statistics:
+
+ (a) ndistinct coefficients
+
+ (b) soft functional dependencies (README.dependencies)
+
+
+Compatible clause types
+-----------------------
+
+Each type of statistics may be used to estimate some subset of clause types.
+
+ (a) functional dependencies - equality clauses (AND), possibly IS NULL
+
+Currently, only OpExprs in the form Var op Const, or Const op Var are
+supported, however it's feasible to expand the code later to also estimate the
+selectivities on clauses such as Var op Var.
+
+
+Complex clauses
+---------------
+
+We also support estimating more complex clauses - essentially AND/OR clauses
+with (Var op Const) as leaves, as long as all the referenced attributes are
+covered by a single statistics.
+
+For example this condition
+
+ (a=1) AND ((b=2) OR ((c=3) AND (d=4)))
+
+may be estimated using statistics on (a,b,c,d). If we only have statistics on
+(b,c,d) we may estimate the second part, and estimate (a=1) using simple stats.
+
+If we only have statistics on (a,b,c) we can't apply it at all at this point,
+but it's worth pointing out clauselist_selectivity() works recursively and when
+handling the second part (the OR-clause), we'll be able to apply the statistics.
+
+Note: The multi-statistics estimation patch also makes it possible to pass some
+clauses as 'conditions' into the deeper parts of the expression tree.
+
+
+Selectivity estimation
+----------------------
+
+Throughout the planner clauselist_selectivity() still remains in charge of
+most selectivity estimate requests. clauselist_selectivity() can be instructed
+to try to make use of any extended statistics on the given RelOptInfo, which
+it will do, if:
+
+ (a) An actual valid RelOptInfo was given. Join relations are passed in as
+ NULL, therefore are invalid.
+
+ (b) The relation given actually has any extended statistics defined which
+ are actually built.
+
+When the above conditions are met, clauselist_selectivity() first attempts to
+pass the clause list off to the extended statistics selectivity estimation
+function. This functions may not find any clauses which is can perform any
+estimations on. In such cases these clauses are simply ignored. When actual
+estimation work is performed in these functions they're expected to mark which
+clauses they've performed estimations for so that any other function
+performing estimations knows which clauses are to be skipped.
Size of sample in ANALYZE
-------------------------
--- /dev/null
+Soft functional dependencies
+============================
+
+Functional dependencies are a concept well described in relational theory,
+particularly in the definition of normalization and "normal forms". Wikipedia
+has a nice definition of a functional dependency [1]:
+
+ In a given table, an attribute Y is said to have a functional dependency
+ on a set of attributes X (written X -> Y) if and only if each X value is
+ associated with precisely one Y value. For example, in an "Employee"
+ table that includes the attributes "Employee ID" and "Employee Date of
+ Birth", the functional dependency
+
+ {Employee ID} -> {Employee Date of Birth}
+
+ would hold. It follows from the previous two sentences that each
+ {Employee ID} is associated with precisely one {Employee Date of Birth}.
+
+ [1] https://en.wikipedia.org/wiki/Functional_dependency
+
+In practical terms, functional dependencies mean that a value in one column
+determines values in some other column. Consider for example this trivial
+table with two integer columns:
+
+ CREATE TABLE t (a INT, b INT)
+ AS SELECT i, i/10 FROM generate_series(1,100000) s(i);
+
+Clearly, knowledge of the value in column 'a' is sufficient to determine the
+value in column 'b', as it's simply (a/10). A more practical example may be
+addresses, where the knowledge of a ZIP code (usually) determines city. Larger
+cities may have multiple ZIP codes, so the dependency can't be reversed.
+
+Many datasets might be normalized not to contain such dependencies, but often
+it's not practical for various reasons. In some cases, it's actually a conscious
+design choice to model the dataset in a denormalized way, either because of
+performance or to make querying easier.
+
+
+Soft dependencies
+-----------------
+
+Real-world data sets often contain data errors, either because of data entry
+mistakes (user mistyping the ZIP code) or perhaps issues in generating the
+data (e.g. a ZIP code mistakenly assigned to two cities in different states).
+
+A strict implementation would either ignore dependencies in such cases,
+rendering the approach mostly useless even for slightly noisy data sets, or
+result in sudden changes in behavior depending on minor differences between
+samples provided to ANALYZE.
+
+For this reason, the statistics implements "soft" functional dependencies,
+associating each functional dependency with a degree of validity (a number
+between 0 and 1). This degree is then used to combine selectivities in a
+smooth manner.
+
+
+Mining dependencies (ANALYZE)
+-----------------------------
+
+The current algorithm is fairly simple - generate all possible functional
+dependencies, and for each one count the number of rows consistent with it.
+Then use the fraction of rows (supporting/total) as the degree.
+
+To count the rows consistent with the dependency (a => b):
+
+ (a) Sort the data lexicographically, i.e. first by 'a' then 'b'.
+
+ (b) For each group of rows with the same 'a' value, count the number of
+ distinct values in 'b'.
+
+ (c) If there's a single distinct value in 'b', the rows are consistent with
+ the functional dependency, otherwise they contradict it.
+
+The algorithm also requires a minimum size of the group to consider it
+consistent (currently 3 rows in the sample). Small groups make it less likely
+to break the consistency.
+
+
+Clause reduction (planner/optimizer)
+------------------------------------
+
+Applying the functional dependencies is fairly simple - given a list of
+equality clauses, we compute selectivities of each clause and then use the
+degree to combine them using this formula
+
+ P(a=?,b=?) = P(a=?) * (d + (1-d) * P(b=?))
+
+Where 'd' is the degree of functional dependence (a=>b).
+
+With more than two equality clauses, this process happens recursively. For
+example for (a,b,c) we first use (a,b=>c) to break the computation into
+
+ P(a=?,b=?,c=?) = P(a=?,b=?) * (d + (1-d)*P(b=?))
+
+and then apply (a=>b) the same way on P(a=?,b=?).
+
+
+Consistency of clauses
+----------------------
+
+Functional dependencies only express general dependencies between columns,
+without referencing particular values. This assumes that the equality clauses
+are in fact consistent with the functional dependency, i.e. that given a
+dependency (a=>b), the value in (b=?) clause is the value determined by (a=?).
+If that's not the case, the clauses are "inconsistent" with the functional
+dependency and the result will be over-estimation.
+
+This may happen, for example, when using conditions on the ZIP code and city
+name with mismatching values (ZIP code for a different city), etc. In such a
+case, the result set will be empty, but we'll estimate the selectivity using
+the ZIP code condition.
+
+In this case, the default estimation based on AVIA principle happens to work
+better, but mostly by chance.
+
+This issue is the price for the simplicity of functional dependencies. If the
+application frequently constructs queries with clauses inconsistent with
+functional dependencies present in the data, the best solution is not to
+use functional dependencies, but one of the more complex types of statistics.
--- /dev/null
+/*-------------------------------------------------------------------------
+ *
+ * dependencies.c
+ * POSTGRES functional dependencies
+ *
+ * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/statistics/dependencies.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/htup_details.h"
+#include "access/sysattr.h"
+#include "catalog/pg_operator.h"
+#include "catalog/pg_statistic_ext.h"
+#include "lib/stringinfo.h"
+#include "optimizer/clauses.h"
+#include "optimizer/cost.h"
+#include "optimizer/var.h"
+#include "nodes/nodes.h"
+#include "nodes/relation.h"
+#include "statistics/extended_stats_internal.h"
+#include "statistics/statistics.h"
+#include "utils/bytea.h"
+#include "utils/fmgroids.h"
+#include "utils/fmgrprotos.h"
+#include "utils/lsyscache.h"
+#include "utils/syscache.h"
+#include "utils/typcache.h"
+
+/*
+ * Internal state for DependencyGenerator of dependencies. Dependencies are similar to
+ * k-permutations of n elements, except that the order does not matter for the
+ * first (k-1) elements. That is, (a,b=>c) and (b,a=>c) are equivalent.
+ */
+typedef struct DependencyGeneratorData
+{
+ int k; /* size of the dependency */
+ int n; /* number of possible attributes */
+ int current; /* next dependency to return (index) */
+ AttrNumber ndependencies; /* number of dependencies generated */
+ AttrNumber *dependencies; /* array of pre-generated dependencies */
+} DependencyGeneratorData;
+
+typedef DependencyGeneratorData *DependencyGenerator;
+
+static void generate_dependencies_recurse(DependencyGenerator state,
+ int index, AttrNumber start, AttrNumber *current);
+static void generate_dependencies(DependencyGenerator state);
+static DependencyGenerator DependencyGenerator_init(int n, int k);
+static void DependencyGenerator_free(DependencyGenerator state);
+static AttrNumber *DependencyGenerator_next(DependencyGenerator state);
+static double dependency_degree(int numrows, HeapTuple *rows, int k,
+ AttrNumber *dependency, VacAttrStats **stats, Bitmapset *attrs);
+static bool dependency_is_fully_matched(MVDependency *dependency,
+ Bitmapset *attnums);
+static bool dependency_implies_attribute(MVDependency *dependency,
+ AttrNumber attnum);
+static bool dependency_is_compatible_clause(Node *clause, Index relid,
+ AttrNumber *attnum);
+static MVDependency *find_strongest_dependency(StatisticExtInfo *stats,
+ MVDependencies *dependencies,
+ Bitmapset *attnums);
+
+static void
+generate_dependencies_recurse(DependencyGenerator state, int index,
+ AttrNumber start, AttrNumber *current)
+{
+ /*
+ * The generator handles the first (k-1) elements differently from the
+ * last element.
+ */
+ if (index < (state->k - 1))
+ {
+ AttrNumber i;
+
+ /*
+ * The first (k-1) values have to be in ascending order, which we
+ * generate recursively.
+ */
+
+ for (i = start; i < state->n; i++)
+ {
+ current[index] = i;
+ generate_dependencies_recurse(state, (index + 1), (i + 1), current);
+ }
+ }
+ else
+ {
+ int i;
+
+ /*
+ * the last element is the implied value, which does not respect the
+ * ascending order. We just need to check that the value is not in the
+ * first (k-1) elements.
+ */
+
+ for (i = 0; i < state->n; i++)
+ {
+ int j;
+ bool match = false;
+
+ current[index] = i;
+
+ for (j = 0; j < index; j++)
+ {
+ if (current[j] == i)
+ {
+ match = true;
+ break;
+ }
+ }
+
+ /*
+ * If the value is not found in the first part of the dependency,
+ * we're done.
+ */
+ if (!match)
+ {
+ state->dependencies = (AttrNumber *) repalloc(state->dependencies,
+ state->k * (state->ndependencies + 1) * sizeof(AttrNumber));
+ memcpy(&state->dependencies[(state->k * state->ndependencies)],
+ current, state->k * sizeof(AttrNumber));
+ state->ndependencies++;
+ }
+ }
+ }
+}
+
+/* generate all dependencies (k-permutations of n elements) */
+static void
+generate_dependencies(DependencyGenerator state)
+{
+ AttrNumber *current = (AttrNumber *) palloc0(sizeof(AttrNumber) * state->k);
+
+ generate_dependencies_recurse(state, 0, 0, current);
+
+ pfree(current);
+}
+
+/*
+ * initialize the DependencyGenerator of variations, and prebuild the variations
+ *
+ * This pre-builds all the variations. We could also generate them in
+ * DependencyGenerator_next(), but this seems simpler.
+ */
+static DependencyGenerator
+DependencyGenerator_init(int n, int k)
+{
+ DependencyGenerator state;
+
+ Assert((n >= k) && (k > 0));
+
+ /* allocate the DependencyGenerator state */
+ state = (DependencyGenerator) palloc0(sizeof(DependencyGeneratorData));
+ state->dependencies = (AttrNumber *) palloc(k * sizeof(AttrNumber));
+
+ state->ndependencies = 0;
+ state->current = 0;
+ state->k = k;
+ state->n = n;
+
+ /* now actually pre-generate all the variations */
+ generate_dependencies(state);
+
+ return state;
+}
+
+/* free the DependencyGenerator state */
+static void
+DependencyGenerator_free(DependencyGenerator state)
+{
+ pfree(state->dependencies);
+ pfree(state);
+
+}
+
+/* generate next combination */
+static AttrNumber *
+DependencyGenerator_next(DependencyGenerator state)
+{
+ if (state->current == state->ndependencies)
+ return NULL;
+
+ return &state->dependencies[state->k * state->current++];
+}
+
+
+/*
+ * validates functional dependency on the data
+ *
+ * An actual work horse of detecting functional dependencies. Given a variation
+ * of k attributes, it checks that the first (k-1) are sufficient to determine
+ * the last one.
+ */
+static double
+dependency_degree(int numrows, HeapTuple *rows, int k, AttrNumber *dependency,
+ VacAttrStats **stats, Bitmapset *attrs)
+{
+ int i,
+ j;
+ int nvalues = numrows * k;
+ MultiSortSupport mss;
+ SortItem *items;
+ Datum *values;
+ bool *isnull;
+ int *attnums;
+
+ /* counters valid within a group */
+ int group_size = 0;
+ int n_violations = 0;
+
+ /* total number of rows supporting (consistent with) the dependency */
+ int n_supporting_rows = 0;
+
+ /* Make sure we have at least two input attributes. */
+ Assert(k >= 2);
+
+ /* sort info for all attributes columns */
+ mss = multi_sort_init(k);
+
+ /* data for the sort */
+ items = (SortItem *) palloc(numrows * sizeof(SortItem));
+ values = (Datum *) palloc(sizeof(Datum) * nvalues);
+ isnull = (bool *) palloc(sizeof(bool) * nvalues);
+
+ /* fix the pointers to values/isnull */
+ for (i = 0; i < numrows; i++)
+ {
+ items[i].values = &values[i * k];
+ items[i].isnull = &isnull[i * k];
+ }
+
+ /*
+ * Transform the bms into an array, to make accessing i-th member easier.
+ */
+ attnums = (int *) palloc(sizeof(int) * bms_num_members(attrs));
+ i = 0;
+ j = -1;
+ while ((j = bms_next_member(attrs, j)) >= 0)
+ attnums[i++] = j;
+
+ /*
+ * Verify the dependency (a,b,...)->z, using a rather simple algorithm:
+ *
+ * (a) sort the data lexicographically
+ *
+ * (b) split the data into groups by first (k-1) columns
+ *
+ * (c) for each group count different values in the last column
+ */
+
+ /* prepare the sort function for the first dimension, and SortItem array */
+ for (i = 0; i < k; i++)
+ {
+ VacAttrStats *colstat = stats[dependency[i]];
+ TypeCacheEntry *type;
+
+ type = lookup_type_cache(colstat->attrtypid, TYPECACHE_LT_OPR);
+ if (type->lt_opr == InvalidOid) /* shouldn't happen */
+ elog(ERROR, "cache lookup failed for ordering operator for type %u",
+ colstat->attrtypid);
+
+ /* prepare the sort function for this dimension */
+ multi_sort_add_dimension(mss, i, type->lt_opr);
+
+ /* accumulate all the data for both columns into an array and sort it */
+ for (j = 0; j < numrows; j++)
+ {
+ items[j].values[i] =
+ heap_getattr(rows[j], attnums[dependency[i]],
+ stats[i]->tupDesc, &items[j].isnull[i]);
+ }
+ }
+
+ /* sort the items so that we can detect the groups */
+ qsort_arg((void *) items, numrows, sizeof(SortItem),
+ multi_sort_compare, mss);
+
+ /*
+ * Walk through the sorted array, split it into rows according to the
+ * first (k-1) columns. If there's a single value in the last column, we
+ * count the group as 'supporting' the functional dependency. Otherwise we
+ * count it as contradicting.
+ *
+ * We also require a group to have a minimum number of rows to be
+ * considered useful for supporting the dependency. Contradicting groups
+ * may be of any size, though.
+ *
+ * XXX The minimum size requirement makes it impossible to identify case
+ * when both columns are unique (or nearly unique), and therefore
+ * trivially functionally dependent.
+ */
+
+ /* start with the first row forming a group */
+ group_size = 1;
+
+ /* loop 1 beyond the end of the array so that we count the final group */
+ for (i = 1; i <= numrows; i++)
+ {
+ /*
+ * Check if the group ended, which may be either because we processed
+ * all the items (i==numrows), or because the i-th item is not equal
+ * to the preceding one.
+ */
+ if (i == numrows ||
+ multi_sort_compare_dims(0, k - 2, &items[i - 1], &items[i], mss) != 0)
+ {
+ /*
+ * If no violations were found in the group then track the rows of
+ * the group as supporting the functional dependency.
+ */
+ if (n_violations == 0)
+ n_supporting_rows += group_size;
+
+ /* Reset counters for the new group */
+ n_violations = 0;
+ group_size = 1;
+ continue;
+ }
+ /* first columns match, but the last one does not (so contradicting) */
+ else if (multi_sort_compare_dim(k - 1, &items[i - 1], &items[i], mss) != 0)
+ n_violations++;
+
+ group_size++;
+ }
+
+ pfree(items);
+ pfree(values);
+ pfree(isnull);
+ pfree(mss);
+
+ /* Compute the 'degree of validity' as (supporting/total). */
+ return (n_supporting_rows * 1.0 / numrows);
+}
+
+/*
+ * detects functional dependencies between groups of columns
+ *
+ * Generates all possible subsets of columns (variations) and computes
+ * the degree of validity for each one. For example with a statistic on
+ * three columns (a,b,c) there are 9 possible dependencies
+ *
+ * two columns three columns
+ * ----------- -------------
+ * (a) -> b (a,b) -> c
+ * (a) -> c (a,c) -> b
+ * (b) -> a (b,c) -> a
+ * (b) -> c
+ * (c) -> a
+ * (c) -> b
+ */
+MVDependencies *
+statext_dependencies_build(int numrows, HeapTuple *rows, Bitmapset *attrs,
+ VacAttrStats **stats)
+{
+ int i,
+ j,
+ k;
+ int numattrs;
+ int *attnums;
+
+ /* result */
+ MVDependencies *dependencies = NULL;
+
+ numattrs = bms_num_members(attrs);
+
+ /*
+ * Transform the bms into an array, to make accessing i-th member easier.
+ */
+ attnums = palloc(sizeof(int) * bms_num_members(attrs));
+ i = 0;
+ j = -1;
+ while ((j = bms_next_member(attrs, j)) >= 0)
+ attnums[i++] = j;
+
+ Assert(numattrs >= 2);
+
+ /*
+ * We'll try build functional dependencies starting from the smallest ones
+ * covering just 2 columns, to the largest ones, covering all columns
+ * included in the statistics. We start from the smallest ones because we
+ * want to be able to skip already implied ones.
+ */
+ for (k = 2; k <= numattrs; k++)
+ {
+ AttrNumber *dependency; /* array with k elements */
+
+ /* prepare a DependencyGenerator of variation */
+ DependencyGenerator DependencyGenerator = DependencyGenerator_init(numattrs, k);
+
+ /* generate all possible variations of k values (out of n) */
+ while ((dependency = DependencyGenerator_next(DependencyGenerator)))
+ {
+ double degree;
+ MVDependency *d;
+
+ /* compute how valid the dependency seems */
+ degree = dependency_degree(numrows, rows, k, dependency, stats, attrs);
+
+ /*
+ * if the dependency seems entirely invalid, don't store it it
+ */
+ if (degree == 0.0)
+ continue;
+
+ d = (MVDependency *) palloc0(offsetof(MVDependency, attributes)
+ + k * sizeof(AttrNumber));
+
+ /* copy the dependency (and keep the indexes into stakeys) */
+ d->degree = degree;
+ d->nattributes = k;
+ for (i = 0; i < k; i++)
+ d->attributes[i] = attnums[dependency[i]];
+
+ /* initialize the list of dependencies */
+ if (dependencies == NULL)
+ {
+ dependencies
+ = (MVDependencies *) palloc0(sizeof(MVDependencies));
+
+ dependencies->magic = STATS_DEPS_MAGIC;
+ dependencies->type = STATS_DEPS_TYPE_BASIC;
+ dependencies->ndeps = 0;
+ }
+
+ dependencies->ndeps++;
+ dependencies = (MVDependencies *) repalloc(dependencies,
+ offsetof(MVDependencies, deps)
+ + dependencies->ndeps * sizeof(MVDependency));
+
+ dependencies->deps[dependencies->ndeps - 1] = d;
+ }
+
+ /*
+ * we're done with variations of k elements, so free the
+ * DependencyGenerator
+ */
+ DependencyGenerator_free(DependencyGenerator);
+ }
+
+ return dependencies;
+}
+
+
+/*
+ * Serialize list of dependencies into a bytea value.
+ */
+bytea *
+statext_dependencies_serialize(MVDependencies * dependencies)
+{
+ int i;
+ bytea *output;
+ char *tmp;
+ Size len;
+
+ /* we need to store ndeps, with a number of attributes for each one */
+ len = VARHDRSZ + SizeOfDependencies
+ + dependencies->ndeps * SizeOfDependency;
+
+ /* and also include space for the actual attribute numbers and degrees */
+ for (i = 0; i < dependencies->ndeps; i++)
+ len += (sizeof(AttrNumber) * dependencies->deps[i]->nattributes);
+
+ output = (bytea *) palloc0(len);
+ SET_VARSIZE(output, len);
+
+ tmp = VARDATA(output);
+
+ /* Store the base struct values (magic, type, ndeps) */
+ memcpy(tmp, &dependencies->magic, sizeof(uint32));
+ tmp += sizeof(uint32);
+ memcpy(tmp, &dependencies->type, sizeof(uint32));
+ tmp += sizeof(uint32);
+ memcpy(tmp, &dependencies->ndeps, sizeof(uint32));
+ tmp += sizeof(uint32);
+
+ /* store number of attributes and attribute numbers for each dependency */
+ for (i = 0; i < dependencies->ndeps; i++)
+ {
+ MVDependency *d = dependencies->deps[i];
+
+ memcpy(tmp, d, SizeOfDependency);
+ tmp += SizeOfDependency;
+
+ memcpy(tmp, d->attributes, sizeof(AttrNumber) * d->nattributes);
+ tmp += sizeof(AttrNumber) * d->nattributes;
+
+ Assert(tmp <= ((char *) output + len));
+ }
+
+ return output;
+}
+
+/*
+ * Reads serialized dependencies into MVDependencies structure.
+ */
+MVDependencies *
+statext_dependencies_deserialize(bytea *data)
+{
+ int i;
+ Size min_expected_size;
+ MVDependencies *dependencies;
+ char *tmp;
+
+ if (data == NULL)
+ return NULL;
+
+ if (VARSIZE_ANY_EXHDR(data) < SizeOfDependencies)
+ elog(ERROR, "invalid MVDependencies size %ld (expected at least %ld)",
+ VARSIZE_ANY_EXHDR(data), SizeOfDependencies);
+
+ /* read the MVDependencies header */
+ dependencies = (MVDependencies *) palloc0(sizeof(MVDependencies));
+
+ /* initialize pointer to the data part (skip the varlena header) */
+ tmp = VARDATA_ANY(data);
+
+ /* read the header fields and perform basic sanity checks */
+ memcpy(&dependencies->magic, tmp, sizeof(uint32));
+ tmp += sizeof(uint32);
+ memcpy(&dependencies->type, tmp, sizeof(uint32));
+ tmp += sizeof(uint32);
+ memcpy(&dependencies->ndeps, tmp, sizeof(uint32));
+ tmp += sizeof(uint32);
+
+ if (dependencies->magic != STATS_DEPS_MAGIC)
+ elog(ERROR, "invalid dependency magic %d (expected %d)",
+ dependencies->magic, STATS_DEPS_MAGIC);
+
+ if (dependencies->type != STATS_DEPS_TYPE_BASIC)
+ elog(ERROR, "invalid dependency type %d (expected %d)",
+ dependencies->type, STATS_DEPS_TYPE_BASIC);
+
+ if (dependencies->ndeps == 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_DATA_CORRUPTED),
+ errmsg("invalid zero-length item array in MVDependencies")));
+
+ /* what minimum bytea size do we expect for those parameters */
+ min_expected_size = SizeOfDependencies +
+ dependencies->ndeps * (SizeOfDependency +
+ sizeof(AttrNumber) * 2);
+
+ if (VARSIZE_ANY_EXHDR(data) < min_expected_size)
+ elog(ERROR, "invalid dependencies size %ld (expected at least %ld)",
+ VARSIZE_ANY_EXHDR(data), min_expected_size);
+
+ /* allocate space for the MCV items */
+ dependencies = repalloc(dependencies, offsetof(MVDependencies, deps)
+ + (dependencies->ndeps * sizeof(MVDependency *)));
+
+ for (i = 0; i < dependencies->ndeps; i++)
+ {
+ double degree;
+ AttrNumber k;
+ MVDependency *d;
+
+ /* degree of validity */
+ memcpy(°ree, tmp, sizeof(double));
+ tmp += sizeof(double);
+
+ /* number of attributes */
+ memcpy(&k, tmp, sizeof(AttrNumber));
+ tmp += sizeof(AttrNumber);
+
+ /* is the number of attributes valid? */
+ Assert((k >= 2) && (k <= STATS_MAX_DIMENSIONS));
+
+ /* now that we know the number of attributes, allocate the dependency */
+ d = (MVDependency *) palloc0(offsetof(MVDependency, attributes)
+ + (k * sizeof(AttrNumber)));
+
+ d->degree = degree;
+ d->nattributes = k;
+
+ /* copy attribute numbers */
+ memcpy(d->attributes, tmp, sizeof(AttrNumber) * d->nattributes);
+ tmp += sizeof(AttrNumber) * d->nattributes;
+
+ dependencies->deps[i] = d;
+
+ /* still within the bytea */
+ Assert(tmp <= ((char *) data + VARSIZE_ANY(data)));
+ }
+
+ /* we should have consumed the whole bytea exactly */
+ Assert(tmp == ((char *) data + VARSIZE_ANY(data)));
+
+ return dependencies;
+}
+
+/*
+ * dependency_is_fully_matched
+ * checks that a functional dependency is fully matched given clauses on
+ * attributes (assuming the clauses are suitable equality clauses)
+ */
+static bool
+dependency_is_fully_matched(MVDependency * dependency, Bitmapset *attnums)
+{
+ int j;
+
+ /*
+ * Check that the dependency actually is fully covered by clauses. We have
+ * to translate all attribute numbers, as those are referenced
+ */
+ for (j = 0; j < dependency->nattributes; j++)
+ {
+ int attnum = dependency->attributes[j];
+
+ if (!bms_is_member(attnum, attnums))
+ return false;
+ }
+
+ return true;
+}
+
+/*
+ * dependency_implies_attribute
+ * check that the attnum matches is implied by the functional dependency
+ */
+static bool
+dependency_implies_attribute(MVDependency * dependency, AttrNumber attnum)
+{
+ if (attnum == dependency->attributes[dependency->nattributes - 1])
+ return true;
+
+ return false;
+}
+
+/*
+ * staext_dependencies_load
+ * Load the functional dependencies for the indicated pg_statistic_ext tuple
+ */
+MVDependencies *
+staext_dependencies_load(Oid mvoid)
+{
+ bool isnull;
+ Datum deps;
+
+ /*
+ * Prepare to scan pg_statistic_ext for entries having indrelid = this
+ * rel.
+ */
+ HeapTuple htup = SearchSysCache1(STATEXTOID, ObjectIdGetDatum(mvoid));
+
+ if (!HeapTupleIsValid(htup))
+ elog(ERROR, "cache lookup failed for extended statistics %u", mvoid);
+
+ deps = SysCacheGetAttr(STATEXTOID, htup,
+ Anum_pg_statistic_ext_stadependencies, &isnull);
+
+ Assert(!isnull);
+
+ ReleaseSysCache(htup);
+
+ return statext_dependencies_deserialize(DatumGetByteaP(deps));
+}
+
+/*
+ * pg_dependencies_in - input routine for type pg_dependencies.
+ *
+ * pg_dependencies is real enough to be a table column, but it has no operations
+ * of its own, and disallows input too
+ */
+Datum
+pg_dependencies_in(PG_FUNCTION_ARGS)
+{
+ /*
+ * pg_node_list stores the data in binary form and parsing text input is
+ * not needed, so disallow this.
+ */
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("cannot accept a value of type %s", "pg_dependencies")));
+
+ PG_RETURN_VOID(); /* keep compiler quiet */
+}
+
+/*
+ * pg_dependencies - output routine for type pg_dependencies.
+ */
+Datum
+pg_dependencies_out(PG_FUNCTION_ARGS)
+{
+ int i,
+ j;
+ StringInfoData str;
+
+ bytea *data = PG_GETARG_BYTEA_PP(0);
+
+ MVDependencies *dependencies = statext_dependencies_deserialize(data);
+
+ initStringInfo(&str);
+ appendStringInfoChar(&str, '[');
+
+ for (i = 0; i < dependencies->ndeps; i++)
+ {
+ MVDependency *dependency = dependencies->deps[i];
+
+ if (i > 0)
+ appendStringInfoString(&str, ", ");
+
+ appendStringInfoChar(&str, '{');
+ for (j = 0; j < dependency->nattributes; j++)
+ {
+ if (j == dependency->nattributes - 1)
+ appendStringInfoString(&str, " => ");
+ else if (j > 0)
+ appendStringInfoString(&str, ", ");
+
+ appendStringInfo(&str, "%d", dependency->attributes[j]);
+ }
+ appendStringInfo(&str, " : %f", dependency->degree);
+ appendStringInfoChar(&str, '}');
+ }
+
+ appendStringInfoChar(&str, ']');
+
+ PG_RETURN_CSTRING(str.data);
+}
+
+/*
+ * pg_dependencies_recv - binary input routine for type pg_dependencies.
+ */
+Datum
+pg_dependencies_recv(PG_FUNCTION_ARGS)
+{
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("cannot accept a value of type %s", "pg_dependencies")));
+
+ PG_RETURN_VOID(); /* keep compiler quiet */
+}
+
+/*
+ * pg_dependencies_send - binary output routine for type pg_dependencies.
+ *
+ * Functional dependencies are serialized in a bytea value (although the type
+ * is named differently), so let's just send that.
+ */
+Datum
+pg_dependencies_send(PG_FUNCTION_ARGS)
+{
+ return byteasend(fcinfo);
+}
+
+/*
+ * dependency_is_compatible_clause
+ * Determines if the clause is compatible with functional dependencies
+ *
+ * Only OpExprs with two arguments using an equality operator are supported.
+ * When returning True attnum is set to the attribute number of the Var within
+ * the supported clause.
+ *
+ * Currently we only support Var = Const, or Const = Var. It may be possible
+ * to expand on this later.
+ */
+static bool
+dependency_is_compatible_clause(Node *clause, Index relid, AttrNumber *attnum)
+{
+ RestrictInfo *rinfo = (RestrictInfo *) clause;
+
+ if (!IsA(rinfo, RestrictInfo))
+ return false;
+
+ /* Pseudoconstants are not really interesting here. */
+ if (rinfo->pseudoconstant)
+ return false;
+
+ /* clauses referencing multiple varnos are incompatible */
+ if (bms_membership(rinfo->clause_relids) != BMS_SINGLETON)
+ return false;
+
+ if (is_opclause(rinfo->clause))
+ {
+ OpExpr *expr = (OpExpr *) rinfo->clause;
+ Var *var;
+ bool varonleft = true;
+ bool ok;
+
+ /* Only expressions with two arguments are considered compatible. */
+ if (list_length(expr->args) != 2)
+ return false;
+
+ /* see if it actually has the right */
+ ok = (NumRelids((Node *) expr) == 1) &&
+ (is_pseudo_constant_clause(lsecond(expr->args)) ||
+ (varonleft = false,
+ is_pseudo_constant_clause(linitial(expr->args))));
+
+ /* unsupported structure (two variables or so) */
+ if (!ok)
+ return false;
+
+ /*
+ * If it's not "=" operator, just ignore the clause, as it's not
+ * compatible with functional dependencies.
+ *
+ * This uses the function for estimating selectivity, not the operator
+ * directly (a bit awkward, but well ...).
+ */
+ if (get_oprrest(expr->opno) != F_EQSEL)
+ return false;
+
+ var = (varonleft) ? linitial(expr->args) : lsecond(expr->args);
+
+ /* We only support plain Vars for now */
+ if (!IsA(var, Var))
+ return false;
+
+ /* Ensure var is from the correct relation */
+ if (var->varno != relid)
+ return false;
+
+ /* we also better ensure the Var is from the current level */
+ if (var->varlevelsup > 0)
+ return false;
+
+ /* Also skip system attributes (we don't allow stats on those). */
+ if (!AttrNumberIsForUserDefinedAttr(var->varattno))
+ return false;
+
+ *attnum = var->varattno;
+ return true;
+ }
+
+ return false;
+}
+
+/*
+ * find_strongest_dependency
+ * find the strongest dependency on the attributes
+ *
+ * When applying functional dependencies, we start with the strongest
+ * dependencies. That is, we select the dependency that:
+ *
+ * (a) has all attributes covered by equality clauses
+ *
+ * (b) has the most attributes
+ *
+ * (c) has the highest degree of validity
+ *
+ * This guarantees that we eliminate the most redundant conditions first
+ * (see the comment in dependencies_clauselist_selectivity).
+ */
+static MVDependency *
+find_strongest_dependency(StatisticExtInfo * stats, MVDependencies * dependencies,
+ Bitmapset *attnums)
+{
+ int i;
+ MVDependency *strongest = NULL;
+
+ /* number of attnums in clauses */
+ int nattnums = bms_num_members(attnums);
+
+ /*
+ * Iterate over the MVDependency items and find the strongest one from the
+ * fully-matched dependencies. We do the cheap checks first, before
+ * matching it against the attnums.
+ */
+ for (i = 0; i < dependencies->ndeps; i++)
+ {
+ MVDependency *dependency = dependencies->deps[i];
+
+ /*
+ * Skip dependencies referencing more attributes than available
+ * clauses, as those can't be fully matched.
+ */
+ if (dependency->nattributes > nattnums)
+ continue;
+
+ if (strongest)
+ {
+ /* skip dependencies on fewer attributes than the strongest. */
+ if (dependency->nattributes < strongest->nattributes)
+ continue;
+
+ /* also skip weaker dependencies when attribute count matches */
+ if (strongest->nattributes == dependency->nattributes &&
+ strongest->degree > dependency->degree)
+ continue;
+ }
+
+ /*
+ * this dependency is stronger, but we must still check that it's
+ * fully matched to these attnums. We perform this check last as it's
+ * slightly more expensive than the previous checks.
+ */
+ if (dependency_is_fully_matched(dependency, attnums))
+ strongest = dependency; /* save new best match */
+ }
+
+ return strongest;
+}
+
+/*
+ * dependencies_clauselist_selectivity
+ * Attempt to estimate selectivity using functional dependency statistics
+ *
+ * Given equality clauses on attributes (a,b) we find the strongest dependency
+ * between them, i.e. either (a=>b) or (b=>a). Assuming (a=>b) is the selected
+ * dependency, we then combine the per-clause selectivities using the formula
+ *
+ * P(a,b) = P(a) * [f + (1-f)*P(b)]
+ *
+ * where 'f' is the degree of the dependency.
+ *
+ * With clauses on more than two attributes, the dependencies are applied
+ * recursively, starting with the widest/strongest dependencies. For example
+ * P(a,b,c) is first split like this:
+ *
+ * P(a,b,c) = P(a,b) * [f + (1-f)*P(c)]
+ *
+ * assuming (a,b=>c) is the strongest dependency.
+ */
+Selectivity
+dependencies_clauselist_selectivity(PlannerInfo *root,
+ List *clauses,
+ int varRelid,
+ JoinType jointype,
+ SpecialJoinInfo *sjinfo,
+ RelOptInfo *rel,
+ Bitmapset **estimatedclauses)
+{
+ Selectivity s1 = 1.0;
+ ListCell *l;
+ Bitmapset *clauses_attnums = NULL;
+ StatisticExtInfo *stat;
+ MVDependencies *dependencies;
+ AttrNumber *list_attnums;
+ int listidx;
+
+
+ /* check if there's any stats that might be useful for us. */
+ if (!has_stats_of_kind(rel->statlist, STATS_EXT_DEPENDENCIES))
+ return 1.0;
+
+ list_attnums = (AttrNumber *) palloc(sizeof(AttrNumber) *
+ list_length(clauses));
+
+ /*
+ * Pre-process the clauses list to extract the attnums seen in each item.
+ * We need to determine if there's any clauses which will be useful for
+ * dependency selectivity estimations. Along the way we'll record all of
+ * the attnums for each clause in a list which we'll reference later so we
+ * don't need to repeat the same work again. We'll also keep track of all
+ * attnums seen.
+ */
+ listidx = 0;
+ foreach(l, clauses)
+ {
+ Node *clause = (Node *) lfirst(l);
+ AttrNumber attnum;
+
+ if (dependency_is_compatible_clause(clause, rel->relid, &attnum))
+ {
+ list_attnums[listidx] = attnum;
+ clauses_attnums = bms_add_member(clauses_attnums, attnum);
+ }
+ else
+ list_attnums[listidx] = InvalidAttrNumber;
+
+ listidx++;
+ }
+
+ /*
+ * If there's not at least two distinct attnums then reject the whole list
+ * of clauses. We must return 1.0 so the calling function's selectivity is
+ * unaffected.
+ */
+ if (bms_num_members(clauses_attnums) < 2)
+ {
+ pfree(list_attnums);
+ return 1.0;
+ }
+
+ /* find the best suited statistics for these attnums */
+ stat = choose_best_statistics(rel->statlist, clauses_attnums,
+ STATS_EXT_DEPENDENCIES);
+
+ /* if no matching stats could be found then we've nothing to do */
+ if (!stat)
+ {
+ pfree(list_attnums);
+ return 1.0;
+ }
+
+ /* load the dependency items stored in the statistics */
+ dependencies = staext_dependencies_load(stat->statOid);
+
+ /*
+ * Apply the dependencies recursively, starting with the widest/strongest
+ * ones, and proceeding to the smaller/weaker ones. At the end of each
+ * round we factor in the selectivity of clauses on the implied attribute,
+ * and remove the clauses from the list.
+ */
+ while (true)
+ {
+ Selectivity s2 = 1.0;
+ MVDependency *dependency;
+
+ /* the widest/strongest dependency, fully matched by clauses */
+ dependency = find_strongest_dependency(stat, dependencies,
+ clauses_attnums);
+
+ /* if no suitable dependency was found, we're done */
+ if (!dependency)
+ break;
+
+ /*
+ * We found an applicable dependency, so find all the clauses on the
+ * implied attribute - with dependency (a,b => c) we look for clauses
+ * on 'c'.
+ */
+ listidx = -1;
+ foreach(l, clauses)
+ {
+ Node *clause;
+
+ listidx++;
+
+ /*
+ * Skip incompatible clauses, and ones we've already estimated on.
+ */
+ if (list_attnums[listidx] == InvalidAttrNumber ||
+ bms_is_member(listidx, *estimatedclauses))
+ continue;
+
+ /*
+ * Technically we could find more than one clause for a given
+ * attnum. Since these clauses must be equality clauses, we choose
+ * to only take the selectivity estimate from the final clause in
+ * the list for this attnum. If the attnum happens to be compared
+ * to a different Const in another clause then no rows will match
+ * anyway. If it happens to be compared to the same Const, then
+ * ignoring the additional clause is just the thing to do.
+ */
+ if (dependency_implies_attribute(dependency,
+ list_attnums[listidx]))
+ {
+ clause = (Node *) lfirst(l);
+
+ s2 = clause_selectivity(root, clause, varRelid, jointype, sjinfo,
+ NULL); /* don't try to use ext stats */
+
+ /* mark this one as done, so we don't touch it again. */
+ *estimatedclauses = bms_add_member(*estimatedclauses, listidx);
+
+ /*
+ * Mark that we've got and used the dependency on this clause.
+ * We'll want to ignore this when looking for the next
+ * strongest dependency above.
+ */
+ clauses_attnums = bms_del_member(clauses_attnums,
+ list_attnums[listidx]);
+ }
+ }
+
+ /*
+ * Now factor in the selectivity for all the "implied" clauses into
+ * the final one, using this formula:
+ *
+ * P(a,b) = P(a) * (f + (1-f) * P(b))
+ *
+ * where 'f' is the degree of validity of the dependency.
+ */
+ s1 *= (dependency->degree + (1 - dependency->degree) * s2);
+ }
+
+ pfree(dependencies);
+ pfree(list_attnums);
+
+ return s1;
+}
static VacAttrStats **lookup_var_attr_stats(Relation rel, Bitmapset *attrs,
int natts, VacAttrStats **vacattrstats);
static void statext_store(Relation pg_stext, Oid relid,
- MVNDistinct *ndistinct,
+ MVNDistinct *ndistinct, MVDependencies *dependencies,
VacAttrStats **stats);
{
StatExtEntry *stat = (StatExtEntry *) lfirst(lc);
MVNDistinct *ndistinct = NULL;
+ MVDependencies *dependencies = NULL;
VacAttrStats **stats;
ListCell *lc2;
if (t == STATS_EXT_NDISTINCT)
ndistinct = statext_ndistinct_build(totalrows, numrows, rows,
stat->columns, stats);
+ else if (t == STATS_EXT_DEPENDENCIES)
+ dependencies = statext_dependencies_build(numrows, rows,
+ stat->columns, stats);
}
/* store the statistics in the catalog */
- statext_store(pg_stext, stat->statOid, ndistinct, stats);
+ statext_store(pg_stext, stat->statOid, ndistinct, dependencies, stats);
}
heap_close(pg_stext, RowExclusiveLock);
attnum = Anum_pg_statistic_ext_standistinct;
break;
+ case STATS_EXT_DEPENDENCIES:
+ attnum = Anum_pg_statistic_ext_stadependencies;
+ break;
+
default:
elog(ERROR, "unexpected statistics type requested: %d", type);
}
enabled = (char *) ARR_DATA_PTR(arr);
for (i = 0; i < ARR_DIMS(arr)[0]; i++)
{
- Assert(enabled[i] == STATS_EXT_NDISTINCT);
+ Assert((enabled[i] == STATS_EXT_NDISTINCT) ||
+ (enabled[i] == STATS_EXT_DEPENDENCIES));
entry->types = lappend_int(entry->types, (int) enabled[i]);
}
*/
static void
statext_store(Relation pg_stext, Oid statOid,
- MVNDistinct *ndistinct,
+ MVNDistinct *ndistinct, MVDependencies *dependencies,
VacAttrStats **stats)
{
HeapTuple stup,
values[Anum_pg_statistic_ext_standistinct - 1] = PointerGetDatum(data);
}
+ if (dependencies != NULL)
+ {
+ bytea *data = statext_dependencies_serialize(dependencies);
+
+ nulls[Anum_pg_statistic_ext_stadependencies - 1] = (data == NULL);
+ values[Anum_pg_statistic_ext_stadependencies - 1] = PointerGetDatum(data);
+ }
+
/* always replace the value (either by bytea or NULL) */
replaces[Anum_pg_statistic_ext_standistinct - 1] = true;
+ replaces[Anum_pg_statistic_ext_stadependencies - 1] = true;
/* there should already be a pg_statistic_ext tuple */
oldtup = SearchSysCache1(STATEXTOID, ObjectIdGetDatum(statOid));
return 0;
}
+
+/*
+ * has_stats_of_kind
+ * Check that the list contains statistic of a given kind
+ */
+bool
+has_stats_of_kind(List *stats, char requiredkind)
+{
+ ListCell *l;
+
+ foreach(l, stats)
+ {
+ StatisticExtInfo *stat = (StatisticExtInfo *) lfirst(l);
+
+ if (stat->kind == requiredkind)
+ return true;
+ }
+
+ return false;
+}
+
+/*
+ * choose_best_statistics
+ * Look for statistics with the specified 'requiredkind' which have keys
+ * that match at least two attnums.
+ *
+ * The current selection criteria is very simple - we choose the statistics
+ * referencing the most attributes with the least keys.
+ *
+ * XXX if multiple statistics exists of the same size matching the same number
+ * of keys, then the statistics which are chosen depend on the order that they
+ * appear in the stats list. Perhaps this needs to be more definitive.
+ */
+StatisticExtInfo *
+choose_best_statistics(List *stats, Bitmapset *attnums, char requiredkind)
+{
+ ListCell *lc;
+ StatisticExtInfo *best_match = NULL;
+ int best_num_matched = 2; /* goal #1: maximize */
+ int best_match_keys = (STATS_MAX_DIMENSIONS + 1); /* goal #2: minimize */
+
+ foreach(lc, stats)
+ {
+ StatisticExtInfo *info = (StatisticExtInfo *) lfirst(lc);
+ int num_matched;
+ int numkeys;
+ Bitmapset *matched;
+
+ /* skip statistics that are not the correct type */
+ if (info->kind != requiredkind)
+ continue;
+
+ /* determine how many attributes of these stats can be matched to */
+ matched = bms_intersect(attnums, info->keys);
+ num_matched = bms_num_members(matched);
+ bms_free(matched);
+
+ /*
+ * save the actual number of keys in the stats so that we can choose
+ * the narrowest stats with the most matching keys.
+ */
+ numkeys = bms_num_members(info->keys);
+
+ /*
+ * Use these statistics when it increases the number of matched
+ * clauses or when it matches the same number of attributes but these
+ * stats have fewer keys than any previous match.
+ */
+ if (num_matched > best_num_matched ||
+ (num_matched == best_num_matched && numkeys < best_match_keys))
+ {
+ best_match = info;
+ best_num_matched = num_matched;
+ best_match_keys = numkeys;
+ }
+ }
+
+ return best_match;
+}
StringInfoData buf;
int colno;
char *nsp;
+ ArrayType *arr;
+ char *enabled;
+ Datum datum;
+ bool isnull;
+ bool ndistinct_enabled;
+ bool dependencies_enabled;
+ int i;
statexttup = SearchSysCache1(STATEXTOID, ObjectIdGetDatum(statextid));
initStringInfo(&buf);
nsp = get_namespace_name(statextrec->stanamespace);
- appendStringInfo(&buf, "CREATE STATISTICS %s ON (",
+ appendStringInfo(&buf, "CREATE STATISTICS %s",
quote_qualified_identifier(nsp,
NameStr(statextrec->staname)));
+ /*
+ * Lookup the staenabled column so that we know how to handle the WITH
+ * clause.
+ */
+ datum = SysCacheGetAttr(STATEXTOID, statexttup,
+ Anum_pg_statistic_ext_staenabled, &isnull);
+ Assert(!isnull);
+ arr = DatumGetArrayTypeP(datum);
+ if (ARR_NDIM(arr) != 1 ||
+ ARR_HASNULL(arr) ||
+ ARR_ELEMTYPE(arr) != CHAROID)
+ elog(ERROR, "staenabled is not a 1-D char array");
+ enabled = (char *) ARR_DATA_PTR(arr);
+
+ ndistinct_enabled = false;
+ dependencies_enabled = false;
+
+ for (i = 0; i < ARR_DIMS(arr)[0]; i++)
+ {
+ if (enabled[i] == STATS_EXT_NDISTINCT)
+ ndistinct_enabled = true;
+ if (enabled[i] == STATS_EXT_DEPENDENCIES)
+ dependencies_enabled = true;
+ }
+
+ /*
+ * If any option is disabled, then we'll need to append a WITH clause to
+ * show which options are enabled. We omit the WITH clause on purpose
+ * when all options are enabled, so a pg_dump/pg_restore will create all
+ * statistics types on a newer postgres version, if the statistics had all
+ * options enabled on the original version.
+ */
+ if (!ndistinct_enabled || !dependencies_enabled)
+ {
+ appendStringInfoString(&buf, " WITH (");
+ if (ndistinct_enabled)
+ appendStringInfoString(&buf, "ndistinct");
+ else if (dependencies_enabled)
+ appendStringInfoString(&buf, "dependencies");
+
+ appendStringInfoChar(&buf, ')');
+ }
+
+ appendStringInfoString(&buf, " ON (");
+
for (colno = 0; colno < statextrec->stakeys.dim1; colno++)
{
AttrNumber attnum = statextrec->stakeys.values[colno];
case IS_NOT_FALSE:
selec = (double) clause_selectivity(root, arg,
varRelid,
- jointype, sjinfo);
+ jointype,
+ sjinfo,
+ NULL);
break;
case IS_FALSE:
case IS_NOT_TRUE:
selec = 1.0 - (double) clause_selectivity(root, arg,
varRelid,
- jointype, sjinfo);
+ jointype,
+ sjinfo,
+ NULL);
break;
default:
elog(ERROR, "unrecognized booltesttype: %d",
indexSelectivity = clauselist_selectivity(root, selectivityQuals,
index->rel->relid,
JOIN_INNER,
- NULL);
+ NULL,
+ index->rel);
/*
* If caller didn't give us an estimate, estimate the number of index
btreeSelectivity = clauselist_selectivity(root, selectivityQuals,
index->rel->relid,
JOIN_INNER,
- NULL);
+ NULL,
+ index->rel);
numIndexTuples = btreeSelectivity * index->rel->tuples;
/*
*indexSelectivity = clauselist_selectivity(root, selectivityQuals,
index->rel->relid,
JOIN_INNER,
- NULL);
+ NULL,
+ index->rel);
/* fetch estimated page cost for tablespace containing index */
get_tablespace_page_costs(index->reltablespace,
*indexSelectivity =
clauselist_selectivity(root, indexQuals,
path->indexinfo->rel->relid,
- JOIN_INNER, NULL);
+ JOIN_INNER, NULL,
+ path->indexinfo->rel);
*indexCorrelation = 1;
/*
" FROM ((SELECT pg_catalog.unnest(stakeys) AS attnum) s\n"
" JOIN pg_catalog.pg_attribute a ON (starelid = a.attrelid AND\n"
"a.attnum = s.attnum AND not attisdropped))) AS columns,\n"
- " (staenabled::char[] @> '{d}'::char[]) AS ndist_enabled\n"
+ " (staenabled::char[] @> '{d}'::char[]) AS ndist_enabled,\n"
+ " (staenabled::char[] @> '{f}'::char[]) AS deps_enabled\n"
"FROM pg_catalog.pg_statistic_ext stat WHERE starelid = '%s'\n"
"ORDER BY 1;",
oid);
for (i = 0; i < tuples; i++)
{
- int cnt = 0;
+ bool gotone = false;
printfPQExpBuffer(&buf, " ");
if (strcmp(PQgetvalue(result, i, 5), "t") == 0)
{
appendPQExpBufferStr(&buf, "ndistinct");
- cnt++;
+ gotone = true;
+ }
+
+ if (strcmp(PQgetvalue(result, i, 6), "t") == 0)
+ {
+ appendPQExpBuffer(&buf, "%sdependencies", gotone ? ", " : "");
}
appendPQExpBuffer(&buf, ") ON (%s)",
DATA(insert ( 3361 17 0 i b ));
DATA(insert ( 3361 25 0 i i ));
+/* pg_dependencies can be coerced to, but not from, bytea and text */
+DATA(insert ( 3402 17 0 i b ));
+DATA(insert ( 3402 25 0 i i ));
+
/*
* Datetime category
*/
DATA(insert OID = 3358 ( pg_ndistinct_send PGNSP PGUID 12 1 0 0 0 f f f f t f s s 1 0 17 "3361" _null_ _null_ _null_ _null_ _null_ pg_ndistinct_send _null_ _null_ _null_ ));
DESCR("I/O");
+DATA(insert OID = 3404 ( pg_dependencies_in PGNSP PGUID 12 1 0 0 0 f f f f t f i s 1 0 3402 "2275" _null_ _null_ _null_ _null_ _null_ pg_dependencies_in _null_ _null_ _null_ ));
+DESCR("I/O");
+DATA(insert OID = 3405 ( pg_dependencies_out PGNSP PGUID 12 1 0 0 0 f f f f t f i s 1 0 2275 "3402" _null_ _null_ _null_ _null_ _null_ pg_dependencies_out _null_ _null_ _null_ ));
+DESCR("I/O");
+DATA(insert OID = 3406 ( pg_dependencies_recv PGNSP PGUID 12 1 0 0 0 f f f f t f s s 1 0 3402 "2281" _null_ _null_ _null_ _null_ _null_ pg_dependencies_recv _null_ _null_ _null_ ));
+DESCR("I/O");
+DATA(insert OID = 3407 ( pg_dependencies_send PGNSP PGUID 12 1 0 0 0 f f f f t f s s 1 0 17 "3402" _null_ _null_ _null_ _null_ _null_ pg_dependencies_send _null_ _null_ _null_ ));
+DESCR("I/O");
+
DATA(insert OID = 1928 ( pg_stat_get_numscans PGNSP PGUID 12 1 0 0 0 f f f f t f s r 1 0 20 "26" _null_ _null_ _null_ _null_ _null_ pg_stat_get_numscans _null_ _null_ _null_ ));
DESCR("statistics: number of scans done for table/index");
DATA(insert OID = 1929 ( pg_stat_get_tuples_returned PGNSP PGUID 12 1 0 0 0 f f f f t f s r 1 0 20 "26" _null_ _null_ _null_ _null_ _null_ pg_stat_get_tuples_returned _null_ _null_ _null_ ));
char staenabled[1] BKI_FORCE_NOT_NULL; /* statistic types
* requested to build */
pg_ndistinct standistinct; /* ndistinct coefficients (serialized) */
+ pg_dependencies stadependencies; /* dependencies (serialized) */
#endif
} FormData_pg_statistic_ext;
* compiler constants for pg_statistic_ext
* ----------------
*/
-#define Natts_pg_statistic_ext 7
+#define Natts_pg_statistic_ext 8
#define Anum_pg_statistic_ext_starelid 1
#define Anum_pg_statistic_ext_staname 2
#define Anum_pg_statistic_ext_stanamespace 3
#define Anum_pg_statistic_ext_stakeys 5
#define Anum_pg_statistic_ext_staenabled 6
#define Anum_pg_statistic_ext_standistinct 7
+#define Anum_pg_statistic_ext_stadependencies 8
-#define STATS_EXT_NDISTINCT 'd'
+#define STATS_EXT_NDISTINCT 'd'
+#define STATS_EXT_DEPENDENCIES 'f'
#endif /* PG_STATISTIC_EXT_H */
DESCR("multivariate ndistinct coefficients");
#define PGNDISTINCTOID 3361
+DATA(insert OID = 3402 ( pg_dependencies PGNSP PGUID -1 f b S f t \054 0 0 0 pg_dependencies_in pg_dependencies_out pg_dependencies_recv pg_dependencies_send - - - i x f 0 -1 0 100 _null_ _null_ _null_ ));
+DESCR("multivariate dependencies");
+#define PGDEPENDENCIESOID 3402
+
DATA(insert OID = 32 ( pg_ddl_command PGNSP PGUID SIZEOF_POINTER t p P f t \054 0 0 0 pg_ddl_command_in pg_ddl_command_out pg_ddl_command_recv pg_ddl_command_send - - - ALIGNOF_POINTER p f 0 -1 0 0 _null_ _null_ _null_ ));
DESCR("internal type for passing CollectedCommand");
#define PGDDLCOMMANDOID 32
List *clauses,
int varRelid,
JoinType jointype,
- SpecialJoinInfo *sjinfo);
+ SpecialJoinInfo *sjinfo,
+ RelOptInfo *rel);
extern Selectivity clause_selectivity(PlannerInfo *root,
Node *clause,
int varRelid,
JoinType jointype,
- SpecialJoinInfo *sjinfo);
+ SpecialJoinInfo *sjinfo,
+ RelOptInfo *rel);
extern void cost_gather_merge(GatherMergePath *path, PlannerInfo *root,
RelOptInfo *rel, ParamPathInfo *param_info,
Cost input_startup_cost, Cost input_total_cost,
extern bytea *statext_ndistinct_serialize(MVNDistinct *ndistinct);
extern MVNDistinct *statext_ndistinct_deserialize(bytea *data);
+extern MVDependencies *statext_dependencies_build(int numrows, HeapTuple *rows,
+ Bitmapset *attrs, VacAttrStats **stats);
+extern bytea *statext_dependencies_serialize(MVDependencies *dependencies);
+extern MVDependencies *statext_dependencies_deserialize(bytea *data);
+
extern MultiSortSupport multi_sort_init(int ndims);
extern void multi_sort_add_dimension(MultiSortSupport mss, int sortdim,
Oid oper);
#define STATISTICS_H
#include "commands/vacuum.h"
+#include "nodes/relation.h"
#define STATS_MAX_DIMENSIONS 8 /* max number of attributes */
#define SizeOfMVNDistinct (offsetof(MVNDistinct, nitems) + sizeof(uint32))
+/* size of the struct excluding the items array */
+#define SizeOfMVNDistinct (offsetof(MVNDistinct, nitems) + sizeof(uint32))
+
+#define STATS_DEPS_MAGIC 0xB4549A2C /* marks serialized bytea */
+#define STATS_DEPS_TYPE_BASIC 1 /* basic dependencies type */
+
+/*
+ * Functional dependencies, tracking column-level relationships (values
+ * in one column determine values in another one).
+ */
+typedef struct MVDependency
+{
+ double degree; /* degree of validity (0-1) */
+ AttrNumber nattributes; /* number of attributes */
+ AttrNumber attributes[FLEXIBLE_ARRAY_MEMBER]; /* attribute numbers */
+} MVDependency;
+
+/* size of the struct excluding the deps array */
+#define SizeOfDependency \
+ (offsetof(MVDependency, nattributes) + sizeof(AttrNumber))
+
+typedef struct MVDependencies
+{
+ uint32 magic; /* magic constant marker */
+ uint32 type; /* type of MV Dependencies (BASIC) */
+ uint32 ndeps; /* number of dependencies */
+ MVDependency *deps[FLEXIBLE_ARRAY_MEMBER]; /* dependencies */
+} MVDependencies;
+
+/* size of the struct excluding the deps array */
+#define SizeOfDependencies (offsetof(MVDependencies, ndeps) + sizeof(uint32))
+
extern MVNDistinct *statext_ndistinct_load(Oid mvoid);
+extern MVDependencies *staext_dependencies_load(Oid mvoid);
extern void BuildRelationExtStatistics(Relation onerel, double totalrows,
int numrows, HeapTuple *rows,
int natts, VacAttrStats **vacattrstats);
extern bool statext_is_kind_built(HeapTuple htup, char kind);
+extern Selectivity dependencies_clauselist_selectivity(PlannerInfo *root,
+ List *clauses,
+ int varRelid,
+ JoinType jointype,
+ SpecialJoinInfo *sjinfo,
+ RelOptInfo *rel,
+ Bitmapset **estimatedclauses);
+extern bool has_stats_of_kind(List *stats, char requiredkind);
+extern StatisticExtInfo *choose_best_statistics(List *stats,
+ Bitmapset *attnums, char requiredkind);
#endif /* STATISTICS_H */
character varying | character | 0 | i
pg_node_tree | text | 0 | i
pg_ndistinct | bytea | 0 | i
+ pg_dependencies | bytea | 0 | i
cidr | inet | 0 | i
xml | text | 0 | a
xml | character varying | 0 | a
xml | character | 0 | a
-(8 rows)
+(9 rows)
-- **************** pg_conversion ****************
-- Look for illegal values in pg_conversion fields.
c.relname AS tablename,
s.staname,
s.stakeys AS attnums,
- length((s.standistinct)::text) AS ndistbytes
+ length((s.standistinct)::bytea) AS ndistbytes,
+ length((s.stadependencies)::bytea) AS depsbytes
FROM ((pg_statistic_ext s
JOIN pg_class c ON ((c.oid = s.starelid)))
LEFT JOIN pg_namespace n ON ((n.oid = c.relnamespace)));
b | integer | | |
c | integer | | |
Statistics:
- "public.ab1_b_c_stats" WITH (ndistinct) ON (b, c)
+ "public.ab1_b_c_stats" WITH (ndistinct, dependencies) ON (b, c)
DROP TABLE ab1;
-- Ensure things work sanely with SET STATISTICS 0
FROM pg_statistic_ext WHERE starelid = 'ndistinct'::regclass;
staenabled | standistinct
------------+------------------------------------------------------------------------------------------------
- {d} | [{(b 3 4), 301.000000}, {(b 3 6), 301.000000}, {(b 4 6), 301.000000}, {(b 3 4 6), 301.000000}]
+ {d,f} | [{(b 3 4), 301.000000}, {(b 3 6), 301.000000}, {(b 4 6), 301.000000}, {(b 3 4 6), 301.000000}]
(1 row)
-- Hash Aggregate, thanks to estimates improved by the statistic
FROM pg_statistic_ext WHERE starelid = 'ndistinct'::regclass;
staenabled | standistinct
------------+----------------------------------------------------------------------------------------------------
- {d} | [{(b 3 4), 2550.000000}, {(b 3 6), 800.000000}, {(b 4 6), 1632.000000}, {(b 3 4 6), 10000.000000}]
+ {d,f} | [{(b 3 4), 2550.000000}, {(b 3 6), 800.000000}, {(b 4 6), 1632.000000}, {(b 3 4 6), 10000.000000}]
(1 row)
-- plans using Group Aggregate, thanks to using correct esimates
(3 rows)
DROP TABLE ndistinct;
+-- functional dependencies tests
+CREATE TABLE functional_dependencies (
+ filler1 TEXT,
+ filler2 NUMERIC,
+ a INT,
+ b TEXT,
+ filler3 DATE,
+ c INT,
+ d TEXT
+);
+SET random_page_cost = 1.2;
+CREATE INDEX fdeps_ab_idx ON functional_dependencies (a, b);
+CREATE INDEX fdeps_abc_idx ON functional_dependencies (a, b, c);
+-- random data (no functional dependencies)
+INSERT INTO functional_dependencies (a, b, c, filler1)
+ SELECT mod(i, 23), mod(i, 29), mod(i, 31), i FROM generate_series(1,5000) s(i);
+ANALYZE functional_dependencies;
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1';
+ QUERY PLAN
+---------------------------------------------------
+ Bitmap Heap Scan on functional_dependencies
+ Recheck Cond: ((a = 1) AND (b = '1'::text))
+ -> Bitmap Index Scan on fdeps_abc_idx
+ Index Cond: ((a = 1) AND (b = '1'::text))
+(4 rows)
+
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1' AND c = 1;
+ QUERY PLAN
+-----------------------------------------------------------
+ Index Scan using fdeps_abc_idx on functional_dependencies
+ Index Cond: ((a = 1) AND (b = '1'::text) AND (c = 1))
+(2 rows)
+
+-- create statistics
+CREATE STATISTICS func_deps_stat WITH (dependencies) ON (a, b, c) FROM functional_dependencies;
+ANALYZE functional_dependencies;
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1';
+ QUERY PLAN
+---------------------------------------------------
+ Bitmap Heap Scan on functional_dependencies
+ Recheck Cond: ((a = 1) AND (b = '1'::text))
+ -> Bitmap Index Scan on fdeps_abc_idx
+ Index Cond: ((a = 1) AND (b = '1'::text))
+(4 rows)
+
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1' AND c = 1;
+ QUERY PLAN
+-----------------------------------------------------------
+ Index Scan using fdeps_abc_idx on functional_dependencies
+ Index Cond: ((a = 1) AND (b = '1'::text) AND (c = 1))
+(2 rows)
+
+-- a => b, a => c, b => c
+TRUNCATE functional_dependencies;
+DROP STATISTICS func_deps_stat;
+INSERT INTO functional_dependencies (a, b, c, filler1)
+ SELECT mod(i,100), mod(i,50), mod(i,25), i FROM generate_series(1,5000) s(i);
+ANALYZE functional_dependencies;
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1';
+ QUERY PLAN
+-----------------------------------------------------------
+ Index Scan using fdeps_abc_idx on functional_dependencies
+ Index Cond: ((a = 1) AND (b = '1'::text))
+(2 rows)
+
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1' AND c = 1;
+ QUERY PLAN
+-----------------------------------------------------------
+ Index Scan using fdeps_abc_idx on functional_dependencies
+ Index Cond: ((a = 1) AND (b = '1'::text) AND (c = 1))
+(2 rows)
+
+-- create statistics
+CREATE STATISTICS func_deps_stat WITH (dependencies) ON (a, b, c) FROM functional_dependencies;
+ANALYZE functional_dependencies;
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1';
+ QUERY PLAN
+---------------------------------------------------
+ Bitmap Heap Scan on functional_dependencies
+ Recheck Cond: ((a = 1) AND (b = '1'::text))
+ -> Bitmap Index Scan on fdeps_abc_idx
+ Index Cond: ((a = 1) AND (b = '1'::text))
+(4 rows)
+
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1' AND c = 1;
+ QUERY PLAN
+---------------------------------------------------
+ Bitmap Heap Scan on functional_dependencies
+ Recheck Cond: ((a = 1) AND (b = '1'::text))
+ Filter: (c = 1)
+ -> Bitmap Index Scan on fdeps_ab_idx
+ Index Cond: ((a = 1) AND (b = '1'::text))
+(5 rows)
+
+RESET random_page_cost;
+DROP TABLE functional_dependencies;
(SELECT 1 FROM pg_type as p2
WHERE p2.typname = ('_' || p1.typname)::name AND
p2.typelem = p1.oid and p1.typarray = p2.oid);
- oid | typname
-------+--------------
+ oid | typname
+------+-----------------
194 | pg_node_tree
3361 | pg_ndistinct
+ 3402 | pg_dependencies
210 | smgr
-(3 rows)
+(4 rows)
-- Make sure typarray points to a varlena array type of our own base
SELECT p1.oid, p1.typname as basetype, p2.typname as arraytype,
SELECT COUNT(*) FROM ndistinct GROUP BY a, d;
DROP TABLE ndistinct;
+
+-- functional dependencies tests
+CREATE TABLE functional_dependencies (
+ filler1 TEXT,
+ filler2 NUMERIC,
+ a INT,
+ b TEXT,
+ filler3 DATE,
+ c INT,
+ d TEXT
+);
+
+SET random_page_cost = 1.2;
+
+CREATE INDEX fdeps_ab_idx ON functional_dependencies (a, b);
+CREATE INDEX fdeps_abc_idx ON functional_dependencies (a, b, c);
+
+-- random data (no functional dependencies)
+INSERT INTO functional_dependencies (a, b, c, filler1)
+ SELECT mod(i, 23), mod(i, 29), mod(i, 31), i FROM generate_series(1,5000) s(i);
+
+ANALYZE functional_dependencies;
+
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1';
+
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1' AND c = 1;
+
+-- create statistics
+CREATE STATISTICS func_deps_stat WITH (dependencies) ON (a, b, c) FROM functional_dependencies;
+
+ANALYZE functional_dependencies;
+
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1';
+
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1' AND c = 1;
+
+-- a => b, a => c, b => c
+TRUNCATE functional_dependencies;
+DROP STATISTICS func_deps_stat;
+
+INSERT INTO functional_dependencies (a, b, c, filler1)
+ SELECT mod(i,100), mod(i,50), mod(i,25), i FROM generate_series(1,5000) s(i);
+
+ANALYZE functional_dependencies;
+
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1';
+
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1' AND c = 1;
+
+-- create statistics
+CREATE STATISTICS func_deps_stat WITH (dependencies) ON (a, b, c) FROM functional_dependencies;
+
+ANALYZE functional_dependencies;
+
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1';
+
+EXPLAIN (COSTS OFF)
+ SELECT * FROM functional_dependencies WHERE a = 1 AND b = '1' AND c = 1;
+
+RESET random_page_cost;
+DROP TABLE functional_dependencies;