*** a/doc/src/sgml/mvcc.sgml
--- b/doc/src/sgml/mvcc.sgml
***************
*** 928,935 **** ERROR:  could not serialize access due to read/write dependencies among transact
          </para>
  
          <para>
!          This lock mode is not automatically acquired on tables by any
!          <productname>PostgreSQL</productname> command.
          </para>
         </listitem>
        </varlistentry>
--- 928,934 ----
          </para>
  
          <para>
!          Acquired by <command>REFRESH MATERIALIZED VIEW CONCURRENTLY</command>.
          </para>
         </listitem>
        </varlistentry>
*** a/doc/src/sgml/ref/refresh_materialized_view.sgml
--- b/doc/src/sgml/ref/refresh_materialized_view.sgml
***************
*** 21,27 **** PostgreSQL documentation
  
   <refsynopsisdiv>
  <synopsis>
! REFRESH MATERIALIZED VIEW <replaceable class="PARAMETER">name</replaceable>
      [ WITH [ NO ] DATA ]
  </synopsis>
   </refsynopsisdiv>
--- 21,27 ----
  
   <refsynopsisdiv>
  <synopsis>
! REFRESH MATERIALIZED VIEW [ CONCURRENTLY ] <replaceable class="PARAMETER">name</replaceable>
      [ WITH [ NO ] DATA ]
  </synopsis>
   </refsynopsisdiv>
***************
*** 38,43 **** REFRESH MATERIALIZED VIEW <replaceable class="PARAMETER">name</replaceable>
--- 38,47 ----
     data is generated and the materialized view is left in an unscannable
     state.
    </para>
+   <para>
+    <literal>CONCURRENTLY</literal> and <literal>WITH NO DATA</literal> may not
+    be specified together.
+   </para>
   </refsect1>
  
   <refsect1>
***************
*** 45,50 **** REFRESH MATERIALIZED VIEW <replaceable class="PARAMETER">name</replaceable>
--- 49,82 ----
  
    <variablelist>
     <varlistentry>
+     <term><literal>CONCURRENTLY</literal></term>
+     <listitem>
+      <para>
+       Refresh the materialized view without locking out concurrent selects on
+       the materialized view.  Without this option a refresh which affects a
+       lot of rows will tend to use fewer resources and complete more quickly,
+       but could block other connections which are trying to read from the
+       materialized view.  This option may be faster in cases where a small
+       number of rows are affected.
+      </para>
+      <para>
+       This option is only allowed if there is at least one
+       <literal>UNIQUE</literal> index on the materialized view which uses only
+       column names and includes all rows; that is, it must not index on any
+       expressions nor include a <literal>WHERE</literal> clause.
+      </para>
+      <para>
+       This option may not be used when the materialized view is not already
+       populated.
+      </para>
+      <para>
+       Even with this option only one <literal>REFRESH</literal> at a time may
+       run against any one materialized view.
+      </para>
+     </listitem>
+    </varlistentry>
+ 
+    <varlistentry>
      <term><replaceable class="PARAMETER">name</replaceable></term>
      <listitem>
       <para>
*** a/src/backend/commands/cluster.c
--- b/src/backend/commands/cluster.c
***************
*** 583,589 **** rebuild_relation(Relation OldHeap, Oid indexOid,
  	heap_close(OldHeap, NoLock);
  
  	/* Create the transient table that will receive the re-ordered data */
! 	OIDNewHeap = make_new_heap(tableOid, tableSpace);
  
  	/* Copy the heap data into the new table in the desired order */
  	copy_heap_data(OIDNewHeap, tableOid, indexOid,
--- 583,590 ----
  	heap_close(OldHeap, NoLock);
  
  	/* Create the transient table that will receive the re-ordered data */
! 	OIDNewHeap = make_new_heap(tableOid, tableSpace, false,
! 							   AccessExclusiveLock);
  
  	/* Copy the heap data into the new table in the desired order */
  	copy_heap_data(OIDNewHeap, tableOid, indexOid,
***************
*** 610,616 **** rebuild_relation(Relation OldHeap, Oid indexOid,
   * data, then call finish_heap_swap to complete the operation.
   */
  Oid
! make_new_heap(Oid OIDOldHeap, Oid NewTableSpace)
  {
  	TupleDesc	OldHeapDesc;
  	char		NewHeapName[NAMEDATALEN];
--- 611,618 ----
   * data, then call finish_heap_swap to complete the operation.
   */
  Oid
! make_new_heap(Oid OIDOldHeap, Oid NewTableSpace, bool forcetemp,
! 			  LOCKMODE lockmode)
  {
  	TupleDesc	OldHeapDesc;
  	char		NewHeapName[NAMEDATALEN];
***************
*** 620,627 **** make_new_heap(Oid OIDOldHeap, Oid NewTableSpace)
  	HeapTuple	tuple;
  	Datum		reloptions;
  	bool		isNull;
  
! 	OldHeap = heap_open(OIDOldHeap, AccessExclusiveLock);
  	OldHeapDesc = RelationGetDescr(OldHeap);
  
  	/*
--- 622,631 ----
  	HeapTuple	tuple;
  	Datum		reloptions;
  	bool		isNull;
+ 	Oid			namespaceid;
+ 	char		relpersistence;
  
! 	OldHeap = heap_open(OIDOldHeap, lockmode);
  	OldHeapDesc = RelationGetDescr(OldHeap);
  
  	/*
***************
*** 642,647 **** make_new_heap(Oid OIDOldHeap, Oid NewTableSpace)
--- 646,662 ----
  	if (isNull)
  		reloptions = (Datum) 0;
  
+ 	if (forcetemp)
+ 	{
+ 		namespaceid = LookupCreationNamespace("pg_temp");
+ 		relpersistence = RELPERSISTENCE_TEMP;
+ 	}
+ 	else
+ 	{
+ 		namespaceid = RelationGetNamespace(OldHeap);
+ 		relpersistence = OldHeap->rd_rel->relpersistence;
+ 	}
+ 
  	/*
  	 * Create the new heap, using a temporary name in the same namespace as
  	 * the existing table.	NOTE: there is some risk of collision with user
***************
*** 657,663 **** make_new_heap(Oid OIDOldHeap, Oid NewTableSpace)
  	snprintf(NewHeapName, sizeof(NewHeapName), "pg_temp_%u", OIDOldHeap);
  
  	OIDNewHeap = heap_create_with_catalog(NewHeapName,
! 										  RelationGetNamespace(OldHeap),
  										  NewTableSpace,
  										  InvalidOid,
  										  InvalidOid,
--- 672,678 ----
  	snprintf(NewHeapName, sizeof(NewHeapName), "pg_temp_%u", OIDOldHeap);
  
  	OIDNewHeap = heap_create_with_catalog(NewHeapName,
! 										  namespaceid,
  										  NewTableSpace,
  										  InvalidOid,
  										  InvalidOid,
***************
*** 665,672 **** make_new_heap(Oid OIDOldHeap, Oid NewTableSpace)
  										  OldHeap->rd_rel->relowner,
  										  OldHeapDesc,
  										  NIL,
! 										  OldHeap->rd_rel->relkind,
! 										  OldHeap->rd_rel->relpersistence,
  										  false,
  										  RelationIsMapped(OldHeap),
  										  true,
--- 680,687 ----
  										  OldHeap->rd_rel->relowner,
  										  OldHeapDesc,
  										  NIL,
! 										  RELKIND_RELATION,
! 										  relpersistence,
  										  false,
  										  RelationIsMapped(OldHeap),
  										  true,
*** a/src/backend/commands/matview.c
--- b/src/backend/commands/matview.c
***************
*** 20,36 ****
--- 20,42 ----
  #include "catalog/catalog.h"
  #include "catalog/indexing.h"
  #include "catalog/namespace.h"
+ #include "catalog/pg_operator.h"
  #include "commands/cluster.h"
  #include "commands/matview.h"
  #include "commands/tablecmds.h"
+ #include "commands/tablespace.h"
  #include "executor/executor.h"
+ #include "executor/spi.h"
  #include "miscadmin.h"
+ #include "parser/parse_relation.h"
  #include "rewrite/rewriteHandler.h"
  #include "storage/smgr.h"
  #include "tcop/tcopprot.h"
+ #include "utils/lsyscache.h"
  #include "utils/rel.h"
  #include "utils/snapmgr.h"
  #include "utils/syscache.h"
+ #include "utils/typcache.h"
  
  
  typedef struct
***************
*** 44,49 **** typedef struct
--- 50,60 ----
  	BulkInsertState bistate;	/* bulk insert state */
  } DR_transientrel;
  
+ #define MAX_QUOTED_NAME_LEN  (NAMEDATALEN*2+3)
+ #define MAX_QUOTED_REL_NAME_LEN  (MAX_QUOTED_NAME_LEN*2)
+ 
+ static int	matview_maintenance_depth = 0;
+ 
  static void transientrel_startup(DestReceiver *self, int operation, TupleDesc typeinfo);
  static void transientrel_receive(TupleTableSlot *slot, DestReceiver *self);
  static void transientrel_shutdown(DestReceiver *self);
***************
*** 51,56 **** static void transientrel_destroy(DestReceiver *self);
--- 62,77 ----
  static void refresh_matview_datafill(DestReceiver *dest, Query *query,
  						 const char *queryString);
  
+ static void quoteOneName(char *buffer, const char *name);
+ static void quoteRelationName(char *buffer, Relation rel);
+ static void mv_GenerateOper(StringInfo buf, Oid opoid);
+ 
+ static void refresh_by_match_merge(Oid matviewOid, Oid tempOid);
+ static void refresh_by_heap_swap(Oid matviewOid, Oid OIDNewHeap);
+ 
+ static void OpenMatViewIncrementalMaintenance(void);
+ static void CloseMatViewIncrementalMaintenance(void);
+ 
  /*
   * SetMatViewPopulatedState
   *		Mark a materialized view as populated, or not.
***************
*** 125,136 **** ExecRefreshMatView(RefreshMatViewStmt *stmt, const char *queryString,
  	Oid			tableSpace;
  	Oid			OIDNewHeap;
  	DestReceiver *dest;
  
  	/*
  	 * Get a lock until end of transaction.
  	 */
  	matviewOid = RangeVarGetRelidExtended(stmt->relation,
! 										  AccessExclusiveLock, false, false,
  										  RangeVarCallbackOwnsTable, NULL);
  	matviewRel = heap_open(matviewOid, NoLock);
  
--- 146,163 ----
  	Oid			tableSpace;
  	Oid			OIDNewHeap;
  	DestReceiver *dest;
+ 	bool		concurrent;
+ 	LOCKMODE	lockmode;
+ 
+ 	/* Determine strength of lock needed. */
+ 	concurrent = stmt->concurrent;
+ 	lockmode = concurrent ? ExclusiveLock : AccessExclusiveLock;
  
  	/*
  	 * Get a lock until end of transaction.
  	 */
  	matviewOid = RangeVarGetRelidExtended(stmt->relation,
! 										  lockmode, false, false,
  										  RangeVarCallbackOwnsTable, NULL);
  	matviewRel = heap_open(matviewOid, NoLock);
  
***************
*** 141,151 **** ExecRefreshMatView(RefreshMatViewStmt *stmt, const char *queryString,
  				 errmsg("\"%s\" is not a materialized view",
  						RelationGetRelationName(matviewRel))));
  
! 	/*
! 	 * We're not using materialized views in the system catalogs.
! 	 */
  	Assert(!IsSystemRelation(matviewRel));
  
  	Assert(!matviewRel->rd_rel->relhasoids);
  
  	/*
--- 168,189 ----
  				 errmsg("\"%s\" is not a materialized view",
  						RelationGetRelationName(matviewRel))));
  
! 	/* Check that CONCURRENTLY is not specified if not populated. */
! 	if (concurrent && !RelationIsPopulated(matviewRel))
! 		ereport(ERROR,
! 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
! 				 errmsg("CONCURRENTLY cannot be used when the materialized view is not populated")));
! 
! 	/* Check that conflicting options have not been specified. */
! 	if (concurrent && stmt->skipData)
! 		ereport(ERROR,
! 				(errcode(ERRCODE_SYNTAX_ERROR),
! 				 errmsg("CONCURRENTLY and WITH NO DATA options cannot be used together")));
! 
! 	/* We're not using materialized views in the system catalogs. */
  	Assert(!IsSystemRelation(matviewRel));
  
+ 	/* We don't allow an oid column for a materialized view. */
  	Assert(!matviewRel->rd_rel->relhasoids);
  
  	/*
***************
*** 197,222 **** ExecRefreshMatView(RefreshMatViewStmt *stmt, const char *queryString,
  	 */
  	SetMatViewPopulatedState(matviewRel, !stmt->skipData);
  
! 	tableSpace = matviewRel->rd_rel->reltablespace;
  
  	heap_close(matviewRel, NoLock);
  
  	/* Create the transient table that will receive the regenerated data. */
! 	OIDNewHeap = make_new_heap(matviewOid, tableSpace);
  	dest = CreateTransientRelDestReceiver(OIDNewHeap);
  
  	/* Generate the data, if wanted. */
  	if (!stmt->skipData)
  		refresh_matview_datafill(dest, dataQuery, queryString);
  
! 	/*
! 	 * Swap the physical files of the target and transient tables, then
! 	 * rebuild the target's indexes and throw away the transient table.
! 	 */
! 	finish_heap_swap(matviewOid, OIDNewHeap, false, false, true, true,
! 					 RecentXmin, ReadNextMultiXactId());
! 
! 	RelationCacheInvalidateEntry(matviewOid);
  }
  
  /*
--- 235,262 ----
  	 */
  	SetMatViewPopulatedState(matviewRel, !stmt->skipData);
  
! 	/* Concurrent refresh builds new data in temp tablespace, and does diff. */
! 	if (concurrent)
! 		tableSpace = GetDefaultTablespace(RELPERSISTENCE_TEMP);
! 	else
! 		tableSpace = matviewRel->rd_rel->reltablespace;
  
  	heap_close(matviewRel, NoLock);
  
  	/* Create the transient table that will receive the regenerated data. */
! 	OIDNewHeap = make_new_heap(matviewOid, tableSpace, concurrent,
! 							   ExclusiveLock);
  	dest = CreateTransientRelDestReceiver(OIDNewHeap);
  
  	/* Generate the data, if wanted. */
  	if (!stmt->skipData)
  		refresh_matview_datafill(dest, dataQuery, queryString);
  
! 	/* Make the matview match the newly generated data. */
! 	if (concurrent)
! 		refresh_by_match_merge(matviewOid, OIDNewHeap);
! 	else
! 		refresh_by_heap_swap(matviewOid, OIDNewHeap);
  }
  
  /*
***************
*** 369,371 **** transientrel_destroy(DestReceiver *self)
--- 409,778 ----
  {
  	pfree(self);
  }
+ 
+ 
+ /*
+  * quoteOneName --- safely quote a single SQL name
+  *
+  * buffer must be MAX_QUOTED_NAME_LEN long (includes room for \0)
+  */
+ static void
+ quoteOneName(char *buffer, const char *name)
+ {
+ 	/* Rather than trying to be smart, just always quote it. */
+ 	*buffer++ = '"';
+ 	while (*name)
+ 	{
+ 		if (*name == '"')
+ 			*buffer++ = '"';
+ 		*buffer++ = *name++;
+ 	}
+ 	*buffer++ = '"';
+ 	*buffer = '\0';
+ }
+ 
+ /*
+  * quoteRelationName --- safely quote a fully qualified relation name
+  *
+  * buffer must be MAX_QUOTED_REL_NAME_LEN long (includes room for \0)
+  */
+ static void
+ quoteRelationName(char *buffer, Relation rel)
+ {
+ 	quoteOneName(buffer, get_namespace_name(RelationGetNamespace(rel)));
+ 	buffer += strlen(buffer);
+ 	*buffer++ = '.';
+ 	quoteOneName(buffer, RelationGetRelationName(rel));
+ }
+ 
+ static void
+ mv_GenerateOper(StringInfo buf, Oid opoid)
+ {
+ 	HeapTuple	opertup;
+ 	Form_pg_operator operform;
+ 	char		nspname[MAX_QUOTED_NAME_LEN];
+ 
+ 	opertup = SearchSysCache1(OPEROID, ObjectIdGetDatum(opoid));
+ 	if (!HeapTupleIsValid(opertup))
+ 		elog(ERROR, "cache lookup failed for operator %u", opoid);
+ 	operform = (Form_pg_operator) GETSTRUCT(opertup);
+ 	Assert(operform->oprkind == 'b');
+ 
+ 	quoteOneName(nspname, get_namespace_name(operform->oprnamespace));
+ 	appendStringInfo(buf, "OPERATOR(%s.%s)",
+ 					 nspname, NameStr(operform->oprname));
+ 
+ 	ReleaseSysCache(opertup);
+ }
+ 
+ /*
+  * refresh_by_match_merge
+  *
+  * Refresh a materialized view with transactional semantics, while allowing
+  * concurrent reads.
+  *
+  * This is called after a new version of the data has been created in a
+  * temporary table.  It performs a full outer join against the old version of
+  * the data, producing "diff" results.	This join cannot work if there are any
+  * duplicated rows in either the old or new versions, in the sense that every
+  * column would compare as equal between the two rows.	It does work correctly
+  * in the face of rows which have at least one NULL value, with all non-NULL
+  * columns equal.  The behavior of NULLs on equality tests and on UNIQUE
+  * indexes turns out to be quite convenient here; the tests we need to make
+  * are consistent with default behavior.  If there is at least one UNIQUE
+  * index on the materialized view, we have exactly the guarantee we need.  By
+  * joining based on equality on all columns which are part of any unique
+  * index, we identify the rows on which we can use UPDATE without any problem.
+  * If any column is NULL in either the old or new version of a row (or both),
+  * we must use DELETE and INSERT, since there could be multiple rows which are
+  * NOT DISTINCT FROM each other, and we could otherwise end up with the wrong
+  * number of occurrences in the updated relation.  The temporary table used to
+  * hold the diff results contains just the TID of the old record (if matched)
+  * and the ROW from the new table as a single column of complex record type
+  * (if matched).
+  *
+  * Once we have the diff table, we perform set-based DELETE, UPDATE, and
+  * INSERT operations against the materialized view, and discard both temporary
+  * tables.
+  *
+  * Everything from the generation of the new data to applying the differences
+  * takes place under cover of an ExclusiveLock, since it seems as though we
+  * would want to prohibit not only concurrent REFRESH operations, but also
+  * incremental maintenance.  It also doesn't seem reasonable or safe to allow
+  * SELECT FOR UPDATE or SELECT FOR SHARE on rows being updated or deleted by
+  * this command.
+  */
+ static void
+ refresh_by_match_merge(Oid matviewOid, Oid tempOid)
+ {
+ 	StringInfoData querybuf;
+ 	Relation	matviewRel;
+ 	Relation	tempRel;
+ 	char		matviewname[MAX_QUOTED_REL_NAME_LEN];
+ 	char		tempname[MAX_QUOTED_REL_NAME_LEN];
+ 	char		diffname[MAX_QUOTED_REL_NAME_LEN];
+ 	TupleDesc	tupdesc;
+ 	bool		foundUniqueIndex;
+ 	List	   *indexoidlist;
+ 	ListCell   *indexoidscan;
+ 	int16		relnatts;
+ 	bool	   *usedForQual;
+ 
+ 	initStringInfo(&querybuf);
+ 	matviewRel = heap_open(matviewOid, NoLock);
+ 	quoteRelationName(matviewname, matviewRel);
+ 	tempRel = heap_open(tempOid, NoLock);
+ 	quoteRelationName(tempname, tempRel);
+ 	strcpy(diffname, tempname);
+ 	strcpy(diffname + strlen(diffname) - 1, "_2\"");
+ 
+ 	relnatts = matviewRel->rd_rel->relnatts;
+ 	usedForQual = (bool *) palloc0(sizeof(bool) * relnatts);
+ 
+ 	/* Open SPI context. */
+ 	if (SPI_connect() != SPI_OK_CONNECT)
+ 		elog(ERROR, "SPI_connect failed");
+ 
+ 	/* Analyze the temp table with the new contents. */
+ 	appendStringInfo(&querybuf, "ANALYZE %s", tempname);
+ 	if (SPI_exec(querybuf.data, 0) != SPI_OK_UTILITY)
+ 		elog(ERROR, "SPI_exec failed: %s", querybuf.data);
+ 
+ 	/* Start building the query for creating the diff table. */
+ 	resetStringInfo(&querybuf);
+ 	appendStringInfo(&querybuf,
+ 					 "CREATE TEMP TABLE %s AS "
+ 					 "SELECT x.ctid AS tid, y FROM %s x FULL JOIN %s y ON (",
+ 					 diffname, matviewname, tempname);
+ 
+ 	/*
+ 	 * Get the list of index OIDs for the table from the relcache, and look up
+ 	 * each one in the pg_index syscache.  We will test for equality on all
+ 	 * columns present in all unique indexes which only reference columns and
+ 	 * include all rows.
+ 	 */
+ 	tupdesc = matviewRel->rd_att;
+ 	foundUniqueIndex = false;
+ 	indexoidlist = RelationGetIndexList(matviewRel);
+ 
+ 	foreach(indexoidscan, indexoidlist)
+ 	{
+ 		Oid			indexoid = lfirst_oid(indexoidscan);
+ 		HeapTuple	indexTuple;
+ 		Form_pg_index index;
+ 
+ 		indexTuple = SearchSysCache1(INDEXRELID, ObjectIdGetDatum(indexoid));
+ 		if (!HeapTupleIsValid(indexTuple))		/* should not happen */
+ 			elog(ERROR, "cache lookup failed for index %u", indexoid);
+ 		index = (Form_pg_index) GETSTRUCT(indexTuple);
+ 
+ 		/* We're only interested if it is unique and valid. */
+ 		if (index->indisunique && IndexIsValid(index))
+ 		{
+ 			int			numatts = index->indnatts;
+ 			int			i;
+ 			bool		expr = false;
+ 			Relation	indexRel;
+ 
+ 			/* Skip any index on an expression. */
+ 			for (i = 0; i < numatts; i++)
+ 			{
+ 				if (index->indkey.values[i] == 0)
+ 				{
+ 					expr = true;
+ 					break;
+ 				}
+ 			}
+ 			if (expr)
+ 			{
+ 				ReleaseSysCache(indexTuple);
+ 				continue;
+ 			}
+ 
+ 			/*
+ 			 * Skip partial indexes.  We count on the ExclusiveLock on the
+ 			 * heap to keep things stable while we check this.
+ 			 */
+ 			indexRel = index_open(index->indexrelid, NoLock);
+ 			if (indexRel->rd_indpred != NIL)
+ 			{
+ 				index_close(indexRel, NoLock);
+ 				ReleaseSysCache(indexTuple);
+ 				continue;
+ 			}
+ 			index_close(indexRel, NoLock);
+ 
+ 			/* Add quals for all columns from this index. */
+ 			for (i = 0; i < numatts; i++)
+ 			{
+ 				int			attnum = index->indkey.values[i];
+ 				Oid			type;
+ 				Oid			op;
+ 				char		colname[MAX_QUOTED_NAME_LEN];
+ 
+ 				/*
+ 				 * Only include the column once regardless of how many times
+ 				 * it shows up in how many indexes.
+ 				 *
+ 				 * This is also useful later to omit columns which can not
+ 				 * have changed from the SET clause of the UPDATE statement.
+ 				 */
+ 				if (usedForQual[attnum - 1])
+ 					continue;
+ 				usedForQual[attnum - 1] = true;
+ 
+ 				/*
+ 				 * Actually add the qual, ANDed with any others.
+ 				 */
+ 				if (foundUniqueIndex)
+ 					appendStringInfoString(&querybuf, " AND ");
+ 
+ 				quoteOneName(colname,
+ 							 NameStr((tupdesc->attrs[attnum - 1])->attname));
+ 				appendStringInfo(&querybuf, "y.%s ", colname);
+ 				type = attnumTypeId(matviewRel, attnum);
+ 				op = lookup_type_cache(type, TYPECACHE_EQ_OPR)->eq_opr;
+ 				mv_GenerateOper(&querybuf, op);
+ 				appendStringInfo(&querybuf, " x.%s", colname);
+ 
+ 				foundUniqueIndex = true;
+ 			}
+ 		}
+ 		ReleaseSysCache(indexTuple);
+ 	}
+ 
+ 	list_free(indexoidlist);
+ 
+ 	if (!foundUniqueIndex)
+ 		ereport(ERROR,
+ 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ 				 errmsg("concurrent refresh requires a unique index on just columns for all rows of the materialized view")));
+ 
+ 	appendStringInfoString(&querybuf,
+ 						   ") WHERE (y.*) IS DISTINCT FROM (x.*)"
+ 						   " ORDER BY tid");
+ 
+ 	/* Create the temporary "diff" table. */
+ 	if (SPI_exec(querybuf.data, 0) != SPI_OK_UTILITY)
+ 		elog(ERROR, "SPI_exec failed: %s", querybuf.data);
+ 
+ 	/*
+ 	 * We have no further use for data from the "full-data" temp table, but we
+ 	 * must keep it around because its type is reference from the diff table.
+ 	 */
+ 
+ 	/* Analyze the diff table. */
+ 	resetStringInfo(&querybuf);
+ 	appendStringInfo(&querybuf, "ANALYZE %s", diffname);
+ 	if (SPI_exec(querybuf.data, 0) != SPI_OK_UTILITY)
+ 		elog(ERROR, "SPI_exec failed: %s", querybuf.data);
+ 
+ 	OpenMatViewIncrementalMaintenance();
+ 
+ 	/* Deletes must come before inserts; do them first. */
+ 	resetStringInfo(&querybuf);
+ 	appendStringInfo(&querybuf,
+ 					 "DELETE FROM %s WHERE ctid IN "
+ 					 "(SELECT d.tid FROM %s d "
+ 					 "WHERE d.tid IS NOT NULL "
+ 					 "AND (d.y) IS NOT DISTINCT FROM NULL)",
+ 					 matviewname, diffname);
+ 	if (SPI_exec(querybuf.data, 0) != SPI_OK_DELETE)
+ 		elog(ERROR, "SPI_exec failed: %s", querybuf.data);
+ 
+ 	/* Updates before inserts gives a better chance at HOT updates. */
+ 	resetStringInfo(&querybuf);
+ 	appendStringInfo(&querybuf, "UPDATE %s x SET ", matviewname);
+ 
+ 	{
+ 		int			i;
+ 		bool		needComma = false;
+ 
+ 		for (i = 0; i < tupdesc->natts; i++)
+ 		{
+ 			char		colname[MAX_QUOTED_NAME_LEN];
+ 
+ 			if (tupdesc->attrs[i]->attisdropped)
+ 				continue;
+ 
+ 			if (usedForQual[i])
+ 				continue;
+ 
+ 			if (needComma)
+ 				appendStringInfoString(&querybuf, ", ");
+ 			needComma = true;
+ 
+ 			quoteOneName(colname,
+ 						 NameStr((tupdesc->attrs[i])->attname));
+ 			appendStringInfo(&querybuf, "%s = (d.y).%s", colname, colname);
+ 		}
+ 	}
+ 
+ 	appendStringInfo(&querybuf,
+ 					 " FROM %s d WHERE d.tid IS NOT NULL AND x.ctid = d.tid",
+ 					 diffname);
+ 
+ 	if (SPI_exec(querybuf.data, 0) != SPI_OK_UPDATE)
+ 		elog(ERROR, "SPI_exec failed: %s", querybuf.data);
+ 
+ 	/* Inserts go last. */
+ 	resetStringInfo(&querybuf);
+ 	appendStringInfo(&querybuf,
+ 					 "INSERT INTO %s SELECT (y).* FROM %s WHERE tid IS NULL",
+ 					 matviewname, diffname);
+ 	if (SPI_exec(querybuf.data, 0) != SPI_OK_INSERT)
+ 		elog(ERROR, "SPI_exec failed: %s", querybuf.data);
+ 
+ 	/* We're done maintaining the materialized view. */
+ 	CloseMatViewIncrementalMaintenance();
+ 	heap_close(tempRel, NoLock);
+ 	heap_close(matviewRel, NoLock);
+ 
+ 	/* Clean up temp tables. */
+ 	resetStringInfo(&querybuf);
+ 	appendStringInfo(&querybuf, "DROP TABLE %s, %s", diffname, tempname);
+ 	if (SPI_exec(querybuf.data, 0) != SPI_OK_UTILITY)
+ 		elog(ERROR, "SPI_exec failed: %s", querybuf.data);
+ 
+ 	/* Close SPI context. */
+ 	if (SPI_finish() != SPI_OK_FINISH)
+ 		elog(ERROR, "SPI_finish failed");
+ }
+ 
+ /*
+  * Swap the physical files of the target and transient tables, then rebuild
+  * the target's indexes and throw away the transient table.
+  */
+ static void
+ refresh_by_heap_swap(Oid matviewOid, Oid OIDNewHeap)
+ {
+ 	finish_heap_swap(matviewOid, OIDNewHeap, false, false, true, true,
+ 					 RecentXmin, ReadNextMultiXactId());
+ 
+ 	RelationCacheInvalidateEntry(matviewOid);
+ }
+ 
+ static void
+ OpenMatViewIncrementalMaintenance(void)
+ {
+ 	matview_maintenance_depth++;
+ }
+ 
+ static void
+ CloseMatViewIncrementalMaintenance(void)
+ {
+ 	matview_maintenance_depth--;
+ 	Assert(matview_maintenance_depth >= 0);
+ }
+ 
+ /*
+  * This should be used to test whether the backend is in a context where it is
+  * OK to allow DML statements to modify materialized views.  We only want to
+  * allow that for internal code driven by the materialized view definition,
+  * not for arbitrary user-supplied code.
+  */
+ bool
+ MatViewIncrementalMaintenanceIsEnabled(void)
+ {
+ 	return matview_maintenance_depth > 0;
+ }
*** a/src/backend/commands/tablecmds.c
--- b/src/backend/commands/tablecmds.c
***************
*** 3529,3535 **** ATRewriteTables(List **wqueue, LOCKMODE lockmode)
  			heap_close(OldHeap, NoLock);
  
  			/* Create transient table that will receive the modified data */
! 			OIDNewHeap = make_new_heap(tab->relid, NewTableSpace);
  
  			/*
  			 * Copy the heap data into the new table with the desired
--- 3529,3536 ----
  			heap_close(OldHeap, NoLock);
  
  			/* Create transient table that will receive the modified data */
! 			OIDNewHeap = make_new_heap(tab->relid, NewTableSpace, false,
! 									   AccessExclusiveLock);
  
  			/*
  			 * Copy the heap data into the new table with the desired
*** a/src/backend/executor/execMain.c
--- b/src/backend/executor/execMain.c
***************
*** 42,47 ****
--- 42,48 ----
  #include "access/transam.h"
  #include "access/xact.h"
  #include "catalog/namespace.h"
+ #include "commands/matview.h"
  #include "commands/trigger.h"
  #include "executor/execdebug.h"
  #include "foreign/fdwapi.h"
***************
*** 999,1008 **** CheckValidResultRel(Relation resultRel, CmdType operation)
  			}
  			break;
  		case RELKIND_MATVIEW:
! 			ereport(ERROR,
! 					(errcode(ERRCODE_WRONG_OBJECT_TYPE),
! 					 errmsg("cannot change materialized view \"%s\"",
! 							RelationGetRelationName(resultRel))));
  			break;
  		case RELKIND_FOREIGN_TABLE:
  			/* Okay only if the FDW supports it */
--- 1000,1010 ----
  			}
  			break;
  		case RELKIND_MATVIEW:
! 			if (!MatViewIncrementalMaintenanceIsEnabled())
! 				ereport(ERROR,
! 						(errcode(ERRCODE_WRONG_OBJECT_TYPE),
! 						 errmsg("cannot change materialized view \"%s\"",
! 								RelationGetRelationName(resultRel))));
  			break;
  		case RELKIND_FOREIGN_TABLE:
  			/* Okay only if the FDW supports it */
*** a/src/backend/executor/nodeModifyTable.c
--- b/src/backend/executor/nodeModifyTable.c
***************
*** 950,956 **** ExecModifyTable(ModifyTableState *node)
  				bool		isNull;
  
  				relkind = resultRelInfo->ri_RelationDesc->rd_rel->relkind;
! 				if (relkind == RELKIND_RELATION)
  				{
  					datum = ExecGetJunkAttribute(slot,
  												 junkfilter->jf_junkAttNo,
--- 950,956 ----
  				bool		isNull;
  
  				relkind = resultRelInfo->ri_RelationDesc->rd_rel->relkind;
! 				if (relkind == RELKIND_RELATION || relkind == RELKIND_MATVIEW)
  				{
  					datum = ExecGetJunkAttribute(slot,
  												 junkfilter->jf_junkAttNo,
***************
*** 1280,1286 **** ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
  					char		relkind;
  
  					relkind = resultRelInfo->ri_RelationDesc->rd_rel->relkind;
! 					if (relkind == RELKIND_RELATION)
  					{
  						j->jf_junkAttNo = ExecFindJunkAttribute(j, "ctid");
  						if (!AttributeNumberIsValid(j->jf_junkAttNo))
--- 1280,1287 ----
  					char		relkind;
  
  					relkind = resultRelInfo->ri_RelationDesc->rd_rel->relkind;
! 					if (relkind == RELKIND_RELATION ||
! 						relkind == RELKIND_MATVIEW)
  					{
  						j->jf_junkAttNo = ExecFindJunkAttribute(j, "ctid");
  						if (!AttributeNumberIsValid(j->jf_junkAttNo))
*** a/src/backend/nodes/copyfuncs.c
--- b/src/backend/nodes/copyfuncs.c
***************
*** 3241,3246 **** _copyRefreshMatViewStmt(const RefreshMatViewStmt *from)
--- 3241,3247 ----
  {
  	RefreshMatViewStmt *newnode = makeNode(RefreshMatViewStmt);
  
+ 	COPY_SCALAR_FIELD(concurrent);
  	COPY_SCALAR_FIELD(skipData);
  	COPY_NODE_FIELD(relation);
  
*** a/src/backend/nodes/equalfuncs.c
--- b/src/backend/nodes/equalfuncs.c
***************
*** 1521,1526 **** _equalCreateTableAsStmt(const CreateTableAsStmt *a, const CreateTableAsStmt *b)
--- 1521,1527 ----
  static bool
  _equalRefreshMatViewStmt(const RefreshMatViewStmt *a, const RefreshMatViewStmt *b)
  {
+ 	COMPARE_SCALAR_FIELD(concurrent);
  	COMPARE_SCALAR_FIELD(skipData);
  	COMPARE_NODE_FIELD(relation);
  
*** a/src/backend/parser/gram.y
--- b/src/backend/parser/gram.y
***************
*** 3284,3294 **** OptNoLog:	UNLOGGED					{ $$ = RELPERSISTENCE_UNLOGGED; }
   *****************************************************************************/
  
  RefreshMatViewStmt:
! 			REFRESH MATERIALIZED VIEW qualified_name opt_with_data
  				{
  					RefreshMatViewStmt *n = makeNode(RefreshMatViewStmt);
! 					n->relation = $4;
! 					n->skipData = !($5);
  					$$ = (Node *) n;
  				}
  		;
--- 3284,3295 ----
   *****************************************************************************/
  
  RefreshMatViewStmt:
! 			REFRESH MATERIALIZED VIEW opt_concurrently qualified_name opt_with_data
  				{
  					RefreshMatViewStmt *n = makeNode(RefreshMatViewStmt);
! 					n->concurrent = $4;
! 					n->relation = $5;
! 					n->skipData = !($6);
  					$$ = (Node *) n;
  				}
  		;
*** a/src/bin/psql/tab-complete.c
--- b/src/bin/psql/tab-complete.c
***************
*** 2871,2876 **** psql_completion(char *text, int start, int end)
--- 2871,2882 ----
  	else if (pg_strcasecmp(prev3_wd, "REFRESH") == 0 &&
  			 pg_strcasecmp(prev2_wd, "MATERIALIZED") == 0 &&
  			 pg_strcasecmp(prev_wd, "VIEW") == 0)
+ 		COMPLETE_WITH_SCHEMA_QUERY(Query_for_list_of_matviews,
+ 								   " UNION SELECT 'CONCURRENTLY'");
+ 	else if (pg_strcasecmp(prev4_wd, "REFRESH") == 0 &&
+ 			 pg_strcasecmp(prev3_wd, "MATERIALIZED") == 0 &&
+ 			 pg_strcasecmp(prev2_wd, "VIEW") == 0 &&
+ 			 pg_strcasecmp(prev_wd, "CONCURRENTLY") == 0)
  		COMPLETE_WITH_SCHEMA_QUERY(Query_for_list_of_matviews, NULL);
  	else if (pg_strcasecmp(prev4_wd, "REFRESH") == 0 &&
  			 pg_strcasecmp(prev3_wd, "MATERIALIZED") == 0 &&
***************
*** 2879,2884 **** psql_completion(char *text, int start, int end)
--- 2885,2895 ----
  	else if (pg_strcasecmp(prev5_wd, "REFRESH") == 0 &&
  			 pg_strcasecmp(prev4_wd, "MATERIALIZED") == 0 &&
  			 pg_strcasecmp(prev3_wd, "VIEW") == 0 &&
+ 			 pg_strcasecmp(prev2_wd, "CONCURRENTLY") == 0)
+ 		COMPLETE_WITH_CONST("WITH DATA");
+ 	else if (pg_strcasecmp(prev5_wd, "REFRESH") == 0 &&
+ 			 pg_strcasecmp(prev4_wd, "MATERIALIZED") == 0 &&
+ 			 pg_strcasecmp(prev3_wd, "VIEW") == 0 &&
  			 pg_strcasecmp(prev_wd, "WITH") == 0)
  	{
  		static const char *const list_WITH_DATA[] =
***************
*** 2889,2894 **** psql_completion(char *text, int start, int end)
--- 2900,2911 ----
  	else if (pg_strcasecmp(prev6_wd, "REFRESH") == 0 &&
  			 pg_strcasecmp(prev5_wd, "MATERIALIZED") == 0 &&
  			 pg_strcasecmp(prev4_wd, "VIEW") == 0 &&
+ 			 pg_strcasecmp(prev3_wd, "CONCURRENTLY") == 0 &&
+ 			 pg_strcasecmp(prev_wd, "WITH") == 0)
+ 		COMPLETE_WITH_CONST("DATA");
+ 	else if (pg_strcasecmp(prev6_wd, "REFRESH") == 0 &&
+ 			 pg_strcasecmp(prev5_wd, "MATERIALIZED") == 0 &&
+ 			 pg_strcasecmp(prev4_wd, "VIEW") == 0 &&
  			 pg_strcasecmp(prev2_wd, "WITH") == 0 &&
  			 pg_strcasecmp(prev_wd, "NO") == 0)
  		COMPLETE_WITH_CONST("DATA");
*** a/src/include/commands/cluster.h
--- b/src/include/commands/cluster.h
***************
*** 25,31 **** extern void check_index_is_clusterable(Relation OldHeap, Oid indexOid,
  						   bool recheck, LOCKMODE lockmode);
  extern void mark_index_clustered(Relation rel, Oid indexOid, bool is_internal);
  
! extern Oid	make_new_heap(Oid OIDOldHeap, Oid NewTableSpace);
  extern void finish_heap_swap(Oid OIDOldHeap, Oid OIDNewHeap,
  				 bool is_system_catalog,
  				 bool swap_toast_by_content,
--- 25,32 ----
  						   bool recheck, LOCKMODE lockmode);
  extern void mark_index_clustered(Relation rel, Oid indexOid, bool is_internal);
  
! extern Oid make_new_heap(Oid OIDOldHeap, Oid NewTableSpace, bool forcetemp,
! 			  LOCKMODE lockmode);
  extern void finish_heap_swap(Oid OIDOldHeap, Oid OIDNewHeap,
  				 bool is_system_catalog,
  				 bool swap_toast_by_content,
*** a/src/include/commands/matview.h
--- b/src/include/commands/matview.h
***************
*** 27,30 **** extern void ExecRefreshMatView(RefreshMatViewStmt *stmt, const char *queryString
--- 27,32 ----
  
  extern DestReceiver *CreateTransientRelDestReceiver(Oid oid);
  
+ extern bool MatViewIncrementalMaintenanceIsEnabled(void);
+ 
  #endif   /* MATVIEW_H */
*** a/src/include/nodes/parsenodes.h
--- b/src/include/nodes/parsenodes.h
***************
*** 2472,2477 **** typedef struct CreateTableAsStmt
--- 2472,2478 ----
  typedef struct RefreshMatViewStmt
  {
  	NodeTag		type;
+ 	bool		concurrent;		/* allow concurrent access? */
  	bool		skipData;		/* true for WITH NO DATA */
  	RangeVar   *relation;		/* relation to insert into */
  } RefreshMatViewStmt;
*** a/src/test/regress/expected/matview.out
--- b/src/test/regress/expected/matview.out
***************
*** 73,78 **** SELECT * FROM tvm;
--- 73,80 ----
  
  CREATE MATERIALIZED VIEW tmm AS SELECT sum(totamt) AS grandtot FROM tm;
  CREATE MATERIALIZED VIEW tvmm AS SELECT sum(totamt) AS grandtot FROM tvm;
+ CREATE UNIQUE INDEX tvmm_expr ON tvmm ((grandtot > 0));
+ CREATE UNIQUE INDEX tvmm_pred ON tvmm (grandtot) WHERE grandtot < 0;
  CREATE VIEW tvv AS SELECT sum(totamt) AS grandtot FROM tv;
  EXPLAIN (costs off)
    CREATE MATERIALIZED VIEW tvvm AS SELECT * FROM tvv;
***************
*** 141,146 **** ALTER MATERIALIZED VIEW tvm SET SCHEMA mvschema;
--- 143,151 ----
    Column  |  Type   | Modifiers | Storage | Stats target | Description 
  ----------+---------+-----------+---------+--------------+-------------
   grandtot | numeric |           | main    |              | 
+ Indexes:
+     "tvmm_expr" UNIQUE, btree ((grandtot > 0::numeric))
+     "tvmm_pred" UNIQUE, btree (grandtot) WHERE grandtot < 0::numeric
  View definition:
   SELECT sum(tvm.totamt) AS grandtot
     FROM mvschema.tvm;
***************
*** 177,183 **** SELECT * FROM tvm ORDER BY type;
   z    |     11
  (3 rows)
  
! REFRESH MATERIALIZED VIEW tm;
  REFRESH MATERIALIZED VIEW tvm;
  SELECT * FROM tm ORDER BY type;
   type | totamt 
--- 182,188 ----
   z    |     11
  (3 rows)
  
! REFRESH MATERIALIZED VIEW CONCURRENTLY tm;
  REFRESH MATERIALIZED VIEW tvm;
  SELECT * FROM tm ORDER BY type;
   type | totamt 
***************
*** 237,242 **** SELECT * FROM tvvm;
--- 242,249 ----
  (1 row)
  
  REFRESH MATERIALIZED VIEW tmm;
+ REFRESH MATERIALIZED VIEW CONCURRENTLY tvmm;
+ ERROR:  concurrent refresh requires a unique index on just columns for all rows of the materialized view
  REFRESH MATERIALIZED VIEW tvmm;
  REFRESH MATERIALIZED VIEW tvvm;
  EXPLAIN (costs off)
***************
*** 281,286 **** SELECT * FROM tvvm;
--- 288,296 ----
  -- test diemv when the mv does not exist
  DROP MATERIALIZED VIEW IF EXISTS no_such_mv;
  NOTICE:  materialized view "no_such_mv" does not exist, skipping
+ -- make sure invalid comination of options is prohibited
+ REFRESH MATERIALIZED VIEW CONCURRENTLY tvmm WITH NO DATA;
+ ERROR:  CONCURRENTLY and WITH NO DATA options cannot be used together
  -- test join of mv and view
  SELECT type, m.totamt AS mtot, v.totamt AS vtot FROM tm m LEFT JOIN tv v USING (type) ORDER BY type;
   type | mtot | vtot 
*** a/src/test/regress/sql/matview.sql
--- b/src/test/regress/sql/matview.sql
***************
*** 29,34 **** CREATE MATERIALIZED VIEW tvm AS SELECT * FROM tv ORDER BY type;
--- 29,36 ----
  SELECT * FROM tvm;
  CREATE MATERIALIZED VIEW tmm AS SELECT sum(totamt) AS grandtot FROM tm;
  CREATE MATERIALIZED VIEW tvmm AS SELECT sum(totamt) AS grandtot FROM tvm;
+ CREATE UNIQUE INDEX tvmm_expr ON tvmm ((grandtot > 0));
+ CREATE UNIQUE INDEX tvmm_pred ON tvmm (grandtot) WHERE grandtot < 0;
  CREATE VIEW tvv AS SELECT sum(totamt) AS grandtot FROM tv;
  EXPLAIN (costs off)
    CREATE MATERIALIZED VIEW tvvm AS SELECT * FROM tvv;
***************
*** 57,63 **** INSERT INTO t VALUES (6, 'z', 13);
  -- confirm pre- and post-refresh contents of fairly simple materialized views
  SELECT * FROM tm ORDER BY type;
  SELECT * FROM tvm ORDER BY type;
! REFRESH MATERIALIZED VIEW tm;
  REFRESH MATERIALIZED VIEW tvm;
  SELECT * FROM tm ORDER BY type;
  SELECT * FROM tvm ORDER BY type;
--- 59,65 ----
  -- confirm pre- and post-refresh contents of fairly simple materialized views
  SELECT * FROM tm ORDER BY type;
  SELECT * FROM tvm ORDER BY type;
! REFRESH MATERIALIZED VIEW CONCURRENTLY tm;
  REFRESH MATERIALIZED VIEW tvm;
  SELECT * FROM tm ORDER BY type;
  SELECT * FROM tvm ORDER BY type;
***************
*** 74,79 **** SELECT * FROM tmm;
--- 76,82 ----
  SELECT * FROM tvmm;
  SELECT * FROM tvvm;
  REFRESH MATERIALIZED VIEW tmm;
+ REFRESH MATERIALIZED VIEW CONCURRENTLY tvmm;
  REFRESH MATERIALIZED VIEW tvmm;
  REFRESH MATERIALIZED VIEW tvvm;
  EXPLAIN (costs off)
***************
*** 89,94 **** SELECT * FROM tvvm;
--- 92,100 ----
  -- test diemv when the mv does not exist
  DROP MATERIALIZED VIEW IF EXISTS no_such_mv;
  
+ -- make sure invalid comination of options is prohibited
+ REFRESH MATERIALIZED VIEW CONCURRENTLY tvmm WITH NO DATA;
+ 
  -- test join of mv and view
  SELECT type, m.totamt AS mtot, v.totamt AS vtot FROM tm m LEFT JOIN tv v USING (type) ORDER BY type;
  
