From c2f84b0616376c579fef419d54aa6e6cb3b34be7 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@enterprisedb.com>
Date: Sat, 25 Sep 2021 21:27:10 +0200
Subject: [PATCH v3 3/6] Add stxdinherit; build inherited extended stats on
 inheritence parents

pg_statistic has an inherited flag which is part of the unique index, but
pg_statistic_ext has never had that.  In back branches, pg_statistic cannot
store both inherited and non-inherited stats.  So it stores non-inherited stats
(FROM ONLY) for inheritence parents and inherited stats for partitioned tables.

This patch for v15 allows storing both inherited and non-inherited stats for
non-empty inheritence parents, and avoids the above, confusing definition.
---
 doc/src/sgml/catalogs.sgml                  |  23 +++
 src/backend/catalog/system_views.sql        |   1 +
 src/backend/commands/analyze.c              |  15 +-
 src/backend/commands/statscmds.c            |  20 ++-
 src/backend/optimizer/util/plancat.c        | 186 +++++++++++---------
 src/backend/statistics/dependencies.c       |  12 +-
 src/backend/statistics/extended_stats.c     |  71 ++++----
 src/backend/statistics/mcv.c                |   9 +-
 src/backend/statistics/mvdistinct.c         |   5 +-
 src/backend/utils/adt/selfuncs.c            |   2 +-
 src/backend/utils/cache/syscache.c          |   6 +-
 src/include/catalog/pg_statistic_ext_data.h |   4 +-
 src/include/nodes/pathnodes.h               |   1 +
 src/include/statistics/statistics.h         |   9 +-
 src/test/regress/expected/rules.out         |   1 +
 15 files changed, 215 insertions(+), 150 deletions(-)

diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c1d11be73f..720e83e523 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -7509,6 +7509,19 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
    created with <link linkend="sql-createstatistics"><command>CREATE STATISTICS</command></link>.
   </para>
 
+  <para>
+   Normally there is one entry, with <structfield>stxdinherit</structfield> =
+   <literal>false</literal>, for each statistics object that has been analyzed.
+   If the table has inheritance children, a second entry with
+   <structfield>stxdinherit</structfield> = <literal>true</literal> is also created.
+   This row represents the statistics object over the inheritance tree, i.e.,
+   statistics for the data you'd see with
+   <literal>SELECT * FROM <replaceable>table</replaceable>*</literal>,
+   whereas the <structfield>stxdinherit</structfield> = <literal>false</literal> row
+   represents the results of
+   <literal>SELECT * FROM ONLY <replaceable>table</replaceable></literal>.
+  </para>
+
   <para>
    Like <link linkend="catalog-pg-statistic"><structname>pg_statistic</structname></link>,
    <structname>pg_statistic_ext_data</structname> should not be
@@ -7548,6 +7561,16 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
       </para></entry>
      </row>
 
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>stxdinherit</structfield> <type>bool</type>
+      </para>
+      <para>
+       If true, the stats include inheritance child columns, not just the
+       values in the specified relation
+      </para></entry>
+     </row>
+
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
        <structfield>stxdndistinct</structfield> <type>pg_ndistinct</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index eb560955cd..36aa3dd3ab 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -266,6 +266,7 @@ CREATE VIEW pg_stats_ext WITH (security_barrier) AS
            ) AS attnames,
            pg_get_statisticsobjdef_expressions(s.oid) as exprs,
            s.stxkind AS kinds,
+           sd.stxdinherit AS inherited,
            sd.stxdndistinct AS n_distinct,
            sd.stxddependencies AS dependencies,
            m.most_common_vals,
diff --git a/src/backend/commands/analyze.c b/src/backend/commands/analyze.c
index acc994e1e8..88c6976ed8 100644
--- a/src/backend/commands/analyze.c
+++ b/src/backend/commands/analyze.c
@@ -548,7 +548,6 @@ do_analyze_rel(Relation onerel, VacuumParams *params,
 	{
 		MemoryContext col_context,
 					old_context;
-		bool		build_ext_stats;
 
 		pgstat_progress_update_param(PROGRESS_ANALYZE_PHASE,
 									 PROGRESS_ANALYZE_PHASE_COMPUTE_STATS);
@@ -612,17 +611,9 @@ do_analyze_rel(Relation onerel, VacuumParams *params,
 							thisdata->attr_cnt, thisdata->vacattrstats);
 		}
 
-		build_ext_stats = (onerel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE) ? inh : (!inh);
-
-		/*
-		 * Build extended statistics (if there are any).
-		 *
-		 * For now we only build extended statistics on individual relations,
-		 * not for relations representing inheritance trees.
-		 */
-		if (build_ext_stats)
-			BuildRelationExtStatistics(onerel, totalrows, numrows, rows,
-									   attr_cnt, vacattrstats);
+		/* Build extended statistics (if there are any). */
+		BuildRelationExtStatistics(onerel, inh, totalrows, numrows, rows,
+								   attr_cnt, vacattrstats);
 	}
 
 	pgstat_progress_update_param(PROGRESS_ANALYZE_PHASE,
diff --git a/src/backend/commands/statscmds.c b/src/backend/commands/statscmds.c
index 8f1550ec80..4395d878c7 100644
--- a/src/backend/commands/statscmds.c
+++ b/src/backend/commands/statscmds.c
@@ -524,6 +524,9 @@ CreateStatistics(CreateStatsStmt *stmt)
 
 	datavalues[Anum_pg_statistic_ext_data_stxoid - 1] = ObjectIdGetDatum(statoid);
 
+	/* create only the "stxdinherit=false", because that always exists */
+	datavalues[Anum_pg_statistic_ext_data_stxdinherit - 1] = ObjectIdGetDatum(false);
+
 	/* no statistics built yet */
 	datanulls[Anum_pg_statistic_ext_data_stxdndistinct - 1] = true;
 	datanulls[Anum_pg_statistic_ext_data_stxddependencies - 1] = true;
@@ -726,6 +729,7 @@ RemoveStatisticsById(Oid statsOid)
 	HeapTuple	tup;
 	Form_pg_statistic_ext statext;
 	Oid			relid;
+	int			inh;
 
 	/*
 	 * First delete the pg_statistic_ext_data tuple holding the actual
@@ -733,14 +737,20 @@ RemoveStatisticsById(Oid statsOid)
 	 */
 	relation = table_open(StatisticExtDataRelationId, RowExclusiveLock);
 
-	tup = SearchSysCache1(STATEXTDATASTXOID, ObjectIdGetDatum(statsOid));
+	/* hack to delete both stxdinherit = true/false */
+	for (inh = 0; inh <= 1; inh++)
+	{
+		tup = SearchSysCache2(STATEXTDATASTXOID, ObjectIdGetDatum(statsOid),
+							  BoolGetDatum(inh));
 
-	if (!HeapTupleIsValid(tup)) /* should not happen */
-		elog(ERROR, "cache lookup failed for statistics data %u", statsOid);
+		if (!HeapTupleIsValid(tup)) /* should not happen */
+			// elog(ERROR, "cache lookup failed for statistics data %u", statsOid);
+			continue;
 
-	CatalogTupleDelete(relation, &tup->t_self);
+		CatalogTupleDelete(relation, &tup->t_self);
 
-	ReleaseSysCache(tup);
+		ReleaseSysCache(tup);
+	}
 
 	table_close(relation, RowExclusiveLock);
 
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index c5194fdbbf..154d48a330 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -30,6 +30,7 @@
 #include "catalog/pg_am.h"
 #include "catalog/pg_proc.h"
 #include "catalog/pg_statistic_ext.h"
+#include "catalog/pg_statistic_ext_data.h"
 #include "foreign/fdwapi.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
@@ -1311,127 +1312,144 @@ get_relation_statistics(RelOptInfo *rel, Relation relation)
 	{
 		Oid			statOid = lfirst_oid(l);
 		Form_pg_statistic_ext staForm;
+		Form_pg_statistic_ext_data dataForm;
 		HeapTuple	htup;
 		HeapTuple	dtup;
 		Bitmapset  *keys = NULL;
 		List	   *exprs = NIL;
 		int			i;
+		int			inh;
 
 		htup = SearchSysCache1(STATEXTOID, ObjectIdGetDatum(statOid));
 		if (!HeapTupleIsValid(htup))
 			elog(ERROR, "cache lookup failed for statistics object %u", statOid);
 		staForm = (Form_pg_statistic_ext) GETSTRUCT(htup);
 
-		dtup = SearchSysCache1(STATEXTDATASTXOID, ObjectIdGetDatum(statOid));
-		if (!HeapTupleIsValid(dtup))
-			elog(ERROR, "cache lookup failed for statistics object %u", statOid);
-
-		/*
-		 * First, build the array of columns covered.  This is ultimately
-		 * wasted if no stats within the object have actually been built, but
-		 * it doesn't seem worth troubling over that case.
-		 */
-		for (i = 0; i < staForm->stxkeys.dim1; i++)
-			keys = bms_add_member(keys, staForm->stxkeys.values[i]);
-
 		/*
-		 * Preprocess expressions (if any). We read the expressions, run them
-		 * through eval_const_expressions, and fix the varnos.
+		 * Hack to load stats with stxdinherit true/false - there should be
+		 * a better way to do this, I guess.
 		 */
+		for (inh = 0; inh <= 1; inh++)
 		{
-			bool		isnull;
-			Datum		datum;
+			dtup = SearchSysCache2(STATEXTDATASTXOID,
+								   ObjectIdGetDatum(statOid), BoolGetDatum((bool) inh));
+			if (!HeapTupleIsValid(dtup))
+				continue;
 
-			/* decode expression (if any) */
-			datum = SysCacheGetAttr(STATEXTOID, htup,
-									Anum_pg_statistic_ext_stxexprs, &isnull);
+			dataForm = (Form_pg_statistic_ext_data) GETSTRUCT(dtup);
 
-			if (!isnull)
+			/*
+			 * First, build the array of columns covered.  This is ultimately
+			 * wasted if no stats within the object have actually been built, but
+			 * it doesn't seem worth troubling over that case.
+			 */
+			for (i = 0; i < staForm->stxkeys.dim1; i++)
+				keys = bms_add_member(keys, staForm->stxkeys.values[i]);
+
+			/*
+			 * Preprocess expressions (if any). We read the expressions, run them
+			 * through eval_const_expressions, and fix the varnos.
+			 */
 			{
-				char	   *exprsString;
+				bool		isnull;
+				Datum		datum;
 
-				exprsString = TextDatumGetCString(datum);
-				exprs = (List *) stringToNode(exprsString);
-				pfree(exprsString);
+				/* decode expression (if any) */
+				datum = SysCacheGetAttr(STATEXTOID, htup,
+										Anum_pg_statistic_ext_stxexprs, &isnull);
 
-				/*
-				 * Run the expressions through eval_const_expressions. This is
-				 * not just an optimization, but is necessary, because the
-				 * planner will be comparing them to similarly-processed qual
-				 * clauses, and may fail to detect valid matches without this.
-				 * We must not use canonicalize_qual, however, since these
-				 * aren't qual expressions.
-				 */
-				exprs = (List *) eval_const_expressions(NULL, (Node *) exprs);
+				if (!isnull)
+				{
+					char	   *exprsString;
 
-				/* May as well fix opfuncids too */
-				fix_opfuncids((Node *) exprs);
+					exprsString = TextDatumGetCString(datum);
+					exprs = (List *) stringToNode(exprsString);
+					pfree(exprsString);
 
-				/*
-				 * Modify the copies we obtain from the relcache to have the
-				 * correct varno for the parent relation, so that they match
-				 * up correctly against qual clauses.
-				 */
-				if (varno != 1)
-					ChangeVarNodes((Node *) exprs, 1, varno, 0);
+					/*
+					 * Run the expressions through eval_const_expressions. This is
+					 * not just an optimization, but is necessary, because the
+					 * planner will be comparing them to similarly-processed qual
+					 * clauses, and may fail to detect valid matches without this.
+					 * We must not use canonicalize_qual, however, since these
+					 * aren't qual expressions.
+					 */
+					exprs = (List *) eval_const_expressions(NULL, (Node *) exprs);
+
+					/* May as well fix opfuncids too */
+					fix_opfuncids((Node *) exprs);
+
+					/*
+					 * Modify the copies we obtain from the relcache to have the
+					 * correct varno for the parent relation, so that they match
+					 * up correctly against qual clauses.
+					 */
+					if (varno != 1)
+						ChangeVarNodes((Node *) exprs, 1, varno, 0);
+				}
 			}
-		}
 
-		/* add one StatisticExtInfo for each kind built */
-		if (statext_is_kind_built(dtup, STATS_EXT_NDISTINCT))
-		{
-			StatisticExtInfo *info = makeNode(StatisticExtInfo);
+			/* add one StatisticExtInfo for each kind built */
+			if (statext_is_kind_built(dtup, STATS_EXT_NDISTINCT))
+			{
+				StatisticExtInfo *info = makeNode(StatisticExtInfo);
 
-			info->statOid = statOid;
-			info->rel = rel;
-			info->kind = STATS_EXT_NDISTINCT;
-			info->keys = bms_copy(keys);
-			info->exprs = exprs;
+				info->statOid = statOid;
+				info->inherit = dataForm->stxdinherit;
+				info->rel = rel;
+				info->kind = STATS_EXT_NDISTINCT;
+				info->keys = bms_copy(keys);
+				info->exprs = exprs;
 
-			stainfos = lappend(stainfos, info);
-		}
+				stainfos = lappend(stainfos, info);
+			}
 
-		if (statext_is_kind_built(dtup, STATS_EXT_DEPENDENCIES))
-		{
-			StatisticExtInfo *info = makeNode(StatisticExtInfo);
+			if (statext_is_kind_built(dtup, STATS_EXT_DEPENDENCIES))
+			{
+				StatisticExtInfo *info = makeNode(StatisticExtInfo);
 
-			info->statOid = statOid;
-			info->rel = rel;
-			info->kind = STATS_EXT_DEPENDENCIES;
-			info->keys = bms_copy(keys);
-			info->exprs = exprs;
+				info->statOid = statOid;
+				info->inherit = dataForm->stxdinherit;
+				info->rel = rel;
+				info->kind = STATS_EXT_DEPENDENCIES;
+				info->keys = bms_copy(keys);
+				info->exprs = exprs;
 
-			stainfos = lappend(stainfos, info);
-		}
+				stainfos = lappend(stainfos, info);
+			}
 
-		if (statext_is_kind_built(dtup, STATS_EXT_MCV))
-		{
-			StatisticExtInfo *info = makeNode(StatisticExtInfo);
+			if (statext_is_kind_built(dtup, STATS_EXT_MCV))
+			{
+				StatisticExtInfo *info = makeNode(StatisticExtInfo);
 
-			info->statOid = statOid;
-			info->rel = rel;
-			info->kind = STATS_EXT_MCV;
-			info->keys = bms_copy(keys);
-			info->exprs = exprs;
+				info->statOid = statOid;
+				info->inherit = dataForm->stxdinherit;
+				info->rel = rel;
+				info->kind = STATS_EXT_MCV;
+				info->keys = bms_copy(keys);
+				info->exprs = exprs;
 
-			stainfos = lappend(stainfos, info);
-		}
+				stainfos = lappend(stainfos, info);
+			}
 
-		if (statext_is_kind_built(dtup, STATS_EXT_EXPRESSIONS))
-		{
-			StatisticExtInfo *info = makeNode(StatisticExtInfo);
+			if (statext_is_kind_built(dtup, STATS_EXT_EXPRESSIONS))
+			{
+				StatisticExtInfo *info = makeNode(StatisticExtInfo);
 
-			info->statOid = statOid;
-			info->rel = rel;
-			info->kind = STATS_EXT_EXPRESSIONS;
-			info->keys = bms_copy(keys);
-			info->exprs = exprs;
+				info->statOid = statOid;
+				info->inherit = dataForm->stxdinherit;
+				info->rel = rel;
+				info->kind = STATS_EXT_EXPRESSIONS;
+				info->keys = bms_copy(keys);
+				info->exprs = exprs;
+
+				stainfos = lappend(stainfos, info);
+			}
 
-			stainfos = lappend(stainfos, info);
+			ReleaseSysCache(dtup);
 		}
 
 		ReleaseSysCache(htup);
-		ReleaseSysCache(dtup);
 		bms_free(keys);
 	}
 
diff --git a/src/backend/statistics/dependencies.c b/src/backend/statistics/dependencies.c
index 36121d5a27..c62fc398ce 100644
--- a/src/backend/statistics/dependencies.c
+++ b/src/backend/statistics/dependencies.c
@@ -618,14 +618,16 @@ dependency_is_fully_matched(MVDependency *dependency, Bitmapset *attnums)
  *		Load the functional dependencies for the indicated pg_statistic_ext tuple
  */
 MVDependencies *
-statext_dependencies_load(Oid mvoid)
+statext_dependencies_load(Oid mvoid, bool inh)
 {
 	MVDependencies *result;
 	bool		isnull;
 	Datum		deps;
 	HeapTuple	htup;
 
-	htup = SearchSysCache1(STATEXTDATASTXOID, ObjectIdGetDatum(mvoid));
+	htup = SearchSysCache2(STATEXTDATASTXOID,
+						   ObjectIdGetDatum(mvoid),
+						   BoolGetDatum(inh));
 	if (!HeapTupleIsValid(htup))
 		elog(ERROR, "cache lookup failed for statistics object %u", mvoid);
 
@@ -1603,6 +1605,10 @@ dependencies_clauselist_selectivity(PlannerInfo *root,
 		if (stat->kind != STATS_EXT_DEPENDENCIES)
 			continue;
 
+		/* skip statistics with mismatching stxdinherit value */
+		if (stat->inherit != rte->inh)
+			continue;
+
 		/*
 		 * Count matching attributes - we have to undo the attnum offsets. The
 		 * input attribute numbers are not offset (expressions are not
@@ -1649,7 +1655,7 @@ dependencies_clauselist_selectivity(PlannerInfo *root,
 		if (nmatched + nexprs < 2)
 			continue;
 
-		deps = statext_dependencies_load(stat->statOid);
+		deps = statext_dependencies_load(stat->statOid, rte->inh);
 
 		/*
 		 * The expressions may be represented by different attnums in the
diff --git a/src/backend/statistics/extended_stats.c b/src/backend/statistics/extended_stats.c
index 5fbdbf0164..ff2ef67dd2 100644
--- a/src/backend/statistics/extended_stats.c
+++ b/src/backend/statistics/extended_stats.c
@@ -77,7 +77,7 @@ typedef struct StatExtEntry
 static List *fetch_statentries_for_relation(Relation pg_statext, Oid relid);
 static VacAttrStats **lookup_var_attr_stats(Relation rel, Bitmapset *attrs, List *exprs,
 											int nvacatts, VacAttrStats **vacatts);
-static void statext_store(Oid statOid,
+static void statext_store(Oid statOid, bool inh,
 						  MVNDistinct *ndistinct, MVDependencies *dependencies,
 						  MCVList *mcv, Datum exprs, VacAttrStats **stats);
 static int	statext_compute_stattarget(int stattarget,
@@ -110,7 +110,7 @@ static StatsBuildData *make_build_data(Relation onerel, StatExtEntry *stat,
  * requested stats, and serializes them back into the catalog.
  */
 void
-BuildRelationExtStatistics(Relation onerel, double totalrows,
+BuildRelationExtStatistics(Relation onerel, bool inh, double totalrows,
 						   int numrows, HeapTuple *rows,
 						   int natts, VacAttrStats **vacattrstats)
 {
@@ -230,7 +230,8 @@ BuildRelationExtStatistics(Relation onerel, double totalrows,
 		}
 
 		/* store the statistics in the catalog */
-		statext_store(stat->statOid, ndistinct, dependencies, mcv, exprstats, stats);
+		statext_store(stat->statOid, inh,
+					  ndistinct, dependencies, mcv, exprstats, stats);
 
 		/* for reporting progress */
 		pgstat_progress_update_param(PROGRESS_ANALYZE_EXT_STATS_COMPUTED,
@@ -781,7 +782,7 @@ lookup_var_attr_stats(Relation rel, Bitmapset *attrs, List *exprs,
  *	tuple.
  */
 static void
-statext_store(Oid statOid,
+statext_store(Oid statOid, bool inh,
 			  MVNDistinct *ndistinct, MVDependencies *dependencies,
 			  MCVList *mcv, Datum exprs, VacAttrStats **stats)
 {
@@ -790,14 +791,19 @@ statext_store(Oid statOid,
 				oldtup;
 	Datum		values[Natts_pg_statistic_ext_data];
 	bool		nulls[Natts_pg_statistic_ext_data];
-	bool		replaces[Natts_pg_statistic_ext_data];
 
 	pg_stextdata = table_open(StatisticExtDataRelationId, RowExclusiveLock);
 
 	memset(nulls, true, sizeof(nulls));
-	memset(replaces, false, sizeof(replaces));
 	memset(values, 0, sizeof(values));
 
+	/* basic info */
+	values[Anum_pg_statistic_ext_data_stxoid - 1] = ObjectIdGetDatum(statOid);
+	nulls[Anum_pg_statistic_ext_data_stxoid - 1] = false;
+
+	values[Anum_pg_statistic_ext_data_stxdinherit - 1] = BoolGetDatum(inh);
+	nulls[Anum_pg_statistic_ext_data_stxdinherit - 1] = false;
+
 	/*
 	 * Construct a new pg_statistic_ext_data tuple, replacing the calculated
 	 * stats.
@@ -830,25 +836,27 @@ statext_store(Oid statOid,
 		values[Anum_pg_statistic_ext_data_stxdexpr - 1] = exprs;
 	}
 
-	/* always replace the value (either by bytea or NULL) */
-	replaces[Anum_pg_statistic_ext_data_stxdndistinct - 1] = true;
-	replaces[Anum_pg_statistic_ext_data_stxddependencies - 1] = true;
-	replaces[Anum_pg_statistic_ext_data_stxdmcv - 1] = true;
-	replaces[Anum_pg_statistic_ext_data_stxdexpr - 1] = true;
-
-	/* there should already be a pg_statistic_ext_data tuple */
-	oldtup = SearchSysCache1(STATEXTDATASTXOID, ObjectIdGetDatum(statOid));
-	if (!HeapTupleIsValid(oldtup))
+	/*
+	 * Delete the old tuple if it exists, and insert a new one. It's easier
+	 * than trying to update or insert, based on various conditions.
+	 *
+	 * There should always be a pg_statistic_ext_data tuple for inh=false,
+	 * but there may be none for inh=true yet.
+	 */
+	oldtup = SearchSysCache2(STATEXTDATASTXOID,
+							 ObjectIdGetDatum(statOid),
+							 BoolGetDatum(inh));
+	if (HeapTupleIsValid(oldtup))
+	{
+		CatalogTupleDelete(pg_stextdata, &(oldtup->t_self));
+		ReleaseSysCache(oldtup);
+	}
+	else if (!inh)
 		elog(ERROR, "cache lookup failed for statistics object %u", statOid);
 
-	/* replace it */
-	stup = heap_modify_tuple(oldtup,
-							 RelationGetDescr(pg_stextdata),
-							 values,
-							 nulls,
-							 replaces);
-	ReleaseSysCache(oldtup);
-	CatalogTupleUpdate(pg_stextdata, &stup->t_self, stup);
+	/* form a new tuple */
+	stup = heap_form_tuple(RelationGetDescr(pg_stextdata), values, nulls);
+	CatalogTupleInsert(pg_stextdata, stup);
 
 	heap_freetuple(stup);
 
@@ -1234,7 +1242,7 @@ stat_covers_expressions(StatisticExtInfo *stat, List *exprs,
  * further tiebreakers are needed.
  */
 StatisticExtInfo *
-choose_best_statistics(List *stats, char requiredkind,
+choose_best_statistics(List *stats, char requiredkind, bool inh,
 					   Bitmapset **clause_attnums, List **clause_exprs,
 					   int nclauses)
 {
@@ -1256,6 +1264,10 @@ choose_best_statistics(List *stats, char requiredkind,
 		if (info->kind != requiredkind)
 			continue;
 
+		/* skip statistics with mismatching inheritance flag */
+		if (info->inherit != inh)
+			continue;
+
 		/*
 		 * Collect attributes and expressions in remaining (unestimated)
 		 * clauses fully covered by this statistic object.
@@ -1694,11 +1706,6 @@ statext_mcv_clauselist_selectivity(PlannerInfo *root, List *clauses, int varReli
 	List	  **list_exprs;		/* expressions matched to any statistic */
 	int			listidx;
 	Selectivity sel = (is_or) ? 0.0 : 1.0;
-	RangeTblEntry *rte = root->simple_rte_array[rel->relid];
-
-	/* If it's an inheritence tree, skip statistics (which do not include child stats) */
-	if (rte->inh && rte->relkind != RELKIND_PARTITIONED_TABLE)
-		return sel;
 
 	/* check if there's any stats that might be useful for us. */
 	if (!has_stats_of_kind(rel->statlist, STATS_EXT_MCV))
@@ -1751,7 +1758,7 @@ statext_mcv_clauselist_selectivity(PlannerInfo *root, List *clauses, int varReli
 		Bitmapset  *simple_clauses;
 
 		/* find the best suited statistics object for these attnums */
-		stat = choose_best_statistics(rel->statlist, STATS_EXT_MCV,
+		stat = choose_best_statistics(rel->statlist, STATS_EXT_MCV, rte->inh,
 									  list_attnums, list_exprs,
 									  list_length(clauses));
 
@@ -1840,7 +1847,7 @@ statext_mcv_clauselist_selectivity(PlannerInfo *root, List *clauses, int varReli
 			MCVList    *mcv_list;
 
 			/* Load the MCV list stored in the statistics object */
-			mcv_list = statext_mcv_load(stat->statOid);
+			mcv_list = statext_mcv_load(stat->statOid, rte->inh);
 
 			/*
 			 * Compute the selectivity of the ORed list of clauses covered by
@@ -2411,7 +2418,7 @@ statext_expressions_load(Oid stxoid, int idx)
 	HeapTupleData tmptup;
 	HeapTuple	tup;
 
-	htup = SearchSysCache1(STATEXTDATASTXOID, ObjectIdGetDatum(stxoid));
+	htup = SearchSysCache2(STATEXTDATASTXOID, ObjectIdGetDatum(stxoid), BoolGetDatum(false));
 	if (!HeapTupleIsValid(htup))
 		elog(ERROR, "cache lookup failed for statistics object %u", stxoid);
 
diff --git a/src/backend/statistics/mcv.c b/src/backend/statistics/mcv.c
index b350fc5f7b..75d7d35caa 100644
--- a/src/backend/statistics/mcv.c
+++ b/src/backend/statistics/mcv.c
@@ -559,12 +559,13 @@ build_column_frequencies(SortItem *groups, int ngroups,
  *		Load the MCV list for the indicated pg_statistic_ext tuple.
  */
 MCVList *
-statext_mcv_load(Oid mvoid)
+statext_mcv_load(Oid mvoid, bool inh)
 {
 	MCVList    *result;
 	bool		isnull;
 	Datum		mcvlist;
-	HeapTuple	htup = SearchSysCache1(STATEXTDATASTXOID, ObjectIdGetDatum(mvoid));
+	HeapTuple	htup = SearchSysCache2(STATEXTDATASTXOID,
+									   ObjectIdGetDatum(mvoid), BoolGetDatum(inh));
 
 	if (!HeapTupleIsValid(htup))
 		elog(ERROR, "cache lookup failed for statistics object %u", mvoid);
@@ -2039,11 +2040,13 @@ mcv_clauselist_selectivity(PlannerInfo *root, StatisticExtInfo *stat,
 	MCVList    *mcv;
 	Selectivity s = 0.0;
 
+	RangeTblEntry *rte = root->simple_rte_array[rel->relid];
+
 	/* match/mismatch bitmap for each MCV item */
 	bool	   *matches = NULL;
 
 	/* load the MCV list stored in the statistics object */
-	mcv = statext_mcv_load(stat->statOid);
+	mcv = statext_mcv_load(stat->statOid, rte->inh);
 
 	/* build a match bitmap for the clauses */
 	matches = mcv_get_match_bitmap(root, clauses, stat->keys, stat->exprs,
diff --git a/src/backend/statistics/mvdistinct.c b/src/backend/statistics/mvdistinct.c
index 4481312d61..ab1f10d6c0 100644
--- a/src/backend/statistics/mvdistinct.c
+++ b/src/backend/statistics/mvdistinct.c
@@ -146,14 +146,15 @@ statext_ndistinct_build(double totalrows, StatsBuildData *data)
  *		Load the ndistinct value for the indicated pg_statistic_ext tuple
  */
 MVNDistinct *
-statext_ndistinct_load(Oid mvoid)
+statext_ndistinct_load(Oid mvoid, bool inh)
 {
 	MVNDistinct *result;
 	bool		isnull;
 	Datum		ndist;
 	HeapTuple	htup;
 
-	htup = SearchSysCache1(STATEXTDATASTXOID, ObjectIdGetDatum(mvoid));
+	htup = SearchSysCache2(STATEXTDATASTXOID,
+						   ObjectIdGetDatum(mvoid), BoolGetDatum(inh));
 	if (!HeapTupleIsValid(htup))
 		elog(ERROR, "cache lookup failed for statistics object %u", mvoid);
 
diff --git a/src/backend/utils/adt/selfuncs.c b/src/backend/utils/adt/selfuncs.c
index b15f14e1a0..aab3bd7696 100644
--- a/src/backend/utils/adt/selfuncs.c
+++ b/src/backend/utils/adt/selfuncs.c
@@ -4008,7 +4008,7 @@ estimate_multivariate_ndistinct(PlannerInfo *root, RelOptInfo *rel,
 
 	Assert(nmatches_vars + nmatches_exprs > 1);
 
-	stats = statext_ndistinct_load(statOid);
+	stats = statext_ndistinct_load(statOid, rte->inh);
 
 	/*
 	 * If we have a match, search it for the specific item that matches (there
diff --git a/src/backend/utils/cache/syscache.c b/src/backend/utils/cache/syscache.c
index 56870b46e4..6194751e99 100644
--- a/src/backend/utils/cache/syscache.c
+++ b/src/backend/utils/cache/syscache.c
@@ -763,11 +763,11 @@ static const struct cachedesc cacheinfo[] = {
 		32
 	},
 	{StatisticExtDataRelationId,	/* STATEXTDATASTXOID */
-		StatisticExtDataStxoidIndexId,
-		1,
+		StatisticExtDataStxoidInhIndexId,
+		2,
 		{
 			Anum_pg_statistic_ext_data_stxoid,
-			0,
+			Anum_pg_statistic_ext_data_stxdinherit,
 			0,
 			0
 		},
diff --git a/src/include/catalog/pg_statistic_ext_data.h b/src/include/catalog/pg_statistic_ext_data.h
index 7b73b790d2..8ffd8b68cd 100644
--- a/src/include/catalog/pg_statistic_ext_data.h
+++ b/src/include/catalog/pg_statistic_ext_data.h
@@ -32,6 +32,7 @@ CATALOG(pg_statistic_ext_data,3429,StatisticExtDataRelationId)
 {
 	Oid			stxoid BKI_LOOKUP(pg_statistic_ext);	/* statistics object
 														 * this data is for */
+	bool		stxdinherit;	/* true if inheritance children are included */
 
 #ifdef CATALOG_VARLEN			/* variable-length fields start here */
 
@@ -53,6 +54,7 @@ typedef FormData_pg_statistic_ext_data * Form_pg_statistic_ext_data;
 
 DECLARE_TOAST(pg_statistic_ext_data, 3430, 3431);
 
-DECLARE_UNIQUE_INDEX_PKEY(pg_statistic_ext_data_stxoid_index, 3433, StatisticExtDataStxoidIndexId, on pg_statistic_ext_data using btree(stxoid oid_ops));
+DECLARE_UNIQUE_INDEX_PKEY(pg_statistic_ext_data_stxoid_inh_index, 3433, StatisticExtDataStxoidInhIndexId, on pg_statistic_ext_data using btree(stxoid oid_ops, stxdinherit bool_ops));
+
 
 #endif							/* PG_STATISTIC_EXT_DATA_H */
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 2a53a6e344..884bda7232 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -934,6 +934,7 @@ typedef struct StatisticExtInfo
 	NodeTag		type;
 
 	Oid			statOid;		/* OID of the statistics row */
+	bool		inherit;		/* includes child relations */
 	RelOptInfo *rel;			/* back-link to statistic's table */
 	char		kind;			/* statistics kind of this entry */
 	Bitmapset  *keys;			/* attnums of the columns covered */
diff --git a/src/include/statistics/statistics.h b/src/include/statistics/statistics.h
index 326cf26fea..02ee41b9f3 100644
--- a/src/include/statistics/statistics.h
+++ b/src/include/statistics/statistics.h
@@ -94,11 +94,11 @@ typedef struct MCVList
 	MCVItem		items[FLEXIBLE_ARRAY_MEMBER];	/* array of MCV items */
 } MCVList;
 
-extern MVNDistinct *statext_ndistinct_load(Oid mvoid);
-extern MVDependencies *statext_dependencies_load(Oid mvoid);
-extern MCVList *statext_mcv_load(Oid mvoid);
+extern MVNDistinct *statext_ndistinct_load(Oid mvoid, bool inh);
+extern MVDependencies *statext_dependencies_load(Oid mvoid, bool inh);
+extern MCVList *statext_mcv_load(Oid mvoid, bool inh);
 
-extern void BuildRelationExtStatistics(Relation onerel, double totalrows,
+extern void BuildRelationExtStatistics(Relation onerel, bool inh, double totalrows,
 									   int numrows, HeapTuple *rows,
 									   int natts, VacAttrStats **vacattrstats);
 extern int	ComputeExtStatisticsRows(Relation onerel,
@@ -121,6 +121,7 @@ extern Selectivity statext_clauselist_selectivity(PlannerInfo *root,
 												  bool is_or);
 extern bool has_stats_of_kind(List *stats, char requiredkind);
 extern StatisticExtInfo *choose_best_statistics(List *stats, char requiredkind,
+												bool inh,
 												Bitmapset **clause_attnums,
 												List **clause_exprs,
 												int nclauses);
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 2fa00a3c29..8ab5187ccb 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2425,6 +2425,7 @@ pg_stats_ext| SELECT cn.nspname AS schemaname,
              JOIN pg_attribute a ON (((a.attrelid = s.stxrelid) AND (a.attnum = k.k))))) AS attnames,
     pg_get_statisticsobjdef_expressions(s.oid) AS exprs,
     s.stxkind AS kinds,
+    sd.stxdinherit AS inherited,
     sd.stxdndistinct AS n_distinct,
     sd.stxddependencies AS dependencies,
     m.most_common_vals,
-- 
2.17.0

