]> granicus.if.org Git - postgresql/commitdiff
Allow a partitioned table to have a default partition.
authorRobert Haas <rhaas@postgresql.org>
Fri, 8 Sep 2017 21:28:04 +0000 (17:28 -0400)
committerRobert Haas <rhaas@postgresql.org>
Fri, 8 Sep 2017 21:28:04 +0000 (17:28 -0400)
Any tuples that don't route to any other partition will route to the
default partition.

Jeevan Ladhe, Beena Emerson, Ashutosh Bapat, Rahila Syed, and Robert
Haas, with review and testing at various stages by (at least) Rushabh
Lathia, Keith Fiske, Amit Langote, Amul Sul, Rajkumar Raghuanshi, Sven
Kunze, Kyotaro Horiguchi, Thom Brown, Rafia Sabih, and Dilip Kumar.

Discussion: http://postgr.es/m/CAH2L28tbN4SYyhS7YV1YBWcitkqbhSWfQCy0G=apRcC_PEO-bg@mail.gmail.com
Discussion: http://postgr.es/m/CAOG9ApEYj34fWMcvBMBQ-YtqR9fTdXhdN82QEKG0SVZ6zeL1xg@mail.gmail.com

31 files changed:
doc/src/sgml/catalogs.sgml
doc/src/sgml/ref/alter_table.sgml
doc/src/sgml/ref/create_table.sgml
src/backend/catalog/heap.c
src/backend/catalog/partition.c
src/backend/commands/tablecmds.c
src/backend/nodes/copyfuncs.c
src/backend/nodes/equalfuncs.c
src/backend/nodes/outfuncs.c
src/backend/nodes/readfuncs.c
src/backend/parser/gram.y
src/backend/parser/parse_utilcmd.c
src/backend/utils/adt/ruleutils.c
src/bin/psql/describe.c
src/bin/psql/tab-complete.c
src/include/catalog/catversion.h
src/include/catalog/partition.h
src/include/catalog/pg_partitioned_table.h
src/include/commands/tablecmds.h
src/include/nodes/parsenodes.h
src/test/regress/expected/alter_table.out
src/test/regress/expected/create_table.out
src/test/regress/expected/insert.out
src/test/regress/expected/plancache.out
src/test/regress/expected/sanity_check.out
src/test/regress/expected/update.out
src/test/regress/sql/alter_table.sql
src/test/regress/sql/create_table.sql
src/test/regress/sql/insert.sql
src/test/regress/sql/plancache.sql
src/test/regress/sql/update.sql

index 4f56188a1cf985687113e655ad45ec424d7bff36..4978b47f0e4735c5ae3db77af84878669fb70c01 100644 (file)
@@ -4738,6 +4738,17 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</>:<replaceable>&lt;salt&gt;<
       <entry>The number of columns in partition key</entry>
      </row>
 
+     <row>
+      <entry><structfield>partdefid</structfield></entry>
+      <entry><type>oid</type></entry>
+      <entry><literal><link linkend="catalog-pg-class"><structname>pg_class</structname></link>.oid</literal></entry>
+      <entry>
+       The OID of the <structname>pg_class</> entry for the default partition
+       of this partitioned table, or zero if this partitioned table does not
+       have a default partition.
+     </entry>
+     </row>
+
      <row>
       <entry><structfield>partattrs</structfield></entry>
       <entry><type>int2vector</type></entry>
index dae63077eedf69e2ed2723e25f11da231e5e9401..0fb385ece7a0cd4928de6a019810f14f30610d02 100644 (file)
@@ -34,7 +34,7 @@ ALTER TABLE [ IF EXISTS ] <replaceable class="PARAMETER">name</replaceable>
 ALTER TABLE ALL IN TABLESPACE <replaceable class="PARAMETER">name</replaceable> [ OWNED BY <replaceable class="PARAMETER">role_name</replaceable> [, ... ] ]
     SET TABLESPACE <replaceable class="PARAMETER">new_tablespace</replaceable> [ NOWAIT ]
 ALTER TABLE [ IF EXISTS ] <replaceable class="PARAMETER">name</replaceable>
-    ATTACH PARTITION <replaceable class="PARAMETER">partition_name</replaceable> FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable>
+    ATTACH PARTITION <replaceable class="PARAMETER">partition_name</replaceable> { FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable> | DEFAULT }
 ALTER TABLE [ IF EXISTS ] <replaceable class="PARAMETER">name</replaceable>
     DETACH PARTITION <replaceable class="PARAMETER">partition_name</replaceable>
 
@@ -765,11 +765,18 @@ ALTER TABLE [ IF EXISTS ] <replaceable class="PARAMETER">name</replaceable>
    </varlistentry>
 
    <varlistentry>
-    <term><literal>ATTACH PARTITION <replaceable class="PARAMETER">partition_name</replaceable> FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable></literal></term>
+    <term><literal>ATTACH PARTITION <replaceable class="PARAMETER">partition_name</replaceable> { FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable> | DEFAULT }</literal></term>
     <listitem>
      <para>
       This form attaches an existing table (which might itself be partitioned)
-      as a partition of the target table using the same syntax for
+      as a partition of the target table. The table can be attached
+      as a partition for specific values using <literal>FOR VALUES
+      </literal> or as a default partition by using <literal>DEFAULT
+      </literal>.
+     </para>
+
+     <para>
+      A partition using <literal>FOR VALUES</literal> uses same syntax for
       <replaceable class="PARAMETER">partition_bound_spec</replaceable> as
       <xref linkend="sql-createtable">.  The partition bound specification
       must correspond to the partitioning strategy and partition key of the
@@ -806,6 +813,17 @@ ALTER TABLE [ IF EXISTS ] <replaceable class="PARAMETER">name</replaceable>
       (See the discussion in <xref linkend="SQL-CREATEFOREIGNTABLE"> about
       constraints on the foreign table.)
      </para>
+
+     <para>
+      When a table has a default partition, defining a new partition changes
+      the partition constraint for the default partition. The default
+      partition can't contain any rows that would need to be moved to the new
+      partition, and will be scanned to verify that none are present. This
+      scan, like the scan of the new partition, can be avoided if an
+      appropriate <literal>CHECK</literal> constraint is present. Also like
+      the scan of the new partition, it is always skipped when the default
+      partition is a foreign table.
+     </para>
     </listitem>
    </varlistentry>
 
@@ -1395,6 +1413,13 @@ ALTER TABLE cities
     ATTACH PARTITION cities_ab FOR VALUES IN ('a', 'b');
 </programlisting></para>
 
+  <para>
+   Attach a default partition to a partitioned table:
+<programlisting>
+ALTER TABLE cities
+    ATTACH PARTITION cities_partdef DEFAULT;
+</programlisting></para>
+
   <para>
    Detach a partition from partitioned table:
 <programlisting>
index a6ca590249976234e03b3c648c1cc85eaa962930..824253de404feb09272fcd9399d63a5ecd7c527b 100644 (file)
@@ -49,7 +49,7 @@ CREATE [ [ GLOBAL | LOCAL ] { TEMPORARY | TEMP } | UNLOGGED ] TABLE [ IF NOT EXI
   { <replaceable class="PARAMETER">column_name</replaceable> [ WITH OPTIONS ] [ <replaceable class="PARAMETER">column_constraint</replaceable> [ ... ] ]
     | <replaceable>table_constraint</replaceable> }
     [, ... ]
-) ] FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable>
+) ] { FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable> | DEFAULT }
 [ PARTITION BY { RANGE | LIST } ( { <replaceable class="parameter">column_name</replaceable> | ( <replaceable class="parameter">expression</replaceable> ) } [ COLLATE <replaceable class="parameter">collation</replaceable> ] [ <replaceable class="parameter">opclass</replaceable> ] [, ... ] ) ]
 [ WITH ( <replaceable class="PARAMETER">storage_parameter</replaceable> [= <replaceable class="PARAMETER">value</replaceable>] [, ... ] ) | WITH OIDS | WITHOUT OIDS ]
 [ ON COMMIT { PRESERVE ROWS | DELETE ROWS | DROP } ]
@@ -250,11 +250,13 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
    </varlistentry>
 
    <varlistentry id="SQL-CREATETABLE-PARTITION">
-    <term><literal>PARTITION OF <replaceable class="PARAMETER">parent_table</replaceable> FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable></literal></term>
+    <term><literal>PARTITION OF <replaceable class="PARAMETER">parent_table</replaceable> { FOR VALUES <replaceable class="PARAMETER">partition_bound_spec</replaceable> | DEFAULT }</literal></term>
     <listitem>
      <para>
       Creates the table as a <firstterm>partition</firstterm> of the specified
-      parent table.
+      parent table. The table can be created either as a partition for specific
+      values using <literal>FOR VALUES</literal> or as a default partition
+      using <literal>DEFAULT</literal>.
      </para>
 
      <para>
@@ -342,6 +344,26 @@ FROM ( { <replaceable class="PARAMETER">numeric_literal</replaceable> | <replace
       allows precisely one value to be stored &mdash; "infinity".
      </para>
 
+     <para>
+      If <literal>DEFAULT</literal> is specified, the table will be
+      created as a default partition of the parent table. The parent can
+      either be a list or range partitioned table. A partition key value
+      not fitting into any other partition of the given parent will be
+      routed to the default partition. There can be only one default
+      partition for a given parent table.
+     </para>
+
+     <para>
+      When a table has an existing <literal>DEFAULT</literal> partition and
+      a new partition is added to it, the existing default partition must
+      be scanned to verify that it does not contain any rows which properly
+      belong in the new partition.  If the default partition contains a
+      large number of rows, this may be slow.  The scan will be skipped if
+      the default partition is a foreign table or if it has a constraint which
+      proves that it cannot contain rows which should be placed in the new
+      partition.
+     </para>
+
      <para>
       A partition must have the same column names and types as the partitioned
       table to which it belongs.  If the parent is specified <literal>WITH
@@ -1679,6 +1701,13 @@ CREATE TABLE cities_ab
 CREATE TABLE cities_ab_10000_to_100000
     PARTITION OF cities_ab FOR VALUES FROM (10000) TO (100000);
 </programlisting></para>
+
+  <para>
+   Create a default partition:
+<programlisting>
+CREATE TABLE cities_partdef
+    PARTITION OF cities DEFAULT;
+</programlisting></para>
  </refsect1>
 
  <refsect1 id="SQL-CREATETABLE-compatibility">
index 45ee9ac8b9cb8045e44b744b7de6627370482932..05e70818e771c5c3b5b5ca50e0f7d5d2cf5738b9 100644 (file)
@@ -1759,7 +1759,8 @@ heap_drop_with_catalog(Oid relid)
 {
        Relation        rel;
        HeapTuple       tuple;
-       Oid                     parentOid = InvalidOid;
+       Oid                     parentOid = InvalidOid,
+                               defaultPartOid = InvalidOid;
 
        /*
         * To drop a partition safely, we must grab exclusive lock on its parent,
@@ -1775,6 +1776,14 @@ heap_drop_with_catalog(Oid relid)
        {
                parentOid = get_partition_parent(relid);
                LockRelationOid(parentOid, AccessExclusiveLock);
+
+               /*
+                * If this is not the default partition, dropping it will change the
+                * default partition's partition constraint, so we must lock it.
+                */
+               defaultPartOid = get_default_partition_oid(parentOid);
+               if (OidIsValid(defaultPartOid) && relid != defaultPartOid)
+                       LockRelationOid(defaultPartOid, AccessExclusiveLock);
        }
 
        ReleaseSysCache(tuple);
@@ -1825,6 +1834,13 @@ heap_drop_with_catalog(Oid relid)
        if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
                RemovePartitionKeyByRelId(relid);
 
+       /*
+        * If the relation being dropped is the default partition itself,
+        * invalidate its entry in pg_partitioned_table.
+        */
+       if (relid == defaultPartOid)
+               update_default_partition_oid(parentOid, InvalidOid);
+
        /*
         * Schedule unlinking of the relation's physical files at commit.
         */
@@ -1884,6 +1900,14 @@ heap_drop_with_catalog(Oid relid)
 
        if (OidIsValid(parentOid))
        {
+               /*
+                * If this is not the default partition, the partition constraint of
+                * the default partition has changed to include the portion of the key
+                * space previously covered by the dropped partition.
+                */
+               if (OidIsValid(defaultPartOid) && relid != defaultPartOid)
+                       CacheInvalidateRelcacheByRelid(defaultPartOid);
+
                /*
                 * Invalidate the parent's relcache so that the partition is no longer
                 * included in its partition descriptor.
@@ -3138,6 +3162,7 @@ StorePartitionKey(Relation rel,
        values[Anum_pg_partitioned_table_partrelid - 1] = ObjectIdGetDatum(RelationGetRelid(rel));
        values[Anum_pg_partitioned_table_partstrat - 1] = CharGetDatum(strategy);
        values[Anum_pg_partitioned_table_partnatts - 1] = Int16GetDatum(partnatts);
+       values[Anum_pg_partitioned_table_partdefid - 1] = ObjectIdGetDatum(InvalidOid);
        values[Anum_pg_partitioned_table_partattrs - 1] = PointerGetDatum(partattrs_vec);
        values[Anum_pg_partitioned_table_partclass - 1] = PointerGetDatum(partopclass_vec);
        values[Anum_pg_partitioned_table_partcollation - 1] = PointerGetDatum(partcollation_vec);
@@ -3223,7 +3248,8 @@ RemovePartitionKeyByRelId(Oid relid)
  *             relispartition to true
  *
  * Also, invalidate the parent's relcache, so that the next rebuild will load
- * the new partition's info into its partition descriptor.
+ * the new partition's info into its partition descriptor.  If there is a
+ * default partition, we must invalidate its relcache entry as well.
  */
 void
 StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
@@ -3234,6 +3260,7 @@ StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
        Datum           new_val[Natts_pg_class];
        bool            new_null[Natts_pg_class],
                                new_repl[Natts_pg_class];
+       Oid                     defaultPartOid;
 
        /* Update pg_class tuple */
        classRel = heap_open(RelationRelationId, RowExclusiveLock);
@@ -3271,5 +3298,15 @@ StorePartitionBound(Relation rel, Relation parent, PartitionBoundSpec *bound)
        heap_freetuple(newtuple);
        heap_close(classRel, RowExclusiveLock);
 
+       /*
+        * The partition constraint for the default partition depends on the
+        * partition bounds of every other partition, so we must invalidate the
+        * relcache entry for that partition every time a partition is added or
+        * removed.
+        */
+       defaultPartOid = get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+       if (OidIsValid(defaultPartOid))
+               CacheInvalidateRelcacheByRelid(defaultPartOid);
+
        CacheInvalidateRelcache(parent);
 }
index c6bd02f77dc04f2c7ce79d12ac9459162551c913..7e426ba9c8813c11f15a748167925332b4b5ca27 100644 (file)
@@ -27,7 +27,9 @@
 #include "catalog/pg_inherits.h"
 #include "catalog/pg_inherits_fn.h"
 #include "catalog/pg_opclass.h"
+#include "catalog/pg_partitioned_table.h"
 #include "catalog/pg_type.h"
+#include "commands/tablecmds.h"
 #include "executor/executor.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
@@ -35,6 +37,7 @@
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/var.h"
 #include "rewrite/rewriteManip.h"
 #include "storage/lmgr.h"
@@ -80,9 +83,12 @@ typedef struct PartitionBoundInfoData
                                                                 * partitioned table) */
        int                     null_index;             /* Index of the null-accepting partition; -1
                                                                 * if there isn't one */
+       int                     default_index;  /* Index of the default partition; -1 if there
+                                                                * isn't one */
 } PartitionBoundInfoData;
 
 #define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
+#define partition_bound_has_default(bi) ((bi)->default_index != -1)
 
 /*
  * When qsort'ing partition bounds after reading from the catalog, each bound
@@ -120,8 +126,10 @@ static void get_range_key_properties(PartitionKey key, int keynum,
                                                 ListCell **partexprs_item,
                                                 Expr **keyCol,
                                                 Const **lower_val, Const **upper_val);
-static List *get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec);
-static List *get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec);
+static List *get_qual_for_list(Relation parent, PartitionBoundSpec *spec);
+static List *get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
+                                  bool for_default);
+static List *get_range_nulltest(PartitionKey key);
 static List *generate_partition_qual(Relation rel);
 
 static PartitionRangeBound *make_one_range_bound(PartitionKey key, int index,
@@ -162,6 +170,7 @@ RelationBuildPartitionDesc(Relation rel)
        MemoryContext oldcxt;
 
        int                     ndatums = 0;
+       int                     default_index = -1;
 
        /* List partitioning specific */
        PartitionListValue **all_values = NULL;
@@ -213,6 +222,22 @@ RelationBuildPartitionDesc(Relation rel)
                                                                &isnull);
                Assert(!isnull);
                boundspec = (Node *) stringToNode(TextDatumGetCString(datum));
+
+               /*
+                * Sanity check: If the PartitionBoundSpec says this is the default
+                * partition, its OID should correspond to whatever's stored in
+                * pg_partitioned_table.partdefid; if not, the catalog is corrupt.
+                */
+               if (castNode(PartitionBoundSpec, boundspec)->is_default)
+               {
+                       Oid                     partdefid;
+
+                       partdefid = get_default_partition_oid(RelationGetRelid(rel));
+                       if (partdefid != inhrelid)
+                               elog(ERROR, "expected partdefid %u, but got %u",
+                                        inhrelid, partdefid);
+               }
+
                boundspecs = lappend(boundspecs, boundspec);
                partoids = lappend_oid(partoids, inhrelid);
                ReleaseSysCache(tuple);
@@ -246,6 +271,18 @@ RelationBuildPartitionDesc(Relation rel)
                                if (spec->strategy != PARTITION_STRATEGY_LIST)
                                        elog(ERROR, "invalid strategy in partition bound spec");
 
+                               /*
+                                * Note the index of the partition bound spec for the default
+                                * partition. There's no datum to add to the list of non-null
+                                * datums for this partition.
+                                */
+                               if (spec->is_default)
+                               {
+                                       default_index = i;
+                                       i++;
+                                       continue;
+                               }
+
                                foreach(c, spec->listdatums)
                                {
                                        Const      *val = castNode(Const, lfirst(c));
@@ -325,6 +362,17 @@ RelationBuildPartitionDesc(Relation rel)
                                if (spec->strategy != PARTITION_STRATEGY_RANGE)
                                        elog(ERROR, "invalid strategy in partition bound spec");
 
+                               /*
+                                * Note the index of the partition bound spec for the default
+                                * partition. There's no datum to add to the allbounds array
+                                * for this partition.
+                                */
+                               if (spec->is_default)
+                               {
+                                       default_index = i++;
+                                       continue;
+                               }
+
                                lower = make_one_range_bound(key, i, spec->lowerdatums,
                                                                                         true);
                                upper = make_one_range_bound(key, i, spec->upperdatums,
@@ -334,10 +382,11 @@ RelationBuildPartitionDesc(Relation rel)
                                i++;
                        }
 
-                       Assert(ndatums == nparts * 2);
+                       Assert(ndatums == nparts * 2 ||
+                                  (default_index != -1 && ndatums == (nparts - 1) * 2));
 
                        /* Sort all the bounds in ascending order */
-                       qsort_arg(all_bounds, 2 * nparts,
+                       qsort_arg(all_bounds, ndatums,
                                          sizeof(PartitionRangeBound *),
                                          qsort_partition_rbound_cmp,
                                          (void *) key);
@@ -421,6 +470,7 @@ RelationBuildPartitionDesc(Relation rel)
                boundinfo = (PartitionBoundInfoData *)
                        palloc0(sizeof(PartitionBoundInfoData));
                boundinfo->strategy = key->strategy;
+               boundinfo->default_index = -1;
                boundinfo->ndatums = ndatums;
                boundinfo->null_index = -1;
                boundinfo->datums = (Datum **) palloc0(ndatums * sizeof(Datum *));
@@ -473,6 +523,21 @@ RelationBuildPartitionDesc(Relation rel)
                                                boundinfo->null_index = mapping[null_index];
                                        }
 
+                                       /* Assign mapped index for the default partition. */
+                                       if (default_index != -1)
+                                       {
+                                               /*
+                                                * The default partition accepts any value not
+                                                * specified in the lists of other partitions, hence
+                                                * it should not get mapped index while assigning
+                                                * those for non-null datums.
+                                                */
+                                               Assert(default_index >= 0 &&
+                                                          mapping[default_index] == -1);
+                                               mapping[default_index] = next_index++;
+                                               boundinfo->default_index = mapping[default_index];
+                                       }
+
                                        /* All partition must now have a valid mapping */
                                        Assert(next_index == nparts);
                                        break;
@@ -527,6 +592,14 @@ RelationBuildPartitionDesc(Relation rel)
                                                        boundinfo->indexes[i] = mapping[orig_index];
                                                }
                                        }
+
+                                       /* Assign mapped index for the default partition. */
+                                       if (default_index != -1)
+                                       {
+                                               Assert(default_index >= 0 && mapping[default_index] == -1);
+                                               mapping[default_index] = next_index++;
+                                               boundinfo->default_index = mapping[default_index];
+                                       }
                                        boundinfo->indexes[i] = -1;
                                        break;
                                }
@@ -577,6 +650,9 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
        if (b1->null_index != b2->null_index)
                return false;
 
+       if (b1->default_index != b2->default_index)
+               return false;
+
        for (i = 0; i < b1->ndatums; i++)
        {
                int                     j;
@@ -635,10 +711,24 @@ check_new_partition_bound(char *relname, Relation parent,
 {
        PartitionKey key = RelationGetPartitionKey(parent);
        PartitionDesc partdesc = RelationGetPartitionDesc(parent);
+       PartitionBoundInfo boundinfo = partdesc->boundinfo;
        ParseState *pstate = make_parsestate(NULL);
        int                     with = -1;
        bool            overlap = false;
 
+       if (spec->is_default)
+       {
+               if (boundinfo == NULL || !partition_bound_has_default(boundinfo))
+                       return;
+
+               /* Default partition already exists, error out. */
+               ereport(ERROR,
+                               (errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
+                                errmsg("partition \"%s\" conflicts with existing default partition \"%s\"",
+                                               relname, get_rel_name(partdesc->oids[boundinfo->default_index])),
+                                parser_errposition(pstate, spec->location)));
+       }
+
        switch (key->strategy)
        {
                case PARTITION_STRATEGY_LIST:
@@ -647,13 +737,13 @@ check_new_partition_bound(char *relname, Relation parent,
 
                                if (partdesc->nparts > 0)
                                {
-                                       PartitionBoundInfo boundinfo = partdesc->boundinfo;
                                        ListCell   *cell;
 
                                        Assert(boundinfo &&
                                                   boundinfo->strategy == PARTITION_STRATEGY_LIST &&
                                                   (boundinfo->ndatums > 0 ||
-                                                       partition_bound_accepts_nulls(boundinfo)));
+                                                       partition_bound_accepts_nulls(boundinfo) ||
+                                                       partition_bound_has_default(boundinfo)));
 
                                        foreach(cell, spec->listdatums)
                                        {
@@ -718,8 +808,10 @@ check_new_partition_bound(char *relname, Relation parent,
                                        int                     offset;
                                        bool            equal;
 
-                                       Assert(boundinfo && boundinfo->ndatums > 0 &&
-                                                  boundinfo->strategy == PARTITION_STRATEGY_RANGE);
+                                       Assert(boundinfo &&
+                                                  boundinfo->strategy == PARTITION_STRATEGY_RANGE &&
+                                                  (boundinfo->ndatums > 0 ||
+                                                       partition_bound_has_default(boundinfo)));
 
                                        /*
                                         * Test whether the new lower bound (which is treated
@@ -796,6 +888,139 @@ check_new_partition_bound(char *relname, Relation parent,
        }
 }
 
+/*
+ * check_default_allows_bound
+ *
+ * This function checks if there exists a row in the default partition that
+ * would properly belong to the new partition being added.  If it finds one,
+ * it throws an error.
+ */
+void
+check_default_allows_bound(Relation parent, Relation default_rel,
+                                                  PartitionBoundSpec *new_spec)
+{
+       List       *new_part_constraints;
+       List       *def_part_constraints;
+       List       *all_parts;
+       ListCell   *lc;
+
+       new_part_constraints = (new_spec->strategy == PARTITION_STRATEGY_LIST)
+               ? get_qual_for_list(parent, new_spec)
+               : get_qual_for_range(parent, new_spec, false);
+       def_part_constraints =
+               get_proposed_default_constraint(new_part_constraints);
+
+       /*
+        * If the existing constraints on the default partition imply that it will
+        * not contain any row that would belong to the new partition, we can
+        * avoid scanning the default partition.
+        */
+       if (PartConstraintImpliedByRelConstraint(default_rel, def_part_constraints))
+       {
+               ereport(INFO,
+                               (errmsg("partition constraint for table \"%s\" is implied by existing constraints",
+                                               RelationGetRelationName(default_rel))));
+               return;
+       }
+
+       /*
+        * Scan the default partition and its subpartitions, and check for rows
+        * that do not satisfy the revised partition constraints.
+        */
+       if (default_rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+               all_parts = find_all_inheritors(RelationGetRelid(default_rel),
+                                                                               AccessExclusiveLock, NULL);
+       else
+               all_parts = list_make1_oid(RelationGetRelid(default_rel));
+
+       foreach(lc, all_parts)
+       {
+               Oid                     part_relid = lfirst_oid(lc);
+               Relation        part_rel;
+               Expr       *constr;
+               Expr       *partition_constraint;
+               EState     *estate;
+               HeapTuple       tuple;
+               ExprState  *partqualstate = NULL;
+               Snapshot        snapshot;
+               TupleDesc       tupdesc;
+               ExprContext *econtext;
+               HeapScanDesc scan;
+               MemoryContext oldCxt;
+               TupleTableSlot *tupslot;
+
+               /* Lock already taken above. */
+               if (part_relid != RelationGetRelid(default_rel))
+                       part_rel = heap_open(part_relid, NoLock);
+               else
+                       part_rel = default_rel;
+
+               /*
+                * Only RELKIND_RELATION relations (i.e. leaf partitions) need to be
+                * scanned.
+                */
+               if (part_rel->rd_rel->relkind != RELKIND_RELATION)
+               {
+                       if (part_rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE)
+                               ereport(WARNING,
+                                               (errcode(ERRCODE_CHECK_VIOLATION),
+                                                errmsg("skipped scanning foreign table \"%s\" which is a partition of default partition \"%s\"",
+                                                               RelationGetRelationName(part_rel),
+                                                               RelationGetRelationName(default_rel))));
+
+                       if (RelationGetRelid(default_rel) != RelationGetRelid(part_rel))
+                               heap_close(part_rel, NoLock);
+
+                       continue;
+               }
+
+               tupdesc = CreateTupleDescCopy(RelationGetDescr(part_rel));
+               constr = linitial(def_part_constraints);
+               partition_constraint = (Expr *)
+                       map_partition_varattnos((List *) constr,
+                                                                       1, part_rel, parent, NULL);
+               estate = CreateExecutorState();
+
+               /* Build expression execution states for partition check quals */
+               partqualstate = ExecPrepareExpr(partition_constraint, estate);
+
+               econtext = GetPerTupleExprContext(estate);
+               snapshot = RegisterSnapshot(GetLatestSnapshot());
+               scan = heap_beginscan(part_rel, snapshot, 0, NULL);
+               tupslot = MakeSingleTupleTableSlot(tupdesc);
+
+               /*
+                * Switch to per-tuple memory context and reset it for each tuple
+                * produced, so we don't leak memory.
+                */
+               oldCxt = MemoryContextSwitchTo(GetPerTupleMemoryContext(estate));
+
+               while ((tuple = heap_getnext(scan, ForwardScanDirection)) != NULL)
+               {
+                       ExecStoreTuple(tuple, tupslot, InvalidBuffer, false);
+                       econtext->ecxt_scantuple = tupslot;
+
+                       if (!ExecCheck(partqualstate, econtext))
+                               ereport(ERROR,
+                                               (errcode(ERRCODE_CHECK_VIOLATION),
+                                                errmsg("updated partition constraint for default partition \"%s\" would be violated by some row",
+                                                               RelationGetRelationName(default_rel))));
+
+                       ResetExprContext(econtext);
+                       CHECK_FOR_INTERRUPTS();
+               }
+
+               MemoryContextSwitchTo(oldCxt);
+               heap_endscan(scan);
+               UnregisterSnapshot(snapshot);
+               ExecDropSingleTupleTableSlot(tupslot);
+               FreeExecutorState(estate);
+
+               if (RelationGetRelid(default_rel) != RelationGetRelid(part_rel))
+                       heap_close(part_rel, NoLock);   /* keep the lock until commit */
+       }
+}
+
 /*
  * get_partition_parent
  *
@@ -860,12 +1085,12 @@ get_qual_from_partbound(Relation rel, Relation parent,
        {
                case PARTITION_STRATEGY_LIST:
                        Assert(spec->strategy == PARTITION_STRATEGY_LIST);
-                       my_qual = get_qual_for_list(key, spec);
+                       my_qual = get_qual_for_list(parent, spec);
                        break;
 
                case PARTITION_STRATEGY_RANGE:
                        Assert(spec->strategy == PARTITION_STRATEGY_RANGE);
-                       my_qual = get_qual_for_range(key, spec);
+                       my_qual = get_qual_for_range(parent, spec, false);
                        break;
 
                default:
@@ -935,7 +1160,8 @@ RelationGetPartitionQual(Relation rel)
  * get_partition_qual_relid
  *
  * Returns an expression tree describing the passed-in relation's partition
- * constraint.
+ * constraint. If there is no partition constraint returns NULL; this can
+ * happen if the default partition is the only partition.
  */
 Expr *
 get_partition_qual_relid(Oid relid)
@@ -948,7 +1174,10 @@ get_partition_qual_relid(Oid relid)
        if (rel->rd_rel->relispartition)
        {
                and_args = generate_partition_qual(rel);
-               if (list_length(and_args) > 1)
+
+               if (and_args == NIL)
+                       result = NULL;
+               else if (list_length(and_args) > 1)
                        result = makeBoolExpr(AND_EXPR, and_args, -1);
                else
                        result = linitial(and_args);
@@ -1263,10 +1492,14 @@ make_partition_op_expr(PartitionKey key, int keynum,
  *
  * Returns an implicit-AND list of expressions to use as a list partition's
  * constraint, given the partition key and bound structures.
+ *
+ * The function returns NIL for a default partition when it's the only
+ * partition since in that case there is no constraint.
  */
 static List *
-get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec)
+get_qual_for_list(Relation parent, PartitionBoundSpec *spec)
 {
+       PartitionKey key = RelationGetPartitionKey(parent);
        List       *result;
        Expr       *keyCol;
        ArrayExpr  *arr;
@@ -1293,15 +1526,63 @@ get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec)
        else
                keyCol = (Expr *) copyObject(linitial(key->partexprs));
 
-       /* Create list of Consts for the allowed values, excluding any nulls */
-       foreach(cell, spec->listdatums)
+       /*
+        * For default list partition, collect datums for all the partitions. The
+        * default partition constraint should check that the partition key is
+        * equal to none of those.
+        */
+       if (spec->is_default)
        {
-               Const      *val = castNode(Const, lfirst(cell));
+               int                     i;
+               int                     ndatums = 0;
+               PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+               PartitionBoundInfo boundinfo = pdesc->boundinfo;
 
-               if (val->constisnull)
-                       list_has_null = true;
-               else
-                       arrelems = lappend(arrelems, copyObject(val));
+               if (boundinfo)
+               {
+                       ndatums = boundinfo->ndatums;
+
+                       if (partition_bound_accepts_nulls(boundinfo))
+                               list_has_null = true;
+               }
+
+               /*
+                * If default is the only partition, there need not be any partition
+                * constraint on it.
+                */
+               if (ndatums == 0 && !list_has_null)
+                       return NIL;
+
+               for (i = 0; i < ndatums; i++)
+               {
+                       Const      *val;
+
+                       /* Construct const from datum */
+                       val = makeConst(key->parttypid[0],
+                                                       key->parttypmod[0],
+                                                       key->parttypcoll[0],
+                                                       key->parttyplen[0],
+                                                       *boundinfo->datums[i],
+                                                       false,  /* isnull */
+                                                       key->parttypbyval[0]);
+
+                       arrelems = lappend(arrelems, val);
+               }
+       }
+       else
+       {
+               /*
+                * Create list of Consts for the allowed values, excluding any nulls.
+                */
+               foreach(cell, spec->listdatums)
+               {
+                       Const      *val = castNode(Const, lfirst(cell));
+
+                       if (val->constisnull)
+                               list_has_null = true;
+                       else
+                               arrelems = lappend(arrelems, copyObject(val));
+               }
        }
 
        if (arrelems)
@@ -1365,6 +1646,18 @@ get_qual_for_list(PartitionKey key, PartitionBoundSpec *spec)
                        result = list_make1(nulltest);
        }
 
+       /*
+        * Note that, in general, applying NOT to a constraint expression doesn't
+        * necessarily invert the set of rows it accepts, because NOT (NULL) is
+        * NULL.  However, the partition constraints we construct here never
+        * evaluate to NULL, so applying NOT works as intended.
+        */
+       if (spec->is_default)
+       {
+               result = list_make1(make_ands_explicit(result));
+               result = list_make1(makeBoolExpr(NOT_EXPR, result, -1));
+       }
+
        return result;
 }
 
@@ -1421,6 +1714,53 @@ get_range_key_properties(PartitionKey key, int keynum,
                *upper_val = NULL;
 }
 
+ /*
+  * get_range_nulltest
+  *
+  * A non-default range partition table does not currently allow partition
+  * keys to be null, so emit an IS NOT NULL expression for each key column.
+  */
+static List *
+get_range_nulltest(PartitionKey key)
+{
+       List       *result = NIL;
+       NullTest   *nulltest;
+       ListCell   *partexprs_item;
+       int                     i;
+
+       partexprs_item = list_head(key->partexprs);
+       for (i = 0; i < key->partnatts; i++)
+       {
+               Expr       *keyCol;
+
+               if (key->partattrs[i] != 0)
+               {
+                       keyCol = (Expr *) makeVar(1,
+                                                                         key->partattrs[i],
+                                                                         key->parttypid[i],
+                                                                         key->parttypmod[i],
+                                                                         key->parttypcoll[i],
+                                                                         0);
+               }
+               else
+               {
+                       if (partexprs_item == NULL)
+                               elog(ERROR, "wrong number of partition key expressions");
+                       keyCol = copyObject(lfirst(partexprs_item));
+                       partexprs_item = lnext(partexprs_item);
+               }
+
+               nulltest = makeNode(NullTest);
+               nulltest->arg = keyCol;
+               nulltest->nulltesttype = IS_NOT_NULL;
+               nulltest->argisrow = false;
+               nulltest->location = -1;
+               result = lappend(result, nulltest);
+       }
+
+       return result;
+}
+
 /*
  * get_qual_for_range
  *
@@ -1459,11 +1799,15 @@ get_range_key_properties(PartitionKey key, int keynum,
  * In most common cases with only one partition column, say a, the following
  * expression tree will be generated: a IS NOT NULL AND a >= al AND a < au
  *
- * If we end up with an empty result list, we return a single-member list
- * containing a constant TRUE, because callers expect a non-empty list.
+ * For default partition, it returns the negation of the constraints of all
+ * the other partitions.
+ *
+ * External callers should pass for_default as false; we set it to true only
+ * when recursing.
  */
 static List *
-get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
+get_qual_for_range(Relation parent, PartitionBoundSpec *spec,
+                                  bool for_default)
 {
        List       *result = NIL;
        ListCell   *cell1,
@@ -1474,10 +1818,10 @@ get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
                                j;
        PartitionRangeDatum *ldatum,
                           *udatum;
+       PartitionKey key = RelationGetPartitionKey(parent);
        Expr       *keyCol;
        Const      *lower_val,
                           *upper_val;
-       NullTest   *nulltest;
        List       *lower_or_arms,
                           *upper_or_arms;
        int                     num_or_arms,
@@ -1487,44 +1831,77 @@ get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
        bool            need_next_lower_arm,
                                need_next_upper_arm;
 
-       lower_or_start_datum = list_head(spec->lowerdatums);
-       upper_or_start_datum = list_head(spec->upperdatums);
-       num_or_arms = key->partnatts;
-
-       /*
-        * A range-partitioned table does not currently allow partition keys to be
-        * null, so emit an IS NOT NULL expression for each key column.
-        */
-       partexprs_item = list_head(key->partexprs);
-       for (i = 0; i < key->partnatts; i++)
+       if (spec->is_default)
        {
-               Expr       *keyCol;
+               List       *or_expr_args = NIL;
+               PartitionDesc pdesc = RelationGetPartitionDesc(parent);
+               Oid                *inhoids = pdesc->oids;
+               int                     nparts = pdesc->nparts,
+                                       i;
 
-               if (key->partattrs[i] != 0)
+               for (i = 0; i < nparts; i++)
                {
-                       keyCol = (Expr *) makeVar(1,
-                                                                         key->partattrs[i],
-                                                                         key->parttypid[i],
-                                                                         key->parttypmod[i],
-                                                                         key->parttypcoll[i],
-                                                                         0);
+                       Oid                     inhrelid = inhoids[i];
+                       HeapTuple       tuple;
+                       Datum           datum;
+                       bool            isnull;
+                       PartitionBoundSpec *bspec;
+
+                       tuple = SearchSysCache1(RELOID, inhrelid);
+                       if (!HeapTupleIsValid(tuple))
+                               elog(ERROR, "cache lookup failed for relation %u", inhrelid);
+
+                       datum = SysCacheGetAttr(RELOID, tuple,
+                                                                       Anum_pg_class_relpartbound,
+                                                                       &isnull);
+
+                       Assert(!isnull);
+                       bspec = (PartitionBoundSpec *)
+                               stringToNode(TextDatumGetCString(datum));
+                       if (!IsA(bspec, PartitionBoundSpec))
+                               elog(ERROR, "expected PartitionBoundSpec");
+
+                       if (!bspec->is_default)
+                       {
+                               List       *part_qual;
+
+                               part_qual = get_qual_for_range(parent, bspec, true);
+
+                               /*
+                                * AND the constraints of the partition and add to
+                                * or_expr_args
+                                */
+                               or_expr_args = lappend(or_expr_args, list_length(part_qual) > 1
+                                                                          ? makeBoolExpr(AND_EXPR, part_qual, -1)
+                                                                          : linitial(part_qual));
+                       }
+                       ReleaseSysCache(tuple);
                }
-               else
+
+               if (or_expr_args != NIL)
                {
-                       if (partexprs_item == NULL)
-                               elog(ERROR, "wrong number of partition key expressions");
-                       keyCol = copyObject(lfirst(partexprs_item));
-                       partexprs_item = lnext(partexprs_item);
+                       /* OR all the non-default partition constraints; then negate it */
+                       result = lappend(result,
+                                                        list_length(or_expr_args) > 1
+                                                        ? makeBoolExpr(OR_EXPR, or_expr_args, -1)
+                                                        : linitial(or_expr_args));
+                       result = list_make1(makeBoolExpr(NOT_EXPR, result, -1));
                }
 
-               nulltest = makeNode(NullTest);
-               nulltest->arg = keyCol;
-               nulltest->nulltesttype = IS_NOT_NULL;
-               nulltest->argisrow = false;
-               nulltest->location = -1;
-               result = lappend(result, nulltest);
+               return result;
        }
 
+       lower_or_start_datum = list_head(spec->lowerdatums);
+       upper_or_start_datum = list_head(spec->upperdatums);
+       num_or_arms = key->partnatts;
+
+       /*
+        * If it is the recursive call for default, we skip the get_range_nulltest
+        * to avoid accumulating the NullTest on the same keys for each partition.
+        */
+       if (!for_default)
+               result = get_range_nulltest(key);
+
        /*
         * Iterate over the key columns and check if the corresponding lower and
         * upper datums are equal using the btree equality operator for the
@@ -1746,9 +2123,16 @@ get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
                                                 ? makeBoolExpr(OR_EXPR, upper_or_arms, -1)
                                                 : linitial(upper_or_arms));
 
-       /* As noted above, caller expects the list to be non-empty. */
+       /*
+        * As noted above, for non-default, we return list with constant TRUE. If
+        * the result is NIL during the recursive call for default, it implies
+        * this is the only other partition which can hold every value of the key
+        * except NULL. Hence we return the NullTest result skipped earlier.
+        */
        if (result == NIL)
-               result = list_make1(makeBoolConst(true, false));
+               result = for_default
+                       ? get_range_nulltest(key)
+                       : list_make1(makeBoolConst(true, false));
 
        return result;
 }
@@ -1756,7 +2140,8 @@ get_qual_for_range(PartitionKey key, PartitionBoundSpec *spec)
 /*
  * generate_partition_qual
  *
- * Generate partition predicate from rel's partition bound expression
+ * Generate partition predicate from rel's partition bound expression. The
+ * function returns a NIL list if there is no predicate.
  *
  * Result expression tree is stored CacheMemoryContext to ensure it survives
  * as long as the relcache entry. But we should be running in a less long-lived
@@ -1932,7 +2317,7 @@ get_partition_for_tuple(PartitionDispatch *pd,
                PartitionDesc partdesc = parent->partdesc;
                TupleTableSlot *myslot = parent->tupslot;
                TupleConversionMap *map = parent->tupmap;
-               int             cur_index = -1;
+               int                     cur_index = -1;
 
                if (myslot != NULL && map != NULL)
                {
@@ -1991,14 +2376,25 @@ get_partition_for_tuple(PartitionDispatch *pd,
 
                        case PARTITION_STRATEGY_RANGE:
                                {
-                                       bool            equal = false;
+                                       bool            equal = false,
+                                                               range_partkey_has_null = false;
                                        int                     cur_offset;
                                        int                     i;
 
-                                       /* No range includes NULL. */
+                                       /*
+                                        * No range includes NULL, so this will be accepted by the
+                                        * default partition if there is one, and otherwise
+                                        * rejected.
+                                        */
                                        for (i = 0; i < key->partnatts; i++)
                                        {
-                                               if (isnull[i])
+                                               if (isnull[i] &&
+                                                       partition_bound_has_default(partdesc->boundinfo))
+                                               {
+                                                       range_partkey_has_null = true;
+                                                       break;
+                                               }
+                                               else if (isnull[i])
                                                {
                                                        *failed_at = parent;
                                                        *failed_slot = slot;
@@ -2007,6 +2403,13 @@ get_partition_for_tuple(PartitionDispatch *pd,
                                                }
                                        }
 
+                                       /*
+                                        * No need to search for partition, as the null key will
+                                        * be routed to the default partition.
+                                        */
+                                       if (range_partkey_has_null)
+                                               break;
+
                                        cur_offset = partition_bound_bsearch(key,
                                                                                                                 partdesc->boundinfo,
                                                                                                                 values,
@@ -2014,9 +2417,9 @@ get_partition_for_tuple(PartitionDispatch *pd,
                                                                                                                 &equal);
 
                                        /*
-                                        * The offset returned is such that the bound at cur_offset
-                                        * is less than or equal to the tuple value, so the bound
-                                        * at offset+1 is the upper bound.
+                                        * The offset returned is such that the bound at
+                                        * cur_offset is less than or equal to the tuple value, so
+                                        * the bound at offset+1 is the upper bound.
                                         */
                                        cur_index = partdesc->boundinfo->indexes[cur_offset + 1];
                                }
@@ -2029,8 +2432,16 @@ get_partition_for_tuple(PartitionDispatch *pd,
 
                /*
                 * cur_index < 0 means we failed to find a partition of this parent.
-                * cur_index >= 0 means we either found the leaf partition, or the
-                * next parent to find a partition of.
+                * Use the default partition, if there is one.
+                */
+               if (cur_index < 0)
+                       cur_index = partdesc->boundinfo->default_index;
+
+               /*
+                * If cur_index is still less than 0 at this point, there's no
+                * partition for this tuple.  Otherwise, we either found the leaf
+                * partition, or a child partitioned table through which we have to
+                * route the tuple.
                 */
                if (cur_index < 0)
                {
@@ -2084,6 +2495,8 @@ make_one_range_bound(PartitionKey key, int index, List *datums, bool lower)
        ListCell   *lc;
        int                     i;
 
+       Assert(datums != NIL);
+
        bound = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
        bound->index = index;
        bound->datums = (Datum *) palloc0(key->partnatts * sizeof(Datum));
@@ -2320,3 +2733,104 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
 
        return lo;
 }
+
+/*
+ * get_default_oid_from_partdesc
+ *
+ * Given a partition descriptor, return the OID of the default partition, if
+ * one exists; else, return InvalidOid.
+ */
+Oid
+get_default_oid_from_partdesc(PartitionDesc partdesc)
+{
+       if (partdesc && partdesc->boundinfo &&
+               partition_bound_has_default(partdesc->boundinfo))
+               return partdesc->oids[partdesc->boundinfo->default_index];
+
+       return InvalidOid;
+}
+
+/*
+ * get_default_partition_oid
+ *
+ * Given a relation OID, return the OID of the default partition, if one
+ * exists.  Use get_default_oid_from_partdesc where possible, for
+ * efficiency.
+ */
+Oid
+get_default_partition_oid(Oid parentId)
+{
+       HeapTuple       tuple;
+       Oid                     defaultPartId = InvalidOid;
+
+       tuple = SearchSysCache1(PARTRELID, ObjectIdGetDatum(parentId));
+
+       if (HeapTupleIsValid(tuple))
+       {
+               Form_pg_partitioned_table part_table_form;
+
+               part_table_form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
+               defaultPartId = part_table_form->partdefid;
+       }
+
+       ReleaseSysCache(tuple);
+       return defaultPartId;
+}
+
+/*
+ * update_default_partition_oid
+ *
+ * Update pg_partition_table.partdefid with a new default partition OID.
+ */
+void
+update_default_partition_oid(Oid parentId, Oid defaultPartId)
+{
+       HeapTuple       tuple;
+       Relation        pg_partitioned_table;
+       Form_pg_partitioned_table part_table_form;
+
+       pg_partitioned_table = heap_open(PartitionedRelationId, RowExclusiveLock);
+
+       tuple = SearchSysCacheCopy1(PARTRELID, ObjectIdGetDatum(parentId));
+
+       if (!HeapTupleIsValid(tuple))
+               elog(ERROR, "cache lookup failed for partition key of relation %u",
+                        parentId);
+
+       part_table_form = (Form_pg_partitioned_table) GETSTRUCT(tuple);
+       part_table_form->partdefid = defaultPartId;
+       CatalogTupleUpdate(pg_partitioned_table, &tuple->t_self, tuple);
+
+       heap_freetuple(tuple);
+       heap_close(pg_partitioned_table, RowExclusiveLock);
+}
+
+/*
+ * get_proposed_default_constraint
+ *
+ * This function returns the negation of new_part_constraints, which
+ * would be an integral part of the default partition constraints after
+ * addition of the partition to which the new_part_constraints belongs.
+ */
+List *
+get_proposed_default_constraint(List *new_part_constraints)
+{
+       Expr       *defPartConstraint;
+
+       defPartConstraint = make_ands_explicit(new_part_constraints);
+
+       /*
+        * Derive the partition constraints of default partition by negating the
+        * given partition constraints. The partition constraint never evaluates
+        * to NULL, so negating it like this is safe.
+        */
+       defPartConstraint = makeBoolExpr(NOT_EXPR,
+                                                                        list_make1(defPartConstraint),
+                                                                        -1);
+       defPartConstraint =
+               (Expr *) eval_const_expressions(NULL,
+                                                                               (Node *) defPartConstraint);
+       defPartConstraint = canonicalize_qual(defPartConstraint);
+
+       return list_make1(defPartConstraint);
+}
index c8fc9cb7fe7832d39e3c1a0234af750d2b3faa69..d2167eda23998bbb28a7518b4afef0611aec7124 100644 (file)
@@ -168,6 +168,8 @@ typedef struct AlteredTableInfo
        bool            chgPersistence; /* T if SET LOGGED/UNLOGGED is used */
        char            newrelpersistence;      /* if above is true */
        Expr       *partition_constraint;       /* for attach partition validation */
+       /* true, if validating default due to some other attach/detach */
+       bool            validate_default;
        /* Objects to rebuild after completing ALTER TYPE operations */
        List       *changedConstraintOids;      /* OIDs of constraints to rebuild */
        List       *changedConstraintDefs;      /* string definitions of same */
@@ -473,11 +475,10 @@ static void CreateInheritance(Relation child_rel, Relation parent_rel);
 static void RemoveInheritance(Relation child_rel, Relation parent_rel);
 static ObjectAddress ATExecAttachPartition(List **wqueue, Relation rel,
                                          PartitionCmd *cmd);
-static bool PartConstraintImpliedByRelConstraint(Relation scanrel,
-                                                                        List *partConstraint);
 static void ValidatePartitionConstraints(List **wqueue, Relation scanrel,
                                                         List *scanrel_children,
-                                                        List *partConstraint);
+                                                        List *partConstraint,
+                                                        bool validate_default);
 static ObjectAddress ATExecDetachPartition(Relation rel, RangeVar *name);
 
 
@@ -774,8 +775,10 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
        {
                PartitionBoundSpec *bound;
                ParseState *pstate;
-               Oid                     parentId = linitial_oid(inheritOids);
-               Relation        parent;
+               Oid                     parentId = linitial_oid(inheritOids),
+                                       defaultPartOid;
+               Relation        parent,
+                                       defaultRel = NULL;
 
                /* Already have strong enough lock on the parent */
                parent = heap_open(parentId, NoLock);
@@ -790,6 +793,30 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
                                         errmsg("\"%s\" is not partitioned",
                                                        RelationGetRelationName(parent))));
 
+               /*
+                * The partition constraint of the default partition depends on the
+                * partition bounds of every other partition. It is possible that
+                * another backend might be about to execute a query on the default
+                * partition table, and that the query relies on previously cached
+                * default partition constraints. We must therefore take a table lock
+                * strong enough to prevent all queries on the default partition from
+                * proceeding until we commit and send out a shared-cache-inval notice
+                * that will make them update their index lists.
+                *
+                * Order of locking: The relation being added won't be visible to
+                * other backends until it is committed, hence here in
+                * DefineRelation() the order of locking the default partition and the
+                * relation being added does not matter. But at all other places we
+                * need to lock the default relation before we lock the relation being
+                * added or removed i.e. we should take the lock in same order at all
+                * the places such that lock parent, lock default partition and then
+                * lock the partition so as to avoid a deadlock.
+                */
+               defaultPartOid =
+                       get_default_oid_from_partdesc(RelationGetPartitionDesc(parent));
+               if (OidIsValid(defaultPartOid))
+                       defaultRel = heap_open(defaultPartOid, AccessExclusiveLock);
+
                /* Tranform the bound values */
                pstate = make_parsestate(NULL);
                pstate->p_sourcetext = queryString;
@@ -798,14 +825,31 @@ DefineRelation(CreateStmt *stmt, char relkind, Oid ownerId,
 
                /*
                 * Check first that the new partition's bound is valid and does not
-                * overlap with any of existing partitions of the parent - note that
-                * it does not return on error.
+                * overlap with any of existing partitions of the parent.
                 */
                check_new_partition_bound(relname, parent, bound);
 
+               /*
+                * If the default partition exists, its partition constraints will
+                * change after the addition of this new partition such that it won't
+                * allow any row that qualifies for this new partition. So, check that
+                * the existing data in the default partition satisfies the constraint
+                * as it will exist after adding this partition.
+                */
+               if (OidIsValid(defaultPartOid))
+               {
+                       check_default_allows_bound(parent, defaultRel, bound);
+                       /* Keep the lock until commit. */
+                       heap_close(defaultRel, NoLock);
+               }
+
                /* Update the pg_class entry. */
                StorePartitionBound(rel, parent, bound);
 
+               /* Update the default partition oid */
+               if (bound->is_default)
+                       update_default_partition_oid(RelationGetRelid(parent), relationId);
+
                heap_close(parent, NoLock);
 
                /*
@@ -4595,9 +4639,16 @@ ATRewriteTable(AlteredTableInfo *tab, Oid OIDNewHeap, LOCKMODE lockmode)
                        }
 
                        if (partqualstate && !ExecCheck(partqualstate, econtext))
-                               ereport(ERROR,
-                                               (errcode(ERRCODE_CHECK_VIOLATION),
-                                                errmsg("partition constraint is violated by some row")));
+                       {
+                               if (tab->validate_default)
+                                       ereport(ERROR,
+                                                       (errcode(ERRCODE_CHECK_VIOLATION),
+                                                        errmsg("updated partition constraint for default partition would be violated by some row")));
+                               else
+                                       ereport(ERROR,
+                                                       (errcode(ERRCODE_CHECK_VIOLATION),
+                                                        errmsg("partition constraint is violated by some row")));
+                       }
 
                        /* Write the tuple out to the new relation */
                        if (newrel)
@@ -13482,7 +13533,7 @@ ComputePartitionAttrs(Relation rel, List *partParams, AttrNumber *partattrs,
  * Existing constraints includes its check constraints and column-level
  * NOT NULL constraints and partConstraint describes the partition constraint.
  */
-static bool
+bool
 PartConstraintImpliedByRelConstraint(Relation scanrel,
                                                                         List *partConstraint)
 {
@@ -13569,7 +13620,8 @@ PartConstraintImpliedByRelConstraint(Relation scanrel,
 static void
 ValidatePartitionConstraints(List **wqueue, Relation scanrel,
                                                         List *scanrel_children,
-                                                        List *partConstraint)
+                                                        List *partConstraint,
+                                                        bool validate_default)
 {
        bool            found_whole_row;
        ListCell   *lc;
@@ -13631,6 +13683,7 @@ ValidatePartitionConstraints(List **wqueue, Relation scanrel,
                /* Grab a work queue entry. */
                tab = ATGetQueueEntry(wqueue, part_rel);
                tab->partition_constraint = (Expr *) linitial(my_partconstr);
+               tab->validate_default = validate_default;
 
                /* keep our lock until commit */
                if (part_rel != scanrel)
@@ -13658,6 +13711,17 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd)
        ObjectAddress address;
        const char *trigger_name;
        bool            found_whole_row;
+       Oid                     defaultPartOid;
+       List       *partBoundConstraint;
+
+       /*
+        * We must lock the default partition, because attaching a new partition
+        * will change its partition constraint.
+        */
+       defaultPartOid =
+               get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
+       if (OidIsValid(defaultPartOid))
+               LockRelationOid(defaultPartOid, AccessExclusiveLock);
 
        attachrel = heap_openrv(cmd->name, AccessExclusiveLock);
 
@@ -13814,6 +13878,11 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd)
        /* OK to create inheritance.  Rest of the checks performed there */
        CreateInheritance(attachrel, rel);
 
+       /* Update the default partition oid */
+       if (cmd->bound->is_default)
+               update_default_partition_oid(RelationGetRelid(rel),
+                                                                        RelationGetRelid(attachrel));
+
        /*
         * Check that the new partition's bound is valid and does not overlap any
         * of existing partitions of the parent - note that it does not return on
@@ -13830,27 +13899,61 @@ ATExecAttachPartition(List **wqueue, Relation rel, PartitionCmd *cmd)
         * If the parent itself is a partition, make sure to include its
         * constraint as well.
         */
-       partConstraint = list_concat(get_qual_from_partbound(attachrel, rel,
-                                                                                                                cmd->bound),
+       partBoundConstraint = get_qual_from_partbound(attachrel, rel, cmd->bound);
+       partConstraint = list_concat(partBoundConstraint,
                                                                 RelationGetPartitionQual(rel));
-       partConstraint = (List *) eval_const_expressions(NULL,
-                                                                                                        (Node *) partConstraint);
-       partConstraint = (List *) canonicalize_qual((Expr *) partConstraint);
-       partConstraint = list_make1(make_ands_explicit(partConstraint));
+
+       /* Skip validation if there are no constraints to validate. */
+       if (partConstraint)
+       {
+               partConstraint =
+                       (List *) eval_const_expressions(NULL,
+                                                                                       (Node *) partConstraint);
+               partConstraint = (List *) canonicalize_qual((Expr *) partConstraint);
+               partConstraint = list_make1(make_ands_explicit(partConstraint));
+
+               /*
+                * Adjust the generated constraint to match this partition's attribute
+                * numbers.
+                */
+               partConstraint = map_partition_varattnos(partConstraint, 1, attachrel,
+                                                                                                rel, &found_whole_row);
+               /* There can never be a whole-row reference here */
+               if (found_whole_row)
+                       elog(ERROR,
+                                "unexpected whole-row reference found in partition key");
+
+               /* Validate partition constraints against the table being attached. */
+               ValidatePartitionConstraints(wqueue, attachrel, attachrel_children,
+                                                                        partConstraint, false);
+       }
 
        /*
-        * Adjust the generated constraint to match this partition's attribute
-        * numbers.
+        * Check whether default partition has a row that would fit the partition
+        * being attached.
         */
-       partConstraint = map_partition_varattnos(partConstraint, 1, attachrel,
-                                                                                        rel, &found_whole_row);
-       /* There can never be a whole-row reference here */
-       if (found_whole_row)
-               elog(ERROR, "unexpected whole-row reference found in partition key");
+       defaultPartOid =
+               get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
+       if (OidIsValid(defaultPartOid))
+       {
+               Relation        defaultrel;
+               List       *defaultrel_children;
+               List       *defPartConstraint;
+
+               /* We already have taken a lock on default partition. */
+               defaultrel = heap_open(defaultPartOid, NoLock);
+               defPartConstraint =
+                       get_proposed_default_constraint(partBoundConstraint);
+               defaultrel_children =
+                       find_all_inheritors(defaultPartOid,
+                                                               AccessExclusiveLock, NULL);
+               ValidatePartitionConstraints(wqueue, defaultrel,
+                                                                        defaultrel_children,
+                                                                        defPartConstraint, true);
 
-       /* Validate partition constraints against the table being attached. */
-       ValidatePartitionConstraints(wqueue, attachrel, attachrel_children,
-                                                                partConstraint);
+               /* keep our lock until commit. */
+               heap_close(defaultrel, NoLock);
+       }
 
        ObjectAddressSet(address, RelationRelationId, RelationGetRelid(attachrel));
 
@@ -13877,6 +13980,16 @@ ATExecDetachPartition(Relation rel, RangeVar *name)
                                new_null[Natts_pg_class],
                                new_repl[Natts_pg_class];
        ObjectAddress address;
+       Oid                     defaultPartOid;
+
+       /*
+        * We must lock the default partition, because detaching this partition
+        * will changing its partition constrant.
+        */
+       defaultPartOid =
+               get_default_oid_from_partdesc(RelationGetPartitionDesc(rel));
+       if (OidIsValid(defaultPartOid))
+               LockRelationOid(defaultPartOid, AccessExclusiveLock);
 
        partRel = heap_openrv(name, AccessShareLock);
 
@@ -13908,6 +14021,24 @@ ATExecDetachPartition(Relation rel, RangeVar *name)
        heap_freetuple(newtuple);
        heap_close(classRel, RowExclusiveLock);
 
+       if (OidIsValid(defaultPartOid))
+       {
+               /*
+                * If the detach relation is the default partition itself, invalidate
+                * its entry in pg_partitioned_table.
+                */
+               if (RelationGetRelid(partRel) == defaultPartOid)
+                       update_default_partition_oid(RelationGetRelid(rel), InvalidOid);
+               else
+               {
+                       /*
+                        * We must invalidate default partition's relcache, for the same
+                        * reasons explained in StorePartitionBound().
+                        */
+                       CacheInvalidateRelcacheByRelid(defaultPartOid);
+               }
+       }
+
        /*
         * Invalidate the parent's relcache so that the partition is no longer
         * included in its partition descriptor.
index 9bae2647fd1c80fb8f1e783f55b0cd18cbb0243a..f1bed14e2bbc47f9ea7313ed9fa3e95ae8aef358 100644 (file)
@@ -4450,6 +4450,7 @@ _copyPartitionBoundSpec(const PartitionBoundSpec *from)
        PartitionBoundSpec *newnode = makeNode(PartitionBoundSpec);
 
        COPY_SCALAR_FIELD(strategy);
+       COPY_SCALAR_FIELD(is_default);
        COPY_NODE_FIELD(listdatums);
        COPY_NODE_FIELD(lowerdatums);
        COPY_NODE_FIELD(upperdatums);
index 11731da80ab3bb9d515a57200e2dcf438d83179d..8b56b9146a1e7692a230d43a909141a6314c9ad2 100644 (file)
@@ -2839,6 +2839,7 @@ static bool
 _equalPartitionBoundSpec(const PartitionBoundSpec *a, const PartitionBoundSpec *b)
 {
        COMPARE_SCALAR_FIELD(strategy);
+       COMPARE_SCALAR_FIELD(is_default);
        COMPARE_NODE_FIELD(listdatums);
        COMPARE_NODE_FIELD(lowerdatums);
        COMPARE_NODE_FIELD(upperdatums);
index 9ee3e23761c832d36840858fc5c925db759f2738..b83d919e408cac5680ddbc4005f135df2732f2c1 100644 (file)
@@ -3573,6 +3573,7 @@ _outPartitionBoundSpec(StringInfo str, const PartitionBoundSpec *node)
        WRITE_NODE_TYPE("PARTITIONBOUNDSPEC");
 
        WRITE_CHAR_FIELD(strategy);
+       WRITE_BOOL_FIELD(is_default);
        WRITE_NODE_FIELD(listdatums);
        WRITE_NODE_FIELD(lowerdatums);
        WRITE_NODE_FIELD(upperdatums);
index 67b9e19d29b4d5c0da9bea2c9ecac72ec8d647a2..fbf8330735835fcbb01c09b6707ebb6202a8118c 100644 (file)
@@ -2390,6 +2390,7 @@ _readPartitionBoundSpec(void)
        READ_LOCALS(PartitionBoundSpec);
 
        READ_CHAR_FIELD(strategy);
+       READ_BOOL_FIELD(is_default);
        READ_NODE_FIELD(listdatums);
        READ_NODE_FIELD(lowerdatums);
        READ_NODE_FIELD(upperdatums);
index 5eb398118e5c050d4b1898e036bbf2e4a75b1116..c303818c9b0cf5d2a7e7be188722de7aca7593dc 100644 (file)
@@ -575,7 +575,7 @@ static Node *makeRecursiveViewSelect(char *relname, List *aliases, Node *query);
 %type <str>                    part_strategy
 %type <partelem>       part_elem
 %type <list>           part_params
-%type <partboundspec> ForValues
+%type <partboundspec> PartitionBoundSpec
 %type <node>           partbound_datum PartitionRangeDatum
 %type <list>           partbound_datum_list range_datum_list
 
@@ -1980,7 +1980,7 @@ alter_table_cmds:
 
 partition_cmd:
                        /* ALTER TABLE <name> ATTACH PARTITION <table_name> FOR VALUES */
-                       ATTACH PARTITION qualified_name ForValues
+                       ATTACH PARTITION qualified_name PartitionBoundSpec
                                {
                                        AlterTableCmd *n = makeNode(AlterTableCmd);
                                        PartitionCmd *cmd = makeNode(PartitionCmd);
@@ -2635,13 +2635,14 @@ alter_identity_column_option:
                                }
                ;
 
-ForValues:
+PartitionBoundSpec:
                        /* a LIST partition */
                        FOR VALUES IN_P '(' partbound_datum_list ')'
                                {
                                        PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
 
                                        n->strategy = PARTITION_STRATEGY_LIST;
+                                       n->is_default = false;
                                        n->listdatums = $5;
                                        n->location = @3;
 
@@ -2654,10 +2655,22 @@ ForValues:
                                        PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
 
                                        n->strategy = PARTITION_STRATEGY_RANGE;
+                                       n->is_default = false;
                                        n->lowerdatums = $5;
                                        n->upperdatums = $9;
                                        n->location = @3;
 
+                                       $$ = n;
+                               }
+
+                       /* a DEFAULT partition */
+                       | DEFAULT
+                               {
+                                       PartitionBoundSpec *n = makeNode(PartitionBoundSpec);
+
+                                       n->is_default = true;
+                                       n->location = @1;
+
                                        $$ = n;
                                }
                ;
@@ -3130,7 +3143,7 @@ CreateStmt:       CREATE OptTemp TABLE qualified_name '(' OptTableElementList ')'
                                        $$ = (Node *)n;
                                }
                | CREATE OptTemp TABLE qualified_name PARTITION OF qualified_name
-                       OptTypedTableElementList ForValues OptPartitionSpec OptWith
+                       OptTypedTableElementList PartitionBoundSpec OptPartitionSpec OptWith
                        OnCommitOption OptTableSpace
                                {
                                        CreateStmt *n = makeNode(CreateStmt);
@@ -3149,7 +3162,7 @@ CreateStmt:       CREATE OptTemp TABLE qualified_name '(' OptTableElementList ')'
                                        $$ = (Node *)n;
                                }
                | CREATE OptTemp TABLE IF_P NOT EXISTS qualified_name PARTITION OF
-                       qualified_name OptTypedTableElementList ForValues OptPartitionSpec
+                       qualified_name OptTypedTableElementList PartitionBoundSpec OptPartitionSpec
                        OptWith OnCommitOption OptTableSpace
                                {
                                        CreateStmt *n = makeNode(CreateStmt);
@@ -4864,7 +4877,7 @@ CreateForeignTableStmt:
                                        $$ = (Node *) n;
                                }
                | CREATE FOREIGN TABLE qualified_name
-                       PARTITION OF qualified_name OptTypedTableElementList ForValues
+                       PARTITION OF qualified_name OptTypedTableElementList PartitionBoundSpec
                        SERVER name create_generic_options
                                {
                                        CreateForeignTableStmt *n = makeNode(CreateForeignTableStmt);
@@ -4885,7 +4898,7 @@ CreateForeignTableStmt:
                                        $$ = (Node *) n;
                                }
                | CREATE FOREIGN TABLE IF_P NOT EXISTS qualified_name
-                       PARTITION OF qualified_name OptTypedTableElementList ForValues
+                       PARTITION OF qualified_name OptTypedTableElementList PartitionBoundSpec
                        SERVER name create_generic_options
                                {
                                        CreateForeignTableStmt *n = makeNode(CreateForeignTableStmt);
index 20586797cc56316d4bfae6500bc77cadcc9a09e8..655da02c10952f36ee7c454b3ee4d961edca7d87 100644 (file)
@@ -3307,6 +3307,18 @@ transformPartitionBound(ParseState *pstate, Relation parent,
        /* Avoid scribbling on input */
        result_spec = copyObject(spec);
 
+       if (spec->is_default)
+       {
+               /*
+                * In case of the default partition, parser had no way to identify the
+                * partition strategy. Assign the parent's strategy to the default
+                * partition bound spec.
+                */
+               result_spec->strategy = strategy;
+
+               return result_spec;
+       }
+
        if (strategy == PARTITION_STRATEGY_LIST)
        {
                ListCell   *cell;
index f9ea7ed771d1e9a98d56e6e65bbb01ed0ab99707..0ea50782181e9cfa44188178c062cb954133e9d4 100644 (file)
@@ -1750,7 +1750,7 @@ pg_get_partition_constraintdef(PG_FUNCTION_ARGS)
 
        constr_expr = get_partition_qual_relid(relationId);
 
-       /* Quick exit if not a partition */
+       /* Quick exit if no partition constraint */
        if (constr_expr == NULL)
                PG_RETURN_NULL();
 
@@ -8699,6 +8699,12 @@ get_rule_expr(Node *node, deparse_context *context,
                                ListCell   *cell;
                                char       *sep;
 
+                               if (spec->is_default)
+                               {
+                                       appendStringInfoString(buf, "DEFAULT");
+                                       break;
+                               }
+
                                switch (spec->strategy)
                                {
                                        case PARTITION_STRATEGY_LIST:
index 6fb9bdd063583fb8b60ad282aeb5256df67942e4..d22ec68431e231d9c781c2256a6030d66e0fd09d 100644 (file)
@@ -1893,19 +1893,20 @@ describeOneTableDetails(const char *schemaname,
                        parent_name = PQgetvalue(result, 0, 0);
                        partdef = PQgetvalue(result, 0, 1);
 
-                       if (PQnfields(result) == 3)
+                       if (PQnfields(result) == 3 && !PQgetisnull(result, 0, 2))
                                partconstraintdef = PQgetvalue(result, 0, 2);
 
                        printfPQExpBuffer(&tmpbuf, _("Partition of: %s %s"), parent_name,
                                                          partdef);
                        printTableAddFooter(&cont, tmpbuf.data);
 
-                       if (partconstraintdef)
-                       {
+                       /* If there isn't any constraint, show that explicitly */
+                       if (partconstraintdef == NULL || partconstraintdef[0] == '\0')
+                               printfPQExpBuffer(&tmpbuf, _("No partition constraint"));
+                       else
                                printfPQExpBuffer(&tmpbuf, _("Partition constraint: %s"),
                                                                  partconstraintdef);
-                               printTableAddFooter(&cont, tmpbuf.data);
-                       }
+                       printTableAddFooter(&cont, tmpbuf.data);
 
                        PQclear(result);
                }
index 2ab8809fa590064f23a3e84ce205fb3dfee0cb97..a09c49d6cf9466ea3617577b23fd0efd743a2c58 100644 (file)
@@ -2053,7 +2053,7 @@ psql_completion(const char *text, int start, int end)
                COMPLETE_WITH_SCHEMA_QUERY(Query_for_list_of_tables, "");
        /* Limited completion support for partition bound specification */
        else if (TailMatches3("ATTACH", "PARTITION", MatchAny))
-               COMPLETE_WITH_CONST("FOR VALUES");
+               COMPLETE_WITH_LIST2("FOR VALUES", "DEFAULT");
        else if (TailMatches2("FOR", "VALUES"))
                COMPLETE_WITH_LIST2("FROM (", "IN (");
 
@@ -2492,7 +2492,7 @@ psql_completion(const char *text, int start, int end)
                COMPLETE_WITH_SCHEMA_QUERY(Query_for_list_of_partitioned_tables, "");
        /* Limited completion support for partition bound specification */
        else if (TailMatches3("PARTITION", "OF", MatchAny))
-               COMPLETE_WITH_CONST("FOR VALUES");
+               COMPLETE_WITH_LIST2("FOR VALUES", "DEFAULT");
 
 /* CREATE TABLESPACE */
        else if (Matches3("CREATE", "TABLESPACE", MatchAny))
index 6525da970d2c1c38d1fb7bdd9c2fcd911293b7e4..56642671b6495fbbd9e5bb99dbde17f1bb07b1d6 100644 (file)
@@ -53,6 +53,6 @@
  */
 
 /*                                                     yyyymmddN */
-#define CATALOG_VERSION_NO     201708311
+#define CATALOG_VERSION_NO     201709081
 
 #endif
index 2283c675e995946abec8c1804cb9392e0ced1aa7..454a940a23bba4762a7f50cded530cc0268f7000 100644 (file)
@@ -99,4 +99,11 @@ extern int get_partition_for_tuple(PartitionDispatch *pd,
                                                EState *estate,
                                                PartitionDispatchData **failed_at,
                                                TupleTableSlot **failed_slot);
+extern Oid     get_default_oid_from_partdesc(PartitionDesc partdesc);
+extern Oid     get_default_partition_oid(Oid parentId);
+extern void update_default_partition_oid(Oid parentId, Oid defaultPartId);
+extern void check_default_allows_bound(Relation parent, Relation defaultRel,
+                                                  PartitionBoundSpec *new_spec);
+extern List *get_proposed_default_constraint(List *new_part_constaints);
+
 #endif                                                 /* PARTITION_H */
index 38d64d6511debcb788b9df4aef91de536c16709e..525e541f930750d3dc6202881a8af4f7f0bc1040 100644 (file)
@@ -32,6 +32,8 @@ CATALOG(pg_partitioned_table,3350) BKI_WITHOUT_OIDS
        Oid                     partrelid;              /* partitioned table oid */
        char            partstrat;              /* partitioning strategy */
        int16           partnatts;              /* number of partition key columns */
+       Oid                     partdefid;              /* default partition oid; InvalidOid if there
+                                                                * isn't one */
 
        /*
         * variable-length fields start here, but we allow direct access to
@@ -62,13 +64,14 @@ typedef FormData_pg_partitioned_table *Form_pg_partitioned_table;
  *             compiler constants for pg_partitioned_table
  * ----------------
  */
-#define Natts_pg_partitioned_table                             7
+#define Natts_pg_partitioned_table                             8
 #define Anum_pg_partitioned_table_partrelid            1
 #define Anum_pg_partitioned_table_partstrat            2
 #define Anum_pg_partitioned_table_partnatts            3
-#define Anum_pg_partitioned_table_partattrs            4
-#define Anum_pg_partitioned_table_partclass            5
-#define Anum_pg_partitioned_table_partcollation 6
-#define Anum_pg_partitioned_table_partexprs            7
+#define Anum_pg_partitioned_table_partdefid            4
+#define Anum_pg_partitioned_table_partattrs            5
+#define Anum_pg_partitioned_table_partclass            6
+#define Anum_pg_partitioned_table_partcollation 7
+#define Anum_pg_partitioned_table_partexprs            8
 
 #endif                                                 /* PG_PARTITIONED_TABLE_H */
index abd31b68d42da42473c28f40e00757b1b51bb982..da3ff5dbeec5073de0280ef386971224ac6afb4c 100644 (file)
@@ -18,6 +18,7 @@
 #include "catalog/dependency.h"
 #include "catalog/objectaddress.h"
 #include "nodes/parsenodes.h"
+#include "catalog/partition.h"
 #include "storage/lock.h"
 #include "utils/relcache.h"
 
@@ -87,4 +88,7 @@ extern void RangeVarCallbackOwnsTable(const RangeVar *relation,
 
 extern void RangeVarCallbackOwnsRelation(const RangeVar *relation,
                                                         Oid relId, Oid oldRelId, void *noCatalogs);
+extern bool PartConstraintImpliedByRelConstraint(Relation scanrel,
+                                                                        List *partConstraint);
+
 #endif                                                 /* TABLECMDS_H */
index 3171815320c4f5c082ce7255acd1c649a2e21852..f3e4c6975395fba22705454d0eacf73b73ca24ad 100644 (file)
@@ -797,6 +797,7 @@ typedef struct PartitionBoundSpec
        NodeTag         type;
 
        char            strategy;               /* see PARTITION_STRATEGY codes above */
+       bool            is_default;             /* is it a default partition bound? */
 
        /* Partitioning info for LIST strategy: */
        List       *listdatums;         /* List of Consts (or A_Consts in raw tree) */
index 0f3642316359844824bc587525c9e0e994a63e99..0d400d97787eb89975dd727db9eded89e6403156 100644 (file)
@@ -3297,6 +3297,14 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
 CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
 ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
 ERROR:  partition "fail_part" would overlap partition "part_1"
+-- check that an existing table can be attached as a default partition
+CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
+ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
+-- check attaching default partition fails if a default partition already
+-- exists
+CREATE TABLE fail_def_part (LIKE part_1 INCLUDING CONSTRAINTS);
+ALTER TABLE list_parted ATTACH PARTITION fail_def_part DEFAULT;
+ERROR:  partition "fail_def_part" conflicts with existing default partition "def_part"
 -- check validation when attaching list partitions
 CREATE TABLE list_parted2 (
        a int,
@@ -3310,6 +3318,15 @@ ERROR:  partition constraint is violated by some row
 -- should be ok after deleting the bad row
 DELETE FROM part_2;
 ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
+-- check partition cannot be attached if default has some row for its values
+CREATE TABLE list_parted2_def PARTITION OF list_parted2 DEFAULT;
+INSERT INTO list_parted2_def VALUES (11, 'z');
+CREATE TABLE part_3 (LIKE list_parted2);
+ALTER TABLE list_parted2 ATTACH PARTITION part_3 FOR VALUES IN (11);
+ERROR:  updated partition constraint for default partition would be violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM list_parted2_def WHERE a = 11;
+ALTER TABLE list_parted2 ATTACH PARTITION part_3 FOR VALUES IN (11);
 -- adding constraints that describe the desired partition constraint
 -- (or more restrictive) will help skip the validation scan
 CREATE TABLE part_3_4 (
@@ -3325,6 +3342,10 @@ ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
 ALTER TABLE part_3_4 ALTER a SET NOT NULL;
 ALTER TABLE list_parted2 ATTACH PARTITION part_3_4 FOR VALUES IN (3, 4);
 INFO:  partition constraint for table "part_3_4" is implied by existing constraints
+-- check if default partition scan skipped
+ALTER TABLE list_parted2_def ADD CONSTRAINT check_a CHECK (a IN (5, 6));
+CREATE TABLE part_55_66 PARTITION OF list_parted2 FOR VALUES IN (55, 66);
+INFO:  partition constraint for table "list_parted2_def" is implied by existing constraints
 -- check validation when attaching range partitions
 CREATE TABLE range_parted (
        a int,
@@ -3350,6 +3371,19 @@ CREATE TABLE part2 (
 );
 ALTER TABLE range_parted ATTACH PARTITION part2 FOR VALUES FROM (1, 10) TO (1, 20);
 INFO:  partition constraint for table "part2" is implied by existing constraints
+-- Create default partition
+CREATE TABLE partr_def1 PARTITION OF range_parted DEFAULT;
+-- Only one default partition is allowed, hence, following should give error
+CREATE TABLE partr_def2 (LIKE part1 INCLUDING CONSTRAINTS);
+ALTER TABLE range_parted ATTACH PARTITION partr_def2 DEFAULT;
+ERROR:  partition "partr_def2" conflicts with existing default partition "partr_def1"
+-- Overlapping partitions cannot be attached, hence, following should give error
+INSERT INTO partr_def1 VALUES (2, 10);
+CREATE TABLE part3 (LIKE range_parted);
+ALTER TABLE range_parted ATTACH partition part3 FOR VALUES FROM (2, 10) TO (2, 20);
+ERROR:  updated partition constraint for default partition would be violated by some row
+-- Attaching partitions should be successful when there are no overlapping rows
+ALTER TABLE range_parted ATTACH partition part3 FOR VALUES FROM (3, 10) TO (3, 20);
 -- check that leaf partitions are scanned when attaching a partitioned
 -- table
 CREATE TABLE part_5 (
@@ -3402,6 +3436,7 @@ ALTER TABLE part_7 ATTACH PARTITION part_7_a_null FOR VALUES IN ('a', null);
 INFO:  partition constraint for table "part_7_a_null" is implied by existing constraints
 ALTER TABLE list_parted2 ATTACH PARTITION part_7 FOR VALUES IN (7);
 INFO:  partition constraint for table "part_7" is implied by existing constraints
+INFO:  partition constraint for table "list_parted2_def" is implied by existing constraints
 -- Same example, but check this time that the constraint correctly detects
 -- violating rows
 ALTER TABLE list_parted2 DETACH PARTITION part_7;
@@ -3415,7 +3450,20 @@ SELECT tableoid::regclass, a, b FROM part_7 order by a;
 (2 rows)
 
 ALTER TABLE list_parted2 ATTACH PARTITION part_7 FOR VALUES IN (7);
+INFO:  partition constraint for table "list_parted2_def" is implied by existing constraints
 ERROR:  partition constraint is violated by some row
+-- check that leaf partitions of default partition are scanned when
+-- attaching a partitioned table.
+ALTER TABLE part_5 DROP CONSTRAINT check_a;
+CREATE TABLE part5_def PARTITION OF part_5 DEFAULT PARTITION BY LIST(a);
+CREATE TABLE part5_def_p1 PARTITION OF part5_def FOR VALUES IN (5);
+INSERT INTO part5_def_p1 VALUES (5, 'y');
+CREATE TABLE part5_p1 (LIKE part_5);
+ALTER TABLE part_5 ATTACH PARTITION part5_p1 FOR VALUES IN ('y');
+ERROR:  updated partition constraint for default partition would be violated by some row
+-- should be ok after deleting the bad row
+DELETE FROM part5_def_p1 WHERE b = 'y';
+ALTER TABLE part_5 ATTACH PARTITION part5_p1 FOR VALUES IN ('y');
 -- check that the table being attached is not already a partition
 ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
 ERROR:  "part_2" is already a partition
@@ -3538,6 +3586,7 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
 ERROR:  cannot alter type of column named in partition key
 -- cleanup
 DROP TABLE list_parted, list_parted2, range_parted;
+DROP TABLE fail_def_part;
 -- more tests for certain multi-level partitioning scenarios
 create table p (a int, b int) partition by range (a, b);
 create table p1 (b int, a int not null) partition by range (b);
index babda8978c741e3a40939e1fff46361ca018f0fd..58c755be505d1157ceb1337afc428ecb64c3bac0 100644 (file)
@@ -467,6 +467,10 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
 ERROR:  invalid bound specification for a list partition
 LINE 1: ...BLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) T...
                                                              ^
+-- check default partition cannot be created more than once
+CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
+CREATE TABLE fail_default_part PARTITION OF list_parted DEFAULT;
+ERROR:  partition "fail_default_part" conflicts with existing default partition "part_default"
 -- specified literal can't be cast to the partition column data type
 CREATE TABLE bools (
        a bool
@@ -558,10 +562,15 @@ CREATE TABLE list_parted2 (
 ) PARTITION BY LIST (a);
 CREATE TABLE part_null_z PARTITION OF list_parted2 FOR VALUES IN (null, 'z');
 CREATE TABLE part_ab PARTITION OF list_parted2 FOR VALUES IN ('a', 'b');
+CREATE TABLE list_parted2_def PARTITION OF list_parted2 DEFAULT;
 CREATE TABLE fail_part PARTITION OF list_parted2 FOR VALUES IN (null);
 ERROR:  partition "fail_part" would overlap partition "part_null_z"
 CREATE TABLE fail_part PARTITION OF list_parted2 FOR VALUES IN ('b', 'c');
 ERROR:  partition "fail_part" would overlap partition "part_ab"
+-- check default partition overlap
+INSERT INTO list_parted2 VALUES('X');
+CREATE TABLE fail_part PARTITION OF list_parted2 FOR VALUES IN ('W', 'X', 'Y');
+ERROR:  updated partition constraint for default partition "list_parted2_def" would be violated by some row
 CREATE TABLE range_parted2 (
        a int
 ) PARTITION BY RANGE (a);
@@ -585,6 +594,16 @@ CREATE TABLE fail_part PARTITION OF range_parted2 FOR VALUES FROM (10) TO (30);
 ERROR:  partition "fail_part" would overlap partition "part2"
 CREATE TABLE fail_part PARTITION OF range_parted2 FOR VALUES FROM (10) TO (50);
 ERROR:  partition "fail_part" would overlap partition "part2"
+-- Create a default partition for range partitioned table
+CREATE TABLE range2_default PARTITION OF range_parted2 DEFAULT;
+-- More than one default partition is not allowed, so this should give error
+CREATE TABLE fail_default_part PARTITION OF range_parted2 DEFAULT;
+ERROR:  partition "fail_default_part" conflicts with existing default partition "range2_default"
+-- Check if the range for default partitions overlap
+INSERT INTO range_parted2 VALUES (85);
+CREATE TABLE fail_part PARTITION OF range_parted2 FOR VALUES FROM (80) TO (90);
+ERROR:  updated partition constraint for default partition "range2_default" would be violated by some row
+CREATE TABLE part4 PARTITION OF range_parted2 FOR VALUES FROM (90) TO (100);
 -- now check for multi-column range partition key
 CREATE TABLE range_parted3 (
        a int,
@@ -598,6 +617,7 @@ CREATE TABLE part11 PARTITION OF range_parted3 FOR VALUES FROM (1, 1) TO (1, 10)
 CREATE TABLE part12 PARTITION OF range_parted3 FOR VALUES FROM (1, 10) TO (1, maxvalue);
 CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, 10) TO (1, 20);
 ERROR:  partition "fail_part" would overlap partition "part12"
+CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
 -- cannot create a partition that says column b is allowed to range
 -- from -infinity to +infinity, while there exist partitions that have
 -- more specific ranges
index e159d62b66f888aae4ab469b078a09deef3b4032..73a5600f19c4859f9d51b12b8d8c0b4637151d90 100644 (file)
@@ -219,17 +219,63 @@ insert into part_null values (null, 0);
 create table part_ee_ff partition of list_parted for values in ('ee', 'ff') partition by range (b);
 create table part_ee_ff1 partition of part_ee_ff for values from (1) to (10);
 create table part_ee_ff2 partition of part_ee_ff for values from (10) to (20);
+-- test default partition
+create table part_default partition of list_parted default;
+-- Negative test: a row, which would fit in other partition, does not fit
+-- default partition, even when inserted directly
+insert into part_default values ('aa', 2);
+ERROR:  new row for relation "part_default" violates partition constraint
+DETAIL:  Failing row contains (aa, 2).
+insert into part_default values (null, 2);
+ERROR:  new row for relation "part_default" violates partition constraint
+DETAIL:  Failing row contains (null, 2).
+-- ok
+insert into part_default values ('Zz', 2);
+-- test if default partition works as expected for multi-level partitioned
+-- table as well as when default partition itself is further partitioned
+drop table part_default;
+create table part_xx_yy partition of list_parted for values in ('xx', 'yy') partition by list (a);
+create table part_xx_yy_p1 partition of part_xx_yy for values in ('xx');
+create table part_xx_yy_defpart partition of part_xx_yy default;
+create table part_default partition of list_parted default partition by range(b);
+create table part_default_p1 partition of part_default for values from (20) to (30);
+create table part_default_p2 partition of part_default for values from (30) to (40);
 -- fail
 insert into part_ee_ff1 values ('EE', 11);
 ERROR:  new row for relation "part_ee_ff1" violates partition constraint
 DETAIL:  Failing row contains (EE, 11).
+insert into part_default_p2 values ('gg', 43);
+ERROR:  new row for relation "part_default_p2" violates partition constraint
+DETAIL:  Failing row contains (gg, 43).
 -- fail (even the parent's, ie, part_ee_ff's partition constraint applies)
 insert into part_ee_ff1 values ('cc', 1);
 ERROR:  new row for relation "part_ee_ff1" violates partition constraint
 DETAIL:  Failing row contains (cc, 1).
+insert into part_default values ('gg', 43);
+ERROR:  no partition of relation "part_default" found for row
+DETAIL:  Partition key of the failing row contains (b) = (43).
 -- ok
 insert into part_ee_ff1 values ('ff', 1);
 insert into part_ee_ff2 values ('ff', 11);
+insert into part_default_p1 values ('cd', 25);
+insert into part_default_p2 values ('de', 35);
+insert into list_parted values ('ab', 21);
+insert into list_parted values ('xx', 1);
+insert into list_parted values ('yy', 2);
+select tableoid::regclass, * from list_parted;
+      tableoid      | a  | b  
+--------------------+----+----
+ part_cc_dd         | cC |  1
+ part_ee_ff1        | ff |  1
+ part_ee_ff2        | ff | 11
+ part_xx_yy_p1      | xx |  1
+ part_xx_yy_defpart | yy |  2
+ part_null          |    |  0
+ part_default_p1    | cd | 25
+ part_default_p1    | ab | 21
+ part_default_p2    | de | 35
+(9 rows)
+
 -- Check tuple routing for partitioned tables
 -- fail
 insert into range_parted values ('a', 0);
@@ -249,6 +295,18 @@ insert into range_parted values ('b', 10);
 insert into range_parted values ('a');
 ERROR:  no partition of relation "range_parted" found for row
 DETAIL:  Partition key of the failing row contains (a, (b + 0)) = (a, null).
+-- Check default partition
+create table part_def partition of range_parted default;
+-- fail
+insert into part_def values ('b', 10);
+ERROR:  new row for relation "part_def" violates partition constraint
+DETAIL:  Failing row contains (b, 10).
+-- ok
+insert into part_def values ('c', 10);
+insert into range_parted values (null, null);
+insert into range_parted values ('a', null);
+insert into range_parted values (null, 19);
+insert into range_parted values ('b', 20);
 select tableoid::regclass, * from range_parted;
  tableoid | a | b  
 ----------+---+----
@@ -258,7 +316,12 @@ select tableoid::regclass, * from range_parted;
  part3    | b |  1
  part4    | b | 10
  part4    | b | 10
-(6 rows)
+ part_def | c | 10
+ part_def |   |   
+ part_def | a |   
+ part_def |   | 19
+ part_def | b | 20
+(11 rows)
 
 -- ok
 insert into list_parted values (null, 1);
@@ -274,17 +337,22 @@ DETAIL:  Partition key of the failing row contains (b) = (0).
 insert into list_parted values ('EE', 1);
 insert into part_ee_ff values ('EE', 10);
 select tableoid::regclass, * from list_parted;
-  tableoid   | a  | b  
--------------+----+----
- part_aa_bb  | aA |   
- part_cc_dd  | cC |  1
- part_ee_ff1 | ff |  1
- part_ee_ff1 | EE |  1
- part_ee_ff2 | ff | 11
- part_ee_ff2 | EE | 10
- part_null   |    |  0
- part_null   |    |  1
-(8 rows)
+      tableoid      | a  | b  
+--------------------+----+----
+ part_aa_bb         | aA |   
+ part_cc_dd         | cC |  1
+ part_ee_ff1        | ff |  1
+ part_ee_ff1        | EE |  1
+ part_ee_ff2        | ff | 11
+ part_ee_ff2        | EE | 10
+ part_xx_yy_p1      | xx |  1
+ part_xx_yy_defpart | yy |  2
+ part_null          |    |  0
+ part_null          |    |  1
+ part_default_p1    | cd | 25
+ part_default_p1    | ab | 21
+ part_default_p2    | de | 35
+(13 rows)
 
 -- some more tests to exercise tuple-routing with multi-level partitioning
 create table part_gg partition of list_parted for values in ('gg') partition by range (b);
@@ -316,6 +384,31 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
 
 -- cleanup
 drop table range_parted, list_parted;
+-- test that a default partition added as the first partition accepts any value
+-- including null
+create table list_parted (a int) partition by list (a);
+create table part_default partition of list_parted default;
+\d+ part_default
+                               Table "public.part_default"
+ Column |  Type   | Collation | Nullable | Default | Storage | Stats target | Description 
+--------+---------+-----------+----------+---------+---------+--------------+-------------
+ a      | integer |           |          |         | plain   |              | 
+Partition of: list_parted DEFAULT
+No partition constraint
+
+insert into part_default values (null);
+insert into part_default values (1);
+insert into part_default values (-1);
+select tableoid::regclass, a from list_parted;
+   tableoid   | a  
+--------------+----
+ part_default |   
+ part_default |  1
+ part_default | -1
+(3 rows)
+
+-- cleanup
+drop table list_parted;
 -- more tests for certain multi-level partitioning scenarios
 create table mlparted (a int, b int) partition by range (a, b);
 create table mlparted1 (b int not null, a int not null) partition by range ((b+0));
@@ -425,6 +518,36 @@ insert into mlparted5 (a, b, c) values (1, 40, 'a');
 ERROR:  new row for relation "mlparted5a" violates partition constraint
 DETAIL:  Failing row contains (b, 1, 40).
 drop table mlparted5;
+alter table mlparted drop constraint check_b;
+-- Check multi-level default partition
+create table mlparted_def partition of mlparted default partition by range(a);
+create table mlparted_def1 partition of mlparted_def for values from (40) to (50);
+create table mlparted_def2 partition of mlparted_def for values from (50) to (60);
+insert into mlparted values (40, 100);
+insert into mlparted_def1 values (42, 100);
+insert into mlparted_def2 values (54, 50);
+-- fail
+insert into mlparted values (70, 100);
+ERROR:  no partition of relation "mlparted_def" found for row
+DETAIL:  Partition key of the failing row contains (a) = (70).
+insert into mlparted_def1 values (52, 50);
+ERROR:  new row for relation "mlparted_def1" violates partition constraint
+DETAIL:  Failing row contains (52, 50, null).
+insert into mlparted_def2 values (34, 50);
+ERROR:  new row for relation "mlparted_def2" violates partition constraint
+DETAIL:  Failing row contains (34, 50, null).
+-- ok
+create table mlparted_defd partition of mlparted_def default;
+insert into mlparted values (70, 100);
+select tableoid::regclass, * from mlparted_def;
+   tableoid    | a  |  b  | c 
+---------------+----+-----+---
+ mlparted_def1 | 40 | 100 | 
+ mlparted_def1 | 42 | 100 | 
+ mlparted_def2 | 54 |  50 | 
+ mlparted_defd | 70 | 100 | 
+(4 rows)
+
 -- check that message shown after failure to find a partition shows the
 -- appropriate key description (or none) in various situations
 create table key_desc (a int, b int) partition by list ((a+0));
index 3f3db337c509570a7f2ab01fb99b82ba2f852aa2..c2eeff161410b4148398112d362694f46f8c286f 100644 (file)
@@ -252,3 +252,29 @@ NOTICE:  3
  
 (1 row)
 
+-- Check that addition or removal of any partition is correctly dealt with by
+-- default partition table when it is being used in prepared statement.
+create table list_parted (a int) partition by list(a);
+create table list_part_null partition of list_parted for values in (null);
+create table list_part_1 partition of list_parted for values in (1);
+create table list_part_def partition of list_parted default;
+prepare pstmt_def_insert (int) as insert into list_part_def values($1);
+-- should fail
+execute pstmt_def_insert(null);
+ERROR:  new row for relation "list_part_def" violates partition constraint
+DETAIL:  Failing row contains (null).
+execute pstmt_def_insert(1);
+ERROR:  new row for relation "list_part_def" violates partition constraint
+DETAIL:  Failing row contains (1).
+create table list_part_2 partition of list_parted for values in (2);
+execute pstmt_def_insert(2);
+ERROR:  new row for relation "list_part_def" violates partition constraint
+DETAIL:  Failing row contains (2).
+alter table list_parted detach partition list_part_null;
+-- should be ok
+execute pstmt_def_insert(null);
+drop table list_part_1;
+-- should be ok
+execute pstmt_def_insert(1);
+drop table list_parted, list_part_null;
+deallocate pstmt_def_insert;
index 6750152e0f4908bcddc8bbe7cbc997ed77606722..e9966405933fa1d372572ee9ffb0ee919cc337f1 100644 (file)
@@ -77,6 +77,10 @@ mlparted12|f
 mlparted2|f
 mlparted3|f
 mlparted4|f
+mlparted_def|f
+mlparted_def1|f
+mlparted_def2|f
+mlparted_defd|f
 money_data|f
 num_data|f
 num_exp_add|t
index 9366f04255cb530de588b5a86f8338161924ca3e..cef70b1a1e1f7df7ab59013ed8f2cd13c9c32a32 100644 (file)
@@ -218,5 +218,38 @@ ERROR:  new row for relation "part_b_10_b_20" violates partition constraint
 DETAIL:  Failing row contains (b, 9).
 -- ok
 update range_parted set b = b + 1 where b = 10;
+-- Creating default partition for range
+create table part_def partition of range_parted default;
+\d+ part_def
+                                  Table "public.part_def"
+ Column |  Type   | Collation | Nullable | Default | Storage  | Stats target | Description 
+--------+---------+-----------+----------+---------+----------+--------------+-------------
+ a      | text    |           |          |         | extended |              | 
+ b      | integer |           |          |         | plain    |              | 
+Partition of: range_parted DEFAULT
+Partition constraint: (NOT (((a = 'a'::text) AND (b >= 1) AND (b < 10)) OR ((a = 'a'::text) AND (b >= 10) AND (b < 20)) OR ((a = 'b'::text) AND (b >= 1) AND (b < 10)) OR ((a = 'b'::text) AND (b >= 10) AND (b < 20))))
+
+insert into range_parted values ('c', 9);
+-- ok
+update part_def set a = 'd' where a = 'c';
+-- fail
+update part_def set a = 'a' where a = 'd';
+ERROR:  new row for relation "part_def" violates partition constraint
+DETAIL:  Failing row contains (a, 9).
+create table list_parted (
+       a text,
+       b int
+) partition by list (a);
+create table list_part1  partition of list_parted for values in ('a', 'b');
+create table list_default partition of list_parted default;
+insert into list_part1 values ('a', 1);
+insert into list_default values ('d', 10);
+-- fail
+update list_default set a = 'a' where a = 'd';
+ERROR:  new row for relation "list_default" violates partition constraint
+DETAIL:  Failing row contains (a, 10).
+-- ok
+update list_default set a = 'x' where a = 'd';
 -- cleanup
 drop table range_parted;
+drop table list_parted;
index e6f6669880b96c0142d22b9d7d8c872d8366be15..37cca72620a75b701f6aff287edc933bff1d4005 100644 (file)
@@ -2111,6 +2111,13 @@ SELECT conislocal, coninhcount FROM pg_constraint WHERE conrelid = 'part_1'::reg
 -- check that the new partition won't overlap with an existing partition
 CREATE TABLE fail_part (LIKE part_1 INCLUDING CONSTRAINTS);
 ALTER TABLE list_parted ATTACH PARTITION fail_part FOR VALUES IN (1);
+-- check that an existing table can be attached as a default partition
+CREATE TABLE def_part (LIKE list_parted INCLUDING CONSTRAINTS);
+ALTER TABLE list_parted ATTACH PARTITION def_part DEFAULT;
+-- check attaching default partition fails if a default partition already
+-- exists
+CREATE TABLE fail_def_part (LIKE part_1 INCLUDING CONSTRAINTS);
+ALTER TABLE list_parted ATTACH PARTITION fail_def_part DEFAULT;
 
 -- check validation when attaching list partitions
 CREATE TABLE list_parted2 (
@@ -2127,6 +2134,15 @@ ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
 DELETE FROM part_2;
 ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
 
+-- check partition cannot be attached if default has some row for its values
+CREATE TABLE list_parted2_def PARTITION OF list_parted2 DEFAULT;
+INSERT INTO list_parted2_def VALUES (11, 'z');
+CREATE TABLE part_3 (LIKE list_parted2);
+ALTER TABLE list_parted2 ATTACH PARTITION part_3 FOR VALUES IN (11);
+-- should be ok after deleting the bad row
+DELETE FROM list_parted2_def WHERE a = 11;
+ALTER TABLE list_parted2 ATTACH PARTITION part_3 FOR VALUES IN (11);
+
 -- adding constraints that describe the desired partition constraint
 -- (or more restrictive) will help skip the validation scan
 CREATE TABLE part_3_4 (
@@ -2144,6 +2160,9 @@ ALTER TABLE list_parted2 DETACH PARTITION part_3_4;
 ALTER TABLE part_3_4 ALTER a SET NOT NULL;
 ALTER TABLE list_parted2 ATTACH PARTITION part_3_4 FOR VALUES IN (3, 4);
 
+-- check if default partition scan skipped
+ALTER TABLE list_parted2_def ADD CONSTRAINT check_a CHECK (a IN (5, 6));
+CREATE TABLE part_55_66 PARTITION OF list_parted2 FOR VALUES IN (55, 66);
 
 -- check validation when attaching range partitions
 CREATE TABLE range_parted (
@@ -2172,6 +2191,21 @@ CREATE TABLE part2 (
 );
 ALTER TABLE range_parted ATTACH PARTITION part2 FOR VALUES FROM (1, 10) TO (1, 20);
 
+-- Create default partition
+CREATE TABLE partr_def1 PARTITION OF range_parted DEFAULT;
+
+-- Only one default partition is allowed, hence, following should give error
+CREATE TABLE partr_def2 (LIKE part1 INCLUDING CONSTRAINTS);
+ALTER TABLE range_parted ATTACH PARTITION partr_def2 DEFAULT;
+
+-- Overlapping partitions cannot be attached, hence, following should give error
+INSERT INTO partr_def1 VALUES (2, 10);
+CREATE TABLE part3 (LIKE range_parted);
+ALTER TABLE range_parted ATTACH partition part3 FOR VALUES FROM (2, 10) TO (2, 20);
+
+-- Attaching partitions should be successful when there are no overlapping rows
+ALTER TABLE range_parted ATTACH partition part3 FOR VALUES FROM (3, 10) TO (3, 20);
+
 -- check that leaf partitions are scanned when attaching a partitioned
 -- table
 CREATE TABLE part_5 (
@@ -2232,6 +2266,18 @@ INSERT INTO part_7 (a, b) VALUES (8, null), (9, 'a');
 SELECT tableoid::regclass, a, b FROM part_7 order by a;
 ALTER TABLE list_parted2 ATTACH PARTITION part_7 FOR VALUES IN (7);
 
+-- check that leaf partitions of default partition are scanned when
+-- attaching a partitioned table.
+ALTER TABLE part_5 DROP CONSTRAINT check_a;
+CREATE TABLE part5_def PARTITION OF part_5 DEFAULT PARTITION BY LIST(a);
+CREATE TABLE part5_def_p1 PARTITION OF part5_def FOR VALUES IN (5);
+INSERT INTO part5_def_p1 VALUES (5, 'y');
+CREATE TABLE part5_p1 (LIKE part_5);
+ALTER TABLE part_5 ATTACH PARTITION part5_p1 FOR VALUES IN ('y');
+-- should be ok after deleting the bad row
+DELETE FROM part5_def_p1 WHERE b = 'y';
+ALTER TABLE part_5 ATTACH PARTITION part5_p1 FOR VALUES IN ('y');
+
 -- check that the table being attached is not already a partition
 ALTER TABLE list_parted2 ATTACH PARTITION part_2 FOR VALUES IN (2);
 
@@ -2327,6 +2373,7 @@ ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
 
 -- cleanup
 DROP TABLE list_parted, list_parted2, range_parted;
+DROP TABLE fail_def_part;
 
 -- more tests for certain multi-level partitioning scenarios
 create table p (a int, b int) partition by range (a, b);
index 1c0ce9276365771232844ff5b001dee92c391a17..eeab5d91ffb44ba670686f9df36819d4660192ae 100644 (file)
@@ -447,6 +447,10 @@ CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES IN ();
 -- trying to specify range for list partitioned table
 CREATE TABLE fail_part PARTITION OF list_parted FOR VALUES FROM (1) TO (2);
 
+-- check default partition cannot be created more than once
+CREATE TABLE part_default PARTITION OF list_parted DEFAULT;
+CREATE TABLE fail_default_part PARTITION OF list_parted DEFAULT;
+
 -- specified literal can't be cast to the partition column data type
 CREATE TABLE bools (
        a bool
@@ -524,9 +528,13 @@ CREATE TABLE list_parted2 (
 ) PARTITION BY LIST (a);
 CREATE TABLE part_null_z PARTITION OF list_parted2 FOR VALUES IN (null, 'z');
 CREATE TABLE part_ab PARTITION OF list_parted2 FOR VALUES IN ('a', 'b');
+CREATE TABLE list_parted2_def PARTITION OF list_parted2 DEFAULT;
 
 CREATE TABLE fail_part PARTITION OF list_parted2 FOR VALUES IN (null);
 CREATE TABLE fail_part PARTITION OF list_parted2 FOR VALUES IN ('b', 'c');
+-- check default partition overlap
+INSERT INTO list_parted2 VALUES('X');
+CREATE TABLE fail_part PARTITION OF list_parted2 FOR VALUES IN ('W', 'X', 'Y');
 
 CREATE TABLE range_parted2 (
        a int
@@ -546,6 +554,17 @@ CREATE TABLE part3 PARTITION OF range_parted2 FOR VALUES FROM (30) TO (40);
 CREATE TABLE fail_part PARTITION OF range_parted2 FOR VALUES FROM (10) TO (30);
 CREATE TABLE fail_part PARTITION OF range_parted2 FOR VALUES FROM (10) TO (50);
 
+-- Create a default partition for range partitioned table
+CREATE TABLE range2_default PARTITION OF range_parted2 DEFAULT;
+
+-- More than one default partition is not allowed, so this should give error
+CREATE TABLE fail_default_part PARTITION OF range_parted2 DEFAULT;
+
+-- Check if the range for default partitions overlap
+INSERT INTO range_parted2 VALUES (85);
+CREATE TABLE fail_part PARTITION OF range_parted2 FOR VALUES FROM (80) TO (90);
+CREATE TABLE part4 PARTITION OF range_parted2 FOR VALUES FROM (90) TO (100);
+
 -- now check for multi-column range partition key
 CREATE TABLE range_parted3 (
        a int,
@@ -559,6 +578,7 @@ CREATE TABLE part10 PARTITION OF range_parted3 FOR VALUES FROM (1, minvalue) TO
 CREATE TABLE part11 PARTITION OF range_parted3 FOR VALUES FROM (1, 1) TO (1, 10);
 CREATE TABLE part12 PARTITION OF range_parted3 FOR VALUES FROM (1, 10) TO (1, maxvalue);
 CREATE TABLE fail_part PARTITION OF range_parted3 FOR VALUES FROM (1, 10) TO (1, 20);
+CREATE TABLE range3_default PARTITION OF range_parted3 DEFAULT;
 
 -- cannot create a partition that says column b is allowed to range
 -- from -infinity to +infinity, while there exist partitions that have
index 6f178720870800c5dae50cc749549ce6649f31db..a2948e4dd0ced3119816e3f13fe707f9fafebba4 100644 (file)
@@ -132,13 +132,39 @@ create table part_ee_ff partition of list_parted for values in ('ee', 'ff') part
 create table part_ee_ff1 partition of part_ee_ff for values from (1) to (10);
 create table part_ee_ff2 partition of part_ee_ff for values from (10) to (20);
 
+-- test default partition
+create table part_default partition of list_parted default;
+-- Negative test: a row, which would fit in other partition, does not fit
+-- default partition, even when inserted directly
+insert into part_default values ('aa', 2);
+insert into part_default values (null, 2);
+-- ok
+insert into part_default values ('Zz', 2);
+-- test if default partition works as expected for multi-level partitioned
+-- table as well as when default partition itself is further partitioned
+drop table part_default;
+create table part_xx_yy partition of list_parted for values in ('xx', 'yy') partition by list (a);
+create table part_xx_yy_p1 partition of part_xx_yy for values in ('xx');
+create table part_xx_yy_defpart partition of part_xx_yy default;
+create table part_default partition of list_parted default partition by range(b);
+create table part_default_p1 partition of part_default for values from (20) to (30);
+create table part_default_p2 partition of part_default for values from (30) to (40);
+
 -- fail
 insert into part_ee_ff1 values ('EE', 11);
+insert into part_default_p2 values ('gg', 43);
 -- fail (even the parent's, ie, part_ee_ff's partition constraint applies)
 insert into part_ee_ff1 values ('cc', 1);
+insert into part_default values ('gg', 43);
 -- ok
 insert into part_ee_ff1 values ('ff', 1);
 insert into part_ee_ff2 values ('ff', 11);
+insert into part_default_p1 values ('cd', 25);
+insert into part_default_p2 values ('de', 35);
+insert into list_parted values ('ab', 21);
+insert into list_parted values ('xx', 1);
+insert into list_parted values ('yy', 2);
+select tableoid::regclass, * from list_parted;
 
 -- Check tuple routing for partitioned tables
 
@@ -154,8 +180,19 @@ insert into range_parted values ('b', 1);
 insert into range_parted values ('b', 10);
 -- fail (partition key (b+0) is null)
 insert into range_parted values ('a');
-select tableoid::regclass, * from range_parted;
 
+-- Check default partition
+create table part_def partition of range_parted default;
+-- fail
+insert into part_def values ('b', 10);
+-- ok
+insert into part_def values ('c', 10);
+insert into range_parted values (null, null);
+insert into range_parted values ('a', null);
+insert into range_parted values (null, 19);
+insert into range_parted values ('b', 20);
+
+select tableoid::regclass, * from range_parted;
 -- ok
 insert into list_parted values (null, 1);
 insert into list_parted (a) values ('aA');
@@ -188,6 +225,18 @@ select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_p
 -- cleanup
 drop table range_parted, list_parted;
 
+-- test that a default partition added as the first partition accepts any value
+-- including null
+create table list_parted (a int) partition by list (a);
+create table part_default partition of list_parted default;
+\d+ part_default
+insert into part_default values (null);
+insert into part_default values (1);
+insert into part_default values (-1);
+select tableoid::regclass, a from list_parted;
+-- cleanup
+drop table list_parted;
+
 -- more tests for certain multi-level partitioning scenarios
 create table mlparted (a int, b int) partition by range (a, b);
 create table mlparted1 (b int not null, a int not null) partition by range ((b+0));
@@ -274,6 +323,24 @@ create function mlparted5abrtrig_func() returns trigger as $$ begin new.c = 'b';
 create trigger mlparted5abrtrig before insert on mlparted5a for each row execute procedure mlparted5abrtrig_func();
 insert into mlparted5 (a, b, c) values (1, 40, 'a');
 drop table mlparted5;
+alter table mlparted drop constraint check_b;
+
+-- Check multi-level default partition
+create table mlparted_def partition of mlparted default partition by range(a);
+create table mlparted_def1 partition of mlparted_def for values from (40) to (50);
+create table mlparted_def2 partition of mlparted_def for values from (50) to (60);
+insert into mlparted values (40, 100);
+insert into mlparted_def1 values (42, 100);
+insert into mlparted_def2 values (54, 50);
+-- fail
+insert into mlparted values (70, 100);
+insert into mlparted_def1 values (52, 50);
+insert into mlparted_def2 values (34, 50);
+-- ok
+create table mlparted_defd partition of mlparted_def default;
+insert into mlparted values (70, 100);
+
+select tableoid::regclass, * from mlparted_def;
 
 -- check that message shown after failure to find a partition shows the
 -- appropriate key description (or none) in various situations
index bc2086166b985d7a3886d2899862443edee928bd..cb2a5514872ed45c7d7ae69ee6f5542af6018fed 100644 (file)
@@ -156,3 +156,24 @@ end$$ language plpgsql;
 
 select cachebug();
 select cachebug();
+
+-- Check that addition or removal of any partition is correctly dealt with by
+-- default partition table when it is being used in prepared statement.
+create table list_parted (a int) partition by list(a);
+create table list_part_null partition of list_parted for values in (null);
+create table list_part_1 partition of list_parted for values in (1);
+create table list_part_def partition of list_parted default;
+prepare pstmt_def_insert (int) as insert into list_part_def values($1);
+-- should fail
+execute pstmt_def_insert(null);
+execute pstmt_def_insert(1);
+create table list_part_2 partition of list_parted for values in (2);
+execute pstmt_def_insert(2);
+alter table list_parted detach partition list_part_null;
+-- should be ok
+execute pstmt_def_insert(null);
+drop table list_part_1;
+-- should be ok
+execute pstmt_def_insert(1);
+drop table list_parted, list_part_null;
+deallocate pstmt_def_insert;
index 663711997b00952e1e9f0f3e7e54ac1d77f100a1..66d1feca10de3bf31c6b31c9baf667c900b986fe 100644 (file)
@@ -125,5 +125,29 @@ update range_parted set b = b - 1 where b = 10;
 -- ok
 update range_parted set b = b + 1 where b = 10;
 
+-- Creating default partition for range
+create table part_def partition of range_parted default;
+\d+ part_def
+insert into range_parted values ('c', 9);
+-- ok
+update part_def set a = 'd' where a = 'c';
+-- fail
+update part_def set a = 'a' where a = 'd';
+
+create table list_parted (
+       a text,
+       b int
+) partition by list (a);
+create table list_part1  partition of list_parted for values in ('a', 'b');
+create table list_default partition of list_parted default;
+insert into list_part1 values ('a', 1);
+insert into list_default values ('d', 10);
+
+-- fail
+update list_default set a = 'a' where a = 'd';
+-- ok
+update list_default set a = 'x' where a = 'd';
+
 -- cleanup
 drop table range_parted;
+drop table list_parted;