*** a/doc/src/sgml/gist.sgml
--- b/doc/src/sgml/gist.sgml
***************
*** 642,647 **** my_distance(PG_FUNCTION_ARGS)
--- 642,679 ----
  
    </variablelist>
  
+  <sect2 id="gist-buffering-build">
+   <title>GiST buffering build</title>
+   <para>
+    Building large GiST indexes by simply inserting all the tuples tends to be
+    slow, because if the index tuples are scattered across the index and the
+    index is large enough to not fit in cache, the insertions need to perform
+    a lot of random I/O. PostgreSQL from version 9.2 supports a more efficient
+    method to build GiST indexes based on buffering, which can dramatically
+    reduce number of random I/O needed for non-ordered data sets. For
+    well-ordered datasets the benefit is smaller or non-existent, because
+    only a small number of pages receive new tuples at a time, and those pages
+    fit in cache even if the index as whole does not.
+   </para>
+ 
+   <para>
+    However, buffering index build needs to call the <function>penalty</>
+    function more often, which consumes some extra CPU resources. Also, it can
+    infuence the quality of the produced index, in both positive and negative
+    directions. That influence depends on various factors, like the
+    distribution of the input data and operator class implementation.
+   </para>
+ 
+   <para>
+    By default, the index build switches to the buffering method when the
+    index size reaches <xref linkend="guc-effective-cache-size">. It can
+    be manually turned on or off by the <literal>BUFFERING</literal> parameter
+    to the CREATE INDEX clause. The default behavior is good for most cases,
+    but turning buffering off might speed up the build somewhat if the input
+    data is ordered.
+   </para>
+ 
+  </sect2>
  </sect1>
  
  <sect1 id="gist-examples">
*** a/doc/src/sgml/ref/create_index.sgml
--- b/doc/src/sgml/ref/create_index.sgml
***************
*** 341,346 **** CREATE [ UNIQUE ] INDEX [ CONCURRENTLY ] [ <replaceable class="parameter">name</
--- 341,366 ----
     </varlistentry>
  
     </variablelist>
+    <para>
+     GiST indexes additionaly accepts parameters:
+    </para>
+ 
+    <variablelist>
+ 
+    <varlistentry>
+     <term><literal>BUFFERING</></term>
+     <listitem>
+     <para>
+      Determines whether the buffering build technique described in
+      <xref linkend="gist-buffering-build"> is used to build the index. With
+      <literal>OFF</> it is disabled, with <literal>ON</> it is enabled, and
+      with <literal>AUTO</> it is initially disabled, but turned on
+      on-the-fly once the index size reaches <xref linkend="guc-effective-cache-size">. The default is <literal>AUTO</>.
+     </para>
+     </listitem>
+    </varlistentry>
+ 
+    </variablelist>
    </refsect2>
  
    <refsect2 id="SQL-CREATEINDEX-CONCURRENTLY">
*** a/src/backend/access/common/reloptions.c
--- b/src/backend/access/common/reloptions.c
***************
*** 219,224 **** static relopt_real realRelOpts[] =
--- 219,235 ----
  
  static relopt_string stringRelOpts[] =
  {
+ 	{
+ 		{
+ 			"buffering",
+ 			"Enables buffering build for this GiST index",
+ 			RELOPT_KIND_GIST
+ 		},
+ 		4,
+ 		false,
+ 		gistValidateBufferingOption,
+ 		"auto"
+ 	},
  	/* list terminator */
  	{{NULL}}
  };
*** a/src/backend/access/gist/Makefile
--- b/src/backend/access/gist/Makefile
***************
*** 13,18 **** top_builddir = ../../../..
  include $(top_builddir)/src/Makefile.global
  
  OBJS = gist.o gistutil.o gistxlog.o gistvacuum.o gistget.o gistscan.o \
!        gistproc.o gistsplit.o
  
  include $(top_srcdir)/src/backend/common.mk
--- 13,18 ----
  include $(top_builddir)/src/Makefile.global
  
  OBJS = gist.o gistutil.o gistxlog.o gistvacuum.o gistget.o gistscan.o \
!        gistproc.o gistsplit.o gistbuild.o gistbuildbuffers.o
  
  include $(top_srcdir)/src/backend/common.mk
*** a/src/backend/access/gist/README
--- b/src/backend/access/gist/README
***************
*** 24,29 **** The current implementation of GiST supports:
--- 24,30 ----
    * provides NULL-safe interface to GiST core
    * Concurrency
    * Recovery support via WAL logging
+   * Buffering build algorithm
  
  The support for concurrency implemented in PostgreSQL was developed based on
  the paper "Access Methods for Next-Generation Database Systems" by
***************
*** 31,36 **** Marcel Kornaker:
--- 32,43 ----
  
      http://www.sai.msu.su/~megera/postgres/gist/papers/concurrency/access-methods-for-next-generation.pdf.gz
  
+ Buffering build algorithm for GiST was developed based on the paper "Efficient
+ Bulk Operations on Dynamic R-trees" by Lars Arge, Klaus Hinrichs, Jan Vahrenhold
+ and Jeffrey Scott Vitter.
+ 
+     http://citeseerx.ist.psu.edu/viewdoc/download?doi=10.1.1.135.9894&rep=rep1&type=pdf
+ 
  The original algorithms were modified in several ways:
  
  * They had to be adapted to PostgreSQL conventions. For example, the SEARCH
***************
*** 278,283 **** would complicate the insertion algorithm. So when an insertion sees a page
--- 285,418 ----
  with F_FOLLOW_RIGHT set, it immediately tries to bring the split that
  crashed in the middle to completion by adding the downlink in the parent.
  
+ Buffering build algorithm
+ -------------------------
+ 
+ In the buffering index build algorithm, some or all internal nodes have a
+ buffer attached to them. When a tuple is inserted at the top, the descend down
+ the tree is stopped as soon as a buffer is reached, and the tuple is pushed to
+ the buffer. When a buffer gets too full, all the tuples in it are flushed to
+ the lower level, where they again hit lower level buffers or leaf pages. This
+ makes the insertions happen in more of a breadth-first than depth-first order,
+ which greatly reduces the amount of random I/O required.
+ 
+ In the algorithm, levels are numbered so that leaf pages have level zero,
+ and internal node levels count up from 1. This numbering ensures that a page's
+ level number never changes, even when the root page is split.
+ 
+ Level                    Tree
+ 
+ 3                         *
+                       /       \
+ 2                *                 *
+               /  |  \           /  |  \
+ 1          *     *     *     *     *     *
+           / \   / \   / \   / \   / \   / \
+ 0        o   o o   o o   o o   o o   o o   o
+ 
+ * - internal page
+ o - leaf page
+ 
+ Internal pages that belong to certain levels have buffers associated with
+ them. Leaf pages never have buffers. Which levels have buffers is controlled
+ by "level step" parameter: level numbers that are multiples of level_step
+ have buffers, while others do not. For example, if level_step = 2, then
+ pages on levels 2, 4, 6, ... have buffers. If level_step = 1 then every
+ internal page has a buffer.
+ 
+ Level        Tree (level_step = 1)                Tree (level_step = 2)
+ 
+ 3                      *(b)                                  *
+                    /       \                             /       \
+ 2             *(b)              *(b)                *(b)              *(b)
+            /  |  \           /  |  \             /  |  \           /  |  \
+ 1       *(b)  *(b)  *(b)  *(b)  *(b)  *(b)    *     *     *     *     *     *
+        / \   / \   / \   / \   / \   / \     / \   / \   / \   / \   / \   / \
+ 0     o   o o   o o   o o   o o   o o   o   o   o o   o o   o o   o o   o o   o
+ 
+ (b) - buffer
+ 
+ Logically, a buffer is just bunch of tuples. Physically, it is divided in
+ pages, backed by a temporary file. Each buffer can be in one of two states:
+ a) Last page of the buffer is kept in main memory. A node buffer is
+ automatically switched to this state when a new index tuple is added to it,
+ or a tuple is removed from it.
+ b) All pages of the buffer are swapped out to disk. When a buffer becomes too
+ full, and we start to flush it, all other buffers are switched to this state.
+ 
+ When an index tuple is inserted, its initial processing can end in one of the
+ following points:
+ 1) Leaf page, if the depth of the index <= level_step, meaning that
+    none of the internal pages have buffers associated with them.
+ 2) Buffer of topmost level page that has buffers.
+ 
+ New index tuples are processed until one of the buffers in the topmost
+ buffered level becomes half-full. When a buffer becomes half-full, it's added
+ to the emptying queue, and will be emptied before a new tuple is processed.
+ 
+ Buffer emptying process means that index tuples from the buffer are moved
+ into buffers at a lower level, or leaf pages. First, all the other buffers are
+ swapped to disk to free up the memory. Then tuples are popped from the buffer
+ one by one, and cascaded down the tree to the next buffer or leaf page below
+ the buffered node.
+ 
+ Emptying a buffer has the interesting dynamic property that any intermediate
+ pages between the buffer being emptied, and the next buffered or leaf level
+ below it, become cached. If there are no more buffers below the node, the leaf
+ pages where the tuples finally land on get cached too. If there are, the last
+ buffer page of each buffer below is kept in memory. This is illustrated in
+ the figures below:
+ 
+    Buffer being emptied to
+      lower-level buffers               Buffer being emptied to leaf pages
+ 
+                +(fb)                                 +(fb)
+             /     \                                /     \
+         +             +                        +             +
+       /   \         /   \                    /   \         /   \
+     *(ab)   *(ab) *(ab)   *(ab)            x       x     x       x
+ 
+ +    - cached internal page
+ x    - cached leaf page
+ *    - non-cached internal page
+ (fb) - buffer being emptied
+ (ab) - buffers being appended to, with last page in memory
+ 
+ In the beginning of the index build, the level-step is chosen so that all those
+ pages involved in emptying one buffer fit in cache, so after each of those
+ pages have been accessed once and cached, emptying a buffer doesn't involve
+ any more I/O. This locality is where the speedup of the buffering algorithm
+ comes from.
+ 
+ Emptying one buffer can fill up one or more of the lower-level buffers,
+ triggering emptying of them as well. Whenever a buffer becomes too full, it's
+ added to the emptying queue, and will be emptied after the current buffer has
+ been processed.
+ 
+ To keep the size of each buffer limited even in the worst case, buffer emptying
+ is scheduled as soon as a buffer becomes half-full, and emptying it continues
+ until 1/2 of the nominal buffer size worth of tuples has been emptied. This
+ guarantees that when buffer emptying begins, all the lower-level buffers
+ are at most half-full. In the worst case that all the tuples are cascaded down
+ to the same lower-level buffer, that buffer therefore has enough space to
+ accommodate all the tuples emptied from the upper-level buffer. There is no
+ hard size limit in any of the data structures used, though, so this only needs
+ to be approximate; small overfilling of some buffers doesn't matter.
+ 
+ If an internal page that has a buffer associated with it is split, the buffer
+ needs to be split too. All tuples in the buffer are scanned through and
+ relocated to the correct sibling buffers, using the penalty function to decide
+ which buffer each tuple should go to.
+ 
+ After all tuples from the heap have been processed, there are still some index
+ tuples in the buffers. At this point, final buffer emptying starts. All buffers
+ are emptied in top-down order. This is slightly complicated by the fact that
+ new buffers can be allocated during the emptying, due to page splits. However,
+ the new buffers will always be siblings of buffers that haven't been fully
+ emptied yet; tuples never move upwards in the tree. The final emptying loops
+ through buffers at a given level until all buffers at that level have been
+ emptied, and then moves down to the next level.
+ 
  
  Authors:
  	Teodor Sigaev	<teodor@sigaev.ru>
*** a/src/backend/access/gist/gist.c
--- b/src/backend/access/gist/gist.c
***************
*** 24,56 ****
  #include "utils/memutils.h"
  #include "utils/rel.h"
  
- /* Working state for gistbuild and its callback */
- typedef struct
- {
- 	GISTSTATE	giststate;
- 	int			numindexattrs;
- 	double		indtuples;
- 	MemoryContext tmpCtx;
- } GISTBuildState;
- 
- /* A List of these is used represent a split-in-progress. */
- typedef struct
- {
- 	Buffer		buf;			/* the split page "half" */
- 	IndexTuple	downlink;		/* downlink for this half. */
- } GISTPageSplitInfo;
- 
  /* non-export function prototypes */
- static void gistbuildCallback(Relation index,
- 				  HeapTuple htup,
- 				  Datum *values,
- 				  bool *isnull,
- 				  bool tupleIsAlive,
- 				  void *state);
- static void gistdoinsert(Relation r,
- 			 IndexTuple itup,
- 			 Size freespace,
- 			 GISTSTATE *GISTstate);
  static void gistfixsplit(GISTInsertState *state, GISTSTATE *giststate);
  static bool gistinserttuples(GISTInsertState *state, GISTInsertStack *stack,
  				 GISTSTATE *giststate,
--- 24,30 ----
***************
*** 89,226 **** createTempGistContext(void)
  }
  
  /*
-  * Routine to build an index.  Basically calls insert over and over.
-  *
-  * XXX: it would be nice to implement some sort of bulk-loading
-  * algorithm, but it is not clear how to do that.
-  */
- Datum
- gistbuild(PG_FUNCTION_ARGS)
- {
- 	Relation	heap = (Relation) PG_GETARG_POINTER(0);
- 	Relation	index = (Relation) PG_GETARG_POINTER(1);
- 	IndexInfo  *indexInfo = (IndexInfo *) PG_GETARG_POINTER(2);
- 	IndexBuildResult *result;
- 	double		reltuples;
- 	GISTBuildState buildstate;
- 	Buffer		buffer;
- 	Page		page;
- 
- 	/*
- 	 * We expect to be called exactly once for any index relation. If that's
- 	 * not the case, big trouble's what we have.
- 	 */
- 	if (RelationGetNumberOfBlocks(index) != 0)
- 		elog(ERROR, "index \"%s\" already contains data",
- 			 RelationGetRelationName(index));
- 
- 	/* no locking is needed */
- 	initGISTstate(&buildstate.giststate, index);
- 
- 	/* initialize the root page */
- 	buffer = gistNewBuffer(index);
- 	Assert(BufferGetBlockNumber(buffer) == GIST_ROOT_BLKNO);
- 	page = BufferGetPage(buffer);
- 
- 	START_CRIT_SECTION();
- 
- 	GISTInitBuffer(buffer, F_LEAF);
- 
- 	MarkBufferDirty(buffer);
- 
- 	if (RelationNeedsWAL(index))
- 	{
- 		XLogRecPtr	recptr;
- 		XLogRecData rdata;
- 
- 		rdata.data = (char *) &(index->rd_node);
- 		rdata.len = sizeof(RelFileNode);
- 		rdata.buffer = InvalidBuffer;
- 		rdata.next = NULL;
- 
- 		recptr = XLogInsert(RM_GIST_ID, XLOG_GIST_CREATE_INDEX, &rdata);
- 		PageSetLSN(page, recptr);
- 		PageSetTLI(page, ThisTimeLineID);
- 	}
- 	else
- 		PageSetLSN(page, GetXLogRecPtrForTemp());
- 
- 	UnlockReleaseBuffer(buffer);
- 
- 	END_CRIT_SECTION();
- 
- 	/* build the index */
- 	buildstate.numindexattrs = indexInfo->ii_NumIndexAttrs;
- 	buildstate.indtuples = 0;
- 
- 	/*
- 	 * create a temporary memory context that is reset once for each tuple
- 	 * inserted into the index
- 	 */
- 	buildstate.tmpCtx = createTempGistContext();
- 
- 	/* do the heap scan */
- 	reltuples = IndexBuildHeapScan(heap, index, indexInfo, true,
- 								   gistbuildCallback, (void *) &buildstate);
- 
- 	/* okay, all heap tuples are indexed */
- 	MemoryContextDelete(buildstate.tmpCtx);
- 
- 	freeGISTstate(&buildstate.giststate);
- 
- 	/*
- 	 * Return statistics
- 	 */
- 	result = (IndexBuildResult *) palloc(sizeof(IndexBuildResult));
- 
- 	result->heap_tuples = reltuples;
- 	result->index_tuples = buildstate.indtuples;
- 
- 	PG_RETURN_POINTER(result);
- }
- 
- /*
-  * Per-tuple callback from IndexBuildHeapScan
-  */
- static void
- gistbuildCallback(Relation index,
- 				  HeapTuple htup,
- 				  Datum *values,
- 				  bool *isnull,
- 				  bool tupleIsAlive,
- 				  void *state)
- {
- 	GISTBuildState *buildstate = (GISTBuildState *) state;
- 	IndexTuple	itup;
- 	MemoryContext oldCtx;
- 
- 	oldCtx = MemoryContextSwitchTo(buildstate->tmpCtx);
- 
- 	/* form an index tuple and point it at the heap tuple */
- 	itup = gistFormTuple(&buildstate->giststate, index,
- 						 values, isnull, true /* size is currently bogus */ );
- 	itup->t_tid = htup->t_self;
- 
- 	/*
- 	 * Since we already have the index relation locked, we call gistdoinsert
- 	 * directly.  Normal access method calls dispatch through gistinsert,
- 	 * which locks the relation for write.	This is the right thing to do if
- 	 * you're inserting single tups, but not when you're initializing the
- 	 * whole index at once.
- 	 *
- 	 * In this path we respect the fillfactor setting, whereas insertions
- 	 * after initial build do not.
- 	 */
- 	gistdoinsert(index, itup,
- 			  RelationGetTargetPageFreeSpace(index, GIST_DEFAULT_FILLFACTOR),
- 				 &buildstate->giststate);
- 
- 	buildstate->indtuples += 1;
- 	MemoryContextSwitchTo(oldCtx);
- 	MemoryContextReset(buildstate->tmpCtx);
- }
- 
- /*
   *	gistbuildempty() -- build an empty gist index in the initialization fork
   */
  Datum
--- 63,68 ----
***************
*** 293,300 **** gistinsert(PG_FUNCTION_ARGS)
   * In that case, we continue to hold the root page locked, and the child
   * pages are released; note that new tuple(s) are *not* on the root page
   * but in one of the new child pages.
   */
! static bool
  gistplacetopage(GISTInsertState *state, GISTSTATE *giststate,
  				Buffer buffer,
  				IndexTuple *itup, int ntup, OffsetNumber oldoffnum,
--- 135,144 ----
   * In that case, we continue to hold the root page locked, and the child
   * pages are released; note that new tuple(s) are *not* on the root page
   * but in one of the new child pages.
+  *
+  * Returns 'true' if the page was split, 'false' otherwise.
   */
! bool
  gistplacetopage(GISTInsertState *state, GISTSTATE *giststate,
  				Buffer buffer,
  				IndexTuple *itup, int ntup, OffsetNumber oldoffnum,
***************
*** 474,480 **** gistplacetopage(GISTInsertState *state, GISTSTATE *giststate,
  			else
  				GistPageGetOpaque(ptr->page)->rightlink = oldrlink;
  
! 			if (ptr->next && !is_rootsplit)
  				GistMarkFollowRight(ptr->page);
  			else
  				GistClearFollowRight(ptr->page);
--- 318,332 ----
  			else
  				GistPageGetOpaque(ptr->page)->rightlink = oldrlink;
  
! 			/*
! 			 * Mark the all but the right-most page with the follow-right
! 			 * flag. It will be cleared as soon as the downlink is inserted
! 			 * into the parent, but this ensures that if we error out before
! 			 * that, the index is still consistent. (in buffering build mode,
! 			 * any error will abort the index build anyway, so this is not
! 			 * needed.)
! 			 */
! 			if (ptr->next && !is_rootsplit && !giststate->gfbb)
  				GistMarkFollowRight(ptr->page);
  			else
  				GistClearFollowRight(ptr->page);
***************
*** 508,514 **** gistplacetopage(GISTInsertState *state, GISTSTATE *giststate,
  		/* Write the WAL record */
  		if (RelationNeedsWAL(state->r))
  			recptr = gistXLogSplit(state->r->rd_node, blkno, is_leaf,
! 								   dist, oldrlink, oldnsn, leftchildbuf);
  		else
  			recptr = GetXLogRecPtrForTemp();
  
--- 360,367 ----
  		/* Write the WAL record */
  		if (RelationNeedsWAL(state->r))
  			recptr = gistXLogSplit(state->r->rd_node, blkno, is_leaf,
! 								   dist, oldrlink, oldnsn, leftchildbuf,
! 								   giststate->gfbb ? true : false);
  		else
  			recptr = GetXLogRecPtrForTemp();
  
***************
*** 570,577 **** gistplacetopage(GISTInsertState *state, GISTSTATE *giststate,
  			recptr = GetXLogRecPtrForTemp();
  			PageSetLSN(page, recptr);
  		}
- 
- 		*splitinfo = NIL;
  	}
  
  	/*
--- 423,428 ----
***************
*** 608,614 **** gistplacetopage(GISTInsertState *state, GISTSTATE *giststate,
   * this routine assumes it is invoked in a short-lived memory context,
   * so it does not bother releasing palloc'd allocations.
   */
! static void
  gistdoinsert(Relation r, IndexTuple itup, Size freespace, GISTSTATE *giststate)
  {
  	ItemId		iid;
--- 459,465 ----
   * this routine assumes it is invoked in a short-lived memory context,
   * so it does not bother releasing palloc'd allocations.
   */
! void
  gistdoinsert(Relation r, IndexTuple itup, Size freespace, GISTSTATE *giststate)
  {
  	ItemId		iid;
***************
*** 1414,1419 **** initGISTstate(GISTSTATE *giststate, Relation index)
--- 1265,1271 ----
  		else
  			giststate->supportCollation[i] = DEFAULT_COLLATION_OID;
  	}
+ 	giststate->gfbb = NULL;
  }
  
  void
*** /dev/null
--- b/src/backend/access/gist/gistbuild.c
***************
*** 0 ****
--- 1,1066 ----
+ /*-------------------------------------------------------------------------
+  *
+  * gistbuild.c
+  *	  build algorithm for GiST indexes implementation.
+  *
+  *
+  * Portions Copyright (c) 1996-2011, PostgreSQL Global Development Group
+  * Portions Copyright (c) 1994, Regents of the University of California
+  *
+  * IDENTIFICATION
+  *	  src/backend/access/gist/gistbuild.c
+  *
+  *-------------------------------------------------------------------------
+  */
+ #include "postgres.h"
+ 
+ #include "access/genam.h"
+ #include "access/gist_private.h"
+ #include "catalog/index.h"
+ #include "catalog/pg_collation.h"
+ #include "miscadmin.h"
+ #include "optimizer/cost.h"
+ #include "storage/bufmgr.h"
+ #include "storage/indexfsm.h"
+ #include "storage/smgr.h"
+ #include "utils/memutils.h"
+ #include "utils/rel.h"
+ 
+ /* Step of index tuples for check whether to switch to buffering build mode */
+ #define BUFFERING_MODE_SWITCH_CHECK_STEP 256
+ 
+ /*
+  * Number of tuples to process in the slow way before switching to buffering
+  * mode, when buffering is explicitly turned on. Also, the number of tuples
+  * to process between readjusting the buffer size parameter, while in
+  * buffering mode.
+  */
+ #define BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET 4096
+ 
+ typedef enum
+ {
+ 	GIST_BUFFERING_DISABLED,	/* in regular build mode and aren't going to
+ 								 * switch */
+ 	GIST_BUFFERING_AUTO,		/* in regular build mode, but will switch to
+ 								 * buffering build mode if the index grows
+ 								 * too big */
+ 	GIST_BUFFERING_STATS,		/* gathering statistics of index tuple size
+ 								 * before switching to the buffering build
+ 								 * mode */
+ 	GIST_BUFFERING_ACTIVE		/* in buffering build mode */
+ } GistBufferingMode;
+ 
+ /* Working state for gistbuild and its callback */
+ typedef struct
+ {
+ 	GISTSTATE	giststate;
+ 	int64		indtuples;
+ 	int64		indtuplesSize;
+ 
+ 	Size		freespace;	/* Amount of free space to leave on pages */
+ 
+ 	GistBufferingMode bufferingMode;
+ 	MemoryContext tmpCtx;
+ } GISTBuildState;
+ 
+ static void gistFreeUnreferencedPath(GISTBufferingInsertStack *path);
+ static bool gistProcessItup(GISTSTATE *giststate, GISTInsertState *state,
+ 				GISTBuildBuffers *gfbb, IndexTuple itup,
+ 				GISTBufferingInsertStack *startparent);
+ static void gistProcessEmptyingStack(GISTSTATE *giststate, GISTInsertState *state);
+ static void gistBufferingBuildInsert(Relation index, IndexTuple itup,
+ 						 GISTBuildState *buildstate);
+ static void gistBuildCallback(Relation index,
+ 				  HeapTuple htup,
+ 				  Datum *values,
+ 				  bool *isnull,
+ 				  bool tupleIsAlive,
+ 				  void *state);
+ static int	gistGetMaxLevel(Relation index);
+ static bool gistInitBuffering(GISTBuildState *buildstate, Relation index);
+ static int	calculatePagesPerBuffer(GISTBuildState *buildstate, Relation index,
+ 						int levelStep);
+ static void gistbufferinginserttuples(GISTInsertState *state, GISTSTATE *giststate,
+ 				Buffer buffer,
+ 				IndexTuple *itup, int ntup, OffsetNumber oldoffnum,
+ 				GISTBufferingInsertStack *path);
+ static void gistBufferingFindCorrectParent(GISTSTATE *giststate, Relation r,
+ 							   GISTBufferingInsertStack *child);
+ 
+ /*
+  * Main entry point to GiST indexbuild. Initially calls insert over and over, 
+  * but switches to more efficient buffering build algorithm after a certain
+  * number of tuples (unless buffering mode is disabled).
+  */
+ Datum
+ gistbuild(PG_FUNCTION_ARGS)
+ {
+ 	Relation	heap = (Relation) PG_GETARG_POINTER(0);
+ 	Relation	index = (Relation) PG_GETARG_POINTER(1);
+ 	IndexInfo  *indexInfo = (IndexInfo *) PG_GETARG_POINTER(2);
+ 	IndexBuildResult *result;
+ 	double		reltuples;
+ 	GISTBuildState buildstate;
+ 	Buffer		buffer;
+ 	Page		page;
+ 	MemoryContext oldcxt = CurrentMemoryContext;
+ 
+ 	buildstate.freespace = RelationGetTargetPageFreeSpace(index,
+ 													  GIST_DEFAULT_FILLFACTOR);
+ 
+ 	if (index->rd_options)
+ 	{
+ 		/* Get buffering mode from the options string */
+ 		GiSTOptions *options = (GiSTOptions *) index->rd_options;
+ 		char	   *bufferingMode = (char *) options + options->bufferingModeOffset;
+ 
+ 		if (strcmp(bufferingMode, "on") == 0)
+ 			buildstate.bufferingMode = GIST_BUFFERING_STATS;
+ 		else if (strcmp(bufferingMode, "off") == 0)
+ 			buildstate.bufferingMode = GIST_BUFFERING_DISABLED;
+ 		else
+ 			buildstate.bufferingMode = GIST_BUFFERING_AUTO;
+ 	}
+ 	else
+ 	{
+ 		/* Automatic buffering mode switching by default */
+ 		buildstate.bufferingMode = GIST_BUFFERING_AUTO;
+ 	}
+ 
+ 	/*
+ 	 * We expect to be called exactly once for any index relation. If that's
+ 	 * not the case, big trouble's what we have.
+ 	 */
+ 	if (RelationGetNumberOfBlocks(index) != 0)
+ 		elog(ERROR, "index \"%s\" already contains data",
+ 			 RelationGetRelationName(index));
+ 
+ 	/* no locking is needed */
+ 	initGISTstate(&buildstate.giststate, index);
+ 
+ 	/* initialize the root page */
+ 	buffer = gistNewBuffer(index);
+ 	Assert(BufferGetBlockNumber(buffer) == GIST_ROOT_BLKNO);
+ 	page = BufferGetPage(buffer);
+ 
+ 	START_CRIT_SECTION();
+ 
+ 	GISTInitBuffer(buffer, F_LEAF);
+ 
+ 	MarkBufferDirty(buffer);
+ 
+ 	if (RelationNeedsWAL(index))
+ 	{
+ 		XLogRecPtr	recptr;
+ 		XLogRecData rdata;
+ 
+ 		rdata.data = (char *) &(index->rd_node);
+ 		rdata.len = sizeof(RelFileNode);
+ 		rdata.buffer = InvalidBuffer;
+ 		rdata.next = NULL;
+ 
+ 		recptr = XLogInsert(RM_GIST_ID, XLOG_GIST_CREATE_INDEX, &rdata);
+ 		PageSetLSN(page, recptr);
+ 		PageSetTLI(page, ThisTimeLineID);
+ 	}
+ 	else
+ 		PageSetLSN(page, GetXLogRecPtrForTemp());
+ 
+ 	UnlockReleaseBuffer(buffer);
+ 
+ 	END_CRIT_SECTION();
+ 
+ 	/* build the index */
+ 	buildstate.indtuples = 0;
+ 	buildstate.indtuplesSize = 0;
+ 
+ 	/*
+ 	 * create a temporary memory context that is reset once for each tuple
+ 	 * processed.
+ 	 */
+ 	buildstate.tmpCtx = createTempGistContext();
+ 
+ 	/*
+ 	 * Do the heap scan.
+ 	 */
+ 	reltuples = IndexBuildHeapScan(heap, index, indexInfo, true,
+ 								   gistBuildCallback, (void *) &buildstate);
+ 
+ 	/*
+ 	 * If buffering build was used, flush out all the tuples that are still
+ 	 * in the buffers.
+ 	 */
+ 	if (buildstate.bufferingMode == GIST_BUFFERING_ACTIVE)
+ 	{
+ 		int			i;
+ 		GISTInsertState insertstate;
+ 		GISTNodeBuffer *nodeBuffer;
+ 		MemoryContext oldCtx;
+ 		GISTBuildBuffers *gfbb = buildstate.giststate.gfbb;
+ 
+ 		elog(DEBUG1, "all tuples processed, emptying buffers");
+ 
+ 		oldCtx = MemoryContextSwitchTo(buildstate.tmpCtx);
+ 
+ 		memset(&insertstate, 0, sizeof(GISTInsertState));
+ 		insertstate.freespace = buildstate.freespace;
+ 		insertstate.r = index;
+ 
+ 		/*
+ 		 * Iterate through the levels from the most higher.
+ 		 */
+ 		for (i = gfbb->buffersOnLevelsLen - 1; i >= 0; i--)
+ 		{
+ 			bool		nonEmpty = true;
+ 
+ 			/*
+ 			 * Empty all buffers on this level. We repeatedly loop through all
+ 			 * the buffers on this level, until we observe that all the
+ 			 * buffers are empty. Looping through the list once is not enough,
+ 			 * because emptying one buffer can cause pages to split and new
+ 			 * buffers to be created on the same (and lower) level.
+ 			 */
+ 			while (nonEmpty)
+ 			{
+ 				ListCell   *p;
+ 
+ 				nonEmpty = false;
+ 
+ 				for (p = list_head(gfbb->buffersOnLevels[i]); p; p = p->next)
+ 				{
+ 					bool		isRoot;
+ 
+ 					/* Get next node buffer */
+ 					nodeBuffer = (GISTNodeBuffer *) p->data.ptr_value;
+ 					isRoot = (nodeBuffer->nodeBlocknum == GIST_ROOT_BLKNO);
+ 
+ 					/* Skip empty node buffer */
+ 					if (nodeBuffer->blocksCount == 0)
+ 						continue;
+ 
+ 					/* Memorize that we saw a non-empty buffer. */
+ 					nonEmpty = true;
+ 
+ 					/* Process emptying of node buffer */
+ 					MemoryContextSwitchTo(gfbb->context);
+ 					gfbb->bufferEmptyingQueue = lcons(nodeBuffer, gfbb->bufferEmptyingQueue);
+ 					MemoryContextSwitchTo(buildstate.tmpCtx);
+ 					gistProcessEmptyingStack(&buildstate.giststate, &insertstate);
+ 
+ 					/*
+ 					 * Root page node buffer is the only node buffer that can
+ 					 * be deleted from the list. So, let's be careful and
+ 					 * restart the scan.
+ 					 */
+ 					if (isRoot)
+ 						break;
+ 				}
+ 			}
+ 		}
+ 		MemoryContextSwitchTo(oldCtx);
+ 	}
+ 
+ 	/* okay, all heap tuples are indexed */
+ 	MemoryContextSwitchTo(oldcxt);
+ 	MemoryContextDelete(buildstate.tmpCtx);
+ 
+ 	freeGISTstate(&buildstate.giststate);
+ 
+ 	/*
+ 	 * Return statistics
+ 	 */
+ 	result = (IndexBuildResult *) palloc(sizeof(IndexBuildResult));
+ 
+ 	result->heap_tuples = reltuples;
+ 	result->index_tuples = (double) buildstate.indtuples;
+ 
+ 	PG_RETURN_POINTER(result);
+ }
+ 
+ 
+ /*
+  * Validator for "buffering" reloption on GiST indexes. Allows "on", "off"
+  * and "auto" values.
+  */
+ void
+ gistValidateBufferingOption(char *value)
+ {
+ 	if (value == NULL ||
+ 		(strcmp(value, "on") != 0 &&
+ 		 strcmp(value, "off") != 0 &&
+ 		 strcmp(value, "auto") != 0))
+ 	{
+ 		ereport(ERROR,
+ 				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ 				 errmsg("invalid value for \"buffering\" option"),
+ 				 errdetail("Valid values are \"on\", \"off\" and \"auto\".")));
+ 	}
+ }
+ 
+ /*
+  * Free unreferenced parts of a path stack.
+  */
+ static void
+ gistFreeUnreferencedPath(GISTBufferingInsertStack *path)
+ {
+ 	while (path->refCount == 0)
+ 	{
+ 		/*
+ 		 * Path part is unreferenced. We can free it and decrease reference
+ 		 * count of parent. If parent becomes unreferenced too procedure
+ 		 * should be repeated for it.
+ 		 */
+ 		GISTBufferingInsertStack *tmp = path->parent;
+ 
+ 		pfree(path);
+ 		path = tmp;
+ 		if (path)
+ 			path->refCount--;
+ 		else
+ 			break;
+ 	}
+ }
+ 
+ /*
+  * Decrease reference count of path part, and free any unreferenced parts of
+  * the path stack.
+  */
+ void
+ gistDecreasePathRefcount(GISTBufferingInsertStack *path)
+ {
+ 	path->refCount--;
+ 	gistFreeUnreferencedPath(path);
+ }
+ 
+ /*
+  * Process an index tuple. Runs the tuple down the tree until we reach a leaf
+  * page or node buffer, and inserts the tuple there. Returns true if we have
+  * to stop buffer emptying process (because one of child buffers can't take
+  * index tuples anymore).
+  */
+ static bool
+ gistProcessItup(GISTSTATE *giststate, GISTInsertState *state,
+ 				GISTBuildBuffers *gfbb, IndexTuple itup,
+ 				GISTBufferingInsertStack *startparent)
+ {
+ 	GISTBufferingInsertStack *path;
+ 	BlockNumber childblkno;
+ 	Buffer		buffer;
+ 	bool		result = false;
+ 
+ 	/*
+ 	 * NULL passed in startparent means that we start index tuple processing
+ 	 * from the root.
+ 	 */
+ 	if (!startparent)
+ 		path = gfbb->rootitem;
+ 	else
+ 		path = startparent;
+ 
+ 	/*
+ 	 * Loop until we reach a leaf page (level == 0) or a level with buffers
+ 	 * (not including the level we start at, because we would otherwise make
+ 	 * no progress).
+ 	 */
+ 	for (;;)
+ 	{
+ 		ItemId		iid;
+ 		IndexTuple	idxtuple,
+ 					newtup;
+ 		Page		page;
+ 		OffsetNumber childoffnum;
+ 		GISTBufferingInsertStack *parent;
+ 
+ 		/* Have we reached a level with buffers? */
+ 		if (LEVEL_HAS_BUFFERS(path->level, gfbb) && path != startparent)
+ 			break;
+ 
+ 		/* Have we reached a leaf page? */
+ 		if (path->level == 0)
+ 			break;
+ 
+ 		/*
+ 		 * Nope. Descend down to the next level then. Choose a child to descend
+ 		 * down to.
+ 		 */
+ 		buffer = ReadBuffer(state->r, path->blkno);
+ 		LockBuffer(buffer, GIST_EXCLUSIVE);
+ 
+ 		page = (Page) BufferGetPage(buffer);
+ 		childoffnum = gistchoose(state->r, page, itup, giststate);
+ 		iid = PageGetItemId(page, childoffnum);
+ 		idxtuple = (IndexTuple) PageGetItem(page, iid);
+ 		childblkno = ItemPointerGetBlockNumber(&(idxtuple->t_tid));
+ 
+ 		/*
+ 		 * Check that the key representing the target child node is
+ 		 * consistent with the key we're inserting. Update it if it's not.
+ 		 */
+ 		newtup = gistgetadjusted(state->r, idxtuple, itup, giststate);
+ 		if (newtup)
+ 			gistbufferinginserttuples(state, giststate, buffer, &newtup, 1,
+ 									  childoffnum, path);
+ 		UnlockReleaseBuffer(buffer);
+ 
+ 		/* Create new path item representing current page */
+ 		parent = path;
+ 		path = (GISTBufferingInsertStack *) MemoryContextAlloc(gfbb->context,
+ 										   sizeof(GISTBufferingInsertStack));
+ 		path->parent = parent;
+ 		path->level = parent->level - 1;
+ 		path->blkno = childblkno;
+ 		path->downlinkoffnum = childoffnum;
+ 		path->refCount = 0;		/* it's unreferenced for now */
+ 
+ 		/* Adjust reference count of parent */
+ 		if (parent)
+ 			parent->refCount++;
+ 	}
+ 
+ 	if (LEVEL_HAS_BUFFERS(path->level, gfbb))
+ 	{
+ 		/*
+ 		 * We've reached level with buffers. Place the index tuple to the
+ 		 * buffer, and add the buffer to the emptying queue if it overflows.
+ 		 */
+ 		GISTNodeBuffer *childNodeBuffer;
+ 
+ 		/* Find the buffer or create a new one */
+ 		childNodeBuffer = gistGetNodeBuffer(gfbb, giststate, path->blkno,
+ 											path->downlinkoffnum, path->parent);
+ 
+ 		/* Add index tuple to it */
+ 		gistPushItupToNodeBuffer(gfbb, childNodeBuffer, itup);
+ 
+ 		if (BUFFER_OVERFLOWED(childNodeBuffer, gfbb))
+ 			result = true;
+ 	}
+ 	else
+ 	{
+ 		/*
+ 		 * We've reached a leaf page. Place the tuple here.
+ 		 */
+ 		buffer = ReadBuffer(state->r, path->blkno);
+ 		LockBuffer(buffer, GIST_EXCLUSIVE);
+ 		gistbufferinginserttuples(state, giststate, buffer, &itup, 1,
+ 								  InvalidOffsetNumber, path);
+ 		UnlockReleaseBuffer(buffer);
+ 	}
+ 
+ 	/*
+ 	 * Free unreferenced path items, if any. Path item may be referenced by
+ 	 * node buffer.
+ 	 */
+ 	gistFreeUnreferencedPath(path);
+ 
+ 	return result;
+ }
+ 
+ /*
+  * Insert tuples to a given page.
+  *
+  * This is analogous with gistinserttuples() in the regular insertion code.
+  */
+ static void
+ gistbufferinginserttuples(GISTInsertState *state, GISTSTATE *giststate,
+ 						  Buffer buffer,
+ 						  IndexTuple *itup, int ntup, OffsetNumber oldoffnum,
+ 						  GISTBufferingInsertStack *path)
+ {
+ 	GISTBuildBuffers *gfbb = giststate->gfbb;
+ 	List	   *splitinfo;
+ 	bool		is_split;
+ 
+ 	is_split = gistplacetopage(state, giststate, buffer,
+ 							   itup, ntup, oldoffnum,
+ 							   InvalidBuffer,
+ 							   &splitinfo);
+ 	/*
+ 	 * If this is a root split, update the root path item kept in memory.
+ 	 * This ensures that all path stacks are always complete, including all
+ 	 * parent nodes up to the root. That simplifies the algorithm to re-find
+ 	 * correct parent.
+ 	 */
+ 	if (is_split && BufferGetBlockNumber(buffer) == GIST_ROOT_BLKNO)
+ 	{
+ 		GISTBufferingInsertStack *oldroot = gfbb->rootitem;
+ 		Page		page = BufferGetPage(buffer);
+ 		ItemId		iid;
+ 		IndexTuple	idxtuple;
+ 		BlockNumber leftmostchild;
+ 
+ 		gfbb->rootitem = (GISTBufferingInsertStack *) MemoryContextAlloc(
+ 			gfbb->context, sizeof(GISTBufferingInsertStack));
+ 		gfbb->rootitem->parent = NULL;
+ 		gfbb->rootitem->blkno = GIST_ROOT_BLKNO;
+ 		gfbb->rootitem->downlinkoffnum = InvalidOffsetNumber;
+ 		gfbb->rootitem->level = oldroot->level + 1;
+ 		gfbb->rootitem->refCount = 1;
+ 
+ 		/*
+ 		 * All the downlinks on the old root page are now on one of the child
+ 		 * pages. Change the block number of the old root entry in the stack
+ 		 * to point to the leftmost child. The other child pages will be
+ 		 * accessible from there by walking right.
+ 		 */
+ 		iid = PageGetItemId(page, FirstOffsetNumber);
+ 		idxtuple = (IndexTuple) PageGetItem(page, iid);
+ 		leftmostchild = ItemPointerGetBlockNumber(&(idxtuple->t_tid));
+ 
+ 		oldroot->parent = gfbb->rootitem;
+ 		oldroot->blkno = leftmostchild;
+ 		oldroot->downlinkoffnum = InvalidOffsetNumber;
+ 	}
+ 
+ 	if (splitinfo)
+ 	{
+ 		/*
+ 		 * Insert the downlinks to the parent. This is analogous with
+ 		 * gistfinishsplit() in the regular insertion code, but the locking
+ 		 * is simpler, and we have to maintain the buffers.
+ 		 */
+ 		IndexTuple *downlinks;
+ 		int			ndownlinks,
+ 					i;
+ 		Buffer		parentBuffer;
+ 		ListCell   *lc;
+ 
+ 		/* Parent may have changed since we memorized this path. */
+ 		gistBufferingFindCorrectParent(giststate, state->r, path);
+ 
+ 		/*
+ 		 * If there's a buffer associated with this page, that needs to
+ 		 * be split too. gistRelocateBuildBuffersOnSplit() will also adjust
+ 		 * the downlinks in 'splitinfo', to make sure they're consistent not
+ 		 * only with the tuples already on the pages, but also the tuples in
+ 		 * the buffers that will eventually be inserted to them.
+ 		 */
+ 		gistRelocateBuildBuffersOnSplit(gfbb, giststate, state->r,
+ 										path, buffer, splitinfo);
+ 
+ 		/* Create an array of all the downlink tuples */
+ 		ndownlinks = list_length(splitinfo);
+ 		downlinks = (IndexTuple *) palloc(sizeof(IndexTuple) * ndownlinks);
+ 		i = 0;
+ 		foreach(lc, splitinfo)
+ 		{
+ 			GISTPageSplitInfo *splitinfo = lfirst(lc);
+ 
+ 			/*
+ 			 * Since there's no concurrent access, we can release the lower
+ 			 * level buffers immediately. Don't release the buffer for the
+ 			 * original page, though, because the caller will release that.
+ 			 */
+ 			if (splitinfo->buf != buffer)
+ 				UnlockReleaseBuffer(splitinfo->buf);
+ 			downlinks[i++] = splitinfo->downlink;
+ 		}
+ 
+ 		/* Insert them into parent. */
+ 		parentBuffer = ReadBuffer(state->r, path->parent->blkno);
+ 		LockBuffer(parentBuffer, GIST_EXCLUSIVE);
+ 		gistbufferinginserttuples(state, giststate, parentBuffer,
+ 								  downlinks, ndownlinks,
+ 								  path->downlinkoffnum, path->parent);
+ 		UnlockReleaseBuffer(parentBuffer);
+ 
+ 		list_free_deep(splitinfo);		/* we don't need this anymore */
+ 	}
+ }
+ 
+ /*
+  * Find correct parent by following rightlinks in buffering index build. This
+  * method of parent searching is possible because no concurrent activity is
+  * possible while index builds.
+  */
+ static void
+ gistBufferingFindCorrectParent(GISTSTATE *giststate, Relation r,
+ 							   GISTBufferingInsertStack *child)
+ {
+ 	GISTBuildBuffers *gfbb = giststate->gfbb;
+ 	GISTBufferingInsertStack *parent = child->parent;
+ 	OffsetNumber i,
+ 				maxoff;
+ 	ItemId		iid;
+ 	IndexTuple	idxtuple;
+ 	Buffer		buffer;
+ 	Page		page;
+ 	bool		copied = false;
+ 
+ 	buffer = ReadBuffer(r, parent->blkno);
+ 	page = BufferGetPage(buffer);
+ 	LockBuffer(buffer, GIST_EXCLUSIVE);
+ 	gistcheckpage(r, buffer);
+ 
+ 	/* Check if it was not moved */
+ 	if (child->downlinkoffnum != InvalidOffsetNumber)
+ 	{
+ 		iid = PageGetItemId(page, child->downlinkoffnum);
+ 		idxtuple = (IndexTuple) PageGetItem(page, iid);
+ 		if (ItemPointerGetBlockNumber(&(idxtuple->t_tid)) == child->blkno)
+ 		{
+ 			/* Still there */
+ 			UnlockReleaseBuffer(buffer);
+ 			return;
+ 		}
+ 	}
+ 
+ 	/* parent has changed, look child in right links until found */
+ 	while (true)
+ 	{
+ 		/* Search for relevant downlink in the current page */
+ 		maxoff = PageGetMaxOffsetNumber(page);
+ 		for (i = FirstOffsetNumber; i <= maxoff; i = OffsetNumberNext(i))
+ 		{
+ 			iid = PageGetItemId(page, i);
+ 			idxtuple = (IndexTuple) PageGetItem(page, iid);
+ 			if (ItemPointerGetBlockNumber(&(idxtuple->t_tid)) == child->blkno)
+ 			{
+ 				/* yes!!, found */
+ 				child->downlinkoffnum = i;
+ 				UnlockReleaseBuffer(buffer);
+ 				return;
+ 			}
+ 		}
+ 
+ 		/*
+ 		 * We should copy parent path item because some other path items can
+ 		 * refer to it.
+ 		 */
+ 		if (!copied)
+ 		{
+ 			parent = (GISTBufferingInsertStack *) MemoryContextAlloc(gfbb->context,
+ 										   sizeof(GISTBufferingInsertStack));
+ 			memcpy(parent, child->parent, sizeof(GISTBufferingInsertStack));
+ 			if (parent->parent)
+ 				parent->parent->refCount++;
+ 			gistDecreasePathRefcount(child->parent);
+ 			child->parent = parent;
+ 			parent->refCount = 1;
+ 			copied = true;
+ 		}
+ 
+ 		/*
+ 		 * Not found in current page. Move towards rightlink.
+ 		 */
+ 		parent->blkno = GistPageGetOpaque(page)->rightlink;
+ 		UnlockReleaseBuffer(buffer);
+ 
+ 		if (parent->blkno == InvalidBlockNumber)
+ 		{
+ 			/*
+ 			 * End of chain and still didn't find parent. Should not happen
+ 			 * during index build.
+ 			 */
+ 			break;
+ 		}
+ 
+ 		/* Get the next page */
+ 		buffer = ReadBuffer(r, parent->blkno);
+ 		page = BufferGetPage(buffer);
+ 		LockBuffer(buffer, GIST_EXCLUSIVE);
+ 		gistcheckpage(r, buffer);
+ 	}
+ 
+ 	elog(ERROR, "failed to re-find parent for block %u", child->blkno);
+ }
+ 
+ /*
+  * Process buffers emptying stack. Emptying of one buffer can cause emptying
+  * of other buffers. This function iterates until this cascading emptying
+  * process finished, e.g. until buffers emptying stack is empty.
+  */
+ static void
+ gistProcessEmptyingStack(GISTSTATE *giststate, GISTInsertState *state)
+ {
+ 	GISTBuildBuffers *gfbb = giststate->gfbb;
+ 
+ 	/* Iterate while we have elements in buffers emptying stack. */
+ 	while (gfbb->bufferEmptyingQueue != NIL)
+ 	{
+ 		GISTNodeBuffer *emptyingNodeBuffer;
+ 
+ 		/* Get node buffer from emptying stack. */
+ 		emptyingNodeBuffer = (GISTNodeBuffer *) linitial(gfbb->bufferEmptyingQueue);
+ 		gfbb->bufferEmptyingQueue = list_delete_first(gfbb->bufferEmptyingQueue);
+ 		emptyingNodeBuffer->queuedForEmptying = false;
+ 
+ 		/*
+ 		 * We are going to load last pages of buffers where emptying will be
+ 		 * to. So let's unload any previously loaded buffers.
+ 		 */
+ 		gistUnloadNodeBuffers(gfbb);
+ 
+ 		/* Variables for split of current emptying buffer detection. */
+ 		gfbb->currentEmptyingBufferSplit = false;
+ 		gfbb->currentEmptyingBufferBlockNumber = emptyingNodeBuffer->nodeBlocknum;
+ 
+ 		while (true)
+ 		{
+ 			IndexTuple	itup;
+ 
+ 			/* Get next index tuple from the buffer */
+ 			if (!gistPopItupFromNodeBuffer(gfbb, emptyingNodeBuffer, &itup))
+ 				break;
+ 
+ 			/* Run it down to the underlying node buffer or leaf page */
+ 			if (gistProcessItup(giststate, state, gfbb, itup, emptyingNodeBuffer->path))
+ 				break;
+ 
+ 			/* Free all the memory allocated during index tuple processing */
+ 			MemoryContextReset(CurrentMemoryContext);
+ 
+ 			/*
+ 			 * If current emptying node buffer split, we have to stop emptying
+ 			 * it, because the buffer might not exist anymore.
+ 			 */
+ 			if (gfbb->currentEmptyingBufferSplit)
+ 				break;
+ 		}
+ 	}
+ }
+ 
+ /*
+  * Insert function for buffering index build.
+  */
+ static void
+ gistBufferingBuildInsert(Relation index, IndexTuple itup,
+ 						 GISTBuildState *buildstate)
+ {
+ 	GISTBuildBuffers *gfbb = buildstate->giststate.gfbb;
+ 	GISTInsertState insertstate;
+ 
+ 	memset(&insertstate, 0, sizeof(GISTInsertState));
+ 	insertstate.freespace = buildstate->freespace;
+ 	insertstate.r = index;
+ 
+ 	/* We are ready for index tuple processing */
+ 	gistProcessItup(&buildstate->giststate, &insertstate, gfbb, itup, NULL);
+ 
+ 	/* Process buffer emptying stack if any */
+ 	gistProcessEmptyingStack(&buildstate->giststate, &insertstate);
+ }
+ 
+ /*
+  * Per-tuple callback from IndexBuildHeapScan.
+  */
+ static void
+ gistBuildCallback(Relation index,
+ 				  HeapTuple htup,
+ 				  Datum *values,
+ 				  bool *isnull,
+ 				  bool tupleIsAlive,
+ 				  void *state)
+ {
+ 	GISTBuildState *buildstate = (GISTBuildState *) state;
+ 	IndexTuple	itup;
+ 	MemoryContext oldCtx;
+ 
+ 	oldCtx = MemoryContextSwitchTo(buildstate->tmpCtx);
+ 
+ 	/* form an index tuple and point it at the heap tuple */
+ 	itup = gistFormTuple(&buildstate->giststate, index, values, isnull, true);
+ 	itup->t_tid = htup->t_self;
+ 
+ 	if (buildstate->bufferingMode == GIST_BUFFERING_ACTIVE)
+ 	{
+ 		/* We have buffers, so use them. */
+ 		gistBufferingBuildInsert(index, itup, buildstate);
+ 	}
+ 	else
+ 	{
+ 		/*
+ 		 * There's no buffers (yet). Since we already have the index relation
+ 		 * locked, we call gistdoinsert directly.
+ 		 *
+ 		 * In this path we respect the fillfactor setting, whereas insertions
+ 		 * after initial build do not.
+ 		 */
+ 		gistdoinsert(index, itup, buildstate->freespace,
+ 					 &buildstate->giststate);
+ 	}
+ 
+ 	/* Increase statistics of index tuples count and their total size. */
+ 	buildstate->indtuples += 1;
+ 	buildstate->indtuplesSize += IndexTupleSize(itup);
+ 
+ 	MemoryContextSwitchTo(oldCtx);
+ 	MemoryContextReset(buildstate->tmpCtx);
+ 
+ 	if (buildstate->bufferingMode == GIST_BUFFERING_ACTIVE &&
+ 		buildstate->indtuples % BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET == 0)
+ 	{
+ 		/* Adjust the target buffer size now */
+ 		buildstate->giststate.gfbb->pagesPerBuffer =
+ 			calculatePagesPerBuffer(buildstate, index,
+ 									buildstate->giststate.gfbb->levelStep);
+ 	}
+ 
+ 	/*
+ 	 * In 'auto' mode, check if the index has grown too large to fit in
+ 	 * cache, and switch to buffering mode if it has.
+ 	 *
+ 	 * To avoid excessive calls to smgrnblocks(), only check this every
+ 	 * BUFFERING_MODE_SWITCH_CHECK_STEP index tuples
+ 	 */
+ 	if ((buildstate->bufferingMode == GIST_BUFFERING_AUTO &&
+ 		 buildstate->indtuples % BUFFERING_MODE_SWITCH_CHECK_STEP == 0 &&
+ 		 effective_cache_size < smgrnblocks(index->rd_smgr, MAIN_FORKNUM)) ||
+ 		(buildstate->bufferingMode == GIST_BUFFERING_STATS &&
+ 		 buildstate->indtuples >= BUFFERING_MODE_TUPLE_SIZE_STATS_TARGET))
+ 	{
+ 		/*
+ 		 * Index doesn't fit in effective cache anymore. Try to switch to
+ 		 * buffering build mode.
+ 		 */
+ 		if (gistInitBuffering(buildstate, index))
+ 		{
+ 			/*
+ 			 * Buffering build is successfully initialized. Now we can set
+ 			 * appropriate flag.
+ 			 */
+ 			buildstate->bufferingMode = GIST_BUFFERING_ACTIVE;
+ 		}
+ 		else
+ 		{
+ 			/*
+ 			 * Failed to switch to buffering build due to not enough memory
+ 			 * settings. Mark that we aren't going to switch anymore.
+ 			 */
+ 			buildstate->bufferingMode = GIST_BUFFERING_DISABLED;
+ 		}
+ 	}
+ }
+ 
+ /*
+  * Calculate pagesPerBuffer parameter for the buffering algorithm.
+  *
+  * Buffer size is chosen so that assuming that tuples are distributed
+  * randomly, emptying half a buffer fills on average one page in every buffer
+  * at the next lower level.
+  */
+ static int
+ calculatePagesPerBuffer(GISTBuildState *buildstate, Relation index,
+ 						int levelStep)
+ {
+ 	double		pagesPerBuffer;
+ 	double		avgIndexTuplesPerPage;
+ 	double		itupAvgSize;
+ 	Size		pageFreeSpace;
+ 
+ 	/* Calc space of index page which is available for index tuples */
+ 	pageFreeSpace = BLCKSZ - SizeOfPageHeaderData - sizeof(GISTPageOpaqueData)
+ 		- sizeof(ItemIdData)
+ 		- buildstate->freespace;
+ 
+ 	/*
+ 	 * Calculate average size of already inserted index tuples using
+ 	 * gathered statistics.
+ 	 */
+ 	itupAvgSize = (double) buildstate->indtuplesSize /
+ 				  (double) buildstate->indtuples;
+ 
+ 	avgIndexTuplesPerPage = pageFreeSpace / itupAvgSize;
+ 
+ 	/*
+ 	 * Recalculate required size of buffers.
+ 	 */
+ 	pagesPerBuffer = 2 * pow(avgIndexTuplesPerPage, levelStep);
+ 
+ 	return round(pagesPerBuffer);
+ }
+ 
+ 
+ /*
+  * Get the depth of the GiST index.
+  */
+ static int
+ gistGetMaxLevel(Relation index)
+ {
+ 	int			maxLevel;
+ 	BlockNumber blkno;
+ 
+ 	/*
+ 	 * Traverse down the tree, starting from the root, until we hit the
+ 	 * leaf level.
+ 	 */
+ 	maxLevel = 0;
+ 	blkno = GIST_ROOT_BLKNO;
+ 	while (true)
+ 	{
+ 		Buffer		buffer;
+ 		Page		page;
+ 		IndexTuple	itup;
+ 
+ 		buffer = ReadBuffer(index, blkno);
+ 		page = (Page) BufferGetPage(buffer);
+ 
+ 		if (GistPageIsLeaf(page))
+ 		{
+ 			/* We hit the bottom, so we're done. */
+ 			ReleaseBuffer(buffer);
+ 			break;
+ 		}
+ 
+ 		/*
+ 		 * Pick the first downlink on the page, and follow it. It doesn't
+ 		 * matter which downlink we choose, the tree has the same depth
+ 		 * everywhere, so we just pick the first one.
+ 		 */
+ 		itup = (IndexTuple) PageGetItem(page,
+ 									 PageGetItemId(page, FirstOffsetNumber));
+ 		blkno = ItemPointerGetBlockNumber(&(itup->t_tid));
+ 		ReleaseBuffer(buffer);
+ 
+ 		/*
+ 		 * We're going down on the tree. It means that there is yet one more
+ 		 * level is the tree.
+ 		 */
+ 		maxLevel++;
+ 	}
+ 	return maxLevel;
+ }
+ 
+ /*
+  * Initial calculations for GiST buffering build.
+  */
+ static bool
+ gistInitBuffering(GISTBuildState *buildstate, Relation index)
+ {
+ 	int			pagesPerBuffer;
+ 	Size		pageFreeSpace;
+ 	Size		itupAvgSize,
+ 				itupMinSize;
+ 	double		avgIndexTuplesPerPage,
+ 				maxIndexTuplesPerPage;
+ 	int			i;
+ 	int			levelStep;
+ 	GISTBuildBuffers *gfbb;
+ 
+ 	/* Calc space of index page which is available for index tuples */
+ 	pageFreeSpace = BLCKSZ - SizeOfPageHeaderData - sizeof(GISTPageOpaqueData)
+ 		- sizeof(ItemIdData)
+ 		- buildstate->freespace;
+ 
+ 	/*
+ 	 * Calculate average size of already inserted index tuples using gathered
+ 	 * statistics.
+ 	 */
+ 	itupAvgSize = (double) buildstate->indtuplesSize /
+ 				  (double) buildstate->indtuples;
+ 
+ 	/*
+ 	 * Calculate minimal possible size of index tuple by index metadata.
+ 	 * Minimal possible size of varlena is VARHDRSZ.
+ 	 *
+ 	 * XXX: that's not actually true, as a short varlen can be just 2 bytes.
+ 	 * And we should take padding into account here.
+ 	 */
+ 	itupMinSize = (Size) MAXALIGN(sizeof(IndexTupleData));
+ 	for (i = 0; i < index->rd_att->natts; i++)
+ 	{
+ 		if (index->rd_att->attrs[i]->attlen < 0)
+ 			itupMinSize += VARHDRSZ;
+ 		else
+ 			itupMinSize += index->rd_att->attrs[i]->attlen;
+ 	}
+ 
+ 	/* Calculate average and maximal number of index tuples which fit to page */
+ 	avgIndexTuplesPerPage = pageFreeSpace / itupAvgSize;
+ 	maxIndexTuplesPerPage = pageFreeSpace / itupMinSize;
+ 
+ 	/*
+ 	 * We need to calculate two parameters for the buffering algorithm:
+ 	 * levelStep and pagesPerBuffer.
+ 	 *
+ 	 * levelStep determines the size of subtree that we operate on, while
+ 	 * emptying a buffer. A higher value is better, as you need fewer buffer
+ 	 * emptying steps to perform the index build. However, if you set it too
+ 	 * high, the subtree doesn't fit in cache anymore, and you quickly lose
+ 	 * the benefit of the buffers.
+ 	 *
+ 	 * In Arge et al's paper, levelStep is chosen as logB(M/4B), where B is
+ 	 * the number of tuples on page (ie. fanout), and M is the amount of
+ 	 * internal memory available. Curiously, they doesn't explain *why* that
+ 	 * setting is optimal. We calculate it by taking the highest levelStep
+ 	 * so that a subtree still fits in cache. For a small B, our way of
+ 	 * calculating levelStep is very close to Arge et al's formula. For a
+ 	 * large B, our formula gives a value that is 2x higher.
+ 	 *
+ 	 * The average size of a subtree of depth n can be calculated as a
+ 	 * geometric series:
+ 	 *
+ 	 *		B^0 + B^1 + B^2 + ... + B^n = (1 - B^(n + 1)) / (1 - B)
+ 	 *
+ 	 * where B is the average number of index tuples on page. The subtree is
+ 	 * cached in the shared buffer cache and the OS cache, so we choose
+ 	 * levelStep so that the subtree size is comfortably smaller than
+ 	 * effective_cache_size, with a safety factor of 4.
+ 	 *
+ 	 * The estimate on the average number of index tuples on page is based on
+ 	 * average tuple sizes observed before switching to buffered build, so the
+ 	 * real subtree size can be somewhat larger. Also, it would selfish to
+ 	 * gobble the whole cache for our index build. The safety factor of 4
+ 	 * should account for those effects.
+ 	 *
+ 	 * The other limiting factor for setting levelStep is that while
+ 	 * processing a subtree, we need to hold one page for each buffer at the
+ 	 * next lower buffered level. The max. number of buffers needed for that
+ 	 * is maxIndexTuplesPerPage^levelStep. This is very conservative, but
+ 	 * hopefully maintenance_work_mem is set high enough that you're
+ 	 * constrained by effective_cache_size rather than maintenance_work_mem.
+ 	 *
+ 	 * XXX: the buffer hash table consumes a fair amount of memory too per
+ 	 * buffer, but that is not currently taken into account. That scales on
+ 	 * the total number of buffers used, ie. the index size and on levelStep.
+ 	 * Note that a higher levelStep *reduces* the amount of memory needed for
+ 	 * the hash table.
+ 	 */
+ 	levelStep = 1;
+ 	while (
+ 		/* subtree must fit in cache (with safety factor of 4) */
+ 		(1 - pow(avgIndexTuplesPerPage, (double) (levelStep + 1))) / (1 - avgIndexTuplesPerPage) < effective_cache_size / 4
+ 		&&
+ 		/* each node in the lowest level of a subtree has one page in memory */
+ 		(pow(maxIndexTuplesPerPage, (double) levelStep) < (maintenance_work_mem * 1024) / BLCKSZ)
+ 		)
+ 	{
+ 		levelStep++;
+ 	}
+ 
+ 	/*
+ 	 * We've just reached unacceptable value of levelStep in previous loop.
+ 	 * So, decrease levelStep to get last acceptable value.
+ 	 */
+ 	levelStep--;
+ 
+ 	/*
+ 	 * If there's not enough cache or maintenance_work_mem, fall back to plain
+ 	 * inserts.
+ 	 */
+ 	if (levelStep <= 0)
+ 	{
+ 		elog(DEBUG1, "failed to switch to buffered GiST build");
+ 		return false;
+ 	}
+ 
+ 	/*
+ 	 * The second parameter to set is pagesPerBuffer, which determines the
+ 	 * size of each buffer. We adjust pagesPerBuffer also during the build,
+ 	 * which is why this calculation is in a separate function.
+ 	 */
+ 	pagesPerBuffer = calculatePagesPerBuffer(buildstate, index, levelStep);
+ 
+ 	elog(DEBUG1, "switching to buffered GiST build; level step = %d, pagesPerBuffer = %d",
+ 		 levelStep, pagesPerBuffer);
+ 
+ 	/* Initialize GISTBuildBuffers with these parameters */
+ 	gfbb = palloc(sizeof(GISTBuildBuffers));
+ 	gfbb->pagesPerBuffer = pagesPerBuffer;
+ 	gfbb->levelStep = levelStep;
+ 	gistInitBuildBuffers(gfbb, gistGetMaxLevel(index));
+ 
+ 	buildstate->giststate.gfbb = gfbb;
+ 
+ 	return true;
+ }
*** /dev/null
--- b/src/backend/access/gist/gistbuildbuffers.c
***************
*** 0 ****
--- 1,795 ----
+ /*-------------------------------------------------------------------------
+  *
+  * gistbuildbuffers.c
+  *	  node buffer management functions for GiST buffering build algorithm.
+  *
+  *
+  * Portions Copyright (c) 1996-2011, PostgreSQL Global Development Group
+  * Portions Copyright (c) 1994, Regents of the University of California
+  *
+  * IDENTIFICATION
+  *	  src/backend/access/gist/gistbuildbuffers.c
+  *
+  *-------------------------------------------------------------------------
+  */
+ #include "postgres.h"
+ 
+ #include "access/genam.h"
+ #include "access/gist_private.h"
+ #include "catalog/index.h"
+ #include "catalog/pg_collation.h"
+ #include "miscadmin.h"
+ #include "storage/buffile.h"
+ #include "storage/bufmgr.h"
+ #include "storage/indexfsm.h"
+ #include "utils/memutils.h"
+ #include "utils/rel.h"
+ 
+ static GISTNodeBufferPage *gistAllocateNewPageBuffer(GISTBuildBuffers *gfbb);
+ static void gistAddLoadedBuffer(GISTBuildBuffers *gfbb, BlockNumber blocknum);
+ static void gistLoadNodeBuffer(GISTBuildBuffers *gfbb, GISTNodeBuffer *nodeBuffer);
+ static void gistUnloadNodeBuffer(GISTBuildBuffers *gfbb, GISTNodeBuffer *nodeBuffer);
+ static void gistPlaceItupToPage(GISTNodeBufferPage *pageBuffer, IndexTuple item);
+ static void gistGetItupFromPage(GISTNodeBufferPage *pageBuffer, IndexTuple *item);
+ static int	gistBuffersFreeBlocksCmp(const void *a, const void *b);
+ static long gistBuffersGetFreeBlock(GISTBuildBuffers *gfbb);
+ static void gistBuffersReleaseBlock(GISTBuildBuffers *gfbb, long blocknum);
+ 
+ /*
+  * Initialize GiST buffering build data structure.
+  */
+ void
+ gistInitBuildBuffers(GISTBuildBuffers *gfbb, int maxLevel)
+ {
+ 	HASHCTL		hashCtl;
+ 
+ 	/*
+ 	 * Create a temporary file to hold buffer pages that are swapped out
+ 	 * of memory. Initialize data structures for free pages management.
+ 	 */
+ 	gfbb->pfile = BufFileCreateTemp(true);
+ 	gfbb->nFileBlocks = 0;
+ 	gfbb->nFreeBlocks = 0;
+ 	gfbb->blocksSorted = false;
+ 	gfbb->freeBlocksLen = 32;
+ 	gfbb->freeBlocks = (long *) palloc(gfbb->freeBlocksLen * sizeof(long));
+ 
+ 	/*
+ 	 * Current memory context will be used for all in-memory data structures
+ 	 * of buffers which are persistent during buffering build.
+ 	 */
+ 	gfbb->context = CurrentMemoryContext;
+ 
+ 	/*
+ 	 * nodeBuffersTab hash is association between index blocks and it's
+ 	 * buffers.
+ 	 */
+ 	hashCtl.keysize = sizeof(BlockNumber);
+ 	hashCtl.entrysize = sizeof(GISTNodeBuffer);
+ 	hashCtl.hcxt = CurrentMemoryContext;
+ 	hashCtl.hash = tag_hash;
+ 	hashCtl.match = memcmp;
+ 	gfbb->nodeBuffersTab = hash_create("gistbuildbuffers",
+ 									   1024,
+ 									   &hashCtl,
+ 									   HASH_ELEM | HASH_CONTEXT
+ 									   | HASH_FUNCTION | HASH_COMPARE);
+ 
+ 	gfbb->bufferEmptyingQueue = NIL;
+ 
+ 	gfbb->currentEmptyingBufferBlockNumber = InvalidBlockNumber;
+ 	gfbb->currentEmptyingBufferSplit = false;
+ 
+ 	/*
+ 	 * Per-level node buffers lists for final buffers emptying process. Node
+ 	 * buffers are inserted here when they are created.
+ 	 */
+ 	gfbb->buffersOnLevelsLen = 1;
+ 	gfbb->buffersOnLevels = (List **) palloc(sizeof(List *) *
+ 											 gfbb->buffersOnLevelsLen);
+ 	gfbb->buffersOnLevels[0] = NIL;
+ 
+ 	/*
+ 	 * Block numbers of node buffers which last pages are currently loaded
+ 	 * into main memory.
+ 	 */
+ 	gfbb->loadedBuffersLen = 32;
+ 	gfbb->loadedBuffers = (BlockNumber *) palloc(gfbb->loadedBuffersLen *
+ 												 sizeof(BlockNumber));
+ 	gfbb->loadedBuffersCount = 0;
+ 
+ 	/*
+ 	 * Root path item of the tree. Updated on each root node split.
+ 	 */
+ 	gfbb->rootitem = (GISTBufferingInsertStack *) MemoryContextAlloc(
+ 							gfbb->context, sizeof(GISTBufferingInsertStack));
+ 	gfbb->rootitem->parent = NULL;
+ 	gfbb->rootitem->blkno = GIST_ROOT_BLKNO;
+ 	gfbb->rootitem->downlinkoffnum = InvalidOffsetNumber;
+ 	gfbb->rootitem->level = maxLevel;
+ 	gfbb->rootitem->refCount = 1;
+ }
+ 
+ /*
+  * Returns a node buffer for given block. The buffer is created if it
+  * doesn't exist yet.
+  */
+ GISTNodeBuffer *
+ gistGetNodeBuffer(GISTBuildBuffers *gfbb, GISTSTATE *giststate,
+ 				  BlockNumber nodeBlocknum,
+ 				  OffsetNumber downlinkoffnum,
+ 				  GISTBufferingInsertStack *parent)
+ {
+ 	GISTNodeBuffer *nodeBuffer;
+ 	bool		found;
+ 
+ 	/* Find node buffer in hash table */
+ 	nodeBuffer = (GISTNodeBuffer *) hash_search(gfbb->nodeBuffersTab,
+ 												(const void *) &nodeBlocknum,
+ 												HASH_ENTER,
+ 												&found);
+ 	if (!found)
+ 	{
+ 		/*
+ 		 * Node buffer wasn't found. Initialize the new buffer as empty.
+ 		 */
+ 		GISTBufferingInsertStack *path;
+ 		int			level;
+ 		MemoryContext oldcxt = MemoryContextSwitchTo(gfbb->context);
+ 
+ 		nodeBuffer->pageBuffer = NULL;
+ 		nodeBuffer->blocksCount = 0;
+ 		nodeBuffer->queuedForEmptying = false;
+ 
+ 		/*
+ 		 * Create a path stack for the page.
+ 		 */
+ 		if (nodeBlocknum != GIST_ROOT_BLKNO)
+ 		{
+ 			path = (GISTBufferingInsertStack *) palloc(
+ 										   sizeof(GISTBufferingInsertStack));
+ 			path->parent = parent;
+ 			path->blkno = nodeBlocknum;
+ 			path->downlinkoffnum = downlinkoffnum;
+ 			path->level = parent->level - 1;
+ 			path->refCount = 0;		/* initially unreferenced */
+ 			parent->refCount++;		/* this path references its parent */
+ 			Assert(path->level > 0);
+ 		}
+ 		else
+ 			path = gfbb->rootitem;
+ 
+ 		nodeBuffer->path = path;
+ 		path->refCount++;
+ 
+ 		/*
+ 		 * Add this buffer to the list of buffers on this level. Enlarge
+ 		 * buffersOnLevels array if needed.
+ 		 */
+ 		level = path->level;
+ 		if (level >= gfbb->buffersOnLevelsLen)
+ 		{
+ 			int			i;
+ 
+ 			gfbb->buffersOnLevels =
+ 				(List **) repalloc(gfbb->buffersOnLevels,
+ 								   (level + 1) * sizeof(List *));
+ 
+ 			/* initialize the enlarged portion */
+ 			for (i = gfbb->buffersOnLevelsLen; i <= level; i++)
+ 				gfbb->buffersOnLevels[i] = NIL;
+ 			gfbb->buffersOnLevelsLen = level + 1;
+ 		}
+ 
+ 		gfbb->buffersOnLevels[level] = lcons(nodeBuffer,
+ 											 gfbb->buffersOnLevels[level]);
+ 
+ 		MemoryContextSwitchTo(oldcxt);
+ 	}
+ 	else
+ 	{
+ 		if (parent != nodeBuffer->path->parent)
+ 		{
+ 			/*
+ 			 * Other parent path item was provided than we've remembered. We
+ 			 * trust caller to provide more correct parent than we have.
+ 			 * Previous parent may be outdated by page split.
+ 			 */
+ 			gistDecreasePathRefcount(nodeBuffer->path->parent);
+ 			nodeBuffer->path->parent = parent;
+ 			parent->refCount++;
+ 		}
+ 	}
+ 
+ 	return nodeBuffer;
+ }
+ 
+ /*
+  * Allocate memory for a buffer page.
+  */
+ static GISTNodeBufferPage *
+ gistAllocateNewPageBuffer(GISTBuildBuffers *gfbb)
+ {
+ 	GISTNodeBufferPage *pageBuffer;
+ 
+ 	pageBuffer = (GISTNodeBufferPage *) MemoryContextAlloc(gfbb->context,
+ 														   BLCKSZ);
+ 	pageBuffer->prev = InvalidBlockNumber;
+ 
+ 	/* Set page free space */
+ 	PAGE_FREE_SPACE(pageBuffer) = BLCKSZ - BUFFER_PAGE_DATA_OFFSET;
+ 	return pageBuffer;
+ }
+ 
+ /*
+  * Add specified block number into loadedBuffers array.
+  */
+ static void
+ gistAddLoadedBuffer(GISTBuildBuffers *gfbb, BlockNumber blocknum)
+ {
+ 	/* Enlarge the array if needed */
+ 	if (gfbb->loadedBuffersCount >= gfbb->loadedBuffersLen)
+ 	{
+ 		gfbb->loadedBuffersLen *= 2;
+ 		gfbb->loadedBuffers = (BlockNumber *) repalloc(gfbb->loadedBuffers,
+ 													 gfbb->loadedBuffersLen *
+ 													   sizeof(BlockNumber));
+ 	}
+ 
+ 	gfbb->loadedBuffers[gfbb->loadedBuffersCount] = blocknum;
+ 	gfbb->loadedBuffersCount++;
+ }
+ 
+ 
+ /*
+  * Load last page of node buffer into main memory.
+  */
+ static void
+ gistLoadNodeBuffer(GISTBuildBuffers *gfbb, GISTNodeBuffer *nodeBuffer)
+ {
+ 	/* Check if we really should load something */
+ 	if (!nodeBuffer->pageBuffer && nodeBuffer->blocksCount > 0)
+ 	{
+ 		/* Allocate memory for page */
+ 		nodeBuffer->pageBuffer = gistAllocateNewPageBuffer(gfbb);
+ 
+ 		/* Read block from temporary file */
+ 		BufFileSeekBlock(gfbb->pfile, nodeBuffer->pageBlocknum);
+ 		BufFileRead(gfbb->pfile, nodeBuffer->pageBuffer, BLCKSZ);
+ 
+ 		/* Mark file block as free */
+ 		gistBuffersReleaseBlock(gfbb, nodeBuffer->pageBlocknum);
+ 
+ 		/* Mark node buffer as loaded */
+ 		gistAddLoadedBuffer(gfbb, nodeBuffer->nodeBlocknum);
+ 		nodeBuffer->pageBlocknum = InvalidBlockNumber;
+ 	}
+ }
+ 
+ /*
+  * Write last page of node buffer to the disk.
+  */
+ static void
+ gistUnloadNodeBuffer(GISTBuildBuffers *gfbb, GISTNodeBuffer *nodeBuffer)
+ {
+ 	/* Check if we have something to write */
+ 	if (nodeBuffer->pageBuffer)
+ 	{
+ 		BlockNumber blkno;
+ 
+ 		/* Get free file block */
+ 		blkno = gistBuffersGetFreeBlock(gfbb);
+ 
+ 		/* Write block to the temporary file */
+ 		BufFileSeekBlock(gfbb->pfile, blkno);
+ 		BufFileWrite(gfbb->pfile, nodeBuffer->pageBuffer, BLCKSZ);
+ 
+ 		/* Free memory of that page */
+ 		pfree(nodeBuffer->pageBuffer);
+ 		nodeBuffer->pageBuffer = NULL;
+ 
+ 		/* Save block number */
+ 		nodeBuffer->pageBlocknum = blkno;
+ 	}
+ }
+ 
+ /*
+  * Write last pages of all node buffers to the disk.
+  */
+ void
+ gistUnloadNodeBuffers(GISTBuildBuffers *gfbb)
+ {
+ 	int			i;
+ 
+ 	/* Iterate over node buffers which last page is loaded into main memory */
+ 	for (i = 0; i < gfbb->loadedBuffersCount; i++)
+ 	{
+ 		GISTNodeBuffer *nodeBuffer;
+ 		bool		found;
+ 
+ 		/* Find node buffer by its block number */
+ 		nodeBuffer = hash_search(gfbb->nodeBuffersTab, &gfbb->loadedBuffers[i],
+ 								 HASH_FIND, &found);
+ 
+ 		/*
+ 		 * Node buffer can be not found. It can disappear during page split.
+ 		 * So, it's ok, just skip it.
+ 		 */
+ 		if (!found)
+ 			continue;
+ 
+ 		/* Unload last page to the disk */
+ 		gistUnloadNodeBuffer(gfbb, nodeBuffer);
+ 	}
+ 	/* Now there are no node buffers with loaded last page */
+ 	gfbb->loadedBuffersCount = 0;
+ }
+ 
+ /*
+  * Add index tuple to buffer page.
+  */
+ static void
+ gistPlaceItupToPage(GISTNodeBufferPage *pageBuffer, IndexTuple itup)
+ {
+ 	/*
+ 	 * Get pointer to the start of free space on the page
+ 	 */
+ 	char	   *ptr = (char *) pageBuffer + BUFFER_PAGE_DATA_OFFSET
+ 	+ PAGE_FREE_SPACE(pageBuffer) - MAXALIGN(IndexTupleSize(itup));
+ 
+ 	/*
+ 	 * There should be enough of space
+ 	 */
+ 	Assert(PAGE_FREE_SPACE(pageBuffer) >= MAXALIGN(IndexTupleSize(itup)));
+ 
+ 	/*
+ 	 * Reduce free space value of page
+ 	 */
+ 	PAGE_FREE_SPACE(pageBuffer) -= MAXALIGN(IndexTupleSize(itup));
+ 
+ 	/*
+ 	 * Copy index tuple to free space
+ 	 */
+ 	memcpy(ptr, itup, IndexTupleSize(itup));
+ }
+ 
+ /*
+  * Get last item from buffer page and remove it from page.
+  */
+ static void
+ gistGetItupFromPage(GISTNodeBufferPage *pageBuffer, IndexTuple *itup)
+ {
+ 	/*
+ 	 * Get pointer to last index tuple
+ 	 */
+ 	IndexTuple	ptr = (IndexTuple) ((char *) pageBuffer
+ 									+ BUFFER_PAGE_DATA_OFFSET
+ 									+ PAGE_FREE_SPACE(pageBuffer));
+ 
+ 	/*
+ 	 * Page shouldn't be empty
+ 	 */
+ 	Assert(!PAGE_IS_EMPTY(pageBuffer));
+ 
+ 	/*
+ 	 * Allocate memory for returned index tuple copy
+ 	 */
+ 	*itup = (IndexTuple) palloc(IndexTupleSize(ptr));
+ 
+ 	/*
+ 	 * Copy data
+ 	 */
+ 	memcpy(*itup, ptr, IndexTupleSize(ptr));
+ 
+ 	/*
+ 	 * Increase free space value of page
+ 	 */
+ 	PAGE_FREE_SPACE(pageBuffer) += MAXALIGN(IndexTupleSize(*itup));
+ }
+ 
+ /*
+  * Push an index tuple to node buffer.
+  */
+ void
+ gistPushItupToNodeBuffer(GISTBuildBuffers *gfbb, GISTNodeBuffer *nodeBuffer,
+ 						 IndexTuple itup)
+ {
+ 	/*
+ 	 * Most part of memory operations will be in buffering build persistent
+ 	 * context. So, let's switch to it.
+ 	 */
+ 	MemoryContext oldcxt = MemoryContextSwitchTo(gfbb->context);
+ 
+ 	/* Is the buffer currently empty? */
+ 	if (nodeBuffer->blocksCount == 0)
+ 	{
+ 		/* It's empty, let's create the first page */
+ 		nodeBuffer->pageBuffer = gistAllocateNewPageBuffer(gfbb);
+ 		nodeBuffer->blocksCount = 1;
+ 		gistAddLoadedBuffer(gfbb, nodeBuffer->nodeBlocknum);
+ 	}
+ 
+ 	/* Load last page of node buffer if it wasn't already */
+ 	if (!nodeBuffer->pageBuffer)
+ 		gistLoadNodeBuffer(gfbb, nodeBuffer);
+ 
+ 	/*
+ 	 * Check if there is enough space on the last page for the tuple
+ 	 */
+ 	if (PAGE_NO_SPACE(nodeBuffer->pageBuffer, itup))
+ 	{
+ 		/*
+ 		 * Nope. Swap previous block to disk and allocate a new one.
+ 		 */
+ 		BlockNumber blkno;
+ 
+ 		/* Write filled page to the disk */
+ 		blkno = gistBuffersGetFreeBlock(gfbb);
+ 		BufFileSeekBlock(gfbb->pfile, blkno);
+ 		BufFileWrite(gfbb->pfile, nodeBuffer->pageBuffer, BLCKSZ);
+ 
+ 		/* Mark space of in-memory page as empty */
+ 		PAGE_FREE_SPACE(nodeBuffer->pageBuffer) =
+ 			BLCKSZ - MAXALIGN(offsetof(GISTNodeBufferPage, tupledata));
+ 
+ 		/* Save block number of the previous page */
+ 		nodeBuffer->pageBuffer->prev = blkno;
+ 
+ 		/* We've just added one more page */
+ 		nodeBuffer->blocksCount++;
+ 	}
+ 
+ 	gistPlaceItupToPage(nodeBuffer->pageBuffer, itup);
+ 
+ 	/*
+ 	 * If the buffer just overflowed, add it to the emptying queue.
+ 	 */
+ 	if (BUFFER_HALF_FILLED(nodeBuffer, gfbb) && !nodeBuffer->queuedForEmptying)
+ 	{
+ 		MemoryContextSwitchTo(gfbb->context);
+ 		gfbb->bufferEmptyingQueue =	lcons(nodeBuffer, gfbb->bufferEmptyingQueue);
+ 		nodeBuffer->queuedForEmptying = true;
+ 	}
+ 
+ 	/* Restore memory context */
+ 	MemoryContextSwitchTo(oldcxt);
+ }
+ 
+ /*
+  * Removes one index tuple from node buffer. Returns true if success and false
+  * if node buffer is empty.
+  */
+ bool
+ gistPopItupFromNodeBuffer(GISTBuildBuffers *gfbb, GISTNodeBuffer *nodeBuffer,
+ 						  IndexTuple *itup)
+ {
+ 	/*
+ 	 * If node buffer is empty then return false.
+ 	 */
+ 	if (nodeBuffer->blocksCount <= 0)
+ 		return false;
+ 
+ 	/* Load last page of node buffer if needed */
+ 	if (!nodeBuffer->pageBuffer)
+ 		gistLoadNodeBuffer(gfbb, nodeBuffer);
+ 
+ 	/*
+ 	 * Get index tuple from last non-empty page.
+ 	 */
+ 	gistGetItupFromPage(nodeBuffer->pageBuffer, itup);
+ 
+ 	/*
+ 	 * Check if the page which the index tuple was got from is now empty
+ 	 */
+ 	if (PAGE_IS_EMPTY(nodeBuffer->pageBuffer))
+ 	{
+ 		BlockNumber prevblkno;
+ 
+ 		/*
+ 		 * If it's empty then we need to release buffer file block and free
+ 		 * page buffer.
+ 		 */
+ 		nodeBuffer->blocksCount--;
+ 
+ 		/*
+ 		 * If there's more pages, fetch previous one
+ 		 */
+ 		prevblkno = nodeBuffer->pageBuffer->prev;
+ 		if (prevblkno != InvalidBlockNumber)
+ 		{
+ 			/* There actually is previous page, so read it. */
+ 			Assert(nodeBuffer->blocksCount > 0);
+ 			BufFileSeekBlock(gfbb->pfile, prevblkno);
+ 			BufFileRead(gfbb->pfile, nodeBuffer->pageBuffer, BLCKSZ);
+ 
+ 			/* Mark block as free */
+ 			gistBuffersReleaseBlock(gfbb, prevblkno);
+ 		}
+ 		else
+ 		{
+ 			/* Actually there are no more pages. Free memory. */
+ 			Assert(nodeBuffer->blocksCount == 0);
+ 			pfree(nodeBuffer->pageBuffer);
+ 			nodeBuffer->pageBuffer = NULL;
+ 		}
+ 	}
+ 	return true;
+ }
+ 
+ /*
+  * qsort comparator for sorting freeBlocks[] into decreasing order.
+  */
+ static int
+ gistBuffersFreeBlocksCmp(const void *a, const void *b)
+ {
+ 	long		ablk = *((const long *) a);
+ 	long		bblk = *((const long *) b);
+ 
+ 	/*
+ 	 * can't just subtract because long might be wider than int
+ 	 */
+ 	if (ablk < bblk)
+ 		return 1;
+ 	if (ablk > bblk)
+ 		return -1;
+ 	return 0;
+ }
+ 
+ /*
+  * Select a currently unused block for writing to.
+  *
+  * NB: should only be called when writer is ready to write immediately,
+  * to ensure that first write pass is sequential.
+  */
+ static long
+ gistBuffersGetFreeBlock(GISTBuildBuffers *gfbb)
+ {
+ 	/*
+ 	 * If there are multiple free blocks, we select the one appearing last in
+ 	 * freeBlocks[] (after sorting the array if needed).  If there are none,
+ 	 * assign the next block at the end of the file.
+ 	 */
+ 	if (gfbb->nFreeBlocks > 0)
+ 	{
+ 		if (!gfbb->blocksSorted)
+ 		{
+ 			qsort((void *) gfbb->freeBlocks, gfbb->nFreeBlocks,
+ 				  sizeof(long), gistBuffersFreeBlocksCmp);
+ 			gfbb->blocksSorted = true;
+ 		}
+ 		return gfbb->freeBlocks[--gfbb->nFreeBlocks];
+ 	}
+ 	else
+ 		return gfbb->nFileBlocks++;
+ }
+ 
+ /*
+  * Return a block# to the freelist.
+  */
+ static void
+ gistBuffersReleaseBlock(GISTBuildBuffers *gfbb, long blocknum)
+ {
+ 	int			ndx;
+ 
+ 	/*
+ 	 * Enlarge freeBlocks array if full.
+ 	 */
+ 	if (gfbb->nFreeBlocks >= gfbb->freeBlocksLen)
+ 	{
+ 		gfbb->freeBlocksLen *= 2;
+ 		gfbb->freeBlocks = (long *) repalloc(gfbb->freeBlocks,
+ 											 gfbb->freeBlocksLen *
+ 											 sizeof(long));
+ 	}
+ 
+ 	/*
+ 	 * Add blocknum to array, and mark the array unsorted if it's no longer in
+ 	 * decreasing order.
+ 	 */
+ 	ndx = gfbb->nFreeBlocks++;
+ 	gfbb->freeBlocks[ndx] = blocknum;
+ 	if (ndx > 0 && gfbb->freeBlocks[ndx - 1] < blocknum)
+ 		gfbb->blocksSorted = false;
+ }
+ 
+ /*
+  * Free buffering build data structure.
+  */
+ void
+ gistFreeBuildBuffers(GISTBuildBuffers *gfbb)
+ {
+ 	/* Close buffers file. */
+ 	BufFileClose(gfbb->pfile);
+ 
+ 	/* All other things will be freed on memory context release */
+ }
+ 
+ /*
+  * Data structure representing information about node buffer for index tuples
+  * relocation from splitted node buffer.
+  */
+ typedef struct
+ {
+ 	GISTENTRY	entry[INDEX_MAX_KEYS];
+ 	bool		isnull[INDEX_MAX_KEYS];
+ 	GISTPageSplitInfo *splitinfo;
+ 	GISTNodeBuffer *nodeBuffer;
+ } RelocationBufferInfo;
+ 
+ /*
+  * Maintain data structures on page split.
+  */
+ void
+ gistRelocateBuildBuffersOnSplit(GISTBuildBuffers *gfbb, GISTSTATE *giststate,
+ 								Relation r, GISTBufferingInsertStack *path,
+ 								Buffer buffer, List *splitinfo)
+ {
+ 	RelocationBufferInfo *relocationBuffersInfos;
+ 	bool		found;
+ 	GISTNodeBuffer *nodeBuffer;
+ 	BlockNumber blocknum;
+ 	IndexTuple	itup;
+ 	int			splitPagesCount = 0,
+ 				i;
+ 	GISTENTRY	entry[INDEX_MAX_KEYS];
+ 	bool		isnull[INDEX_MAX_KEYS];
+ 	GISTNodeBuffer nodebuf;
+ 	ListCell   *lc;
+ 
+ 	/*
+ 	 * If the splitted page level doesn't have buffers, we have nothing to do.
+ 	 */
+ 	if (!LEVEL_HAS_BUFFERS(path->level, gfbb))
+ 		return;
+ 
+ 	/*
+ 	 * Get pointer to node buffer of splitted page.
+ 	 */
+ 	blocknum = BufferGetBlockNumber(buffer);
+ 	nodeBuffer = hash_search(gfbb->nodeBuffersTab, &blocknum,
+ 							 HASH_FIND, &found);
+ 	if (!found)
+ 	{
+ 		/*
+ 		 * Node buffer should exist at this point. If it didn't exist before,
+ 		 * the insertion that caused the page to split should've created it.
+ 		 */
+ 		elog(ERROR, "node buffer of page being split (%u) does not exist",
+ 			 blocknum);
+ 	}
+ 
+ 	/*
+ 	 * Make a copy of the old buffer, as we're going reuse the old one as
+ 	 * the buffer for the new left page, which is on the same block as the
+ 	 * old page. That's not true for the root page, but that's fine because
+ 	 * we never have a buffer on the root page anyway. The original algorithm
+ 	 * as described by Arge et al did, but it's of no use, as you might as
+ 	 * well read the tuples straight from the heap instead of the root buffer.
+ 	 */
+ 	Assert(blocknum != GIST_ROOT_BLKNO);
+ 	memcpy(&nodebuf, nodeBuffer, sizeof(GISTNodeBuffer));
+ 
+ 	/* Reset the old buffer, used for the new left page from now on */
+ 	nodeBuffer->blocksCount = 0;
+ 	nodeBuffer->pageBuffer = NULL;
+ 	nodeBuffer->pageBlocknum = InvalidBlockNumber;
+ 
+ 	/* Reassign pointer to the saved copy. */
+ 	nodeBuffer = &nodebuf;
+ 
+ 	/*
+ 	 * Allocate memory for information about relocation buffers.
+ 	 */
+ 	splitPagesCount = list_length(splitinfo);
+ 	relocationBuffersInfos =
+ 		(RelocationBufferInfo *) palloc(sizeof(RelocationBufferInfo) *
+ 										splitPagesCount);
+ 
+ 	/*
+ 	 * Fill relocation buffers information for node buffers of pages produced
+ 	 * by split.
+ 	 */
+ 	i = 0;
+ 	foreach(lc, splitinfo)
+ 	{
+ 		GISTPageSplitInfo *si = (GISTPageSplitInfo *) lfirst(lc);
+ 		GISTNodeBuffer *newNodeBuffer;
+ 
+ 		/* Decompress parent index tuple of node buffer page. */
+ 		gistDeCompressAtt(giststate, r,
+ 						  si->downlink, NULL, (OffsetNumber) 0,
+ 						  relocationBuffersInfos[i].entry,
+ 						  relocationBuffersInfos[i].isnull);
+ 
+ 		newNodeBuffer = gistGetNodeBuffer(gfbb, giststate, BufferGetBlockNumber(si->buf),
+ 								   path->downlinkoffnum, path->parent);
+ 
+ 		relocationBuffersInfos[i].nodeBuffer = newNodeBuffer;
+ 		relocationBuffersInfos[i].splitinfo = si;
+ 
+ 		i++;
+ 	}
+ 
+ 	/*
+ 	 * Loop through all index tuples on the buffer on the splitted page,
+ 	 * moving all the tuples to the buffers on the new pages.
+ 	 */
+ 	while (gistPopItupFromNodeBuffer(gfbb, nodeBuffer, &itup))
+ 	{
+ 		float		sum_grow,
+ 					which_grow[INDEX_MAX_KEYS];
+ 		int			i,
+ 					which;
+ 		IndexTuple	newtup;
+ 
+ 		/*
+ 		 * Choose which page this tuple should go to.
+ 		 */
+ 		gistDeCompressAtt(giststate, r,
+ 						  itup, NULL, (OffsetNumber) 0, entry, isnull);
+ 
+ 		which = -1;
+ 		*which_grow = -1.0f;
+ 		sum_grow = 1.0f;
+ 
+ 		for (i = 0; i < splitPagesCount && sum_grow; i++)
+ 		{
+ 			int			j;
+ 			RelocationBufferInfo *splitPageInfo = &relocationBuffersInfos[i];
+ 
+ 			sum_grow = 0.0f;
+ 			for (j = 0; j < r->rd_att->natts; j++)
+ 			{
+ 				float		usize;
+ 
+ 				usize = gistpenalty(giststate, j,
+ 									&splitPageInfo->entry[j],
+ 									splitPageInfo->isnull[j],
+ 									&entry[j], isnull[j]);
+ 
+ 				if (which_grow[j] < 0 || usize < which_grow[j])
+ 				{
+ 					which = i;
+ 					which_grow[j] = usize;
+ 					if (j < r->rd_att->natts - 1 && i == 0)
+ 						which_grow[j + 1] = -1;
+ 					sum_grow += which_grow[j];
+ 				}
+ 				else if (which_grow[j] == usize)
+ 					sum_grow += usize;
+ 				else
+ 				{
+ 					sum_grow = 1;
+ 					break;
+ 				}
+ 			}
+ 		}
+ 
+ 		/*
+ 		 * push item to selected node buffer
+ 		 */
+ 		gistPushItupToNodeBuffer(gfbb, relocationBuffersInfos[which].nodeBuffer,
+ 								 itup);
+ 
+ 		/*
+ 		 * Adjust the downlink for this page, if needed.
+ 		 */
+ 		newtup = gistgetadjusted(r, relocationBuffersInfos[which].splitinfo->downlink,
+ 								 itup, giststate);
+ 		if (newtup)
+ 		{
+ 			gistDeCompressAtt(giststate, r,
+ 							  newtup, NULL, (OffsetNumber) 0,
+ 							  relocationBuffersInfos[which].entry,
+ 							  relocationBuffersInfos[which].isnull);
+ 
+ 			relocationBuffersInfos[which].splitinfo->downlink = newtup;
+ 		}
+ 	}
+ 
+ 	/* Report about splitting for current emptying buffer */
+ 	if (blocknum == gfbb->currentEmptyingBufferBlockNumber)
+ 		gfbb->currentEmptyingBufferSplit = true;
+ 
+ 	pfree(relocationBuffersInfos);
+ }
*** a/src/backend/access/gist/gistutil.c
--- b/src/backend/access/gist/gistutil.c
***************
*** 670,682 **** gistoptions(PG_FUNCTION_ARGS)
  {
  	Datum		reloptions = PG_GETARG_DATUM(0);
  	bool		validate = PG_GETARG_BOOL(1);
! 	bytea	   *result;
  
! 	result = default_reloptions(reloptions, validate, RELOPT_KIND_GIST);
  
- 	if (result)
- 		PG_RETURN_BYTEA_P(result);
- 	PG_RETURN_NULL();
  }
  
  /*
--- 670,699 ----
  {
  	Datum		reloptions = PG_GETARG_DATUM(0);
  	bool		validate = PG_GETARG_BOOL(1);
! 	relopt_value *options;
! 	GiSTOptions *rdopts;
! 	int			numoptions;
! 	static const relopt_parse_elt tab[] = {
! 		{"fillfactor", RELOPT_TYPE_INT, offsetof(GiSTOptions, fillfactor)},
! 		{"buffering", RELOPT_TYPE_STRING, offsetof(GiSTOptions, bufferingModeOffset)}
! 	};
  
! 	options = parseRelOptions(reloptions, validate, RELOPT_KIND_GIST,
! 							  &numoptions);
! 
! 	/* if none set, we're done */
! 	if (numoptions == 0)
! 		PG_RETURN_NULL();
! 
! 	rdopts = allocateReloptStruct(sizeof(GiSTOptions), options, numoptions);
! 
! 	fillRelOptions((void *) rdopts, sizeof(GiSTOptions), options, numoptions,
! 				   validate, tab, lengthof(tab));
! 
! 	pfree(options);
! 
! 	PG_RETURN_BYTEA_P(rdopts);
  
  }
  
  /*
*** a/src/backend/access/gist/gistxlog.c
--- b/src/backend/access/gist/gistxlog.c
***************
*** 266,272 **** gistRedoPageSplitRecord(XLogRecPtr lsn, XLogRecord *record)
  			else
  				GistPageGetOpaque(page)->rightlink = xldata->origrlink;
  			GistPageGetOpaque(page)->nsn = xldata->orignsn;
! 			if (i < xlrec.data->npage - 1 && !isrootsplit)
  				GistMarkFollowRight(page);
  			else
  				GistClearFollowRight(page);
--- 266,273 ----
  			else
  				GistPageGetOpaque(page)->rightlink = xldata->origrlink;
  			GistPageGetOpaque(page)->nsn = xldata->orignsn;
! 			if (i < xlrec.data->npage - 1 && !isrootsplit &&
! 				!xldata->noFollowRight)
  				GistMarkFollowRight(page);
  			else
  				GistClearFollowRight(page);
***************
*** 414,420 **** XLogRecPtr
  gistXLogSplit(RelFileNode node, BlockNumber blkno, bool page_is_leaf,
  			  SplitedPageLayout *dist,
  			  BlockNumber origrlink, GistNSN orignsn,
! 			  Buffer leftchildbuf)
  {
  	XLogRecData *rdata;
  	gistxlogPageSplit xlrec;
--- 415,421 ----
  gistXLogSplit(RelFileNode node, BlockNumber blkno, bool page_is_leaf,
  			  SplitedPageLayout *dist,
  			  BlockNumber origrlink, GistNSN orignsn,
! 			  Buffer leftchildbuf, bool noFollowFight)
  {
  	XLogRecData *rdata;
  	gistxlogPageSplit xlrec;
***************
*** 436,441 **** gistXLogSplit(RelFileNode node, BlockNumber blkno, bool page_is_leaf,
--- 437,443 ----
  	xlrec.npage = (uint16) npage;
  	xlrec.leftchild =
  		BufferIsValid(leftchildbuf) ? BufferGetBlockNumber(leftchildbuf) : InvalidBlockNumber;
+ 	xlrec.noFollowRight = noFollowFight;
  
  	rdata[0].data = (char *) &xlrec;
  	rdata[0].len = sizeof(gistxlogPageSplit);
*** a/src/include/access/gist_private.h
--- b/src/include/access/gist_private.h
***************
*** 17,29 ****
--- 17,72 ----
  #include "access/gist.h"
  #include "access/itup.h"
  #include "storage/bufmgr.h"
+ #include "storage/buffile.h"
  #include "utils/rbtree.h"
+ #include "utils/hsearch.h"
+ 
+ /* Has specified level buffers? */
+ #define LEVEL_HAS_BUFFERS(nlevel, gfbb) ((nlevel) != 0 && (nlevel) % (gfbb)->levelStep == 0 && nlevel != (gfbb)->rootitem->level)
+ /* Is specified buffer at least half-filled (should be planned for emptying)?*/
+ #define BUFFER_HALF_FILLED(nodeBuffer, gfbb) ((nodeBuffer)->blocksCount > (gfbb)->pagesPerBuffer / 2)
+ /* Is specified buffer overflowed (can't take index tuples anymore)?*/
+ #define BUFFER_OVERFLOWED(nodeBuffer, gfbb) ((nodeBuffer)->blocksCount > (gfbb)->pagesPerBuffer)
  
  /* Buffer lock modes */
  #define GIST_SHARE	BUFFER_LOCK_SHARE
  #define GIST_EXCLUSIVE	BUFFER_LOCK_EXCLUSIVE
  #define GIST_UNLOCK BUFFER_LOCK_UNLOCK
  
+ typedef struct
+ {
+ 	BlockNumber prev;
+ 	uint32		freespace;
+ 	char		tupledata[1];
+ } GISTNodeBufferPage;
+ 
+ #define BUFFER_PAGE_DATA_OFFSET MAXALIGN(offsetof(GISTNodeBufferPage, tupledata))
+ /* Returns free space in node buffer page */
+ #define PAGE_FREE_SPACE(nbp) (nbp->freespace)
+ /* Checks if node buffer page is empty */
+ #define PAGE_IS_EMPTY(nbp) (nbp->freespace == BLCKSZ - BUFFER_PAGE_DATA_OFFSET)
+ /* Checks if node buffers page don't contain sufficient space for index tuple */
+ #define PAGE_NO_SPACE(nbp, itup) (PAGE_FREE_SPACE(nbp) < \
+ 										MAXALIGN(IndexTupleSize(itup)))
+ 
+ /* Buffer of tree node data structure */
+ typedef struct
+ {
+ 	/* number of page containing node */
+ 	BlockNumber nodeBlocknum;
+ 
+ 	/* count of blocks occupied by buffer */
+ 	int32		blocksCount;
+ 
+ 	BlockNumber pageBlocknum;
+ 	GISTNodeBufferPage *pageBuffer;
+ 
+ 	/* is this buffer queued for emptying? */
+ 	bool		queuedForEmptying;
+ 
+ 	struct GISTBufferingInsertStack *path;
+ } GISTNodeBuffer;
+ 
  /*
   * GISTSTATE: information needed for any GiST index operation
   *
***************
*** 44,49 **** typedef struct GISTSTATE
--- 87,94 ----
  	/* Collations to pass to the support functions */
  	Oid			supportCollation[INDEX_MAX_KEYS];
  
+ 	struct GISTBuildBuffers *gfbb;
+ 
  	TupleDesc	tupdesc;
  } GISTSTATE;
  
***************
*** 170,175 **** typedef struct gistxlogPageSplit
--- 215,221 ----
  
  	BlockNumber leftchild;		/* like in gistxlogPageUpdate */
  	uint16		npage;			/* # of pages in the split */
+ 	bool		noFollowRight;	/* skip followRight flag setting */
  
  	/*
  	 * follow: 1. gistxlogPage and array of IndexTupleData per page
***************
*** 225,230 **** typedef struct GISTInsertStack
--- 271,346 ----
  	struct GISTInsertStack *parent;
  } GISTInsertStack;
  
+ /*
+  * Extended GISTInsertStack for buffering GiST index build. It additionally hold
+  * level number of page.
+  */
+ typedef struct GISTBufferingInsertStack
+ {
+ 	/* current page */
+ 	BlockNumber blkno;
+ 
+ 	/* offset of the downlink in the parent page, that points to this page */
+ 	OffsetNumber downlinkoffnum;
+ 
+ 	/* pointer to parent */
+ 	struct GISTBufferingInsertStack *parent;
+ 
+ 	int			refCount;
+ 
+ 	/* level number */
+ 	int			level;
+ }	GISTBufferingInsertStack;
+ 
+ /*
+  * Data structure with general information about build buffers.
+  */
+ typedef struct GISTBuildBuffers
+ {
+ 	/* memory context which is persistent during buffering build */
+ 	MemoryContext context;
+ 	/* underlying files */
+ 	BufFile    *pfile;
+ 	/* # of blocks used in underlying files */
+ 	long		nFileBlocks;
+ 	/* is freeBlocks[] currently in order? */
+ 	bool		blocksSorted;
+ 	/* resizable array of free blocks */
+ 	long	   *freeBlocks;
+ 	/* # of currently free blocks */
+ 	int			nFreeBlocks;
+ 	/* current allocated length of freeBlocks[] */
+ 	int			freeBlocksLen;
+ 
+ 	/* hash for buffers by block number */
+ 	HTAB	   *nodeBuffersTab;
+ 
+ 	/* stack of buffers for emptying */
+ 	List	   *bufferEmptyingQueue;
+ 	/* number of currently emptying buffer */
+ 	BlockNumber currentEmptyingBufferBlockNumber;
+ 	/* whether currently emptying buffer was split - a signal to stop emptying */
+ 	bool		currentEmptyingBufferSplit;
+ 
+ 	/* step of levels for buffers location */
+ 	int			levelStep;
+ 	/* maximal number of pages occupied by buffer */
+ 	int			pagesPerBuffer;
+ 
+ 	/* array of lists of non-empty buffers on levels for final emptying */
+ 	List	  **buffersOnLevels;
+ 	int			buffersOnLevelsLen;
+ 
+ 	/*
+ 	 * Dynamically-sized array of block numbers of buffers loaded into main
+ 	 * memory.
+ 	 */
+ 	BlockNumber *loadedBuffers;
+ 	int			loadedBuffersCount;		/* entries currently in loadedBuffers */
+ 	int			loadedBuffersLen;		/* allocated size of loadedBuffers */
+ 	GISTBufferingInsertStack *rootitem;
+ }	GISTBuildBuffers;
+ 
  typedef struct GistSplitVector
  {
  	GIST_SPLITVEC splitVector;	/* to/from PickSplit method */
***************
*** 286,291 **** extern Datum gistinsert(PG_FUNCTION_ARGS);
--- 402,424 ----
  extern MemoryContext createTempGistContext(void);
  extern void initGISTstate(GISTSTATE *giststate, Relation index);
  extern void freeGISTstate(GISTSTATE *giststate);
+ extern void gistdoinsert(Relation r,
+ 			 IndexTuple itup,
+ 			 Size freespace,
+ 			 GISTSTATE *GISTstate);
+ 
+ /* A List of these is returned from gistplacetopage() in *splitinfo */
+ typedef struct
+ {
+ 	Buffer		buf;			/* the split page "half" */
+ 	IndexTuple	downlink;		/* downlink for this half. */
+ } GISTPageSplitInfo;
+ 
+ extern bool gistplacetopage(GISTInsertState *state, GISTSTATE *giststate,
+ 				Buffer buffer,
+ 				IndexTuple *itup, int ntup, OffsetNumber oldoffnum,
+ 				Buffer leftchildbuf,
+ 				List **splitinfo);
  
  extern SplitedPageLayout *gistSplit(Relation r, Page page, IndexTuple *itup,
  		  int len, GISTSTATE *giststate);
***************
*** 305,311 **** extern XLogRecPtr gistXLogSplit(RelFileNode node,
  			  BlockNumber blkno, bool page_is_leaf,
  			  SplitedPageLayout *dist,
  			  BlockNumber origrlink, GistNSN oldnsn,
! 			  Buffer leftchild);
  
  /* gistget.c */
  extern Datum gistgettuple(PG_FUNCTION_ARGS);
--- 438,444 ----
  			  BlockNumber blkno, bool page_is_leaf,
  			  SplitedPageLayout *dist,
  			  BlockNumber origrlink, GistNSN oldnsn,
! 			  Buffer leftchild, bool noFollowFight);
  
  /* gistget.c */
  extern Datum gistgettuple(PG_FUNCTION_ARGS);
***************
*** 313,318 **** extern Datum gistgetbitmap(PG_FUNCTION_ARGS);
--- 446,461 ----
  
  /* gistutil.c */
  
+ /*
+  * Storage type for GiST's reloptions
+  */
+ typedef struct GiSTOptions
+ {
+ 	int32		vl_len_;		/* varlena header (do not touch directly!) */
+ 	int			fillfactor;		/* page fill factor in percent (0..100) */
+ 	int			bufferingModeOffset;	/* use buffering build? */
+ }	GiSTOptions;
+ 
  #define GiSTPageSize   \
  	( BLCKSZ - SizeOfPageHeaderData - MAXALIGN(sizeof(GISTPageOpaqueData)) )
  
***************
*** 380,383 **** extern void gistSplitByKey(Relation r, Page page, IndexTuple *itup,
--- 523,546 ----
  			   GistSplitVector *v, GistEntryVector *entryvec,
  			   int attno);
  
+ /* gistbuild.c */
+ extern void gistDecreasePathRefcount(GISTBufferingInsertStack *path);
+ extern void gistValidateBufferingOption(char *value);
+ 
+ /* gistbuildbuffers.c */
+ extern void gistInitBuildBuffers(GISTBuildBuffers *gfbb, int maxLevel);
+ GISTNodeBuffer *gistGetNodeBuffer(GISTBuildBuffers *gfbb, GISTSTATE *giststate,
+ 				  BlockNumber blkno, OffsetNumber downlinkoffnu,
+ 				  GISTBufferingInsertStack *parent);
+ extern void gistPushItupToNodeBuffer(GISTBuildBuffers *gfbb,
+ 						 GISTNodeBuffer *nodeBuffer, IndexTuple item);
+ extern bool gistPopItupFromNodeBuffer(GISTBuildBuffers *gfbb,
+ 						  GISTNodeBuffer *nodeBuffer, IndexTuple *item);
+ extern void gistFreeBuildBuffers(GISTBuildBuffers *gfbb);
+ extern void gistRelocateBuildBuffersOnSplit(GISTBuildBuffers *gfbb,
+ 								GISTSTATE *giststate, Relation r,
+ 							  GISTBufferingInsertStack *path, Buffer buffer,
+ 								List *splitinfo);
+ extern void gistUnloadNodeBuffers(GISTBuildBuffers *gfbb);
+ 
  #endif   /* GIST_PRIVATE_H */
