Parallel bitmap heap scan

Started by Dilip Kumarover 9 years ago128 messages
#1Dilip Kumar
dilipbalaut@gmail.com
5 attachment(s)

Hi Hackers,

I would like to propose parallel bitmap heap scan feature. After
running TPCH benchmark, It was observed that many of TPCH queries are
using bitmap scan (@TPCH_plan.tar.gz attached below). Keeping this
point in mind we thought that many query will get benefited with
parallel bitmap scan.

Robert has also pointed out the same thing in his blog related to parallel query
http://rhaas.blogspot.in/2016/04/postgresql-96-with-parallel-query-vs.html

Currently Bitmap heap plan look like this :
------------------------------------------------------
Bitmap Heap Scan
-> Bitmap Index Scan

After this patch :
---------------------
Parallel Bitmap Heap Scan
-> Bitmap Index Scan

As part of this work I have implemented parallel processing in
BitmapHeapScan node. BitmapIndexScan is still non parallel.

Brief design idea:
-----------------------
#1. Shared TIDBitmap creation and initialization
First worker to see the state as parallel bitmap info as PBM_INITIAL
become leader and set the state to PBM_INPROGRESS All other workers
see the state as PBM_INPROGRESS will wait for leader to complete the
TIDBitmap.

#2 At this level TIDBitmap is ready and all workers are awake.

#3. Bitmap processing (Iterate and process the pages).
In this phase each worker will iterate over page and chunk array and
select heap pages one by one. If prefetch is enable then there will be
two iterator. Since multiple worker are iterating over same page and
chunk array we need to have a shared iterator, so we grab a spin lock
and iterate within a lock, so that each worker get and different page
to process.

Note: For more detail on design, please refer comment of
BitmapHeapNext API in "parallel-bitmap-heap-scan-v1.patch" file.

Attached patch details:
------------------------------
1. parallel-bitmap-heap-scan-v1.patch: This is the main patch to make
bitmap heap scan node parallel aware.

2. dht-return-dsa-v1.patch: This patch will provide new API, where we
can scan full DHT[1], and get the dsa_pointers (a relative pointer).
The dsa_pointer values can be shared with other processes. We need
this because, after TIDBitmap is created, only one worker will process
whole TIDBitmap and convert it to a page and chunk array. So we need
to store the generic pointer, so that later on each worker can convert
those to their local pointer before start processing.

My patch depends on following patches.
------------------------------------------------------
1. conditional_variable
/messages/by-id/CAEepm=0zshYwB6wDeJCkrRJeoBM=jPYBe+-k_VtKRU_8zMLEfA@mail.gmail.com

2. dsa_area
/messages/by-id/CAEepm=024p-MeAsDmG=R3+tR4EGhuGJs_+rjFKF0eRoSTmMJnA@mail.gmail.com

3. Creating a DSA area to provide work space for parallel execution
/messages/by-id/CAEepm=0HmRefi1+xDJ99Gj5APHr8Qr05KZtAxrMj8b+ay3o6sA@mail.gmail.com

4. Hash table in dynamic shared memory (DHT) [1]
/messages/by-id/CAEepm=0VrMt3s_REDhQv6z1pHL7FETOD7Rt9V2MQ3r-2ss2ccA@mail.gmail.com

Order in which patches should be applied:
--------------------------------------------------------
1. conditional_variable
2. dsa_area
3. Creating a DSA area to provide work space for parallel execution
4. Hash table in dynamic shared memory.
5. dht-return-dsa-v1.patch
6. parallel-bitmap-heap-scan-v1.patch

Performance Results:
-----------------------------
Summary :
1. After this patch, I observed currently 4 queries are getting
significant improvement (Q4, Q6, Q14, Q15).
- Q4, is converting from parallel seqscan to parallel bitmap heap scan.
- Other queries are converted from a regular bitmap heap scan to a
parallel bitmap heap scan.
2. Benefit is more visible at lower workers (upto 4), after that some
of the queries are selecting ParallelSeqScan over ParallelBitmapScan.
And, I think this is expected, because so far we have only made
BitmapHeap node as parallel whereas ParallelSeqScan is completely
parallel so at higher worker count ParallelSeqScan is better choice.
3. Detailed result is attached @TPCH_PBMS.pdf
4. Explain analyse output is attached @TPCH_plan.tar.gz (for all
changed queries at worker 2)

TPCH query plan changed example (TPCH Q6):
----------------------------------------------------------------
On Head:
-------------

QUERY PLAN
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
Limit (cost=1558475.95..1558475.96 rows=1 width=32) (actual
time=40921.437..40921.438 rows=1 loops=1)
-> Aggregate (cost=1558475.95..1558475.96 rows=1 width=32)
(actual time=40921.435..40921.435 rows=1 loops=1)
-> Bitmap Heap Scan on lineitem (cost=291783.32..1552956.39
rows=1103911 width=12) (actual time=7032.075..38997.369 rows=1140434
loops=1)
Recheck Cond: ((l_shipdate >= '1994-01-01'::date) AND
(l_shipdate < '1995-01-01 00:00:00'::timestamp without time zone) AND
(l_discount >= 0.01) AND (l_discount <= 0.03) AND (l_quantity <
'24'::numeric))
Rows Removed by Index Recheck: 25284232
Heap Blocks: exact=134904 lossy=530579
-> Bitmap Index Scan on idx_lineitem_shipdate
(cost=0.00..291507.35 rows=1103911 width=0) (actual
time=6951.408..6951.408 rows=1140434 loops=1)
Index Cond: ((l_shipdate >= '1994-01-01'::date)
AND (l_shipdate < '1995-01-01 00:00:00'::timestamp without time zone)
AND (l_discount >= 0.01) AND (l_discount <= 0.03) AND (l_quantity <
'24'::numeric))
Planning time: 1.126 ms
Execution time: 40922.569 ms
(10 rows)

After Patch:
----------------

QUERY PLAN
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
Limit (cost=1541767.60..1541767.61 rows=1 width=32) (actual
time=21895.008..21895.009 rows=1 loops=1)
-> Finalize Aggregate (cost=1541767.60..1541767.61 rows=1
width=32) (actual time=21895.006..21895.006 rows=1 loops=1)
-> Gather (cost=1541767.38..1541767.59 rows=2 width=32)
(actual time=21894.341..21894.970 rows=3 loops=1)
Workers Planned: 2
Workers Launched: 2
-> Partial Aggregate (cost=1540767.38..1540767.39
rows=1 width=32) (actual time=21890.990..21890.990 rows=1 loops=3)
-> Parallel Bitmap Heap Scan on lineitem
(cost=291783.32..1538467.56 rows=459963 width=12) (actual
time=8517.126..21215.469 rows=380145 loops=3)
Recheck Cond: ((l_shipdate >=
'1994-01-01'::date) AND (l_shipdate < '1995-01-01 00:00:00'::timestamp
without time zone) AND (l_discount >= 0.01) AND (l_discount <= 0.03)
AND (l_quantity < '24'::numeric))
Rows Removed by Index Recheck: 8427921
Heap Blocks: exact=47761 lossy=187096
-> Bitmap Index Scan on
idx_lineitem_shipdate (cost=0.00..291507.35 rows=1103911 width=0)
(actual time=8307.291..8307.291 rows=1140434 loops=1)
Index Cond: ((l_shipdate >=
'1994-01-01'::date) AND (l_shipdate < '1995-01-01 00:00:00'::timestamp
without time zone) AND (l_discount >= 0.01) AND (l_discount <= 0.03)
AND (l_quantity < '24'::numeric))
Planning time: 1.173 ms
Execution time: 21915.931 ms
(14 rows)

* Thanks to Robert Haas and Amit Kapila, for helping in design review
(off list) and many valuable inputs.
* Thanks to Thomas Munro for DSA and DHT work on which my patch is based on.
* Thanks to Rafia sabih for helping with performance test.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

benchmark_machine_info.txttext/plain; charset=US-ASCII; name=benchmark_machine_info.txtDownload
dht-return-dsa-v1.patchapplication/octet-stream; name=dht-return-dsa-v1.patchDownload
diff --git a/src/backend/storage/ipc/dht.c b/src/backend/storage/ipc/dht.c
index 0916a3f..6974b91 100644
--- a/src/backend/storage/ipc/dht.c
+++ b/src/backend/storage/ipc/dht.c
@@ -51,21 +51,6 @@
 #include "storage/spin.h"
 #include "utils/memutils.h"
 
-/*
- * An item in the hash table.  This wraps the user's entry object in an
- * envelop that holds a pointer back to the bucket and a pointer to the next
- * item in the bucket.
- */
-struct dht_hash_table_item
-{
-	/* The hashed key, to avoid having to recompute it. */
-	dht_hash	hash;
-	/* The next item in the same bucket. */
-	dsa_pointer next;
-	/* The user's entry object follows here. */
-	char		entry[FLEXIBLE_ARRAY_MEMBER];
-};
-
 /* The number of partitions for locking purposes. */
 #define DHT_NUM_PARTITIONS_LOG2 7
 #define DHT_NUM_PARTITIONS (1 << DHT_NUM_PARTITIONS_LOG2)
@@ -699,6 +684,30 @@ dht_iterate_next(dht_iterator *iterator)
 {
 	dsa_pointer item_pointer;
 
+	item_pointer = dht_iterate_next_dsa(iterator);
+
+	if (DsaPointerIsValid(item_pointer))
+	{
+		iterator->item = dsa_get_address(iterator->hash_table->area,
+										 item_pointer);
+		return &(iterator->item->entry);
+	}
+
+	return NULL;
+}
+
+/*
+ * dht_iterate_next_dsa
+ *
+ * Move to the next item in the hash table. Returns a dsa_pointer to
+ * next item, or InvalidDsaPointer if the end of the hash table has
+ * been reached.
+ */
+dsa_pointer
+dht_iterate_next_dsa(dht_iterator *iterator)
+{
+	dsa_pointer item_pointer;
+
 	Assert(iterator->hash_table->control->magic == DHT_MAGIC);
 
 	while (iterator->partition < DHT_NUM_PARTITIONS)
@@ -716,9 +725,8 @@ dht_iterate_next(dht_iterator *iterator)
 		{
 			/* Remember this item, so that we can step over it next time. */
 			iterator->last_item_pointer = item_pointer;
-			iterator->item = dsa_get_address(iterator->hash_table->area,
-											 item_pointer);
-			return &(iterator->item->entry);
+
+			return item_pointer;
 		}
 
 		/* We have reached the end of the bucket. */
@@ -751,7 +759,7 @@ dht_iterate_next(dht_iterator *iterator)
 		}
 	}
 	iterator->item = NULL;
-	return NULL;
+	return InvalidDsaPointer;
 }
 
 /*
diff --git a/src/include/storage/dht.h b/src/include/storage/dht.h
index ccf9d17..8655f55 100644
--- a/src/include/storage/dht.h
+++ b/src/include/storage/dht.h
@@ -61,11 +61,25 @@ typedef struct
 
 /* Forward declaration of private types for use only by dht.c. */
 struct dht_hash_table_bucket;
-struct dht_hash_table_item;
-typedef struct dht_hash_table_item dht_hash_table_item;
+
 typedef struct dht_hash_table_bucket dht_hash_table_bucket;
 
 /*
+ * An item in the hash table.  This wraps the user's entry object in an
+ * envelop that holds a pointer back to the bucket and a pointer to the next
+ * item in the bucket.
+ */
+typedef struct dht_hash_table_item
+{
+	/* The hashed key, to avoid having to recompute it. */
+	dht_hash	hash;
+	/* The next item in the same bucket. */
+	dsa_pointer next;
+	/* The user's entry object follows here. */
+	char		entry[FLEXIBLE_ARRAY_MEMBER];
+}dht_hash_table_item;
+
+/*
  * The state used to track a walk over all entries in a hash table.  The
  * members of this struct are only for use by code in dht.c, but it is
  * included in the header because it's useful to be able to create objects of
@@ -101,6 +115,7 @@ extern void *dht_iterate_next(dht_iterator *iterator);
 extern void dht_iterate_delete(dht_iterator *iterator);
 extern void dht_iterate_release(dht_iterator *iterator);
 extern void dht_iterate_end(dht_iterator *iterator);
+extern dsa_pointer dht_iterate_next_dsa(dht_iterator *iterator);
 
 /* Finding, creating, deleting entries. */
 extern void *dht_find(dht_hash_table *hash_table,
parallel-bitmap-heap-scan-v1.patchapplication/octet-stream; name=parallel-bitmap-heap-scan-v1.patchDownload
diff --git a/src/backend/access/gin/ginget.c b/src/backend/access/gin/ginget.c
index 9ed9fd2..c8034d0 100644
--- a/src/backend/access/gin/ginget.c
+++ b/src/backend/access/gin/ginget.c
@@ -362,7 +362,7 @@ restartScanEntry:
 
 		if (entry->matchBitmap && !tbm_is_empty(entry->matchBitmap))
 		{
-			entry->matchIterator = tbm_begin_iterate(entry->matchBitmap);
+			entry->matchIterator = tbm_begin_iterate(entry->matchBitmap, NULL);
 			entry->isFinished = FALSE;
 		}
 	}
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index b019bc1..1dfd492 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_bm_update_snapshot
+ *
+ *		Update snpashot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_bm_update_snapshot(HeapScanDesc scan, Snapshot	snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 72bacd5..1e34f26 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -203,6 +204,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+										e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -255,6 +260,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+										d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -724,6 +734,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									(BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index 449aacb..c3ad77b 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -47,16 +47,63 @@
 #include "utils/spccache.h"
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
-
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
-
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static void pbms_set_parallel(PlanState *node);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+									ParallelIterator *parallel_iterator,
+									bool is_parallel);
+static void prefetch_pages(int *prefetch_pages, int prefetch_target,
+						BitmapHeapScanState *node, HeapScanDesc scan);
+static void update_prefetch_target(int *prefetch_target, int prefetch_maximum);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
  *
  *		Retrieve next tuple from the BitmapHeapScan node's currentRelation
+ *
+ *
+ *	[PARALLEL BITMAP HEAP SCAN ALGORITHM]
+ *
+ *	#1. Shared TIDBitmap creation and initialization
+ *		a) First worker to see the state as parallel bitmap info as
+ *		PBM_INITIAL become leader and set the state to PBM_INPROGRESS
+ *		All other workers see the state as PBM_INPROGRESS will wait for
+ *		leader to complete the TIDBitmap.
+ *
+ *		Leader Worker Processing:
+ *		(Leader is responsible for creating shared TIDBitmap and create
+ *		shared page and chunk array from TIDBitmap.)
+ *			1) Create TIDBitmap using DHT.
+ *			2) Begin Iterate: convert hash table into shared page and chunk
+ *			array.
+ *			3) Restore local TIDBitmap variable information into
+ *			ParallelBitmapInfo so that other worker can see those.
+ *			4) set state to PBM_FINISHED.
+ *			5) Wake up other workers.
+ *
+ *		Other Worker Processing:
+ *			1) Wait until leader create shared TIDBitmap and shared page
+ *			and chunk array.
+ *			2) Attach to shared page table, copy TIDBitmap from
+ *			ParallelBitmapInfo to local TIDBitmap, we copy this to local
+ *			TIDBitmap so that next level processing can read information
+ *			same as in non parallel case and we can avoid extra changes
+ *			in code.
+ *
+ *	# At this level TIDBitmap is ready and all workers are awake #
+ *
+ *	#2. Bitmap processing (Iterate and process the pages).
+ *		. In this phase each worker will iterate over page and chunk array and
+ *		select heap pages one by one. If prefetch is enable then there will
+ *		be two iterator.
+ *		. Since multiple worker are iterating over same page and chunk array
+ *		we need to have a shared iterator, so we grab a spin lock and iterate
+ *		within a lock.
  * ----------------------------------------------------------------
  */
 static TupleTableSlot *
@@ -67,12 +114,19 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TIDBitmap  *tbm;
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
-
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+	bool		is_parallel = node->parallel_bitmap ? true : false;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (is_parallel)
+		parallel_tbm = (ParallelTIDBitmap*)((char *)pbminfo +
+					pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -101,36 +155,106 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running
+		 * in non parallel mode or we are leader worker.
+		 *
+		 * In parallel mode leader worker will immediately come out
+		 * of the function, but all other worker will be blocked
+		 * until leader worker wake them up.
+		 */
+		if (!is_parallel || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer
+			 * node and set parallel flag in lower level bitmap index scan.
+			 * Later bitmap index node will use this flag to indicate
+			 * tidbitmap that it needs to create an shared page table.
+			 */
+			if (is_parallel)
+			{
+				node->is_leader = true;
+				pbms_set_parallel(outerPlanState(node));
+			}
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM.
+			 * Here we need to create a local TBM and copy information from
+			 * shared location. We also need to attach to shared page table
+			 * using hash table handle stored in parallel_tbm(shared memory).
+			 */
+			tbm = tbm_create(work_mem * 1024L);
+			tbm_set_parallel(tbm, node->ss.ps.state->es_query_area);
+			tbm_attach_to_pagetable(tbm, parallel_tbm);
+		}
 
 		if (!tbm || !IsA(tbm, TIDBitmap))
 			elog(ERROR, "unrecognized result from subplan");
 
 		node->tbm = tbm;
-		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+		node->tbmiterator = tbmiterator
+							= tbm_begin_iterate(tbm, parallel_tbm);
 		node->tbmres = tbmres = NULL;
 
 #ifdef USE_PREFETCH
 		if (node->prefetch_maximum > 0)
 		{
-			node->prefetch_iterator = prefetch_iterator = tbm_begin_iterate(tbm);
-			node->prefetch_pages = 0;
-			node->prefetch_target = -1;
+			node->prefetch_iterator = prefetch_iterator =
+							tbm_begin_iterate(tbm, parallel_tbm);
+
+			/* These variable are used only in case of non parallel mode */
+			if (!is_parallel)
+			{
+				node->prefetch_pages = 0;
+				node->prefetch_target = -1;
+			}
 		}
 #endif   /* USE_PREFETCH */
+
+		/*
+		 * If we are in parallel mode and we are leader worker then
+		 * copy the local TBM information to shared location, and wake
+		 * up other workers.
+		 */
+		if (node->is_leader)
+		{
+			/*
+			 * copy TBM local information in shared memory before waking
+			 * up other workers. Other workers will create there own
+			 * TBM and copy information from shared memory, they will
+			 * also use hash table handle from shared memory for attaching
+			 * to shared memory hash table.
+			 */
+			tbm_update_shared_members(tbm, parallel_tbm);
+
+			/* Change the state under a lock */
+			SpinLockAcquire(&pbminfo->state_mutex);
+			pbminfo->state = PBM_FINISHED;
+			SpinLockRelease(&pbminfo->state_mutex);
+
+			/* Wake up all other workers. */
+			ConditionVariableBroadcast(&pbminfo->cv);
+		}
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+													&pbminfo->tbmiterator,
+													is_parallel);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +262,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (is_parallel)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages --;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+											&pbminfo->prefetch_iterator,
+											is_parallel);
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not
+				 * ensure that current blockno in main iterator and prefetch
+				 * iterator is same. It's possible that whatever blockno we
+				 * are prefetching is getting processed by some other worker.
+				 */
+				if (!is_parallel &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +333,25 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				if (!is_parallel)
+					update_prefetch_target(&node->prefetch_target,
+										 node->prefetch_maximum);
+				else
+				{
+					/*
+					 * If we are in parallel mode then grab prefetch_mutex
+					 * before updating prefetch target.
+					 */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+					update_prefetch_target(&pbminfo->prefetch_target,
+										 node->prefetch_maximum);
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+				}
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +367,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (!is_parallel)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex
+				 * before updating prefetch target.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+					pbminfo->prefetch_target++;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +406,29 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			if (pbminfo == NULL)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				prefetch_pages(&node->prefetch_pages,
+							node->prefetch_target, node, scan);
+			}
+			else if(node->prefetch_pages < node->prefetch_target)
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex
+				 * before going for prefetch.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
 
-				if (tbmpre == NULL)
-				{
-					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
-					break;
-				}
-				node->prefetch_pages++;
-				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+				prefetch_pages(&pbminfo->prefetch_pages,
+							pbminfo->prefetch_target, node, scan);
+
+				SpinLockRelease(&pbminfo->prefetch_mutex);
 			}
+
+			/* Restore the prefetch_iterator */
+			prefetch_iterator = node->prefetch_iterator;
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +643,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +761,8 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->is_leader = false;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -653,3 +849,311 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool needWait = false;
+	bool queuedSelf = false;
+	bool leader = false;
+
+	for(;;)
+	{
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		/*
+		 * if state is initial then we are the first one to come here
+		 * set the state to in progress and mark ourself as leader
+		 */
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+
+		/* bitmap create is in progress so we need to wait */
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/*
+		 * If we are a leader or else leader has already created a
+		 * tid bitmap.
+		 */
+		if (leader || !needWait)
+			break;
+
+		/* We need to queue */
+		if (queuedSelf)
+		{
+			/* Sleep until leader send wake up signal */
+			ConditionVariableSleep(WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+			queuedSelf = false;
+			needWait = false;
+		}
+		else if (needWait)
+		{
+			/* Add ourself to wait queue */
+			ConditionVariablePrepareToSleep(&pbminfo->cv);
+			queuedSelf = true;
+		}
+	}
+
+	/* Cancel the sleep before return */
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node
+	 * as parallel, because only first node will create the main bitmap
+	 * other bitmaps will be merged to the first bitmap so no need to
+	 * create them in shared memory.
+	 */
+	switch(node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState*)node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState*)node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState*)node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/*
+ * prefetch_pages
+ *
+ * Prefetch pages before going for actual processing of the page.
+ */
+static void
+prefetch_pages(int *prefetch_pages, int prefetch_target,
+			BitmapHeapScanState *node, HeapScanDesc scan)
+{
+	TBMIterator *iterator = node->prefetch_iterator;
+	ParallelIterator *parallel_iteartor;
+
+	/*
+	 * If parallel bitmap info available means we are running
+	 * in parallel mode. So use parallel iterator for prefetching.
+	 */
+	if (node->parallel_bitmap)
+		parallel_iteartor = &node->parallel_bitmap->prefetch_iterator;
+
+	while (*prefetch_pages < prefetch_target)
+	{
+		TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+										parallel_iteartor,
+										node->parallel_bitmap ? true:false);
+		if (tbmpre == NULL)
+		{
+			/* No more pages to prefetch */
+			tbm_end_iterate(iterator);
+			node->prefetch_iterator = NULL;
+			break;
+		}
+
+		(*prefetch_pages)++;
+		PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+	}
+}
+
+/*
+ * update_prefetch_target
+ *
+ * Update the value of prefetch target
+ */
+static void
+update_prefetch_target(int *prefetch_target, int prefetch_maximum)
+{
+	/*
+	 * Increase prefetch target if it's not yet at the max.  Note that
+	 * we will increase it to zero after fetching the very first
+	 * page/tuple, then to one after the second tuple is fetched, then
+	 * it doubles as later pages are fetched.
+	 */
+	if (*prefetch_target >= prefetch_maximum)
+		 /* don't increase any further */ ;
+	else if (*prefetch_target >= prefetch_maximum / 2)
+		*prefetch_target = prefetch_maximum;
+	else if (*prefetch_target > 0)
+		*prefetch_target *= 2;
+	else
+		(*prefetch_target)++;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Acquire a iterator lock.
+ * copy iterator state from shared iterator to local iterator.
+ * Call tbm_iterate and restore the state back to shared iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator,
+				ParallelIterator *parallel_iterator,
+				bool is_parallel)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (!is_parallel)
+		return tbm_iterate(iterator);
+
+	/*
+	 * We are in parallel mode so grab parallel iterator mutex
+	 * before calling iterator.
+	 */
+	SpinLockAcquire(&parallel_iterator->mutex);
+
+	/*
+	 * Now we have got lock on iterator so copy information from
+	 * shared location to our local iterator.
+	 */
+	iterator->spageptr = parallel_iterator->spageptr;
+	iterator->schunkptr = parallel_iterator->schunkptr;
+	iterator->schunkbit = parallel_iterator->schunkbit;
+
+	output = tbm_iterate(iterator);
+
+	/*
+	 * tbm_iterate would have changed the iterator value
+	 * in local iterator so copy them back to shared location
+	 * before releasing the lock.
+	 */
+	parallel_iterator->spageptr = iterator->spageptr;
+	parallel_iterator->schunkptr = iterator->schunkptr;
+	parallel_iterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&parallel_iterator->mutex);
+
+	return output;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+							phs_snapshot_data),
+							EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+						 ParallelContext *pcxt)
+{
+	ParallelBitmapInfo	*pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Snapshot	snapshot;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+								phs_snapshot_data),
+								EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+
+	heap_bm_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo	*pbminfo;
+	Snapshot			snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_bm_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index a364098..9cc5088 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate
+	 * that we need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_area);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index dfeb7d5..9207741 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -44,6 +44,11 @@
 #include "nodes/bitmapset.h"
 #include "nodes/tidbitmap.h"
 #include "utils/hsearch.h"
+#include "storage/condition_variable.h"
+#include "storage/lwlock.h"
+#include "storage/shmem.h"
+#include "storage/dht.h"
+#include "storage/dsa.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -80,6 +85,8 @@
 /* number of active words for a lossy chunk: */
 #define WORDS_PER_CHUNK  ((PAGES_PER_CHUNK - 1) / BITS_PER_BITMAPWORD + 1)
 
+#define TBM_IS_SHARED(tbm) (tbm)->shared
+
 /*
  * The hashtable entries are represented by this data structure.  For
  * an exact page, blockno is the page number and bit k of the bitmap
@@ -138,24 +145,28 @@ struct TIDBitmap
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	bool			shared;		/* need to build shared tbm if set*/
+	dht_hash_table_handle	hash_handle;	/* shared hash table handle */
+	dht_hash_table	*shared_pagetable;	/* dynamic hash table */
+	dsa_area		*area;		/* reference to per-query shared memory area */
 };
 
 /*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
  */
-struct TBMIterator
+struct ParallelTIDBitmap
 {
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+	dht_hash_table_handle	hash_handle;	/* shared hash table handle */
+	dsa_pointer		dsa_pages;		/* dsa pointers for all kind of pages */
+	int				nentries;		/* number of entries in pagetable */
+	int				maxentries;		/* limit on same to meet maxbytes */
+	int				npages;			/* number of exact entries in pagetable */
+	int				nchunks;		/* number of lossy entries in pagetable */
+	int				item_count;		/* total item in dsa_pages */
+	bool			inited;			/* set true after leader converts page */
+									/* table to dsa_pointer's array. */
 };
 
-
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
 static bool tbm_intersect_page(TIDBitmap *a, PagetableEntry *apage,
@@ -167,8 +178,11 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
-
-
+static bool tbm_delete_entry(TIDBitmap *tbm, BlockNumber pageno);
+static PagetableEntry* tbm_find_or_insert(TIDBitmap *tbm, BlockNumber pageno,
+						bool *found);
+static PagetableEntry* tbm_find(const TIDBitmap *tbm, BlockNumber pageno);
+static void* tbm_seq_search(TIDBitmap *tbm, void *iterator);
 /*
  * tbm_create - create an initially-empty bitmap
  *
@@ -244,6 +258,33 @@ tbm_create_pagetable(TIDBitmap *tbm)
 }
 
 /*
+ * tbm_create_shared_pagetable
+ *
+ * Creates shared hash table using DHT for parallel bitmap scan.
+ */
+static void
+tbm_create_shared_pagetable(TIDBitmap *tbm)
+{
+	dht_parameters params = {0};
+
+	params.key_size = sizeof(BlockNumber);
+	params.entry_size = sizeof(PagetableEntry);
+	params.compare_function = memcmp;
+	params.hash_function = tag_hash;
+
+	params.tranche_id = LWLockNewTrancheId();
+
+	/* Create a dynamic hash table */
+	tbm->shared_pagetable = dht_create(tbm->area, &params);
+	if (tbm->shared_pagetable == NULL)
+		elog(ERROR, "could not create hash table");
+
+	/* Get the handle so that other backend can attach using this handle */
+	tbm->hash_handle = dht_get_hash_table_handle(tbm->shared_pagetable);
+	tbm->status = TBM_HASH;
+}
+
+/*
  * tbm_free - free a TIDBitmap
  */
 void
@@ -255,6 +296,11 @@ tbm_free(TIDBitmap *tbm)
 		pfree(tbm->spages);
 	if (tbm->schunks)
 		pfree(tbm->schunks);
+
+	/* If we have shared page table then detach from it */
+	if (tbm->shared_pagetable)
+		dht_detach(tbm->shared_pagetable);
+
 	pfree(tbm);
 }
 
@@ -431,6 +477,14 @@ void
 tbm_intersect(TIDBitmap *a, const TIDBitmap *b)
 {
 	Assert(!a->iterating);
+
+	/*
+	 * In case of BitmapAnd, only first node will create shared TBM
+	 * and all other node will have local TBM which will be merged
+	 * with first shared TBM.
+	 */
+	Assert(!TBM_IS_SHARED(b));
+
 	/* Nothing to do if a is empty */
 	if (a->nentries == 0)
 		return;
@@ -449,12 +503,34 @@ tbm_intersect(TIDBitmap *a, const TIDBitmap *b)
 	}
 	else
 	{
+		void		   *iterator;
+		dht_iterator 	dhtiterator;
 		HASH_SEQ_STATUS status;
 		PagetableEntry *apage;
 
 		Assert(a->status == TBM_HASH);
-		hash_seq_init(&status, a->pagetable);
-		while ((apage = (PagetableEntry *) hash_seq_search(&status)) != NULL)
+
+		/*
+		 * If we are using shared page table then use DHT iterator.
+		 */
+		if (TBM_IS_SHARED(a))
+		{
+			iterator = &dhtiterator;
+			dht_iterate_begin(a->shared_pagetable, &dhtiterator, false);
+		}
+		else
+		{
+			iterator = &status;
+			hash_seq_init(&status, a->pagetable);
+		}
+
+		/*
+		 * scan complete hash table using unified tbm_seq_search function this
+		 * function will take care to scan DHT if it's shared page table
+		 * otherwise dynahash.
+		 */
+		while ((apage =
+				(PagetableEntry *) tbm_seq_search(a, iterator)) != NULL)
 		{
 			if (tbm_intersect_page(a, apage, b))
 			{
@@ -464,12 +540,21 @@ tbm_intersect(TIDBitmap *a, const TIDBitmap *b)
 				else
 					a->npages--;
 				a->nentries--;
-				if (hash_search(a->pagetable,
-								(void *) &apage->blockno,
-								HASH_REMOVE, NULL) == NULL)
+
+				/*
+				 * If we are using shared hash then we need to
+				 * release the lock on element.
+				 */
+				if (TBM_IS_SHARED(a))
+					dht_iterate_release(&dhtiterator);
+
+				if (!tbm_delete_entry(a, apage->blockno))
 					elog(ERROR, "hash table corrupted");
 			}
 		}
+
+		if (TBM_IS_SHARED(a))
+			dht_iterate_end(&dhtiterator);
 	}
 }
 
@@ -579,7 +664,7 @@ tbm_is_empty(const TIDBitmap *tbm)
  * contents repeatedly, including parallel scans.
  */
 TBMIterator *
-tbm_begin_iterate(TIDBitmap *tbm)
+tbm_begin_iterate(TIDBitmap *tbm, ParallelTIDBitmap *parallel_info)
 {
 	TBMIterator *iterator;
 
@@ -592,11 +677,16 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	iterator->tbm = tbm;
 
 	/*
-	 * Initialize iteration pointers.
+	 * Initialize iteration pointers, only if it's not shared tbm.
+	 * In case of shared tbm, we will copy these values from
+	 * shared iterator before calling tbm_iterate.
 	 */
-	iterator->spageptr = 0;
-	iterator->schunkptr = 0;
-	iterator->schunkbit = 0;
+	if (!TBM_IS_SHARED(tbm))
+	{
+		iterator->spageptr = 0;
+		iterator->schunkptr = 0;
+		iterator->schunkbit = 0;
+	}
 
 	/*
 	 * If we have a hashtable, create and fill the sorted page lists, unless
@@ -606,7 +696,6 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	 */
 	if (tbm->status == TBM_HASH && !tbm->iterating)
 	{
-		HASH_SEQ_STATUS status;
 		PagetableEntry *page;
 		int			npages;
 		int			nchunks;
@@ -620,15 +709,84 @@ tbm_begin_iterate(TIDBitmap *tbm)
 				MemoryContextAlloc(tbm->mcxt,
 								   tbm->nchunks * sizeof(PagetableEntry *));
 
-		hash_seq_init(&status, tbm->pagetable);
-		npages = nchunks = 0;
-		while ((page = (PagetableEntry *) hash_seq_search(&status)) != NULL)
+		/*
+		 * If we have shared TBM means we are running in parallel mode.
+		 * So iterate over DHT and construct page and chunk array.
+		 *
+		 * First leader worker will create array of dsa pointers which
+		 * will holds dsa pointers for both pages and chunks, later
+		 * while converting to local pointers we will identify them
+		 * and copy in their respective array.
+		 */
+		if (TBM_IS_SHARED(tbm))
 		{
-			if (page->ischunk)
-				tbm->schunks[nchunks++] = page;
-			else
-				tbm->spages[npages++] = page;
+			dsa_pointer *dsa_spages;
+			dht_hash_table_item *item;
+			int			ncount = 0;
+			int			i;
+
+			/*
+			 * Iterate over DHT and create array of dsa_pointers.
+			 * Only leader will perform this step after that inited
+			 * flag will be set.
+			 */
+			if (!(parallel_info->inited) &&
+				(tbm->npages > 0 || tbm->nchunks > 0))
+			{
+				dht_iterator dhtiterator;
+				dsa_pointer	 dsa_page;
+
+				parallel_info->dsa_pages = dsa_allocate(tbm->area,
+						(tbm->nchunks + tbm->npages) * sizeof(dsa_pointer));
+
+				dsa_spages =
+						dsa_get_address(tbm->area, parallel_info->dsa_pages);
+
+				dht_iterate_begin(tbm->shared_pagetable, &dhtiterator, false);
+				while ((dsa_page = dht_iterate_next_dsa(&dhtiterator)) !=
+						InvalidDsaPointer)
+					dsa_spages[ncount++] = dsa_page;
+
+				parallel_info->inited = true;
+				parallel_info->item_count = ncount;
+			}
+
+			/*
+			 * This step will be done by all the workers including leader.
+			 * Here we need to convert array of dsa pointers to local
+			 * page and chunk array.
+			 */
+			dsa_spages = dsa_get_address(tbm->area, parallel_info->dsa_pages);
+			npages = nchunks = 0;
+			for (i = 0; i < parallel_info->item_count; i++)
+			{
+				item = dsa_get_address(tbm->area, dsa_spages[i]);
+				page = (PagetableEntry*)&(item->entry);
+
+				if (page->ischunk)
+					tbm->schunks[nchunks++] = page;
+				else
+					tbm->spages[npages++] = page;
+			}
+		}
+		else
+		{
+			HASH_SEQ_STATUS status;
+
+			/* Process local hash table, if we are not in parallel mode */
+			npages = nchunks = 0;
+
+			hash_seq_init(&status, tbm->pagetable);
+			while ((page =
+					(PagetableEntry *) hash_seq_search(&status)) != NULL)
+			{
+				if (page->ischunk)
+					tbm->schunks[nchunks++] = page;
+				else
+					tbm->spages[npages++] = page;
+			}
 		}
+
 		Assert(npages == tbm->npages);
 		Assert(nchunks == tbm->nchunks);
 		if (npages > 1)
@@ -791,11 +949,18 @@ tbm_find_pageentry(const TIDBitmap *tbm, BlockNumber pageno)
 		return page;
 	}
 
-	page = (PagetableEntry *) hash_search(tbm->pagetable,
-										  (void *) &pageno,
-										  HASH_FIND, NULL);
+	page = (PagetableEntry *) tbm_find(tbm, pageno);
 	if (page == NULL)
 		return NULL;
+
+	/*
+	 * If it's from shared hash table then release the entry.
+	 * Only one worker is building hash table so it's okay to
+	 * release it here.
+	 */
+	if (TBM_IS_SHARED(tbm))
+		dht_release(tbm->shared_pagetable, (void*)page);
+
 	if (page->ischunk)
 		return NULL;			/* don't want a lossy chunk header */
 	return page;
@@ -815,7 +980,11 @@ tbm_get_pageentry(TIDBitmap *tbm, BlockNumber pageno)
 	PagetableEntry *page;
 	bool		found;
 
-	if (tbm->status == TBM_EMPTY)
+	/*
+	 * Use fixed slot only if it's local tidbitmap, If shared bitmap
+	 * then directly insert into shared hash table.
+	 */
+	if ((tbm->status == TBM_EMPTY) && !TBM_IS_SHARED(tbm))
 	{
 		/* Use the fixed slot */
 		page = &tbm->entry1;
@@ -824,6 +993,7 @@ tbm_get_pageentry(TIDBitmap *tbm, BlockNumber pageno)
 	}
 	else
 	{
+		/* In case of shared TBM, status will never be TBM_ONE_PAGE */
 		if (tbm->status == TBM_ONE_PAGE)
 		{
 			page = &tbm->entry1;
@@ -833,10 +1003,17 @@ tbm_get_pageentry(TIDBitmap *tbm, BlockNumber pageno)
 			tbm_create_pagetable(tbm);
 		}
 
+		/*
+		 * In case of parallel bitmap scan, we don't switch from TBM_EMPTY
+		 * to TBM_ONE_PAGE. So even if we are here we may not have hash
+		 * table ready, so if tbm->status is not yet TBM_HASH then create
+		 * shared page table.
+		 */
+		if (tbm->status != TBM_HASH && TBM_IS_SHARED(tbm))
+			tbm_create_shared_pagetable(tbm);
+
 		/* Look up or create an entry */
-		page = (PagetableEntry *) hash_search(tbm->pagetable,
-											  (void *) &pageno,
-											  HASH_ENTER, &found);
+		page = tbm_find_or_insert(tbm, pageno, &found);
 	}
 
 	/* Initialize it if not present before */
@@ -849,6 +1026,10 @@ tbm_get_pageentry(TIDBitmap *tbm, BlockNumber pageno)
 		tbm->npages++;
 	}
 
+	/* Release the entry lock if it's from shared hash table */
+	if (page && TBM_IS_SHARED(tbm))
+		dht_release(tbm->shared_pagetable, (void*)page);
+
 	return page;
 }
 
@@ -869,9 +1050,17 @@ tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno)
 
 	bitno = pageno % PAGES_PER_CHUNK;
 	chunk_pageno = pageno - bitno;
-	page = (PagetableEntry *) hash_search(tbm->pagetable,
-										  (void *) &chunk_pageno,
-										  HASH_FIND, NULL);
+
+	page = tbm_find(tbm, chunk_pageno);
+
+	/*
+	 * If entry is from shared page table then release the entry.
+	 * Currently only one worker is building the bitmap so it's
+	 * fine to release it here.
+	 */
+	if (page && TBM_IS_SHARED(tbm))
+		dht_release(tbm->shared_pagetable, (void*)page);
+
 	if (page != NULL && page->ischunk)
 	{
 		int			wordnum = WORDNUM(bitno);
@@ -901,7 +1090,13 @@ tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno)
 
 	/* We force the bitmap into hashtable mode whenever it's lossy */
 	if (tbm->status != TBM_HASH)
-		tbm_create_pagetable(tbm);
+	{
+		/* Create shared page table if we are in parallel mode. */
+		if (TBM_IS_SHARED(tbm))
+			tbm_create_shared_pagetable(tbm);
+		else
+			tbm_create_pagetable(tbm);
+	}
 
 	bitno = pageno % PAGES_PER_CHUNK;
 	chunk_pageno = pageno - bitno;
@@ -912,20 +1107,16 @@ tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno)
 	 */
 	if (bitno != 0)
 	{
-		if (hash_search(tbm->pagetable,
-						(void *) &pageno,
-						HASH_REMOVE, NULL) != NULL)
+		if (tbm_delete_entry(tbm, pageno))
 		{
-			/* It was present, so adjust counts */
-			tbm->nentries--;
-			tbm->npages--;		/* assume it must have been non-lossy */
+				/* It was present, so adjust counts */
+				tbm->nentries--;
+				tbm->npages--;		/* assume it must have been non-lossy */
 		}
 	}
 
 	/* Look up or create entry for chunk-header page */
-	page = (PagetableEntry *) hash_search(tbm->pagetable,
-										  (void *) &chunk_pageno,
-										  HASH_ENTER, &found);
+	page = (PagetableEntry *)tbm_find_or_insert(tbm, chunk_pageno, &found);
 
 	/* Initialize it if not present before */
 	if (!found)
@@ -954,6 +1145,10 @@ tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno)
 	wordnum = WORDNUM(bitno);
 	bitnum = BITNUM(bitno);
 	page->words[wordnum] |= ((bitmapword) 1 << bitnum);
+
+	/* Unlock an entry which was locked by dht_find_or_insert. */
+	if (TBM_IS_SHARED(tbm))
+		dht_release(tbm->shared_pagetable, (void*)page);
 }
 
 /*
@@ -964,21 +1159,28 @@ tbm_lossify(TIDBitmap *tbm)
 {
 	HASH_SEQ_STATUS status;
 	PagetableEntry *page;
+	dht_iterator	dhtiterator;
+	void *iterator;
 
-	/*
-	 * XXX Really stupid implementation: this just lossifies pages in
-	 * essentially random order.  We should be paying some attention to the
-	 * number of bits set in each page, instead.
-	 *
-	 * Since we are called as soon as nentries exceeds maxentries, we should
-	 * push nentries down to significantly less than maxentries, or else we'll
-	 * just end up doing this again very soon.  We shoot for maxentries/2.
-	 */
 	Assert(!tbm->iterating);
 	Assert(tbm->status == TBM_HASH);
 
-	hash_seq_init(&status, tbm->pagetable);
-	while ((page = (PagetableEntry *) hash_seq_search(&status)) != NULL)
+	/*
+	 * If we are using shared page table then use DHT iterator otherwise
+	 * dyna hash iterator
+	 */
+	if (TBM_IS_SHARED(tbm))
+	{
+		iterator = &dhtiterator;
+		dht_iterate_begin(tbm->shared_pagetable, &dhtiterator, false);
+	}
+	else
+	{
+		iterator = &status;
+		hash_seq_init(&status, tbm->pagetable);
+	}
+
+	while ((page = (PagetableEntry *) tbm_seq_search(tbm, iterator)) != NULL)
 	{
 		if (page->ischunk)
 			continue;			/* already a chunk header */
@@ -990,21 +1192,23 @@ tbm_lossify(TIDBitmap *tbm)
 		if ((page->blockno % PAGES_PER_CHUNK) == 0)
 			continue;
 
+		/* If we are using shared hash then release the lock on element. */
+		if (TBM_IS_SHARED(tbm))
+			dht_iterate_release(&dhtiterator);
+
 		/* This does the dirty work ... */
 		tbm_mark_page_lossy(tbm, page->blockno);
 
 		if (tbm->nentries <= tbm->maxentries / 2)
 		{
 			/* we have done enough */
-			hash_seq_term(&status);
+			if (TBM_IS_SHARED(tbm))
+				dht_iterate_end(&dhtiterator);
+			else
+				hash_seq_term(&status);
+
 			break;
 		}
-
-		/*
-		 * Note: tbm_mark_page_lossy may have inserted a lossy chunk into the
-		 * hashtable.  We can continue the same seq_search scan since we do
-		 * not care whether we visit lossy chunks or not.
-		 */
 	}
 
 	/*
@@ -1036,3 +1240,171 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_attach_to_pagetable
+ *
+ * Attach worker to shared TID bitmap created by leader worker.
+ */
+void
+tbm_attach_to_pagetable(TIDBitmap *tbm, ParallelTIDBitmap *stbm)
+{
+	dht_parameters params = {0};
+
+	params.key_size = sizeof(BlockNumber);
+	params.entry_size = sizeof(PagetableEntry);
+	params.compare_function = memcmp;
+	params.hash_function = tag_hash;
+
+	tbm->shared_pagetable = dht_attach(tbm->area, &params, stbm->hash_handle);
+
+	tbm->status = TBM_HASH;
+	tbm->nchunks = stbm->nchunks;
+	tbm->nentries = stbm->nentries;
+	tbm->npages = stbm->npages;
+	tbm->maxentries = stbm->maxentries;
+	tbm->shared = true;
+}
+
+/*
+ * tbm_update_shared_members
+ *
+ * Restore leaders private tbm state to shared location. This must
+ * be called before waking up the other worker.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before
+	 * waking up the other workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->hash_handle = tbm->hash_handle;
+}
+
+/*
+ * tbm_delete_entry
+ *
+ * Unified function to delete an entry from tbm page table
+ * if tbm is shared then it will operate on from shared hash
+ * table otherwise on local hash table.
+ */
+static bool
+tbm_delete_entry(TIDBitmap *tbm, BlockNumber pageno)
+{
+	bool result = false;
+	if (TBM_IS_SHARED(tbm))
+	{
+		/* Look up or create an entry */
+		if (dht_delete_key(tbm->shared_pagetable, (void *) &pageno))
+			result = true;
+	}
+	else
+	{
+		if (hash_search(tbm->pagetable,
+						(void *) &pageno,
+						HASH_REMOVE, NULL) != NULL)
+		result = true;
+	}
+
+	return result;
+}
+
+/*
+ * tbm_find_or_insert
+ *
+ * Unified function to find or insert an entry in page table.
+ * If tbm is shared then it will operate on from shared hash
+ * table otherwise on local hash table.
+ */
+static PagetableEntry*
+tbm_find_or_insert(TIDBitmap *tbm, BlockNumber chunk_pageno, bool *found)
+{
+	PagetableEntry *page;
+
+	if (TBM_IS_SHARED(tbm))
+		page = (PagetableEntry *) dht_find_or_insert(tbm->shared_pagetable,
+										  (void *) &chunk_pageno,
+										  found);
+	else
+		page = (PagetableEntry *) hash_search(tbm->pagetable,
+											  (void *) &chunk_pageno,
+											  HASH_ENTER, found);
+
+	return page;
+}
+
+/*
+ * tbm_find
+ *
+ * Unified function to find an entry from tbm page table
+ * if tbm is shared then it will operate on shared hash
+ * table otherwise on local hash table.
+ */
+static PagetableEntry*
+tbm_find(const TIDBitmap *tbm, BlockNumber pageno)
+{
+	PagetableEntry *page;
+
+	if (TBM_IS_SHARED(tbm))
+		page = (PagetableEntry *) dht_find(tbm->shared_pagetable,
+										  (void *) &pageno,
+										  false);
+	else
+		page = (PagetableEntry *) hash_search(tbm->pagetable,
+											  (void *) &pageno,
+											  HASH_FIND, NULL);
+	return page;
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->shared = true;
+	tbm->area = (dsa_area*)area;
+}
+
+/*
+ * tbm_seq_search
+ *
+ * Unified function to scan full tbm page table
+ * if tbm is shared then it will operate on shared hash
+ * table otherwise on local hash table.
+ */
+static void*
+tbm_seq_search(TIDBitmap *tbm, void *iterator)
+{
+	if (TBM_IS_SHARED(tbm))
+	{
+		dht_iterator *dhtiterator = (dht_iterator*)iterator;
+
+		return dht_iterate_next(dhtiterator);
+	}
+	else
+	{
+		HASH_SEQ_STATUS *status = (HASH_SEQ_STATUS*)iterator;
+
+		return hash_seq_search(status);
+	}
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index e42ef98..058e55a 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -126,6 +126,7 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static int compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 
 
 /*
@@ -678,49 +679,7 @@ create_plain_partial_paths(PlannerInfo *root, RelOptInfo *rel)
 {
 	int			parallel_workers;
 
-	/*
-	 * If the user has set the parallel_workers reloption, use that; otherwise
-	 * select a default number of workers.
-	 */
-	if (rel->rel_parallel_workers != -1)
-		parallel_workers = rel->rel_parallel_workers;
-	else
-	{
-		int			parallel_threshold;
-
-		/*
-		 * If this relation is too small to be worth a parallel scan, just
-		 * return without doing anything ... unless it's an inheritance child.
-		 * In that case, we want to generate a parallel path here anyway.  It
-		 * might not be worthwhile just for this relation, but when combined
-		 * with all of its inheritance siblings it may well pay off.
-		 */
-		if (rel->pages < (BlockNumber) min_parallel_relation_size &&
-			rel->reloptkind == RELOPT_BASEREL)
-			return;
-
-		/*
-		 * Select the number of workers based on the log of the size of the
-		 * relation.  This probably needs to be a good deal more
-		 * sophisticated, but we need something here for now.  Note that the
-		 * upper limit of the min_parallel_relation_size GUC is chosen to
-		 * prevent overflow here.
-		 */
-		parallel_workers = 1;
-		parallel_threshold = Max(min_parallel_relation_size, 1);
-		while (rel->pages >= (BlockNumber) (parallel_threshold * 3))
-		{
-			parallel_workers++;
-			parallel_threshold *= 3;
-			if (parallel_threshold > INT_MAX / 3)
-				break;			/* avoid overflow */
-		}
-	}
-
-	/*
-	 * In no case use more than max_parallel_workers_per_gather workers.
-	 */
-	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+	parallel_workers = compute_parallel_worker(rel, rel->pages);
 
 	/* If any limit was set to zero, the user doesn't want a parallel scan. */
 	if (parallel_workers <= 0)
@@ -2866,6 +2825,84 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int				parallel_workers;
+	double			pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path*)create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+static int
+compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
+{
+	int			parallel_workers;
+
+	/*
+	 * If the user has set the parallel_workers reloption, use that; otherwise
+	 * select a default number of workers.
+	 */
+	if (rel->rel_parallel_workers != -1)
+		parallel_workers = rel->rel_parallel_workers;
+	else
+	{
+		int			parallel_threshold;
+
+		/*
+		 * If this relation is too small to be worth a parallel scan, just
+		 * return without doing anything ... unless it's an inheritance child.
+		 * In that case, we want to generate a parallel path here anyway.  It
+		 * might not be worthwhile just for this relation, but when combined
+		 * with all of its inheritance siblings it may well pay off.
+		 */
+		if (pages < (BlockNumber) min_parallel_relation_size &&
+			rel->reloptkind == RELOPT_BASEREL)
+			return 0;
+
+		/*
+		 * Select the number of workers based on the log of the size of the
+		 * relation.  This probably needs to be a good deal more
+		 * sophisticated, but we need something here for now.  Note that the
+		 * upper limit of the min_parallel_relation_size GUC is chosen to
+		 * prevent overflow here.
+		 */
+		parallel_workers = 1;
+		parallel_threshold = Max(min_parallel_relation_size, 1);
+		while (pages >= (BlockNumber) (parallel_threshold * 3))
+		{
+			parallel_workers++;
+			parallel_threshold *= 3;
+			if (parallel_threshold > INT_MAX / 3)
+				break;			/* avoid overflow */
+		}
+	}
+
+	/*
+	 * In no case use more than max_parallel_workers_per_gather workers.
+	 */
+	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+
+	return parallel_workers;
+}
+
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 2a49639..c08997a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -161,7 +161,7 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
 static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
 static double relation_byte_size(double tuples, int width);
 static double page_size(double tuples, int width);
-
+static Cost adjust_cost_for_parallelism(Path *path, Cost cpu_run_cost);
 
 /*
  * clamp_row_est
@@ -237,44 +237,7 @@ cost_seqscan(Path *path, PlannerInfo *root,
 
 	/* Adjust costing for parallelism, if used. */
 	if (path->parallel_workers > 0)
-	{
-		double		parallel_divisor = path->parallel_workers;
-		double		leader_contribution;
-
-		/*
-		 * Early experience with parallel query suggests that when there is
-		 * only one worker, the leader often makes a very substantial
-		 * contribution to executing the parallel portion of the plan, but as
-		 * more workers are added, it does less and less, because it's busy
-		 * reading tuples from the workers and doing whatever non-parallel
-		 * post-processing is needed.  By the time we reach 4 workers, the
-		 * leader no longer makes a meaningful contribution.  Thus, for now,
-		 * estimate that the leader spends 30% of its time servicing each
-		 * worker, and the remainder executing the parallel plan.
-		 */
-		leader_contribution = 1.0 - (0.3 * path->parallel_workers);
-		if (leader_contribution > 0)
-			parallel_divisor += leader_contribution;
-
-		/*
-		 * In the case of a parallel plan, the row count needs to represent
-		 * the number of tuples processed per worker.  Otherwise, higher-level
-		 * plan nodes that appear below the gather will be costed incorrectly,
-		 * because they'll anticipate receiving more rows than any given copy
-		 * will actually get.
-		 */
-		path->rows = clamp_row_est(path->rows / parallel_divisor);
-
-		/* The CPU cost is divided among all the workers. */
-		cpu_run_cost /= parallel_divisor;
-
-		/*
-		 * It may be possible to amortize some of the I/O cost, but probably
-		 * not very much, because most operating systems already do aggressive
-		 * prefetching.  For now, we assume that the disk run cost can't be
-		 * amortized at all.
-		 */
-	}
+		cpu_run_cost = adjust_cost_for_parallelism(path, cpu_run_cost);
 
 	path->startup_cost = startup_cost;
 	path->total_cost = startup_cost + cpu_run_cost + disk_run_cost;
@@ -831,10 +794,10 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
 	Cost		indexTotalCost;
-	Selectivity indexSelectivity;
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -855,13 +818,12 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	if (!enable_bitmapscan)
 		startup_cost += disable_cost;
 
-	/*
-	 * Fetch total cost of obtaining the bitmap, as well as its total
-	 * selectivity.
-	 */
-	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+	pages_fetched = compute_bitmap_pages(root, baserel, bitmapqual,
+										loop_count, &indexTotalCost,
+										&tuples_fetched);
 
 	startup_cost += indexTotalCost;
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
 
 	/* Fetch estimated page costs for tablespace containing table. */
 	get_tablespace_page_costs(baserel->reltablespace,
@@ -869,41 +831,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 							  &spc_seq_page_cost);
 
 	/*
-	 * Estimate number of main-table pages fetched.
-	 */
-	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
-
-	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
-
-	if (loop_count > 1)
-	{
-		/*
-		 * For repeated bitmap scans, scale up the number of tuples fetched in
-		 * the Mackert and Lohman formula by the number of scans, so that we
-		 * estimate the number of pages fetched by all the scans. Then
-		 * pro-rate for one scan.
-		 */
-		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
-											baserel->pages,
-											get_indexpath_pages(bitmapqual),
-											root);
-		pages_fetched /= loop_count;
-	}
-	else
-	{
-		/*
-		 * For a single scan, the number of heap pages that need to be fetched
-		 * is the same as the Mackert and Lohman formula for the case T <= b
-		 * (ie, no re-reads needed).
-		 */
-		pages_fetched = (2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
-	}
-	if (pages_fetched >= T)
-		pages_fetched = T;
-	else
-		pages_fetched = ceil(pages_fetched);
-
-	/*
 	 * For small numbers of pages we should charge spc_random_page_cost
 	 * apiece, while if nearly all the table's pages are being read, it's more
 	 * appropriate to charge spc_seq_page_cost apiece.  The effect is
@@ -932,8 +859,13 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+		cpu_run_cost = adjust_cost_for_parallelism(path, cpu_run_cost);
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
@@ -944,6 +876,56 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 }
 
 /*
+ * adjust_cost_for_parallelism
+ *
+ * Adjust the cpu cost based on number of parallel workers, also update
+ * the number of rows processed at each worker.
+ */
+static Cost
+adjust_cost_for_parallelism(Path *path, Cost cpu_run_cost)
+{
+	double		parallel_divisor = path->parallel_workers;
+	double		leader_contribution;
+	Cost		cpu_cost = cpu_run_cost;
+
+	/*
+	 * Early experience with parallel query suggests that when there is
+	 * only one worker, the leader often makes a very substantial
+	 * contribution to executing the parallel portion of the plan, but as
+	 * more workers are added, it does less and less, because it's busy
+	 * reading tuples from the workers and doing whatever non-parallel
+	 * post-processing is needed.  By the time we reach 4 workers, the
+	 * leader no longer makes a meaningful contribution.  Thus, for now,
+	 * estimate that the leader spends 30% of its time servicing each
+	 * worker, and the remainder executing the parallel plan.
+	 */
+	leader_contribution = 1.0 - (0.3 * path->parallel_workers);
+	if (leader_contribution > 0)
+		parallel_divisor += leader_contribution;
+
+	/*
+	 * In the case of a parallel plan, the row count needs to represent
+	 * the number of tuples processed per worker.  Otherwise, higher-level
+	 * plan nodes that appear below the gather will be costed incorrectly,
+	 * because they'll anticipate receiving more rows than any given copy
+	 * will actually get.
+	 */
+	path->rows = clamp_row_est(path->rows / parallel_divisor);
+
+	/* The CPU cost is divided among all the workers. */
+	cpu_cost /= parallel_divisor;
+
+	/*
+	 * It may be possible to amortize some of the I/O cost, but probably
+	 * not very much, because most operating systems already do aggressive
+	 * prefetching.  For now, we assume that the disk run cost can't be
+	 * amortized at all.
+	 */
+
+	return cpu_cost;
+}
+
+/*
  * cost_bitmap_tree_node
  *		Extract cost and selectivity from a bitmap tree node (index/and/or)
  */
@@ -4765,3 +4747,69 @@ page_size(double tuples, int width)
 {
 	return ceil(relation_byte_size(tuples, width) / BLCKSZ);
 }
+
+/*
+ * compute_bitmap_pages
+ *
+ * compute number of pages fetched from heap in bitmap heap scan.
+ */
+double
+compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel, Path *bitmapqual,
+			int loop_count, Cost *cost, double *tuple)
+{
+	Cost			indexTotalCost;
+	Selectivity		indexSelectivity;
+	double			T;
+	double			pages_fetched;
+	double			tuples_fetched;
+
+	/*
+	 * Fetch total cost of obtaining the bitmap, as well as its total
+	 * selectivity.
+	 */
+	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+
+	/*
+	 * Estimate number of main-table pages fetched.
+	 */
+	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
+
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
+
+	if (loop_count > 1)
+	{
+		/*
+		 * For repeated bitmap scans, scale up the number of tuples fetched in
+		 * the Mackert and Lohman formula by the number of scans, so that we
+		 * estimate the number of pages fetched by all the scans. Then
+		 * pro-rate for one scan.
+		 */
+		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
+											baserel->pages,
+											get_indexpath_pages(bitmapqual),
+											root);
+		pages_fetched /= loop_count;
+	}
+	else
+	{
+		/*
+		 * For a single scan, the number of heap pages that need to be fetched
+		 * is the same as the Mackert and Lohman formula for the case T <= b
+		 * (ie, no re-reads needed).
+		 */
+		pages_fetched =
+				(2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
+	}
+
+	if (pages_fetched >= T)
+		pages_fetched = T;
+	else
+		pages_fetched = ceil(pages_fetched);
+
+	if (cost)
+		*cost = indexTotalCost;
+	if (tuple)
+		*tuple = tuples_fetched;
+
+	return pages_fetched;
+}
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 2952bfb..1a7bde0 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 47158f6..4ffcf87 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2803,7 +2803,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index abb7507..0801b6e 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -47,7 +47,6 @@ typedef enum
 
 static List *translate_sub_tlist(List *tlist, int relid);
 
-
 /*****************************************************************************
  *		MISC. PATH UTILITIES
  *****************************************************************************/
@@ -1071,7 +1070,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,11 +1080,10 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
-
 	pathnode->bitmapqual = bitmapqual;
 
 	cost_bitmap_heap_scan(&pathnode->path, root, rel,
@@ -3192,7 +3191,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 5c6cb6b..3e48afe 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3382,6 +3382,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 0d12bbb..7cf0a3c 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -23,7 +23,6 @@
 #include "utils/relcache.h"
 #include "utils/snapshot.h"
 
-
 /* "options" flag bits for heap_insert */
 #define HEAP_INSERT_SKIP_WAL	0x0001
 #define HEAP_INSERT_SKIP_FSM	0x0002
@@ -178,6 +177,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_bm_update_snapshot(HeapScanDesc scan, Snapshot	snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index 0ed9c78..8afecb7 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+								ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+								ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+										shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index bb1f56a..78f1ac8 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1393,6 +1395,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t	mutex;			/* mutual exclusion for below three fields */
+	int 	spageptr;		/* next spages index */
+	int 	schunkptr;		/* next schunks index */
+	int 	schunkbit;		/* next bit to check in current schunk */
+} ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *	 						parallel bitmap heap scan.
+ *
+ *	 	PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *	 						first worker to see this state will become
+ *	 						leader and will create TIDbitmap. This will
+ *	 						also set the state to PBM_INPROGRESS.
+ *	 	PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *	 						so workers need to sleep until leader set the
+ *	 						state to PBM_FINISHED and wake us up.
+ *	 	PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *	 						can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *	 	relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid						relid;
+	ParallelIterator		tbmiterator;
+	ParallelIterator		prefetch_iterator;
+	slock_t					prefetch_mutex;
+	int						prefetch_pages;
+	int						prefetch_target;
+	slock_t					state_mutex;
+	ConditionVariable		cv;
+	PBMState				state;
+	Size					ptbm_offset;
+	char					phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1407,6 +1475,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1423,6 +1493,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1438,7 +1509,11 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		is_leader		   is_leader is set true, if this worker become
+ *						   leader for parallel bitmap heap scan.
+ *		parallel_bitmap	   shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1453,6 +1528,9 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	bool		is_leader;
+	ParallelBitmapInfo	*parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 9303299..e215256 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+}TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -58,9 +75,14 @@ extern void tbm_union(TIDBitmap *a, const TIDBitmap *b);
 extern void tbm_intersect(TIDBitmap *a, const TIDBitmap *b);
 
 extern bool tbm_is_empty(const TIDBitmap *tbm);
-
-extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
+extern TBMIterator * tbm_begin_iterate(TIDBitmap *tbm,
+							ParallelTIDBitmap *parallel_info);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_attach_to_pagetable(TIDBitmap *tbm, ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+									ParallelTIDBitmap *parallel_tbm);
+void tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..e1baacc 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -183,6 +183,8 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
 					   double cte_rows);
 extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
 extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern double compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel,
+				Path *bitmapqual, int loop_count, Cost *cost, double *tuple);
 
 /*
  * prototypes for clausesel.c
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..397d266 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 44abe83..3264f44 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 27be549..7f7992a 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -784,6 +784,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
TPCH_PBMS.pdfapplication/pdf; name=TPCH_PBMS.pdfDownload
%PDF-1.4
%��������
2 0 obj
<</Length 3 0 R/Filter/FlateDecode>>
stream
x��YK��6����90�*=-��3���4���d!�����#�d��r{����2�q�R=�*U�����/s�`,��~�����������������G5�t;���
_?.���������9J�_:*����0n������?=�g&��&���?3&������0�H��������Q��i/�	B�I6��G�v4*B^����
�Ie���Q#NL�e��N2�G{	���
��On.�Z����Q�w��Ko?n7�q,����m����@�{I2��e���h�i
;��M���+�^OO�{���L�(�� N����Us��������� �����4�0��y�`0�����U11P�3"��u������L�2J'����+L�si;�g\1E~W"�g2B��E|a��i�]�f��8/D�`w�.�9�6���T�H�R�r�- ��3i�=�SW��ZD�1��b]4
�=��	#�zQ0��<�8��H��V����aPl�%�����s9cJ���l���Q�F��e�P�������	����%���
e�(5��@�B���KJ)^��)I5�����h���J'V�pbk�!
[2�$\|{����?F%�F�����n��\Rg���.�:(B���:��':T���F�FF�b��?3*53������T�����L@C�MM~e�n�)��Lx�����f[�if��J)Xsj����1�e
F ���p�lN��������R�,|�����q����Cqy>��L��;k�PT��q&V<�;��o*����n�C�����/8c�R�d������5���zK�!���}�^CU�}���J����t����u����oG��]
��Pw[��UD�cT�Q���J��mJ��#m�Na|��_)�~Q�� 
������;&���}��Te����3+�J�({��e��^�A+1�g������F�~!�S���z`�����a��-��C����>uCjTR�.��
��4���6d�kVA�����W����<?3�"}�m�C���6�x�K���mzTP|��-N���Hy%3]H�=��UC������h����c���}<�H���RXSW��T���~�y���5��f,�8e�)i|i��f?�bq�j���=�������w��F=�=z`r��Gv,��pn���r���'+���=i���P���T��]]��G��}�M��f�I��=�����������F�!��V�Z&��T�@6k�H�����S�YGAH�<���z.�C���|�����|<�K��uJ/�]�p�����������N����8��c�����d���~��P��Q�Dn8''L���(y,���L&��������	Xf\�.aESd�4�}��S�����:��-	z]���s\���o>�"S�����b��^�O@
�,C�l��_XV	I��6]��'��0m��x����)�� ����f K�[������d� �T�����$��9������P'���I|�����������W�!�V�&��c��;�T���&�E����G�U��'|/�7j4��.!����>]rH��C�����7�����1�m��4 �����;��?D�sVD_{+�,U�$1���(4MQ�y��C Pb03W-A�l?Rm%P�"��mD���?�`)$��=G�u�X�0�l�b%�!�^e���:ZZ]e����6
0�O��J�C�V�5
�j�"C#���C��peU/gQ��	�'+0UR��<�|x��g5|�g�p�*�������d8�
endstream
endobj

3 0 obj
1793
endobj

4 0 obj
<</Type/XObject
/Subtype/Form
/BBox[ -79 395 692 395.1 ]
/Group<</S/Transparency/CS/DeviceRGB/K true>>
/Length 25
/Filter/FlateDecode
>>
stream
x�+T0P0��t.�.��@.)�
endstream
endobj

5 0 obj
<</CA 0.5
   /ca 0.5
>>
endobj

7 0 obj
<</Length 8 0 R/Filter/FlateDecode/Length1 24296>>
stream
x���xS��0:k����%Y[�%[�mI�������-{�,����6���6~@x$��$8	%���6$
y!!�I���$�M��6�i~���I��6NO�����-�W�����~���+i�=�5kf��Y�����B4�$�\�1�P���B�!���G��j��J�������Y�sA��;BJ�g����g�R�"��]�m���-$����$�?u��E	$���v��+�����I������W��#�8����v�5X��aH|����k�
�>"�0B*ip`xd?��F��2��,\��-H�~A��|�GC�
��)�*>F������g2�'X�I�)6�#���������r�z���<_~Aa����{�{��mC&�Q���aKQ����'4v�>����V�"���Mt�%k���_�%��z^D~���s��>t��O�V���#���v��=Nj>��M%|�����s��_�
�~�A��G���~�p�f�W�^����1���h7��!�C{	|;:-h9I�|��.4p�q4��F��,����M�7�^�6i�n�g?�C�n*�,|N����%����m&SdV�S_{�DF=���I;df�*NG����/ZZ0��uw�k�Us���*+��e�%�E��D�'�������Lu����.V���UJ�2���h��3��l�3��qgI��)�=l'I�[a��v�~+�D �o��"��uH���<�m�v��?�r�OC��F~���d_����0�!G�$�p�����*{����������*���:f�snWL��Q�����Y����UrgU��H���������phacu���h�u��c�Ur�+�+���2J{m:�c?�����V��4����e�a���g�����]�lgU8{�G	��]a���:��X�]���F����}�o�t�y��[S:�)�t�o������=0�>�qzzl��.8��k4�����(�HJ����k8�@SXh���hg�����-�a���v���t:��}�L��e#BBBS��v|�i	� �����H��VXO ��j
�v�39�cj�9c39���;�h�-n�����jB�=����V��p
������Ao/�6��v����>{�� d!�n.@8��H�g��E+� Co��8	���Y����M ���p�����TERGt����^R���Q_�<|a�s0��s}<i���7�E���qs��}e�T�[]Ek�W��WE�@q96��|�����/�P>j�������2��;���vk'�i��F�#,5�nr6v5QF#��@�s�5���������������D2(:6��64�Fk
a��*]eo�V��
$� ��rr+�U���T��s���`E3���l{uWU��oA�Qv�����Q�gn��hrD>�nL����I	%jp&�I'���a�FN��L�<ootv9�����j�}����%�L��X����X�L�A�g"�����z3q�5r�z4x[v�L�}\��[<N�;�iymQ���Vy����t�ILf�<���K���t��;k;����eh"A��n�uP����ua6��v-<.������	D��U�x��>��x�k|�N�
9�T�H#v����J���&!4&��r�_y����I��4��	3i����4IN�2J	���D~W�;��li�oo�<���"�apV�8+�Vh�1��9a�sM�����tMW�3��7����%����*r����D�����'���b�q���&At���M>�T�W�OM���t��Q��Si��T/������SE�N�����*�����PU����B�>���P;�����mJ>����a��c`q��K���L�^9����Y
�8{�oVI�1�R�7#�`�l��f���]�]���L��OO_��$iC�����3��`���m9��u9����46����E<T��sFT�kmm�(�[/JJZ[]����}^��D������g���@�����N{ZfQ
��������
�+�//�+=�35��H8�c�^=qn���S�+������t7<�]R9�d{�������o
����y�o�,��x�����5�fo[�s�v4�_�?�RzQ������9�K�3�Z��t��@��e9��6��Pa��|��raa�AG2���1y��L!��}
��'�4�:�H!
�Lk��9������Nv�	:'�9�y����������E�����NeID�	B��*/�U�*)��]�s��[7DI�.����I����� A��PM	��@}gE��y�������S�u�$&���h^^|q��������j�����k�r�M���S����w)��v>JB�h��iv�r���U)���"�[��������9��jlz�����g3�L�5c����B���=d�P�����0������u���y����������-,��%c����N�L�*!�Y;}�����}`��+wnw�zu������+�KY����=%����-�^H���������6%�?���;�����6}�pndF���Y���1vw,���>�F�n-����k�k�-��,Kb���y��5Z3#x�^'pvN�X%7�:EHS1��b�����w1Z���|������]�@f���u���Wzg���g����83�<��n(��������U&�JV���?�����Z��J�B6�����+-��v~��<�����^��U0�D������9X��s�C4m��XF�P�J�0����Efg����m.�
�8�\A:L�'�g��0�Xz�-���~%���/������%��Qi�Q���A�y�0�L�p�&]�����I �8e3R	*I��m�8o������q�.����8F'��`\����3J��y�EgP�!|#�:qFrD���Lae~	;�
�?�7��8��^x�w�TL92��{� ���C�U��q���������\��a��yh6��������7jvk0�"��%r��@��%��pzzJ@���[��3�s��jL���LIA�����#��������|'���0�\���}mq������t�D��!�<D2����yRy�0
�'3�2/��2U\�.�m.5�J�J(�
e&�	fS��L����t�2�D��A������SG�����N*]Z�__��x#���`�{?�)��������zEr��Q��m
9�2gCsA[��W��������K����>��z����S��B�T��E�����W�)hbA��\��N��dj)s��C�P5�P
n�5>�W�T��
h�tips��R��28����-��w���ds�����,�
Hy�b,l�=�ck��ua|R���kt��h��j��5�j�55�6t�&k`A
�����o
>[s��@��K-
�������
�����<�h�������T�
�����::�����`Qb��O&���0��W�xy������-��
��Z'���W�;���W�0�y����880|������p�=+�SX�r��&��*�Y�7�4�fEyb��������]W�������}x^_���(��]���W��?~������������C+r<�OF�H��'�a�S����{w�`���/J�\�)�!�YK��L��I3��M�>
^��*��M�� 6k�h�DD_L�2��p����6l������1/cj9�� lG�#5:�.�Bj*���m�8Q-�������Kt���?Y&Z�\'���(��J��Q��nS�������J���^t$�2pO�qP�x�[��S���������s���.~`����������OL}v�	��4����Y�y^a~C�#�����??���[�4ux*Q\:'c�@#�L��?3�<����`��	�Q;�����0��#oJO�z��M�t5��f����
p��
�
{�!I�/i���x���R�����8�����BR0J�!�mk$%e[��Lv��0����X"kV3�*����<Db�����$��&�)�0�Xf�H��S�Co�^{���css��6��������������}��e�BSO�p���<xp�cmnn[f������.��y�3�bj�����w�./O�b�;)�Zi�e	;o�����x�0.��tX�Nf�����9!�	�ZwX��
����J��	J*��U����8��=3d'R��[JF�22�2��cH4`�v8A	�����XB	�����k�F����������WP��*\e�)�S���o[�V�VZ�v�Wy��3����#��}
����]���o}��-[v-)o����7�����U��bby���������{!D�'��e�A)������%�n���m�M�}��5�}�S�5�V�F�n%^��d�}J�/M�&+J�XJ?UD,�:�s���BW���=K�IhQ���BDV*��J��6	�:��8��s�����'Wu�~��Y���Y���`��fh����??�����_@�3s�>3��_<{~GQ������o�/+���d��\Q`2W��-i���l�	A
�Q^&������$mL+���#��I�F�����^&)��$��4M2���4�{YN�G�{�A�Y*�����y�]�����?��|�=���O=p�oy LA�{�x�v�{@���x�2-���#s�������< ��4&X.P�<x��=0HKWy:=L��H5�
����4���#�{(�~.�X�a�<�=��-����C!h�[$-�H�-���NRp��ha\G@����D�0F�\��<������)�e(IJ��$�IVf��)��K�Mc������R]Q��H�Am�t�\'��t�l�.��g��g(����������k9�G����'�i�W�C�_XTX�P����a232��) =���W0E��sK0�L�Nk�M��9�W����z�����+�Ag�1�`�S����|��>������*1���\o|AIq��'�j=���7�rN� ����c~������C��I��J���?�?�?���y������n�-#�/o��g<���=�x-����|������p��lR��8�sV<X�l��(���#A���p���<�X���yxl5����?�������+�y����h;���e|a�x��$`��tn+x��c�=���y�a���g/�p�?��tv��6$l�WI�C$��D<]�aLj�'��<3�C�/$�,�x��a���c�����i.�p�"l'��l%v�Jb��B3��2vPyXV2v��+)��~��c��m��
�<"��#���y�>�k������u�[a����(�M�%�!
&|��SI�N��W����x"2��DW�H�t&zQZ@MF<���������U�%��4�*
V%A���I^��w�CN��x��L*�a�\K(�=K<����@T��T�v6G�
B�=8��Dm6�u�nP7�����)t��6�E��z�������Z��������U���:TQ�X�31	w�v�T����E��]�x"c��3����~qI�1@G�<5��Uu�f�Q��w�_={��]�%T:$+K���Z9O��mH�z�du�����c�<b�vL�������^��Z���X}P�4#V`%�Q��-��������R*y�axb��x��^�H�l��z8Q[d���\l�5��3�g�����m�^�����:������#���M ,�
��2�&�a�KZ!�� B��TCX7t���>��?8w.�7�a�7������)mBc����T�	�Tj�n�k���K:���
����A�)>%�!��q���r����$���l����1\�������}�����=A���O�K�{��|���������V���\��<ka7��YV�4+���R�k�D���1{��?��V�0Yp�[L���TJ�x4d���t��Q3(2�d����!�[uH���a
 ��3
S�����|$���������WRE{u�����;V��K��z�!pa���7����3��.>&],H����=�������py����������-��F���+�a%����
�29���`t�m������v�nO��A��������.�v9:I�*�d���uf������Vz���u	$1:o�e�v �Pd_�R���m�S|y����'��f7���<�#i_����#Yqj��*z�h�nY^�������w7�������m^��y"�?vL����Ki�}KZ�&nK�qLo�&CW*,U@�,h���\Xc�b|��$��81�9��
n�X�\�U�`�k���<�j���H�tH/�����b�tV�[��c�8�����qfcL��L�J�1[es���?�%%��bFEm61B���{�G�������'�o���NSA�b_���2k�XG����C�e'�;�,p���`��76���������y���e�����wF
Rs�U�������������,��������g��f,�%>�i�%��H��Q_V��:8��]
�!|M��H0�z�N�~3c�$�)���������` ^a�0�/`Lo"� �"K���l�B����IM���-8�#��V"�[�Eh�Z#KQf|&�e��r0�����w�����L���8c\��
�S����SW�|���%��k�}�J�&*I_��]a���w��������yZ����=�8�;��}�}���d�}��l��[�ne6'�I<��4g���M�c!!F,TA:��+�x���#��(%c��)�����o�=��,��3�������������*c�������u����B�N��a���[�3�Xd	�Z)�=,E7�W�h�j���~���A�IZj76��T�����������
���o�����
f:��y=��o�������������x��Y��[�6�y���c����<
��j65�R�#:A�l�R�g���I=��}d�W(PH����u=CdGjGc�CH3�Up	3��:*H���2(b��������jV��y#��d�55�{�[S������N�j(��"��&2��d���-Dg���Y;���c��&�H��I;��h���)~<o2��&
lR���&��ob�<�w`�P�U����p��U�q~�)���K�-0��f��#SWa���
o��
f�*���s3M7w��d�w4Q9o4ju�n�K{]������K�["\.���M�J�G���rD�d��)�an���ilb���-[^�P�]�UX�Z�(|f����BGe���{���������I���PCO���T5�d�@��Z�h�*v7?�_Q����yw--���������oeI���w��Z���1K��w4V:���9�A�'�x�)_[mn���W�����V�)Dod-�yxL0�&��7)����Y�lb�z��q���9�d�S;SGR�����#��E�K���DHO,H�NMd��`��#��M?��'"�:�_����:R2�1�9�v{~���t��M:e~��k��&2�n����t�'���������"�/L=K��s�Oo�W�����==��_H������sg�nb��jZpU`V�B_V��9�{���U���,?p�
H�n{�����ea`��hi���`������W�9������Mt�-"4M�~����D����ep�	�\].�����3NO�Jj"*�IE�X����i�a��
���8���6xM�Lx�0�g,����������+%q8+KHq(��a�}j���uQ&��)9�d&�����N���um���D��?�hC�����M���4e����M��y}���CscO������=�9��s��]�2�k�Y���}�����U�����MSV]��U�i����BRz������i�[�5?��e�W��n�����x����G�;��fS�x:n�����l $\�����j3!���2��2`���LK���?k%��k<dd&�b�S�_���2��(<��2�dl�`b23p�s�����a�z�zX���yY)]#�:/un�F����z$��qq������V�R��� B�g��&w�,TXK�����{y�����u��;v�����^��o)�q��G5��C��lZ��_jp`~�=��aG��}E�W���/���s�����������4�:��[��k^��%��5F���.��dA�R�hlN�����/����*��2a��n&�1�f���uL�B�
��Hl���l�F�������ugT�����Zu�����~���\��'���J����O��+����G��������+��}@y'�t�����oJj&�����jc�n�JL�	+Xp���������&�M>���%W����3���/%+��HG��diIg0Y�t��br{2sLb�d�,����.dQ��D�(j���^�r�#V<���S�>�>�rY�r�f�`L��D��;���������D}�!����9���Z���6�u����g�.�����y�%����t�Q���@�#�Q�`,������g��W}H=�f��c8�v)
�4!ML���������p��L<�	�� o�k��A�"d3M���-�������?#B|2{�~�~�����d�Ci|~C���O���3�I�X�*�|o}&[zm��U���
w/��W_L��+*9wIY����d���.�S:���VA�H
H�L`�Y��H$�1�B��>�A����!�1�8�/s@��9kjzp���I�,w�# @�Q����F����M:5~�{���%����:T������������fH/�+��Av��S�)8���DA���x�x�����dL]#YR�11� �?L����<V%��%3'XYB	�\v���`�NK��!�]=��cD�9��]����������w]�u��]oBp�Qh
);�{U���T��S}��g8?1V���,
���~�����-^�j�lq�[3���PW�������;mK
���Y�96�>��{�������)q���s�d&j��;�1b+�����.NE��iiC�����]�\��^k��|��<X`#�V�����N������>������a��R1~�P��x�8\����d1,(�����XW�-�g�/���V���
)����[���p��MN��W��.��yDwQe���&n�?���L����,�[���1w�'V.�!�n����d��#P�j�'���I	����/�d�;�������d_��x\��p{�����k���;��k�.z���1���I%����IH!��4!t�����Lv2t�0�����E���
ZQ{A{Y���TF�����(��s?����P~��0&��a ����~iF
\{�Y|�x���7�#T�"}�%\2"�m�������I�Z�x�a���<���&N�����j��a����H:����i�&�Hq������U\(FH����&��u_��!b�/S0����u'6������h���uk���g���UT���)SM�������b���=p����+�l���Y�+�����I�W��������J���H$�Z@H�	|����Oy!�����e^x�{��K��^v��{��
��p
����|[
�Y[��X��+~���C����~8��=~��o��?����/�����g?|��w���
   ��?�����������}������?�s${�
�Z�^��~ 5��[�����,���~|���I�V�-�j?|}�������c�A?�J������i2(��:�A�k�w����-��6?�m���?�����#U�z�O��u�\��+Wh�K����B��"m*C��)-�����!�V�!_.��C��$���9��Z$�7&R�����4y��%�����'����~��.���$��J-e	�����GyB�9���f���Y�ns}a����m�m�d��5{�hDx�_O��T��
�`��}�!A��^T�l���""���C��7�I(uJ�6��vb������7DG�U��kR���'�O��i���hI��t��a�LP:����Vh�K<t��W��q�c���l��r,�L���.�FlZK��u��S����e�nD�U���p|�G������_���K�_�C|z����7�8��k(��������HLm�Cc�
TD{��T��e4�o�:Y������z�|������&�C��b`���P�9KmS*�%2�5sJ���*J�*Z�����W�r��k�QY��<���)���e�D��+����� SB�eU|d��<a~���x�g
m���e9���z���{���5����d<�VwC��Gv2|�D���gb�v���������q�-��v��R�$���]*�nt��6�=�b�l���6���������f����klKm=6��\h�]��9��&@�0"��9@@9	�}A�G��)D�@,l��������L��[%6'�4��c����*k��ZF�*�q������R���)�����<���
l���>�a[�2r����9`Q
�@�1���?(ss&�����Y���43������\�Q����=�����x��<p�{=��h������c�#Y��v���:�N���)�m���*]��.�.����:�F'I����n����o"S(�%"/f�X!B��#n�-�*�#�A�Q�!�R���!������g��"�'B�X,6�����������*�"���(������� �?���"|;���.�;�Nn�����8�-"����"~W�������="��[�KD���#��"�I��G"�a�x@<J�i��b��d��(�F��k"|"�oDxOi�M��	�-"�a�u"���!I�WE�������x��[EX+B��D���"NA'���\��"��D��{)��"n���D��*�V��+"\��?�u^���f}��"��9�9�������H��!7�n���r��"�6p��U<$������W)��X�����LIg�l�C����k@��F��Gp�e�������������������r��C-���M���QA�Q��IuS�|��.C|���%�^�#�u��������\�5��d<���)�a��3�@�9����)2�C"t3�������jK�F�����}4���5�M�U�e�N����S�t�JABB���w�������"�;��6n��m�[f��W�M�(/�g�^}4���"in��wc>�5j_���jtE���a�
�+a	�������|b�	�������/�����Q���Q����Dj�^=���f�����G�O�t�����S��J&����lgm�J-��6����i#��Z�D+-^l��i�[��yj�D�l�:����J���1�J�!"���2��0T�s=$�[{]t8�}dE+��
�K����������^=�x`��G��<	�~>U?�������w�k����m����7��������8ot��J����)��am��Z���;a?`���$��"��`/���d����U�PY_��N%�1q������P�����*2�����H��tR/����}3
�B���[��;l�����|�,e�%����)+���y}��v����S���ieb������`&{q�)��d1�6O}2�;y�����F�\��&��ZJ��y(��}��W����l��;����4J#�J����k�MJ���O����El���YI�A.�J�Ce9�O�c[9L��d��r���
f���*q�7����!�"���/zY�G^�p�L�A����"-��<���fB��;�O�V��,�n>���������&N�,���4_ib~��z���:�XS��~�~zX�����Jb���"�HE����(��N��R�:�u��6�v�����_c��a60�3D� �K7i*H�h5<�	�K�,`V�����za��OxK�@P��F��
�
�LQL�q�9Xm��[�����p�����A5m���al7�3�vT�
�n�g��\�@h&T�h��J3����P<I��F����_l�L�l��
�P�Vc$=R��2R����&(*
�u����3q�^����:�����W�����+����_~��wc��	4��^���g$��w�#�a���O�:����'�>����
�Q��
�1����[zM2WVU�L�s*���_�|f��&���4F��,",��{���	�0�Yzj���`��#sq���
\>���0������_��[�,^�	n�����-�����}�^2��W>mEMc����������o����3h�`�u��?=�}1�h����>��U���K��C�������.s*;7A��\���\Q~�"�����zt���6�=�/���g�m��]��R�*��
�/�=�������7� ����������%�7R E �1�N#a����1c�1�d	(AE���p�;m�d��
,6 &��f���z��G�#��v�w�t<�8�x����3��z��0O���[��x��E����a&Y���4#��~�S\p��?s��8��{�8���0�����`wv�#��,�q�t`��1��2`�#���n	�	2������� ���
�/����!���a��}��t�*G�����8�����o9>p���+"���A� E��|�A���c�	����:9V��F��N����8Q.�Ux����������~9����fDFk+�Dd$x�����Ou��8��=�t��n$�2��k~j��(�����O~c�%`&>9%������@�$���m.���s��� ��Z��W!<����Q�@<����	!��R��0i��6	&�I2�J�)�2a�������E��/�g-,x/}�/d�:��B7��-�����B+�����J�1�������I�YSiY@����Y� �gi{-x���b��������-@1�-��B��2~�TP�����-,m���ErfI{IH�f�Z/X8��:!�[g�#�R�`�j,%�������q9��6����
T�"��
W}T�)�n��}�O�
[�����i��YI1�Gs����,��q���i*�NO���sH�������������~������c�	
�4!MX�Lhk��f�fd5T�k����j������w]�*W�)��B����q�q�sg5�{�`i�������>��k�}���a�Ki�����;�������g3B��Gz��)L$�B�p�e�L����$\�IT���%D�x��Bm@��9|$&+T �Plm,��}0VkNNRrf9��	f��I�#�9B�]�7/X��9������9��#9�?6�@aN �;�����9p�fm�����s����H��F�*G��J�k�Q�����������9���n���n7����pc�>u��nx�
��p�
;�0"���!����
7��sZ���"b��Ey���I���������fH	-��Gn���o�a��x�
���Un�:{�37|��37>��g���
�i;�x�;��Y7���W7��
o���������gz�Fa��}�~��	��o���r��q�L�}�Q���L���v� �a�(���?�]�������1�C���*�[�������w�]�v��4RS|������nvL&k��(Y�������H������L�w)�����O��J7�n��UJj�d���s����T%(�������s	O�O3������W\�.��B0���N��/�_"�o����Kn��+/��
�3��,���!y�E"��o+��������qS����UzU�����g�~z�Ue�R�R�*A����T
$�R)u�3a�k(���ug,�]�����������4�d��fI����$����<�a��~��!j�K)��N����Up��^�Y����(fCN��M,Y`�.����_���Q�b@��</����q<�3H-d�K��U����?S3o�a���)5�]
u�:��V�T���	����WlA5�o/Kj��)apC��4r����Z���-Ti�PiZ0k��z��I-���v�>�-�����Li��ZxK��p�n�K�;����h���R����OX~���W'��)�n�z-C�eh�� z�h�3�S�6���V�i���N-ss�_�w�\'�����[���hM�-����K��K�V�-���&�e�����{�r��s�%ZH���}�?�t�/s0'�0����2#Zh�B=}���`��\4��<�%���i�Z��aY%0X��!�����G'�����Q��h�}1���e�i��\d�#M~�B�s���������m�n.����SuY9�~3u���m��w 4?�zv�SU��c�Z��k�^�Yd���O�lb#Y�.�/�����1���T���Y�!�a#&�
�^�!�1c�OzI�I�Y�+k��K#�N���k�8f/w���G��B�%gT0�6$���o��s5�7d�!��w48������D��9��egJ��,^�H�w[�[\�}?g�]���k�]���L���|���A����o���
�"����sg��_�*�;�VI�����B����h������������%��F�k).A6�|��=K��ZB����D�z�R<G���<�;��\���I��/aZJ�;H�K�vr����\��ZD���2�$��A��L�\v��O�������V>�x3N�����������$��;�?!���;��H<��A_�w������y�.�kf��|��=�HP<����~���KLq������j�4�i�cU�'t!!V��S�L�0��Wc����^����>���Lby���2������HYb3��8�r���(���"��G<
��f"���&I�$Xr},��� �A��
D�JD��a��LD��E��a	�
+�&�z4�BqP
�(j�a5i��������hX����p,���X��&��hP

c���a�3��0K`��a%1D�
~6V�O���a�bOG�<Jb/D�jT�^��5h�
k�o��h8mQ���8���;b�Z�m��"���N{�c�m��_���^��.�������wuz�w���^4��v�|{����>2�����ch�}����w�������/��^��3��ch�������!{��6���K���ix�G,�����
�����=}�#]C$�����Y���:F��G������tw����Wv
�t���^��U�C}��}+im�����;048m�H��.�������������R�w��
��(�J�Y9���Uy#�:���z�I�=�#k��I�?L>*�HZs3��d`�D`����.;E?L�wwu��

��Z9���n�[���������@C�D��?+Md�D��C=�� ;�B+Q6y�!�|�Hh�B��D�MB���@yHh6ZC���0��.��"��rY
y')5Ul�Q=	/@�Ij��A��A`��Z�B�I������$�W����>�Or�X?�K���Q�>�o6IYIR��:�\��������K������H�(�<�v_V�����Q"B����;�'�n �e��\�RaD��_������IyJ��+e�#$�<@��Qz�"��[�)����0������7D�o�6������'����D�z�� *%+�m��s+��Q�9��@�?-7Bf��L�.y�{ld�=2������R�_�wJ������?�������5���#K���L��������Pm����dj{����}d�H����������������/�f�:_&����s���J��k����C0y
�]t
b\��[(���@���9�������KXwi���K{/����Q��?~��~��f��_���p����t�W�H������~��<0
�f�m�_�~�����k����������2lo|7�6��N�;�����������W�:���W�z�U����'�'�	�x����P�^�|����Xx"������0�=Vy~1�"�|�����B����0�������{�b����g�Ne�8�f���pf?�$���/��u��}���a�a<�0���'�����{���x��������e��
������DHh���>�AA��N����,0����5��1�����yL�:����N/�������0=+��T����$8k�u,��x,�<S�t
B��j6��R����cu:�n�n@�Ww^7�SV��K:���cf��4L�_�r��VN/�+C-a�N_L�����bW54�4x�i���9�u���������p'	H40FB�q3��42<2*�-Dh���!�?�"r\�$��
����(v
���0��#$}����0MR�\��z�� ^N��H��0�&�������r�
endstream
endobj

8 0 obj
16520
endobj

9 0 obj
<</Type/FontDescriptor/FontName/BAAAAA+LiberationSans
/Flags 4
/FontBBox[-203 -303 1049 910]/ItalicAngle 0
/Ascent 905
/Descent -211
/CapHeight 910
/StemV 80
/FontFile2 7 0 R
>>
endobj

10 0 obj
<</Length 464/Filter/FlateDecode>>
stream
x�]�M��0���>n��vb�J��E�������J�p�����n+�z����<���;��R~�Sw�:c�m��.�S�c����n�+����\�)���-�z��jU������i�O���(��>�a����cZ���+\����X�U����v��^C)Y��>}��sJ�����2��T��>����/�XU�Z���u���oM��������ShU5v����%7�Z���F�j�cn������
�����F6�-k
�����x����+��Y������y��v`�;������gM+1���`�[8k�[9��V�g�Jg����&��
�=�
��L�\C7��q/���`�;�����}�{�����.��
�b�����wO���{q���$��78����������H<�
�h���`���OK���G�_'�/���X��c	B�L�0��s:O3���%���
endstream
endobj

11 0 obj
<</Type/Font/Subtype/TrueType/BaseFont/BAAAAA+LiberationSans
/FirstChar 0
/LastChar 55
/Widths[365 666 556 556 277 556 666 556 556 277 777 666 556 556 833 610
722 722 277 610 556 500 777 556 556 556 556 556 556 556 722 277
556 222 500 556 556 556 833 777 333 222 943 500 556 556 500 277
333 277 500 222 277 722 500 722 ]
/FontDescriptor 9 0 R
/ToUnicode 10 0 R
>>
endobj

12 0 obj
<</Length 13 0 R/Filter/FlateDecode/Length1 9252>>
stream
x��X{T[������2��������]���g, ��A�l�dI~''q��v��C���I�����/N��4���g�H�v7M�=��n��������d����o��_M���9��^43���7�|����;�hx���)� ~h�*L�� ��Fk��E���?T}!b	�Nm�!I
Br�����?���DH�	����;|6�UB�/���1�z�Q9�_46��1�Z��p<8���*Wo>o��#d��H�x6����\(�?����P0������=6��f���@�	2�������HerER���|�oK����AYh�X��HjP&������������Bo^F�F��S�}4��@��i��G_�7��,:�����
�:�9���5�5�Fg�)t���<�M�(��>��P
���= ��YA1�C�`.�64�~3?>}�.�]�L�����?D����P�
�cTJ��b�
���=0�aE�A�Sq��x��`f��G>������g���+�����E��~�/�N#?�r����F�0:X;B���=�r��|��ks�B��������,���(�5���M���a�,M��xG�������c������-�UNGsSco���]YS����bY��Rj6.1q�z]Nf�Z���LNR�eR	C02;8��Ab�\�R�s^xoxD��u�#���k��9r�&��ohb5[�jK���c�w�9v�w��>������H���� 2����0�u��5�����m,��4��er��K.5��d%�J�#���z,����!H�J��b�wX��t;��z}_��EH���.�$�dM�\4�����;c>;8�F=��an�{�[`�06�8b�}B�I(����]���}��kv&j����<�7����X����,���r�������#J:�X�����'�������j.6��9a���Q����
��}��3�k�uv�
�n�;�1/H�g��+��������F����t�fy�a���Y�Q{�VS�@<���BOV/��Z��1��A4[��1AR�2�9�^aj#��&
N-�}��s1M:[m�uY��e��
R��n�B���"��I�����t
[��j��9<����0����)���7�{1r��Ya��!�7���\H��o�����w��!�aBf��<C�Q���Lgf1Os�j��t��l��g����lh9�k���M�WG�=<"�<�a�i#�[��>p����D�J.�tzqF�4��[����~���#�jNR�������@�	�b�&Z�� `�@p��P�b5.Ji�6��n�E���P�:|�	=��fTJ����`MFY����������L��ML#T�BS'��E������q}�+�n�6
��r�D�zn�n`Bz�^`(�����\a���`]wt�,t�1������A����0�"]+�~��9�61�hq?�fx���1�mc\�p��v���p����E���V���Xj���q���;gx�������V�{�g&M����"�s��"��RB�TH�2�R0
Q_�����+"?4��(S,�0�%q�zAF@&��xQF�R�`�������o,���9���as��W?��,EH�|���k�r;���r��!3p6.5�����9���5C5,��[�Yf0����K2��gd����a�h��b)���2?�=�����O/����	{���>&������w����\g��F���")�������<���K�}�����Lj@�����i�4�U��d�������n��
�4���ee�8M*�*�H��J/����O���}�p�����������������~���sx5����O����_�l�N�|dCw�
Q�6���60���X��n�2��s�D�{���!�X<l�C���J��X5�� ��[��s�9l���[./ Y�2���a��B*����z)SN\����y���&?�<��S�x�[��]Id�
m�'���W��6���S��/������~������X}��o���W�z��P��?��r��_�=����o��
u@U�;e�J2����p����&p�Ve�w��k��Q�8��x���y���_\Jv����*����>-�O��N����D���R��Hf����&���W��]3C��\���� g2������s�%7��'e�r�fcDr)	�$uIJB���\O.����e�H6��m�@����j���m�f�~S�mV��	���)��d�( 1�`�\��361i�FbIU���29��1��k��3*�����u�bo�����#��������L
,+q�<.�{N^oP�y�
���������w�2;y��_�p����cP%�7*}-��t;���v�h%���n���b�
b���-����*^UFT*��W�����\��LS�0ANh��<�	�Z����d�40Yz�������xn�+�O����u������#Y�ei
��U���8p H�T��|PS��S'���L�C��M����L��`�f�8����Y@��v^r��x���{�j\�S�o�����g�Mc�+��y�y]���������y�C]�;K
~�cG����	�#e�{����+%������gv:�Gtyc�+S%�GG��V\�����n�����k�����*��/*������XB�y�#8za��i>�-9~K�����$xWJ,��bq6�/����P��HQ�6KK�$k���������d]2	&O&N�O�$'/m5>0�� �����y>`����
��	j�F�����!��D�+�C�n�X�c�B���&1=Gh�� P���U��Ru��L���������O�w}ea]�m�+���t���H��D�����8��H~�?9xl������m��������������=�{|�3���$��2�W/�S�~���Z[A���,��hS��"�����������|4E��fgeg3��9�*��*��kkIY%_I�����j�4%?"'A�����������O�t��k�k�p]]�������;�S�#f)%<@\6��,3�
�Tx�*�"*UJ$��/���[�#`�\�����Y9��a�	��@y��t�E����?�00�*RY���zEa~uW�����`�����y�����������F���N�u�ZV�Z�,m�����*����yuE��`���X�&;�L�_���=�{��������2��jKI����]��������������n����7�{2f��j+�����zF���J�t��rR^.��d����ZcPZ�^*��@]@

J�#�(#D�5U^A�7NOM�����r�s.��Q����{���RS�)d��,���,E�����'�C�o�V���7G�w�2���\s ��������H�k������`���wY��*w�3]{��������ot����E�n����P1t���N�����;-u���D�w?(��
����d���gH7KH��f%�+�[�S�����R(�R���d�w�(Sfq�Z*��J��u>��%���\kNJR ,���c���e&Q*�Pd��R�2`C��2�9���n��E��r���lV��v�������y�����-�:��m������ee�5Ez3\��
����zfM����"Z����7_�v
���������7?m{�q^s����+-b���C�g���������$�@O*�+)��y�����I�aiN.���&���I�a#c4���NWP�)�1U��jCU�JRU��E�X���sK#�������`���p������g!El0LS%C��7_U��}������m��������J�<Y^<�����z��r�
t��T6T�v�-+��ky���UiJ��������[~�����BX���V���O�m?�������/�wk6�2V�>��{x}�ssK��T����������BF�
���B�LB�Y%v	� F���$9
2�&�H�MF��vI���I5�0,��\����E6�Z&��:�����*���e�I]����}Ru<C��m������b��W��������}����X.������D��]���S�|��U��H�����u-|�����dA
(��v����AM7��
��%TJ��fI�J�F�Z���*C�cN�<fBW�?M������C�������(��;O�?5��~����� ��,�w%�N#�I���
8�%C�� =���%(M'h)JC/$h���-G����@��%A'�4H�8�6��Bn��t*
�Wt�'E0;�$w�lL�p�2�	���O�jd������-f^O�2��y?A��G���(�(A'����	Z�VHW$�t�4��S�/�8����=���G���q��-/+�b�|���5�-�!�0>��
6�����|����������v
���^6��&���lp���m����7��no �����"C���/���w��q�����G�h����R~S��������1X��?��A����n����Q�f{n\;2����!_8��`t���5�����l��55��`���o��m�F��H00��j�����[�	�!��'��/�3��E��X�e,:1�"��VqF��V��k<�cf#>K�G���o\���|CQK0<j�����������f���<�h83���w�0��Q4������*��N�2TT����"����-P
h��[,DD���m�X�����5����y��(���m/����a�dA4�7�o���yh����
\������A��!��3�A�T��o�go�����K����Z�R,-��g�Y�R�_���C,�Q�JT�����{A�[��GR���lQ��3f\3����n�m���-�K �	������8nam����q�9�,
���n�8g�(��9G��D.�j�-eE��?��ny(a�"R��?�q��1�x�-��	�g[���/(B[oYc���G���w��mvhdiK�.xI�?"�G-up��h[D���F?��������&dwz�����������"(K���9��������i,����s������wurut�t�W��J�W�W'���z��]Q_��t\	]��"K����_���S�
��rf�~y��;w���K�����y������K���cz/a�����N�V�7��?���?���}��a�y���N�e�Y�lh��_�f5�N���W�,��y���b��J�N�����/1I!$�#�S� 0S/y�<���"�|?�����
�"�SkO=y��)���'M:���t'z������!�(	P��r
3�c��_z�H����t<�O��t�N�.N_�&��u�����G������_:.Q��<~���q���ju��E�N�XR�Suo8����G��zt���?�����a�09�������9ue��Cd���)���_R�%i�N6V#{p��L����[/m�����G#v]@�H����ag�n>d0;�PY��@��9����^�������O��	<��o�����h�+�cPc�b�K�*��,V��K����uw;���~h3�5�R����
2X��2�����]�h$_�*]��h��H�v�w��b���,6:/t`vM���X�+t&���W�����������8��������:O�=�FZ�U�'�s�bVA9����W�d�����z���W]��%�b�u:�]�A5���TV�ZUPuXuI5���AvU��
|*K�,>2��m2�����Zy��������;��~���g0~�o��C�1�U(�v���Va�S@��g�Qc_4�j���1�D2jIA���N�G��).�RY$.�@D�����?*�P�	������5���(��U�:.�����.J�#�6�����������Fr����
endstream
endobj

13 0 obj
5990
endobj

14 0 obj
<</Type/FontDescriptor/FontName/DAAAAA+LiberationSans-Italic
/Flags 68
/FontBBox[-271 -303 1061 1014]/ItalicAngle -30
/Ascent 905
/Descent -211
/CapHeight 1014
/StemV 80
/FontFile2 12 0 R
>>
endobj

15 0 obj
<</Length 270/Filter/FlateDecode>>
stream
x�]��n� �;O�q{�����1�n�������0Z�
�����&=h�qf�0f�����g���<���&3;��Ai�T*��(���-�Bo�L�F���H�r�w�\����d/N�Sz���k�v��F��2R�TB�y�������m#CZ�eZ�
���q�FF�d����b����V��_._[�^|rJ�P�Xy���d�.��G�.}����'�!�@�K�)��>'����3�>9�C�������5P1;V����V~��5���
'���
endstream
endobj

16 0 obj
<</Type/Font/Subtype/TrueType/BaseFont/DAAAAA+LiberationSans-Italic
/FirstChar 0
/LastChar 11
/Widths[365 556 556 277 556 556 556 556 556 556 556 556 ]
/FontDescriptor 14 0 R
/ToUnicode 15 0 R
>>
endobj

17 0 obj
<</Length 18 0 R/Filter/FlateDecode/Length1 14380>>
stream
x��zt���9gf$�d�lK�[#���<����0��m�
F6���c��!HpR��@~�Ch/�k�J�)i�����i�O.m�
m��8p�Hoo��3��I����}k����3��>����>g���
m��4�$�l��~h�B�-�pl��aa���3��Y�:�fC�����y)�5����~}��i������{��O�"��x��"�W	����6o�f|m'���������7��nC����
��I��OCY���{`��w��B���?8�k�;�P�@�C��q���CYB��p�����	�r
��Wkb�:��o2'$&%[f��Z�����������A&�E~�v��(����g�t�^�W
���I8}�>�V�C��>
�s��q����c�et
��u\��/����]�S�������~��`�������	�<��h/���`:�
rm!��a�+xU�	t	�{]B��t�2IP��S�4sy�	2/&{7�~���.\������ �����A�[�	t�-,�r�5��*2N���^�5�����FW��x?�dV��9}}�RY��%�r�(<�����?����/
���?��9�H���������G�Jt�������5�?��T���������a������������3[��UUYQ^6������,������H������F�^���y�R���(����B�]!6�^W�O��n@tOCt�@yn�		]2�p;����D)E(�)Jl*Qe~�PkBo���1�����{���E2�f�-l6h!�&��!�%��<��Fk�j��1�z�}�O�����5j
e��p�,,$���A*-�6�d�v����jk,6[{~����^#W��2��bnH)����h�p,����1Z������v�h1��v���2:B9��P���a��P���6��\��L�S�K�2va����v;�;�Qd�@��zGG=v�3�5�=61��.���bbF��a����&�����i��pyt��%�����m!������v�L���>I��u����lt���$�

����HY@�,���t��H�9;Ycj�5#�5S���0���m�!6c~��t��;4�
��Z:vCH�W��>k���2�R���B\&�ZMo+�65��_#�qt�i���������vE�7�%!?/T��L}K[H�@���Q�1�Ztw�����r��x�����b��7��M��B�sC��'�*����=��]5(/{S�)���t�H�|���P{
%6��u�Y;���:d�������,����no���������CdnK[}�����mfT�He�f�~����aK.��P	m�������Jx��*�
�pK���J�
[�$5��j}5Q:Z��)G����In
Z>s�,�v[���#P-D;�*����*&<���QT��t�mv����'���6:6�Y�Qe�:��U�m�i�5!TO�2C�e�rC���T��K��'��Q���y�2�G"�|~�%,�4Zd���l�t��E��<zL��-�Q�����7�U���A���I��E���eN~8�9�����c~�������-m�	&s���K���S�2�P,E��@��(�dz�)	����r�g#���a�3F"8�$����$G/���>�1��Z���������v���4�8���@;�Y�0Q���v����>���)�:�WP�V6���;G
��/���IP
<z�V�����F���J6m\<��~[y�!�cEs}\����<�)�m�3lF[
����p���j��ip�*&>��T�6�����-�����#vXs�O���5z�U�����jN'r�����t���A��0.��!
�����A�8���ee��9n��BN@�]��{Z��He�bIq�\q�,�SIW��i:b�8���G����_=�5�����-K����Wn���6���w�����b�0��{���d����9�nZ]����	T��Z�t..�.�._��������E��y��������T����t���e���b��B4kE��7qI�T�Ej�%�*Ox�~�a	�������b��f���06q�{��u�-�T1u	�(�7Y
��U�������pt����C�;A��ga:p��B;�6#�I����x����������+m�?�$�
�j[@
<�vs	YY�_���������:avY�6�v*'>cG���V��W���:6qU��iV��fy
ZX"j<igs�����4�n�`�<y9������y��5�q�K���V������������������Z�E���[���������4��"�������]���#Vf�b"J�iw��&cO����p�R����nq)��o]Z0w���OL�����$.|!F�����@�2�>CHM���O]�S+f�������y%mUi���;w����������W���m\�{�{��2+sL�'j���OXi�����UnX�G�0��R��SH?��	^���zf�1u&�u������b��k=jebR<��Yt9���B��kP�!7���v�O.��A#(�.7]Y�T���1'��9(L�}�!��eXSnYcER��R������/�W������*:g�y��
���vs��#�>�*�B
�&y��3<*MV������i����s�p��y�qIz���Dm07!CNJ����\�xdBAV��tVa�N�"��l�'����1�P[&6���i�;�z�����K�V-X������t���|D5_�����_y���;U5[��$�-+c��T�z�����X��NW��k�2<m�&�rm��R��wVw^G\:������8Ca��e=���?x���A�S^���V�}��E�E�����	x�
t�1	�1l��;W?��,\����U-5�c�� ���g7;J��R~��2'G�STb�_�r��-[�y��\P�Vg�%Ix�u�ak����l���������HZ�2���3�"�F���`�[����D����@�B�+���37�^�h�.Q���D�M�'�Kd�$��D{�W����X�7����������]1��e�.�
���V�P�b\�)�K���)4�pk�>���f5������^�^�[�������-ZS��������{�;���j��o�����6��eY���� >�>��H��v:�l9��R��@�(^���F�R��?�p	��P���J��+ii$Mv�`CiI��s�)�aqV��+��*�\��F��:�0�������/�/�R����a�a��/���@k&[~�������K��[��)�$�������q���^��hA�N!5�)t(gR�H
�X�z��'aQ��� ^��qu$�)v��$N�#K�%�?�v��?:;����h/vGg�
����&�A��g�M��s��3#6?.���Kf�����g�/�S+^a9�P(��NuN K�
k�E�s
�`�7RY�D�t�����G���A�6d�'�g��o$A.���4��9���2(\f��\b��6��b<e$O?0��F��I�q����v�8D�$�OD�������.�9��1/���/�e�E�E��i5��LZ���.��!~"��"�M�V��r������"~R<%h~���rW�
"I�F���������"��Et��"��:<!��"�'�a�6k�"���uj�9M�h%�F�0T@S�����Wd������R���7��F�J�?�Oh�D/�-����mh��"~T�00����N����D��,a��\$.GvU���,�a*�_�'2}Q�!q@�z��WdT9��D�S0BK����!�����Azy�"W9U34uM'��\9��N�����������ne�LwlB�����GBz�`�gef)`�W2���Rq�9��tS&R�PPX��:��x
��Z�s���~;B�dM�Z�S��Z�e���"&&��[���{;g�jWiE��Dwa^��w�{n\���-��J����7c+��S=�fV��,c�x�����~ ]����2�2$y�:�Ao�6pf���N	{�x��N�d8p���G��H��;�v�|�g���>��}U��{��v��dV��4�������<z��-ohiy��;�r�L����c�d����=(��@8�e�5j���Z��X��3Q	��������=xQ�F��b%�9�b6�������qIuD��X�T�9���n�&��vtNF/4v��v��X�<�6���Xl�13'|q
[��� �?��gKx-������p���0��6�G-�@v5�xU���T�����4!�u���
��f����z�zq��8e;��"bj�����`�L��V�����S�P�����G�k�H�i�C$������Q�D��K���@/�����P��-����.�z�t���,"~F#���"K�w�H;�.�"=���LMw�4�T$�fB����T*�t�qV�j�?M�@Q!F4@�q]TF�+�N�DD��K��L�JS�&��Z�sUv��#^v�� Q�RHHY!��RI�"!3!�����eW�&�jh"N������qn�4�;��
����2����bl��s�����4�W�LN�����|�6'c�,$Y�8U����"��c�x3�6�w���:A��ie��7��Y�^�_,6�K�*��n|���UJ���g��f�x��g�����"�)/7S�$�};���g!�(C�=��'�����.�V��<i�=����������W\�`+*����*��
g&�
H�q��|��
�P��� P��AZA�����Zj�q`�Z��1�������Z���� ��)q�$
����9�r����JV0�R��Y���������p�r�����=��H������u�=�	%��u� �;������L�_�[�s;J���)B�h�{����'�/||�����J&�N�b�:kX����d�

ka��������BF��^`9�T�5[��S�:����Mr�|J�kH=YNv79M���!���$E���,?����#�	���T���n��'-<���<y�*���/��4�w���Gxf�'	�yr���'!�,O�������<�D��x�(����������O]��N���T�X�c�.���:�/��� 9�c*������'�R���<~������U_����k{�''x������h��O��<9������,������l�w�y��4�&��y��+�z�a)I=0���L<&��s�E�A�����*�����"��7j�+o7��-�������r���������N�w�D��?�.M@���p��e���3�{�g._��|��`]�PZ|
��>���"�rdqY�,����Zp��CP�TD��:�q������+��<��G�X��#�rL�`r�E�/��Uw��~�d> [v�qou����>���x�S�/��Eah��#}��i����rdnw���K�&��4�m�R��"9�[�p�8c���!Z1��5Y,&V�\�.�0w�������8$����y 8$Y ���; 'M��i������GO����;�����"ck��s{N��mm}+���{�gF����}��uO�8�����\�U����@����C�������}K��7KY�#�����E��
�B�In����m��K�2#��Z��J�uuJ��.VbK��J��h�+�d��<��~M���k��T��EV��������H!����7p����L4"�%*XI%X�Z^�I��<�����U���f�Yg�X~d/w�K�]����rN(�ZN��]��X-���X��h,��9�,�����z�r�������)��|�Z����Y��F��5�r�~:lr����m�%5�)���d���`�|�����|�/���WV��+�{���`_Qq�S��O�\�g{�OnI*���������<������G�YZ�Z�����7R+��������X
�el��4N>��8�:���X}u�q(�2�
��1��P��0#��J�=-���)���F���3�-M��O��J����p��7f���?]�����XS�j��\
y],����J�W����e8�C@.�>�����H.rw����^��^4��$b���w�T�����6�-z�b[n�G��40K�{.����{����b�}�U��S+&sjH�?��r-W(�Z��`����<��0JV�`�a`E��~X�t���4TWpah��������_���!-�J5��*N�D�swY�3���&:!�r�PO�X(�m�y�,����������HC�u�E���7��_�=�����?�Bx���7'�����n���;0n��B>$�R)����-��xF�y��6���-��z�H�I������N#g�
z�&$�%��pD�������4_�cLW�`���kq�z���~{���Z8�{�B]s����G}���]�� [����!n&�-����11^p��j���50^��0���Ww����kP�*{����2�"n?���e��R��m��uI�I���m���x[���+��s���cw�"w��~���z��e�1�KB������� �`�C�(p@�/*������|�'�y��c+���x����y����	5cD������:8(R�����$�x�7������������V�2�O!k���\�_��W��Ay^���,Q;�mCK� ~�F�B� �K�6�>X�Q"�p�G�[�a���������}�?+��=q
����#��"vwU.�k�'�J��$	u�����9�."�,����*TE�*����Z4q���_��@.��:sInn���$��_���W.�#8��w������n�1������R��g���.fj�+����|�bX��m'�k�sW���crg5����
[��kv�I��/�����d�ez�'��7�~MU�C���r��e���������j��0#��������[mi���X���N��+3�RLz��~������o��J6I��jp�D�Y�j�A������9�_j"�	���Xs��Ux A1B���	g�Z_GHbb�U���
N����V��Zc1��),R�48�������6a�I)�f����vs;i����gb��8�%E>��������=f�
���Yciy�^��,0��t=����`N���'%��uz��B�
��t�`�R�D���I��g��������gq��!��7�x�����F��+�wx��g_�<�<k����	� �����[����������W?�����M��������Z��=g|t�I\#Gs!�l�p����ig��VN�o��qy[������"��s�6�^�6J��Y����0+��09�Q��Q����t�:uIR09����y(�df��yJC�4]��&�9������~�y
�=i������2�e�~KH��.�5L;���]���#K�K�$J�����Oo���*�*l3�J}������V�.��uw.[�
�9������v|�$}���e#K����'��y��'�������^�����[_����-���.g���Z�Z�g�m~Y7p?5}����W~�����{���[?O/S������L�!���b4��O����6��pKQ���Q%��v���-�����]o'�#�r/"7B�>��h���~���.�k���2���������I�k�R�up[��'�� ����S��H*������Y��b�7:���s���Yu���74��5��Y�t�g�l�����e}��[��l���DP����t��������N��kJ�����X�����`fQ<z8
s+}'
+ ><���N��(��gGa�pk��=S��-������KQX�f��;d�P:K:�0F�Lr�@���4���0�����0�f0���
���(
+�5��(�B��o�0�fpl�����c�
�-
k����(�C��������{r��*��z����<a�@O�0{�zA&
C��oh���@X8N���-��A�[���m�Z'�W��~a�*�P�p�@h��������
����|�K���*�R�P�"
\��-
J���F��@0�5��a� �������{�70,t�
-S
V��������p7���@�������=��`��h�����X��M>aQ���/���;�w�qGAw��hz����nxK��������
oX����A����`��Gh����'P�A�������k}=���5�;���;#���8o��\����Z�������P?Z���0������[�d��JZ�|��u�(�������h=�	�8���>xo��R���j��%���D3�~���a��Z��!�p~����������5�@?��P�s��Z���d�C@�/����
S��w���4�)�B����4�U<&9��'=��4��52�a�w��_��
�2U����hX�m@�j�������j�e��{��~����^�6��#��������?�
]�C�*�_��n���"?,�1Z�'��v#'�C�+��9�D�����?m7�����{
�F��@��fsaTC�P
m�6��n�n
z�w�������,}����������h-O?��'k�@�����s��t�������,��,����e���Md�_�����F�����<�Yi>���
7��:����/�q����[�W]%�+�y�{�����W�?]��\e���,��/UY/V����*��wc8�x��9[�S��<�%����8U�N���-3aE������1���/2�]�����}���.Gs�w�R<�>��Z���a���g^��J�sf{��iY�1l���VY�;�����NJ'�NNr������WOrcX��u@�J�+��+�_��,�+�G�{�c"2C�w���F �$);3�c=�<Z}��QVKGufz)���K�����D^x���|c����d>���*�?�������*�������gfY���i�G�'<��C�}��[��Z%����>�P�u����=�V�^�^�����}��^V�����������e�E�q���������l!6�=w���0���W�?H_�0����T�@�u�#Z�qbk�;�U�fZ0;����+Z��^�Qg]���.��l��Z���V3������g��m�Rsv�GjNM�G\�gIS���a����������@�p�������$Y�<6�<��<�lM�F�o5��V�Q+F�1l<n��e���m�Hb��%`a��j�W�]���N}���������+#�+z�gHUG���cx���f��~L9��>�l\��2��Sj�)����m�0~����{��������Ji�� Q`C�13��oGf�#�D�)��&��et08<<�"M�� r��@��Rr�-�����2+J�4
E;C2���/�!��������
endstream
endobj

18 0 obj
9568
endobj

19 0 obj
<</Type/FontDescriptor/FontName/CAAAAA+LiberationSans-Bold
/Flags 4
/FontBBox[-184 -303 1061 1033]/ItalicAngle 0
/Ascent 905
/Descent -211
/CapHeight 1033
/StemV 80
/FontFile2 17 0 R
>>
endobj

20 0 obj
<</Length 367/Filter/FlateDecode>>
stream
x�]��n�0E�|���"�l$��� ��C�������_�m�.���;�q�~Y�k=,�����ZY����@\�6hO�B
��������|W���c��)�=�����]����+<x��U`}���q��n���xE!�n���<�#�T���[�u�J����X�J7)�M��m�
�<
�WU��V����K�}��Z�.IT8��G��Y"��Y���|��2��q ?��?p-���O�O|n�\�|�|f>!_x�3r���2�Z<Wn�x�d��"�?��H�O�.���2���&�?�<�����Ogm�T�����3��Dy������8d�(�m��V�[�y��[��O���4��I3���7���
endstream
endobj

21 0 obj
<</Type/Font/Subtype/TrueType/BaseFont/CAAAAA+LiberationSans-Bold
/FirstChar 0
/LastChar 32
/Widths[365 666 610 556 389 556 610 277 722 610 333 943 610 556 833 889
610 666 722 722 777 556 666 556 556 333 277 610 556 556 556 277
556 ]
/FontDescriptor 19 0 R
/ToUnicode 20 0 R
>>
endobj

22 0 obj
<</F1 11 0 R/F2 21 0 R/F3 16 0 R
>>
endobj

23 0 obj
<</Font 22 0 R
/XObject<</Tr4 4 0 R>>
/ExtGState<</EGS5 5 0 R>>
/ProcSet[/PDF/Text/ImageC/ImageI/ImageB]
>>
endobj

1 0 obj
<</Type/Page/Parent 6 0 R/Resources 23 0 R/MediaBox[0 0 612 792]/Group<</S/Transparency/CS/DeviceRGB/I true>>/Contents 2 0 R>>
endobj

24 0 obj
<</Count 1/First 25 0 R/Last 25 0 R
>>
endobj

25 0 obj
<</Count 0/Title<FEFF0053006800650065007400310031>
/Dest[1 0 R/XYZ 0 792 0]/Parent 24 0 R>>
endobj

6 0 obj
<</Type/Pages
/Resources 23 0 R
/MediaBox[ 0 0 612 792 ]
/Kids[ 1 0 R ]
/Count 1>>
endobj

26 0 obj
<</Type/Catalog/Pages 6 0 R
/OpenAction[1 0 R /Fit]
/Outlines 24 0 R
>>
endobj

27 0 obj
<</Creator<FEFF00430061006C0063>
/Producer<FEFF004C0069006200720065004F0066006600690063006500200034002E0033>
/CreationDate(D:20161006113121+05'30')>>
endobj

xref
0 28
0000000000 65535 f 
0000037525 00000 n 
0000000019 00000 n 
0000001883 00000 n 
0000001904 00000 n 
0000002100 00000 n 
0000037833 00000 n 
0000002140 00000 n 
0000018745 00000 n 
0000018767 00000 n 
0000018961 00000 n 
0000019495 00000 n 
0000019876 00000 n 
0000025952 00000 n 
0000025974 00000 n 
0000026182 00000 n 
0000026522 00000 n 
0000026735 00000 n 
0000036390 00000 n 
0000036412 00000 n 
0000036615 00000 n 
0000037052 00000 n 
0000037347 00000 n 
0000037400 00000 n 
0000037668 00000 n 
0000037724 00000 n 
0000037932 00000 n 
0000038021 00000 n 
trailer
<</Size 28/Root 26 0 R
/Info 27 0 R
/ID [ <0979E97CCC2C23A4D6D7CA822984A982>
<0979E97CCC2C23A4D6D7CA822984A982> ]
/DocChecksum /E3CCB573B5358A12900EDFC878B2AF86
>>
startxref
38188
%%EOF
TPCH_Plan.tar.gzapplication/x-gzip; name=TPCH_Plan.tar.gzDownload
#2Dilip Kumar
dilipbalaut@gmail.com
In reply to: Dilip Kumar (#1)
1 attachment(s)
Re: Parallel bitmap heap scan

There is major chance in tidbitmap.c file after efficient hash table
commit [1]commit-http://git.postgresql.org/pg/commitdiff/75ae538bc3168bf44475240d4e0487ee2f3bb376 and my patch need to be rebased.

Only parallel-bitmap-heap-scan need to be rebased, all other patch can
be applied on head as is.
Rebased version (v2) of parallel-bitmap-heap-scan is attached.

[1]: commit-http://git.postgresql.org/pg/commitdiff/75ae538bc3168bf44475240d4e0487ee2f3bb376

On Fri, Oct 7, 2016 at 11:46 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

Hi Hackers,

I would like to propose parallel bitmap heap scan feature. After
running TPCH benchmark, It was observed that many of TPCH queries are
using bitmap scan (@TPCH_plan.tar.gz attached below). Keeping this
point in mind we thought that many query will get benefited with
parallel bitmap scan.

Robert has also pointed out the same thing in his blog related to parallel query
http://rhaas.blogspot.in/2016/04/postgresql-96-with-parallel-query-vs.html

Currently Bitmap heap plan look like this :
------------------------------------------------------
Bitmap Heap Scan
-> Bitmap Index Scan

After this patch :
---------------------
Parallel Bitmap Heap Scan
-> Bitmap Index Scan

As part of this work I have implemented parallel processing in
BitmapHeapScan node. BitmapIndexScan is still non parallel.

Brief design idea:
-----------------------
#1. Shared TIDBitmap creation and initialization
First worker to see the state as parallel bitmap info as PBM_INITIAL
become leader and set the state to PBM_INPROGRESS All other workers
see the state as PBM_INPROGRESS will wait for leader to complete the
TIDBitmap.

#2 At this level TIDBitmap is ready and all workers are awake.

#3. Bitmap processing (Iterate and process the pages).
In this phase each worker will iterate over page and chunk array and
select heap pages one by one. If prefetch is enable then there will be
two iterator. Since multiple worker are iterating over same page and
chunk array we need to have a shared iterator, so we grab a spin lock
and iterate within a lock, so that each worker get and different page
to process.

Note: For more detail on design, please refer comment of
BitmapHeapNext API in "parallel-bitmap-heap-scan-v1.patch" file.

Attached patch details:
------------------------------
1. parallel-bitmap-heap-scan-v1.patch: This is the main patch to make
bitmap heap scan node parallel aware.

2. dht-return-dsa-v1.patch: This patch will provide new API, where we
can scan full DHT[1], and get the dsa_pointers (a relative pointer).
The dsa_pointer values can be shared with other processes. We need
this because, after TIDBitmap is created, only one worker will process
whole TIDBitmap and convert it to a page and chunk array. So we need
to store the generic pointer, so that later on each worker can convert
those to their local pointer before start processing.

My patch depends on following patches.
------------------------------------------------------
1. conditional_variable
/messages/by-id/CAEepm=0zshYwB6wDeJCkrRJeoBM=jPYBe+-k_VtKRU_8zMLEfA@mail.gmail.com

2. dsa_area
/messages/by-id/CAEepm=024p-MeAsDmG=R3+tR4EGhuGJs_+rjFKF0eRoSTmMJnA@mail.gmail.com

3. Creating a DSA area to provide work space for parallel execution
/messages/by-id/CAEepm=0HmRefi1+xDJ99Gj5APHr8Qr05KZtAxrMj8b+ay3o6sA@mail.gmail.com

4. Hash table in dynamic shared memory (DHT) [1]
/messages/by-id/CAEepm=0VrMt3s_REDhQv6z1pHL7FETOD7Rt9V2MQ3r-2ss2ccA@mail.gmail.com

Order in which patches should be applied:
--------------------------------------------------------
1. conditional_variable
2. dsa_area
3. Creating a DSA area to provide work space for parallel execution
4. Hash table in dynamic shared memory.
5. dht-return-dsa-v1.patch
6. parallel-bitmap-heap-scan-v1.patch

Performance Results:
-----------------------------
Summary :
1. After this patch, I observed currently 4 queries are getting
significant improvement (Q4, Q6, Q14, Q15).
- Q4, is converting from parallel seqscan to parallel bitmap heap scan.
- Other queries are converted from a regular bitmap heap scan to a
parallel bitmap heap scan.
2. Benefit is more visible at lower workers (upto 4), after that some
of the queries are selecting ParallelSeqScan over ParallelBitmapScan.
And, I think this is expected, because so far we have only made
BitmapHeap node as parallel whereas ParallelSeqScan is completely
parallel so at higher worker count ParallelSeqScan is better choice.
3. Detailed result is attached @TPCH_PBMS.pdf
4. Explain analyse output is attached @TPCH_plan.tar.gz (for all
changed queries at worker 2)

TPCH query plan changed example (TPCH Q6):
----------------------------------------------------------------
On Head:
-------------

QUERY PLAN
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
Limit (cost=1558475.95..1558475.96 rows=1 width=32) (actual
time=40921.437..40921.438 rows=1 loops=1)
-> Aggregate (cost=1558475.95..1558475.96 rows=1 width=32)
(actual time=40921.435..40921.435 rows=1 loops=1)
-> Bitmap Heap Scan on lineitem (cost=291783.32..1552956.39
rows=1103911 width=12) (actual time=7032.075..38997.369 rows=1140434
loops=1)
Recheck Cond: ((l_shipdate >= '1994-01-01'::date) AND
(l_shipdate < '1995-01-01 00:00:00'::timestamp without time zone) AND
(l_discount >= 0.01) AND (l_discount <= 0.03) AND (l_quantity <
'24'::numeric))
Rows Removed by Index Recheck: 25284232
Heap Blocks: exact=134904 lossy=530579
-> Bitmap Index Scan on idx_lineitem_shipdate
(cost=0.00..291507.35 rows=1103911 width=0) (actual
time=6951.408..6951.408 rows=1140434 loops=1)
Index Cond: ((l_shipdate >= '1994-01-01'::date)
AND (l_shipdate < '1995-01-01 00:00:00'::timestamp without time zone)
AND (l_discount >= 0.01) AND (l_discount <= 0.03) AND (l_quantity <
'24'::numeric))
Planning time: 1.126 ms
Execution time: 40922.569 ms
(10 rows)

After Patch:
----------------

QUERY PLAN
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
Limit (cost=1541767.60..1541767.61 rows=1 width=32) (actual
time=21895.008..21895.009 rows=1 loops=1)
-> Finalize Aggregate (cost=1541767.60..1541767.61 rows=1
width=32) (actual time=21895.006..21895.006 rows=1 loops=1)
-> Gather (cost=1541767.38..1541767.59 rows=2 width=32)
(actual time=21894.341..21894.970 rows=3 loops=1)
Workers Planned: 2
Workers Launched: 2
-> Partial Aggregate (cost=1540767.38..1540767.39
rows=1 width=32) (actual time=21890.990..21890.990 rows=1 loops=3)
-> Parallel Bitmap Heap Scan on lineitem
(cost=291783.32..1538467.56 rows=459963 width=12) (actual
time=8517.126..21215.469 rows=380145 loops=3)
Recheck Cond: ((l_shipdate >=
'1994-01-01'::date) AND (l_shipdate < '1995-01-01 00:00:00'::timestamp
without time zone) AND (l_discount >= 0.01) AND (l_discount <= 0.03)
AND (l_quantity < '24'::numeric))
Rows Removed by Index Recheck: 8427921
Heap Blocks: exact=47761 lossy=187096
-> Bitmap Index Scan on
idx_lineitem_shipdate (cost=0.00..291507.35 rows=1103911 width=0)
(actual time=8307.291..8307.291 rows=1140434 loops=1)
Index Cond: ((l_shipdate >=
'1994-01-01'::date) AND (l_shipdate < '1995-01-01 00:00:00'::timestamp
without time zone) AND (l_discount >= 0.01) AND (l_discount <= 0.03)
AND (l_quantity < '24'::numeric))
Planning time: 1.173 ms
Execution time: 21915.931 ms
(14 rows)

* Thanks to Robert Haas and Amit Kapila, for helping in design review
(off list) and many valuable inputs.
* Thanks to Thomas Munro for DSA and DHT work on which my patch is based on.
* Thanks to Rafia sabih for helping with performance test.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

parallel-bitmap-heap-scan-v2.patchapplication/octet-stream; name=parallel-bitmap-heap-scan-v2.patchDownload
diff --git a/src/backend/access/gin/ginget.c b/src/backend/access/gin/ginget.c
index 9ed9fd2..c8034d0 100644
--- a/src/backend/access/gin/ginget.c
+++ b/src/backend/access/gin/ginget.c
@@ -362,7 +362,7 @@ restartScanEntry:
 
 		if (entry->matchBitmap && !tbm_is_empty(entry->matchBitmap))
 		{
-			entry->matchIterator = tbm_begin_iterate(entry->matchBitmap);
+			entry->matchIterator = tbm_begin_iterate(entry->matchBitmap, NULL);
 			entry->isFinished = FALSE;
 		}
 	}
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index b019bc1..1dfd492 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_bm_update_snapshot
+ *
+ *		Update snpashot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_bm_update_snapshot(HeapScanDesc scan, Snapshot	snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 72bacd5..1e34f26 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -203,6 +204,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+										e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -255,6 +260,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+										d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -724,6 +734,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									(BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index 449aacb..c3ad77b 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -47,16 +47,63 @@
 #include "utils/spccache.h"
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
-
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
-
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static void pbms_set_parallel(PlanState *node);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+									ParallelIterator *parallel_iterator,
+									bool is_parallel);
+static void prefetch_pages(int *prefetch_pages, int prefetch_target,
+						BitmapHeapScanState *node, HeapScanDesc scan);
+static void update_prefetch_target(int *prefetch_target, int prefetch_maximum);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
  *
  *		Retrieve next tuple from the BitmapHeapScan node's currentRelation
+ *
+ *
+ *	[PARALLEL BITMAP HEAP SCAN ALGORITHM]
+ *
+ *	#1. Shared TIDBitmap creation and initialization
+ *		a) First worker to see the state as parallel bitmap info as
+ *		PBM_INITIAL become leader and set the state to PBM_INPROGRESS
+ *		All other workers see the state as PBM_INPROGRESS will wait for
+ *		leader to complete the TIDBitmap.
+ *
+ *		Leader Worker Processing:
+ *		(Leader is responsible for creating shared TIDBitmap and create
+ *		shared page and chunk array from TIDBitmap.)
+ *			1) Create TIDBitmap using DHT.
+ *			2) Begin Iterate: convert hash table into shared page and chunk
+ *			array.
+ *			3) Restore local TIDBitmap variable information into
+ *			ParallelBitmapInfo so that other worker can see those.
+ *			4) set state to PBM_FINISHED.
+ *			5) Wake up other workers.
+ *
+ *		Other Worker Processing:
+ *			1) Wait until leader create shared TIDBitmap and shared page
+ *			and chunk array.
+ *			2) Attach to shared page table, copy TIDBitmap from
+ *			ParallelBitmapInfo to local TIDBitmap, we copy this to local
+ *			TIDBitmap so that next level processing can read information
+ *			same as in non parallel case and we can avoid extra changes
+ *			in code.
+ *
+ *	# At this level TIDBitmap is ready and all workers are awake #
+ *
+ *	#2. Bitmap processing (Iterate and process the pages).
+ *		. In this phase each worker will iterate over page and chunk array and
+ *		select heap pages one by one. If prefetch is enable then there will
+ *		be two iterator.
+ *		. Since multiple worker are iterating over same page and chunk array
+ *		we need to have a shared iterator, so we grab a spin lock and iterate
+ *		within a lock.
  * ----------------------------------------------------------------
  */
 static TupleTableSlot *
@@ -67,12 +114,19 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TIDBitmap  *tbm;
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
-
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+	bool		is_parallel = node->parallel_bitmap ? true : false;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (is_parallel)
+		parallel_tbm = (ParallelTIDBitmap*)((char *)pbminfo +
+					pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -101,36 +155,106 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running
+		 * in non parallel mode or we are leader worker.
+		 *
+		 * In parallel mode leader worker will immediately come out
+		 * of the function, but all other worker will be blocked
+		 * until leader worker wake them up.
+		 */
+		if (!is_parallel || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer
+			 * node and set parallel flag in lower level bitmap index scan.
+			 * Later bitmap index node will use this flag to indicate
+			 * tidbitmap that it needs to create an shared page table.
+			 */
+			if (is_parallel)
+			{
+				node->is_leader = true;
+				pbms_set_parallel(outerPlanState(node));
+			}
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM.
+			 * Here we need to create a local TBM and copy information from
+			 * shared location. We also need to attach to shared page table
+			 * using hash table handle stored in parallel_tbm(shared memory).
+			 */
+			tbm = tbm_create(work_mem * 1024L);
+			tbm_set_parallel(tbm, node->ss.ps.state->es_query_area);
+			tbm_attach_to_pagetable(tbm, parallel_tbm);
+		}
 
 		if (!tbm || !IsA(tbm, TIDBitmap))
 			elog(ERROR, "unrecognized result from subplan");
 
 		node->tbm = tbm;
-		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+		node->tbmiterator = tbmiterator
+							= tbm_begin_iterate(tbm, parallel_tbm);
 		node->tbmres = tbmres = NULL;
 
 #ifdef USE_PREFETCH
 		if (node->prefetch_maximum > 0)
 		{
-			node->prefetch_iterator = prefetch_iterator = tbm_begin_iterate(tbm);
-			node->prefetch_pages = 0;
-			node->prefetch_target = -1;
+			node->prefetch_iterator = prefetch_iterator =
+							tbm_begin_iterate(tbm, parallel_tbm);
+
+			/* These variable are used only in case of non parallel mode */
+			if (!is_parallel)
+			{
+				node->prefetch_pages = 0;
+				node->prefetch_target = -1;
+			}
 		}
 #endif   /* USE_PREFETCH */
+
+		/*
+		 * If we are in parallel mode and we are leader worker then
+		 * copy the local TBM information to shared location, and wake
+		 * up other workers.
+		 */
+		if (node->is_leader)
+		{
+			/*
+			 * copy TBM local information in shared memory before waking
+			 * up other workers. Other workers will create there own
+			 * TBM and copy information from shared memory, they will
+			 * also use hash table handle from shared memory for attaching
+			 * to shared memory hash table.
+			 */
+			tbm_update_shared_members(tbm, parallel_tbm);
+
+			/* Change the state under a lock */
+			SpinLockAcquire(&pbminfo->state_mutex);
+			pbminfo->state = PBM_FINISHED;
+			SpinLockRelease(&pbminfo->state_mutex);
+
+			/* Wake up all other workers. */
+			ConditionVariableBroadcast(&pbminfo->cv);
+		}
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+													&pbminfo->tbmiterator,
+													is_parallel);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +262,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (is_parallel)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages --;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+											&pbminfo->prefetch_iterator,
+											is_parallel);
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not
+				 * ensure that current blockno in main iterator and prefetch
+				 * iterator is same. It's possible that whatever blockno we
+				 * are prefetching is getting processed by some other worker.
+				 */
+				if (!is_parallel &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +333,25 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				if (!is_parallel)
+					update_prefetch_target(&node->prefetch_target,
+										 node->prefetch_maximum);
+				else
+				{
+					/*
+					 * If we are in parallel mode then grab prefetch_mutex
+					 * before updating prefetch target.
+					 */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+					update_prefetch_target(&pbminfo->prefetch_target,
+										 node->prefetch_maximum);
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+				}
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +367,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (!is_parallel)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex
+				 * before updating prefetch target.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+					pbminfo->prefetch_target++;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +406,29 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			if (pbminfo == NULL)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				prefetch_pages(&node->prefetch_pages,
+							node->prefetch_target, node, scan);
+			}
+			else if(node->prefetch_pages < node->prefetch_target)
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex
+				 * before going for prefetch.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
 
-				if (tbmpre == NULL)
-				{
-					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
-					break;
-				}
-				node->prefetch_pages++;
-				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+				prefetch_pages(&pbminfo->prefetch_pages,
+							pbminfo->prefetch_target, node, scan);
+
+				SpinLockRelease(&pbminfo->prefetch_mutex);
 			}
+
+			/* Restore the prefetch_iterator */
+			prefetch_iterator = node->prefetch_iterator;
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +643,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +761,8 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->is_leader = false;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -653,3 +849,311 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool needWait = false;
+	bool queuedSelf = false;
+	bool leader = false;
+
+	for(;;)
+	{
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		/*
+		 * if state is initial then we are the first one to come here
+		 * set the state to in progress and mark ourself as leader
+		 */
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+
+		/* bitmap create is in progress so we need to wait */
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/*
+		 * If we are a leader or else leader has already created a
+		 * tid bitmap.
+		 */
+		if (leader || !needWait)
+			break;
+
+		/* We need to queue */
+		if (queuedSelf)
+		{
+			/* Sleep until leader send wake up signal */
+			ConditionVariableSleep(WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+			queuedSelf = false;
+			needWait = false;
+		}
+		else if (needWait)
+		{
+			/* Add ourself to wait queue */
+			ConditionVariablePrepareToSleep(&pbminfo->cv);
+			queuedSelf = true;
+		}
+	}
+
+	/* Cancel the sleep before return */
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node
+	 * as parallel, because only first node will create the main bitmap
+	 * other bitmaps will be merged to the first bitmap so no need to
+	 * create them in shared memory.
+	 */
+	switch(node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState*)node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState*)node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState*)node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/*
+ * prefetch_pages
+ *
+ * Prefetch pages before going for actual processing of the page.
+ */
+static void
+prefetch_pages(int *prefetch_pages, int prefetch_target,
+			BitmapHeapScanState *node, HeapScanDesc scan)
+{
+	TBMIterator *iterator = node->prefetch_iterator;
+	ParallelIterator *parallel_iteartor;
+
+	/*
+	 * If parallel bitmap info available means we are running
+	 * in parallel mode. So use parallel iterator for prefetching.
+	 */
+	if (node->parallel_bitmap)
+		parallel_iteartor = &node->parallel_bitmap->prefetch_iterator;
+
+	while (*prefetch_pages < prefetch_target)
+	{
+		TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+										parallel_iteartor,
+										node->parallel_bitmap ? true:false);
+		if (tbmpre == NULL)
+		{
+			/* No more pages to prefetch */
+			tbm_end_iterate(iterator);
+			node->prefetch_iterator = NULL;
+			break;
+		}
+
+		(*prefetch_pages)++;
+		PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+	}
+}
+
+/*
+ * update_prefetch_target
+ *
+ * Update the value of prefetch target
+ */
+static void
+update_prefetch_target(int *prefetch_target, int prefetch_maximum)
+{
+	/*
+	 * Increase prefetch target if it's not yet at the max.  Note that
+	 * we will increase it to zero after fetching the very first
+	 * page/tuple, then to one after the second tuple is fetched, then
+	 * it doubles as later pages are fetched.
+	 */
+	if (*prefetch_target >= prefetch_maximum)
+		 /* don't increase any further */ ;
+	else if (*prefetch_target >= prefetch_maximum / 2)
+		*prefetch_target = prefetch_maximum;
+	else if (*prefetch_target > 0)
+		*prefetch_target *= 2;
+	else
+		(*prefetch_target)++;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Acquire a iterator lock.
+ * copy iterator state from shared iterator to local iterator.
+ * Call tbm_iterate and restore the state back to shared iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator,
+				ParallelIterator *parallel_iterator,
+				bool is_parallel)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (!is_parallel)
+		return tbm_iterate(iterator);
+
+	/*
+	 * We are in parallel mode so grab parallel iterator mutex
+	 * before calling iterator.
+	 */
+	SpinLockAcquire(&parallel_iterator->mutex);
+
+	/*
+	 * Now we have got lock on iterator so copy information from
+	 * shared location to our local iterator.
+	 */
+	iterator->spageptr = parallel_iterator->spageptr;
+	iterator->schunkptr = parallel_iterator->schunkptr;
+	iterator->schunkbit = parallel_iterator->schunkbit;
+
+	output = tbm_iterate(iterator);
+
+	/*
+	 * tbm_iterate would have changed the iterator value
+	 * in local iterator so copy them back to shared location
+	 * before releasing the lock.
+	 */
+	parallel_iterator->spageptr = iterator->spageptr;
+	parallel_iterator->schunkptr = iterator->schunkptr;
+	parallel_iterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&parallel_iterator->mutex);
+
+	return output;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+							phs_snapshot_data),
+							EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+						 ParallelContext *pcxt)
+{
+	ParallelBitmapInfo	*pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Snapshot	snapshot;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+								phs_snapshot_data),
+								EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+
+	heap_bm_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo	*pbminfo;
+	Snapshot			snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_bm_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index a364098..9cc5088 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate
+	 * that we need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_area);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 826fead..7d3ebb2 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -43,6 +43,11 @@
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
 #include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
+#include "storage/lwlock.h"
+#include "storage/shmem.h"
+#include "storage/dht.h"
+#include "storage/dsa.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -79,6 +84,8 @@
 /* number of active words for a lossy chunk: */
 #define WORDS_PER_CHUNK  ((PAGES_PER_CHUNK - 1) / BITS_PER_BITMAPWORD + 1)
 
+#define TBM_IS_SHARED(tbm) (tbm)->shared
+
 /*
  * The hashtable entries are represented by this data structure.  For
  * an exact page, blockno is the page number and bit k of the bitmap
@@ -139,24 +146,28 @@ struct TIDBitmap
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	bool			shared;		/* need to build shared tbm if set*/
+	dht_hash_table_handle	hash_handle;	/* shared hash table handle */
+	dht_hash_table	*shared_pagetable;	/* dynamic hash table */
+	dsa_area		*area;		/* reference to per-query shared memory area */
 };
 
 /*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
  */
-struct TBMIterator
+struct ParallelTIDBitmap
 {
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+	dht_hash_table_handle	hash_handle;	/* shared hash table handle */
+	dsa_pointer		dsa_pages;		/* dsa pointers for all kind of pages */
+	int				nentries;		/* number of entries in pagetable */
+	int				maxentries;		/* limit on same to meet maxbytes */
+	int				npages;			/* number of exact entries in pagetable */
+	int				nchunks;		/* number of lossy entries in pagetable */
+	int				item_count;		/* total item in dsa_pages */
+	bool			inited;			/* set true after leader converts page */
+									/* table to dsa_pointer's array. */
 };
 
-
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
 static bool tbm_intersect_page(TIDBitmap *a, PagetableEntry *apage,
@@ -168,6 +179,7 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+static void* tbm_pagetable_iterate(TIDBitmap *tbm, void *iterator);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -266,6 +278,33 @@ tbm_create_pagetable(TIDBitmap *tbm)
 }
 
 /*
+ * tbm_create_shared_pagetable
+ *
+ * Creates shared hash table using DHT for parallel bitmap scan.
+ */
+static void
+tbm_create_shared_pagetable(TIDBitmap *tbm)
+{
+	dht_parameters params = {0};
+
+	params.key_size = sizeof(BlockNumber);
+	params.entry_size = sizeof(PagetableEntry);
+	params.compare_function = memcmp;
+	params.hash_function = tag_hash;
+
+	params.tranche_id = LWLockNewTrancheId();
+
+	/* Create a dynamic hash table */
+	tbm->shared_pagetable = dht_create(tbm->area, &params);
+	if (tbm->shared_pagetable == NULL)
+		elog(ERROR, "could not create hash table");
+
+	/* Get the handle so that other backend can attach using this handle */
+	tbm->hash_handle = dht_get_hash_table_handle(tbm->shared_pagetable);
+	tbm->status = TBM_HASH;
+}
+
+/*
  * tbm_free - free a TIDBitmap
  */
 void
@@ -277,6 +316,11 @@ tbm_free(TIDBitmap *tbm)
 		pfree(tbm->spages);
 	if (tbm->schunks)
 		pfree(tbm->schunks);
+
+	/* If we have shared page table then detach from it */
+	if (tbm->shared_pagetable)
+		dht_detach(tbm->shared_pagetable);
+
 	pfree(tbm);
 }
 
@@ -453,6 +497,14 @@ void
 tbm_intersect(TIDBitmap *a, const TIDBitmap *b)
 {
 	Assert(!a->iterating);
+
+	/*
+	 * In case of BitmapAnd, only first node will create shared TBM
+	 * and all other node will have local TBM which will be merged
+	 * with first shared TBM.
+	 */
+	Assert(!TBM_IS_SHARED(b));
+
 	/* Nothing to do if a is empty */
 	if (a->nentries == 0)
 		return;
@@ -473,10 +525,27 @@ tbm_intersect(TIDBitmap *a, const TIDBitmap *b)
 	{
 		pagetable_iterator i;
 		PagetableEntry *apage;
+		void		   *iterator;
+		dht_iterator 	dhtiterator;
 
 		Assert(a->status == TBM_HASH);
 		pagetable_start_iterate(a->pagetable, &i);
-		while ((apage = pagetable_iterate(a->pagetable, &i)) != NULL)
+		/*
+		 * If we are using shared page table then use DHT iterator.
+		 */
+		if (TBM_IS_SHARED(a))
+		{
+			iterator = &dhtiterator;
+			dht_iterate_begin(a->shared_pagetable, &dhtiterator, false);
+		}
+		else
+		{
+			iterator = &i;
+			pagetable_start_iterate(a->pagetable, &i);
+		}
+
+		while ((apage =
+				(PagetableEntry *)tbm_pagetable_iterate(a, iterator)) != NULL)
 		{
 			if (tbm_intersect_page(a, apage, b))
 			{
@@ -486,8 +555,23 @@ tbm_intersect(TIDBitmap *a, const TIDBitmap *b)
 				else
 					a->npages--;
 				a->nentries--;
-				if (!pagetable_delete(a->pagetable, apage->blockno))
-					elog(ERROR, "hash table corrupted");
+
+				/*
+				 * If we are using shared hash then we need to release the
+				 * lock on DHT element before deleting it.
+				 */
+				if (TBM_IS_SHARED(a))
+				{
+					dht_iterate_release(&dhtiterator);
+					if (dht_delete_key(a->shared_pagetable,
+									(void *)&(apage->blockno)))
+						elog(ERROR, "hash table corrupted");
+				}
+				else
+				{
+					if (!pagetable_delete(a->pagetable, apage->blockno))
+						elog(ERROR, "hash table corrupted");
+				}
 			}
 		}
 	}
@@ -599,7 +683,7 @@ tbm_is_empty(const TIDBitmap *tbm)
  * contents repeatedly, including parallel scans.
  */
 TBMIterator *
-tbm_begin_iterate(TIDBitmap *tbm)
+tbm_begin_iterate(TIDBitmap *tbm, ParallelTIDBitmap *parallel_info)
 {
 	TBMIterator *iterator;
 
@@ -612,11 +696,16 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	iterator->tbm = tbm;
 
 	/*
-	 * Initialize iteration pointers.
+	 * Initialize iteration pointers, only if it's not shared tbm.
+	 * In case of shared tbm, we will copy these values from
+	 * shared iterator before calling tbm_iterate.
 	 */
-	iterator->spageptr = 0;
-	iterator->schunkptr = 0;
-	iterator->schunkbit = 0;
+	if (!TBM_IS_SHARED(tbm))
+	{
+		iterator->spageptr = 0;
+		iterator->schunkptr = 0;
+		iterator->schunkbit = 0;
+	}
 
 	/*
 	 * If we have a hashtable, create and fill the sorted page lists, unless
@@ -640,15 +729,79 @@ tbm_begin_iterate(TIDBitmap *tbm)
 				MemoryContextAlloc(tbm->mcxt,
 								   tbm->nchunks * sizeof(PagetableEntry *));
 
-		npages = nchunks = 0;
-		pagetable_start_iterate(tbm->pagetable, &i);
-		while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+		/*
+		 * If we have shared TBM means we are running in parallel mode.
+		 * So iterate over DHT and construct page and chunk array.
+		 *
+		 * First leader worker will create array of dsa pointers which
+		 * will holds dsa pointers for both pages and chunks, later
+		 * while converting to local pointers we will identify them
+		 * and copy in their respective array.
+		 */
+		if (TBM_IS_SHARED(tbm))
 		{
-			if (page->ischunk)
-				tbm->schunks[nchunks++] = page;
-			else
-				tbm->spages[npages++] = page;
+			dsa_pointer *dsa_spages;
+			dht_hash_table_item *item;
+			int			ncount = 0;
+			int			i;
+
+			/*
+			 * Iterate over DHT and create array of dsa_pointers.
+			 * Only leader will perform this step after that inited
+			 * flag will be set.
+			 */
+			if (!(parallel_info->inited) &&
+				(tbm->npages > 0 || tbm->nchunks > 0))
+			{
+				dht_iterator dhtiterator;
+				dsa_pointer	 dsa_page;
+
+				parallel_info->dsa_pages = dsa_allocate(tbm->area,
+						(tbm->nchunks + tbm->npages) * sizeof(dsa_pointer));
+
+				dsa_spages =
+						dsa_get_address(tbm->area, parallel_info->dsa_pages);
+
+				dht_iterate_begin(tbm->shared_pagetable, &dhtiterator, false);
+				while ((dsa_page = dht_iterate_next_dsa(&dhtiterator)) !=
+						InvalidDsaPointer)
+					dsa_spages[ncount++] = dsa_page;
+
+				parallel_info->inited = true;
+				parallel_info->item_count = ncount;
+			}
+
+			/*
+			 * This step will be done by all the workers including leader.
+			 * Here we need to convert array of dsa pointers to local
+			 * page and chunk array.
+			 */
+			dsa_spages = dsa_get_address(tbm->area, parallel_info->dsa_pages);
+			npages = nchunks = 0;
+			for (i = 0; i < parallel_info->item_count; i++)
+			{
+				item = dsa_get_address(tbm->area, dsa_spages[i]);
+				page = (PagetableEntry*)&(item->entry);
+
+				if (page->ischunk)
+					tbm->schunks[nchunks++] = page;
+				else
+					tbm->spages[npages++] = page;
+			}
 		}
+		else
+		{
+			npages = nchunks = 0;
+			pagetable_start_iterate(tbm->pagetable, &i);
+			while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+			{
+				if (page->ischunk)
+					tbm->schunks[nchunks++] = page;
+				else
+					tbm->spages[npages++] = page;
+			}
+		}
+
 		Assert(npages == tbm->npages);
 		Assert(nchunks == tbm->nchunks);
 		if (npages > 1)
@@ -811,11 +964,23 @@ tbm_find_pageentry(const TIDBitmap *tbm, BlockNumber pageno)
 		return page;
 	}
 
-	page = pagetable_lookup(tbm->pagetable, pageno);
+	/* If TBM has shared page table then find in DHT */
+	if (TBM_IS_SHARED(tbm))
+		page = (PagetableEntry *) dht_find(tbm->shared_pagetable,
+										  (void *) &pageno,
+										  false);
+	else
+		page = pagetable_lookup(tbm->pagetable, pageno);
+
 	if (page == NULL)
 		return NULL;
 	if (page->ischunk)
 		return NULL;			/* don't want a lossy chunk header */
+
+	/* If entry is from shared page table then release the entry. */
+	if (TBM_IS_SHARED(tbm))
+		dht_release(tbm->shared_pagetable, (void*)page);
+
 	return page;
 }
 
@@ -833,7 +998,11 @@ tbm_get_pageentry(TIDBitmap *tbm, BlockNumber pageno)
 	PagetableEntry *page;
 	bool		found;
 
-	if (tbm->status == TBM_EMPTY)
+	/*
+	 * Use fixed slot only if it's local tidbitmap, If shared bitmap
+	 * then directly insert into shared hash table.
+	 */
+	if ((tbm->status == TBM_EMPTY) && !TBM_IS_SHARED(tbm))
 	{
 		/* Use the fixed slot */
 		page = &tbm->entry1;
@@ -842,6 +1011,7 @@ tbm_get_pageentry(TIDBitmap *tbm, BlockNumber pageno)
 	}
 	else
 	{
+		/* In case of shared TBM, status will never be TBM_ONE_PAGE */
 		if (tbm->status == TBM_ONE_PAGE)
 		{
 			page = &tbm->entry1;
@@ -851,8 +1021,23 @@ tbm_get_pageentry(TIDBitmap *tbm, BlockNumber pageno)
 			tbm_create_pagetable(tbm);
 		}
 
-		/* Look up or create an entry */
-		page = pagetable_insert(tbm->pagetable, pageno, &found);
+		/*
+		 * In case of parallel bitmap scan, we don't switch from TBM_EMPTY
+		 * to TBM_ONE_PAGE. So even if we are here we may not have hash
+		 * table ready, so if tbm->status is not yet TBM_HASH then create
+		 * shared page table.
+		 */
+		if (TBM_IS_SHARED(tbm))
+		{
+			if (tbm->status != TBM_HASH)
+				tbm_create_shared_pagetable(tbm);
+
+			page = (PagetableEntry *) dht_find_or_insert(tbm->shared_pagetable,
+											  (void *) &pageno, &found);
+		}
+		else
+			/* Look up or create an entry */
+			page = pagetable_insert(tbm->pagetable, pageno, &found);
 	}
 
 	/* Initialize it if not present before */
@@ -868,6 +1053,10 @@ tbm_get_pageentry(TIDBitmap *tbm, BlockNumber pageno)
 		tbm->npages++;
 	}
 
+	/* Release the entry lock if it's from shared hash table */
+	if (page && TBM_IS_SHARED(tbm))
+		dht_release(tbm->shared_pagetable, (void*)page);
+
 	return page;
 }
 
@@ -889,7 +1078,20 @@ tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno)
 	bitno = pageno % PAGES_PER_CHUNK;
 	chunk_pageno = pageno - bitno;
 
-	page = pagetable_lookup(tbm->pagetable, chunk_pageno);
+	/* If tbm has shared page table then use DHT to find the entry. */
+	if (TBM_IS_SHARED(tbm))
+		page = (PagetableEntry *) dht_find(tbm->shared_pagetable,
+										(void *) &pageno, false);
+	else
+		page = pagetable_lookup(tbm->pagetable, chunk_pageno);
+
+	/*
+	 * If entry is from shared page table then release the entry.
+	 * Currently only one worker is building the bitmap so it's
+	 * fine to release it here.
+	 */
+	if (TBM_IS_SHARED(tbm) && page != NULL)
+		dht_release(tbm->shared_pagetable, (void*)page);
 
 	if (page != NULL && page->ischunk)
 	{
@@ -920,7 +1122,13 @@ tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno)
 
 	/* We force the bitmap into hashtable mode whenever it's lossy */
 	if (tbm->status != TBM_HASH)
-		tbm_create_pagetable(tbm);
+	{
+		/* Create shared page table if we are in parallel mode. */
+		if (TBM_IS_SHARED(tbm))
+			tbm_create_shared_pagetable(tbm);
+		else
+			tbm_create_pagetable(tbm);
+	}
 
 	bitno = pageno % PAGES_PER_CHUNK;
 	chunk_pageno = pageno - bitno;
@@ -931,7 +1139,15 @@ tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno)
 	 */
 	if (bitno != 0)
 	{
-		if (pagetable_delete(tbm->pagetable, pageno))
+		bool result = false;
+
+		/* Look up or create an entry */
+		if (TBM_IS_SHARED(tbm))
+			result = dht_delete_key(tbm->shared_pagetable, (void *) &pageno);
+		else
+			result = pagetable_delete(tbm->pagetable, pageno);
+
+		if (result)
 		{
 			/* It was present, so adjust counts */
 			tbm->nentries--;
@@ -940,7 +1156,11 @@ tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno)
 	}
 
 	/* Look up or create entry for chunk-header page */
-	page = pagetable_insert(tbm->pagetable, chunk_pageno, &found);
+	if (TBM_IS_SHARED(tbm))
+		page = (PagetableEntry *) dht_find_or_insert(tbm->shared_pagetable,
+										  (void *) &chunk_pageno, &found);
+	else
+		page = pagetable_insert(tbm->pagetable, chunk_pageno, &found);
 
 	/* Initialize it if not present before */
 	if (!found)
@@ -975,6 +1195,10 @@ tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno)
 	wordnum = WORDNUM(bitno);
 	bitnum = BITNUM(bitno);
 	page->words[wordnum] |= ((bitmapword) 1 << bitnum);
+
+	/* Unlock an entry which was locked by dht_find_or_insert. */
+	if (TBM_IS_SHARED(tbm))
+		dht_release(tbm->shared_pagetable, (void*)page);
 }
 
 /*
@@ -985,7 +1209,8 @@ tbm_lossify(TIDBitmap *tbm)
 {
 	pagetable_iterator i;
 	PagetableEntry *page;
-
+	dht_iterator	dhtiterator;
+	void		   *iterator;
 	/*
 	 * XXX Really stupid implementation: this just lossifies pages in
 	 * essentially random order.  We should be paying some attention to the
@@ -998,8 +1223,24 @@ tbm_lossify(TIDBitmap *tbm)
 	Assert(!tbm->iterating);
 	Assert(tbm->status == TBM_HASH);
 
-	pagetable_start_iterate_at(tbm->pagetable, &i, tbm->lossify_start);
-	while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+	/*
+	 * If we are using shared page table then use DHT iterator otherwise
+	 * dyna hash iterator
+	 */
+	if (TBM_IS_SHARED(tbm))
+	{
+		iterator = &dhtiterator;
+		dht_iterate_begin(tbm->shared_pagetable, &dhtiterator, false);
+	}
+	else
+	{
+		iterator = &i;
+		pagetable_start_iterate_at(tbm->pagetable, &i, tbm->lossify_start);
+	}
+
+
+	while ((page = (PagetableEntry *)
+				tbm_pagetable_iterate(tbm, iterator)) != NULL)
 	{
 		if (page->ischunk)
 			continue;			/* already a chunk header */
@@ -1011,11 +1252,19 @@ tbm_lossify(TIDBitmap *tbm)
 		if ((page->blockno % PAGES_PER_CHUNK) == 0)
 			continue;
 
+		/* If we are using shared hash then release the lock on element. */
+		if (TBM_IS_SHARED(tbm))
+			dht_iterate_release(&dhtiterator);
+
 		/* This does the dirty work ... */
 		tbm_mark_page_lossy(tbm, page->blockno);
 
 		if (tbm->nentries <= tbm->maxentries / 2)
 		{
+			/* we have done enough */
+			if (TBM_IS_SHARED(tbm))
+				dht_iterate_end(&dhtiterator);
+
 			/*
 			 * We have made enough room. Remember where to start lossifying
 			 * next round, so we evenly iterate over the hashtable.
@@ -1061,3 +1310,96 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_attach_to_pagetable
+ *
+ * Attach worker to shared TID bitmap created by leader worker.
+ */
+void
+tbm_attach_to_pagetable(TIDBitmap *tbm, ParallelTIDBitmap *stbm)
+{
+	dht_parameters params = {0};
+
+	params.key_size = sizeof(BlockNumber);
+	params.entry_size = sizeof(PagetableEntry);
+	params.compare_function = memcmp;
+	params.hash_function = tag_hash;
+
+	tbm->shared_pagetable = dht_attach(tbm->area, &params, stbm->hash_handle);
+
+	tbm->status = TBM_HASH;
+	tbm->nchunks = stbm->nchunks;
+	tbm->nentries = stbm->nentries;
+	tbm->npages = stbm->npages;
+	tbm->maxentries = stbm->maxentries;
+	tbm->shared = true;
+}
+
+/*
+ * tbm_update_shared_members
+ *
+ * Restore leaders private tbm state to shared location. This must
+ * be called before waking up the other worker.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before
+	 * waking up the other workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->hash_handle = tbm->hash_handle;
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->shared = true;
+	tbm->area = (dsa_area*)area;
+}
+
+/*
+ * tbm_pagetable_iterate
+ *
+ * Unified function to scan full tbm page table
+ * if tbm is shared then it will operate on shared hash
+ * table otherwise on local hash table.
+ */
+static void*
+tbm_pagetable_iterate(TIDBitmap *tbm, void *iterator)
+{
+	if (TBM_IS_SHARED(tbm))
+	{
+		dht_iterator *dhtiterator = (dht_iterator*)iterator;
+
+		return dht_iterate_next(dhtiterator);
+	}
+	else
+	{
+		pagetable_iterator *i = (pagetable_iterator*)iterator;
+
+		return pagetable_iterate(tbm->pagetable, i);
+	}
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index e42ef98..058e55a 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -126,6 +126,7 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static int compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 
 
 /*
@@ -678,49 +679,7 @@ create_plain_partial_paths(PlannerInfo *root, RelOptInfo *rel)
 {
 	int			parallel_workers;
 
-	/*
-	 * If the user has set the parallel_workers reloption, use that; otherwise
-	 * select a default number of workers.
-	 */
-	if (rel->rel_parallel_workers != -1)
-		parallel_workers = rel->rel_parallel_workers;
-	else
-	{
-		int			parallel_threshold;
-
-		/*
-		 * If this relation is too small to be worth a parallel scan, just
-		 * return without doing anything ... unless it's an inheritance child.
-		 * In that case, we want to generate a parallel path here anyway.  It
-		 * might not be worthwhile just for this relation, but when combined
-		 * with all of its inheritance siblings it may well pay off.
-		 */
-		if (rel->pages < (BlockNumber) min_parallel_relation_size &&
-			rel->reloptkind == RELOPT_BASEREL)
-			return;
-
-		/*
-		 * Select the number of workers based on the log of the size of the
-		 * relation.  This probably needs to be a good deal more
-		 * sophisticated, but we need something here for now.  Note that the
-		 * upper limit of the min_parallel_relation_size GUC is chosen to
-		 * prevent overflow here.
-		 */
-		parallel_workers = 1;
-		parallel_threshold = Max(min_parallel_relation_size, 1);
-		while (rel->pages >= (BlockNumber) (parallel_threshold * 3))
-		{
-			parallel_workers++;
-			parallel_threshold *= 3;
-			if (parallel_threshold > INT_MAX / 3)
-				break;			/* avoid overflow */
-		}
-	}
-
-	/*
-	 * In no case use more than max_parallel_workers_per_gather workers.
-	 */
-	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+	parallel_workers = compute_parallel_worker(rel, rel->pages);
 
 	/* If any limit was set to zero, the user doesn't want a parallel scan. */
 	if (parallel_workers <= 0)
@@ -2866,6 +2825,84 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int				parallel_workers;
+	double			pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path*)create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+static int
+compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
+{
+	int			parallel_workers;
+
+	/*
+	 * If the user has set the parallel_workers reloption, use that; otherwise
+	 * select a default number of workers.
+	 */
+	if (rel->rel_parallel_workers != -1)
+		parallel_workers = rel->rel_parallel_workers;
+	else
+	{
+		int			parallel_threshold;
+
+		/*
+		 * If this relation is too small to be worth a parallel scan, just
+		 * return without doing anything ... unless it's an inheritance child.
+		 * In that case, we want to generate a parallel path here anyway.  It
+		 * might not be worthwhile just for this relation, but when combined
+		 * with all of its inheritance siblings it may well pay off.
+		 */
+		if (pages < (BlockNumber) min_parallel_relation_size &&
+			rel->reloptkind == RELOPT_BASEREL)
+			return 0;
+
+		/*
+		 * Select the number of workers based on the log of the size of the
+		 * relation.  This probably needs to be a good deal more
+		 * sophisticated, but we need something here for now.  Note that the
+		 * upper limit of the min_parallel_relation_size GUC is chosen to
+		 * prevent overflow here.
+		 */
+		parallel_workers = 1;
+		parallel_threshold = Max(min_parallel_relation_size, 1);
+		while (pages >= (BlockNumber) (parallel_threshold * 3))
+		{
+			parallel_workers++;
+			parallel_threshold *= 3;
+			if (parallel_threshold > INT_MAX / 3)
+				break;			/* avoid overflow */
+		}
+	}
+
+	/*
+	 * In no case use more than max_parallel_workers_per_gather workers.
+	 */
+	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+
+	return parallel_workers;
+}
+
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 2a49639..c08997a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -161,7 +161,7 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
 static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
 static double relation_byte_size(double tuples, int width);
 static double page_size(double tuples, int width);
-
+static Cost adjust_cost_for_parallelism(Path *path, Cost cpu_run_cost);
 
 /*
  * clamp_row_est
@@ -237,44 +237,7 @@ cost_seqscan(Path *path, PlannerInfo *root,
 
 	/* Adjust costing for parallelism, if used. */
 	if (path->parallel_workers > 0)
-	{
-		double		parallel_divisor = path->parallel_workers;
-		double		leader_contribution;
-
-		/*
-		 * Early experience with parallel query suggests that when there is
-		 * only one worker, the leader often makes a very substantial
-		 * contribution to executing the parallel portion of the plan, but as
-		 * more workers are added, it does less and less, because it's busy
-		 * reading tuples from the workers and doing whatever non-parallel
-		 * post-processing is needed.  By the time we reach 4 workers, the
-		 * leader no longer makes a meaningful contribution.  Thus, for now,
-		 * estimate that the leader spends 30% of its time servicing each
-		 * worker, and the remainder executing the parallel plan.
-		 */
-		leader_contribution = 1.0 - (0.3 * path->parallel_workers);
-		if (leader_contribution > 0)
-			parallel_divisor += leader_contribution;
-
-		/*
-		 * In the case of a parallel plan, the row count needs to represent
-		 * the number of tuples processed per worker.  Otherwise, higher-level
-		 * plan nodes that appear below the gather will be costed incorrectly,
-		 * because they'll anticipate receiving more rows than any given copy
-		 * will actually get.
-		 */
-		path->rows = clamp_row_est(path->rows / parallel_divisor);
-
-		/* The CPU cost is divided among all the workers. */
-		cpu_run_cost /= parallel_divisor;
-
-		/*
-		 * It may be possible to amortize some of the I/O cost, but probably
-		 * not very much, because most operating systems already do aggressive
-		 * prefetching.  For now, we assume that the disk run cost can't be
-		 * amortized at all.
-		 */
-	}
+		cpu_run_cost = adjust_cost_for_parallelism(path, cpu_run_cost);
 
 	path->startup_cost = startup_cost;
 	path->total_cost = startup_cost + cpu_run_cost + disk_run_cost;
@@ -831,10 +794,10 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
 	Cost		indexTotalCost;
-	Selectivity indexSelectivity;
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -855,13 +818,12 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	if (!enable_bitmapscan)
 		startup_cost += disable_cost;
 
-	/*
-	 * Fetch total cost of obtaining the bitmap, as well as its total
-	 * selectivity.
-	 */
-	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+	pages_fetched = compute_bitmap_pages(root, baserel, bitmapqual,
+										loop_count, &indexTotalCost,
+										&tuples_fetched);
 
 	startup_cost += indexTotalCost;
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
 
 	/* Fetch estimated page costs for tablespace containing table. */
 	get_tablespace_page_costs(baserel->reltablespace,
@@ -869,41 +831,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 							  &spc_seq_page_cost);
 
 	/*
-	 * Estimate number of main-table pages fetched.
-	 */
-	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
-
-	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
-
-	if (loop_count > 1)
-	{
-		/*
-		 * For repeated bitmap scans, scale up the number of tuples fetched in
-		 * the Mackert and Lohman formula by the number of scans, so that we
-		 * estimate the number of pages fetched by all the scans. Then
-		 * pro-rate for one scan.
-		 */
-		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
-											baserel->pages,
-											get_indexpath_pages(bitmapqual),
-											root);
-		pages_fetched /= loop_count;
-	}
-	else
-	{
-		/*
-		 * For a single scan, the number of heap pages that need to be fetched
-		 * is the same as the Mackert and Lohman formula for the case T <= b
-		 * (ie, no re-reads needed).
-		 */
-		pages_fetched = (2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
-	}
-	if (pages_fetched >= T)
-		pages_fetched = T;
-	else
-		pages_fetched = ceil(pages_fetched);
-
-	/*
 	 * For small numbers of pages we should charge spc_random_page_cost
 	 * apiece, while if nearly all the table's pages are being read, it's more
 	 * appropriate to charge spc_seq_page_cost apiece.  The effect is
@@ -932,8 +859,13 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+		cpu_run_cost = adjust_cost_for_parallelism(path, cpu_run_cost);
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
@@ -944,6 +876,56 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 }
 
 /*
+ * adjust_cost_for_parallelism
+ *
+ * Adjust the cpu cost based on number of parallel workers, also update
+ * the number of rows processed at each worker.
+ */
+static Cost
+adjust_cost_for_parallelism(Path *path, Cost cpu_run_cost)
+{
+	double		parallel_divisor = path->parallel_workers;
+	double		leader_contribution;
+	Cost		cpu_cost = cpu_run_cost;
+
+	/*
+	 * Early experience with parallel query suggests that when there is
+	 * only one worker, the leader often makes a very substantial
+	 * contribution to executing the parallel portion of the plan, but as
+	 * more workers are added, it does less and less, because it's busy
+	 * reading tuples from the workers and doing whatever non-parallel
+	 * post-processing is needed.  By the time we reach 4 workers, the
+	 * leader no longer makes a meaningful contribution.  Thus, for now,
+	 * estimate that the leader spends 30% of its time servicing each
+	 * worker, and the remainder executing the parallel plan.
+	 */
+	leader_contribution = 1.0 - (0.3 * path->parallel_workers);
+	if (leader_contribution > 0)
+		parallel_divisor += leader_contribution;
+
+	/*
+	 * In the case of a parallel plan, the row count needs to represent
+	 * the number of tuples processed per worker.  Otherwise, higher-level
+	 * plan nodes that appear below the gather will be costed incorrectly,
+	 * because they'll anticipate receiving more rows than any given copy
+	 * will actually get.
+	 */
+	path->rows = clamp_row_est(path->rows / parallel_divisor);
+
+	/* The CPU cost is divided among all the workers. */
+	cpu_cost /= parallel_divisor;
+
+	/*
+	 * It may be possible to amortize some of the I/O cost, but probably
+	 * not very much, because most operating systems already do aggressive
+	 * prefetching.  For now, we assume that the disk run cost can't be
+	 * amortized at all.
+	 */
+
+	return cpu_cost;
+}
+
+/*
  * cost_bitmap_tree_node
  *		Extract cost and selectivity from a bitmap tree node (index/and/or)
  */
@@ -4765,3 +4747,69 @@ page_size(double tuples, int width)
 {
 	return ceil(relation_byte_size(tuples, width) / BLCKSZ);
 }
+
+/*
+ * compute_bitmap_pages
+ *
+ * compute number of pages fetched from heap in bitmap heap scan.
+ */
+double
+compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel, Path *bitmapqual,
+			int loop_count, Cost *cost, double *tuple)
+{
+	Cost			indexTotalCost;
+	Selectivity		indexSelectivity;
+	double			T;
+	double			pages_fetched;
+	double			tuples_fetched;
+
+	/*
+	 * Fetch total cost of obtaining the bitmap, as well as its total
+	 * selectivity.
+	 */
+	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+
+	/*
+	 * Estimate number of main-table pages fetched.
+	 */
+	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
+
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
+
+	if (loop_count > 1)
+	{
+		/*
+		 * For repeated bitmap scans, scale up the number of tuples fetched in
+		 * the Mackert and Lohman formula by the number of scans, so that we
+		 * estimate the number of pages fetched by all the scans. Then
+		 * pro-rate for one scan.
+		 */
+		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
+											baserel->pages,
+											get_indexpath_pages(bitmapqual),
+											root);
+		pages_fetched /= loop_count;
+	}
+	else
+	{
+		/*
+		 * For a single scan, the number of heap pages that need to be fetched
+		 * is the same as the Mackert and Lohman formula for the case T <= b
+		 * (ie, no re-reads needed).
+		 */
+		pages_fetched =
+				(2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
+	}
+
+	if (pages_fetched >= T)
+		pages_fetched = T;
+	else
+		pages_fetched = ceil(pages_fetched);
+
+	if (cost)
+		*cost = indexTotalCost;
+	if (tuple)
+		*tuple = tuples_fetched;
+
+	return pages_fetched;
+}
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 2952bfb..1a7bde0 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 47158f6..4ffcf87 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2803,7 +2803,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index abb7507..0801b6e 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -47,7 +47,6 @@ typedef enum
 
 static List *translate_sub_tlist(List *tlist, int relid);
 
-
 /*****************************************************************************
  *		MISC. PATH UTILITIES
  *****************************************************************************/
@@ -1071,7 +1070,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,11 +1080,10 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
-
 	pathnode->bitmapqual = bitmapqual;
 
 	cost_bitmap_heap_scan(&pathnode->path, root, rel,
@@ -3192,7 +3191,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 5112d6d..01a6e49 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3387,6 +3387,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 0d12bbb..7cf0a3c 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -23,7 +23,6 @@
 #include "utils/relcache.h"
 #include "utils/snapshot.h"
 
-
 /* "options" flag bits for heap_insert */
 #define HEAP_INSERT_SKIP_WAL	0x0001
 #define HEAP_INSERT_SKIP_FSM	0x0002
@@ -178,6 +177,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_bm_update_snapshot(HeapScanDesc scan, Snapshot	snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index 0ed9c78..8afecb7 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+								ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+								ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+										shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 2fadf76..7a1ff63 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1399,6 +1401,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t	mutex;			/* mutual exclusion for below three fields */
+	int 	spageptr;		/* next spages index */
+	int 	schunkptr;		/* next schunks index */
+	int 	schunkbit;		/* next bit to check in current schunk */
+} ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *	 						parallel bitmap heap scan.
+ *
+ *	 	PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *	 						first worker to see this state will become
+ *	 						leader and will create TIDbitmap. This will
+ *	 						also set the state to PBM_INPROGRESS.
+ *	 	PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *	 						so workers need to sleep until leader set the
+ *	 						state to PBM_FINISHED and wake us up.
+ *	 	PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *	 						can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *	 	relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid						relid;
+	ParallelIterator		tbmiterator;
+	ParallelIterator		prefetch_iterator;
+	slock_t					prefetch_mutex;
+	int						prefetch_pages;
+	int						prefetch_target;
+	slock_t					state_mutex;
+	ConditionVariable		cv;
+	PBMState				state;
+	Size					ptbm_offset;
+	char					phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1413,6 +1481,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1429,6 +1499,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1444,7 +1515,11 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		is_leader		   is_leader is set true, if this worker become
+ *						   leader for parallel bitmap heap scan.
+ *		parallel_bitmap	   shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1459,6 +1534,9 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	bool		is_leader;
+	ParallelBitmapInfo	*parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 9303299..e215256 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+}TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -58,9 +75,14 @@ extern void tbm_union(TIDBitmap *a, const TIDBitmap *b);
 extern void tbm_intersect(TIDBitmap *a, const TIDBitmap *b);
 
 extern bool tbm_is_empty(const TIDBitmap *tbm);
-
-extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
+extern TBMIterator * tbm_begin_iterate(TIDBitmap *tbm,
+							ParallelTIDBitmap *parallel_info);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_attach_to_pagetable(TIDBitmap *tbm, ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+									ParallelTIDBitmap *parallel_tbm);
+void tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..e1baacc 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -183,6 +183,8 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
 					   double cte_rows);
 extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
 extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern double compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel,
+				Path *bitmapqual, int loop_count, Cost *cost, double *tuple);
 
 /*
  * prototypes for clausesel.c
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..397d266 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 44abe83..3264f44 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 1c9bf13..0aa2ab5 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -784,6 +784,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#3Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#2)
Re: Parallel bitmap heap scan

On Mon, Oct 17, 2016 at 1:23 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

There is major chance in tidbitmap.c file after efficient hash table
commit [1] and my patch need to be rebased.

Only parallel-bitmap-heap-scan need to be rebased, all other patch can
be applied on head as is.
Rebased version (v2) of parallel-bitmap-heap-scan is attached.

But what's the impact on performance? Presumably parallel bitmap heap
scan was already slower than the non-parallel version, and that commit
presumably widens the gap. Seems like something to worry about...

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#4Andres Freund
andres@anarazel.de
In reply to: Dilip Kumar (#1)
Re: Parallel bitmap heap scan

Hi,

On 2016-10-07 11:46:40 +0530, Dilip Kumar wrote:

Brief design idea:
-----------------------
#1. Shared TIDBitmap creation and initialization
First worker to see the state as parallel bitmap info as PBM_INITIAL
become leader and set the state to PBM_INPROGRESS All other workers
see the state as PBM_INPROGRESS will wait for leader to complete the
TIDBitmap.

#2 At this level TIDBitmap is ready and all workers are awake.

#3. Bitmap processing (Iterate and process the pages).
In this phase each worker will iterate over page and chunk array and
select heap pages one by one. If prefetch is enable then there will be
two iterator. Since multiple worker are iterating over same page and
chunk array we need to have a shared iterator, so we grab a spin lock
and iterate within a lock, so that each worker get and different page
to process.

I don't quite understand why the bitmap has to be parallel at all. As
far as I understand your approach as described here, the only thing that
needs to be shared are the iteration arrays. Since they never need to
be resized and such, it seems to make a lot more sense to just add an
API to share those, instead of the whole underlying hash.

Greetings,

Andres Freund

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#5Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#3)
Re: Parallel bitmap heap scan

On Tue, Oct 18, 2016 at 1:42 AM, Robert Haas <robertmhaas@gmail.com> wrote:

But what's the impact on performance? Presumably parallel bitmap heap
scan was already slower than the non-parallel version, and that commit
presumably widens the gap. Seems like something to worry about...

I have checked the performance in my local machine and there is no
impact on the gap.
If you see the explain analyze output of all queries which got
benefited which parallel bitmap map heap scan, BitmapIndex node is
taking very less time compare to BitmapHeap.

Actual execution time on head (before efficient hash table patch)
BitmapHeapNode BitmapIndexNode
Q6 38997 6951
Q14 14516 569
Q15 28530 1442

Out of 4 queries, Q4 is converted from parallel seqscan to parallel
bitmap scan so no impact.
Q14, Q15 time spent in BitmapIndex node is < 5% of time spent in
BitmapHeap Node. Q6 it's 20% but I did not see much impact on this in
my local machine. However I will take the complete performance reading
and post the data on my actual performance machine.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#6Dilip Kumar
dilipbalaut@gmail.com
In reply to: Andres Freund (#4)
Re: Parallel bitmap heap scan

On Tue, Oct 18, 2016 at 1:45 AM, Andres Freund <andres@anarazel.de> wrote:

I don't quite understand why the bitmap has to be parallel at all. As
far as I understand your approach as described here, the only thing that
needs to be shared are the iteration arrays. Since they never need to
be resized and such, it seems to make a lot more sense to just add an
API to share those, instead of the whole underlying hash.

You are right that we only share iteration arrays. But only point is
that each entry of iteration array is just a pointer to hash entry.
So either we need to build hash in shared memory (my current approach)
or we need to copy each hash element at shared location (I think this
is going to be expensive).

Let me know if I am missing something..

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#7Andres Freund
andres@anarazel.de
In reply to: Dilip Kumar (#5)
Re: Parallel bitmap heap scan

On 2016-10-19 09:43:10 +0530, Dilip Kumar wrote:

On Tue, Oct 18, 2016 at 1:42 AM, Robert Haas <robertmhaas@gmail.com> wrote:

But what's the impact on performance? Presumably parallel bitmap heap
scan was already slower than the non-parallel version, and that commit
presumably widens the gap. Seems like something to worry about...

I have checked the performance in my local machine and there is no
impact on the gap.

Try measuring with something more heavy on bitmap scan time
itself. E.g.
SELECT SUM(l_extendedprice) FROM lineitem WHERE (l_shipdate >= '1995-01-01'::date) AND (l_shipdate <= '1996-12-31'::date);
or similar. The tpch queries don't actually spend that much time in the
bitmapscan itself - the parallization of the rest of the query is what
matters...

Greetings,

Andres Freund

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#8Dilip Kumar
dilipbalaut@gmail.com
In reply to: Andres Freund (#7)
1 attachment(s)
Re: Parallel bitmap heap scan

On Wed, Oct 19, 2016 at 12:39 PM, Andres Freund <andres@anarazel.de> wrote:

Try measuring with something more heavy on bitmap scan time
itself. E.g.
SELECT SUM(l_extendedprice) FROM lineitem WHERE (l_shipdate >= '1995-01-01'::date) AND (l_shipdate <= '1996-12-31'::date);
or similar. The tpch queries don't actually spend that much time in the
bitmapscan itself - the parallization of the rest of the query is what
matters...

Yeah, I agree.

I have tested with this query, with exact filter condition it was
taking parallel sequence scan, so I have modified the filter a bit and
tested.

Tested with all default configuration in my local machine. I think I
will generate more such test cases and do detail testing in my
performance machine.

Explain Analyze results:
---------------------------------
On Head:
------------
postgres=# explain analyze SELECT SUM(l_extendedprice) FROM lineitem
WHERE (l_shipdate >= '1995-01-01'::date) AND (l_shipdate <=
'1996-03-31'::date);

QUERY PLAN
-----------------------------------------------------------------------------------------------------------------------------------------------------------
Aggregate (cost=848805.90..848805.91 rows=1 width=32) (actual
time=12440.165..12440.166 rows=1 loops=1)
-> Bitmap Heap Scan on lineitem (cost=143372.40..834833.25
rows=5589057 width=8) (actual time=1106.217..11183.722 rows=5678841
loops=1)
Recheck Cond: ((l_shipdate >= '1995-01-01'::date) AND
(l_shipdate <= '1996-03-31'::date))
Rows Removed by Index Recheck: 20678739
Heap Blocks: exact=51196 lossy=528664
-> Bitmap Index Scan on idx_lineitem_shipdate
(cost=0.00..141975.13 rows=5589057 width=0) (actual
time=1093.376..1093.376 rows=5678841 loops=1)
Index Cond: ((l_shipdate >= '1995-01-01'::date) AND
(l_shipdate <= '1996-03-31'::date))
Planning time: 0.185 ms
Execution time: 12440.819 ms
(9 rows)

After Patch:
---------------
postgres=# explain analyze SELECT SUM(l_extendedprice) FROM lineitem
WHERE (l_shipdate >= '1995-01-01'::date) AND (l_shipdate <=
'1996-03-31'::date);

QUERY PLAN

--------------------------------------------------------------------------------------------------------------------------------------------------------------
---------
Finalize Aggregate (cost=792751.16..792751.17 rows=1 width=32)
(actual time=6660.157..6660.157 rows=1 loops=1)
-> Gather (cost=792750.94..792751.15 rows=2 width=32) (actual
time=6659.378..6660.117 rows=3 loops=1)
Workers Planned: 2
Workers Launched: 2
-> Partial Aggregate (cost=791750.94..791750.95 rows=1
width=32) (actual time=6655.941..6655.941 rows=1 loops=3)
-> Parallel Bitmap Heap Scan on lineitem
(cost=143372.40..785929.00 rows=2328774 width=8) (actual
time=1980.797..6204.232 rows=1892947 loops=
3)
Recheck Cond: ((l_shipdate >= '1995-01-01'::date)
AND (l_shipdate <= '1996-03-31'::date))
Rows Removed by Index Recheck: 6930269
Heap Blocks: exact=17090 lossy=176443
-> Bitmap Index Scan on idx_lineitem_shipdate
(cost=0.00..141975.13 rows=5589057 width=0) (actual
time=1933.454..1933.454 rows=5678841
loops=1)
Index Cond: ((l_shipdate >=
'1995-01-01'::date) AND (l_shipdate <= '1996-03-31'::date))
Planning time: 0.207 ms
Execution time: 6669.195 ms
(13 rows)

Summary:
-> With patch overall execution is 2 time faster compared to head.
-> Bitmap creation with patch is bit slower compared to head and thats
because of DHT vs efficient hash table.

I found one defect in v2 patch, that I induced during last rebasing.
That is fixed in v3.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

parallel-bitmap-heap-scan-v3.patchapplication/octet-stream; name=parallel-bitmap-heap-scan-v3.patchDownload
diff --git a/src/backend/access/gin/ginget.c b/src/backend/access/gin/ginget.c
index 9ed9fd2..c8034d0 100644
--- a/src/backend/access/gin/ginget.c
+++ b/src/backend/access/gin/ginget.c
@@ -362,7 +362,7 @@ restartScanEntry:
 
 		if (entry->matchBitmap && !tbm_is_empty(entry->matchBitmap))
 		{
-			entry->matchIterator = tbm_begin_iterate(entry->matchBitmap);
+			entry->matchIterator = tbm_begin_iterate(entry->matchBitmap, NULL);
 			entry->isFinished = FALSE;
 		}
 	}
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index b019bc1..1dfd492 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_bm_update_snapshot
+ *
+ *		Update snpashot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_bm_update_snapshot(HeapScanDesc scan, Snapshot	snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 72bacd5..1e34f26 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -203,6 +204,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+										e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -255,6 +260,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+										d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -724,6 +734,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									(BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index 449aacb..c3ad77b 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -47,16 +47,63 @@
 #include "utils/spccache.h"
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
-
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
-
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static void pbms_set_parallel(PlanState *node);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+									ParallelIterator *parallel_iterator,
+									bool is_parallel);
+static void prefetch_pages(int *prefetch_pages, int prefetch_target,
+						BitmapHeapScanState *node, HeapScanDesc scan);
+static void update_prefetch_target(int *prefetch_target, int prefetch_maximum);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
  *
  *		Retrieve next tuple from the BitmapHeapScan node's currentRelation
+ *
+ *
+ *	[PARALLEL BITMAP HEAP SCAN ALGORITHM]
+ *
+ *	#1. Shared TIDBitmap creation and initialization
+ *		a) First worker to see the state as parallel bitmap info as
+ *		PBM_INITIAL become leader and set the state to PBM_INPROGRESS
+ *		All other workers see the state as PBM_INPROGRESS will wait for
+ *		leader to complete the TIDBitmap.
+ *
+ *		Leader Worker Processing:
+ *		(Leader is responsible for creating shared TIDBitmap and create
+ *		shared page and chunk array from TIDBitmap.)
+ *			1) Create TIDBitmap using DHT.
+ *			2) Begin Iterate: convert hash table into shared page and chunk
+ *			array.
+ *			3) Restore local TIDBitmap variable information into
+ *			ParallelBitmapInfo so that other worker can see those.
+ *			4) set state to PBM_FINISHED.
+ *			5) Wake up other workers.
+ *
+ *		Other Worker Processing:
+ *			1) Wait until leader create shared TIDBitmap and shared page
+ *			and chunk array.
+ *			2) Attach to shared page table, copy TIDBitmap from
+ *			ParallelBitmapInfo to local TIDBitmap, we copy this to local
+ *			TIDBitmap so that next level processing can read information
+ *			same as in non parallel case and we can avoid extra changes
+ *			in code.
+ *
+ *	# At this level TIDBitmap is ready and all workers are awake #
+ *
+ *	#2. Bitmap processing (Iterate and process the pages).
+ *		. In this phase each worker will iterate over page and chunk array and
+ *		select heap pages one by one. If prefetch is enable then there will
+ *		be two iterator.
+ *		. Since multiple worker are iterating over same page and chunk array
+ *		we need to have a shared iterator, so we grab a spin lock and iterate
+ *		within a lock.
  * ----------------------------------------------------------------
  */
 static TupleTableSlot *
@@ -67,12 +114,19 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TIDBitmap  *tbm;
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
-
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+	bool		is_parallel = node->parallel_bitmap ? true : false;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (is_parallel)
+		parallel_tbm = (ParallelTIDBitmap*)((char *)pbminfo +
+					pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -101,36 +155,106 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running
+		 * in non parallel mode or we are leader worker.
+		 *
+		 * In parallel mode leader worker will immediately come out
+		 * of the function, but all other worker will be blocked
+		 * until leader worker wake them up.
+		 */
+		if (!is_parallel || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer
+			 * node and set parallel flag in lower level bitmap index scan.
+			 * Later bitmap index node will use this flag to indicate
+			 * tidbitmap that it needs to create an shared page table.
+			 */
+			if (is_parallel)
+			{
+				node->is_leader = true;
+				pbms_set_parallel(outerPlanState(node));
+			}
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM.
+			 * Here we need to create a local TBM and copy information from
+			 * shared location. We also need to attach to shared page table
+			 * using hash table handle stored in parallel_tbm(shared memory).
+			 */
+			tbm = tbm_create(work_mem * 1024L);
+			tbm_set_parallel(tbm, node->ss.ps.state->es_query_area);
+			tbm_attach_to_pagetable(tbm, parallel_tbm);
+		}
 
 		if (!tbm || !IsA(tbm, TIDBitmap))
 			elog(ERROR, "unrecognized result from subplan");
 
 		node->tbm = tbm;
-		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+		node->tbmiterator = tbmiterator
+							= tbm_begin_iterate(tbm, parallel_tbm);
 		node->tbmres = tbmres = NULL;
 
 #ifdef USE_PREFETCH
 		if (node->prefetch_maximum > 0)
 		{
-			node->prefetch_iterator = prefetch_iterator = tbm_begin_iterate(tbm);
-			node->prefetch_pages = 0;
-			node->prefetch_target = -1;
+			node->prefetch_iterator = prefetch_iterator =
+							tbm_begin_iterate(tbm, parallel_tbm);
+
+			/* These variable are used only in case of non parallel mode */
+			if (!is_parallel)
+			{
+				node->prefetch_pages = 0;
+				node->prefetch_target = -1;
+			}
 		}
 #endif   /* USE_PREFETCH */
+
+		/*
+		 * If we are in parallel mode and we are leader worker then
+		 * copy the local TBM information to shared location, and wake
+		 * up other workers.
+		 */
+		if (node->is_leader)
+		{
+			/*
+			 * copy TBM local information in shared memory before waking
+			 * up other workers. Other workers will create there own
+			 * TBM and copy information from shared memory, they will
+			 * also use hash table handle from shared memory for attaching
+			 * to shared memory hash table.
+			 */
+			tbm_update_shared_members(tbm, parallel_tbm);
+
+			/* Change the state under a lock */
+			SpinLockAcquire(&pbminfo->state_mutex);
+			pbminfo->state = PBM_FINISHED;
+			SpinLockRelease(&pbminfo->state_mutex);
+
+			/* Wake up all other workers. */
+			ConditionVariableBroadcast(&pbminfo->cv);
+		}
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+													&pbminfo->tbmiterator,
+													is_parallel);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +262,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (is_parallel)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages --;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+											&pbminfo->prefetch_iterator,
+											is_parallel);
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not
+				 * ensure that current blockno in main iterator and prefetch
+				 * iterator is same. It's possible that whatever blockno we
+				 * are prefetching is getting processed by some other worker.
+				 */
+				if (!is_parallel &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +333,25 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				if (!is_parallel)
+					update_prefetch_target(&node->prefetch_target,
+										 node->prefetch_maximum);
+				else
+				{
+					/*
+					 * If we are in parallel mode then grab prefetch_mutex
+					 * before updating prefetch target.
+					 */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+					update_prefetch_target(&pbminfo->prefetch_target,
+										 node->prefetch_maximum);
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+				}
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +367,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (!is_parallel)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex
+				 * before updating prefetch target.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+					pbminfo->prefetch_target++;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +406,29 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			if (pbminfo == NULL)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				prefetch_pages(&node->prefetch_pages,
+							node->prefetch_target, node, scan);
+			}
+			else if(node->prefetch_pages < node->prefetch_target)
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex
+				 * before going for prefetch.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
 
-				if (tbmpre == NULL)
-				{
-					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
-					break;
-				}
-				node->prefetch_pages++;
-				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+				prefetch_pages(&pbminfo->prefetch_pages,
+							pbminfo->prefetch_target, node, scan);
+
+				SpinLockRelease(&pbminfo->prefetch_mutex);
 			}
+
+			/* Restore the prefetch_iterator */
+			prefetch_iterator = node->prefetch_iterator;
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +643,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +761,8 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->is_leader = false;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -653,3 +849,311 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool needWait = false;
+	bool queuedSelf = false;
+	bool leader = false;
+
+	for(;;)
+	{
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		/*
+		 * if state is initial then we are the first one to come here
+		 * set the state to in progress and mark ourself as leader
+		 */
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+
+		/* bitmap create is in progress so we need to wait */
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/*
+		 * If we are a leader or else leader has already created a
+		 * tid bitmap.
+		 */
+		if (leader || !needWait)
+			break;
+
+		/* We need to queue */
+		if (queuedSelf)
+		{
+			/* Sleep until leader send wake up signal */
+			ConditionVariableSleep(WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+			queuedSelf = false;
+			needWait = false;
+		}
+		else if (needWait)
+		{
+			/* Add ourself to wait queue */
+			ConditionVariablePrepareToSleep(&pbminfo->cv);
+			queuedSelf = true;
+		}
+	}
+
+	/* Cancel the sleep before return */
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node
+	 * as parallel, because only first node will create the main bitmap
+	 * other bitmaps will be merged to the first bitmap so no need to
+	 * create them in shared memory.
+	 */
+	switch(node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState*)node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState*)node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState*)node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/*
+ * prefetch_pages
+ *
+ * Prefetch pages before going for actual processing of the page.
+ */
+static void
+prefetch_pages(int *prefetch_pages, int prefetch_target,
+			BitmapHeapScanState *node, HeapScanDesc scan)
+{
+	TBMIterator *iterator = node->prefetch_iterator;
+	ParallelIterator *parallel_iteartor;
+
+	/*
+	 * If parallel bitmap info available means we are running
+	 * in parallel mode. So use parallel iterator for prefetching.
+	 */
+	if (node->parallel_bitmap)
+		parallel_iteartor = &node->parallel_bitmap->prefetch_iterator;
+
+	while (*prefetch_pages < prefetch_target)
+	{
+		TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+										parallel_iteartor,
+										node->parallel_bitmap ? true:false);
+		if (tbmpre == NULL)
+		{
+			/* No more pages to prefetch */
+			tbm_end_iterate(iterator);
+			node->prefetch_iterator = NULL;
+			break;
+		}
+
+		(*prefetch_pages)++;
+		PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+	}
+}
+
+/*
+ * update_prefetch_target
+ *
+ * Update the value of prefetch target
+ */
+static void
+update_prefetch_target(int *prefetch_target, int prefetch_maximum)
+{
+	/*
+	 * Increase prefetch target if it's not yet at the max.  Note that
+	 * we will increase it to zero after fetching the very first
+	 * page/tuple, then to one after the second tuple is fetched, then
+	 * it doubles as later pages are fetched.
+	 */
+	if (*prefetch_target >= prefetch_maximum)
+		 /* don't increase any further */ ;
+	else if (*prefetch_target >= prefetch_maximum / 2)
+		*prefetch_target = prefetch_maximum;
+	else if (*prefetch_target > 0)
+		*prefetch_target *= 2;
+	else
+		(*prefetch_target)++;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Acquire a iterator lock.
+ * copy iterator state from shared iterator to local iterator.
+ * Call tbm_iterate and restore the state back to shared iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator,
+				ParallelIterator *parallel_iterator,
+				bool is_parallel)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (!is_parallel)
+		return tbm_iterate(iterator);
+
+	/*
+	 * We are in parallel mode so grab parallel iterator mutex
+	 * before calling iterator.
+	 */
+	SpinLockAcquire(&parallel_iterator->mutex);
+
+	/*
+	 * Now we have got lock on iterator so copy information from
+	 * shared location to our local iterator.
+	 */
+	iterator->spageptr = parallel_iterator->spageptr;
+	iterator->schunkptr = parallel_iterator->schunkptr;
+	iterator->schunkbit = parallel_iterator->schunkbit;
+
+	output = tbm_iterate(iterator);
+
+	/*
+	 * tbm_iterate would have changed the iterator value
+	 * in local iterator so copy them back to shared location
+	 * before releasing the lock.
+	 */
+	parallel_iterator->spageptr = iterator->spageptr;
+	parallel_iterator->schunkptr = iterator->schunkptr;
+	parallel_iterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&parallel_iterator->mutex);
+
+	return output;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+							phs_snapshot_data),
+							EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+						 ParallelContext *pcxt)
+{
+	ParallelBitmapInfo	*pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Snapshot	snapshot;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+								phs_snapshot_data),
+								EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+
+	heap_bm_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo	*pbminfo;
+	Snapshot			snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_bm_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index a364098..9cc5088 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate
+	 * that we need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_area);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 826fead..bdf5368 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -43,6 +43,11 @@
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
 #include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
+#include "storage/lwlock.h"
+#include "storage/shmem.h"
+#include "storage/dht.h"
+#include "storage/dsa.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -79,6 +84,8 @@
 /* number of active words for a lossy chunk: */
 #define WORDS_PER_CHUNK  ((PAGES_PER_CHUNK - 1) / BITS_PER_BITMAPWORD + 1)
 
+#define TBM_IS_SHARED(tbm) (tbm)->shared
+
 /*
  * The hashtable entries are represented by this data structure.  For
  * an exact page, blockno is the page number and bit k of the bitmap
@@ -139,24 +146,28 @@ struct TIDBitmap
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	bool			shared;		/* need to build shared tbm if set*/
+	dht_hash_table_handle	hash_handle;	/* shared hash table handle */
+	dht_hash_table	*shared_pagetable;	/* dynamic hash table */
+	dsa_area		*area;		/* reference to per-query shared memory area */
 };
 
 /*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
  */
-struct TBMIterator
+struct ParallelTIDBitmap
 {
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+	dht_hash_table_handle	hash_handle;	/* shared hash table handle */
+	dsa_pointer		dsa_pages;		/* dsa pointers for all kind of pages */
+	int				nentries;		/* number of entries in pagetable */
+	int				maxentries;		/* limit on same to meet maxbytes */
+	int				npages;			/* number of exact entries in pagetable */
+	int				nchunks;		/* number of lossy entries in pagetable */
+	int				item_count;		/* total item in dsa_pages */
+	bool			inited;			/* set true after leader converts page */
+									/* table to dsa_pointer's array. */
 };
 
-
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
 static bool tbm_intersect_page(TIDBitmap *a, PagetableEntry *apage,
@@ -168,6 +179,7 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+static void* tbm_pagetable_iterate(TIDBitmap *tbm, void *iterator);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -266,6 +278,33 @@ tbm_create_pagetable(TIDBitmap *tbm)
 }
 
 /*
+ * tbm_create_shared_pagetable
+ *
+ * Creates shared hash table using DHT for parallel bitmap scan.
+ */
+static void
+tbm_create_shared_pagetable(TIDBitmap *tbm)
+{
+	dht_parameters params = {0};
+
+	params.key_size = sizeof(BlockNumber);
+	params.entry_size = sizeof(PagetableEntry);
+	params.compare_function = memcmp;
+	params.hash_function = tag_hash;
+
+	params.tranche_id = LWLockNewTrancheId();
+
+	/* Create a dynamic hash table */
+	tbm->shared_pagetable = dht_create(tbm->area, &params);
+	if (tbm->shared_pagetable == NULL)
+		elog(ERROR, "could not create hash table");
+
+	/* Get the handle so that other backend can attach using this handle */
+	tbm->hash_handle = dht_get_hash_table_handle(tbm->shared_pagetable);
+	tbm->status = TBM_HASH;
+}
+
+/*
  * tbm_free - free a TIDBitmap
  */
 void
@@ -277,6 +316,11 @@ tbm_free(TIDBitmap *tbm)
 		pfree(tbm->spages);
 	if (tbm->schunks)
 		pfree(tbm->schunks);
+
+	/* If we have shared page table then detach from it */
+	if (tbm->shared_pagetable)
+		dht_detach(tbm->shared_pagetable);
+
 	pfree(tbm);
 }
 
@@ -453,6 +497,14 @@ void
 tbm_intersect(TIDBitmap *a, const TIDBitmap *b)
 {
 	Assert(!a->iterating);
+
+	/*
+	 * In case of BitmapAnd, only first node will create shared TBM
+	 * and all other node will have local TBM which will be merged
+	 * with first shared TBM.
+	 */
+	Assert(!TBM_IS_SHARED(b));
+
 	/* Nothing to do if a is empty */
 	if (a->nentries == 0)
 		return;
@@ -473,10 +525,27 @@ tbm_intersect(TIDBitmap *a, const TIDBitmap *b)
 	{
 		pagetable_iterator i;
 		PagetableEntry *apage;
+		void		   *iterator;
+		dht_iterator 	dhtiterator;
 
 		Assert(a->status == TBM_HASH);
 		pagetable_start_iterate(a->pagetable, &i);
-		while ((apage = pagetable_iterate(a->pagetable, &i)) != NULL)
+		/*
+		 * If we are using shared page table then use DHT iterator.
+		 */
+		if (TBM_IS_SHARED(a))
+		{
+			iterator = &dhtiterator;
+			dht_iterate_begin(a->shared_pagetable, &dhtiterator, false);
+		}
+		else
+		{
+			iterator = &i;
+			pagetable_start_iterate(a->pagetable, &i);
+		}
+
+		while ((apage =
+				(PagetableEntry *)tbm_pagetable_iterate(a, iterator)) != NULL)
 		{
 			if (tbm_intersect_page(a, apage, b))
 			{
@@ -486,8 +555,23 @@ tbm_intersect(TIDBitmap *a, const TIDBitmap *b)
 				else
 					a->npages--;
 				a->nentries--;
-				if (!pagetable_delete(a->pagetable, apage->blockno))
-					elog(ERROR, "hash table corrupted");
+
+				/*
+				 * If we are using shared hash then we need to release the
+				 * lock on DHT element before deleting it.
+				 */
+				if (TBM_IS_SHARED(a))
+				{
+					dht_iterate_release(&dhtiterator);
+					if (dht_delete_key(a->shared_pagetable,
+									(void *)&(apage->blockno)))
+						elog(ERROR, "hash table corrupted");
+				}
+				else
+				{
+					if (!pagetable_delete(a->pagetable, apage->blockno))
+						elog(ERROR, "hash table corrupted");
+				}
 			}
 		}
 	}
@@ -599,7 +683,7 @@ tbm_is_empty(const TIDBitmap *tbm)
  * contents repeatedly, including parallel scans.
  */
 TBMIterator *
-tbm_begin_iterate(TIDBitmap *tbm)
+tbm_begin_iterate(TIDBitmap *tbm, ParallelTIDBitmap *parallel_info)
 {
 	TBMIterator *iterator;
 
@@ -612,11 +696,16 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	iterator->tbm = tbm;
 
 	/*
-	 * Initialize iteration pointers.
+	 * Initialize iteration pointers, only if it's not shared tbm.
+	 * In case of shared tbm, we will copy these values from
+	 * shared iterator before calling tbm_iterate.
 	 */
-	iterator->spageptr = 0;
-	iterator->schunkptr = 0;
-	iterator->schunkbit = 0;
+	if (!TBM_IS_SHARED(tbm))
+	{
+		iterator->spageptr = 0;
+		iterator->schunkptr = 0;
+		iterator->schunkbit = 0;
+	}
 
 	/*
 	 * If we have a hashtable, create and fill the sorted page lists, unless
@@ -640,15 +729,79 @@ tbm_begin_iterate(TIDBitmap *tbm)
 				MemoryContextAlloc(tbm->mcxt,
 								   tbm->nchunks * sizeof(PagetableEntry *));
 
-		npages = nchunks = 0;
-		pagetable_start_iterate(tbm->pagetable, &i);
-		while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+		/*
+		 * If we have shared TBM means we are running in parallel mode.
+		 * So iterate over DHT and construct page and chunk array.
+		 *
+		 * First leader worker will create array of dsa pointers which
+		 * will holds dsa pointers for both pages and chunks, later
+		 * while converting to local pointers we will identify them
+		 * and copy in their respective array.
+		 */
+		if (TBM_IS_SHARED(tbm))
 		{
-			if (page->ischunk)
-				tbm->schunks[nchunks++] = page;
-			else
-				tbm->spages[npages++] = page;
+			dsa_pointer *dsa_spages;
+			dht_hash_table_item *item;
+			int			ncount = 0;
+			int			i;
+
+			/*
+			 * Iterate over DHT and create array of dsa_pointers.
+			 * Only leader will perform this step after that inited
+			 * flag will be set.
+			 */
+			if (!(parallel_info->inited) &&
+				(tbm->npages > 0 || tbm->nchunks > 0))
+			{
+				dht_iterator dhtiterator;
+				dsa_pointer	 dsa_page;
+
+				parallel_info->dsa_pages = dsa_allocate(tbm->area,
+						(tbm->nchunks + tbm->npages) * sizeof(dsa_pointer));
+
+				dsa_spages =
+						dsa_get_address(tbm->area, parallel_info->dsa_pages);
+
+				dht_iterate_begin(tbm->shared_pagetable, &dhtiterator, false);
+				while ((dsa_page = dht_iterate_next_dsa(&dhtiterator)) !=
+						InvalidDsaPointer)
+					dsa_spages[ncount++] = dsa_page;
+
+				parallel_info->inited = true;
+				parallel_info->item_count = ncount;
+			}
+
+			/*
+			 * This step will be done by all the workers including leader.
+			 * Here we need to convert array of dsa pointers to local
+			 * page and chunk array.
+			 */
+			dsa_spages = dsa_get_address(tbm->area, parallel_info->dsa_pages);
+			npages = nchunks = 0;
+			for (i = 0; i < parallel_info->item_count; i++)
+			{
+				item = dsa_get_address(tbm->area, dsa_spages[i]);
+				page = (PagetableEntry*)&(item->entry);
+
+				if (page->ischunk)
+					tbm->schunks[nchunks++] = page;
+				else
+					tbm->spages[npages++] = page;
+			}
 		}
+		else
+		{
+			npages = nchunks = 0;
+			pagetable_start_iterate(tbm->pagetable, &i);
+			while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+			{
+				if (page->ischunk)
+					tbm->schunks[nchunks++] = page;
+				else
+					tbm->spages[npages++] = page;
+			}
+		}
+
 		Assert(npages == tbm->npages);
 		Assert(nchunks == tbm->nchunks);
 		if (npages > 1)
@@ -811,11 +964,23 @@ tbm_find_pageentry(const TIDBitmap *tbm, BlockNumber pageno)
 		return page;
 	}
 
-	page = pagetable_lookup(tbm->pagetable, pageno);
+	/* If TBM has shared page table then find in DHT */
+	if (TBM_IS_SHARED(tbm))
+		page = (PagetableEntry *) dht_find(tbm->shared_pagetable,
+										  (void *) &pageno,
+										  false);
+	else
+		page = pagetable_lookup(tbm->pagetable, pageno);
+
 	if (page == NULL)
 		return NULL;
 	if (page->ischunk)
 		return NULL;			/* don't want a lossy chunk header */
+
+	/* If entry is from shared page table then release the entry. */
+	if (TBM_IS_SHARED(tbm))
+		dht_release(tbm->shared_pagetable, (void*)page);
+
 	return page;
 }
 
@@ -833,7 +998,11 @@ tbm_get_pageentry(TIDBitmap *tbm, BlockNumber pageno)
 	PagetableEntry *page;
 	bool		found;
 
-	if (tbm->status == TBM_EMPTY)
+	/*
+	 * Use fixed slot only if it's local tidbitmap, If shared bitmap
+	 * then directly insert into shared hash table.
+	 */
+	if ((tbm->status == TBM_EMPTY) && !TBM_IS_SHARED(tbm))
 	{
 		/* Use the fixed slot */
 		page = &tbm->entry1;
@@ -842,6 +1011,7 @@ tbm_get_pageentry(TIDBitmap *tbm, BlockNumber pageno)
 	}
 	else
 	{
+		/* In case of shared TBM, status will never be TBM_ONE_PAGE */
 		if (tbm->status == TBM_ONE_PAGE)
 		{
 			page = &tbm->entry1;
@@ -851,8 +1021,23 @@ tbm_get_pageentry(TIDBitmap *tbm, BlockNumber pageno)
 			tbm_create_pagetable(tbm);
 		}
 
-		/* Look up or create an entry */
-		page = pagetable_insert(tbm->pagetable, pageno, &found);
+		/*
+		 * In case of parallel bitmap scan, we don't switch from TBM_EMPTY
+		 * to TBM_ONE_PAGE. So even if we are here we may not have hash
+		 * table ready, so if tbm->status is not yet TBM_HASH then create
+		 * shared page table.
+		 */
+		if (TBM_IS_SHARED(tbm))
+		{
+			if (tbm->status != TBM_HASH)
+				tbm_create_shared_pagetable(tbm);
+
+			page = (PagetableEntry *) dht_find_or_insert(tbm->shared_pagetable,
+											  (void *) &pageno, &found);
+		}
+		else
+			/* Look up or create an entry */
+			page = pagetable_insert(tbm->pagetable, pageno, &found);
 	}
 
 	/* Initialize it if not present before */
@@ -868,6 +1053,10 @@ tbm_get_pageentry(TIDBitmap *tbm, BlockNumber pageno)
 		tbm->npages++;
 	}
 
+	/* Release the entry lock if it's from shared hash table */
+	if (page && TBM_IS_SHARED(tbm))
+		dht_release(tbm->shared_pagetable, (void*)page);
+
 	return page;
 }
 
@@ -889,7 +1078,20 @@ tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno)
 	bitno = pageno % PAGES_PER_CHUNK;
 	chunk_pageno = pageno - bitno;
 
-	page = pagetable_lookup(tbm->pagetable, chunk_pageno);
+	/* If tbm has shared page table then use DHT to find the entry. */
+	if (TBM_IS_SHARED(tbm))
+		page = (PagetableEntry *) dht_find(tbm->shared_pagetable,
+										(void *) &chunk_pageno, false);
+	else
+		page = pagetable_lookup(tbm->pagetable, chunk_pageno);
+
+	/*
+	 * If entry is from shared page table then release the entry.
+	 * Currently only one worker is building the bitmap so it's
+	 * fine to release it here.
+	 */
+	if (TBM_IS_SHARED(tbm) && page != NULL)
+		dht_release(tbm->shared_pagetable, (void*)page);
 
 	if (page != NULL && page->ischunk)
 	{
@@ -920,7 +1122,13 @@ tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno)
 
 	/* We force the bitmap into hashtable mode whenever it's lossy */
 	if (tbm->status != TBM_HASH)
-		tbm_create_pagetable(tbm);
+	{
+		/* Create shared page table if we are in parallel mode. */
+		if (TBM_IS_SHARED(tbm))
+			tbm_create_shared_pagetable(tbm);
+		else
+			tbm_create_pagetable(tbm);
+	}
 
 	bitno = pageno % PAGES_PER_CHUNK;
 	chunk_pageno = pageno - bitno;
@@ -931,7 +1139,15 @@ tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno)
 	 */
 	if (bitno != 0)
 	{
-		if (pagetable_delete(tbm->pagetable, pageno))
+		bool result = false;
+
+		/* Look up or create an entry */
+		if (TBM_IS_SHARED(tbm))
+			result = dht_delete_key(tbm->shared_pagetable, (void *) &pageno);
+		else
+			result = pagetable_delete(tbm->pagetable, pageno);
+
+		if (result)
 		{
 			/* It was present, so adjust counts */
 			tbm->nentries--;
@@ -940,7 +1156,11 @@ tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno)
 	}
 
 	/* Look up or create entry for chunk-header page */
-	page = pagetable_insert(tbm->pagetable, chunk_pageno, &found);
+	if (TBM_IS_SHARED(tbm))
+		page = (PagetableEntry *) dht_find_or_insert(tbm->shared_pagetable,
+										  (void *) &chunk_pageno, &found);
+	else
+		page = pagetable_insert(tbm->pagetable, chunk_pageno, &found);
 
 	/* Initialize it if not present before */
 	if (!found)
@@ -975,6 +1195,10 @@ tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno)
 	wordnum = WORDNUM(bitno);
 	bitnum = BITNUM(bitno);
 	page->words[wordnum] |= ((bitmapword) 1 << bitnum);
+
+	/* Unlock an entry which was locked by dht_find_or_insert. */
+	if (TBM_IS_SHARED(tbm))
+		dht_release(tbm->shared_pagetable, (void*)page);
 }
 
 /*
@@ -985,7 +1209,8 @@ tbm_lossify(TIDBitmap *tbm)
 {
 	pagetable_iterator i;
 	PagetableEntry *page;
-
+	dht_iterator	dhtiterator;
+	void		   *iterator;
 	/*
 	 * XXX Really stupid implementation: this just lossifies pages in
 	 * essentially random order.  We should be paying some attention to the
@@ -998,8 +1223,24 @@ tbm_lossify(TIDBitmap *tbm)
 	Assert(!tbm->iterating);
 	Assert(tbm->status == TBM_HASH);
 
-	pagetable_start_iterate_at(tbm->pagetable, &i, tbm->lossify_start);
-	while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+	/*
+	 * If we are using shared page table then use DHT iterator otherwise
+	 * dyna hash iterator
+	 */
+	if (TBM_IS_SHARED(tbm))
+	{
+		iterator = &dhtiterator;
+		dht_iterate_begin(tbm->shared_pagetable, &dhtiterator, false);
+	}
+	else
+	{
+		iterator = &i;
+		pagetable_start_iterate_at(tbm->pagetable, &i, tbm->lossify_start);
+	}
+
+
+	while ((page = (PagetableEntry *)
+				tbm_pagetable_iterate(tbm, iterator)) != NULL)
 	{
 		if (page->ischunk)
 			continue;			/* already a chunk header */
@@ -1011,11 +1252,19 @@ tbm_lossify(TIDBitmap *tbm)
 		if ((page->blockno % PAGES_PER_CHUNK) == 0)
 			continue;
 
+		/* If we are using shared hash then release the lock on element. */
+		if (TBM_IS_SHARED(tbm))
+			dht_iterate_release(&dhtiterator);
+
 		/* This does the dirty work ... */
 		tbm_mark_page_lossy(tbm, page->blockno);
 
 		if (tbm->nentries <= tbm->maxentries / 2)
 		{
+			/* we have done enough */
+			if (TBM_IS_SHARED(tbm))
+				dht_iterate_end(&dhtiterator);
+
 			/*
 			 * We have made enough room. Remember where to start lossifying
 			 * next round, so we evenly iterate over the hashtable.
@@ -1061,3 +1310,96 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_attach_to_pagetable
+ *
+ * Attach worker to shared TID bitmap created by leader worker.
+ */
+void
+tbm_attach_to_pagetable(TIDBitmap *tbm, ParallelTIDBitmap *stbm)
+{
+	dht_parameters params = {0};
+
+	params.key_size = sizeof(BlockNumber);
+	params.entry_size = sizeof(PagetableEntry);
+	params.compare_function = memcmp;
+	params.hash_function = tag_hash;
+
+	tbm->shared_pagetable = dht_attach(tbm->area, &params, stbm->hash_handle);
+
+	tbm->status = TBM_HASH;
+	tbm->nchunks = stbm->nchunks;
+	tbm->nentries = stbm->nentries;
+	tbm->npages = stbm->npages;
+	tbm->maxentries = stbm->maxentries;
+	tbm->shared = true;
+}
+
+/*
+ * tbm_update_shared_members
+ *
+ * Restore leaders private tbm state to shared location. This must
+ * be called before waking up the other worker.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before
+	 * waking up the other workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->hash_handle = tbm->hash_handle;
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->shared = true;
+	tbm->area = (dsa_area*)area;
+}
+
+/*
+ * tbm_pagetable_iterate
+ *
+ * Unified function to scan full tbm page table
+ * if tbm is shared then it will operate on shared hash
+ * table otherwise on local hash table.
+ */
+static void*
+tbm_pagetable_iterate(TIDBitmap *tbm, void *iterator)
+{
+	if (TBM_IS_SHARED(tbm))
+	{
+		dht_iterator *dhtiterator = (dht_iterator*)iterator;
+
+		return dht_iterate_next(dhtiterator);
+	}
+	else
+	{
+		pagetable_iterator *i = (pagetable_iterator*)iterator;
+
+		return pagetable_iterate(tbm->pagetable, i);
+	}
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index e42ef98..058e55a 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -126,6 +126,7 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static int compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 
 
 /*
@@ -678,49 +679,7 @@ create_plain_partial_paths(PlannerInfo *root, RelOptInfo *rel)
 {
 	int			parallel_workers;
 
-	/*
-	 * If the user has set the parallel_workers reloption, use that; otherwise
-	 * select a default number of workers.
-	 */
-	if (rel->rel_parallel_workers != -1)
-		parallel_workers = rel->rel_parallel_workers;
-	else
-	{
-		int			parallel_threshold;
-
-		/*
-		 * If this relation is too small to be worth a parallel scan, just
-		 * return without doing anything ... unless it's an inheritance child.
-		 * In that case, we want to generate a parallel path here anyway.  It
-		 * might not be worthwhile just for this relation, but when combined
-		 * with all of its inheritance siblings it may well pay off.
-		 */
-		if (rel->pages < (BlockNumber) min_parallel_relation_size &&
-			rel->reloptkind == RELOPT_BASEREL)
-			return;
-
-		/*
-		 * Select the number of workers based on the log of the size of the
-		 * relation.  This probably needs to be a good deal more
-		 * sophisticated, but we need something here for now.  Note that the
-		 * upper limit of the min_parallel_relation_size GUC is chosen to
-		 * prevent overflow here.
-		 */
-		parallel_workers = 1;
-		parallel_threshold = Max(min_parallel_relation_size, 1);
-		while (rel->pages >= (BlockNumber) (parallel_threshold * 3))
-		{
-			parallel_workers++;
-			parallel_threshold *= 3;
-			if (parallel_threshold > INT_MAX / 3)
-				break;			/* avoid overflow */
-		}
-	}
-
-	/*
-	 * In no case use more than max_parallel_workers_per_gather workers.
-	 */
-	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+	parallel_workers = compute_parallel_worker(rel, rel->pages);
 
 	/* If any limit was set to zero, the user doesn't want a parallel scan. */
 	if (parallel_workers <= 0)
@@ -2866,6 +2825,84 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int				parallel_workers;
+	double			pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path*)create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+static int
+compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
+{
+	int			parallel_workers;
+
+	/*
+	 * If the user has set the parallel_workers reloption, use that; otherwise
+	 * select a default number of workers.
+	 */
+	if (rel->rel_parallel_workers != -1)
+		parallel_workers = rel->rel_parallel_workers;
+	else
+	{
+		int			parallel_threshold;
+
+		/*
+		 * If this relation is too small to be worth a parallel scan, just
+		 * return without doing anything ... unless it's an inheritance child.
+		 * In that case, we want to generate a parallel path here anyway.  It
+		 * might not be worthwhile just for this relation, but when combined
+		 * with all of its inheritance siblings it may well pay off.
+		 */
+		if (pages < (BlockNumber) min_parallel_relation_size &&
+			rel->reloptkind == RELOPT_BASEREL)
+			return 0;
+
+		/*
+		 * Select the number of workers based on the log of the size of the
+		 * relation.  This probably needs to be a good deal more
+		 * sophisticated, but we need something here for now.  Note that the
+		 * upper limit of the min_parallel_relation_size GUC is chosen to
+		 * prevent overflow here.
+		 */
+		parallel_workers = 1;
+		parallel_threshold = Max(min_parallel_relation_size, 1);
+		while (pages >= (BlockNumber) (parallel_threshold * 3))
+		{
+			parallel_workers++;
+			parallel_threshold *= 3;
+			if (parallel_threshold > INT_MAX / 3)
+				break;			/* avoid overflow */
+		}
+	}
+
+	/*
+	 * In no case use more than max_parallel_workers_per_gather workers.
+	 */
+	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+
+	return parallel_workers;
+}
+
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 2a49639..c08997a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -161,7 +161,7 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
 static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
 static double relation_byte_size(double tuples, int width);
 static double page_size(double tuples, int width);
-
+static Cost adjust_cost_for_parallelism(Path *path, Cost cpu_run_cost);
 
 /*
  * clamp_row_est
@@ -237,44 +237,7 @@ cost_seqscan(Path *path, PlannerInfo *root,
 
 	/* Adjust costing for parallelism, if used. */
 	if (path->parallel_workers > 0)
-	{
-		double		parallel_divisor = path->parallel_workers;
-		double		leader_contribution;
-
-		/*
-		 * Early experience with parallel query suggests that when there is
-		 * only one worker, the leader often makes a very substantial
-		 * contribution to executing the parallel portion of the plan, but as
-		 * more workers are added, it does less and less, because it's busy
-		 * reading tuples from the workers and doing whatever non-parallel
-		 * post-processing is needed.  By the time we reach 4 workers, the
-		 * leader no longer makes a meaningful contribution.  Thus, for now,
-		 * estimate that the leader spends 30% of its time servicing each
-		 * worker, and the remainder executing the parallel plan.
-		 */
-		leader_contribution = 1.0 - (0.3 * path->parallel_workers);
-		if (leader_contribution > 0)
-			parallel_divisor += leader_contribution;
-
-		/*
-		 * In the case of a parallel plan, the row count needs to represent
-		 * the number of tuples processed per worker.  Otherwise, higher-level
-		 * plan nodes that appear below the gather will be costed incorrectly,
-		 * because they'll anticipate receiving more rows than any given copy
-		 * will actually get.
-		 */
-		path->rows = clamp_row_est(path->rows / parallel_divisor);
-
-		/* The CPU cost is divided among all the workers. */
-		cpu_run_cost /= parallel_divisor;
-
-		/*
-		 * It may be possible to amortize some of the I/O cost, but probably
-		 * not very much, because most operating systems already do aggressive
-		 * prefetching.  For now, we assume that the disk run cost can't be
-		 * amortized at all.
-		 */
-	}
+		cpu_run_cost = adjust_cost_for_parallelism(path, cpu_run_cost);
 
 	path->startup_cost = startup_cost;
 	path->total_cost = startup_cost + cpu_run_cost + disk_run_cost;
@@ -831,10 +794,10 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
 	Cost		indexTotalCost;
-	Selectivity indexSelectivity;
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -855,13 +818,12 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	if (!enable_bitmapscan)
 		startup_cost += disable_cost;
 
-	/*
-	 * Fetch total cost of obtaining the bitmap, as well as its total
-	 * selectivity.
-	 */
-	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+	pages_fetched = compute_bitmap_pages(root, baserel, bitmapqual,
+										loop_count, &indexTotalCost,
+										&tuples_fetched);
 
 	startup_cost += indexTotalCost;
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
 
 	/* Fetch estimated page costs for tablespace containing table. */
 	get_tablespace_page_costs(baserel->reltablespace,
@@ -869,41 +831,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 							  &spc_seq_page_cost);
 
 	/*
-	 * Estimate number of main-table pages fetched.
-	 */
-	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
-
-	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
-
-	if (loop_count > 1)
-	{
-		/*
-		 * For repeated bitmap scans, scale up the number of tuples fetched in
-		 * the Mackert and Lohman formula by the number of scans, so that we
-		 * estimate the number of pages fetched by all the scans. Then
-		 * pro-rate for one scan.
-		 */
-		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
-											baserel->pages,
-											get_indexpath_pages(bitmapqual),
-											root);
-		pages_fetched /= loop_count;
-	}
-	else
-	{
-		/*
-		 * For a single scan, the number of heap pages that need to be fetched
-		 * is the same as the Mackert and Lohman formula for the case T <= b
-		 * (ie, no re-reads needed).
-		 */
-		pages_fetched = (2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
-	}
-	if (pages_fetched >= T)
-		pages_fetched = T;
-	else
-		pages_fetched = ceil(pages_fetched);
-
-	/*
 	 * For small numbers of pages we should charge spc_random_page_cost
 	 * apiece, while if nearly all the table's pages are being read, it's more
 	 * appropriate to charge spc_seq_page_cost apiece.  The effect is
@@ -932,8 +859,13 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+		cpu_run_cost = adjust_cost_for_parallelism(path, cpu_run_cost);
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
@@ -944,6 +876,56 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 }
 
 /*
+ * adjust_cost_for_parallelism
+ *
+ * Adjust the cpu cost based on number of parallel workers, also update
+ * the number of rows processed at each worker.
+ */
+static Cost
+adjust_cost_for_parallelism(Path *path, Cost cpu_run_cost)
+{
+	double		parallel_divisor = path->parallel_workers;
+	double		leader_contribution;
+	Cost		cpu_cost = cpu_run_cost;
+
+	/*
+	 * Early experience with parallel query suggests that when there is
+	 * only one worker, the leader often makes a very substantial
+	 * contribution to executing the parallel portion of the plan, but as
+	 * more workers are added, it does less and less, because it's busy
+	 * reading tuples from the workers and doing whatever non-parallel
+	 * post-processing is needed.  By the time we reach 4 workers, the
+	 * leader no longer makes a meaningful contribution.  Thus, for now,
+	 * estimate that the leader spends 30% of its time servicing each
+	 * worker, and the remainder executing the parallel plan.
+	 */
+	leader_contribution = 1.0 - (0.3 * path->parallel_workers);
+	if (leader_contribution > 0)
+		parallel_divisor += leader_contribution;
+
+	/*
+	 * In the case of a parallel plan, the row count needs to represent
+	 * the number of tuples processed per worker.  Otherwise, higher-level
+	 * plan nodes that appear below the gather will be costed incorrectly,
+	 * because they'll anticipate receiving more rows than any given copy
+	 * will actually get.
+	 */
+	path->rows = clamp_row_est(path->rows / parallel_divisor);
+
+	/* The CPU cost is divided among all the workers. */
+	cpu_cost /= parallel_divisor;
+
+	/*
+	 * It may be possible to amortize some of the I/O cost, but probably
+	 * not very much, because most operating systems already do aggressive
+	 * prefetching.  For now, we assume that the disk run cost can't be
+	 * amortized at all.
+	 */
+
+	return cpu_cost;
+}
+
+/*
  * cost_bitmap_tree_node
  *		Extract cost and selectivity from a bitmap tree node (index/and/or)
  */
@@ -4765,3 +4747,69 @@ page_size(double tuples, int width)
 {
 	return ceil(relation_byte_size(tuples, width) / BLCKSZ);
 }
+
+/*
+ * compute_bitmap_pages
+ *
+ * compute number of pages fetched from heap in bitmap heap scan.
+ */
+double
+compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel, Path *bitmapqual,
+			int loop_count, Cost *cost, double *tuple)
+{
+	Cost			indexTotalCost;
+	Selectivity		indexSelectivity;
+	double			T;
+	double			pages_fetched;
+	double			tuples_fetched;
+
+	/*
+	 * Fetch total cost of obtaining the bitmap, as well as its total
+	 * selectivity.
+	 */
+	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+
+	/*
+	 * Estimate number of main-table pages fetched.
+	 */
+	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
+
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
+
+	if (loop_count > 1)
+	{
+		/*
+		 * For repeated bitmap scans, scale up the number of tuples fetched in
+		 * the Mackert and Lohman formula by the number of scans, so that we
+		 * estimate the number of pages fetched by all the scans. Then
+		 * pro-rate for one scan.
+		 */
+		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
+											baserel->pages,
+											get_indexpath_pages(bitmapqual),
+											root);
+		pages_fetched /= loop_count;
+	}
+	else
+	{
+		/*
+		 * For a single scan, the number of heap pages that need to be fetched
+		 * is the same as the Mackert and Lohman formula for the case T <= b
+		 * (ie, no re-reads needed).
+		 */
+		pages_fetched =
+				(2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
+	}
+
+	if (pages_fetched >= T)
+		pages_fetched = T;
+	else
+		pages_fetched = ceil(pages_fetched);
+
+	if (cost)
+		*cost = indexTotalCost;
+	if (tuple)
+		*tuple = tuples_fetched;
+
+	return pages_fetched;
+}
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 2952bfb..1a7bde0 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 47158f6..4ffcf87 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2803,7 +2803,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index abb7507..0801b6e 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -47,7 +47,6 @@ typedef enum
 
 static List *translate_sub_tlist(List *tlist, int relid);
 
-
 /*****************************************************************************
  *		MISC. PATH UTILITIES
  *****************************************************************************/
@@ -1071,7 +1070,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,11 +1080,10 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
-
 	pathnode->bitmapqual = bitmapqual;
 
 	cost_bitmap_heap_scan(&pathnode->path, root, rel,
@@ -3192,7 +3191,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 5112d6d..01a6e49 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3387,6 +3387,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 0d12bbb..7cf0a3c 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -23,7 +23,6 @@
 #include "utils/relcache.h"
 #include "utils/snapshot.h"
 
-
 /* "options" flag bits for heap_insert */
 #define HEAP_INSERT_SKIP_WAL	0x0001
 #define HEAP_INSERT_SKIP_FSM	0x0002
@@ -178,6 +177,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_bm_update_snapshot(HeapScanDesc scan, Snapshot	snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index 0ed9c78..8afecb7 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+								ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+								ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+										shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 2fadf76..7a1ff63 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1399,6 +1401,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t	mutex;			/* mutual exclusion for below three fields */
+	int 	spageptr;		/* next spages index */
+	int 	schunkptr;		/* next schunks index */
+	int 	schunkbit;		/* next bit to check in current schunk */
+} ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *	 						parallel bitmap heap scan.
+ *
+ *	 	PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *	 						first worker to see this state will become
+ *	 						leader and will create TIDbitmap. This will
+ *	 						also set the state to PBM_INPROGRESS.
+ *	 	PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *	 						so workers need to sleep until leader set the
+ *	 						state to PBM_FINISHED and wake us up.
+ *	 	PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *	 						can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *	 	relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid						relid;
+	ParallelIterator		tbmiterator;
+	ParallelIterator		prefetch_iterator;
+	slock_t					prefetch_mutex;
+	int						prefetch_pages;
+	int						prefetch_target;
+	slock_t					state_mutex;
+	ConditionVariable		cv;
+	PBMState				state;
+	Size					ptbm_offset;
+	char					phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1413,6 +1481,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1429,6 +1499,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1444,7 +1515,11 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		is_leader		   is_leader is set true, if this worker become
+ *						   leader for parallel bitmap heap scan.
+ *		parallel_bitmap	   shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1459,6 +1534,9 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	bool		is_leader;
+	ParallelBitmapInfo	*parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 9303299..e215256 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+}TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -58,9 +75,14 @@ extern void tbm_union(TIDBitmap *a, const TIDBitmap *b);
 extern void tbm_intersect(TIDBitmap *a, const TIDBitmap *b);
 
 extern bool tbm_is_empty(const TIDBitmap *tbm);
-
-extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
+extern TBMIterator * tbm_begin_iterate(TIDBitmap *tbm,
+							ParallelTIDBitmap *parallel_info);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_attach_to_pagetable(TIDBitmap *tbm, ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+									ParallelTIDBitmap *parallel_tbm);
+void tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..e1baacc 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -183,6 +183,8 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
 					   double cte_rows);
 extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
 extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern double compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel,
+				Path *bitmapqual, int loop_count, Cost *cost, double *tuple);
 
 /*
  * prototypes for clausesel.c
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..397d266 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 44abe83..3264f44 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 1c9bf13..0aa2ab5 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -784,6 +784,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#9Amit Kapila
amit.kapila16@gmail.com
In reply to: Dilip Kumar (#8)
Re: Parallel bitmap heap scan

On Wed, Oct 19, 2016 at 9:23 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Wed, Oct 19, 2016 at 12:39 PM, Andres Freund <andres@anarazel.de> wrote:

Try measuring with something more heavy on bitmap scan time
itself. E.g.
SELECT SUM(l_extendedprice) FROM lineitem WHERE (l_shipdate >= '1995-01-01'::date) AND (l_shipdate <= '1996-12-31'::date);
or similar. The tpch queries don't actually spend that much time in the
bitmapscan itself - the parallization of the rest of the query is what
matters...

Yeah, I agree.

I have tested with this query, with exact filter condition it was
taking parallel sequence scan, so I have modified the filter a bit and
tested.

Tested with all default configuration in my local machine. I think I
will generate more such test cases and do detail testing in my
performance machine.

Explain Analyze results:
---------------------------------
On Head:
------------
postgres=# explain analyze SELECT SUM(l_extendedprice) FROM lineitem
WHERE (l_shipdate >= '1995-01-01'::date) AND (l_shipdate <=
'1996-03-31'::date);

QUERY PLAN
-----------------------------------------------------------------------------------------------------------------------------------------------------------
Aggregate (cost=848805.90..848805.91 rows=1 width=32) (actual
time=12440.165..12440.166 rows=1 loops=1)
-> Bitmap Heap Scan on lineitem (cost=143372.40..834833.25
rows=5589057 width=8) (actual time=1106.217..11183.722 rows=5678841
loops=1)
Recheck Cond: ((l_shipdate >= '1995-01-01'::date) AND
(l_shipdate <= '1996-03-31'::date))
Rows Removed by Index Recheck: 20678739
Heap Blocks: exact=51196 lossy=528664
-> Bitmap Index Scan on idx_lineitem_shipdate
(cost=0.00..141975.13 rows=5589057 width=0) (actual
time=1093.376..1093.376 rows=5678841 loops=1)
Index Cond: ((l_shipdate >= '1995-01-01'::date) AND
(l_shipdate <= '1996-03-31'::date))
Planning time: 0.185 ms
Execution time: 12440.819 ms
(9 rows)

After Patch:
---------------
postgres=# explain analyze SELECT SUM(l_extendedprice) FROM lineitem
WHERE (l_shipdate >= '1995-01-01'::date) AND (l_shipdate <=
'1996-03-31'::date);

QUERY PLAN

--------------------------------------------------------------------------------------------------------------------------------------------------------------
---------
Finalize Aggregate (cost=792751.16..792751.17 rows=1 width=32)
(actual time=6660.157..6660.157 rows=1 loops=1)
-> Gather (cost=792750.94..792751.15 rows=2 width=32) (actual
time=6659.378..6660.117 rows=3 loops=1)
Workers Planned: 2
Workers Launched: 2
-> Partial Aggregate (cost=791750.94..791750.95 rows=1
width=32) (actual time=6655.941..6655.941 rows=1 loops=3)
-> Parallel Bitmap Heap Scan on lineitem
(cost=143372.40..785929.00 rows=2328774 width=8) (actual
time=1980.797..6204.232 rows=1892947 loops=
3)
Recheck Cond: ((l_shipdate >= '1995-01-01'::date)
AND (l_shipdate <= '1996-03-31'::date))
Rows Removed by Index Recheck: 6930269
Heap Blocks: exact=17090 lossy=176443
-> Bitmap Index Scan on idx_lineitem_shipdate
(cost=0.00..141975.13 rows=5589057 width=0) (actual
time=1933.454..1933.454 rows=5678841
loops=1)
Index Cond: ((l_shipdate >=
'1995-01-01'::date) AND (l_shipdate <= '1996-03-31'::date))
Planning time: 0.207 ms
Execution time: 6669.195 ms
(13 rows)

Summary:
-> With patch overall execution is 2 time faster compared to head.
-> Bitmap creation with patch is bit slower compared to head and thats
because of DHT vs efficient hash table.

I think here the impact of slowness due to Bitmap Index Scan is not
much visible, as the time it takes as compare to overall time is less.
However, I think there is an advantage of using DHT as that will allow
us to build the hash table by multiple workers using parallel index
scan in future.

--
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#10Amit Khandekar
amitdkhan.pg@gmail.com
In reply to: Amit Kapila (#9)
Re: Parallel bitmap heap scan

On 19 October 2016 at 09:47, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Tue, Oct 18, 2016 at 1:45 AM, Andres Freund <andres@anarazel.de> wrote:

I don't quite understand why the bitmap has to be parallel at all. As
far as I understand your approach as described here, the only thing that
needs to be shared are the iteration arrays. Since they never need to
be resized and such, it seems to make a lot more sense to just add an
API to share those, instead of the whole underlying hash.

You are right that we only share iteration arrays. But only point is
that each entry of iteration array is just a pointer to hash entry.
So either we need to build hash in shared memory (my current approach)
or we need to copy each hash element at shared location (I think this
is going to be expensive).

While the discussion is going on regarding implementation for creating
shared tidbitmap, meanwhile I am starting with review of the bitmap
heap scan part, i.e. nodeBitmapHeapscan.c, since this looks mostly
independent of tidbitmap implementation.

Brief design idea:
-----------------------
#1. Shared TIDBitmap creation and initialization
First worker to see the state as parallel bitmap info as PBM_INITIAL
become leader and set the state to PBM_INPROGRESS All other workers
see the state as PBM_INPROGRESS will wait for leader to complete the
TIDBitmap.

#2 At this level TIDBitmap is ready and all workers are awake.

As far as correctness is concerned, the logic where the first worker
becomes leader while others synchronously wait, looks good. Workers
get allocated right from the beginning even though they would stay
idle for some percentage of time (5-20% ?) , but I guess there is
nothing we can do about it with the current parallel query
infrastructure.

In pbms_is_leader() , I didn't clearly understand the significance of
the for-loop. If it is a worker, it can call
ConditionVariablePrepareToSleep() followed by
ConditionVariableSleep(). Once it comes out of
ConditionVariableSleep(), isn't it guaranteed that the leader has
finished the bitmap ? If yes, then it looks like it is not necessary
to again iterate and go back through the pbminfo->state checking.
Also, with this, variable queuedSelf also might not be needed. But I
might be missing something here. Not sure what happens if worker calls
ConditionVariable[Prepare]Sleep() but leader has already called
ConditionVariableBroadcast(). Does the for loop have something to do
with this ? But this can happen even with the current for-loop, it
seems.

#3. Bitmap processing (Iterate and process the pages).
In this phase each worker will iterate over page and chunk array and
select heap pages one by one. If prefetch is enable then there will be
two iterator. Since multiple worker are iterating over same page and
chunk array we need to have a shared iterator, so we grab a spin lock
and iterate within a lock, so that each worker get and different page
to process.

tbm_iterate() call under SpinLock :
For parallel tbm iteration, tbm_iterate() is called while SpinLock is
held. Generally we try to keep code inside Spinlock call limited to a
few lines, and that too without occurrence of a function call.
Although tbm_iterate() code itself looks safe under a spinlock, I was
checking if we can squeeze SpinlockAcquire() and SpinLockRelease()
closer to each other. One thought is : in tbm_iterate(), acquire the
SpinLock before the while loop that iterates over lossy chunks. Then,
if both chunk and per-page data remain, release spinlock just before
returning (the first return stmt). And then just before scanning
bitmap of an exact page, i.e. just after "if (iterator->spageptr <
tbm->npages)", save the page handle, increment iterator->spageptr,
release Spinlock, and then use the saved page handle to iterate over
the page bitmap.

prefetch_pages() call under Spinlock :
Here again, prefetch_pages() is called while pbminfo->prefetch_mutex
Spinlock is held. Effectively, heavy functions like PrefetchBuffer()
would get called while under the Spinlock. These can even ereport().
One option is to use mutex lock for this purpose. But I think that
would slow things down. Moreover, the complete set of prefetch pages
would be scanned by a single worker, and others might wait for this
one. Instead, what I am thinking is: grab the pbminfo->prefetch_mutex
Spinlock only while incrementing pbminfo->prefetch_pages. The rest
part viz : iterating over the prefetch pages, and doing the
PrefetchBuffer() need not be synchronised using this
pgbinfo->prefetch_mutex Spinlock. pbms_parallel_iterate() already has
its own iterator spinlock. Only thing is, workers may not do the
actual PrefetchBuffer() sequentially. One of them might shoot ahead
and prefetch 3-4 pages while the other is lagging with the
sequentially lesser page number; but I believe this is fine, as long
as they all prefetch all the required blocks.

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#11Dilip Kumar
dilipbalaut@gmail.com
In reply to: Amit Khandekar (#10)
Re: Parallel bitmap heap scan

On Fri, Nov 18, 2016 at 9:59 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:

Thanks for the review..

In pbms_is_leader() , I didn't clearly understand the significance of
the for-loop. If it is a worker, it can call
ConditionVariablePrepareToSleep() followed by
ConditionVariableSleep(). Once it comes out of
ConditionVariableSleep(), isn't it guaranteed that the leader has
finished the bitmap ? If yes, then it looks like it is not necessary
to again iterate and go back through the pbminfo->state checking.
Also, with this, variable queuedSelf also might not be needed. But I
might be missing something here.

I have taken this logic from example posted on conditional variable thread[1]/messages/by-id/CA+Tgmoaj2aPti0yho7FeEf2qt-JgQPRWb0gci_o1Hfr=C56Xng@mail.gmail.com

for (;;)
{
ConditionVariablePrepareToSleep(cv);
if (condition for which we are waiting is satisfied)
break;
ConditionVariableSleep();
}
ConditionVariableCancelSleep();

[1]: /messages/by-id/CA+Tgmoaj2aPti0yho7FeEf2qt-JgQPRWb0gci_o1Hfr=C56Xng@mail.gmail.com

So it appears to me even if we come out of ConditionVariableSleep();
we need to verify our condition and then only break.

Not sure what happens if worker calls

ConditionVariable[Prepare]Sleep() but leader has already called
ConditionVariableBroadcast(). Does the for loop have something to do
with this ? But this can happen even with the current for-loop, it
seems.

If leader has already called ConditionVariableBroadcast, but after
ConditionVariablePrepareToSleep we will check the condition again
before calling ConditionVariableSleep. And condition check is under
SpinLockAcquire(&pbminfo->state_mutex);

However I think there is one problem in my code (I think you might be
pointing same), that after ConditionVariablePrepareToSleep, if
pbminfo->state is already PBM_FINISHED, I am not resetting needWait to
false, and which can lead to the problem.

#3. Bitmap processing (Iterate and process the pages).
In this phase each worker will iterate over page and chunk array and
select heap pages one by one. If prefetch is enable then there will be
two iterator. Since multiple worker are iterating over same page and
chunk array we need to have a shared iterator, so we grab a spin lock
and iterate within a lock, so that each worker get and different page
to process.

tbm_iterate() call under SpinLock :
For parallel tbm iteration, tbm_iterate() is called while SpinLock is
held. Generally we try to keep code inside Spinlock call limited to a
few lines, and that too without occurrence of a function call.
Although tbm_iterate() code itself looks safe under a spinlock, I was
checking if we can squeeze SpinlockAcquire() and SpinLockRelease()
closer to each other. One thought is : in tbm_iterate(), acquire the
SpinLock before the while loop that iterates over lossy chunks. Then,
if both chunk and per-page data remain, release spinlock just before
returning (the first return stmt). And then just before scanning
bitmap of an exact page, i.e. just after "if (iterator->spageptr <
tbm->npages)", save the page handle, increment iterator->spageptr,
release Spinlock, and then use the saved page handle to iterate over
the page bitmap.

Main reason to keep Spin lock out of this function to avoid changes
inside this function, and also this function takes local iterator as
input which don't have spin lock reference to it. But that can be
changed, we can pass shared iterator to it.

I will think about this logic and try to update in next version.

prefetch_pages() call under Spinlock :
Here again, prefetch_pages() is called while pbminfo->prefetch_mutex
Spinlock is held. Effectively, heavy functions like PrefetchBuffer()
would get called while under the Spinlock. These can even ereport().
One option is to use mutex lock for this purpose. But I think that
would slow things down. Moreover, the complete set of prefetch pages
would be scanned by a single worker, and others might wait for this
one. Instead, what I am thinking is: grab the pbminfo->prefetch_mutex
Spinlock only while incrementing pbminfo->prefetch_pages. The rest
part viz : iterating over the prefetch pages, and doing the
PrefetchBuffer() need not be synchronised using this
pgbinfo->prefetch_mutex Spinlock. pbms_parallel_iterate() already has
its own iterator spinlock. Only thing is, workers may not do the
actual PrefetchBuffer() sequentially. One of them might shoot ahead
and prefetch 3-4 pages while the other is lagging with the
sequentially lesser page number; but I believe this is fine, as long
as they all prefetch all the required blocks.

I agree with your point, will try to fix this as well.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#12Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#8)
Re: Parallel bitmap heap scan

On Wed, Oct 19, 2016 at 11:53 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

I found one defect in v2 patch, that I induced during last rebasing.
That is fixed in v3.

So, I had a brief look at this tonight. This is not a full review,
but just some things I noticed:

+ * Update snpashot info in heap scan descriptor.

Typo. Also, why should we have a function for this at all? And if we
do have a function for this, why should it have "bm" in the name when
it's stored in heapam.c?

+ *    [PARALLEL BITMAP HEAP SCAN ALGORITHM]
+ *
+ *    #1. Shared TIDBitmap creation and initialization
+ *        a) First worker to see the state as parallel bitmap info as
+ *        PBM_INITIAL become leader and set the state to PBM_INPROGRESS
+ *        All other workers see the state as PBM_INPROGRESS will wait for
+ *        leader to complete the TIDBitmap.
+ *
+ *        Leader Worker Processing:
+ *        (Leader is responsible for creating shared TIDBitmap and create
+ *        shared page and chunk array from TIDBitmap.)
+ *            1) Create TIDBitmap using DHT.
+ *            2) Begin Iterate: convert hash table into shared page and chunk
+ *            array.
+ *            3) Restore local TIDBitmap variable information into
+ *            ParallelBitmapInfo so that other worker can see those.
+ *            4) set state to PBM_FINISHED.
+ *            5) Wake up other workers.
+ *
+ *        Other Worker Processing:
+ *            1) Wait until leader create shared TIDBitmap and shared page
+ *            and chunk array.
+ *            2) Attach to shared page table, copy TIDBitmap from
+ *            ParallelBitmapInfo to local TIDBitmap, we copy this to local
+ *            TIDBitmap so that next level processing can read information
+ *            same as in non parallel case and we can avoid extra changes
+ *            in code.
+ *
+ *    # At this level TIDBitmap is ready and all workers are awake #
+ *
+ *    #2. Bitmap processing (Iterate and process the pages).
+ *        . In this phase each worker will iterate over page and
chunk array and
+ *        select heap pages one by one. If prefetch is enable then there will
+ *        be two iterator.
+ *        . Since multiple worker are iterating over same page and chunk array
+ *        we need to have a shared iterator, so we grab a spin lock and iterate
+ *        within a lock.

The formatting of this comment is completely haphazard. "Leader
worker" is not a term that has any meaning. A given backend involved
in a parallel query is either a leader or a worker, not both.

+    /* reset parallel bitmap scan info, if present */
+    if (node->parallel_bitmap)
+    {
+        ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+        pbminfo->state = PBM_INITIAL;
+        pbminfo->tbmiterator.schunkbit = 0;
+        pbminfo->tbmiterator.spageptr = 0;
+        pbminfo->tbmiterator.schunkptr = 0;
+        pbminfo->prefetch_iterator.schunkbit = 0;
+        pbminfo->prefetch_iterator.spageptr = 0;
+        pbminfo->prefetch_iterator.schunkptr = 0;
+        pbminfo->prefetch_pages = 0;
+        pbminfo->prefetch_target = -1;
+    }

This is obviously not going to work in the face of concurrent
activity. When we did Parallel Seq Scan, we didn't make any changes
to the rescan code at all. I think we are assuming that there's no
way to cause a rescan of the driving table of a parallel query; if
that's wrong, we'll need some fix, but this isn't it. I would just
leave this out.

+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)

I think you should see if you can use Thomas Munro's barrier stuff for
this instead.

+    SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+
+    shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+    node->parallel_bitmap = pbminfo;
+    snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+
+    heap_bm_update_snapshot(node->ss.ss_currentScanDesc, snapshot);

This doesn't make any sense. You serialize the snapshot from the
estate, then restore it, then shove it into the scan descriptor. But
presumably that's already the snapshot the scan descriptor is using.
The workers need to do this, perhaps, but not the leader!

+ dht_parameters params = {0};

Not PostgreSQL style.

+#define TBM_IS_SHARED(tbm) (tbm)->shared

Seems pointless.

+ bool shared; /* need to build shared tbm if set*/

Style.

+ params.tranche_id = LWLockNewTrancheId();

You absolutely, positively cannot burn through tranche IDs like this.

+    if (tbm->shared_pagetable)
+        dht_detach(tbm->shared_pagetable);

Hmm, would we leak references if we errored out?

@@ -47,7 +47,6 @@ typedef enum

static List *translate_sub_tlist(List *tlist, int relid);

-
/*****************************************************************************
* MISC. PATH UTILITIES
*****************************************************************************/

Useless whitespace change.

@@ -23,7 +23,6 @@
#include "utils/relcache.h"
#include "utils/snapshot.h"

-
/* "options" flag bits for heap_insert */
#define HEAP_INSERT_SKIP_WAL 0x0001
#define HEAP_INSERT_SKIP_FSM 0x0002

Useless whitespace change.

WAIT_EVENT_MQ_RECEIVE,
WAIT_EVENT_MQ_SEND,
WAIT_EVENT_PARALLEL_FINISH,
+ WAIT_EVENT_PARALLEL_BITMAP_SCAN,
WAIT_EVENT_SAFE_SNAPSHOT,
WAIT_EVENT_SYNC_REP

Missing a documentation update.

In general, the amount of change in nodeBitmapHeapScan.c seems larger
than I would have expected. My copy of that file has 655 lines; this
patch adds 544 additional lines. I think/hope that some of that can
be simplified away.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#13Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#12)
Re: Parallel bitmap heap scan

On Wed, Nov 23, 2016 at 7:24 AM, Robert Haas <robertmhaas@gmail.com> wrote:

So, I had a brief look at this tonight. This is not a full review,
but just some things I noticed:

Thanks for the review..

+ * Update snpashot info in heap scan descriptor.

Typo. Also, why should we have a function for this at all? And if we
do have a function for this, why should it have "bm" in the name when
it's stored in heapam.c?

We are updating snapshot in HeapScanDesc, that's the reason I am using
function and kept in heapam.c file like other function
heap_beginscan_bm.
But I think we can change it's name bm is not required in this, this
function don't do anything specific for bm. I will change this.

+ *    [PARALLEL BITMAP HEAP SCAN ALGORITHM]
+ *
+ *    #1. Shared TIDBitmap creation and initialization
+ *        a) First worker to see the state as parallel bitmap info as
+ *        PBM_INITIAL become leader and set the state to PBM_INPROGRESS
+ *        All other workers see the state as PBM_INPROGRESS will wait for
+ *        leader to complete the TIDBitmap.
+ *
+ *        Leader Worker Processing:
+ *        (Leader is responsible for creating shared TIDBitmap and create
+ *        shared page and chunk array from TIDBitmap.)
+ *            1) Create TIDBitmap using DHT.
+ *            2) Begin Iterate: convert hash table into shared page and chunk
+ *            array.
+ *            3) Restore local TIDBitmap variable information into
+ *            ParallelBitmapInfo so that other worker can see those.
+ *            4) set state to PBM_FINISHED.
+ *            5) Wake up other workers.
+ *
+ *        Other Worker Processing:
+ *            1) Wait until leader create shared TIDBitmap and shared page
+ *            and chunk array.
+ *            2) Attach to shared page table, copy TIDBitmap from
+ *            ParallelBitmapInfo to local TIDBitmap, we copy this to local
+ *            TIDBitmap so that next level processing can read information
+ *            same as in non parallel case and we can avoid extra changes
+ *            in code.
+ *
+ *    # At this level TIDBitmap is ready and all workers are awake #
+ *
+ *    #2. Bitmap processing (Iterate and process the pages).
+ *        . In this phase each worker will iterate over page and
chunk array and
+ *        select heap pages one by one. If prefetch is enable then there will
+ *        be two iterator.
+ *        . Since multiple worker are iterating over same page and chunk array
+ *        we need to have a shared iterator, so we grab a spin lock and iterate
+ *        within a lock.

The formatting of this comment is completely haphazard.

I will fix this..

"Leader

worker" is not a term that has any meaning. A given backend involved
in a parallel query is either a leader or a worker, not both.

I agree this is confusing, but we can't call it directly a leader
because IMHO we meant by a leader who, actually spawns all worker and
gather the results. But here I meant by "leader worker" is the worker
which takes responsibility of building tidbitmap, which can be any
worker. So I named it "leader worker". Let me think what we can call
it.

+    /* reset parallel bitmap scan info, if present */
+    if (node->parallel_bitmap)
+    {
+        ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+        pbminfo->state = PBM_INITIAL;
+        pbminfo->tbmiterator.schunkbit = 0;
+        pbminfo->tbmiterator.spageptr = 0;
+        pbminfo->tbmiterator.schunkptr = 0;
+        pbminfo->prefetch_iterator.schunkbit = 0;
+        pbminfo->prefetch_iterator.spageptr = 0;
+        pbminfo->prefetch_iterator.schunkptr = 0;
+        pbminfo->prefetch_pages = 0;
+        pbminfo->prefetch_target = -1;
+    }

This is obviously not going to work in the face of concurrent
activity. When we did Parallel Seq Scan, we didn't make any changes
to the rescan code at all. I think we are assuming that there's no
way to cause a rescan of the driving table of a parallel query; if
that's wrong, we'll need some fix, but this isn't it. I would just
leave this out.

In heap_rescan function we have similar change..

if (scan->rs_parallel != NULL)
{
parallel_scan = scan->rs_parallel;
SpinLockAcquire(&parallel_scan->phs_mutex);
parallel_scan->phs_cblock = parallel_scan->phs_startblock;
SpinLockRelease(&parallel_scan->phs_mutex);
}

And this is not for driving table, it's required for the case when
gather is as inner node for JOIN.
consider below example. I know it's a bad plan but we can produce this plan :)

Outer Node Inner Node
SeqScan(t1) NLJ (Gather -> Parallel SeqScan (t2)).

Reason for doing same is that, during ExecReScanGather we don't
recreate new DSM instead of that we just reinitialise the DSM
(ExecParallelReinitialize).

+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)

I think you should see if you can use Thomas Munro's barrier stuff for
this instead.

Okay, I will think over it.

+    SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+
+    shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+    node->parallel_bitmap = pbminfo;
+    snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+
+    heap_bm_update_snapshot(node->ss.ss_currentScanDesc, snapshot);

This doesn't make any sense. You serialize the snapshot from the
estate, then restore it, then shove it into the scan descriptor. But
presumably that's already the snapshot the scan descriptor is using.
The workers need to do this, perhaps, but not the leader!

This is wrong, will fix this.

+ dht_parameters params = {0};

Not PostgreSQL style.

I will fix..

+#define TBM_IS_SHARED(tbm) (tbm)->shared

Seems pointless.

Ok..

+ bool shared; /* need to build shared tbm if set*/

Style.

Ok.

+ params.tranche_id = LWLockNewTrancheId();

You absolutely, positively cannot burn through tranche IDs like this.

+    if (tbm->shared_pagetable)
+        dht_detach(tbm->shared_pagetable);

Hmm, would we leak references if we errored out?

I will check on this part. Anyway, In my next version I am working on
making my patch independent of DHT, so this part will be removed.

@@ -47,7 +47,6 @@ typedef enum

static List *translate_sub_tlist(List *tlist, int relid);

-
/*****************************************************************************
* MISC. PATH UTILITIES
*****************************************************************************/

Useless whitespace change.

@@ -23,7 +23,6 @@
#include "utils/relcache.h"
#include "utils/snapshot.h"

-
/* "options" flag bits for heap_insert */
#define HEAP_INSERT_SKIP_WAL 0x0001
#define HEAP_INSERT_SKIP_FSM 0x0002

Useless whitespace change.

WAIT_EVENT_MQ_RECEIVE,
WAIT_EVENT_MQ_SEND,
WAIT_EVENT_PARALLEL_FINISH,
+ WAIT_EVENT_PARALLEL_BITMAP_SCAN,
WAIT_EVENT_SAFE_SNAPSHOT,
WAIT_EVENT_SYNC_REP

Missing a documentation update.

I will fix these, in next version.

In general, the amount of change in nodeBitmapHeapScan.c seems larger
than I would have expected. My copy of that file has 655 lines; this
patch adds 544 additional lines. I think/hope that some of that can
be simplified away.

I will work on this.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#14Dilip Kumar
dilipbalaut@gmail.com
In reply to: Dilip Kumar (#13)
Re: Parallel bitmap heap scan

On Wed, Nov 23, 2016 at 12:31 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

I tried to address these comments in my new version, All comments are
fixed except below

+ *
+ *    #2. Bitmap processing (Iterate and process the pages).
+ *        . In this phase each worker will iterate over page and
chunk array and
+ *        select heap pages one by one. If prefetch is enable then there will
+ *        be two iterator.
+ *        . Since multiple worker are iterating over same page and chunk array
+ *        we need to have a shared iterator, so we grab a spin lock and iterate
+ *        within a lock.

The formatting of this comment is completely haphazard.

I will fix this..

I have changed the formatting, and also moved the algorithm
description inside function body.
I am not sure does this meet your expectation or we should change it further ?

+    /* reset parallel bitmap scan info, if present */
+    if (node->parallel_bitmap)
+    {
+        ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+        pbminfo->state = PBM_INITIAL;
+        pbminfo->tbmiterator.schunkbit = 0;
+        pbminfo->tbmiterator.spageptr = 0;
+        pbminfo->tbmiterator.schunkptr = 0;
+        pbminfo->prefetch_iterator.schunkbit = 0;
+        pbminfo->prefetch_iterator.spageptr = 0;
+        pbminfo->prefetch_iterator.schunkptr = 0;
+        pbminfo->prefetch_pages = 0;
+        pbminfo->prefetch_target = -1;
+    }

This is obviously not going to work in the face of concurrent
activity. When we did Parallel Seq Scan, we didn't make any changes
to the rescan code at all. I think we are assuming that there's no
way to cause a rescan of the driving table of a parallel query; if
that's wrong, we'll need some fix, but this isn't it. I would just
leave this out.

In heap_rescan function we have similar change..

if (scan->rs_parallel != NULL)
{
parallel_scan = scan->rs_parallel;
SpinLockAcquire(&parallel_scan->phs_mutex);
parallel_scan->phs_cblock = parallel_scan->phs_startblock;
SpinLockRelease(&parallel_scan->phs_mutex);
}

And this is not for driving table, it's required for the case when
gather is as inner node for JOIN.
consider below example. I know it's a bad plan but we can produce this plan :)

Outer Node Inner Node
SeqScan(t1) NLJ (Gather -> Parallel SeqScan (t2)).

Reason for doing same is that, during ExecReScanGather we don't
recreate new DSM instead of that we just reinitialise the DSM
(ExecParallelReinitialize).

This is not fixed, reason is already explained.

+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)

I think you should see if you can use Thomas Munro's barrier stuff for
this instead.

Okay, I will think over it.

IMHO, barrier is used when multiple worker are doing some work
together in phase1, and before moving to next phase all have to
complete phase1, so we put barrier, so that before starting next phase
all cross the barrier.

But here case is different, only one worker need to finish the phase1,
and as soon as it's over all can start phase2. But we don't have
requirement that all worker should cross certain barrier. In this case
even though some worker did not start, other worker can do their work.

In general, the amount of change in nodeBitmapHeapScan.c seems larger
than I would have expected. My copy of that file has 655 lines; this
patch adds 544 additional lines. I think/hope that some of that can
be simplified away.

I will work on this.

I have removed some function which was actually not required, and code
can be merged in main function. Almost reduced by 100 lines.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#15Dilip Kumar
dilipbalaut@gmail.com
In reply to: Dilip Kumar (#11)
Re: Parallel bitmap heap scan

On Tue, Nov 22, 2016 at 9:05 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Fri, Nov 18, 2016 at 9:59 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:

Thanks for the review..

I have worked on these comments..

In pbms_is_leader() , I didn't clearly understand the significance of
the for-loop. If it is a worker, it can call
ConditionVariablePrepareToSleep() followed by
ConditionVariableSleep(). Once it comes out of
ConditionVariableSleep(), isn't it guaranteed that the leader has
finished the bitmap ? If yes, then it looks like it is not necessary
to again iterate and go back through the pbminfo->state checking.
Also, with this, variable queuedSelf also might not be needed. But I
might be missing something here.

I have taken this logic from example posted on conditional variable thread[1]

for (;;)
{
ConditionVariablePrepareToSleep(cv);
if (condition for which we are waiting is satisfied)
break;
ConditionVariableSleep();
}
ConditionVariableCancelSleep();

[1] /messages/by-id/CA+Tgmoaj2aPti0yho7FeEf2qt-JgQPRWb0gci_o1Hfr=C56Xng@mail.gmail.com

So it appears to me even if we come out of ConditionVariableSleep();
we need to verify our condition and then only break.

Not sure what happens if worker calls

ConditionVariable[Prepare]Sleep() but leader has already called
ConditionVariableBroadcast(). Does the for loop have something to do
with this ? But this can happen even with the current for-loop, it
seems.

If leader has already called ConditionVariableBroadcast, but after
ConditionVariablePrepareToSleep we will check the condition again
before calling ConditionVariableSleep. And condition check is under
SpinLockAcquire(&pbminfo->state_mutex);

However I think there is one problem in my code (I think you might be
pointing same), that after ConditionVariablePrepareToSleep, if
pbminfo->state is already PBM_FINISHED, I am not resetting needWait to
false, and which can lead to the problem.

I have fixed the defect what I have mentioned above,

#3. Bitmap processing (Iterate and process the pages).
In this phase each worker will iterate over page and chunk array and
select heap pages one by one. If prefetch is enable then there will be
two iterator. Since multiple worker are iterating over same page and
chunk array we need to have a shared iterator, so we grab a spin lock
and iterate within a lock, so that each worker get and different page
to process.

tbm_iterate() call under SpinLock :
For parallel tbm iteration, tbm_iterate() is called while SpinLock is
held. Generally we try to keep code inside Spinlock call limited to a
few lines, and that too without occurrence of a function call.
Although tbm_iterate() code itself looks safe under a spinlock, I was
checking if we can squeeze SpinlockAcquire() and SpinLockRelease()
closer to each other. One thought is : in tbm_iterate(), acquire the
SpinLock before the while loop that iterates over lossy chunks. Then,
if both chunk and per-page data remain, release spinlock just before
returning (the first return stmt). And then just before scanning
bitmap of an exact page, i.e. just after "if (iterator->spageptr <
tbm->npages)", save the page handle, increment iterator->spageptr,
release Spinlock, and then use the saved page handle to iterate over
the page bitmap.

Main reason to keep Spin lock out of this function to avoid changes
inside this function, and also this function takes local iterator as
input which don't have spin lock reference to it. But that can be
changed, we can pass shared iterator to it.

I will think about this logic and try to update in next version.

Still this issue is not addressed.
Logic inside tbm_iterate is using same variable, like spageptr,
multiple places. IMHO this complete logic needs to be done under one
spin lock.

prefetch_pages() call under Spinlock :
Here again, prefetch_pages() is called while pbminfo->prefetch_mutex
Spinlock is held. Effectively, heavy functions like PrefetchBuffer()
would get called while under the Spinlock. These can even ereport().
One option is to use mutex lock for this purpose. But I think that
would slow things down. Moreover, the complete set of prefetch pages
would be scanned by a single worker, and others might wait for this
one. Instead, what I am thinking is: grab the pbminfo->prefetch_mutex
Spinlock only while incrementing pbminfo->prefetch_pages. The rest
part viz : iterating over the prefetch pages, and doing the
PrefetchBuffer() need not be synchronised using this
pgbinfo->prefetch_mutex Spinlock. pbms_parallel_iterate() already has
its own iterator spinlock. Only thing is, workers may not do the
actual PrefetchBuffer() sequentially. One of them might shoot ahead
and prefetch 3-4 pages while the other is lagging with the
sequentially lesser page number; but I believe this is fine, as long
as they all prefetch all the required blocks.

I agree with your point, will try to fix this as well.

I have fixed this part.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#16Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#14)
Re: Parallel bitmap heap scan

On Sat, Nov 26, 2016 at 7:40 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

IMHO, barrier is used when multiple worker are doing some work
together in phase1, and before moving to next phase all have to
complete phase1, so we put barrier, so that before starting next phase
all cross the barrier.

But here case is different, only one worker need to finish the phase1,
and as soon as it's over all can start phase2. But we don't have
requirement that all worker should cross certain barrier. In this case
even though some worker did not start, other worker can do their work.

I think the Barrier stuff has a process for choosing one worker to
conduct a particular phase. So it seems like if the Barrier API is
well-designed, you should be able to use it to decide who will conduct
the index scan, and then when that's done everyone can proceed to
scanning the heap. If that can't work for some reason, Thomas should
probably adjust his API so it does. He's presenting that as a
generally-useful primitive...

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#17Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#16)
Re: Parallel bitmap heap scan

On Sun, Nov 27, 2016 at 3:15 AM, Robert Haas <robertmhaas@gmail.com> wrote:

I think the Barrier stuff has a process for choosing one worker to
conduct a particular phase. So it seems like if the Barrier API is
well-designed, you should be able to use it to decide who will conduct
the index scan, and then when that's done everyone can proceed to
scanning the heap. If that can't work for some reason, Thomas should
probably adjust his API so it does. He's presenting that as a
generally-useful primitive...

If I understand the barrier API correctly, It has two Part.
1. BarrierInit 2. BarrierWait.

1. In BarrierInit we defined that, how many worker(lets say nworkers)
should cross the barrier, before we are allowed to cross the
BarriedWait.

2. BarrierWait, will actually make calling process wait until
BarrierWait is not called for nworkers times.

So I am not very clear, If we call BarrierInit with nworkers=1, then
first question is when should we call BarrierWait, because as soon as
we call BarrierWait count will reach 1, and now everyone is allowed to
proceed. so obviously It should be called once the Bitmap is Ready.

Second question is, if it's called only after Bitmap is ready, then
what about other process, how they are supposed to wait until bitmap
is not ready. If they wait using BarrierWait, it again make the count
1 and everyone is allowed to proceed. Which doesn't seems correct.

Correct me if I am missing something ?

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#18Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Dilip Kumar (#17)
Re: Parallel bitmap heap scan

On Sun, Nov 27, 2016 at 3:34 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Sun, Nov 27, 2016 at 3:15 AM, Robert Haas <robertmhaas@gmail.com> wrote:

I think the Barrier stuff has a process for choosing one worker to
conduct a particular phase. So it seems like if the Barrier API is
well-designed, you should be able to use it to decide who will conduct
the index scan, and then when that's done everyone can proceed to
scanning the heap. If that can't work for some reason, Thomas should
probably adjust his API so it does. He's presenting that as a
generally-useful primitive...

If I understand the barrier API correctly, It has two Part.
1. BarrierInit 2. BarrierWait.

1. In BarrierInit we defined that, how many worker(lets say nworkers)
should cross the barrier, before we are allowed to cross the
BarriedWait.

2. BarrierWait, will actually make calling process wait until
BarrierWait is not called for nworkers times.

So I am not very clear, If we call BarrierInit with nworkers=1, then
first question is when should we call BarrierWait, because as soon as
we call BarrierWait count will reach 1, and now everyone is allowed to
proceed. so obviously It should be called once the Bitmap is Ready.

Second question is, if it's called only after Bitmap is ready, then
what about other process, how they are supposed to wait until bitmap
is not ready. If they wait using BarrierWait, it again make the count
1 and everyone is allowed to proceed. Which doesn't seems correct.

Correct me if I am missing something ?

I'm not sure if it's the right tool for this job or not and haven't
studied this patch yet. I will. But here is one way to use barrier.c
for something like this, based on the description above. It's
slightly more complicated than you said because you don't know whether
the leader is going to participate or how many of the planned workers
will actually be able to start up, so there would be no way to provide
that 'participants' argument to BarrierInit and any given participant
might already have missed some of the 'BarrierWait' calls by the time
it starts running, so merely calling BarrierWait the right number of
times isn't enough to stay in sync. So instead you do this:

#define PBS_PHASE INIT 0
#define PBS_PHASE_BUILDING 1
#define PBS_PHASE_SCANNING 2

Initialise the barrier with BarrierInit(&something->barrier, 0), which
says that you don't know how many participants there will be.

Somewhere in each participant you need to do this exactly once:

BarrierAttach(&something->barrier);

I think you need to track whether you've called that yet and do so on
demand in your ExecBitmapHeap function. You can't just do it in
ExecBitmapHeapInitializeWorker because the leader needs to do it too,
but *only* if it runs the plan. Then you need something like this:

switch (BarrierPhase(&something->barrier)
{
case PBS_PHASE_INIT:
if (BarrierWait(&something->barrier, WAIT_EVENT_PBS_PHASE_INIT))
{
/* Serial phase that will run in only one chosen participant. */
build_the_bitmap();
}
/* Fall through. */

case PBS_PHASE_BUILDING:
BarrierWait(&something->barrier, WAIT_EVENT_PBS_PHASE_BUILDING);
/* Fall through. */

case PBS_PHASE_SCANNING:
scan_the_bitmap_and_emit_one_tuple();
}

When a new participant arrives here, if it finds that we're still in
the INIT phase, then it enters an election to see if it can build the
bitmap; one lucky participant wins and does that, while any other
participants twiddle their thumbs at the next BarrierWait call. If a
new participant finds that we're already in the BUILDING phase when it
arrives, then it has missed that election and just has to wait for the
building to be completed. Once they all agree that building has
finished, we move onto scanning. If a new arrival finds that we're in
SCANNING phase, then it happily scans and emits tuples. Does that
make sense?

Not sure exactly how to coordinate rescans yet, but probably with
BarrierWaitSet(&something->barrier, PBS_PHASE_INIT).

--
Thomas Munro
http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#19Amit Kapila
amit.kapila16@gmail.com
In reply to: Thomas Munro (#18)
Re: Parallel bitmap heap scan

On Mon, Nov 28, 2016 at 8:11 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

When a new participant arrives here, if it finds that we're still in
the INIT phase, then it enters an election to see if it can build the
bitmap; one lucky participant wins and does that, while any other
participants twiddle their thumbs at the next BarrierWait call. If a
new participant finds that we're already in the BUILDING phase when it
arrives, then it has missed that election and just has to wait for the
building to be completed. Once they all agree that building has
finished, we move onto scanning. If a new arrival finds that we're in
SCANNING phase, then it happily scans and emits tuples. Does that
make sense?

Not sure exactly how to coordinate rescans yet, but probably with
BarrierWaitSet(&something->barrier, PBS_PHASE_INIT).

Do you think that using barrier's will simplify the patch as compared
to using condition variables because in that case, it will make sense
to use barriers?

--
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#20Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Amit Kapila (#19)
Re: Parallel bitmap heap scan

On Mon, Nov 28, 2016 at 3:49 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:

Do you think that using barrier's will simplify the patch as compared
to using condition variables because in that case, it will make sense
to use barriers?

It would work, but I suppose you might call it overkill. If they were
cooperating to build the bitmap in parallel then a barrier might look
more tempting, because then they'd all be waiting for each other to
agree that they've all finished doing that and are ready to scan.
When they're all just waiting for one guy to flip a single bit, then
it's debatable whether a barrier is any simpler than a condition
variable + a spinlock + a bit!

--
Thomas Munro
http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#21Dilip Kumar
dilipbalaut@gmail.com
In reply to: Thomas Munro (#20)
Re: Parallel bitmap heap scan

On Mon, Nov 28, 2016 at 9:20 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

It would work, but I suppose you might call it overkill. If they were
cooperating to build the bitmap in parallel then a barrier might look
more tempting, because then they'd all be waiting for each other to
agree that they've all finished doing that and are ready to scan.
When they're all just waiting for one guy to flip a single bit, then
it's debatable whether a barrier is any simpler than a condition
variable + a spinlock + a bit!

+1

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#22Dilip Kumar
dilipbalaut@gmail.com
In reply to: Dilip Kumar (#1)
3 attachment(s)
Re: Parallel bitmap heap scan

On Fri, Nov 25, 2016 at 6:55 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

I have changed the design to directly make it based on DSA instead of using DHT.
In new version we no longer use DHT. Instead, of that I have made some
change in simplehash[1], so that it can allow external allocator. In
tidbitmap.c, I have register the allocator to simplehash and those
allocator functions will allocate memory directly from DSA.

simplehash is always using one single memory (during expand it copy
from old memory to new memory). Which makes remaining processing very
simple for us.

In tbm_begin_iterate, we need not to scan complete hash table instead
of that we can just process dsa memory and convert into page array and
chunk array.

I have tested the performance in my local machine and I observed that
it's slightly better than older
DHT based version (complete performance number will be published soon).

Dependency on other patches:
1. dsa_area (dsa-v7.patch)
/messages/by-id/CAEepm=024p-MeAsDmG=R3+tR4EGhuGJs_+rjFKF0eRoSTmMJnA@mail.gmail.com

2. Creating a DSA area to provide work space for parallel execution
(dsa-area-for-executor-v3.patch)
/messages/by-id/CAEepm=0HmRefi1+xDJ99Gj5APHr8Qr05KZtAxrMj8b+ay3o6sA@mail.gmail.com

patch details
1. hash_support_alloc_free_v1.patch [1].
2. parallel-bitmap-heap-scan-v3.patch

I just realised that, my latest patch I just sent to Andres, instead
of replying to all.
Forwarding the same mail to Hackers.

Performance reading with new patch..
TPCH-scale factor 10. work_mem 20MB, Power 4 socket machine

Query Head Patch Improvement
Q4 4811 3290 1.5x
Q6 13136 6198 2.1x
Q14 8119 5057 1.6x
Q15 25652 20143 1.2x

Explained analyzed results are attached with the mail..

* I have also applied Andres patch from below link, for taking this
performance (both for head and for patch).
/messages/by-id/20161123083351.5vramz52nmdokhzz@alap3.anarazel.de

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

hash_support_alloc_free_v1.patchapplication/octet-stream; name=hash_support_alloc_free_v1.patchDownload
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 94cc59d..ba8a8be 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -314,7 +314,7 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
 	hashtable->in_hash_funcs = NULL;
 	hashtable->cur_eq_funcs = NULL;
 
-	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
+	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, NULL);
 	hashtable->hashtab->private_data = hashtable;
 
 	return hashtable;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 826fead..ea79de7 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -244,7 +244,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index 12aedbc..2328413 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -66,6 +66,7 @@
 #define SH_STATUS_EMPTY SH_MAKE_NAME(EMPTY)
 #define SH_STATUS_IN_USE SH_MAKE_NAME(IN_USE)
 #define SH_ITERATOR SH_MAKE_NAME(iterator)
+#define SH_ALLOCATOR SH_MAKE_NAME(alloc)
 
 /* function declarations */
 #define SH_CREATE SH_MAKE_NAME(create)
@@ -90,6 +91,22 @@
 /* generate forward declarations necessary to use the hash table */
 #ifdef SH_DECLARE
 
+/* Memory allocator function pointers */
+typedef void *(*HashAlloc) (Size size, void *args);
+typedef void (*HashFree) (void *pointer, void *args);
+
+typedef struct SH_ALLOCATOR
+{
+	/* Allocation function */
+	void 	*(*HashAlloc) (Size size, void *args);
+
+	/* Free function*/
+	void 	 (*HashFree) (void *pointer, void *args);
+
+	/* Arguments */
+	void 	*args;
+} SH_ALLOCATOR;
+
 /* type definitions */
 typedef struct SH_TYPE
 {
@@ -112,6 +129,9 @@ typedef struct SH_TYPE
 	/* hash buckets */
 	SH_ELEMENT_TYPE *data;
 
+	/* hash allocator */
+	SH_ALLOCATOR	*alloc;
+
 	/* memory context to use for allocations */
 	MemoryContext ctx;
 
@@ -133,7 +153,8 @@ typedef struct SH_ITERATOR
 } SH_ITERATOR;
 
 /* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
+						SH_ALLOCATOR *alloc);
 SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
 SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
 SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -281,7 +302,7 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
  * allocating required memory in the passed-in context.
  */
 SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements)
+SH_CREATE(MemoryContext ctx, uint32 nelements, SH_ALLOCATOR *alloc)
 {
 	SH_TYPE    *tb;
 	uint64		size;
@@ -294,7 +315,17 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 
 	SH_COMPUTE_PARAMETERS(tb, size);
 
-	tb->data = MemoryContextAllocExtended(tb->ctx,
+	tb->alloc = alloc;
+
+	/*
+	 * If allocation handle is passed then use allocation function from the
+	 * handle otherwise use standard allocator.
+	 */
+	if (tb->alloc != NULL)
+		tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+										tb->alloc->args);
+	else
+		tb->data = MemoryContextAllocExtended(tb->ctx,
 										  sizeof(SH_ELEMENT_TYPE) * tb->size,
 										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
 
@@ -333,8 +364,16 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 	/* compute parameters for new table */
 	SH_COMPUTE_PARAMETERS(tb, newsize);
 
-	tb->data = MemoryContextAllocExtended(
-								 tb->ctx, sizeof(SH_ELEMENT_TYPE) * tb->size,
+	/*
+	 * If allocation handle is valid then use allocation function from the
+	 * handle otherwise use standard allocator.
+	 */
+	if (tb->alloc != NULL)
+		tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
+	else
+		tb->data = MemoryContextAllocExtended(tb->ctx,
+										  sizeof(SH_ELEMENT_TYPE) * tb->size,
 										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
 
 	newdata = tb->data;
@@ -421,7 +460,11 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 		}
 	}
 
-	pfree(olddata);
+	if (tb->alloc == NULL)
+		pfree(olddata);
+	else
+		tb->alloc->HashFree(olddata, tb->alloc->args);
+
 }
 
 /*
parallel-bitmap-heap-scan-v3.patchapplication/octet-stream; name=parallel-bitmap-heap-scan-v3.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3de489e..4684cdc 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1234,6 +1234,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index b019bc1..6feda21 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snpashot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index ee1d76e..802a517 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -204,6 +205,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+										e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -256,6 +261,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+										d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -727,6 +737,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									(BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index 449aacb..5cc5c75 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -47,11 +47,15 @@
 #include "utils/spccache.h"
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
-
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
-
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+									ParallelIterator *parallel_iterator);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -67,12 +71,18 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TIDBitmap  *tbm;
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
-
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (pbminfo)
+		parallel_tbm = (ParallelTIDBitmap*)((char *)pbminfo +
+						pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,6 +97,35 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	prefetch_iterator = node->prefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *  Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	First worker to see the state as parallel bitmap info as PBM_INITIAL
+	 *	become leader and set the state to PBM_INPROGRESS All other workers
+	 *	see the state as PBM_INPROGRESS will wait for leader to complete the
+	 *	TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *	  ParallelBitmapInfo so that other worker can see those.
+	 *	  set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader create shared TIDBitmap.
+	 *	  copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over page and chunk array
+	 *	     and select heap pages one by one. If prefetch is enable then
+	 *	     there will be two iterator.
+	 *	  b) Since multiple worker are iterating over same page and chunk
+	 *	     array we need to have a shared iterator, so we grab a spin
+	 *	     lock and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,7 +140,42 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running
+		 * in non parallel mode or we are leader worker.
+		 *
+		 * In parallel mode leader worker will immediately come out
+		 * of the function, but all other worker will be blocked
+		 * until leader worker wake them up.
+		 */
+		if (pbminfo == NULL || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer
+			 * node and set parallel flag in lower level bitmap index scan.
+			 * Later bitmap index node will use this flag to indicate
+			 * tidbitmap that it needs to create an shared page table.
+			 */
+			if (pbminfo)
+			{
+				node->is_leader = true;
+				pbms_set_parallel(outerPlanState(node));
+			}
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM.
+			 * Here we need to create a local TBM and copy information from
+			 * shared location. We also need to attach to shared page table
+			 * using hash table handle stored in parallel_tbm(shared memory).
+			 */
+			tbm = tbm_create(work_mem * 1024L);
+			tbm_set_parallel(tbm, node->ss.ps.state->es_query_area);
+			tbm_restore_shared_members(tbm, parallel_tbm);
+		}
 
 		if (!tbm || !IsA(tbm, TIDBitmap))
 			elog(ERROR, "unrecognized result from subplan");
@@ -118,19 +192,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			node->prefetch_target = -1;
 		}
 #endif   /* USE_PREFETCH */
+
+		/*
+		 * If we are in parallel mode and we are leader worker then
+		 * copy the local TBM information to shared location, and wake
+		 * up other workers.
+		 */
+		if (node->is_leader)
+		{
+			/*
+			 * copy TBM local information in shared memory before waking
+			 * up other workers. Other workers will create there own
+			 * TBM and copy information from shared memory.
+			 */
+			tbm_update_shared_members(tbm, parallel_tbm);
+
+			/* Change the state under a lock */
+			SpinLockAcquire(&pbminfo->state_mutex);
+			pbminfo->state = PBM_FINISHED;
+			SpinLockRelease(&pbminfo->state_mutex);
+
+			/* Wake up all other workers. */
+			ConditionVariableBroadcast(&pbminfo->cv);
+		}
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+									   pbminfo ? &pbminfo->tbmiterator : NULL);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +237,43 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (pbminfo)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages --;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+								pbminfo ? &pbminfo->prefetch_iterator : NULL);
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not
+				 * ensure that current blockno in main iterator and prefetch
+				 * iterator is same. It's possible that whatever blockno we
+				 * are prefetching is getting processed by some other worker.
+				 */
+				if ((pbminfo == NULL)&&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +307,39 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				int *prefetch_target;
+
+				if (pbminfo == NULL)
+					prefetch_target = &node->prefetch_target;
+				else
+				{
+					prefetch_target = &pbminfo->prefetch_target;
+					/* If we are in parallel mode then grab prefetch_mutex */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+				}
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +355,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (pbminfo == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex
+				 * before updating prefetch target.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+					pbminfo->prefetch_target++;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +394,52 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		*prefetch_pages;
+			int		 prefetch_target;
+			TBMIterator *iterator = node->prefetch_iterator;
+			ParallelIterator *parallel_iteartor = NULL;
+			ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+			/*
+			 * If parallel bitmap info available means we are running
+			 * in parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (pbminfo)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				parallel_iteartor = &pbminfo->prefetch_iterator;
+				prefetch_pages = &pbminfo->prefetch_pages;
+				prefetch_target = pbminfo->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
 
+			while (*prefetch_pages < prefetch_target)
+			{
+				TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+															parallel_iteartor);
 				if (tbmpre == NULL)
 				{
 					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					tbm_end_iterate(iterator);
+					prefetch_iterator = node->prefetch_iterator = NULL;
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +654,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +772,8 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->is_leader = false;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -653,3 +860,237 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool needWait = false;
+	bool queuedSelf = false;
+	bool leader = false;
+
+	for(;;)
+	{
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		/*
+		 * Check the current state
+		 *
+		 * If state is
+		 * PBM_INITIAL    -> then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS -> then we need to wait till leader create BITMAP
+		 * PBM_FINISHED   -> BITMAP is ready so no need to wait.
+		 *
+		 */
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/*
+		 * If we are a leader or else leader has already created a
+		 * tid bitmap.
+		 */
+		if (leader || !needWait)
+			break;
+
+		/* We need to queue */
+		if (queuedSelf)
+		{
+			/* Sleep until leader send wake up signal */
+			ConditionVariableSleep(
+							&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+			queuedSelf = false;
+			needWait = false;
+		}
+		else if (needWait)
+		{
+			/* Add ourself to wait queue */
+			ConditionVariablePrepareToSleep(&pbminfo->cv);
+			queuedSelf = true;
+		}
+	}
+
+	/* Cancel the sleep before return */
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Wrapper over tbm_iterate, which take care of copying and restoring iterator
+ * variables from parallel iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator, ParallelIterator *piterator)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (piterator == NULL)
+		return tbm_iterate(iterator);
+
+	/* We are in parallel mode so grab parallel iterator mutex */
+	SpinLockAcquire(&piterator->mutex);
+
+	/*
+	 * Copy information from shared location to local iterator and
+	 * call iterate. Then restore back to shared iterator.
+	 */
+	iterator->spageptr = piterator->spageptr;
+	iterator->schunkptr = piterator->schunkptr;
+	iterator->schunkbit = piterator->schunkbit;
+	output = tbm_iterate(iterator);
+	piterator->spageptr = iterator->spageptr;
+	piterator->schunkptr = iterator->schunkptr;
+	piterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&piterator->mutex);
+
+	return output;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node
+	 * as parallel, because only first node will create the main bitmap
+	 * other bitmaps will be merged to the first bitmap so no need to
+	 * create them in shared memory.
+	 */
+	switch(node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState*)node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState*)node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState*)node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+							phs_snapshot_data),
+							EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+						 ParallelContext *pcxt)
+{
+	ParallelBitmapInfo	*pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+								phs_snapshot_data),
+								EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo	*pbminfo;
+	Snapshot			snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index a364098..9cc5088 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate
+	 * that we need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_area);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index ea79de7..5f9ac41 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -43,6 +43,12 @@
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
 #include "nodes/tidbitmap.h"
+#include "utils/hsearch.h"
+#include "storage/condition_variable.h"
+#include "storage/spin.h"
+#include "storage/lwlock.h"
+#include "nodes/execnodes.h"
+#include "storage/shmem.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -139,24 +145,28 @@ struct TIDBitmap
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	bool			is_shared;	/* need to build shared tbm if set*/
+	dsa_pointer		dsa_data;	/* dsa_pointer to the element array */
+	struct pagetable_alloc *allocator; /* shared memory allocator */
+	int			dsa_entries;	/* actual allocated entries */
+	dsa_area		*area;		/* reference to per-query shared memory area */
 };
 
 /*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
  */
-struct TBMIterator
+struct ParallelTIDBitmap
 {
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+	dsa_pointer		dsa_data;		/* dsa pointers for all kind of pages */
+	int				nentries;		/* number of entries in pagetable */
+	int				maxentries;		/* limit on same to meet maxbytes */
+	int				npages;			/* number of exact entries in pagetable */
+	int				nchunks;		/* number of lossy entries in pagetable */
+	int				dsa_entries;	/* total item in dsa_pages */
+	bool			inited;			/* set true after leader converts page */
+									/* table to dsa_pointer's array. */
 };
 
-
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
 static bool tbm_intersect_page(TIDBitmap *a, PagetableEntry *apage,
@@ -168,7 +178,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
-
+void * tbm_alloc_shared(Size size, void *arg);
+void tbm_free_shared(void *pointer, void *arg);
 /*
  * Simple inline murmur hash implementation for the exact width required, for
  * performance.
@@ -231,6 +242,8 @@ tbm_create(long maxbytes)
 	tbm->maxentries = (int) nbuckets;
 	tbm->lossify_start = 0;
 
+	tbm->allocator = palloc(sizeof(pagetable_alloc));
+
 	return tbm;
 }
 
@@ -244,7 +257,16 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	if (tbm->is_shared)
+	{
+		tbm->allocator->args = tbm;
+		tbm->allocator->HashAlloc = tbm_alloc_shared;
+		tbm->allocator->HashFree = tbm_free_shared;
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, tbm->allocator);
+		tbm->dsa_entries = tbm->pagetable->size;
+	}
+	else
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -271,7 +293,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 void
 tbm_free(TIDBitmap *tbm)
 {
-	if (tbm->pagetable)
+	if (!tbm->is_shared && tbm->pagetable)
 		pagetable_destroy(tbm->pagetable);
 	if (tbm->spages)
 		pfree(tbm->spages);
@@ -612,11 +634,16 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	iterator->tbm = tbm;
 
 	/*
-	 * Initialize iteration pointers.
+	 * Initialize iteration pointers, only if it's not shared tbm.
+	 * In case of shared tbm, we will copy these values from
+	 * shared iterator before calling tbm_iterate.
 	 */
-	iterator->spageptr = 0;
-	iterator->schunkptr = 0;
-	iterator->schunkbit = 0;
+	if (!tbm->is_shared)
+	{
+		iterator->spageptr = 0;
+		iterator->schunkptr = 0;
+		iterator->schunkbit = 0;
+	}
 
 	/*
 	 * If we have a hashtable, create and fill the sorted page lists, unless
@@ -626,7 +653,7 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	 */
 	if (tbm->status == TBM_HASH && !tbm->iterating)
 	{
-		pagetable_iterator i;
+		pagetable_iterator itr;
 		PagetableEntry *page;
 		int			npages;
 		int			nchunks;
@@ -640,15 +667,49 @@ tbm_begin_iterate(TIDBitmap *tbm)
 				MemoryContextAlloc(tbm->mcxt,
 								   tbm->nchunks * sizeof(PagetableEntry *));
 
-		npages = nchunks = 0;
-		pagetable_start_iterate(tbm->pagetable, &i);
-		while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+		/*
+		 * If we have shared TBM means we are running in parallel mode.
+		 * So directly convert dsa array to page and chunk array.
+		 */
+		if (tbm->is_shared)
 		{
-			if (page->ischunk)
-				tbm->schunks[nchunks++] = page;
-			else
-				tbm->spages[npages++] = page;
+			PagetableEntry *dsa_entry;
+			int			i;
+
+			/*
+			 * This step will be done by all the workers including leader.
+			 * Here we need to convert array of dsa pointers to local
+			 * page and chunk array.
+			 */
+			dsa_entry = dsa_get_address(tbm->area, tbm->dsa_data);
+			dsa_entry = (void*)(((char*)dsa_entry) + sizeof(dsa_pointer));
+			npages = nchunks = 0;
+			for (i = 0; i < tbm->dsa_entries; i++)
+			{
+				page = (PagetableEntry*)(dsa_entry + i);
+
+				if (page->status != pagetable_IN_USE)
+					continue;
+
+				if (page->ischunk)
+					tbm->schunks[nchunks++] = page;
+				else
+					tbm->spages[npages++] = page;
+			}
+		}
+		else
+		{
+			npages = nchunks = 0;
+			pagetable_start_iterate(tbm->pagetable, &itr);
+			while ((page = pagetable_iterate(tbm->pagetable, &itr)) != NULL)
+			{
+				if (page->ischunk)
+					tbm->schunks[nchunks++] = page;
+				else
+					tbm->spages[npages++] = page;
+			}
 		}
+
 		Assert(npages == tbm->npages);
 		Assert(nchunks == tbm->nchunks);
 		if (npages > 1)
@@ -1061,3 +1122,114 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_update_shared_members
+ *
+ * Restore leaders private tbm state to shared location. This must
+ * be called before waking up the other worker.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before
+	 * waking up the other workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->dsa_data = tbm->dsa_data;
+	parallel_tbm->dsa_entries = tbm->dsa_entries;
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->is_shared = true;
+	tbm->area = (dsa_area*)area;
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
+
+/*
+ * tbm_restore_shared_members
+ *
+ * Attach worker to shared TID bitmap created by leader worker.
+ */
+void
+tbm_restore_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *stbm)
+{
+	tbm->status = TBM_HASH;
+	tbm->nchunks = stbm->nchunks;
+	tbm->nentries = stbm->nentries;
+	tbm->npages = stbm->npages;
+	tbm->maxentries = stbm->maxentries;
+	tbm->dsa_data = stbm->dsa_data;
+	tbm->dsa_entries = stbm->dsa_entries;
+	tbm->is_shared = true;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Memory allocation function for bitmap hash.
+ * It allocated memory from DSA and also stores dsa_pointer in the memory
+ * so that during free we can directly get the dsa_pointe and free it.
+ */
+void *
+tbm_alloc_shared(Size size, void *arg)
+{
+	TIDBitmap	*tbm = arg;
+	dsa_pointer  dsaptr;
+	char 		*ptr;
+
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->area, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	/* Keep track of actual number of entries */
+	if (tbm->pagetable)
+		tbm->dsa_entries = tbm->pagetable->size;
+
+	ptr = dsa_get_address(tbm->area, dsaptr);
+
+	/* Store dsa_pointer */
+	*((dsa_pointer*)ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * tbm_free_shared
+ *
+ * Memory free function for tid bitmap
+ */
+void
+tbm_free_shared(void *pointer, void *arg)
+{
+	TIDBitmap	*tbm = arg;
+
+	dsa_pointer dsa_data =
+			*((dsa_pointer*)((char*)pointer - sizeof (dsa_pointer)));
+
+	dsa_free(tbm->area, dsa_data);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index e42ef98..058e55a 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -126,6 +126,7 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static int compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 
 
 /*
@@ -678,49 +679,7 @@ create_plain_partial_paths(PlannerInfo *root, RelOptInfo *rel)
 {
 	int			parallel_workers;
 
-	/*
-	 * If the user has set the parallel_workers reloption, use that; otherwise
-	 * select a default number of workers.
-	 */
-	if (rel->rel_parallel_workers != -1)
-		parallel_workers = rel->rel_parallel_workers;
-	else
-	{
-		int			parallel_threshold;
-
-		/*
-		 * If this relation is too small to be worth a parallel scan, just
-		 * return without doing anything ... unless it's an inheritance child.
-		 * In that case, we want to generate a parallel path here anyway.  It
-		 * might not be worthwhile just for this relation, but when combined
-		 * with all of its inheritance siblings it may well pay off.
-		 */
-		if (rel->pages < (BlockNumber) min_parallel_relation_size &&
-			rel->reloptkind == RELOPT_BASEREL)
-			return;
-
-		/*
-		 * Select the number of workers based on the log of the size of the
-		 * relation.  This probably needs to be a good deal more
-		 * sophisticated, but we need something here for now.  Note that the
-		 * upper limit of the min_parallel_relation_size GUC is chosen to
-		 * prevent overflow here.
-		 */
-		parallel_workers = 1;
-		parallel_threshold = Max(min_parallel_relation_size, 1);
-		while (rel->pages >= (BlockNumber) (parallel_threshold * 3))
-		{
-			parallel_workers++;
-			parallel_threshold *= 3;
-			if (parallel_threshold > INT_MAX / 3)
-				break;			/* avoid overflow */
-		}
-	}
-
-	/*
-	 * In no case use more than max_parallel_workers_per_gather workers.
-	 */
-	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+	parallel_workers = compute_parallel_worker(rel, rel->pages);
 
 	/* If any limit was set to zero, the user doesn't want a parallel scan. */
 	if (parallel_workers <= 0)
@@ -2866,6 +2825,84 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int				parallel_workers;
+	double			pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path*)create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+static int
+compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
+{
+	int			parallel_workers;
+
+	/*
+	 * If the user has set the parallel_workers reloption, use that; otherwise
+	 * select a default number of workers.
+	 */
+	if (rel->rel_parallel_workers != -1)
+		parallel_workers = rel->rel_parallel_workers;
+	else
+	{
+		int			parallel_threshold;
+
+		/*
+		 * If this relation is too small to be worth a parallel scan, just
+		 * return without doing anything ... unless it's an inheritance child.
+		 * In that case, we want to generate a parallel path here anyway.  It
+		 * might not be worthwhile just for this relation, but when combined
+		 * with all of its inheritance siblings it may well pay off.
+		 */
+		if (pages < (BlockNumber) min_parallel_relation_size &&
+			rel->reloptkind == RELOPT_BASEREL)
+			return 0;
+
+		/*
+		 * Select the number of workers based on the log of the size of the
+		 * relation.  This probably needs to be a good deal more
+		 * sophisticated, but we need something here for now.  Note that the
+		 * upper limit of the min_parallel_relation_size GUC is chosen to
+		 * prevent overflow here.
+		 */
+		parallel_workers = 1;
+		parallel_threshold = Max(min_parallel_relation_size, 1);
+		while (pages >= (BlockNumber) (parallel_threshold * 3))
+		{
+			parallel_workers++;
+			parallel_threshold *= 3;
+			if (parallel_threshold > INT_MAX / 3)
+				break;			/* avoid overflow */
+		}
+	}
+
+	/*
+	 * In no case use more than max_parallel_workers_per_gather workers.
+	 */
+	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+
+	return parallel_workers;
+}
+
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index e42895d..64bb8b6 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -161,7 +161,7 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
 static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
 static double relation_byte_size(double tuples, int width);
 static double page_size(double tuples, int width);
-
+static Cost adjust_cost_for_parallelism(Path *path, Cost cpu_run_cost);
 
 /*
  * clamp_row_est
@@ -237,44 +237,7 @@ cost_seqscan(Path *path, PlannerInfo *root,
 
 	/* Adjust costing for parallelism, if used. */
 	if (path->parallel_workers > 0)
-	{
-		double		parallel_divisor = path->parallel_workers;
-		double		leader_contribution;
-
-		/*
-		 * Early experience with parallel query suggests that when there is
-		 * only one worker, the leader often makes a very substantial
-		 * contribution to executing the parallel portion of the plan, but as
-		 * more workers are added, it does less and less, because it's busy
-		 * reading tuples from the workers and doing whatever non-parallel
-		 * post-processing is needed.  By the time we reach 4 workers, the
-		 * leader no longer makes a meaningful contribution.  Thus, for now,
-		 * estimate that the leader spends 30% of its time servicing each
-		 * worker, and the remainder executing the parallel plan.
-		 */
-		leader_contribution = 1.0 - (0.3 * path->parallel_workers);
-		if (leader_contribution > 0)
-			parallel_divisor += leader_contribution;
-
-		/*
-		 * In the case of a parallel plan, the row count needs to represent
-		 * the number of tuples processed per worker.  Otherwise, higher-level
-		 * plan nodes that appear below the gather will be costed incorrectly,
-		 * because they'll anticipate receiving more rows than any given copy
-		 * will actually get.
-		 */
-		path->rows = clamp_row_est(path->rows / parallel_divisor);
-
-		/* The CPU cost is divided among all the workers. */
-		cpu_run_cost /= parallel_divisor;
-
-		/*
-		 * It may be possible to amortize some of the I/O cost, but probably
-		 * not very much, because most operating systems already do aggressive
-		 * prefetching.  For now, we assume that the disk run cost can't be
-		 * amortized at all.
-		 */
-	}
+		cpu_run_cost = adjust_cost_for_parallelism(path, cpu_run_cost);
 
 	path->startup_cost = startup_cost;
 	path->total_cost = startup_cost + cpu_run_cost + disk_run_cost;
@@ -831,10 +794,10 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
 	Cost		indexTotalCost;
-	Selectivity indexSelectivity;
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -855,13 +818,12 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	if (!enable_bitmapscan)
 		startup_cost += disable_cost;
 
-	/*
-	 * Fetch total cost of obtaining the bitmap, as well as its total
-	 * selectivity.
-	 */
-	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+	pages_fetched = compute_bitmap_pages(root, baserel, bitmapqual,
+										loop_count, &indexTotalCost,
+										&tuples_fetched);
 
 	startup_cost += indexTotalCost;
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
 
 	/* Fetch estimated page costs for tablespace containing table. */
 	get_tablespace_page_costs(baserel->reltablespace,
@@ -869,41 +831,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 							  &spc_seq_page_cost);
 
 	/*
-	 * Estimate number of main-table pages fetched.
-	 */
-	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
-
-	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
-
-	if (loop_count > 1)
-	{
-		/*
-		 * For repeated bitmap scans, scale up the number of tuples fetched in
-		 * the Mackert and Lohman formula by the number of scans, so that we
-		 * estimate the number of pages fetched by all the scans. Then
-		 * pro-rate for one scan.
-		 */
-		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
-											baserel->pages,
-											get_indexpath_pages(bitmapqual),
-											root);
-		pages_fetched /= loop_count;
-	}
-	else
-	{
-		/*
-		 * For a single scan, the number of heap pages that need to be fetched
-		 * is the same as the Mackert and Lohman formula for the case T <= b
-		 * (ie, no re-reads needed).
-		 */
-		pages_fetched = (2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
-	}
-	if (pages_fetched >= T)
-		pages_fetched = T;
-	else
-		pages_fetched = ceil(pages_fetched);
-
-	/*
 	 * For small numbers of pages we should charge spc_random_page_cost
 	 * apiece, while if nearly all the table's pages are being read, it's more
 	 * appropriate to charge spc_seq_page_cost apiece.  The effect is
@@ -932,8 +859,13 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+		cpu_run_cost = adjust_cost_for_parallelism(path, cpu_run_cost);
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
@@ -944,6 +876,56 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 }
 
 /*
+ * adjust_cost_for_parallelism
+ *
+ * Adjust the cpu cost based on number of parallel workers, also update
+ * the number of rows processed at each worker.
+ */
+static Cost
+adjust_cost_for_parallelism(Path *path, Cost cpu_run_cost)
+{
+	double		parallel_divisor = path->parallel_workers;
+	double		leader_contribution;
+	Cost		cpu_cost = cpu_run_cost;
+
+	/*
+	 * Early experience with parallel query suggests that when there is
+	 * only one worker, the leader often makes a very substantial
+	 * contribution to executing the parallel portion of the plan, but as
+	 * more workers are added, it does less and less, because it's busy
+	 * reading tuples from the workers and doing whatever non-parallel
+	 * post-processing is needed.  By the time we reach 4 workers, the
+	 * leader no longer makes a meaningful contribution.  Thus, for now,
+	 * estimate that the leader spends 30% of its time servicing each
+	 * worker, and the remainder executing the parallel plan.
+	 */
+	leader_contribution = 1.0 - (0.3 * path->parallel_workers);
+	if (leader_contribution > 0)
+		parallel_divisor += leader_contribution;
+
+	/*
+	 * In the case of a parallel plan, the row count needs to represent
+	 * the number of tuples processed per worker.  Otherwise, higher-level
+	 * plan nodes that appear below the gather will be costed incorrectly,
+	 * because they'll anticipate receiving more rows than any given copy
+	 * will actually get.
+	 */
+	path->rows = clamp_row_est(path->rows / parallel_divisor);
+
+	/* The CPU cost is divided among all the workers. */
+	cpu_cost /= parallel_divisor;
+
+	/*
+	 * It may be possible to amortize some of the I/O cost, but probably
+	 * not very much, because most operating systems already do aggressive
+	 * prefetching.  For now, we assume that the disk run cost can't be
+	 * amortized at all.
+	 */
+
+	return cpu_cost;
+}
+
+/*
  * cost_bitmap_tree_node
  *		Extract cost and selectivity from a bitmap tree node (index/and/or)
  */
@@ -4764,3 +4746,69 @@ page_size(double tuples, int width)
 {
 	return ceil(relation_byte_size(tuples, width) / BLCKSZ);
 }
+
+/*
+ * compute_bitmap_pages
+ *
+ * compute number of pages fetched from heap in bitmap heap scan.
+ */
+double
+compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel, Path *bitmapqual,
+			int loop_count, Cost *cost, double *tuple)
+{
+	Cost			indexTotalCost;
+	Selectivity		indexSelectivity;
+	double			T;
+	double			pages_fetched;
+	double			tuples_fetched;
+
+	/*
+	 * Fetch total cost of obtaining the bitmap, as well as its total
+	 * selectivity.
+	 */
+	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+
+	/*
+	 * Estimate number of main-table pages fetched.
+	 */
+	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
+
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
+
+	if (loop_count > 1)
+	{
+		/*
+		 * For repeated bitmap scans, scale up the number of tuples fetched in
+		 * the Mackert and Lohman formula by the number of scans, so that we
+		 * estimate the number of pages fetched by all the scans. Then
+		 * pro-rate for one scan.
+		 */
+		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
+											baserel->pages,
+											get_indexpath_pages(bitmapqual),
+											root);
+		pages_fetched /= loop_count;
+	}
+	else
+	{
+		/*
+		 * For a single scan, the number of heap pages that need to be fetched
+		 * is the same as the Mackert and Lohman formula for the case T <= b
+		 * (ie, no re-reads needed).
+		 */
+		pages_fetched =
+				(2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
+	}
+
+	if (pages_fetched >= T)
+		pages_fetched = T;
+	else
+		pages_fetched = ceil(pages_fetched);
+
+	if (cost)
+		*cost = indexTotalCost;
+	if (tuple)
+		*tuple = tuples_fetched;
+
+	return pages_fetched;
+}
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 2952bfb..1a7bde0 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index ad49674..0b544a1 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2803,7 +2803,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 6d3ccfd..00b5c01 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,11 +1081,10 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
-
 	pathnode->bitmapqual = bitmapqual;
 
 	cost_bitmap_heap_scan(&pathnode->path, root, rel,
@@ -3192,7 +3192,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index a392197..80c80ab 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3387,6 +3387,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 0d12bbb..ad9c693 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -178,6 +178,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index 0ed9c78..8afecb7 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+								ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+								ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+										shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 7967383..e1cc891 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1399,6 +1401,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t	mutex;			/* mutual exclusion for below three fields */
+	int 	spageptr;		/* next spages index */
+	int 	schunkptr;		/* next schunks index */
+	int 	schunkbit;		/* next bit to check in current schunk */
+} ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *	 						parallel bitmap heap scan.
+ *
+ *	 	PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *	 						first worker to see this state will become
+ *	 						leader and will create TIDbitmap. This will
+ *	 						also set the state to PBM_INPROGRESS.
+ *	 	PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *	 						so workers need to sleep until leader set the
+ *	 						state to PBM_FINISHED and wake us up.
+ *	 	PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *	 						can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *	 	relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid						relid;
+	ParallelIterator		tbmiterator;
+	ParallelIterator		prefetch_iterator;
+	slock_t					prefetch_mutex;
+	int						prefetch_pages;
+	int						prefetch_target;
+	slock_t					state_mutex;
+	ConditionVariable		cv;
+	PBMState				state;
+	Size					ptbm_offset;
+	char					phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1413,6 +1481,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1429,6 +1499,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1444,7 +1515,11 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		is_leader		   is_leader is set true, if this worker become
+ *						   leader for parallel bitmap heap scan.
+ *		parallel_bitmap	   shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1459,6 +1534,9 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	bool		is_leader;
+	ParallelBitmapInfo	*parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 9303299..94e7756 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+}TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -58,9 +75,14 @@ extern void tbm_union(TIDBitmap *a, const TIDBitmap *b);
 extern void tbm_intersect(TIDBitmap *a, const TIDBitmap *b);
 
 extern bool tbm_is_empty(const TIDBitmap *tbm);
-
-extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
+extern TBMIterator * tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+									ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+									ParallelTIDBitmap *parallel_tbm);
+void tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..e1baacc 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -183,6 +183,8 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
 					   double cte_rows);
 extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
 extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern double compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel,
+				Path *bitmapqual, int loop_count, Cost *cost, double *tuple);
 
 /*
  * prototypes for clausesel.c
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..397d266 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 44abe83..3264f44 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 0b85b7a..07ea852 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -784,6 +784,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
analyze.tar.gzapplication/x-gzip; name=analyze.tar.gzDownload
#23Haribabu Kommi
kommi.haribabu@gmail.com
In reply to: Dilip Kumar (#22)
Re: Parallel bitmap heap scan

On Wed, Nov 30, 2016 at 4:38 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Fri, Nov 25, 2016 at 6:55 PM, Dilip Kumar <dilipbalaut@gmail.com>
wrote:

I have changed the design to directly make it based on DSA instead of

using DHT.

In new version we no longer use DHT. Instead, of that I have made some
change in simplehash[1], so that it can allow external allocator. In
tidbitmap.c, I have register the allocator to simplehash and those
allocator functions will allocate memory directly from DSA.

simplehash is always using one single memory (during expand it copy
from old memory to new memory). Which makes remaining processing very
simple for us.

In tbm_begin_iterate, we need not to scan complete hash table instead
of that we can just process dsa memory and convert into page array and
chunk array.

I have tested the performance in my local machine and I observed that
it's slightly better than older
DHT based version (complete performance number will be published soon).

Dependency on other patches:
1. dsa_area (dsa-v7.patch)
/messages/by-id/CAEepm=024p-

MeAsDmG%3DR3%2BtR4EGhuGJs_%2BrjFKF0eRoSTmMJnA%40mail.gmail.com

2. Creating a DSA area to provide work space for parallel execution
(dsa-area-for-executor-v3.patch)
/messages/by-id/CAEepm=0HmRefi1%25

2BxDJ99Gj5APHr8Qr05KZtAxrMj8b%2Bay3o6sA%40mail.gmail.com

patch details
1. hash_support_alloc_free_v1.patch [1].
2. parallel-bitmap-heap-scan-v3.patch

I just realised that, my latest patch I just sent to Andres, instead
of replying to all.
Forwarding the same mail to Hackers.

Performance reading with new patch..
TPCH-scale factor 10. work_mem 20MB, Power 4 socket machine

Query Head Patch Improvement
Q4 4811 3290 1.5x
Q6 13136 6198 2.1x
Q14 8119 5057 1.6x
Q15 25652 20143 1.2x

Explained analyzed results are attached with the mail..

* I have also applied Andres patch from below link, for taking this
performance (both for head and for patch).
/messages/by-id/20161123083351.
5vramz52nmdokhzz%40alap3.anarazel.de

Moved to next CF with "needs review" status.

Regards,
Hari Babu
Fujitsu Australia

#24Amit Kapila
amit.kapila16@gmail.com
In reply to: Dilip Kumar (#22)
Re: Parallel bitmap heap scan

On Wed, Nov 30, 2016 at 11:08 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

patch details
1. hash_support_alloc_free_v1.patch [1].
2. parallel-bitmap-heap-scan-v3.patch

Few assorted comments:

1.
+ else if (needWait)
+ {
+ /* Add ourself to wait queue */
+ ConditionVariablePrepareToSleep(&pbminfo->cv);
+ queuedSelf = true;
+ }

With the committed version of condition variables, you can avoid
calling ConditionVariablePrepareToSleep(). Refer latest parallel
index scan patch [1].

2.
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader to complete the TidBitmap.</entry>
+        </row>
+        <row>

Isn't it better to write it as below:

Waiting for the leader backend to form the TidBitmap.

3.
+ * Update snpashot info in heap scan descriptor.

/snpashot/snapshot

4.
 #include "utils/tqual.h"
-
+#include "miscadmin.h"
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
-
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
  TBMIterateResult *tbmres;
-
+ ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 static int tbm_comparator(const void *left, const void *right);
-
+void * tbm_alloc_shared(Size size, void *arg);

It seems line deletes at above places are spurious. Please check for
similar occurrences at other places in patch.

5.
+ bool is_shared; /* need to build shared tbm if set*/

space is required towards the end of the comment (set */).

6.
+ /*
+ * If we have shared TBM means we are running in parallel mode.
+ * So directly convert dsa array to page and chunk array.
+ */

I think the above comment can be simplified as "For shared TBM,
directly convert dsa array to page and chunk array"

7.
+ dsa_entry = (void*)(((char*)dsa_entry) + sizeof(dsa_pointer));

extra space is missing at multiple places in above line. It should be
written as below:

dsa_entry = (void *)(((char *) dsa_entry) + sizeof(dsa_pointer));

Similar stuff needs to be taken care at other places in the patch as
well. I think it will be better if you run pgindent on your patch.

--
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#25Dilip Kumar
dilipbalaut@gmail.com
In reply to: Amit Kapila (#24)
2 attachment(s)
Re: Parallel bitmap heap scan

On Sat, Dec 10, 2016 at 5:36 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:

Few assorted comments:

Thanks for the review

1.
+ else if (needWait)
+ {
+ /* Add ourself to wait queue */
+ ConditionVariablePrepareToSleep(&pbminfo->cv);
+ queuedSelf = true;
+ }

With the committed version of condition variables, you can avoid
calling ConditionVariablePrepareToSleep(). Refer latest parallel
index scan patch [1].

Oh, I see, Fixed.

2.
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader to complete the TidBitmap.</entry>
+        </row>
+        <row>

Isn't it better to write it as below:

Waiting for the leader backend to form the TidBitmap.

Done this way..

3.
+ * Update snpashot info in heap scan descriptor.

/snpashot/snapshot

Fixed

4.
#include "utils/tqual.h"
-
+#include "miscadmin.h"
static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
-
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
TBMIterateResult *tbmres;
-
+ ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
static int tbm_comparator(const void *left, const void *right);
-
+void * tbm_alloc_shared(Size size, void *arg);

It seems line deletes at above places are spurious. Please check for
similar occurrences at other places in patch.

Fixed

5.
+ bool is_shared; /* need to build shared tbm if set*/

space is required towards the end of the comment (set */).

Fixed

6.
+ /*
+ * If we have shared TBM means we are running in parallel mode.
+ * So directly convert dsa array to page and chunk array.
+ */

I think the above comment can be simplified as "For shared TBM,
directly convert dsa array to page and chunk array"

Done this way..

7.
+ dsa_entry = (void*)(((char*)dsa_entry) + sizeof(dsa_pointer));

extra space is missing at multiple places in above line. It should be
written as below:

dsa_entry = (void *)(((char *) dsa_entry) + sizeof(dsa_pointer));

Fixed..

Similar stuff needs to be taken care at other places in the patch as
well. I think it will be better if you run pgindent on your patch.

I have run the pgindent, and taken all the changes whichever was
applicable for added code.

There are some cleanup for "hash-support-alloc-free" also, so
attaching a new patch for this as well.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

parallel-bitmap-heap-scan-v4.patchapplication/octet-stream; name=parallel-bitmap-heap-scan-v4.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 128ee13..70fcb65 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1236,6 +1236,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index b019bc1..e99ab66 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 7d88489..bb1a010 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -203,6 +204,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -255,6 +260,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -723,6 +733,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index 449aacb..1fd2298 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,16 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+					  ParallelIterator *parallel_iterator);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -68,11 +74,18 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
 
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (pbminfo)
+		parallel_tbm = (ParallelTIDBitmap *) ((char *) pbminfo +
+											  pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,6 +100,35 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	prefetch_iterator = node->prefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	First worker to see the state as parallel bitmap info as PBM_INITIAL
+	 *	become leader and set the state to PBM_INPROGRESS All other workers
+	 *	see the state as PBM_INPROGRESS will wait for leader to complete the
+	 *	TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *	  ParallelBitmapInfo so that other worker can see those.
+	 *	  set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader create shared TIDBitmap.
+	 *	  copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over page and chunk array
+	 *		 and select heap pages one by one. If prefetch is enable then
+	 *		 there will be two iterator.
+	 *	  b) Since multiple worker are iterating over same page and chunk
+	 *		 array we need to have a shared iterator, so we grab a spin
+	 *		 lock and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,7 +143,41 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.
+		 *
+		 * In parallel mode leader will immediately come out of the function,
+		 * but all other worker will be blocked until leader wake them up.
+		 */
+		if (pbminfo == NULL || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag in lower level bitmap index scan. Later
+			 * bitmap index node will use this flag to indicate tidbitmap that
+			 * it needs to create an shared page table.
+			 */
+			if (pbminfo)
+			{
+				node->is_leader = true;
+				pbms_set_parallel(outerPlanState(node));
+			}
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM. Here we
+			 * need to create a local TBM and copy information from shared
+			 * location. We also need to attach to shared page table using
+			 * hash table handle stored in parallel_tbm(shared memory).
+			 */
+			tbm = tbm_create(work_mem * 1024L);
+			tbm_set_parallel(tbm, node->ss.ps.state->es_query_area);
+			tbm_restore_shared_members(tbm, parallel_tbm);
+		}
 
 		if (!tbm || !IsA(tbm, TIDBitmap))
 			elog(ERROR, "unrecognized result from subplan");
@@ -118,19 +194,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			node->prefetch_target = -1;
 		}
 #endif   /* USE_PREFETCH */
+
+		/*
+		 * If we are in parallel mode and we are leader worker then copy the
+		 * local TBM information to shared location, and wake up other
+		 * workers.
+		 */
+		if (node->is_leader)
+		{
+			/*
+			 * copy TBM local information in shared memory before waking up
+			 * other workers. Other workers will create there own TBM and copy
+			 * information from shared memory.
+			 */
+			tbm_update_shared_members(tbm, parallel_tbm);
+
+			/* Change the state under a lock */
+			SpinLockAcquire(&pbminfo->state_mutex);
+			pbminfo->state = PBM_FINISHED;
+			SpinLockRelease(&pbminfo->state_mutex);
+
+			/* Wake up all other workers. */
+			ConditionVariableBroadcast(&pbminfo->cv);
+		}
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+									 pbminfo ? &pbminfo->tbmiterator : NULL);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +239,43 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (pbminfo)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+							   pbminfo ? &pbminfo->prefetch_iterator : NULL);
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not
+				 * ensure that current blockno in main iterator and prefetch
+				 * iterator is same. It's possible that whatever blockno we
+				 * are prefetching is getting processed by some other worker.
+				 */
+				if ((pbminfo == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +309,39 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				int		   *prefetch_target;
+
+				if (pbminfo == NULL)
+					prefetch_target = &node->prefetch_target;
+				else
+				{
+					prefetch_target = &pbminfo->prefetch_target;
+					/* If we are in parallel mode then grab prefetch_mutex */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+				}
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +357,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (pbminfo == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+					pbminfo->prefetch_target++;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +396,52 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+			TBMIterator *iterator = node->prefetch_iterator;
+			ParallelIterator *parallel_iteartor = NULL;
+			ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (pbminfo)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				parallel_iteartor = &pbminfo->prefetch_iterator;
+				prefetch_pages = &pbminfo->prefetch_pages;
+				prefetch_target = pbminfo->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
 
+			while (*prefetch_pages < prefetch_target)
+			{
+				TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+														  parallel_iteartor);
 				if (tbmpre == NULL)
 				{
 					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					tbm_end_iterate(iterator);
+					prefetch_iterator = node->prefetch_iterator = NULL;
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +656,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +774,8 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->is_leader = false;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -653,3 +862,219 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*
+		 * Check the current state
+		 *
+		 * If state is
+		 * PBM_INITIAL    -> then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS -> then we need to wait till leader create BITMAP
+		 * PBM_FINISHED   -> BITMAP is ready so no need to wait.
+		 *
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Wrapper over tbm_iterate, which take care of copying and restoring iterator
+ * variables from parallel iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator, ParallelIterator *piterator)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (piterator == NULL)
+		return tbm_iterate(iterator);
+
+	/* We are in parallel mode so grab parallel iterator mutex */
+	SpinLockAcquire(&piterator->mutex);
+
+	/*
+	 * Copy information from shared location to local iterator and call
+	 * iterate. Then restore back to shared iterator.
+	 */
+	iterator->spageptr = piterator->spageptr;
+	iterator->schunkptr = piterator->schunkptr;
+	iterator->schunkbit = piterator->schunkbit;
+	output = tbm_iterate(iterator);
+	piterator->spageptr = iterator->spageptr;
+	piterator->schunkptr = iterator->schunkptr;
+	piterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&piterator->mutex);
+
+	return output;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node as
+	 * parallel, because only first node will create the main bitmap other
+	 * bitmaps will be merged to the first bitmap so no need to create them in
+	 * shared memory.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState *) node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index a364098..9cc5088 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate
+	 * that we need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_area);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index ea79de7..c0b49e5 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -43,6 +43,12 @@
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
 #include "nodes/tidbitmap.h"
+#include "utils/hsearch.h"
+#include "storage/condition_variable.h"
+#include "storage/spin.h"
+#include "storage/lwlock.h"
+#include "nodes/execnodes.h"
+#include "storage/shmem.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -139,21 +145,26 @@ struct TIDBitmap
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	bool		is_shared;		/* need to build shared tbm if set */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	struct pagetable_alloc *allocator;	/* shared memory allocator */
+	int			dsa_entries;	/* actual allocated entries */
+	dsa_area   *area;			/* reference to per-query shared memory area */
 };
 
 /*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
  */
-struct TBMIterator
+struct ParallelTIDBitmap
 {
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+	dsa_pointer dsa_data;		/* dsa pointers for all kind of pages */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	int			dsa_entries;	/* total item in dsa_pages */
+	bool		inited;			/* set true after leader converts page */
+	/* table to dsa_pointer's array. */
 };
 
 
@@ -168,6 +179,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+void	   *tbm_alloc_shared(Size size, void *arg);
+void		tbm_free_shared(void *pointer, void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -231,6 +244,8 @@ tbm_create(long maxbytes)
 	tbm->maxentries = (int) nbuckets;
 	tbm->lossify_start = 0;
 
+	tbm->allocator = palloc(sizeof(pagetable_alloc));
+
 	return tbm;
 }
 
@@ -244,7 +259,16 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	if (tbm->is_shared)
+	{
+		tbm->allocator->args = tbm;
+		tbm->allocator->HashAlloc = tbm_alloc_shared;
+		tbm->allocator->HashFree = tbm_free_shared;
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, tbm->allocator);
+		tbm->dsa_entries = tbm->pagetable->size;
+	}
+	else
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -271,7 +295,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 void
 tbm_free(TIDBitmap *tbm)
 {
-	if (tbm->pagetable)
+	if (!tbm->is_shared && tbm->pagetable)
 		pagetable_destroy(tbm->pagetable);
 	if (tbm->spages)
 		pfree(tbm->spages);
@@ -612,11 +636,16 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	iterator->tbm = tbm;
 
 	/*
-	 * Initialize iteration pointers.
+	 * Initialize iteration pointers, only if it's not shared tbm. For shared
+	 * TBM, we will copy these values from shared iterator before calling
+	 * tbm_iterate.
 	 */
-	iterator->spageptr = 0;
-	iterator->schunkptr = 0;
-	iterator->schunkbit = 0;
+	if (!tbm->is_shared)
+	{
+		iterator->spageptr = 0;
+		iterator->schunkptr = 0;
+		iterator->schunkbit = 0;
+	}
 
 	/*
 	 * If we have a hashtable, create and fill the sorted page lists, unless
@@ -626,7 +655,7 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	 */
 	if (tbm->status == TBM_HASH && !tbm->iterating)
 	{
-		pagetable_iterator i;
+		pagetable_iterator itr;
 		PagetableEntry *page;
 		int			npages;
 		int			nchunks;
@@ -640,15 +669,48 @@ tbm_begin_iterate(TIDBitmap *tbm)
 				MemoryContextAlloc(tbm->mcxt,
 								   tbm->nchunks * sizeof(PagetableEntry *));
 
-		npages = nchunks = 0;
-		pagetable_start_iterate(tbm->pagetable, &i);
-		while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+		/*
+		 * For shared TBM, directly convert dsa array to page and chunk array.
+		 */
+		if (tbm->is_shared)
 		{
-			if (page->ischunk)
-				tbm->schunks[nchunks++] = page;
-			else
-				tbm->spages[npages++] = page;
+			PagetableEntry *dsa_entry;
+			int			i;
+
+			/*
+			 * This step will be done by all the workers including leader.
+			 * Here we need to convert array of dsa pointers to local page and
+			 * chunk array.
+			 */
+			dsa_entry = dsa_get_address(tbm->area, tbm->dsa_data);
+			dsa_entry = (void *) (((char *) dsa_entry) + sizeof(dsa_pointer));
+			npages = nchunks = 0;
+			for (i = 0; i < tbm->dsa_entries; i++)
+			{
+				page = (PagetableEntry *) (dsa_entry + i);
+
+				if (page->status != pagetable_IN_USE)
+					continue;
+
+				if (page->ischunk)
+					tbm->schunks[nchunks++] = page;
+				else
+					tbm->spages[npages++] = page;
+			}
+		}
+		else
+		{
+			npages = nchunks = 0;
+			pagetable_start_iterate(tbm->pagetable, &itr);
+			while ((page = pagetable_iterate(tbm->pagetable, &itr)) != NULL)
+			{
+				if (page->ischunk)
+					tbm->schunks[nchunks++] = page;
+				else
+					tbm->spages[npages++] = page;
+			}
 		}
+
 		Assert(npages == tbm->npages);
 		Assert(nchunks == tbm->nchunks);
 		if (npages > 1)
@@ -1061,3 +1123,114 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_update_shared_members
+ *
+ * Restore leaders private tbm state to shared location. This must
+ * be called before waking up the other worker.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before waking up the other
+	 * workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->dsa_data = tbm->dsa_data;
+	parallel_tbm->dsa_entries = tbm->dsa_entries;
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->is_shared = true;
+	tbm->area = (dsa_area *) area;
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size
+tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
+
+/*
+ * tbm_restore_shared_members
+ *
+ * Attach worker to shared TID bitmap created by leader worker.
+ */
+void
+tbm_restore_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *stbm)
+{
+	tbm->status = TBM_HASH;
+	tbm->nchunks = stbm->nchunks;
+	tbm->nentries = stbm->nentries;
+	tbm->npages = stbm->npages;
+	tbm->maxentries = stbm->maxentries;
+	tbm->dsa_data = stbm->dsa_data;
+	tbm->dsa_entries = stbm->dsa_entries;
+	tbm->is_shared = true;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Memory allocation function for bitmap hash.
+ * It allocated memory from DSA and also stores dsa_pointer in the memory
+ * so that during free we can directly get the dsa_pointe and free it.
+ */
+void *
+tbm_alloc_shared(Size size, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->area, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	/* Keep track of actual number of entries */
+	if (tbm->pagetable)
+		tbm->dsa_entries = tbm->pagetable->size;
+
+	ptr = dsa_get_address(tbm->area, dsaptr);
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * tbm_free_shared
+ *
+ * Memory free function for tid bitmap
+ */
+void
+tbm_free_shared(void *pointer, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+
+	dsa_pointer dsa_data =
+			*((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+
+	dsa_free(tbm->area, dsa_data);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 9753a26..a51b11d 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -126,6 +126,7 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static int	compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 
 
 /*
@@ -678,49 +679,7 @@ create_plain_partial_paths(PlannerInfo *root, RelOptInfo *rel)
 {
 	int			parallel_workers;
 
-	/*
-	 * If the user has set the parallel_workers reloption, use that; otherwise
-	 * select a default number of workers.
-	 */
-	if (rel->rel_parallel_workers != -1)
-		parallel_workers = rel->rel_parallel_workers;
-	else
-	{
-		int			parallel_threshold;
-
-		/*
-		 * If this relation is too small to be worth a parallel scan, just
-		 * return without doing anything ... unless it's an inheritance child.
-		 * In that case, we want to generate a parallel path here anyway.  It
-		 * might not be worthwhile just for this relation, but when combined
-		 * with all of its inheritance siblings it may well pay off.
-		 */
-		if (rel->pages < (BlockNumber) min_parallel_relation_size &&
-			rel->reloptkind == RELOPT_BASEREL)
-			return;
-
-		/*
-		 * Select the number of workers based on the log of the size of the
-		 * relation.  This probably needs to be a good deal more
-		 * sophisticated, but we need something here for now.  Note that the
-		 * upper limit of the min_parallel_relation_size GUC is chosen to
-		 * prevent overflow here.
-		 */
-		parallel_workers = 1;
-		parallel_threshold = Max(min_parallel_relation_size, 1);
-		while (rel->pages >= (BlockNumber) (parallel_threshold * 3))
-		{
-			parallel_workers++;
-			parallel_threshold *= 3;
-			if (parallel_threshold > INT_MAX / 3)
-				break;			/* avoid overflow */
-		}
-	}
-
-	/*
-	 * In no case use more than max_parallel_workers_per_gather workers.
-	 */
-	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+	parallel_workers = compute_parallel_worker(rel, rel->pages);
 
 	/* If any limit was set to zero, the user doesn't want a parallel scan. */
 	if (parallel_workers <= 0)
@@ -2866,6 +2825,84 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+static int
+compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
+{
+	int			parallel_workers;
+
+	/*
+	 * If the user has set the parallel_workers reloption, use that; otherwise
+	 * select a default number of workers.
+	 */
+	if (rel->rel_parallel_workers != -1)
+		parallel_workers = rel->rel_parallel_workers;
+	else
+	{
+		int			parallel_threshold;
+
+		/*
+		 * If this relation is too small to be worth a parallel scan, just
+		 * return without doing anything ... unless it's an inheritance child.
+		 * In that case, we want to generate a parallel path here anyway.  It
+		 * might not be worthwhile just for this relation, but when combined
+		 * with all of its inheritance siblings it may well pay off.
+		 */
+		if (pages < (BlockNumber) min_parallel_relation_size &&
+			rel->reloptkind == RELOPT_BASEREL)
+			return 0;
+
+		/*
+		 * Select the number of workers based on the log of the size of the
+		 * relation.  This probably needs to be a good deal more
+		 * sophisticated, but we need something here for now.  Note that the
+		 * upper limit of the min_parallel_relation_size GUC is chosen to
+		 * prevent overflow here.
+		 */
+		parallel_workers = 1;
+		parallel_threshold = Max(min_parallel_relation_size, 1);
+		while (pages >= (BlockNumber) (parallel_threshold * 3))
+		{
+			parallel_workers++;
+			parallel_threshold *= 3;
+			if (parallel_threshold > INT_MAX / 3)
+				break;			/* avoid overflow */
+		}
+	}
+
+	/*
+	 * In no case use more than max_parallel_workers_per_gather workers.
+	 */
+	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+
+	return parallel_workers;
+}
+
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index e42895d..785736d 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -161,6 +161,7 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
 static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
 static double relation_byte_size(double tuples, int width);
 static double page_size(double tuples, int width);
+static Cost adjust_cost_for_parallelism(Path *path, Cost cpu_run_cost);
 
 
 /*
@@ -237,44 +238,7 @@ cost_seqscan(Path *path, PlannerInfo *root,
 
 	/* Adjust costing for parallelism, if used. */
 	if (path->parallel_workers > 0)
-	{
-		double		parallel_divisor = path->parallel_workers;
-		double		leader_contribution;
-
-		/*
-		 * Early experience with parallel query suggests that when there is
-		 * only one worker, the leader often makes a very substantial
-		 * contribution to executing the parallel portion of the plan, but as
-		 * more workers are added, it does less and less, because it's busy
-		 * reading tuples from the workers and doing whatever non-parallel
-		 * post-processing is needed.  By the time we reach 4 workers, the
-		 * leader no longer makes a meaningful contribution.  Thus, for now,
-		 * estimate that the leader spends 30% of its time servicing each
-		 * worker, and the remainder executing the parallel plan.
-		 */
-		leader_contribution = 1.0 - (0.3 * path->parallel_workers);
-		if (leader_contribution > 0)
-			parallel_divisor += leader_contribution;
-
-		/*
-		 * In the case of a parallel plan, the row count needs to represent
-		 * the number of tuples processed per worker.  Otherwise, higher-level
-		 * plan nodes that appear below the gather will be costed incorrectly,
-		 * because they'll anticipate receiving more rows than any given copy
-		 * will actually get.
-		 */
-		path->rows = clamp_row_est(path->rows / parallel_divisor);
-
-		/* The CPU cost is divided among all the workers. */
-		cpu_run_cost /= parallel_divisor;
-
-		/*
-		 * It may be possible to amortize some of the I/O cost, but probably
-		 * not very much, because most operating systems already do aggressive
-		 * prefetching.  For now, we assume that the disk run cost can't be
-		 * amortized at all.
-		 */
-	}
+		cpu_run_cost = adjust_cost_for_parallelism(path, cpu_run_cost);
 
 	path->startup_cost = startup_cost;
 	path->total_cost = startup_cost + cpu_run_cost + disk_run_cost;
@@ -831,10 +795,10 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
 	Cost		indexTotalCost;
-	Selectivity indexSelectivity;
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -855,13 +819,12 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	if (!enable_bitmapscan)
 		startup_cost += disable_cost;
 
-	/*
-	 * Fetch total cost of obtaining the bitmap, as well as its total
-	 * selectivity.
-	 */
-	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+	pages_fetched = compute_bitmap_pages(root, baserel, bitmapqual,
+										 loop_count, &indexTotalCost,
+										 &tuples_fetched);
 
 	startup_cost += indexTotalCost;
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
 
 	/* Fetch estimated page costs for tablespace containing table. */
 	get_tablespace_page_costs(baserel->reltablespace,
@@ -869,41 +832,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 							  &spc_seq_page_cost);
 
 	/*
-	 * Estimate number of main-table pages fetched.
-	 */
-	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
-
-	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
-
-	if (loop_count > 1)
-	{
-		/*
-		 * For repeated bitmap scans, scale up the number of tuples fetched in
-		 * the Mackert and Lohman formula by the number of scans, so that we
-		 * estimate the number of pages fetched by all the scans. Then
-		 * pro-rate for one scan.
-		 */
-		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
-											baserel->pages,
-											get_indexpath_pages(bitmapqual),
-											root);
-		pages_fetched /= loop_count;
-	}
-	else
-	{
-		/*
-		 * For a single scan, the number of heap pages that need to be fetched
-		 * is the same as the Mackert and Lohman formula for the case T <= b
-		 * (ie, no re-reads needed).
-		 */
-		pages_fetched = (2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
-	}
-	if (pages_fetched >= T)
-		pages_fetched = T;
-	else
-		pages_fetched = ceil(pages_fetched);
-
-	/*
 	 * For small numbers of pages we should charge spc_random_page_cost
 	 * apiece, while if nearly all the table's pages are being read, it's more
 	 * appropriate to charge spc_seq_page_cost apiece.  The effect is
@@ -932,8 +860,13 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+		cpu_run_cost = adjust_cost_for_parallelism(path, cpu_run_cost);
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
@@ -944,6 +877,56 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 }
 
 /*
+ * adjust_cost_for_parallelism
+ *
+ * Adjust the cpu cost based on number of parallel workers, also update
+ * the number of rows processed at each worker.
+ */
+static Cost
+adjust_cost_for_parallelism(Path *path, Cost cpu_run_cost)
+{
+	double		parallel_divisor = path->parallel_workers;
+	double		leader_contribution;
+	Cost		cpu_cost = cpu_run_cost;
+
+	/*
+	 * Early experience with parallel query suggests that when there is
+	 * only one worker, the leader often makes a very substantial
+	 * contribution to executing the parallel portion of the plan, but as
+	 * more workers are added, it does less and less, because it's busy
+	 * reading tuples from the workers and doing whatever non-parallel
+	 * post-processing is needed.  By the time we reach 4 workers, the
+	 * leader no longer makes a meaningful contribution.  Thus, for now,
+	 * estimate that the leader spends 30% of its time servicing each
+	 * worker, and the remainder executing the parallel plan.
+	 */
+	leader_contribution = 1.0 - (0.3 * path->parallel_workers);
+	if (leader_contribution > 0)
+		parallel_divisor += leader_contribution;
+
+	/*
+	 * In the case of a parallel plan, the row count needs to represent
+	 * the number of tuples processed per worker.  Otherwise, higher-level
+	 * plan nodes that appear below the gather will be costed incorrectly,
+	 * because they'll anticipate receiving more rows than any given copy
+	 * will actually get.
+	 */
+	path->rows = clamp_row_est(path->rows / parallel_divisor);
+
+	/* The CPU cost is divided among all the workers. */
+	cpu_cost /= parallel_divisor;
+
+	/*
+	 * It may be possible to amortize some of the I/O cost, but probably
+	 * not very much, because most operating systems already do aggressive
+	 * prefetching.  For now, we assume that the disk run cost can't be
+	 * amortized at all.
+	 */
+
+	return cpu_cost;
+}
+
+/*
  * cost_bitmap_tree_node
  *		Extract cost and selectivity from a bitmap tree node (index/and/or)
  */
@@ -4764,3 +4747,69 @@ page_size(double tuples, int width)
 {
 	return ceil(relation_byte_size(tuples, width) / BLCKSZ);
 }
+
+/*
+ * compute_bitmap_pages
+ *
+ * compute number of pages fetched from heap in bitmap heap scan.
+ */
+double
+compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel, Path *bitmapqual,
+			int loop_count, Cost *cost, double *tuple)
+{
+	Cost		indexTotalCost;
+	Selectivity indexSelectivity;
+	double		T;
+	double		pages_fetched;
+	double		tuples_fetched;
+
+	/*
+	 * Fetch total cost of obtaining the bitmap, as well as its total
+	 * selectivity.
+	 */
+	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+
+	/*
+	 * Estimate number of main-table pages fetched.
+	 */
+	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
+
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
+
+	if (loop_count > 1)
+	{
+		/*
+		 * For repeated bitmap scans, scale up the number of tuples fetched in
+		 * the Mackert and Lohman formula by the number of scans, so that we
+		 * estimate the number of pages fetched by all the scans. Then
+		 * pro-rate for one scan.
+		 */
+		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
+											baserel->pages,
+											get_indexpath_pages(bitmapqual),
+											root);
+		pages_fetched /= loop_count;
+	}
+	else
+	{
+		/*
+		 * For a single scan, the number of heap pages that need to be fetched
+		 * is the same as the Mackert and Lohman formula for the case T <= b
+		 * (ie, no re-reads needed).
+		 */
+		pages_fetched =
+			(2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
+	}
+
+	if (pages_fetched >= T)
+		pages_fetched = T;
+	else
+		pages_fetched = ceil(pages_fetched);
+
+	if (cost)
+		*cost = indexTotalCost;
+	if (tuple)
+		*tuple = tuples_fetched;
+
+	return pages_fetched;
+}
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 2952bfb..1a7bde0 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index ad49674..0b544a1 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2803,7 +2803,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 6d3ccfd..b461c72 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3192,7 +3193,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index c7584cb..558a6c1 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3390,6 +3390,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 0d12bbb..ad9c693 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -178,6 +178,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index 0ed9c78..aab708d 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index cdf4104..2a47781 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -27,6 +27,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1413,6 +1415,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t		mutex;			/* mutual exclusion for below three fields */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+}	ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	ParallelIterator tbmiterator;
+	ParallelIterator prefetch_iterator;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	Size		ptbm_offset;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1427,6 +1495,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1443,6 +1513,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1458,7 +1529,11 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		is_leader		   is_leader is set true, if this worker become
+ *						   leader for parallel bitmap heap scan.
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1473,6 +1548,9 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	bool		is_leader;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 9303299..9a6a44a 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+} TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -62,5 +79,11 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+						   ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+						  ParallelTIDBitmap *parallel_tbm);
+void		tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..e1baacc 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -183,6 +183,8 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
 					   double cte_rows);
 extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
 extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern double compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel,
+				Path *bitmapqual, int loop_count, Cost *cost, double *tuple);
 
 /*
  * prototypes for clausesel.c
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..397d266 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 44abe83..3264f44 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 152ff06..a8f7928 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -785,6 +785,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
hash-support-alloc-free-v4.patchapplication/octet-stream; name=hash-support-alloc-free-v4.patchDownload
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 94cc59d..ba8a8be 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -314,7 +314,7 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
 	hashtable->in_hash_funcs = NULL;
 	hashtable->cur_eq_funcs = NULL;
 
-	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
+	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, NULL);
 	hashtable->hashtab->private_data = hashtable;
 
 	return hashtable;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 826fead..ea79de7 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -244,7 +244,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index 12aedbc..b9a05e6 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -66,6 +66,7 @@
 #define SH_STATUS_EMPTY SH_MAKE_NAME(EMPTY)
 #define SH_STATUS_IN_USE SH_MAKE_NAME(IN_USE)
 #define SH_ITERATOR SH_MAKE_NAME(iterator)
+#define SH_ALLOCATOR SH_MAKE_NAME(alloc)
 
 /* function declarations */
 #define SH_CREATE SH_MAKE_NAME(create)
@@ -90,6 +91,18 @@
 /* generate forward declarations necessary to use the hash table */
 #ifdef SH_DECLARE
 
+typedef struct SH_ALLOCATOR
+{
+	/* Allocation function */
+	void 	*(*HashAlloc) (Size size, void *args);
+
+	/* Free function */
+	void 	 (*HashFree) (void *pointer, void *args);
+
+	/* Arguments to be passed to alloc and free functions */
+	void 	*args;
+} SH_ALLOCATOR;
+
 /* type definitions */
 typedef struct SH_TYPE
 {
@@ -112,6 +125,9 @@ typedef struct SH_TYPE
 	/* hash buckets */
 	SH_ELEMENT_TYPE *data;
 
+	/* hash allocator */
+	SH_ALLOCATOR	*alloc;
+
 	/* memory context to use for allocations */
 	MemoryContext ctx;
 
@@ -133,7 +149,8 @@ typedef struct SH_ITERATOR
 } SH_ITERATOR;
 
 /* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
+							SH_ALLOCATOR *alloc);
 SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
 SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
 SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -281,7 +298,7 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
  * allocating required memory in the passed-in context.
  */
 SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements)
+SH_CREATE(MemoryContext ctx, uint32 nelements, SH_ALLOCATOR *alloc)
 {
 	SH_TYPE    *tb;
 	uint64		size;
@@ -294,9 +311,19 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 
 	SH_COMPUTE_PARAMETERS(tb, size);
 
-	tb->data = MemoryContextAllocExtended(tb->ctx,
-										  sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	tb->alloc = alloc;
+
+	/*
+	 * If allocation handle is passed then use allocation function from the
+	 * handle otherwise use standard allocator.
+	 */
+	if (tb->alloc != NULL)
+		tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+										tb->alloc->args);
+	else
+		tb->data = MemoryContextAllocExtended(tb->ctx,
+										sizeof(SH_ELEMENT_TYPE) * tb->size,
+										MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
 
 	return tb;
 }
@@ -333,9 +360,17 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 	/* compute parameters for new table */
 	SH_COMPUTE_PARAMETERS(tb, newsize);
 
-	tb->data = MemoryContextAllocExtended(
-								 tb->ctx, sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	/*
+	 * If allocation handle is valid then use allocation function from the
+	 * handle otherwise use standard allocator.
+	 */
+	if (tb->alloc != NULL)
+		tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+										tb->alloc->args);
+	else
+		tb->data = MemoryContextAllocExtended(tb->ctx,
+										sizeof(SH_ELEMENT_TYPE) * tb->size,
+										MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
 
 	newdata = tb->data;
 
@@ -421,7 +456,11 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 		}
 	}
 
-	pfree(olddata);
+	if (tb->alloc == NULL)
+		pfree(olddata);
+	else
+		tb->alloc->HashFree(olddata, tb->alloc->args);
+
 }
 
 /*
#26Dilip Kumar
dilipbalaut@gmail.com
In reply to: Dilip Kumar (#25)
2 attachment(s)
Re: Parallel bitmap heap scan

Rebased on the current head.

On Tue, Dec 13, 2016 at 12:06 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Sat, Dec 10, 2016 at 5:36 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:

Few assorted comments:

Thanks for the review

1.
+ else if (needWait)
+ {
+ /* Add ourself to wait queue */
+ ConditionVariablePrepareToSleep(&pbminfo->cv);
+ queuedSelf = true;
+ }

With the committed version of condition variables, you can avoid
calling ConditionVariablePrepareToSleep(). Refer latest parallel
index scan patch [1].

Oh, I see, Fixed.

2.
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader to complete the TidBitmap.</entry>
+        </row>
+        <row>

Isn't it better to write it as below:

Waiting for the leader backend to form the TidBitmap.

Done this way..

3.
+ * Update snpashot info in heap scan descriptor.

/snpashot/snapshot

Fixed

4.
#include "utils/tqual.h"
-
+#include "miscadmin.h"
static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
-
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
TBMIterateResult *tbmres;
-
+ ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
static int tbm_comparator(const void *left, const void *right);
-
+void * tbm_alloc_shared(Size size, void *arg);

It seems line deletes at above places are spurious. Please check for
similar occurrences at other places in patch.

Fixed

5.
+ bool is_shared; /* need to build shared tbm if set*/

space is required towards the end of the comment (set */).

Fixed

6.
+ /*
+ * If we have shared TBM means we are running in parallel mode.
+ * So directly convert dsa array to page and chunk array.
+ */

I think the above comment can be simplified as "For shared TBM,
directly convert dsa array to page and chunk array"

Done this way..

7.
+ dsa_entry = (void*)(((char*)dsa_entry) + sizeof(dsa_pointer));

extra space is missing at multiple places in above line. It should be
written as below:

dsa_entry = (void *)(((char *) dsa_entry) + sizeof(dsa_pointer));

Fixed..

Similar stuff needs to be taken care at other places in the patch as
well. I think it will be better if you run pgindent on your patch.

I have run the pgindent, and taken all the changes whichever was
applicable for added code.

There are some cleanup for "hash-support-alloc-free" also, so
attaching a new patch for this as well.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

hash-support-alloc-free-v5.patchapplication/octet-stream; name=hash-support-alloc-free-v5.patchDownload
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 3149fbe..055b1ca 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -330,7 +330,7 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
 	else
 		hashtable->hash_iv = 0;
 
-	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
+	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, NULL);
 	hashtable->hashtab->private_data = hashtable;
 
 	return hashtable;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 826fead..ea79de7 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -244,7 +244,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index 12aedbc..b9a05e6 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -66,6 +66,7 @@
 #define SH_STATUS_EMPTY SH_MAKE_NAME(EMPTY)
 #define SH_STATUS_IN_USE SH_MAKE_NAME(IN_USE)
 #define SH_ITERATOR SH_MAKE_NAME(iterator)
+#define SH_ALLOCATOR SH_MAKE_NAME(alloc)
 
 /* function declarations */
 #define SH_CREATE SH_MAKE_NAME(create)
@@ -90,6 +91,18 @@
 /* generate forward declarations necessary to use the hash table */
 #ifdef SH_DECLARE
 
+typedef struct SH_ALLOCATOR
+{
+	/* Allocation function */
+	void 	*(*HashAlloc) (Size size, void *args);
+
+	/* Free function */
+	void 	 (*HashFree) (void *pointer, void *args);
+
+	/* Arguments to be passed to alloc and free functions */
+	void 	*args;
+} SH_ALLOCATOR;
+
 /* type definitions */
 typedef struct SH_TYPE
 {
@@ -112,6 +125,9 @@ typedef struct SH_TYPE
 	/* hash buckets */
 	SH_ELEMENT_TYPE *data;
 
+	/* hash allocator */
+	SH_ALLOCATOR	*alloc;
+
 	/* memory context to use for allocations */
 	MemoryContext ctx;
 
@@ -133,7 +149,8 @@ typedef struct SH_ITERATOR
 } SH_ITERATOR;
 
 /* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
+							SH_ALLOCATOR *alloc);
 SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
 SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
 SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -281,7 +298,7 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
  * allocating required memory in the passed-in context.
  */
 SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements)
+SH_CREATE(MemoryContext ctx, uint32 nelements, SH_ALLOCATOR *alloc)
 {
 	SH_TYPE    *tb;
 	uint64		size;
@@ -294,9 +311,19 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 
 	SH_COMPUTE_PARAMETERS(tb, size);
 
-	tb->data = MemoryContextAllocExtended(tb->ctx,
-										  sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	tb->alloc = alloc;
+
+	/*
+	 * If allocation handle is passed then use allocation function from the
+	 * handle otherwise use standard allocator.
+	 */
+	if (tb->alloc != NULL)
+		tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+										tb->alloc->args);
+	else
+		tb->data = MemoryContextAllocExtended(tb->ctx,
+										sizeof(SH_ELEMENT_TYPE) * tb->size,
+										MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
 
 	return tb;
 }
@@ -333,9 +360,17 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 	/* compute parameters for new table */
 	SH_COMPUTE_PARAMETERS(tb, newsize);
 
-	tb->data = MemoryContextAllocExtended(
-								 tb->ctx, sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	/*
+	 * If allocation handle is valid then use allocation function from the
+	 * handle otherwise use standard allocator.
+	 */
+	if (tb->alloc != NULL)
+		tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+										tb->alloc->args);
+	else
+		tb->data = MemoryContextAllocExtended(tb->ctx,
+										sizeof(SH_ELEMENT_TYPE) * tb->size,
+										MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
 
 	newdata = tb->data;
 
@@ -421,7 +456,11 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 		}
 	}
 
-	pfree(olddata);
+	if (tb->alloc == NULL)
+		pfree(olddata);
+	else
+		tb->alloc->HashFree(olddata, tb->alloc->args);
+
 }
 
 /*
parallel-bitmap-heap-scan-v5.patchapplication/octet-stream; name=parallel-bitmap-heap-scan-v5.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1545f03..cf2dbbd 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1232,6 +1232,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index b019bc1..e99ab66 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 8a6f844..7aea080 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -203,6 +204,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -255,6 +260,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -732,6 +742,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index 449aacb..5d63459 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,16 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+					  ParallelIterator *parallel_iterator);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -68,11 +74,18 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
 
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (pbminfo)
+		parallel_tbm = (ParallelTIDBitmap *) ((char *) pbminfo +
+											  pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,6 +100,35 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	prefetch_iterator = node->prefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	First worker to see the state as parallel bitmap info as PBM_INITIAL
+	 *	become leader and set the state to PBM_INPROGRESS All other workers
+	 *	see the state as PBM_INPROGRESS will wait for leader to complete the
+	 *	TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *	  ParallelBitmapInfo so that other worker can see those.
+	 *	  set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader create shared TIDBitmap.
+	 *	  copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over page and chunk array
+	 *		 and select heap pages one by one. If prefetch is enable then
+	 *		 there will be two iterator.
+	 *	  b) Since multiple worker are iterating over same page and chunk
+	 *		 array we need to have a shared iterator, so we grab a spin
+	 *		 lock and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,7 +143,41 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.
+		 *
+		 * In parallel mode leader will immediately come out of the function,
+		 * but all other worker will be blocked until leader wake them up.
+		 */
+		if (pbminfo == NULL || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag in lower level bitmap index scan. Later
+			 * bitmap index node will use this flag to indicate tidbitmap that
+			 * it needs to create an shared page table.
+			 */
+			if (pbminfo)
+			{
+				node->is_leader = true;
+				pbms_set_parallel(outerPlanState(node));
+			}
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM. Here we
+			 * need to create a local TBM and copy information from shared
+			 * location. We also need to attach to shared page table using
+			 * hash table handle stored in parallel_tbm(shared memory).
+			 */
+			tbm = tbm_create(work_mem * 1024L);
+			tbm_set_parallel(tbm, node->ss.ps.state->es_query_dsa);
+			tbm_restore_shared_members(tbm, parallel_tbm);
+		}
 
 		if (!tbm || !IsA(tbm, TIDBitmap))
 			elog(ERROR, "unrecognized result from subplan");
@@ -118,19 +194,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			node->prefetch_target = -1;
 		}
 #endif   /* USE_PREFETCH */
+
+		/*
+		 * If we are in parallel mode and we are leader worker then copy the
+		 * local TBM information to shared location, and wake up other
+		 * workers.
+		 */
+		if (node->is_leader)
+		{
+			/*
+			 * copy TBM local information in shared memory before waking up
+			 * other workers. Other workers will create there own TBM and copy
+			 * information from shared memory.
+			 */
+			tbm_update_shared_members(tbm, parallel_tbm);
+
+			/* Change the state under a lock */
+			SpinLockAcquire(&pbminfo->state_mutex);
+			pbminfo->state = PBM_FINISHED;
+			SpinLockRelease(&pbminfo->state_mutex);
+
+			/* Wake up all other workers. */
+			ConditionVariableBroadcast(&pbminfo->cv);
+		}
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+									 pbminfo ? &pbminfo->tbmiterator : NULL);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +239,43 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (pbminfo)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+							   pbminfo ? &pbminfo->prefetch_iterator : NULL);
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not
+				 * ensure that current blockno in main iterator and prefetch
+				 * iterator is same. It's possible that whatever blockno we
+				 * are prefetching is getting processed by some other worker.
+				 */
+				if ((pbminfo == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +309,39 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				int		   *prefetch_target;
+
+				if (pbminfo == NULL)
+					prefetch_target = &node->prefetch_target;
+				else
+				{
+					prefetch_target = &pbminfo->prefetch_target;
+					/* If we are in parallel mode then grab prefetch_mutex */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+				}
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +357,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (pbminfo == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+					pbminfo->prefetch_target++;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +396,52 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+			TBMIterator *iterator = node->prefetch_iterator;
+			ParallelIterator *parallel_iteartor = NULL;
+			ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (pbminfo)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				parallel_iteartor = &pbminfo->prefetch_iterator;
+				prefetch_pages = &pbminfo->prefetch_pages;
+				prefetch_target = pbminfo->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
 
+			while (*prefetch_pages < prefetch_target)
+			{
+				TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+														  parallel_iteartor);
 				if (tbmpre == NULL)
 				{
 					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					tbm_end_iterate(iterator);
+					prefetch_iterator = node->prefetch_iterator = NULL;
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +656,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +774,8 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->is_leader = false;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -653,3 +862,219 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*
+		 * Check the current state
+		 *
+		 * If state is
+		 * PBM_INITIAL    -> then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS -> then we need to wait till leader create BITMAP
+		 * PBM_FINISHED   -> BITMAP is ready so no need to wait.
+		 *
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Wrapper over tbm_iterate, which take care of copying and restoring iterator
+ * variables from parallel iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator, ParallelIterator *piterator)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (piterator == NULL)
+		return tbm_iterate(iterator);
+
+	/* We are in parallel mode so grab parallel iterator mutex */
+	SpinLockAcquire(&piterator->mutex);
+
+	/*
+	 * Copy information from shared location to local iterator and call
+	 * iterate. Then restore back to shared iterator.
+	 */
+	iterator->spageptr = piterator->spageptr;
+	iterator->schunkptr = piterator->schunkptr;
+	iterator->schunkbit = piterator->schunkbit;
+	output = tbm_iterate(iterator);
+	piterator->spageptr = iterator->spageptr;
+	piterator->schunkptr = iterator->schunkptr;
+	piterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&piterator->mutex);
+
+	return output;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node as
+	 * parallel, because only first node will create the main bitmap other
+	 * bitmaps will be merged to the first bitmap so no need to create them in
+	 * shared memory.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState *) node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index a364098..8acb927 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate
+	 * that we need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_dsa);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index ea79de7..41292f8 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -43,6 +43,13 @@
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
 #include "nodes/tidbitmap.h"
+#include "utils/hsearch.h"
+#include "utils/dsa.h"
+#include "storage/condition_variable.h"
+#include "storage/spin.h"
+#include "storage/lwlock.h"
+#include "nodes/execnodes.h"
+#include "storage/shmem.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -139,21 +146,26 @@ struct TIDBitmap
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	bool		is_shared;		/* need to build shared tbm if set */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	struct pagetable_alloc *allocator;	/* shared memory allocator */
+	int			dsa_entries;	/* actual allocated entries */
+	dsa_area   *area;			/* reference to per-query shared memory area */
 };
 
 /*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
  */
-struct TBMIterator
+struct ParallelTIDBitmap
 {
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+	dsa_pointer dsa_data;		/* dsa pointers for all kind of pages */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	int			dsa_entries;	/* total item in dsa_pages */
+	bool		inited;			/* set true after leader converts page */
+	/* table to dsa_pointer's array. */
 };
 
 
@@ -168,6 +180,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+void	   *tbm_alloc_shared(Size size, void *arg);
+void		tbm_free_shared(void *pointer, void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -231,6 +245,8 @@ tbm_create(long maxbytes)
 	tbm->maxentries = (int) nbuckets;
 	tbm->lossify_start = 0;
 
+	tbm->allocator = palloc(sizeof(pagetable_alloc));
+
 	return tbm;
 }
 
@@ -244,7 +260,16 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	if (tbm->is_shared)
+	{
+		tbm->allocator->args = tbm;
+		tbm->allocator->HashAlloc = tbm_alloc_shared;
+		tbm->allocator->HashFree = tbm_free_shared;
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, tbm->allocator);
+		tbm->dsa_entries = tbm->pagetable->size;
+	}
+	else
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -271,7 +296,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 void
 tbm_free(TIDBitmap *tbm)
 {
-	if (tbm->pagetable)
+	if (!tbm->is_shared && tbm->pagetable)
 		pagetable_destroy(tbm->pagetable);
 	if (tbm->spages)
 		pfree(tbm->spages);
@@ -612,11 +637,16 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	iterator->tbm = tbm;
 
 	/*
-	 * Initialize iteration pointers.
+	 * Initialize iteration pointers, only if it's not shared tbm. For shared
+	 * TBM, we will copy these values from shared iterator before calling
+	 * tbm_iterate.
 	 */
-	iterator->spageptr = 0;
-	iterator->schunkptr = 0;
-	iterator->schunkbit = 0;
+	if (!tbm->is_shared)
+	{
+		iterator->spageptr = 0;
+		iterator->schunkptr = 0;
+		iterator->schunkbit = 0;
+	}
 
 	/*
 	 * If we have a hashtable, create and fill the sorted page lists, unless
@@ -626,7 +656,7 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	 */
 	if (tbm->status == TBM_HASH && !tbm->iterating)
 	{
-		pagetable_iterator i;
+		pagetable_iterator itr;
 		PagetableEntry *page;
 		int			npages;
 		int			nchunks;
@@ -640,15 +670,48 @@ tbm_begin_iterate(TIDBitmap *tbm)
 				MemoryContextAlloc(tbm->mcxt,
 								   tbm->nchunks * sizeof(PagetableEntry *));
 
-		npages = nchunks = 0;
-		pagetable_start_iterate(tbm->pagetable, &i);
-		while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+		/*
+		 * For shared TBM, directly convert dsa array to page and chunk array.
+		 */
+		if (tbm->is_shared)
 		{
-			if (page->ischunk)
-				tbm->schunks[nchunks++] = page;
-			else
-				tbm->spages[npages++] = page;
+			PagetableEntry *dsa_entry;
+			int			i;
+
+			/*
+			 * This step will be done by all the workers including leader.
+			 * Here we need to convert array of dsa pointers to local page and
+			 * chunk array.
+			 */
+			dsa_entry = dsa_get_address(tbm->area, tbm->dsa_data);
+			dsa_entry = (void *) (((char *) dsa_entry) + sizeof(dsa_pointer));
+			npages = nchunks = 0;
+			for (i = 0; i < tbm->dsa_entries; i++)
+			{
+				page = (PagetableEntry *) (dsa_entry + i);
+
+				if (page->status != pagetable_IN_USE)
+					continue;
+
+				if (page->ischunk)
+					tbm->schunks[nchunks++] = page;
+				else
+					tbm->spages[npages++] = page;
+			}
+		}
+		else
+		{
+			npages = nchunks = 0;
+			pagetable_start_iterate(tbm->pagetable, &itr);
+			while ((page = pagetable_iterate(tbm->pagetable, &itr)) != NULL)
+			{
+				if (page->ischunk)
+					tbm->schunks[nchunks++] = page;
+				else
+					tbm->spages[npages++] = page;
+			}
 		}
+
 		Assert(npages == tbm->npages);
 		Assert(nchunks == tbm->nchunks);
 		if (npages > 1)
@@ -1061,3 +1124,114 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_update_shared_members
+ *
+ * Restore leaders private tbm state to shared location. This must
+ * be called before waking up the other worker.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before waking up the other
+	 * workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->dsa_data = tbm->dsa_data;
+	parallel_tbm->dsa_entries = tbm->dsa_entries;
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->is_shared = true;
+	tbm->area = (dsa_area *) area;
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size
+tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
+
+/*
+ * tbm_restore_shared_members
+ *
+ * Attach worker to shared TID bitmap created by leader worker.
+ */
+void
+tbm_restore_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *stbm)
+{
+	tbm->status = TBM_HASH;
+	tbm->nchunks = stbm->nchunks;
+	tbm->nentries = stbm->nentries;
+	tbm->npages = stbm->npages;
+	tbm->maxentries = stbm->maxentries;
+	tbm->dsa_data = stbm->dsa_data;
+	tbm->dsa_entries = stbm->dsa_entries;
+	tbm->is_shared = true;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Memory allocation function for bitmap hash.
+ * It allocated memory from DSA and also stores dsa_pointer in the memory
+ * so that during free we can directly get the dsa_pointe and free it.
+ */
+void *
+tbm_alloc_shared(Size size, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->area, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	/* Keep track of actual number of entries */
+	if (tbm->pagetable)
+		tbm->dsa_entries = tbm->pagetable->size;
+
+	ptr = dsa_get_address(tbm->area, dsaptr);
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * tbm_free_shared
+ *
+ * Memory free function for tid bitmap
+ */
+void
+tbm_free_shared(void *pointer, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+
+	dsa_pointer dsa_data =
+			*((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+
+	dsa_free(tbm->area, dsa_data);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 9753a26..a51b11d 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -126,6 +126,7 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static int	compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 
 
 /*
@@ -678,49 +679,7 @@ create_plain_partial_paths(PlannerInfo *root, RelOptInfo *rel)
 {
 	int			parallel_workers;
 
-	/*
-	 * If the user has set the parallel_workers reloption, use that; otherwise
-	 * select a default number of workers.
-	 */
-	if (rel->rel_parallel_workers != -1)
-		parallel_workers = rel->rel_parallel_workers;
-	else
-	{
-		int			parallel_threshold;
-
-		/*
-		 * If this relation is too small to be worth a parallel scan, just
-		 * return without doing anything ... unless it's an inheritance child.
-		 * In that case, we want to generate a parallel path here anyway.  It
-		 * might not be worthwhile just for this relation, but when combined
-		 * with all of its inheritance siblings it may well pay off.
-		 */
-		if (rel->pages < (BlockNumber) min_parallel_relation_size &&
-			rel->reloptkind == RELOPT_BASEREL)
-			return;
-
-		/*
-		 * Select the number of workers based on the log of the size of the
-		 * relation.  This probably needs to be a good deal more
-		 * sophisticated, but we need something here for now.  Note that the
-		 * upper limit of the min_parallel_relation_size GUC is chosen to
-		 * prevent overflow here.
-		 */
-		parallel_workers = 1;
-		parallel_threshold = Max(min_parallel_relation_size, 1);
-		while (rel->pages >= (BlockNumber) (parallel_threshold * 3))
-		{
-			parallel_workers++;
-			parallel_threshold *= 3;
-			if (parallel_threshold > INT_MAX / 3)
-				break;			/* avoid overflow */
-		}
-	}
-
-	/*
-	 * In no case use more than max_parallel_workers_per_gather workers.
-	 */
-	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+	parallel_workers = compute_parallel_worker(rel, rel->pages);
 
 	/* If any limit was set to zero, the user doesn't want a parallel scan. */
 	if (parallel_workers <= 0)
@@ -2866,6 +2825,84 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+static int
+compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
+{
+	int			parallel_workers;
+
+	/*
+	 * If the user has set the parallel_workers reloption, use that; otherwise
+	 * select a default number of workers.
+	 */
+	if (rel->rel_parallel_workers != -1)
+		parallel_workers = rel->rel_parallel_workers;
+	else
+	{
+		int			parallel_threshold;
+
+		/*
+		 * If this relation is too small to be worth a parallel scan, just
+		 * return without doing anything ... unless it's an inheritance child.
+		 * In that case, we want to generate a parallel path here anyway.  It
+		 * might not be worthwhile just for this relation, but when combined
+		 * with all of its inheritance siblings it may well pay off.
+		 */
+		if (pages < (BlockNumber) min_parallel_relation_size &&
+			rel->reloptkind == RELOPT_BASEREL)
+			return 0;
+
+		/*
+		 * Select the number of workers based on the log of the size of the
+		 * relation.  This probably needs to be a good deal more
+		 * sophisticated, but we need something here for now.  Note that the
+		 * upper limit of the min_parallel_relation_size GUC is chosen to
+		 * prevent overflow here.
+		 */
+		parallel_workers = 1;
+		parallel_threshold = Max(min_parallel_relation_size, 1);
+		while (pages >= (BlockNumber) (parallel_threshold * 3))
+		{
+			parallel_workers++;
+			parallel_threshold *= 3;
+			if (parallel_threshold > INT_MAX / 3)
+				break;			/* avoid overflow */
+		}
+	}
+
+	/*
+	 * In no case use more than max_parallel_workers_per_gather workers.
+	 */
+	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+
+	return parallel_workers;
+}
+
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 415edad..3e0adb5 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -161,6 +161,7 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
 static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
 static double relation_byte_size(double tuples, int width);
 static double page_size(double tuples, int width);
+static Cost adjust_cost_for_parallelism(Path *path, Cost cpu_run_cost);
 
 
 /*
@@ -237,44 +238,7 @@ cost_seqscan(Path *path, PlannerInfo *root,
 
 	/* Adjust costing for parallelism, if used. */
 	if (path->parallel_workers > 0)
-	{
-		double		parallel_divisor = path->parallel_workers;
-		double		leader_contribution;
-
-		/*
-		 * Early experience with parallel query suggests that when there is
-		 * only one worker, the leader often makes a very substantial
-		 * contribution to executing the parallel portion of the plan, but as
-		 * more workers are added, it does less and less, because it's busy
-		 * reading tuples from the workers and doing whatever non-parallel
-		 * post-processing is needed.  By the time we reach 4 workers, the
-		 * leader no longer makes a meaningful contribution.  Thus, for now,
-		 * estimate that the leader spends 30% of its time servicing each
-		 * worker, and the remainder executing the parallel plan.
-		 */
-		leader_contribution = 1.0 - (0.3 * path->parallel_workers);
-		if (leader_contribution > 0)
-			parallel_divisor += leader_contribution;
-
-		/*
-		 * In the case of a parallel plan, the row count needs to represent
-		 * the number of tuples processed per worker.  Otherwise, higher-level
-		 * plan nodes that appear below the gather will be costed incorrectly,
-		 * because they'll anticipate receiving more rows than any given copy
-		 * will actually get.
-		 */
-		path->rows = clamp_row_est(path->rows / parallel_divisor);
-
-		/* The CPU cost is divided among all the workers. */
-		cpu_run_cost /= parallel_divisor;
-
-		/*
-		 * It may be possible to amortize some of the I/O cost, but probably
-		 * not very much, because most operating systems already do aggressive
-		 * prefetching.  For now, we assume that the disk run cost can't be
-		 * amortized at all.
-		 */
-	}
+		cpu_run_cost = adjust_cost_for_parallelism(path, cpu_run_cost);
 
 	path->startup_cost = startup_cost;
 	path->total_cost = startup_cost + cpu_run_cost + disk_run_cost;
@@ -831,10 +795,10 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
 	Cost		indexTotalCost;
-	Selectivity indexSelectivity;
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -855,13 +819,12 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	if (!enable_bitmapscan)
 		startup_cost += disable_cost;
 
-	/*
-	 * Fetch total cost of obtaining the bitmap, as well as its total
-	 * selectivity.
-	 */
-	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+	pages_fetched = compute_bitmap_pages(root, baserel, bitmapqual,
+										 loop_count, &indexTotalCost,
+										 &tuples_fetched);
 
 	startup_cost += indexTotalCost;
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
 
 	/* Fetch estimated page costs for tablespace containing table. */
 	get_tablespace_page_costs(baserel->reltablespace,
@@ -869,41 +832,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 							  &spc_seq_page_cost);
 
 	/*
-	 * Estimate number of main-table pages fetched.
-	 */
-	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
-
-	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
-
-	if (loop_count > 1)
-	{
-		/*
-		 * For repeated bitmap scans, scale up the number of tuples fetched in
-		 * the Mackert and Lohman formula by the number of scans, so that we
-		 * estimate the number of pages fetched by all the scans. Then
-		 * pro-rate for one scan.
-		 */
-		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
-											baserel->pages,
-											get_indexpath_pages(bitmapqual),
-											root);
-		pages_fetched /= loop_count;
-	}
-	else
-	{
-		/*
-		 * For a single scan, the number of heap pages that need to be fetched
-		 * is the same as the Mackert and Lohman formula for the case T <= b
-		 * (ie, no re-reads needed).
-		 */
-		pages_fetched = (2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
-	}
-	if (pages_fetched >= T)
-		pages_fetched = T;
-	else
-		pages_fetched = ceil(pages_fetched);
-
-	/*
 	 * For small numbers of pages we should charge spc_random_page_cost
 	 * apiece, while if nearly all the table's pages are being read, it's more
 	 * appropriate to charge spc_seq_page_cost apiece.  The effect is
@@ -932,8 +860,13 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+		cpu_run_cost = adjust_cost_for_parallelism(path, cpu_run_cost);
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
@@ -944,6 +877,56 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 }
 
 /*
+ * adjust_cost_for_parallelism
+ *
+ * Adjust the cpu cost based on number of parallel workers, also update
+ * the number of rows processed at each worker.
+ */
+static Cost
+adjust_cost_for_parallelism(Path *path, Cost cpu_run_cost)
+{
+	double		parallel_divisor = path->parallel_workers;
+	double		leader_contribution;
+	Cost		cpu_cost = cpu_run_cost;
+
+	/*
+	 * Early experience with parallel query suggests that when there is
+	 * only one worker, the leader often makes a very substantial
+	 * contribution to executing the parallel portion of the plan, but as
+	 * more workers are added, it does less and less, because it's busy
+	 * reading tuples from the workers and doing whatever non-parallel
+	 * post-processing is needed.  By the time we reach 4 workers, the
+	 * leader no longer makes a meaningful contribution.  Thus, for now,
+	 * estimate that the leader spends 30% of its time servicing each
+	 * worker, and the remainder executing the parallel plan.
+	 */
+	leader_contribution = 1.0 - (0.3 * path->parallel_workers);
+	if (leader_contribution > 0)
+		parallel_divisor += leader_contribution;
+
+	/*
+	 * In the case of a parallel plan, the row count needs to represent
+	 * the number of tuples processed per worker.  Otherwise, higher-level
+	 * plan nodes that appear below the gather will be costed incorrectly,
+	 * because they'll anticipate receiving more rows than any given copy
+	 * will actually get.
+	 */
+	path->rows = clamp_row_est(path->rows / parallel_divisor);
+
+	/* The CPU cost is divided among all the workers. */
+	cpu_cost /= parallel_divisor;
+
+	/*
+	 * It may be possible to amortize some of the I/O cost, but probably
+	 * not very much, because most operating systems already do aggressive
+	 * prefetching.  For now, we assume that the disk run cost can't be
+	 * amortized at all.
+	 */
+
+	return cpu_cost;
+}
+
+/*
  * cost_bitmap_tree_node
  *		Extract cost and selectivity from a bitmap tree node (index/and/or)
  */
@@ -4798,3 +4781,69 @@ page_size(double tuples, int width)
 {
 	return ceil(relation_byte_size(tuples, width) / BLCKSZ);
 }
+
+/*
+ * compute_bitmap_pages
+ *
+ * compute number of pages fetched from heap in bitmap heap scan.
+ */
+double
+compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel, Path *bitmapqual,
+			int loop_count, Cost *cost, double *tuple)
+{
+	Cost		indexTotalCost;
+	Selectivity indexSelectivity;
+	double		T;
+	double		pages_fetched;
+	double		tuples_fetched;
+
+	/*
+	 * Fetch total cost of obtaining the bitmap, as well as its total
+	 * selectivity.
+	 */
+	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+
+	/*
+	 * Estimate number of main-table pages fetched.
+	 */
+	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
+
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
+
+	if (loop_count > 1)
+	{
+		/*
+		 * For repeated bitmap scans, scale up the number of tuples fetched in
+		 * the Mackert and Lohman formula by the number of scans, so that we
+		 * estimate the number of pages fetched by all the scans. Then
+		 * pro-rate for one scan.
+		 */
+		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
+											baserel->pages,
+											get_indexpath_pages(bitmapqual),
+											root);
+		pages_fetched /= loop_count;
+	}
+	else
+	{
+		/*
+		 * For a single scan, the number of heap pages that need to be fetched
+		 * is the same as the Mackert and Lohman formula for the case T <= b
+		 * (ie, no re-reads needed).
+		 */
+		pages_fetched =
+			(2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
+	}
+
+	if (pages_fetched >= T)
+		pages_fetched = T;
+	else
+		pages_fetched = ceil(pages_fetched);
+
+	if (cost)
+		*cost = indexTotalCost;
+	if (tuple)
+		*tuple = tuples_fetched;
+
+	return pages_fetched;
+}
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 2952bfb..1a7bde0 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index ad49674..0b544a1 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2803,7 +2803,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 6d3ccfd..b461c72 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3192,7 +3193,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 61e6a2c..391413f 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3386,6 +3386,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 0d12bbb..ad9c693 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -178,6 +178,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index 0ed9c78..aab708d 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 5c3b868..eca8eea 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1413,6 +1415,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t		mutex;			/* mutual exclusion for below three fields */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+}	ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	ParallelIterator tbmiterator;
+	ParallelIterator prefetch_iterator;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	Size		ptbm_offset;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1427,6 +1495,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1443,6 +1513,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1458,7 +1529,11 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		is_leader		   is_leader is set true, if this worker become
+ *						   leader for parallel bitmap heap scan.
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1473,6 +1548,9 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	bool		is_leader;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 9303299..9a6a44a 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+} TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -62,5 +79,11 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+						   ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+						  ParallelTIDBitmap *parallel_tbm);
+void		tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..e1baacc 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -183,6 +183,8 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
 					   double cte_rows);
 extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
 extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern double compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel,
+				Path *bitmapqual, int loop_count, Cost *cost, double *tuple);
 
 /*
  * prototypes for clausesel.c
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..397d266 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 44abe83..3264f44 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 282f8ae..44eb80e 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -784,6 +784,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#27Andres Freund
andres@anarazel.de
In reply to: Dilip Kumar (#26)
Re: Parallel bitmap heap scan

Hi,

On 2016-12-21 10:36:13 +0530, Dilip Kumar wrote:

diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index 12aedbc..b9a05e6 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -66,6 +66,7 @@
#define SH_STATUS_EMPTY SH_MAKE_NAME(EMPTY)
#define SH_STATUS_IN_USE SH_MAKE_NAME(IN_USE)
#define SH_ITERATOR SH_MAKE_NAME(iterator)
+#define SH_ALLOCATOR SH_MAKE_NAME(alloc)

/* function declarations */
#define SH_CREATE SH_MAKE_NAME(create)
@@ -90,6 +91,18 @@
/* generate forward declarations necessary to use the hash table */
#ifdef SH_DECLARE

+typedef struct SH_ALLOCATOR
+{
+	/* Allocation function */
+	void 	*(*HashAlloc) (Size size, void *args);
+
+	/* Free function */
+	void 	 (*HashFree) (void *pointer, void *args);
+
+	/* Arguments to be passed to alloc and free functions */
+	void 	*args;
+} SH_ALLOCATOR;
+
/* type definitions */
typedef struct SH_TYPE
{
@@ -112,6 +125,9 @@ typedef struct SH_TYPE
/* hash buckets */
SH_ELEMENT_TYPE *data;
+	/* hash allocator */
+	SH_ALLOCATOR	*alloc;
+
/* memory context to use for allocations */
MemoryContext ctx;

@@ -133,7 +149,8 @@ typedef struct SH_ITERATOR
} SH_ITERATOR;

/* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
+							SH_ALLOCATOR *alloc);
SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -281,7 +298,7 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
* allocating required memory in the passed-in context.
*/
SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements)
+SH_CREATE(MemoryContext ctx, uint32 nelements, SH_ALLOCATOR *alloc)
{
SH_TYPE    *tb;
uint64		size;
@@ -294,9 +311,19 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)

SH_COMPUTE_PARAMETERS(tb, size);

-	tb->data = MemoryContextAllocExtended(tb->ctx,
-										  sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	tb->alloc = alloc;
+
+	/*
+	 * If allocation handle is passed then use allocation function from the
+	 * handle otherwise use standard allocator.
+	 */
+	if (tb->alloc != NULL)
+		tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+										tb->alloc->args);
+	else
+		tb->data = MemoryContextAllocExtended(tb->ctx,
+										sizeof(SH_ELEMENT_TYPE) * tb->size,
+										MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);

If we go there, it seems better to also wrap the memory context based approach
in the allocator. This also needs docs, including a warning that just
using an allocator in shared memory does *NOT* allow the hashtable to be
used in shared memory in the general case.

Greetings,

Andres Freund

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#28Dilip Kumar
dilipbalaut@gmail.com
In reply to: Andres Freund (#27)
Re: Parallel bitmap heap scan

On Wed, Dec 21, 2016 at 3:23 PM, Andres Freund <andres@anarazel.de> wrote:

Sorry for the delayed response.

If we go there, it seems better to also wrap the memory context based approach
in the allocator.

One option is we can keep default allocator in the simple hash, and if
the caller doesn't supply any functions we can use default functions
as I have shown below.

+/* default allocator function */
+static void *
+SH_DEFAULT_ALLOC(Size size, void *args)
+{
+ MemoryContext context = (MemoryContext) args;
+
+ return MemoryContextAllocExtended(context, size,
+  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+}
+
+/* default allocator function */
+static void
+SH_DEFAULT_FREE(void *pointer, void *args)
+{
+ pfree(pointer);
+}
+
+SH_CREATE(MemoryContext ctx, uint32 nelements, SH_ALLOCATOR *alloc)
 {
- SH_TYPE    *tb;
- uint64 size;
+ SH_TYPE     *tb;
+ uint64 size;

tb = MemoryContextAllocZero(ctx, sizeof(SH_TYPE));
tb->ctx = ctx;
@@ -294,9 +336,18 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)

SH_COMPUTE_PARAMETERS(tb, size);

- tb->data = MemoryContextAllocExtended(tb->ctx,
-  sizeof(SH_ELEMENT_TYPE) * tb->size,
-  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+ if (!alloc)
+ {
+ tb->alloc = palloc(sizeof(SH_ALLOCATOR));
+ tb->alloc->HashAlloc = SH_DEFAULT_ALLOC;
+ tb->alloc->HashFree = SH_DEFAULT_FREE;
+ tb->alloc->args = tb->ctx;
+ }
+ else
+ tb->alloc = alloc;
+
+ tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+ tb->alloc->args);

Other the another option is, that we can always make caller to provide
an allocator. But this way every new user for simple hash need to take
care of having allocator.

What is your opinion?

This also needs docs, including a warning that just
using an allocator in shared memory does *NOT* allow the hash table to be
used in shared memory in the general case.

Make sense.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#29Dilip Kumar
dilipbalaut@gmail.com
In reply to: Dilip Kumar (#28)
2 attachment(s)
Re: Parallel bitmap heap scan

On Mon, Dec 26, 2016 at 3:14 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

Other the another option is, that we can always make caller to provide
an allocator. But this way every new user for simple hash need to take
care of having allocator.

What is your opinion?

Attached is the new version of the patch which implements it the way I
described.

This also needs docs, including a warning that just
using an allocator in shared memory does *NOT* allow the hash table to be
used in shared memory in the general case.

Make sense.

Added the Warning.

I have also fixed some bug in parallel bitmap heap scan
(path.parallel_workers was not initialised before calling
cost_bitmap_heap_scan in some cases, so it was taking the
uninitialized value). Patch attached.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

hash-support-alloc-free-v6.patchapplication/octet-stream; name=hash-support-alloc-free-v6.patchDownload
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 47c9656..14f8c2d 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -330,7 +330,7 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
 	else
 		hashtable->hash_iv = 0;
 
-	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
+	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, NULL);
 	hashtable->hashtab->private_data = hashtable;
 
 	return hashtable;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 7b31948..0885812 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -244,7 +244,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index 12aedbc..a28afc3 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -66,6 +66,7 @@
 #define SH_STATUS_EMPTY SH_MAKE_NAME(EMPTY)
 #define SH_STATUS_IN_USE SH_MAKE_NAME(IN_USE)
 #define SH_ITERATOR SH_MAKE_NAME(iterator)
+#define SH_ALLOCATOR SH_MAKE_NAME(alloc)
 
 /* function declarations */
 #define SH_CREATE SH_MAKE_NAME(create)
@@ -90,6 +91,18 @@
 /* generate forward declarations necessary to use the hash table */
 #ifdef SH_DECLARE
 
+typedef struct SH_ALLOCATOR
+{
+	/* Allocation function */
+	void	   *(*HashAlloc) (Size size, void *args);
+
+	/* Free function */
+	void		(*HashFree) (void *pointer, void *args);
+
+	/* Arguments to be passed to alloc and free functions */
+	void	   *args;
+}	SH_ALLOCATOR;
+
 /* type definitions */
 typedef struct SH_TYPE
 {
@@ -112,6 +125,9 @@ typedef struct SH_TYPE
 	/* hash buckets */
 	SH_ELEMENT_TYPE *data;
 
+	/* hash allocator */
+	SH_ALLOCATOR *alloc;
+
 	/* memory context to use for allocations */
 	MemoryContext ctx;
 
@@ -133,7 +149,8 @@ typedef struct SH_ITERATOR
 } SH_ITERATOR;
 
 /* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
+							SH_ALLOCATOR *alloc);
 SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
 SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
 SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -276,15 +293,38 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
 #endif
 }
 
+/* default memory allocator function */
+static void *
+SH_DEFAULT_ALLOC(Size size, void *args)
+{
+	MemoryContext context = (MemoryContext) args;
+
+	return MemoryContextAllocExtended(context, size,
+									  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+}
+
+/* default memory free function */
+static void
+SH_DEFAULT_FREE(void *pointer, void *args)
+{
+	pfree(pointer);
+}
+
 /*
  * Create a hash table with enough space for `nelements` distinct members,
  * allocating required memory in the passed-in context.
+ *
+ * WARNING : alloc is an allocator handle which provides memory allocator
+ * function for storing hash elements. User needs to be aware that providing
+ * allocator does not store complete hash table in the allocator memory i.e
+ * hash table will still be allocated in local memory. However all the
+ * elements will be stored in the memory provided by the allocator.
  */
 SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements)
+SH_CREATE(MemoryContext ctx, uint32 nelements, SH_ALLOCATOR *alloc)
 {
-	SH_TYPE    *tb;
-	uint64		size;
+	SH_TYPE     *tb;
+	uint64		 size;
 
 	tb = MemoryContextAllocZero(ctx, sizeof(SH_TYPE));
 	tb->ctx = ctx;
@@ -294,9 +334,18 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 
 	SH_COMPUTE_PARAMETERS(tb, size);
 
-	tb->data = MemoryContextAllocExtended(tb->ctx,
-										  sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	if (!alloc)
+	{
+		tb->alloc = palloc(sizeof(SH_ALLOCATOR));
+		tb->alloc->HashAlloc = SH_DEFAULT_ALLOC;
+		tb->alloc->HashFree = SH_DEFAULT_FREE;
+		tb->alloc->args = tb->ctx;
+	}
+	else
+		tb->alloc = alloc;
+
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	return tb;
 }
@@ -305,6 +354,7 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 SH_SCOPE void
 SH_DESTROY(SH_TYPE *tb)
 {
+	pfree(tb->alloc);
 	pfree(tb->data);
 	pfree(tb);
 }
@@ -333,9 +383,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 	/* compute parameters for new table */
 	SH_COMPUTE_PARAMETERS(tb, newsize);
 
-	tb->data = MemoryContextAllocExtended(
-								 tb->ctx, sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	newdata = tb->data;
 
@@ -421,7 +470,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 		}
 	}
 
-	pfree(olddata);
+	tb->alloc->HashFree(olddata, tb->alloc->args);
+
 }
 
 /*
parallel-bitmap-heap-scan-v6.patchapplication/octet-stream; name=parallel-bitmap-heap-scan-v6.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1545f03..cf2dbbd 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1232,6 +1232,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1ce42ea..76a99a1 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 821f3e9..f108b33 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -203,6 +204,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -255,6 +260,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -732,6 +742,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index d5fd57a..6a2a19a 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,16 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+					  ParallelIterator *parallel_iterator);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -68,11 +74,18 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
 
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (pbminfo)
+		parallel_tbm = (ParallelTIDBitmap *) ((char *) pbminfo +
+											  pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,6 +100,35 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	prefetch_iterator = node->prefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	First worker to see the state as parallel bitmap info as PBM_INITIAL
+	 *	become leader and set the state to PBM_INPROGRESS All other workers
+	 *	see the state as PBM_INPROGRESS will wait for leader to complete the
+	 *	TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *	  ParallelBitmapInfo so that other worker can see those.
+	 *	  set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader create shared TIDBitmap.
+	 *	  copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over page and chunk array
+	 *		 and select heap pages one by one. If prefetch is enable then
+	 *		 there will be two iterator.
+	 *	  b) Since multiple worker are iterating over same page and chunk
+	 *		 array we need to have a shared iterator, so we grab a spin
+	 *		 lock and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,7 +143,41 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.
+		 *
+		 * In parallel mode leader will immediately come out of the function,
+		 * but all other worker will be blocked until leader wake them up.
+		 */
+		if (pbminfo == NULL || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag in lower level bitmap index scan. Later
+			 * bitmap index node will use this flag to indicate tidbitmap that
+			 * it needs to create an shared page table.
+			 */
+			if (pbminfo)
+			{
+				node->is_leader = true;
+				pbms_set_parallel(outerPlanState(node));
+			}
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM. Here we
+			 * need to create a local TBM and copy information from shared
+			 * location. We also need to attach to shared page table using
+			 * hash table handle stored in parallel_tbm(shared memory).
+			 */
+			tbm = tbm_create(work_mem * 1024L);
+			tbm_set_parallel(tbm, node->ss.ps.state->es_query_dsa);
+			tbm_restore_shared_members(tbm, parallel_tbm);
+		}
 
 		if (!tbm || !IsA(tbm, TIDBitmap))
 			elog(ERROR, "unrecognized result from subplan");
@@ -118,19 +194,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			node->prefetch_target = -1;
 		}
 #endif   /* USE_PREFETCH */
+
+		/*
+		 * If we are in parallel mode and we are leader worker then copy the
+		 * local TBM information to shared location, and wake up other
+		 * workers.
+		 */
+		if (node->is_leader)
+		{
+			/*
+			 * copy TBM local information in shared memory before waking up
+			 * other workers. Other workers will create there own TBM and copy
+			 * information from shared memory.
+			 */
+			tbm_update_shared_members(tbm, parallel_tbm);
+
+			/* Change the state under a lock */
+			SpinLockAcquire(&pbminfo->state_mutex);
+			pbminfo->state = PBM_FINISHED;
+			SpinLockRelease(&pbminfo->state_mutex);
+
+			/* Wake up all other workers. */
+			ConditionVariableBroadcast(&pbminfo->cv);
+		}
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+									 pbminfo ? &pbminfo->tbmiterator : NULL);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +239,43 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (pbminfo)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+							   pbminfo ? &pbminfo->prefetch_iterator : NULL);
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not
+				 * ensure that current blockno in main iterator and prefetch
+				 * iterator is same. It's possible that whatever blockno we
+				 * are prefetching is getting processed by some other worker.
+				 */
+				if ((pbminfo == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +309,39 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				int		   *prefetch_target;
+
+				if (pbminfo == NULL)
+					prefetch_target = &node->prefetch_target;
+				else
+				{
+					prefetch_target = &pbminfo->prefetch_target;
+					/* If we are in parallel mode then grab prefetch_mutex */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+				}
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +357,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (pbminfo == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+					pbminfo->prefetch_target++;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +396,52 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+			TBMIterator *iterator = node->prefetch_iterator;
+			ParallelIterator *parallel_iteartor = NULL;
+			ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (pbminfo)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				parallel_iteartor = &pbminfo->prefetch_iterator;
+				prefetch_pages = &pbminfo->prefetch_pages;
+				prefetch_target = pbminfo->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
 
+			while (*prefetch_pages < prefetch_target)
+			{
+				TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+														  parallel_iteartor);
 				if (tbmpre == NULL)
 				{
 					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					tbm_end_iterate(iterator);
+					prefetch_iterator = node->prefetch_iterator = NULL;
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +656,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +774,8 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->is_leader = false;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -653,3 +862,219 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*
+		 * Check the current state
+		 *
+		 * If state is
+		 * PBM_INITIAL    -> then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS -> then we need to wait till leader create BITMAP
+		 * PBM_FINISHED   -> BITMAP is ready so no need to wait.
+		 *
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Wrapper over tbm_iterate, which take care of copying and restoring iterator
+ * variables from parallel iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator, ParallelIterator *piterator)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (piterator == NULL)
+		return tbm_iterate(iterator);
+
+	/* We are in parallel mode so grab parallel iterator mutex */
+	SpinLockAcquire(&piterator->mutex);
+
+	/*
+	 * Copy information from shared location to local iterator and call
+	 * iterate. Then restore back to shared iterator.
+	 */
+	iterator->spageptr = piterator->spageptr;
+	iterator->schunkptr = piterator->schunkptr;
+	iterator->schunkbit = piterator->schunkbit;
+	output = tbm_iterate(iterator);
+	piterator->spageptr = iterator->spageptr;
+	piterator->schunkptr = iterator->schunkptr;
+	piterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&piterator->mutex);
+
+	return output;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node as
+	 * parallel, because only first node will create the main bitmap other
+	 * bitmaps will be merged to the first bitmap so no need to create them in
+	 * shared memory.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState *) node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..3d4ed13 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate
+	 * that we need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_dsa);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..aa6fedb 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -43,6 +43,13 @@
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
 #include "nodes/tidbitmap.h"
+#include "utils/hsearch.h"
+#include "utils/dsa.h"
+#include "storage/condition_variable.h"
+#include "storage/spin.h"
+#include "storage/lwlock.h"
+#include "nodes/execnodes.h"
+#include "storage/shmem.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -139,21 +146,26 @@ struct TIDBitmap
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	bool		is_shared;		/* need to build shared tbm if set */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	struct pagetable_alloc *allocator;	/* shared memory allocator */
+	int			dsa_entries;	/* actual allocated entries */
+	dsa_area   *area;			/* reference to per-query shared memory area */
 };
 
 /*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
  */
-struct TBMIterator
+struct ParallelTIDBitmap
 {
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+	dsa_pointer dsa_data;		/* dsa pointers for all kind of pages */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	int			dsa_entries;	/* total item in dsa_pages */
+	bool		inited;			/* set true after leader converts page */
+	/* table to dsa_pointer's array. */
 };
 
 
@@ -168,6 +180,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+void	   *tbm_alloc_shared(Size size, void *arg);
+void		tbm_free_shared(void *pointer, void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -231,6 +245,8 @@ tbm_create(long maxbytes)
 	tbm->maxentries = (int) nbuckets;
 	tbm->lossify_start = 0;
 
+	tbm->allocator = palloc(sizeof(pagetable_alloc));
+
 	return tbm;
 }
 
@@ -244,7 +260,16 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	if (tbm->is_shared)
+	{
+		tbm->allocator->args = tbm;
+		tbm->allocator->HashAlloc = tbm_alloc_shared;
+		tbm->allocator->HashFree = tbm_free_shared;
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, tbm->allocator);
+		tbm->dsa_entries = tbm->pagetable->size;
+	}
+	else
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -271,7 +296,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 void
 tbm_free(TIDBitmap *tbm)
 {
-	if (tbm->pagetable)
+	if (!tbm->is_shared && tbm->pagetable)
 		pagetable_destroy(tbm->pagetable);
 	if (tbm->spages)
 		pfree(tbm->spages);
@@ -612,11 +637,16 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	iterator->tbm = tbm;
 
 	/*
-	 * Initialize iteration pointers.
+	 * Initialize iteration pointers, only if it's not shared tbm. For shared
+	 * TBM, we will copy these values from shared iterator before calling
+	 * tbm_iterate.
 	 */
-	iterator->spageptr = 0;
-	iterator->schunkptr = 0;
-	iterator->schunkbit = 0;
+	if (!tbm->is_shared)
+	{
+		iterator->spageptr = 0;
+		iterator->schunkptr = 0;
+		iterator->schunkbit = 0;
+	}
 
 	/*
 	 * If we have a hashtable, create and fill the sorted page lists, unless
@@ -626,7 +656,7 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	 */
 	if (tbm->status == TBM_HASH && !tbm->iterating)
 	{
-		pagetable_iterator i;
+		pagetable_iterator itr;
 		PagetableEntry *page;
 		int			npages;
 		int			nchunks;
@@ -640,15 +670,48 @@ tbm_begin_iterate(TIDBitmap *tbm)
 				MemoryContextAlloc(tbm->mcxt,
 								   tbm->nchunks * sizeof(PagetableEntry *));
 
-		npages = nchunks = 0;
-		pagetable_start_iterate(tbm->pagetable, &i);
-		while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+		/*
+		 * For shared TBM, directly convert dsa array to page and chunk array.
+		 */
+		if (tbm->is_shared)
 		{
-			if (page->ischunk)
-				tbm->schunks[nchunks++] = page;
-			else
-				tbm->spages[npages++] = page;
+			PagetableEntry *dsa_entry;
+			int			i;
+
+			/*
+			 * This step will be done by all the workers including leader.
+			 * Here we need to convert array of dsa pointers to local page and
+			 * chunk array.
+			 */
+			dsa_entry = dsa_get_address(tbm->area, tbm->dsa_data);
+			dsa_entry = (void *) (((char *) dsa_entry) + sizeof(dsa_pointer));
+			npages = nchunks = 0;
+			for (i = 0; i < tbm->dsa_entries; i++)
+			{
+				page = (PagetableEntry *) (dsa_entry + i);
+
+				if (page->status != pagetable_IN_USE)
+					continue;
+
+				if (page->ischunk)
+					tbm->schunks[nchunks++] = page;
+				else
+					tbm->spages[npages++] = page;
+			}
+		}
+		else
+		{
+			npages = nchunks = 0;
+			pagetable_start_iterate(tbm->pagetable, &itr);
+			while ((page = pagetable_iterate(tbm->pagetable, &itr)) != NULL)
+			{
+				if (page->ischunk)
+					tbm->schunks[nchunks++] = page;
+				else
+					tbm->spages[npages++] = page;
+			}
 		}
+
 		Assert(npages == tbm->npages);
 		Assert(nchunks == tbm->nchunks);
 		if (npages > 1)
@@ -1061,3 +1124,114 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_update_shared_members
+ *
+ * Restore leaders private tbm state to shared location. This must
+ * be called before waking up the other worker.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before waking up the other
+	 * workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->dsa_data = tbm->dsa_data;
+	parallel_tbm->dsa_entries = tbm->dsa_entries;
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->is_shared = true;
+	tbm->area = (dsa_area *) area;
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size
+tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
+
+/*
+ * tbm_restore_shared_members
+ *
+ * Attach worker to shared TID bitmap created by leader worker.
+ */
+void
+tbm_restore_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *stbm)
+{
+	tbm->status = TBM_HASH;
+	tbm->nchunks = stbm->nchunks;
+	tbm->nentries = stbm->nentries;
+	tbm->npages = stbm->npages;
+	tbm->maxentries = stbm->maxentries;
+	tbm->dsa_data = stbm->dsa_data;
+	tbm->dsa_entries = stbm->dsa_entries;
+	tbm->is_shared = true;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Memory allocation function for bitmap hash.
+ * It allocated memory from DSA and also stores dsa_pointer in the memory
+ * so that during free we can directly get the dsa_pointe and free it.
+ */
+void *
+tbm_alloc_shared(Size size, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->area, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	/* Keep track of actual number of entries */
+	if (tbm->pagetable)
+		tbm->dsa_entries = tbm->pagetable->size;
+
+	ptr = dsa_get_address(tbm->area, dsaptr);
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * tbm_free_shared
+ *
+ * Memory free function for tid bitmap
+ */
+void
+tbm_free_shared(void *pointer, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+
+	dsa_pointer dsa_data =
+			*((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+
+	dsa_free(tbm->area, dsa_data);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 46d7d06..bb262a6 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -126,6 +126,7 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static int	compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 
 
 /*
@@ -678,49 +679,7 @@ create_plain_partial_paths(PlannerInfo *root, RelOptInfo *rel)
 {
 	int			parallel_workers;
 
-	/*
-	 * If the user has set the parallel_workers reloption, use that; otherwise
-	 * select a default number of workers.
-	 */
-	if (rel->rel_parallel_workers != -1)
-		parallel_workers = rel->rel_parallel_workers;
-	else
-	{
-		int			parallel_threshold;
-
-		/*
-		 * If this relation is too small to be worth a parallel scan, just
-		 * return without doing anything ... unless it's an inheritance child.
-		 * In that case, we want to generate a parallel path here anyway.  It
-		 * might not be worthwhile just for this relation, but when combined
-		 * with all of its inheritance siblings it may well pay off.
-		 */
-		if (rel->pages < (BlockNumber) min_parallel_relation_size &&
-			rel->reloptkind == RELOPT_BASEREL)
-			return;
-
-		/*
-		 * Select the number of workers based on the log of the size of the
-		 * relation.  This probably needs to be a good deal more
-		 * sophisticated, but we need something here for now.  Note that the
-		 * upper limit of the min_parallel_relation_size GUC is chosen to
-		 * prevent overflow here.
-		 */
-		parallel_workers = 1;
-		parallel_threshold = Max(min_parallel_relation_size, 1);
-		while (rel->pages >= (BlockNumber) (parallel_threshold * 3))
-		{
-			parallel_workers++;
-			parallel_threshold *= 3;
-			if (parallel_threshold > INT_MAX / 3)
-				break;			/* avoid overflow */
-		}
-	}
-
-	/*
-	 * In no case use more than max_parallel_workers_per_gather workers.
-	 */
-	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+	parallel_workers = compute_parallel_worker(rel, rel->pages);
 
 	/* If any limit was set to zero, the user doesn't want a parallel scan. */
 	if (parallel_workers <= 0)
@@ -2866,6 +2825,84 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+static int
+compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
+{
+	int			parallel_workers;
+
+	/*
+	 * If the user has set the parallel_workers reloption, use that; otherwise
+	 * select a default number of workers.
+	 */
+	if (rel->rel_parallel_workers != -1)
+		parallel_workers = rel->rel_parallel_workers;
+	else
+	{
+		int			parallel_threshold;
+
+		/*
+		 * If this relation is too small to be worth a parallel scan, just
+		 * return without doing anything ... unless it's an inheritance child.
+		 * In that case, we want to generate a parallel path here anyway.  It
+		 * might not be worthwhile just for this relation, but when combined
+		 * with all of its inheritance siblings it may well pay off.
+		 */
+		if (pages < (BlockNumber) min_parallel_relation_size &&
+			rel->reloptkind == RELOPT_BASEREL)
+			return 0;
+
+		/*
+		 * Select the number of workers based on the log of the size of the
+		 * relation.  This probably needs to be a good deal more
+		 * sophisticated, but we need something here for now.  Note that the
+		 * upper limit of the min_parallel_relation_size GUC is chosen to
+		 * prevent overflow here.
+		 */
+		parallel_workers = 1;
+		parallel_threshold = Max(min_parallel_relation_size, 1);
+		while (pages >= (BlockNumber) (parallel_threshold * 3))
+		{
+			parallel_workers++;
+			parallel_threshold *= 3;
+			if (parallel_threshold > INT_MAX / 3)
+				break;			/* avoid overflow */
+		}
+	}
+
+	/*
+	 * In no case use more than max_parallel_workers_per_gather workers.
+	 */
+	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+
+	return parallel_workers;
+}
+
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a52eb7e..8289e3f 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -161,6 +161,7 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
 static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
 static double relation_byte_size(double tuples, int width);
 static double page_size(double tuples, int width);
+static Cost adjust_cost_for_parallelism(Path *path, Cost cpu_run_cost);
 
 
 /*
@@ -237,44 +238,7 @@ cost_seqscan(Path *path, PlannerInfo *root,
 
 	/* Adjust costing for parallelism, if used. */
 	if (path->parallel_workers > 0)
-	{
-		double		parallel_divisor = path->parallel_workers;
-		double		leader_contribution;
-
-		/*
-		 * Early experience with parallel query suggests that when there is
-		 * only one worker, the leader often makes a very substantial
-		 * contribution to executing the parallel portion of the plan, but as
-		 * more workers are added, it does less and less, because it's busy
-		 * reading tuples from the workers and doing whatever non-parallel
-		 * post-processing is needed.  By the time we reach 4 workers, the
-		 * leader no longer makes a meaningful contribution.  Thus, for now,
-		 * estimate that the leader spends 30% of its time servicing each
-		 * worker, and the remainder executing the parallel plan.
-		 */
-		leader_contribution = 1.0 - (0.3 * path->parallel_workers);
-		if (leader_contribution > 0)
-			parallel_divisor += leader_contribution;
-
-		/*
-		 * In the case of a parallel plan, the row count needs to represent
-		 * the number of tuples processed per worker.  Otherwise, higher-level
-		 * plan nodes that appear below the gather will be costed incorrectly,
-		 * because they'll anticipate receiving more rows than any given copy
-		 * will actually get.
-		 */
-		path->rows = clamp_row_est(path->rows / parallel_divisor);
-
-		/* The CPU cost is divided among all the workers. */
-		cpu_run_cost /= parallel_divisor;
-
-		/*
-		 * It may be possible to amortize some of the I/O cost, but probably
-		 * not very much, because most operating systems already do aggressive
-		 * prefetching.  For now, we assume that the disk run cost can't be
-		 * amortized at all.
-		 */
-	}
+		cpu_run_cost = adjust_cost_for_parallelism(path, cpu_run_cost);
 
 	path->startup_cost = startup_cost;
 	path->total_cost = startup_cost + cpu_run_cost + disk_run_cost;
@@ -831,10 +795,10 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
 	Cost		indexTotalCost;
-	Selectivity indexSelectivity;
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -855,13 +819,12 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	if (!enable_bitmapscan)
 		startup_cost += disable_cost;
 
-	/*
-	 * Fetch total cost of obtaining the bitmap, as well as its total
-	 * selectivity.
-	 */
-	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+	pages_fetched = compute_bitmap_pages(root, baserel, bitmapqual,
+										 loop_count, &indexTotalCost,
+										 &tuples_fetched);
 
 	startup_cost += indexTotalCost;
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
 
 	/* Fetch estimated page costs for tablespace containing table. */
 	get_tablespace_page_costs(baserel->reltablespace,
@@ -869,41 +832,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 							  &spc_seq_page_cost);
 
 	/*
-	 * Estimate number of main-table pages fetched.
-	 */
-	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
-
-	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
-
-	if (loop_count > 1)
-	{
-		/*
-		 * For repeated bitmap scans, scale up the number of tuples fetched in
-		 * the Mackert and Lohman formula by the number of scans, so that we
-		 * estimate the number of pages fetched by all the scans. Then
-		 * pro-rate for one scan.
-		 */
-		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
-											baserel->pages,
-											get_indexpath_pages(bitmapqual),
-											root);
-		pages_fetched /= loop_count;
-	}
-	else
-	{
-		/*
-		 * For a single scan, the number of heap pages that need to be fetched
-		 * is the same as the Mackert and Lohman formula for the case T <= b
-		 * (ie, no re-reads needed).
-		 */
-		pages_fetched = (2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
-	}
-	if (pages_fetched >= T)
-		pages_fetched = T;
-	else
-		pages_fetched = ceil(pages_fetched);
-
-	/*
 	 * For small numbers of pages we should charge spc_random_page_cost
 	 * apiece, while if nearly all the table's pages are being read, it's more
 	 * appropriate to charge spc_seq_page_cost apiece.  The effect is
@@ -932,8 +860,13 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+		cpu_run_cost = adjust_cost_for_parallelism(path, cpu_run_cost);
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
@@ -944,6 +877,56 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 }
 
 /*
+ * adjust_cost_for_parallelism
+ *
+ * Adjust the cpu cost based on number of parallel workers, also update
+ * the number of rows processed at each worker.
+ */
+static Cost
+adjust_cost_for_parallelism(Path *path, Cost cpu_run_cost)
+{
+	double		parallel_divisor = path->parallel_workers;
+	double		leader_contribution;
+	Cost		cpu_cost = cpu_run_cost;
+
+	/*
+	 * Early experience with parallel query suggests that when there is
+	 * only one worker, the leader often makes a very substantial
+	 * contribution to executing the parallel portion of the plan, but as
+	 * more workers are added, it does less and less, because it's busy
+	 * reading tuples from the workers and doing whatever non-parallel
+	 * post-processing is needed.  By the time we reach 4 workers, the
+	 * leader no longer makes a meaningful contribution.  Thus, for now,
+	 * estimate that the leader spends 30% of its time servicing each
+	 * worker, and the remainder executing the parallel plan.
+	 */
+	leader_contribution = 1.0 - (0.3 * path->parallel_workers);
+	if (leader_contribution > 0)
+		parallel_divisor += leader_contribution;
+
+	/*
+	 * In the case of a parallel plan, the row count needs to represent
+	 * the number of tuples processed per worker.  Otherwise, higher-level
+	 * plan nodes that appear below the gather will be costed incorrectly,
+	 * because they'll anticipate receiving more rows than any given copy
+	 * will actually get.
+	 */
+	path->rows = clamp_row_est(path->rows / parallel_divisor);
+
+	/* The CPU cost is divided among all the workers. */
+	cpu_cost /= parallel_divisor;
+
+	/*
+	 * It may be possible to amortize some of the I/O cost, but probably
+	 * not very much, because most operating systems already do aggressive
+	 * prefetching.  For now, we assume that the disk run cost can't be
+	 * amortized at all.
+	 */
+
+	return cpu_cost;
+}
+
+/*
  * cost_bitmap_tree_node
  *		Extract cost and selectivity from a bitmap tree node (index/and/or)
  */
@@ -4798,3 +4781,69 @@ page_size(double tuples, int width)
 {
 	return ceil(relation_byte_size(tuples, width) / BLCKSZ);
 }
+
+/*
+ * compute_bitmap_pages
+ *
+ * compute number of pages fetched from heap in bitmap heap scan.
+ */
+double
+compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel, Path *bitmapqual,
+			int loop_count, Cost *cost, double *tuple)
+{
+	Cost		indexTotalCost;
+	Selectivity indexSelectivity;
+	double		T;
+	double		pages_fetched;
+	double		tuples_fetched;
+
+	/*
+	 * Fetch total cost of obtaining the bitmap, as well as its total
+	 * selectivity.
+	 */
+	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+
+	/*
+	 * Estimate number of main-table pages fetched.
+	 */
+	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
+
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
+
+	if (loop_count > 1)
+	{
+		/*
+		 * For repeated bitmap scans, scale up the number of tuples fetched in
+		 * the Mackert and Lohman formula by the number of scans, so that we
+		 * estimate the number of pages fetched by all the scans. Then
+		 * pro-rate for one scan.
+		 */
+		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
+											baserel->pages,
+											get_indexpath_pages(bitmapqual),
+											root);
+		pages_fetched /= loop_count;
+	}
+	else
+	{
+		/*
+		 * For a single scan, the number of heap pages that need to be fetched
+		 * is the same as the Mackert and Lohman formula for the case T <= b
+		 * (ie, no re-reads needed).
+		 */
+		pages_fetched =
+			(2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
+	}
+
+	if (pages_fetched >= T)
+		pages_fetched = T;
+	else
+		pages_fetched = ceil(pages_fetched);
+
+	if (cost)
+		*cost = indexTotalCost;
+	if (tuple)
+		*tuple = tuples_fetched;
+
+	return pages_fetched;
+}
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 7b43c4a..e74a717 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1557,6 +1561,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1599,6 +1608,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index c7bcd9b..25fe7b6 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2803,7 +2803,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3b7c56d..bcf85a2 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3192,7 +3193,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index f37a0bf..0f15d04 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3386,6 +3386,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index ee7e05a..d7ebac4 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -178,6 +178,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 486555e..62bbbec 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1417,6 +1419,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t		mutex;			/* mutual exclusion for below three fields */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+}	ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	ParallelIterator tbmiterator;
+	ParallelIterator prefetch_iterator;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	Size		ptbm_offset;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1431,6 +1499,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1447,6 +1517,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1462,7 +1533,11 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		is_leader		   is_leader is set true, if this worker become
+ *						   leader for parallel bitmap heap scan.
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1477,6 +1552,9 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	bool		is_leader;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..9504395 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+} TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -62,5 +79,11 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+						   ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+						  ParallelTIDBitmap *parallel_tbm);
+void		tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 39376ec..1987d2e 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -183,6 +183,8 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
 					   double cte_rows);
 extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
 extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern double compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel,
+				Path *bitmapqual, int loop_count, Cost *cost, double *tuple);
 
 /*
  * prototypes for clausesel.c
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index d16f879..8209749 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 480f25f..0341b57 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 5b37894..5c2cc0c 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -784,6 +784,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#30Amit Khandekar
amitdkhan.pg@gmail.com
In reply to: Dilip Kumar (#15)
Re: Parallel bitmap heap scan

Sorry for the delay in my next response. I still haven't exhaustively
gone through all changes, but meanwhile, below are some more points.

On 26 November 2016 at 18:18, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Tue, Nov 22, 2016 at 9:05 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Fri, Nov 18, 2016 at 9:59 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:

Thanks for the review..

I have worked on these comments..

In pbms_is_leader() , I didn't clearly understand the significance of
the for-loop. If it is a worker, it can call
ConditionVariablePrepareToSleep() followed by
ConditionVariableSleep(). Once it comes out of
ConditionVariableSleep(), isn't it guaranteed that the leader has
finished the bitmap ? If yes, then it looks like it is not necessary
to again iterate and go back through the pbminfo->state checking.
Also, with this, variable queuedSelf also might not be needed. But I
might be missing something here.

I have taken this logic from example posted on conditional variable thread[1]

for (;;)
{
ConditionVariablePrepareToSleep(cv);
if (condition for which we are waiting is satisfied)
break;
ConditionVariableSleep();
}
ConditionVariableCancelSleep();

[1] /messages/by-id/CA+Tgmoaj2aPti0yho7FeEf2qt-JgQPRWb0gci_o1Hfr=C56Xng@mail.gmail.com

With the latest patches, this looks fine to me.

tbm_iterate() call under SpinLock :
For parallel tbm iteration, tbm_iterate() is called while SpinLock is
held. Generally we try to keep code inside Spinlock call limited to a
few lines, and that too without occurrence of a function call.
Although tbm_iterate() code itself looks safe under a spinlock, I was
checking if we can squeeze SpinlockAcquire() and SpinLockRelease()
closer to each other. One thought is : in tbm_iterate(), acquire the
SpinLock before the while loop that iterates over lossy chunks. Then,
if both chunk and per-page data remain, release spinlock just before
returning (the first return stmt). And then just before scanning
bitmap of an exact page, i.e. just after "if (iterator->spageptr <
tbm->npages)", save the page handle, increment iterator->spageptr,
release Spinlock, and then use the saved page handle to iterate over
the page bitmap.

Main reason to keep Spin lock out of this function to avoid changes
inside this function, and also this function takes local iterator as
input which don't have spin lock reference to it. But that can be
changed, we can pass shared iterator to it.

I will think about this logic and try to update in next version.

Still this issue is not addressed.
Logic inside tbm_iterate is using same variable, like spageptr,
multiple places. IMHO this complete logic needs to be done under one
spin lock.

I think we both agree on the part that the mutex handle can be passed
to tbm_iterate() to do this. I am yet to give more thought on how
clumsy it will be if we add SpinlockRelease() calls in intermediate
places in the function rather than in the end.

prefetch_pages() call under Spinlock :
Here again, prefetch_pages() is called while pbminfo->prefetch_mutex
Spinlock is held. Effectively, heavy functions like PrefetchBuffer()
would get called while under the Spinlock. These can even ereport().
One option is to use mutex lock for this purpose. But I think that
would slow things down. Moreover, the complete set of prefetch pages
would be scanned by a single worker, and others might wait for this
one. Instead, what I am thinking is: grab the pbminfo->prefetch_mutex
Spinlock only while incrementing pbminfo->prefetch_pages. The rest
part viz : iterating over the prefetch pages, and doing the
PrefetchBuffer() need not be synchronised using this
pgbinfo->prefetch_mutex Spinlock. pbms_parallel_iterate() already has
its own iterator spinlock. Only thing is, workers may not do the
actual PrefetchBuffer() sequentially. One of them might shoot ahead
and prefetch 3-4 pages while the other is lagging with the
sequentially lesser page number; but I believe this is fine, as long
as they all prefetch all the required blocks.

I agree with your point, will try to fix this as well.

I have fixed this part.

This looks good now.

Further points below ....

===== nodeBItmapHeapscan.c ======

In BitmapHeapNext(), the following code is relevant only for tbm
returned from MultiExecProcNode().
if (!tbm || !IsA(tbm, TIDBitmap))
elog(ERROR, "unrecognized result from subplan");

So it should be moved just below MultiExecProcNode(), so that it does
not get called when it is created from tbm_create().

--------------

BitmapHeapScanState->is_leader field looks unnecessary. Instead, a
local variable is_leader in BitmapHeapNext() will solve the purpose.
This is because is_leader is used only in BitmapHeapNext().

--------------

In BitmapHeapNext(), just before tbm_restore_shared_members() is
called, we create tbm using tbm_create(). I think tbm_create() does
not make sense for shared tbm. Whatever fields are required, will be
restored in tbm_restore_shared_members(). The other fields which do
not make sense in a restored tbm are not required to be initialized
using tbm_create(). So I think tbm_restore_shared_members() itself can
call makeNode(TIDBitmap). (Also it is not required to initialize
tbm->allocator; see note below in tidbitmap.c section). So
tbm_restore_shared_members() itself can call tbm_set_parallel().
Looking at all this, it looks better to have the function name changed
to tbm_attach(parallel_tbm) or tbm_restore(parallel_tbm) rather than
tbm_restore_shared_members(). The function header anyways (rightly)
says : Attach worker to shared TID bitmap.

-------------

From what I understand, the leader worker does not have to create its
iterators before waking up the other workers, as long as it makes sure
it copies tbm fields into shared memory before waking workers. But in
the patch, tbm_begin_iterate() is called *before* the
ConditionVariableBroadcast() is called. So I feel, we can shift the
code inside the "if (node->is_leader)" to a place inside the "if
(pbminfo == NULL || pbms_is_leader(pbminfo))" condition block, just
after MultiExecProcNode() call. (And we won't even need is_leader
local variable as well). This way now the other workers will start
working sooner.

====== tidbitmap.c =======

The new #include's are not in alphabetical order.

--------------

ParallelTIDBitmap.inited is unused, and I believe, not required.

--------------

For leader worker, the new TIDBitmap fields added for parallel bitmap
*are* valid while the tid is being built. So the below comment should
be shifted accordingly :
/* these are valid when iterating is true: */
Better still, the shared tbm-related fields can be kept in the end,
and a comment should be added that these are for shared tbm.

--------------

It seems, the comment below the last ParallelTIDBitmap field is not relevant :
/* table to dsa_pointer's array. */

--------------

tbm->allocator field does not seem to be required. A new allocator can
be just palloc'ed in tbm_create_pagetable(), and passed to
pagetable_create(). SH_CREATE() stores this allocator in the
SH_TYPE->alloc field, and fetches the same whenever it needs it for
calling any of the allocator functions. So we can remove the
tbm->allocator field and shift "palloc(sizeof(pagetable_alloc))" call
from tbm_create() to tbm_create_pagetable().

--------------

In tbm_free() :
I think we should call pagetable_destroy() even when tbm is shared, so
that the hash implementation gets a chance to free the hash table
structure. I understand that the hash table structure itself is not
big, so it will only be a small memory leak, but it's better to not
assume that. Instead, let SH_DESTROY() call HashFree(). Then, in
tbm_free_shared(), we can skip the dsa_free() call if tbm->iterating
is false. Basically, tbm bitmap implementation should deduce from the
bitmap state whether it should free the shared data, rather than
preventing a call to SH_DESTROY().

-----------

In tbm_begin_iterate(), for shared tbm, internal structures from
simplehash.h are assumed to be known. For e.g., the hash entries will
always be present in one single array, and also the entry status is
evaluated using pagetable_IN_USE. Is simplehash.h designed keeping in
mind that these things are suppose to be exposed ?

I understand that the hash table handle is local to the leader worker,
and so it is not accessible to other workers. And so, we cannot use
pagetable_iterate() to scan the hash table. So, how about copying the
SH_TYPE structure and making it accessible to other workers ? If we
have something like SH_ATTACH() or SH_COPY(), this will copy the
relevant fields that are sufficient to restore the SH_TYPE structure,
and other workers can start using this hash table after assigning dsa
array back to tb->data. Something like HASH_ATTACH used in dynahash.c.

--------------

In the tbm_update_shared_members() header comments :
* Restore leaders private tbm state to shared location. This must
* be called before waking up the other worker.

Above can be changed to:
* Store leader's private tbm state to shared location. This must
* be called before waking up other workers.

--------------

To be consistent with other function header comments in this file, a
blank line is required between these two lines :

* tbm_estimate_parallel_tidbitmap
* Estimate size of shared TIDBitmap related info.

-------------

tbm->is_shared is set in both tbm_set_parallel() and
tbm_restore_shared_members(). I think it is needed only in
tbm_set_parallel().

--------------

tbm_alloc_shared() Function header comments need some typo correction :
* It allocated memory from DSA and also stores dsa_pointer in the memory
allocated => allocates

--------------

We prepend the dsa_pointer value into the shared memory data allocated
for the pagetable entries; and we save the address of the first page
table entry in tbm->data. But the same dsa_pointer is also stored in
tbm->dsa_data. And tbm is accessible in tbm_free_shared(). So it does
not look necessary to prepend dsa_pointer before the page table
entries. In tbm_free_shared(), we can do dsa_free(tbm->area,
tbm->dsa_data).

--------------

Below are my suggested changes in the algorithm comments :

@@ -103,27 +103,27 @@ BitmapHeapNext(BitmapHeapScanState *node)
        /* --------------------------------------------------------------------
         *      Parallel Bitmap Heap Scan Algorithm
         *
-        *      First worker to see the state as parallel bitmap info
as PBM_INITIAL
-        *      become leader and set the state to PBM_INPROGRESS All
other workers
-        *      see the state as PBM_INPROGRESS will wait for leader
to complete the
-        *      TIDBitmap.
+        *      The first worker to see the state of
ParallelBitmapInfo as PBM_INITIAL
+        *      becomes the leader and sets the state to
PBM_INPROGRESS. All other
+        *      workers see the state as PBM_INPROGRESS, and will wait
for leader to
+        *      finish building the TIDBitmap.
         *
         *      Leader Processing:
         *        Create TIDBitmap using DSA memory.
         *        Restore local TIDBitmap variable information into
-        *        ParallelBitmapInfo so that other worker can see those.
-        *        set state to PBM_FINISHED.
+        *           ParallelBitmapInfo so that other worker can see those.
+        *        Set state to PBM_FINISHED.
         *        Wake up other workers.
         *
         *      Other Worker Processing:
-        *        Wait until leader create shared TIDBitmap.
-        *        copy TIDBitmap from info from ParallelBitmapInfo to
local TIDBitmap.
+        *        Wait until leader creates shared TIDBitmap.
+        *        Copy TIDBitmap from ParallelBitmapInfo to local TIDBitmap.
         *
         *      Iterate and process the pages.
         *        a) In this phase each worker will iterate over page
and chunk array
-        *               and select heap pages one by one. If prefetch
is enable then
-        *               there will be two iterator.
-        *        b) Since multiple worker are iterating over same
page and chunk
+        *               and select heap pages one by one. If prefetch
is enabled then
+        *               there will be two iterators.
+        *        b) Since multiple workers are iterating over same
page and chunk
         *               array we need to have a shared iterator, so
we grab a spin
         *               lock and iterate within a lock.

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#31Dilip Kumar
dilipbalaut@gmail.com
In reply to: Amit Khandekar (#30)
3 attachment(s)
Re: Parallel bitmap heap scan

On Fri, Jan 6, 2017 at 10:47 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:

This looks good now.

Thanks..

Further points below ....

Thanks for the review.

===== nodeBItmapHeapscan.c ======

In BitmapHeapNext(), the following code is relevant only for tbm
returned from MultiExecProcNode().
if (!tbm || !IsA(tbm, TIDBitmap))
elog(ERROR, "unrecognized result from subplan");

Fixed

--------------

BitmapHeapScanState->is_leader field looks unnecessary. Instead, a
local variable is_leader in BitmapHeapNext() will solve the purpose.
This is because is_leader is used only in BitmapHeapNext().

Fixed

--------------

In BitmapHeapNext(), just before tbm_restore_shared_members() is
called, we create tbm using tbm_create(). I think tbm_create() does
not make sense for shared tbm. Whatever fields are required, will be
restored in tbm_restore_shared_members(). The other fields which do
not make sense in a restored tbm are not required to be initialized
using tbm_create(). So I think tbm_restore_shared_members() itself can
call makeNode(TIDBitmap). (Also it is not required to initialize
tbm->allocator; see note below in tidbitmap.c section). So
tbm_restore_shared_members() itself can call tbm_set_parallel().
Looking at all this, it looks better to have the function name changed
to tbm_attach(parallel_tbm) or tbm_restore(parallel_tbm) rather than
tbm_restore_shared_members(). The function header anyways (rightly)
says : Attach worker to shared TID bitmap.

Fixed

-------------

From what I understand, the leader worker does not have to create its
iterators before waking up the other workers, as long as it makes sure
it copies tbm fields into shared memory before waking workers. But in
the patch, tbm_begin_iterate() is called *before* the
ConditionVariableBroadcast() is called. So I feel, we can shift the
code inside the "if (node->is_leader)" to a place inside the "if
(pbminfo == NULL || pbms_is_leader(pbminfo))" condition block, just
after MultiExecProcNode() call. (And we won't even need is_leader
local variable as well). This way now the other workers will start
working sooner.

Correct, Fixed.

====== tidbitmap.c =======

The new #include's are not in alphabetical order.

Done..

--------------

ParallelTIDBitmap.inited is unused, and I believe, not required.

Fixed

--------------

For leader worker, the new TIDBitmap fields added for parallel bitmap
*are* valid while the tid is being built. So the below comment should
be shifted accordingly :
/* these are valid when iterating is true: */
Better still, the shared tbm-related fields can be kept in the end,
and a comment should be added that these are for shared tbm.

Done

--------------

It seems, the comment below the last ParallelTIDBitmap field is not relevant :
/* table to dsa_pointer's array. */

Fixed..

--------------

tbm->allocator field does not seem to be required. A new allocator can
be just palloc'ed in tbm_create_pagetable(), and passed to
pagetable_create(). SH_CREATE() stores this allocator in the
SH_TYPE->alloc field, and fetches the same whenever it needs it for
calling any of the allocator functions. So we can remove the
tbm->allocator field and shift "palloc(sizeof(pagetable_alloc))" call
from tbm_create() to tbm_create_pagetable().

Done

--------------

In tbm_free() :
I think we should call pagetable_destroy() even when tbm is shared, so
that the hash implementation gets a chance to free the hash table
structure. I understand that the hash table structure itself is not
big, so it will only be a small memory leak, but it's better to not
assume that. Instead, let SH_DESTROY() call HashFree(). Then, in
tbm_free_shared(), we can skip the dsa_free() call if tbm->iterating
is false. Basically, tbm bitmap implementation should deduce from the
bitmap state whether it should free the shared data, rather than
preventing a call to SH_DESTROY().

Fixed

-----------

In tbm_begin_iterate(), for shared tbm, internal structures from
simplehash.h are assumed to be known. For e.g., the hash entries will
always be present in one single array, and also the entry status is
evaluated using pagetable_IN_USE. Is simplehash.h designed keeping in
mind that these things are suppose to be exposed ?

I understand that the hash table handle is local to the leader worker,
and so it is not accessible to other workers. And so, we cannot use
pagetable_iterate() to scan the hash table. So, how about copying the
SH_TYPE structure and making it accessible to other workers ? If we
have something like SH_ATTACH() or SH_COPY(), this will copy the
relevant fields that are sufficient to restore the SH_TYPE structure,
and other workers can start using this hash table after assigning dsa
array back to tb->data. Something like HASH_ATTACH used in dynahash.c.

This looks cleaner, and also avoid processing the data of the hash
directly. I have changed as per the suggestion.

--------------

In the tbm_update_shared_members() header comments :
* Restore leaders private tbm state to shared location. This must
* be called before waking up the other worker.

Above can be changed to:
* Store leader's private tbm state to shared location. This must
* be called before waking up other workers.

Done

--------------

To be consistent with other function header comments in this file, a
blank line is required between these two lines :

* tbm_estimate_parallel_tidbitmap
* Estimate size of shared TIDBitmap related info.

Done

-------------

tbm->is_shared is set in both tbm_set_parallel() and
tbm_restore_shared_members(). I think it is needed only in
tbm_set_parallel().

Done

--------------

tbm_alloc_shared() Function header comments need some typo correction :
* It allocated memory from DSA and also stores dsa_pointer in the memory
allocated => allocates

--------------

We prepend the dsa_pointer value into the shared memory data allocated
for the pagetable entries; and we save the address of the first page
table entry in tbm->data. But the same dsa_pointer is also stored in
tbm->dsa_data. And tbm is accessible in tbm_free_shared(). So it does
not look necessary to prepend dsa_pointer before the page table
entries. In tbm_free_shared(), we can do dsa_free(tbm->area,
tbm->dsa_data).

tbm->dsa_data allocates the latest allocated dsa_pointer. And, hash
table first allocates the new memory then free the old pointer. So
whenever it allocates the new memory we will store latest in
tbm->dsa_data, and when the hash table will call free it will call
with the local pointer and we don't have any reference to dsa pointer
for that old memory, that's the reason we store dsa pointer as header.

--------------

Below are my suggested changes in the algorithm comments :

@@ -103,27 +103,27 @@ BitmapHeapNext(BitmapHeapScanState *node)

Changed.

Apart from these changes, I have created a separate
patch(0001-opt-parallelcost-refactoring-v7.patch) for code refactoring
in optimizer required for parallelbitmap heap scan. (earlier this was
part of my main patch)

This patch can be used by 0003-parallel-bitmap-heap-scan-v7.patch
attached in the mail and also parallel-index-scan[2]/messages/by-id/CAA4eK1KthrAvNjmB2cWuUHz+p3ZTTtbD7o2KUw49PC8HK4r1Wg@mail.gmail.com can be rebased on
this, if this get committed,

[2]: /messages/by-id/CAA4eK1KthrAvNjmB2cWuUHz+p3ZTTtbD7o2KUw49PC8HK4r1Wg@mail.gmail.com
/messages/by-id/CAA4eK1KthrAvNjmB2cWuUHz+p3ZTTtbD7o2KUw49PC8HK4r1Wg@mail.gmail.com

I will send the performance data with the new patch in the separate mail.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0001-opt-parallelcost-refactoring-v7.patchapplication/octet-stream; name=0001-opt-parallelcost-refactoring-v7.patchDownload
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 46d7d06..3cf9417 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -126,6 +126,7 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static int	compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 
 
 /*
@@ -678,49 +679,7 @@ create_plain_partial_paths(PlannerInfo *root, RelOptInfo *rel)
 {
 	int			parallel_workers;
 
-	/*
-	 * If the user has set the parallel_workers reloption, use that; otherwise
-	 * select a default number of workers.
-	 */
-	if (rel->rel_parallel_workers != -1)
-		parallel_workers = rel->rel_parallel_workers;
-	else
-	{
-		int			parallel_threshold;
-
-		/*
-		 * If this relation is too small to be worth a parallel scan, just
-		 * return without doing anything ... unless it's an inheritance child.
-		 * In that case, we want to generate a parallel path here anyway.  It
-		 * might not be worthwhile just for this relation, but when combined
-		 * with all of its inheritance siblings it may well pay off.
-		 */
-		if (rel->pages < (BlockNumber) min_parallel_relation_size &&
-			rel->reloptkind == RELOPT_BASEREL)
-			return;
-
-		/*
-		 * Select the number of workers based on the log of the size of the
-		 * relation.  This probably needs to be a good deal more
-		 * sophisticated, but we need something here for now.  Note that the
-		 * upper limit of the min_parallel_relation_size GUC is chosen to
-		 * prevent overflow here.
-		 */
-		parallel_workers = 1;
-		parallel_threshold = Max(min_parallel_relation_size, 1);
-		while (rel->pages >= (BlockNumber) (parallel_threshold * 3))
-		{
-			parallel_workers++;
-			parallel_threshold *= 3;
-			if (parallel_threshold > INT_MAX / 3)
-				break;			/* avoid overflow */
-		}
-	}
-
-	/*
-	 * In no case use more than max_parallel_workers_per_gather workers.
-	 */
-	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+	parallel_workers = compute_parallel_worker(rel, rel->pages);
 
 	/* If any limit was set to zero, the user doesn't want a parallel scan. */
 	if (parallel_workers <= 0)
@@ -2866,6 +2825,59 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+static int
+compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
+{
+	int			parallel_workers;
+
+	/*
+	 * If the user has set the parallel_workers reloption, use that; otherwise
+	 * select a default number of workers.
+	 */
+	if (rel->rel_parallel_workers != -1)
+		parallel_workers = rel->rel_parallel_workers;
+	else
+	{
+		int			parallel_threshold;
+
+		/*
+		 * If this relation is too small to be worth a parallel scan, just
+		 * return without doing anything ... unless it's an inheritance child.
+		 * In that case, we want to generate a parallel path here anyway.  It
+		 * might not be worthwhile just for this relation, but when combined
+		 * with all of its inheritance siblings it may well pay off.
+		 */
+		if (pages < (BlockNumber) min_parallel_relation_size &&
+			rel->reloptkind == RELOPT_BASEREL)
+			return 0;
+
+		/*
+		 * Select the number of workers based on the log of the size of the
+		 * relation.  This probably needs to be a good deal more
+		 * sophisticated, but we need something here for now.  Note that the
+		 * upper limit of the min_parallel_relation_size GUC is chosen to
+		 * prevent overflow here.
+		 */
+		parallel_workers = 1;
+		parallel_threshold = Max(min_parallel_relation_size, 1);
+		while (pages >= (BlockNumber) (parallel_threshold * 3))
+		{
+			parallel_workers++;
+			parallel_threshold *= 3;
+			if (parallel_threshold > INT_MAX / 3)
+				break;			/* avoid overflow */
+		}
+	}
+
+	/*
+	 * In no case use more than max_parallel_workers_per_gather workers.
+	 */
+	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+
+	return parallel_workers;
+}
+
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a52eb7e..66edc54 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -161,6 +161,7 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
 static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
 static double relation_byte_size(double tuples, int width);
 static double page_size(double tuples, int width);
+static Cost update_cost_for_parallelism(Path *path, Cost cpu_run_cost);
 
 
 /*
@@ -237,44 +238,7 @@ cost_seqscan(Path *path, PlannerInfo *root,
 
 	/* Adjust costing for parallelism, if used. */
 	if (path->parallel_workers > 0)
-	{
-		double		parallel_divisor = path->parallel_workers;
-		double		leader_contribution;
-
-		/*
-		 * Early experience with parallel query suggests that when there is
-		 * only one worker, the leader often makes a very substantial
-		 * contribution to executing the parallel portion of the plan, but as
-		 * more workers are added, it does less and less, because it's busy
-		 * reading tuples from the workers and doing whatever non-parallel
-		 * post-processing is needed.  By the time we reach 4 workers, the
-		 * leader no longer makes a meaningful contribution.  Thus, for now,
-		 * estimate that the leader spends 30% of its time servicing each
-		 * worker, and the remainder executing the parallel plan.
-		 */
-		leader_contribution = 1.0 - (0.3 * path->parallel_workers);
-		if (leader_contribution > 0)
-			parallel_divisor += leader_contribution;
-
-		/*
-		 * In the case of a parallel plan, the row count needs to represent
-		 * the number of tuples processed per worker.  Otherwise, higher-level
-		 * plan nodes that appear below the gather will be costed incorrectly,
-		 * because they'll anticipate receiving more rows than any given copy
-		 * will actually get.
-		 */
-		path->rows = clamp_row_est(path->rows / parallel_divisor);
-
-		/* The CPU cost is divided among all the workers. */
-		cpu_run_cost /= parallel_divisor;
-
-		/*
-		 * It may be possible to amortize some of the I/O cost, but probably
-		 * not very much, because most operating systems already do aggressive
-		 * prefetching.  For now, we assume that the disk run cost can't be
-		 * amortized at all.
-		 */
-	}
+		cpu_run_cost = update_cost_for_parallelism(path, cpu_run_cost);
 
 	path->startup_cost = startup_cost;
 	path->total_cost = startup_cost + cpu_run_cost + disk_run_cost;
@@ -831,10 +795,10 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
 	Cost		indexTotalCost;
-	Selectivity indexSelectivity;
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -855,13 +819,12 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	if (!enable_bitmapscan)
 		startup_cost += disable_cost;
 
-	/*
-	 * Fetch total cost of obtaining the bitmap, as well as its total
-	 * selectivity.
-	 */
-	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+	pages_fetched = compute_bitmap_pages(root, baserel, bitmapqual,
+										 loop_count, &indexTotalCost,
+										 &tuples_fetched);
 
 	startup_cost += indexTotalCost;
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
 
 	/* Fetch estimated page costs for tablespace containing table. */
 	get_tablespace_page_costs(baserel->reltablespace,
@@ -869,41 +832,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 							  &spc_seq_page_cost);
 
 	/*
-	 * Estimate number of main-table pages fetched.
-	 */
-	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
-
-	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
-
-	if (loop_count > 1)
-	{
-		/*
-		 * For repeated bitmap scans, scale up the number of tuples fetched in
-		 * the Mackert and Lohman formula by the number of scans, so that we
-		 * estimate the number of pages fetched by all the scans. Then
-		 * pro-rate for one scan.
-		 */
-		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
-											baserel->pages,
-											get_indexpath_pages(bitmapqual),
-											root);
-		pages_fetched /= loop_count;
-	}
-	else
-	{
-		/*
-		 * For a single scan, the number of heap pages that need to be fetched
-		 * is the same as the Mackert and Lohman formula for the case T <= b
-		 * (ie, no re-reads needed).
-		 */
-		pages_fetched = (2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
-	}
-	if (pages_fetched >= T)
-		pages_fetched = T;
-	else
-		pages_fetched = ceil(pages_fetched);
-
-	/*
 	 * For small numbers of pages we should charge spc_random_page_cost
 	 * apiece, while if nearly all the table's pages are being read, it's more
 	 * appropriate to charge spc_seq_page_cost apiece.  The effect is
@@ -944,6 +872,56 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 }
 
 /*
+ * update_cost_for_parallelism
+ *
+ * Adjust the cpu cost based on number of parallel workers, also update
+ * the number of rows processed at each worker.
+ */
+static Cost
+update_cost_for_parallelism(Path *path, Cost cpu_run_cost)
+{
+	double		parallel_divisor = path->parallel_workers;
+	double		leader_contribution;
+	Cost		cpu_cost = cpu_run_cost;
+
+	/*
+	 * Early experience with parallel query suggests that when there is only
+	 * one worker, the leader often makes a very substantial contribution to
+	 * executing the parallel portion of the plan, but as more workers are
+	 * added, it does less and less, because it's busy reading tuples from the
+	 * workers and doing whatever non-parallel post-processing is needed.  By
+	 * the time we reach 4 workers, the leader no longer makes a meaningful
+	 * contribution.  Thus, for now, estimate that the leader spends 30% of
+	 * its time servicing each worker, and the remainder executing the
+	 * parallel plan.
+	 */
+	leader_contribution = 1.0 - (0.3 * path->parallel_workers);
+	if (leader_contribution > 0)
+		parallel_divisor += leader_contribution;
+
+	/*
+	 * In the case of a parallel plan, the row count needs to represent the
+	 * number of tuples processed per worker.  Otherwise, higher-level plan
+	 * nodes that appear below the gather will be costed incorrectly, because
+	 * they'll anticipate receiving more rows than any given copy will
+	 * actually get.
+	 */
+	path->rows = clamp_row_est(path->rows / parallel_divisor);
+
+	/* The CPU cost is divided among all the workers. */
+	cpu_cost /= parallel_divisor;
+
+	/*
+	 * It may be possible to amortize some of the I/O cost, but probably not
+	 * very much, because most operating systems already do aggressive
+	 * prefetching.  For now, we assume that the disk run cost can't be
+	 * amortized at all.
+	 */
+
+	return cpu_cost;
+}
+
+/*
  * cost_bitmap_tree_node
  *		Extract cost and selectivity from a bitmap tree node (index/and/or)
  */
@@ -4798,3 +4776,69 @@ page_size(double tuples, int width)
 {
 	return ceil(relation_byte_size(tuples, width) / BLCKSZ);
 }
+
+/*
+ * compute_bitmap_pages
+ *
+ * compute number of pages fetched from heap in bitmap heap scan.
+ */
+double
+compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel, Path *bitmapqual,
+					 int loop_count, Cost *cost, double *tuple)
+{
+	Cost		indexTotalCost;
+	Selectivity indexSelectivity;
+	double		T;
+	double		pages_fetched;
+	double		tuples_fetched;
+
+	/*
+	 * Fetch total cost of obtaining the bitmap, as well as its total
+	 * selectivity.
+	 */
+	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+
+	/*
+	 * Estimate number of main-table pages fetched.
+	 */
+	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
+
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
+
+	if (loop_count > 1)
+	{
+		/*
+		 * For repeated bitmap scans, scale up the number of tuples fetched in
+		 * the Mackert and Lohman formula by the number of scans, so that we
+		 * estimate the number of pages fetched by all the scans. Then
+		 * pro-rate for one scan.
+		 */
+		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
+											baserel->pages,
+											get_indexpath_pages(bitmapqual),
+											root);
+		pages_fetched /= loop_count;
+	}
+	else
+	{
+		/*
+		 * For a single scan, the number of heap pages that need to be fetched
+		 * is the same as the Mackert and Lohman formula for the case T <= b
+		 * (ie, no re-reads needed).
+		 */
+		pages_fetched =
+			(2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
+	}
+
+	if (pages_fetched >= T)
+		pages_fetched = T;
+	else
+		pages_fetched = ceil(pages_fetched);
+
+	if (cost)
+		*cost = indexTotalCost;
+	if (tuple)
+		*tuple = tuples_fetched;
+
+	return pages_fetched;
+}
0002-hash-support-alloc-free-v7.patchapplication/octet-stream; name=0002-hash-support-alloc-free-v7.patchDownload
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 47c9656..14f8c2d 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -330,7 +330,7 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
 	else
 		hashtable->hash_iv = 0;
 
-	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
+	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, NULL);
 	hashtable->hashtab->private_data = hashtable;
 
 	return hashtable;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 7b31948..0885812 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -244,7 +244,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index 12aedbc..7d66b59 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -66,6 +66,7 @@
 #define SH_STATUS_EMPTY SH_MAKE_NAME(EMPTY)
 #define SH_STATUS_IN_USE SH_MAKE_NAME(IN_USE)
 #define SH_ITERATOR SH_MAKE_NAME(iterator)
+#define SH_ALLOCATOR SH_MAKE_NAME(alloc)
 
 /* function declarations */
 #define SH_CREATE SH_MAKE_NAME(create)
@@ -78,6 +79,7 @@
 #define SH_START_ITERATE_AT SH_MAKE_NAME(start_iterate_at)
 #define SH_ITERATE SH_MAKE_NAME(iterate)
 #define SH_STAT SH_MAKE_NAME(stat)
+#define SH_COPY SH_MAKE_NAME(copy)
 
 /* internal helper functions (no externally visible prototypes) */
 #define SH_COMPUTE_PARAMETERS SH_MAKE_NAME(compute_parameters)
@@ -90,6 +92,18 @@
 /* generate forward declarations necessary to use the hash table */
 #ifdef SH_DECLARE
 
+typedef struct SH_ALLOCATOR
+{
+	/* Allocation function */
+	void	   *(*HashAlloc) (Size size, void *args);
+
+	/* Free function */
+	void		(*HashFree) (void *pointer, void *args);
+
+	/* Arguments to be passed to alloc and free functions */
+	void	   *args;
+}	SH_ALLOCATOR;
+
 /* type definitions */
 typedef struct SH_TYPE
 {
@@ -112,6 +126,9 @@ typedef struct SH_TYPE
 	/* hash buckets */
 	SH_ELEMENT_TYPE *data;
 
+	/* hash allocator */
+	SH_ALLOCATOR *alloc;
+
 	/* memory context to use for allocations */
 	MemoryContext ctx;
 
@@ -133,7 +150,8 @@ typedef struct SH_ITERATOR
 } SH_ITERATOR;
 
 /* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
+							SH_ALLOCATOR *alloc);
 SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
 SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
 SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -143,6 +161,7 @@ SH_SCOPE void SH_START_ITERATE(SH_TYPE *tb, SH_ITERATOR *iter);
 SH_SCOPE void SH_START_ITERATE_AT(SH_TYPE *tb, SH_ITERATOR *iter, uint32 at);
 SH_SCOPE SH_ELEMENT_TYPE *SH_ITERATE(SH_TYPE *tb, SH_ITERATOR *iter);
 SH_SCOPE void SH_STAT(SH_TYPE *tb);
+SH_SCOPE void SH_COPY(SH_TYPE *src_tb, SH_TYPE *dst_tb, SH_ELEMENT_TYPE *data);
 
 #endif   /* SH_DECLARE */
 
@@ -276,15 +295,38 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
 #endif
 }
 
+/* default memory allocator function */
+static void *
+SH_DEFAULT_ALLOC(Size size, void *args)
+{
+	MemoryContext context = (MemoryContext) args;
+
+	return MemoryContextAllocExtended(context, size,
+									  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+}
+
+/* default memory free function */
+static void
+SH_DEFAULT_FREE(void *pointer, void *args)
+{
+	pfree(pointer);
+}
+
 /*
  * Create a hash table with enough space for `nelements` distinct members,
  * allocating required memory in the passed-in context.
+ *
+ * WARNING : alloc is an allocator handle which provides memory allocator
+ * function for storing hash elements. User needs to be aware that providing
+ * allocator does not store complete hash table in the allocator memory i.e
+ * hash table will still be allocated in local memory. However all the
+ * elements will be stored in the memory provided by the allocator.
  */
 SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements)
+SH_CREATE(MemoryContext ctx, uint32 nelements, SH_ALLOCATOR *alloc)
 {
-	SH_TYPE    *tb;
-	uint64		size;
+	SH_TYPE     *tb;
+	uint64		 size;
 
 	tb = MemoryContextAllocZero(ctx, sizeof(SH_TYPE));
 	tb->ctx = ctx;
@@ -294,9 +336,18 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 
 	SH_COMPUTE_PARAMETERS(tb, size);
 
-	tb->data = MemoryContextAllocExtended(tb->ctx,
-										  sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	if (!alloc)
+	{
+		tb->alloc = palloc(sizeof(SH_ALLOCATOR));
+		tb->alloc->HashAlloc = SH_DEFAULT_ALLOC;
+		tb->alloc->HashFree = SH_DEFAULT_FREE;
+		tb->alloc->args = tb->ctx;
+	}
+	else
+		tb->alloc = alloc;
+
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	return tb;
 }
@@ -305,7 +356,12 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 SH_SCOPE void
 SH_DESTROY(SH_TYPE *tb)
 {
-	pfree(tb->data);
+	if (tb->alloc)
+	{
+		pfree(tb->alloc);
+		tb->alloc->HashFree(tb->data, tb->alloc->args);
+	}
+
 	pfree(tb);
 }
 
@@ -333,9 +389,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 	/* compute parameters for new table */
 	SH_COMPUTE_PARAMETERS(tb, newsize);
 
-	tb->data = MemoryContextAllocExtended(
-								 tb->ctx, sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	newdata = tb->data;
 
@@ -421,7 +476,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 		}
 	}
 
-	pfree(olddata);
+	tb->alloc->HashFree(olddata, tb->alloc->args);
+
 }
 
 /*
@@ -826,6 +882,18 @@ SH_STAT(SH_TYPE *tb)
 		 total_collisions, max_collisions, avg_collisions);
 }
 
+/*
+ * Copy information from src_tb to dst_tb, it only copies those informations
+ * which are required by SH_ITERATE. Also store input data as hashtable data.
+ */
+SH_SCOPE void
+SH_COPY (SH_TYPE *src_tb, SH_TYPE *dst_tb, SH_ELEMENT_TYPE *data)
+{
+	dst_tb->size = src_tb->size;
+	dst_tb->sizemask = src_tb->sizemask;
+	dst_tb->data = data;
+}
+
 #endif   /* SH_DEFINE */
 
 
0003-parallel-bitmap-heap-scan-v7.patchapplication/octet-stream; name=0003-parallel-bitmap-heap-scan-v7.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1545f03..cf2dbbd 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1232,6 +1232,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1ce42ea..76a99a1 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 86d9fb5..404c7ac 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -203,6 +204,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -255,6 +260,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -731,6 +741,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index d5fd57a..9df6dfd 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,16 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+					  ParallelIterator *parallel_iterator);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -68,11 +74,18 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
 
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (pbminfo)
+		parallel_tbm = (ParallelTIDBitmap *) ((char *) pbminfo +
+											  pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,6 +100,35 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	prefetch_iterator = node->prefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	The first worker to see the state of ParallelBitmapInfo as PBM_INITIAL
+	 *	becomes leader and sets the state to PBM_INPROGRESS. All other workers
+	 *	see the state as PBM_INPROGRESS, and will wait for leader to finish
+	 *	building the TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *		ParallelBitmapInfo so that other worker can see those.
+	 *	  Set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader creates shared TIDBitmap.
+	 *	  Copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over page and chunk array
+	 *			and select heap pages one by one. If prefetch is enable then
+	 *			there will be two iterators.
+	 *	  b) Since multiple workers are iterating over same page and chunk
+	 *			array we need to have a shared iterator, so we grab a spin
+	 *			lock and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,10 +143,56 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.
+		 *
+		 * In parallel mode leader will immediately come out of the function,
+		 * but all other worker will be blocked until leader wake them up.
+		 */
+		if (pbminfo == NULL || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag in lower level bitmap index scan. Later
+			 * bitmap index node will use this flag to indicate tidbitmap that
+			 * it needs to create an shared page table.
+			 */
+			if (pbminfo)
+				pbms_set_parallel(outerPlanState(node));
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			/*
+			 * copy TBM local information in shared memory before waking up
+			 * other workers. Other workers will create there own TBM and copy
+			 * information from shared memory.
+			 */
+			if (pbminfo)
+			{
+				tbm_update_shared_members(tbm, parallel_tbm);
+
+				/* Change the state under a lock */
+				SpinLockAcquire(&pbminfo->state_mutex);
+				pbminfo->state = PBM_FINISHED;
+				SpinLockRelease(&pbminfo->state_mutex);
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+				/* Wake up all other workers. */
+				ConditionVariableBroadcast(&pbminfo->cv);
+			}
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM. Here we
+			 * need to create a local TBM and copy information from shared
+			 * location. We also need to attach to shared page table using
+			 * hash table handle stored in parallel_tbm (shared memory).
+			 */
+			tbm = tbm_attach(parallel_tbm, node->ss.ps.state->es_query_dsa);
+		}
 
 		node->tbm = tbm;
 		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
@@ -118,19 +206,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			node->prefetch_target = -1;
 		}
 #endif   /* USE_PREFETCH */
+
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+									 pbminfo ? &pbminfo->tbmiterator : NULL);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +229,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (pbminfo)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+							   pbminfo ? &pbminfo->prefetch_iterator : NULL);
+
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not ensure
+				 * that current blockno in main iterator and prefetch iterator
+				 * is same. It's possible that whatever blockno we are
+				 * prefetching is getting processed by some other worker.
+				 */
+				if ((pbminfo == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +300,39 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				int		   *prefetch_target;
+
+				if (pbminfo == NULL)
+					prefetch_target = &node->prefetch_target;
+				else
+				{
+					prefetch_target = &pbminfo->prefetch_target;
+					/* If we are in parallel mode then grab prefetch_mutex */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+				}
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +348,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (pbminfo == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+					pbminfo->prefetch_target++;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +387,53 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+			TBMIterator *iterator = node->prefetch_iterator;
+			ParallelIterator *parallel_iteartor = NULL;
+			ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (pbminfo)
+			{
+				parallel_iteartor = &pbminfo->prefetch_iterator;
+				prefetch_pages = &pbminfo->prefetch_pages;
+				prefetch_target = pbminfo->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
+
+			while (*prefetch_pages < prefetch_target)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+														  parallel_iteartor);
 
 				if (tbmpre == NULL)
 				{
 					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					tbm_end_iterate(iterator);
+					prefetch_iterator = node->prefetch_iterator = NULL;
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +648,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +766,7 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -653,3 +853,218 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : then we need to wait till leader creates BITMAP
+		 * PBM_FINISHED	  : BITMAP is ready so no need to wait.
+		 *---------------
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Wrapper over tbm_iterate, which take care of copying and restoring iterator
+ * variables from parallel iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator, ParallelIterator *piterator)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (piterator == NULL)
+		return tbm_iterate(iterator);
+
+	/* We are in parallel mode so grab parallel iterator mutex */
+	SpinLockAcquire(&piterator->mutex);
+
+	/*
+	 * Copy information from shared location to local iterator and call
+	 * iterate. Then restore back to shared iterator.
+	 */
+	iterator->spageptr = piterator->spageptr;
+	iterator->schunkptr = piterator->schunkptr;
+	iterator->schunkbit = piterator->schunkbit;
+	output = tbm_iterate(iterator);
+	piterator->spageptr = iterator->spageptr;
+	piterator->schunkptr = iterator->schunkptr;
+	piterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&piterator->mutex);
+
+	return output;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node as
+	 * parallel, because only first node will create the main bitmap other
+	 * bitmaps will be merged to the first bitmap so no need to create them in
+	 * shared memory.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState *) node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..56f8376 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate that we
+	 * need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_dsa);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..4ff14c2 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -42,7 +42,14 @@
 
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
+#include "nodes/execnodes.h"
 #include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
+#include "storage/lwlock.h"
+#include "storage/shmem.h"
+#include "storage/spin.h"
+#include "utils/dsa.h"
+#include "utils/hsearch.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -136,27 +143,15 @@ struct TIDBitmap
 	bool		iterating;		/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
+	bool		is_shared;		/* need to build shared tbm if set */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	int			dsa_entries;	/* actual allocated entries */
+	dsa_area   *area;			/* reference to per-query shared memory area */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
 };
 
-/*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
- */
-struct TBMIterator
-{
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
-};
-
-
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
 static bool tbm_intersect_page(TIDBitmap *a, PagetableEntry *apage,
@@ -168,6 +163,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+void	   *tbm_alloc_shared(Size size, void *arg);
+void		tbm_free_shared(void *pointer, void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -198,6 +195,19 @@ hash_blockno(BlockNumber b)
 #define SH_DECLARE
 #include "lib/simplehash.h"
 
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+struct ParallelTIDBitmap
+{
+	dsa_pointer dsa_data;		/* dsa pointers for all kind of pages */
+	pagetable_hash pagetable;	/* hash table of PagetableEntry's */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	int			dsa_entries;	/* total item in dsa_pages */
+};
 
 /*
  * tbm_create - create an initially-empty bitmap
@@ -244,7 +254,18 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	if (tbm->is_shared)
+	{
+		pagetable_alloc *allocator = palloc(sizeof(pagetable_alloc));
+
+		allocator->args = tbm;
+		allocator->HashAlloc = tbm_alloc_shared;
+		allocator->HashFree = tbm_free_shared;
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, allocator);
+		tbm->dsa_entries = tbm->pagetable->size;
+	}
+	else
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -1061,3 +1082,136 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_update_shared_members
+ *
+ * Store leader's private tbm state to shared location. This must
+ * be called before waking up other workers.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before waking up the other
+	 * workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->dsa_data = tbm->dsa_data;
+	parallel_tbm->dsa_entries = tbm->dsa_entries;
+	pagetable_copy(tbm->pagetable, &parallel_tbm->pagetable, NULL);
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->is_shared = true;
+	tbm->area = (dsa_area *) area;
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ *
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size
+tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
+
+/*
+ * tbm_attach
+ *
+ * Create a local TIDBitmap for worker and Attach it to shared TID bitmap
+ * created by leader.
+ */
+TIDBitmap *
+tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area)
+{
+	TIDBitmap  *tbm = makeNode(TIDBitmap);
+	PagetableEntry *dsa_entry;
+
+	tbm->mcxt = CurrentMemoryContext;
+	tbm->status = TBM_HASH;
+	tbm->nchunks = parallel_tbm->nchunks;
+	tbm->nentries = parallel_tbm->nentries;
+	tbm->npages = parallel_tbm->npages;
+	tbm->maxentries = parallel_tbm->maxentries;
+	tbm->dsa_data = parallel_tbm->dsa_data;
+	tbm->dsa_entries = parallel_tbm->dsa_entries;
+	dsa_entry = dsa_get_address(area, tbm->dsa_data);
+	dsa_entry = (void *) (((char *) dsa_entry) + sizeof(dsa_pointer));
+	tbm->pagetable = palloc(sizeof(struct pagetable_hash));
+	pagetable_copy(&parallel_tbm->pagetable, tbm->pagetable, dsa_entry);
+
+	/* Mark the tbm as parallel and also keep the DSA reference in it. */
+	tbm_set_parallel(tbm, area);
+
+	return tbm;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Memory allocation function for bitmap hash.
+ * It allocates memory from DSA and also stores dsa_pointer in the memory
+ * so that during free we can directly get the dsa_pointe and free it.
+ */
+void *
+tbm_alloc_shared(Size size, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->area, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	/* Keep track of actual number of entries */
+	if (tbm->pagetable)
+		tbm->dsa_entries = tbm->pagetable->size;
+
+	ptr = dsa_get_address(tbm->area, dsaptr);
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * tbm_free_shared
+ *
+ * Memory free function for tid bitmap
+ */
+void
+tbm_free_shared(void *pointer, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsa_data;
+
+	/*
+	 * If TBM is in iterating phase means pagetable was already created and we
+	 * have come here during tbm_free. So we need not to do anything because
+	 * by this time DSA would have been already freed.
+	 */
+	if (tbm->iterating)
+		return;
+
+	dsa_data = *((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+	dsa_free(tbm->area, dsa_data);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 3cf9417..bb262a6 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2825,6 +2825,31 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
 static int
 compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
 {
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 66edc54..4535016 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -860,8 +860,13 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+		cpu_run_cost = update_cost_for_parallelism(path, cpu_run_cost);
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 7b43c4a..e74a717 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1557,6 +1561,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1599,6 +1608,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index c7bcd9b..25fe7b6 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2803,7 +2803,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3b7c56d..bcf85a2 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3192,7 +3193,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index f37a0bf..0f15d04 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3386,6 +3386,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index ee7e05a..d7ebac4 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -178,6 +178,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index ce13bf7..06b9883 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1416,6 +1418,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t		mutex;			/* mutual exclusion for below three fields */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+}	ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	ParallelIterator tbmiterator;
+	ParallelIterator prefetch_iterator;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	Size		ptbm_offset;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1430,6 +1498,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1446,6 +1516,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1461,7 +1532,9 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1476,6 +1549,8 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..77d0fc6 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+} TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -62,5 +79,12 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+						   ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+						  ParallelTIDBitmap *parallel_tbm);
+void		tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
+TIDBitmap *tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 39376ec..1987d2e 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -183,6 +183,8 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
 					   double cte_rows);
 extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
 extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern double compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel,
+				Path *bitmapqual, int loop_count, Cost *cost, double *tuple);
 
 /*
  * prototypes for clausesel.c
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index d16f879..8209749 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 480f25f..0341b57 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 5b37894..5c2cc0c 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -784,6 +784,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#32tushar
tushar.ahuja@enterprisedb.com
In reply to: Dilip Kumar (#31)
Re: Parallel bitmap heap scan

On 01/09/2017 01:05 PM, Dilip Kumar wrote:

This patch can be used by 0003-parallel-bitmap-heap-scan-v7.patch
attached in the mail and also parallel-index-scan[2] can be rebased on
this, if this get committed,

After applying your patches against the fresh sources of PG v10 , not
able to perform initdb

centos@tusharcentos7 bin]$ ./initdb -D data
The files belonging to this database system will be owned by user "centos".
This user must also own the server process.

The database cluster will be initialized with locale "en_US.utf8".
The default database encoding has accordingly been set to "UTF8".
The default text search configuration will be set to "english".

Data page checksums are disabled.

creating directory data ... ok
creating subdirectories ... ok
selecting default max_connections ... 100
selecting default shared_buffers ... 128MB
selecting dynamic shared memory implementation ... posix
creating configuration files ... ok
running bootstrap script ... ok
performing post-bootstrap initialization ... sh: line 1: 30709
Segmentation fault
"/home/centos/PG10_9ja/postgresql/edbpsql/bin/postgres" --single -F -O
-j -c search_path=pg_catalog -c exit_on_error=true template1 > /dev/null
child process exited with exit code 139
initdb: removing data directory "data"
[centos@tusharcentos7 bin]$

--
regards,tushar
EnterpriseDB https://www.enterprisedb.com/
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#33Dilip Kumar
dilipbalaut@gmail.com
In reply to: tushar (#32)
Re: Parallel bitmap heap scan

On Mon, Jan 9, 2017 at 3:07 PM, tushar <tushar.ahuja@enterprisedb.com> wrote:

creating directory data ... ok
creating subdirectories ... ok
selecting default max_connections ... 100
selecting default shared_buffers ... 128MB
selecting dynamic shared memory implementation ... posix
creating configuration files ... ok
running bootstrap script ... ok
performing post-bootstrap initialization ... sh: line 1: 30709 Segmentation
fault "/home/centos/PG10_9ja/postgresql/edbpsql/bin/postgres" --single -F -O
-j -c search_path=pg_catalog -c exit_on_error=true template1 > /dev/null
child process exited with exit code 139
initdb: removing data directory "data"

I have taken the latest code, applied all 3 patches and compiled.
Initdb is working fine for me.

Can you please verify, do you have any extra patch along with my patch?
Did you properly clean the code?

I have asked one of my colleague to verify this and the result is
same, No crash.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#34tushar
tushar.ahuja@enterprisedb.com
In reply to: Dilip Kumar (#33)
Re: Parallel bitmap heap scan

On 01/09/2017 04:36 PM, Dilip Kumar wrote:

I have taken the latest code, applied all 3 patches and compiled.
Initdb is working fine for me.

Can you please verify, do you have any extra patch along with my patch?
Did you properly clean the code?

Thanks Dilip. issue is reproducible if we uses '--enable-cassert'
switch in the configure. We are able to reproduce it only with
--enable-cassert' .

--
regards,tushar
EnterpriseDB https://www.enterprisedb.com/
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#35Dilip Kumar
dilipbalaut@gmail.com
In reply to: tushar (#34)
3 attachment(s)
Re: Parallel bitmap heap scan

On Mon, Jan 9, 2017 at 5:01 PM, tushar <tushar.ahuja@enterprisedb.com> wrote:

Thanks Dilip. issue is reproducible if we uses '--enable-cassert' switch
in the configure. We are able to reproduce it only with --enable-cassert' .

Thanks, Tushar. I have fixed it. The defect was in 0002. I have also
observed another issue related to code refactoring, Actually, there
was some code present in 0001 which supposed to be in 0003.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0001-opt-parallelcost-refactoring-v8.patchapplication/octet-stream; name=0001-opt-parallelcost-refactoring-v8.patchDownload
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 46d7d06..3cf9417 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -126,6 +126,7 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static int	compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 
 
 /*
@@ -678,49 +679,7 @@ create_plain_partial_paths(PlannerInfo *root, RelOptInfo *rel)
 {
 	int			parallel_workers;
 
-	/*
-	 * If the user has set the parallel_workers reloption, use that; otherwise
-	 * select a default number of workers.
-	 */
-	if (rel->rel_parallel_workers != -1)
-		parallel_workers = rel->rel_parallel_workers;
-	else
-	{
-		int			parallel_threshold;
-
-		/*
-		 * If this relation is too small to be worth a parallel scan, just
-		 * return without doing anything ... unless it's an inheritance child.
-		 * In that case, we want to generate a parallel path here anyway.  It
-		 * might not be worthwhile just for this relation, but when combined
-		 * with all of its inheritance siblings it may well pay off.
-		 */
-		if (rel->pages < (BlockNumber) min_parallel_relation_size &&
-			rel->reloptkind == RELOPT_BASEREL)
-			return;
-
-		/*
-		 * Select the number of workers based on the log of the size of the
-		 * relation.  This probably needs to be a good deal more
-		 * sophisticated, but we need something here for now.  Note that the
-		 * upper limit of the min_parallel_relation_size GUC is chosen to
-		 * prevent overflow here.
-		 */
-		parallel_workers = 1;
-		parallel_threshold = Max(min_parallel_relation_size, 1);
-		while (rel->pages >= (BlockNumber) (parallel_threshold * 3))
-		{
-			parallel_workers++;
-			parallel_threshold *= 3;
-			if (parallel_threshold > INT_MAX / 3)
-				break;			/* avoid overflow */
-		}
-	}
-
-	/*
-	 * In no case use more than max_parallel_workers_per_gather workers.
-	 */
-	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+	parallel_workers = compute_parallel_worker(rel, rel->pages);
 
 	/* If any limit was set to zero, the user doesn't want a parallel scan. */
 	if (parallel_workers <= 0)
@@ -2866,6 +2825,59 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+static int
+compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
+{
+	int			parallel_workers;
+
+	/*
+	 * If the user has set the parallel_workers reloption, use that; otherwise
+	 * select a default number of workers.
+	 */
+	if (rel->rel_parallel_workers != -1)
+		parallel_workers = rel->rel_parallel_workers;
+	else
+	{
+		int			parallel_threshold;
+
+		/*
+		 * If this relation is too small to be worth a parallel scan, just
+		 * return without doing anything ... unless it's an inheritance child.
+		 * In that case, we want to generate a parallel path here anyway.  It
+		 * might not be worthwhile just for this relation, but when combined
+		 * with all of its inheritance siblings it may well pay off.
+		 */
+		if (pages < (BlockNumber) min_parallel_relation_size &&
+			rel->reloptkind == RELOPT_BASEREL)
+			return 0;
+
+		/*
+		 * Select the number of workers based on the log of the size of the
+		 * relation.  This probably needs to be a good deal more
+		 * sophisticated, but we need something here for now.  Note that the
+		 * upper limit of the min_parallel_relation_size GUC is chosen to
+		 * prevent overflow here.
+		 */
+		parallel_workers = 1;
+		parallel_threshold = Max(min_parallel_relation_size, 1);
+		while (pages >= (BlockNumber) (parallel_threshold * 3))
+		{
+			parallel_workers++;
+			parallel_threshold *= 3;
+			if (parallel_threshold > INT_MAX / 3)
+				break;			/* avoid overflow */
+		}
+	}
+
+	/*
+	 * In no case use more than max_parallel_workers_per_gather workers.
+	 */
+	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+
+	return parallel_workers;
+}
+
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a52eb7e..deb973b 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -161,6 +161,7 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
 static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
 static double relation_byte_size(double tuples, int width);
 static double page_size(double tuples, int width);
+static Cost update_cost_for_parallelism(Path *path, Cost cpu_run_cost);
 
 
 /*
@@ -237,44 +238,7 @@ cost_seqscan(Path *path, PlannerInfo *root,
 
 	/* Adjust costing for parallelism, if used. */
 	if (path->parallel_workers > 0)
-	{
-		double		parallel_divisor = path->parallel_workers;
-		double		leader_contribution;
-
-		/*
-		 * Early experience with parallel query suggests that when there is
-		 * only one worker, the leader often makes a very substantial
-		 * contribution to executing the parallel portion of the plan, but as
-		 * more workers are added, it does less and less, because it's busy
-		 * reading tuples from the workers and doing whatever non-parallel
-		 * post-processing is needed.  By the time we reach 4 workers, the
-		 * leader no longer makes a meaningful contribution.  Thus, for now,
-		 * estimate that the leader spends 30% of its time servicing each
-		 * worker, and the remainder executing the parallel plan.
-		 */
-		leader_contribution = 1.0 - (0.3 * path->parallel_workers);
-		if (leader_contribution > 0)
-			parallel_divisor += leader_contribution;
-
-		/*
-		 * In the case of a parallel plan, the row count needs to represent
-		 * the number of tuples processed per worker.  Otherwise, higher-level
-		 * plan nodes that appear below the gather will be costed incorrectly,
-		 * because they'll anticipate receiving more rows than any given copy
-		 * will actually get.
-		 */
-		path->rows = clamp_row_est(path->rows / parallel_divisor);
-
-		/* The CPU cost is divided among all the workers. */
-		cpu_run_cost /= parallel_divisor;
-
-		/*
-		 * It may be possible to amortize some of the I/O cost, but probably
-		 * not very much, because most operating systems already do aggressive
-		 * prefetching.  For now, we assume that the disk run cost can't be
-		 * amortized at all.
-		 */
-	}
+		cpu_run_cost = update_cost_for_parallelism(path, cpu_run_cost);
 
 	path->startup_cost = startup_cost;
 	path->total_cost = startup_cost + cpu_run_cost + disk_run_cost;
@@ -831,7 +795,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
 	Cost		indexTotalCost;
-	Selectivity indexSelectivity;
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
@@ -855,13 +818,12 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	if (!enable_bitmapscan)
 		startup_cost += disable_cost;
 
-	/*
-	 * Fetch total cost of obtaining the bitmap, as well as its total
-	 * selectivity.
-	 */
-	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+	pages_fetched = compute_bitmap_pages(root, baserel, bitmapqual,
+										 loop_count, &indexTotalCost,
+										 &tuples_fetched);
 
 	startup_cost += indexTotalCost;
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
 
 	/* Fetch estimated page costs for tablespace containing table. */
 	get_tablespace_page_costs(baserel->reltablespace,
@@ -869,41 +831,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 							  &spc_seq_page_cost);
 
 	/*
-	 * Estimate number of main-table pages fetched.
-	 */
-	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
-
-	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
-
-	if (loop_count > 1)
-	{
-		/*
-		 * For repeated bitmap scans, scale up the number of tuples fetched in
-		 * the Mackert and Lohman formula by the number of scans, so that we
-		 * estimate the number of pages fetched by all the scans. Then
-		 * pro-rate for one scan.
-		 */
-		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
-											baserel->pages,
-											get_indexpath_pages(bitmapqual),
-											root);
-		pages_fetched /= loop_count;
-	}
-	else
-	{
-		/*
-		 * For a single scan, the number of heap pages that need to be fetched
-		 * is the same as the Mackert and Lohman formula for the case T <= b
-		 * (ie, no re-reads needed).
-		 */
-		pages_fetched = (2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
-	}
-	if (pages_fetched >= T)
-		pages_fetched = T;
-	else
-		pages_fetched = ceil(pages_fetched);
-
-	/*
 	 * For small numbers of pages we should charge spc_random_page_cost
 	 * apiece, while if nearly all the table's pages are being read, it's more
 	 * appropriate to charge spc_seq_page_cost apiece.  The effect is
@@ -944,6 +871,56 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 }
 
 /*
+ * update_cost_for_parallelism
+ *
+ * Adjust the cpu cost based on number of parallel workers, also update
+ * the number of rows processed at each worker.
+ */
+static Cost
+update_cost_for_parallelism(Path *path, Cost cpu_run_cost)
+{
+	double		parallel_divisor = path->parallel_workers;
+	double		leader_contribution;
+	Cost		cpu_cost = cpu_run_cost;
+
+	/*
+	 * Early experience with parallel query suggests that when there is only
+	 * one worker, the leader often makes a very substantial contribution to
+	 * executing the parallel portion of the plan, but as more workers are
+	 * added, it does less and less, because it's busy reading tuples from the
+	 * workers and doing whatever non-parallel post-processing is needed.  By
+	 * the time we reach 4 workers, the leader no longer makes a meaningful
+	 * contribution.  Thus, for now, estimate that the leader spends 30% of
+	 * its time servicing each worker, and the remainder executing the
+	 * parallel plan.
+	 */
+	leader_contribution = 1.0 - (0.3 * path->parallel_workers);
+	if (leader_contribution > 0)
+		parallel_divisor += leader_contribution;
+
+	/*
+	 * In the case of a parallel plan, the row count needs to represent the
+	 * number of tuples processed per worker.  Otherwise, higher-level plan
+	 * nodes that appear below the gather will be costed incorrectly, because
+	 * they'll anticipate receiving more rows than any given copy will
+	 * actually get.
+	 */
+	path->rows = clamp_row_est(path->rows / parallel_divisor);
+
+	/* The CPU cost is divided among all the workers. */
+	cpu_cost /= parallel_divisor;
+
+	/*
+	 * It may be possible to amortize some of the I/O cost, but probably not
+	 * very much, because most operating systems already do aggressive
+	 * prefetching.  For now, we assume that the disk run cost can't be
+	 * amortized at all.
+	 */
+
+	return cpu_cost;
+}
+
+/*
  * cost_bitmap_tree_node
  *		Extract cost and selectivity from a bitmap tree node (index/and/or)
  */
@@ -4798,3 +4775,69 @@ page_size(double tuples, int width)
 {
 	return ceil(relation_byte_size(tuples, width) / BLCKSZ);
 }
+
+/*
+ * compute_bitmap_pages
+ *
+ * compute number of pages fetched from heap in bitmap heap scan.
+ */
+double
+compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel, Path *bitmapqual,
+					 int loop_count, Cost *cost, double *tuple)
+{
+	Cost		indexTotalCost;
+	Selectivity indexSelectivity;
+	double		T;
+	double		pages_fetched;
+	double		tuples_fetched;
+
+	/*
+	 * Fetch total cost of obtaining the bitmap, as well as its total
+	 * selectivity.
+	 */
+	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+
+	/*
+	 * Estimate number of main-table pages fetched.
+	 */
+	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
+
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
+
+	if (loop_count > 1)
+	{
+		/*
+		 * For repeated bitmap scans, scale up the number of tuples fetched in
+		 * the Mackert and Lohman formula by the number of scans, so that we
+		 * estimate the number of pages fetched by all the scans. Then
+		 * pro-rate for one scan.
+		 */
+		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
+											baserel->pages,
+											get_indexpath_pages(bitmapqual),
+											root);
+		pages_fetched /= loop_count;
+	}
+	else
+	{
+		/*
+		 * For a single scan, the number of heap pages that need to be fetched
+		 * is the same as the Mackert and Lohman formula for the case T <= b
+		 * (ie, no re-reads needed).
+		 */
+		pages_fetched =
+			(2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
+	}
+
+	if (pages_fetched >= T)
+		pages_fetched = T;
+	else
+		pages_fetched = ceil(pages_fetched);
+
+	if (cost)
+		*cost = indexTotalCost;
+	if (tuple)
+		*tuple = tuples_fetched;
+
+	return pages_fetched;
+}
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 39376ec..0e68264 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -183,6 +183,8 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
 					   double cte_rows);
 extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
 extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern double compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel,
+				  Path *bitmapqual, int loop_count, Cost *cost, double *tuple);
 
 /*
  * prototypes for clausesel.c
0002-hash-support-alloc-free-v8.patchapplication/octet-stream; name=0002-hash-support-alloc-free-v8.patchDownload
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 47c9656..14f8c2d 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -330,7 +330,7 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
 	else
 		hashtable->hash_iv = 0;
 
-	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
+	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, NULL);
 	hashtable->hashtab->private_data = hashtable;
 
 	return hashtable;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 7b31948..0885812 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -244,7 +244,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index 12aedbc..46604fe 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -66,6 +66,7 @@
 #define SH_STATUS_EMPTY SH_MAKE_NAME(EMPTY)
 #define SH_STATUS_IN_USE SH_MAKE_NAME(IN_USE)
 #define SH_ITERATOR SH_MAKE_NAME(iterator)
+#define SH_ALLOCATOR SH_MAKE_NAME(alloc)
 
 /* function declarations */
 #define SH_CREATE SH_MAKE_NAME(create)
@@ -78,6 +79,7 @@
 #define SH_START_ITERATE_AT SH_MAKE_NAME(start_iterate_at)
 #define SH_ITERATE SH_MAKE_NAME(iterate)
 #define SH_STAT SH_MAKE_NAME(stat)
+#define SH_COPY SH_MAKE_NAME(copy)
 
 /* internal helper functions (no externally visible prototypes) */
 #define SH_COMPUTE_PARAMETERS SH_MAKE_NAME(compute_parameters)
@@ -90,6 +92,18 @@
 /* generate forward declarations necessary to use the hash table */
 #ifdef SH_DECLARE
 
+typedef struct SH_ALLOCATOR
+{
+	/* Allocation function */
+	void	   *(*HashAlloc) (Size size, void *args);
+
+	/* Free function */
+	void		(*HashFree) (void *pointer, void *args);
+
+	/* Arguments to be passed to alloc and free functions */
+	void	   *args;
+}	SH_ALLOCATOR;
+
 /* type definitions */
 typedef struct SH_TYPE
 {
@@ -112,6 +126,9 @@ typedef struct SH_TYPE
 	/* hash buckets */
 	SH_ELEMENT_TYPE *data;
 
+	/* hash allocator */
+	SH_ALLOCATOR *alloc;
+
 	/* memory context to use for allocations */
 	MemoryContext ctx;
 
@@ -133,7 +150,8 @@ typedef struct SH_ITERATOR
 } SH_ITERATOR;
 
 /* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
+							SH_ALLOCATOR *alloc);
 SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
 SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
 SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -143,6 +161,7 @@ SH_SCOPE void SH_START_ITERATE(SH_TYPE *tb, SH_ITERATOR *iter);
 SH_SCOPE void SH_START_ITERATE_AT(SH_TYPE *tb, SH_ITERATOR *iter, uint32 at);
 SH_SCOPE SH_ELEMENT_TYPE *SH_ITERATE(SH_TYPE *tb, SH_ITERATOR *iter);
 SH_SCOPE void SH_STAT(SH_TYPE *tb);
+SH_SCOPE void SH_COPY(SH_TYPE *src_tb, SH_TYPE *dst_tb, SH_ELEMENT_TYPE *data);
 
 #endif   /* SH_DECLARE */
 
@@ -276,15 +295,38 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
 #endif
 }
 
+/* default memory allocator function */
+static void *
+SH_DEFAULT_ALLOC(Size size, void *args)
+{
+	MemoryContext context = (MemoryContext) args;
+
+	return MemoryContextAllocExtended(context, size,
+									  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+}
+
+/* default memory free function */
+static void
+SH_DEFAULT_FREE(void *pointer, void *args)
+{
+	pfree(pointer);
+}
+
 /*
  * Create a hash table with enough space for `nelements` distinct members,
  * allocating required memory in the passed-in context.
+ *
+ * WARNING : alloc is an allocator handle which provides memory allocator
+ * function for storing hash elements. User needs to be aware that providing
+ * allocator does not store complete hash table in the allocator memory i.e
+ * hash table will still be allocated in local memory. However all the
+ * elements will be stored in the memory provided by the allocator.
  */
 SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements)
+SH_CREATE(MemoryContext ctx, uint32 nelements, SH_ALLOCATOR *alloc)
 {
-	SH_TYPE    *tb;
-	uint64		size;
+	SH_TYPE     *tb;
+	uint64		 size;
 
 	tb = MemoryContextAllocZero(ctx, sizeof(SH_TYPE));
 	tb->ctx = ctx;
@@ -294,9 +336,18 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 
 	SH_COMPUTE_PARAMETERS(tb, size);
 
-	tb->data = MemoryContextAllocExtended(tb->ctx,
-										  sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	if (!alloc)
+	{
+		tb->alloc = palloc(sizeof(SH_ALLOCATOR));
+		tb->alloc->HashAlloc = SH_DEFAULT_ALLOC;
+		tb->alloc->HashFree = SH_DEFAULT_FREE;
+		tb->alloc->args = tb->ctx;
+	}
+	else
+		tb->alloc = alloc;
+
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	return tb;
 }
@@ -305,7 +356,12 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 SH_SCOPE void
 SH_DESTROY(SH_TYPE *tb)
 {
-	pfree(tb->data);
+	if (tb->alloc)
+	{
+		tb->alloc->HashFree(tb->data, tb->alloc->args);
+		pfree(tb->alloc);
+	}
+
 	pfree(tb);
 }
 
@@ -333,9 +389,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 	/* compute parameters for new table */
 	SH_COMPUTE_PARAMETERS(tb, newsize);
 
-	tb->data = MemoryContextAllocExtended(
-								 tb->ctx, sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	newdata = tb->data;
 
@@ -421,7 +476,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 		}
 	}
 
-	pfree(olddata);
+	tb->alloc->HashFree(olddata, tb->alloc->args);
+
 }
 
 /*
@@ -826,6 +882,18 @@ SH_STAT(SH_TYPE *tb)
 		 total_collisions, max_collisions, avg_collisions);
 }
 
+/*
+ * Copy information from src_tb to dst_tb, it only copies those informations
+ * which are required by SH_ITERATE. Also store input data as hashtable data.
+ */
+SH_SCOPE void
+SH_COPY (SH_TYPE *src_tb, SH_TYPE *dst_tb, SH_ELEMENT_TYPE *data)
+{
+	dst_tb->size = src_tb->size;
+	dst_tb->sizemask = src_tb->sizemask;
+	dst_tb->data = data;
+}
+
 #endif   /* SH_DEFINE */
 
 
0003-parallel-bitmap-heap-scan-v8.patchapplication/octet-stream; name=0003-parallel-bitmap-heap-scan-v8.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1545f03..cf2dbbd 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1232,6 +1232,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1ce42ea..76a99a1 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 86d9fb5..404c7ac 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -203,6 +204,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -255,6 +260,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -731,6 +741,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index d5fd57a..9df6dfd 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,16 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+					  ParallelIterator *parallel_iterator);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -68,11 +74,18 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
 
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (pbminfo)
+		parallel_tbm = (ParallelTIDBitmap *) ((char *) pbminfo +
+											  pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,6 +100,35 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	prefetch_iterator = node->prefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	The first worker to see the state of ParallelBitmapInfo as PBM_INITIAL
+	 *	becomes leader and sets the state to PBM_INPROGRESS. All other workers
+	 *	see the state as PBM_INPROGRESS, and will wait for leader to finish
+	 *	building the TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *		ParallelBitmapInfo so that other worker can see those.
+	 *	  Set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader creates shared TIDBitmap.
+	 *	  Copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over page and chunk array
+	 *			and select heap pages one by one. If prefetch is enable then
+	 *			there will be two iterators.
+	 *	  b) Since multiple workers are iterating over same page and chunk
+	 *			array we need to have a shared iterator, so we grab a spin
+	 *			lock and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,10 +143,56 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.
+		 *
+		 * In parallel mode leader will immediately come out of the function,
+		 * but all other worker will be blocked until leader wake them up.
+		 */
+		if (pbminfo == NULL || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag in lower level bitmap index scan. Later
+			 * bitmap index node will use this flag to indicate tidbitmap that
+			 * it needs to create an shared page table.
+			 */
+			if (pbminfo)
+				pbms_set_parallel(outerPlanState(node));
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			/*
+			 * copy TBM local information in shared memory before waking up
+			 * other workers. Other workers will create there own TBM and copy
+			 * information from shared memory.
+			 */
+			if (pbminfo)
+			{
+				tbm_update_shared_members(tbm, parallel_tbm);
+
+				/* Change the state under a lock */
+				SpinLockAcquire(&pbminfo->state_mutex);
+				pbminfo->state = PBM_FINISHED;
+				SpinLockRelease(&pbminfo->state_mutex);
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+				/* Wake up all other workers. */
+				ConditionVariableBroadcast(&pbminfo->cv);
+			}
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM. Here we
+			 * need to create a local TBM and copy information from shared
+			 * location. We also need to attach to shared page table using
+			 * hash table handle stored in parallel_tbm (shared memory).
+			 */
+			tbm = tbm_attach(parallel_tbm, node->ss.ps.state->es_query_dsa);
+		}
 
 		node->tbm = tbm;
 		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
@@ -118,19 +206,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			node->prefetch_target = -1;
 		}
 #endif   /* USE_PREFETCH */
+
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+									 pbminfo ? &pbminfo->tbmiterator : NULL);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +229,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (pbminfo)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+							   pbminfo ? &pbminfo->prefetch_iterator : NULL);
+
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not ensure
+				 * that current blockno in main iterator and prefetch iterator
+				 * is same. It's possible that whatever blockno we are
+				 * prefetching is getting processed by some other worker.
+				 */
+				if ((pbminfo == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +300,39 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				int		   *prefetch_target;
+
+				if (pbminfo == NULL)
+					prefetch_target = &node->prefetch_target;
+				else
+				{
+					prefetch_target = &pbminfo->prefetch_target;
+					/* If we are in parallel mode then grab prefetch_mutex */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+				}
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +348,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (pbminfo == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+					pbminfo->prefetch_target++;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +387,53 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+			TBMIterator *iterator = node->prefetch_iterator;
+			ParallelIterator *parallel_iteartor = NULL;
+			ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (pbminfo)
+			{
+				parallel_iteartor = &pbminfo->prefetch_iterator;
+				prefetch_pages = &pbminfo->prefetch_pages;
+				prefetch_target = pbminfo->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
+
+			while (*prefetch_pages < prefetch_target)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+														  parallel_iteartor);
 
 				if (tbmpre == NULL)
 				{
 					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					tbm_end_iterate(iterator);
+					prefetch_iterator = node->prefetch_iterator = NULL;
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +648,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +766,7 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -653,3 +853,218 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : then we need to wait till leader creates BITMAP
+		 * PBM_FINISHED	  : BITMAP is ready so no need to wait.
+		 *---------------
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Wrapper over tbm_iterate, which take care of copying and restoring iterator
+ * variables from parallel iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator, ParallelIterator *piterator)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (piterator == NULL)
+		return tbm_iterate(iterator);
+
+	/* We are in parallel mode so grab parallel iterator mutex */
+	SpinLockAcquire(&piterator->mutex);
+
+	/*
+	 * Copy information from shared location to local iterator and call
+	 * iterate. Then restore back to shared iterator.
+	 */
+	iterator->spageptr = piterator->spageptr;
+	iterator->schunkptr = piterator->schunkptr;
+	iterator->schunkbit = piterator->schunkbit;
+	output = tbm_iterate(iterator);
+	piterator->spageptr = iterator->spageptr;
+	piterator->schunkptr = iterator->schunkptr;
+	piterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&piterator->mutex);
+
+	return output;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node as
+	 * parallel, because only first node will create the main bitmap other
+	 * bitmaps will be merged to the first bitmap so no need to create them in
+	 * shared memory.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState *) node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..56f8376 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate that we
+	 * need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_dsa);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..4ff14c2 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -42,7 +42,14 @@
 
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
+#include "nodes/execnodes.h"
 #include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
+#include "storage/lwlock.h"
+#include "storage/shmem.h"
+#include "storage/spin.h"
+#include "utils/dsa.h"
+#include "utils/hsearch.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -136,27 +143,15 @@ struct TIDBitmap
 	bool		iterating;		/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
+	bool		is_shared;		/* need to build shared tbm if set */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	int			dsa_entries;	/* actual allocated entries */
+	dsa_area   *area;			/* reference to per-query shared memory area */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
 };
 
-/*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
- */
-struct TBMIterator
-{
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
-};
-
-
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
 static bool tbm_intersect_page(TIDBitmap *a, PagetableEntry *apage,
@@ -168,6 +163,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+void	   *tbm_alloc_shared(Size size, void *arg);
+void		tbm_free_shared(void *pointer, void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -198,6 +195,19 @@ hash_blockno(BlockNumber b)
 #define SH_DECLARE
 #include "lib/simplehash.h"
 
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+struct ParallelTIDBitmap
+{
+	dsa_pointer dsa_data;		/* dsa pointers for all kind of pages */
+	pagetable_hash pagetable;	/* hash table of PagetableEntry's */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	int			dsa_entries;	/* total item in dsa_pages */
+};
 
 /*
  * tbm_create - create an initially-empty bitmap
@@ -244,7 +254,18 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	if (tbm->is_shared)
+	{
+		pagetable_alloc *allocator = palloc(sizeof(pagetable_alloc));
+
+		allocator->args = tbm;
+		allocator->HashAlloc = tbm_alloc_shared;
+		allocator->HashFree = tbm_free_shared;
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, allocator);
+		tbm->dsa_entries = tbm->pagetable->size;
+	}
+	else
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -1061,3 +1082,136 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_update_shared_members
+ *
+ * Store leader's private tbm state to shared location. This must
+ * be called before waking up other workers.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before waking up the other
+	 * workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->dsa_data = tbm->dsa_data;
+	parallel_tbm->dsa_entries = tbm->dsa_entries;
+	pagetable_copy(tbm->pagetable, &parallel_tbm->pagetable, NULL);
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->is_shared = true;
+	tbm->area = (dsa_area *) area;
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ *
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size
+tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
+
+/*
+ * tbm_attach
+ *
+ * Create a local TIDBitmap for worker and Attach it to shared TID bitmap
+ * created by leader.
+ */
+TIDBitmap *
+tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area)
+{
+	TIDBitmap  *tbm = makeNode(TIDBitmap);
+	PagetableEntry *dsa_entry;
+
+	tbm->mcxt = CurrentMemoryContext;
+	tbm->status = TBM_HASH;
+	tbm->nchunks = parallel_tbm->nchunks;
+	tbm->nentries = parallel_tbm->nentries;
+	tbm->npages = parallel_tbm->npages;
+	tbm->maxentries = parallel_tbm->maxentries;
+	tbm->dsa_data = parallel_tbm->dsa_data;
+	tbm->dsa_entries = parallel_tbm->dsa_entries;
+	dsa_entry = dsa_get_address(area, tbm->dsa_data);
+	dsa_entry = (void *) (((char *) dsa_entry) + sizeof(dsa_pointer));
+	tbm->pagetable = palloc(sizeof(struct pagetable_hash));
+	pagetable_copy(&parallel_tbm->pagetable, tbm->pagetable, dsa_entry);
+
+	/* Mark the tbm as parallel and also keep the DSA reference in it. */
+	tbm_set_parallel(tbm, area);
+
+	return tbm;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Memory allocation function for bitmap hash.
+ * It allocates memory from DSA and also stores dsa_pointer in the memory
+ * so that during free we can directly get the dsa_pointe and free it.
+ */
+void *
+tbm_alloc_shared(Size size, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->area, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	/* Keep track of actual number of entries */
+	if (tbm->pagetable)
+		tbm->dsa_entries = tbm->pagetable->size;
+
+	ptr = dsa_get_address(tbm->area, dsaptr);
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * tbm_free_shared
+ *
+ * Memory free function for tid bitmap
+ */
+void
+tbm_free_shared(void *pointer, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsa_data;
+
+	/*
+	 * If TBM is in iterating phase means pagetable was already created and we
+	 * have come here during tbm_free. So we need not to do anything because
+	 * by this time DSA would have been already freed.
+	 */
+	if (tbm->iterating)
+		return;
+
+	dsa_data = *((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+	dsa_free(tbm->area, dsa_data);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 3cf9417..bb262a6 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2825,6 +2825,31 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
 static int
 compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
 {
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index deb973b..4535016 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -798,6 +798,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -859,8 +860,13 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+		cpu_run_cost = update_cost_for_parallelism(path, cpu_run_cost);
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 7b43c4a..e74a717 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1557,6 +1561,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1599,6 +1608,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index c7bcd9b..25fe7b6 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2803,7 +2803,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3b7c56d..bcf85a2 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3192,7 +3193,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index f37a0bf..0f15d04 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3386,6 +3386,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index ee7e05a..d7ebac4 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -178,6 +178,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index ce13bf7..06b9883 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1416,6 +1418,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t		mutex;			/* mutual exclusion for below three fields */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+}	ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	ParallelIterator tbmiterator;
+	ParallelIterator prefetch_iterator;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	Size		ptbm_offset;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1430,6 +1498,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1446,6 +1516,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1461,7 +1532,9 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1476,6 +1549,8 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..77d0fc6 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+} TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -62,5 +79,12 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+						   ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+						  ParallelTIDBitmap *parallel_tbm);
+void		tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
+TIDBitmap *tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index d16f879..8209749 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 480f25f..0341b57 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 5b37894..5c2cc0c 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -784,6 +784,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#36tushar
tushar.ahuja@enterprisedb.com
In reply to: Dilip Kumar (#35)
Re: Parallel bitmap heap scan

On 01/09/2017 07:22 PM, Dilip Kumar wrote:

Thanks, Tushar. I have fixed it. The defect was in 0002. I have also
observed another issue related to code refactoring, Actually, there
was some code present in 0001 which supposed to be in 0003.

Thanks, I have checked at my end and it is fixed now.

--
regards,tushar
EnterpriseDB https://www.enterprisedb.com/
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#37tushar
tushar.ahuja@enterprisedb.com
In reply to: tushar (#36)
Re: Parallel bitmap heap scan

On 01/10/2017 11:29 AM, tushar wrote:

On 01/09/2017 07:22 PM, Dilip Kumar wrote:

Thanks, Tushar. I have fixed it. The defect was in 0002. I have also
observed another issue related to code refactoring, Actually, there
was some code present in 0001 which supposed to be in 0003.

Thanks, I have checked at my end and it is fixed now.

We found a regression , earlier the testcase was working fine (against
the older patches of Parallel bitmap heap scan) but now getting a
server crash
against v8 patches.

Testcase - (one of the table of TPC-H )

postgres=#explain analyze verbose
SELECT SUM(l_extendedprice) FROM lineitem
WHERE (l_shipdate >= '1995-01-01'::date)
AND (l_shipdate <='1996-03-31'::date);
WARNING: terminating connection because of crash of another server process
DETAIL: The postmaster has commanded this server process to roll back
the current transaction and exit, because another server process exited
abnormally and possibly corrupted shared memory.
HINT: In a moment you should be able to reconnect to the database and
repeat your command.
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
connection to server was lost

Here is the stack trace - ( Two core dump file generated)

[centos@centos-cpula bin]$ gdb -q -c data/core.25434
/home/centos/PG10_10jan/postgresql/edbpsql/bin/postgres
Reading symbols from
/home/centos/PG10_10jan/postgresql/edbpsql/bin/postgres...done.
[New Thread 25434]
Missing separate debuginfo for
Try: yum --enablerepo='*-debug*' install
/usr/lib/debug/.build-id/7f/719af91ee951b4fcb6647e7868f95f766a616b
Reading symbols from /usr/lib64/libssl.so.10...(no debugging symbols
found)...done.
Loaded symbols for /usr/lib64/libssl.so.10
Reading symbols from /usr/lib64/libcrypto.so.10...(no debugging symbols
found)...done.
Loaded symbols for /usr/lib64/libcrypto.so.10
Reading symbols from /lib64/librt.so.1...(no debugging symbols
found)...done.
Loaded symbols for /lib64/librt.so.1
Reading symbols from /lib64/libdl.so.2...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libdl.so.2
Reading symbols from /lib64/libm.so.6...(no debugging symbols found)...done.
Loaded symbols for /lib64/libm.so.6
Reading symbols from /lib64/libc.so.6...(no debugging symbols found)...done.
Loaded symbols for /lib64/libc.so.6
Reading symbols from /lib64/libpthread.so.0...(no debugging symbols
found)...done.
[Thread debugging using libthread_db enabled]
Loaded symbols for /lib64/libpthread.so.0
Reading symbols from /lib64/libgssapi_krb5.so.2...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libgssapi_krb5.so.2
Reading symbols from /lib64/libkrb5.so.3...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libkrb5.so.3
Reading symbols from /lib64/libcom_err.so.2...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libcom_err.so.2
Reading symbols from /lib64/libk5crypto.so.3...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libk5crypto.so.3
Reading symbols from /lib64/libz.so.1...(no debugging symbols found)...done.
Loaded symbols for /lib64/libz.so.1
Reading symbols from /lib64/ld-linux-x86-64.so.2...(no debugging symbols
found)...done.
Loaded symbols for /lib64/ld-linux-x86-64.so.2
Reading symbols from /lib64/libkrb5support.so.0...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libkrb5support.so.0
Reading symbols from /lib64/libkeyutils.so.1...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libkeyutils.so.1
Reading symbols from /lib64/libresolv.so.2...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libresolv.so.2
Reading symbols from /lib64/libselinux.so.1...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libselinux.so.1
Reading symbols from /lib64/libnss_files.so.2...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libnss_files.so.2
Core was generated by `postgres: bgworker: parallel worker for PID
25433 '.
Program terminated with signal 11, Segmentation fault.
#0 0x00000000006f2fa6 in pagetable_destroy (tb=0x2079bf0) at
../../../src/include/lib/simplehash.h:361
361 tb->alloc->HashFree(tb->data, tb->alloc->args);
Missing separate debuginfos, use: debuginfo-install
glibc-2.12-1.192.el6.x86_64 keyutils-libs-1.4-5.el6.x86_64
krb5-libs-1.10.3-57.el6.x86_64 libcom_err-1.41.12-22.el6.x86_64
libselinux-2.0.94-7.el6.x86_64 openssl-1.0.1e-48.el6_8.1.x86_64
zlib-1.2.3-29.el6.x86_64
(gdb) bt
#0 0x00000000006f2fa6 in pagetable_destroy (tb=0x2079bf0) at
../../../src/include/lib/simplehash.h:361
#1 0x00000000006f3b52 in tbm_free (tbm=0x2077fe0) at tidbitmap.c:296
#2 0x00000000006ab29b in ExecEndBitmapHeapScan (node=0x207e760) at
nodeBitmapHeapscan.c:717
#3 0x0000000000691701 in ExecEndNode (node=0x207e760) at execProcnode.c:689
#4 0x00000000006a8f86 in ExecEndAgg (node=0x207e878) at nodeAgg.c:3563
#5 0x0000000000691800 in ExecEndNode (node=0x207e878) at execProcnode.c:755
#6 0x000000000068cb0f in ExecEndPlan (planstate=0x207e878,
estate=0x207df58) at execMain.c:1469
#7 0x000000000068afb7 in standard_ExecutorEnd (queryDesc=0x20592e8) at
execMain.c:470
#8 0x000000000068aef6 in ExecutorEnd (queryDesc=0x20592e8) at
execMain.c:441
#9 0x0000000000690ba5 in ParallelQueryMain (seg=0x1fe1e58,
toc=0x7fcbc3677000) at execParallel.c:819
#10 0x0000000000518620 in ParallelWorkerMain (main_arg=300704707) at
parallel.c:1109
#11 0x0000000000793db2 in StartBackgroundWorker () at bgworker.c:747
#12 0x00000000007a6265 in do_start_bgworker (rw=0x20039f0) at
postmaster.c:5560
#13 0x00000000007a659f in maybe_start_bgworker () at postmaster.c:5744
#14 0x00000000007a5706 in sigusr1_handler (postgres_signal_arg=10) at
postmaster.c:4998
#15 <signal handler called>
#16 0x0000003746ee1503 in __select_nocancel () from /lib64/libc.so.6
#17 0x00000000007a10ca in ServerLoop () at postmaster.c:1676
#18 0x00000000007a08bf in PostmasterMain (argc=3, argv=0x1fdfd20) at
postmaster.c:1320
#19 0x00000000006e7816 in main (argc=3, argv=0x1fdfd20) at main.c:228
(gdb)

[centos@centos-cpula bin]$ gdb -q -c data/core.25435
/home/centos/PG10_10jan/postgresql/edbpsql/bin/postgres
Reading symbols from
/home/centos/PG10_10jan/postgresql/edbpsql/bin/postgres...done.
[New Thread 25435]
Missing separate debuginfo for
Try: yum --enablerepo='*-debug*' install
/usr/lib/debug/.build-id/7f/719af91ee951b4fcb6647e7868f95f766a616b
Reading symbols from /usr/lib64/libssl.so.10...(no debugging symbols
found)...done.
Loaded symbols for /usr/lib64/libssl.so.10
Reading symbols from /usr/lib64/libcrypto.so.10...(no debugging symbols
found)...done.
Loaded symbols for /usr/lib64/libcrypto.so.10
Reading symbols from /lib64/librt.so.1...(no debugging symbols
found)...done.
Loaded symbols for /lib64/librt.so.1
Reading symbols from /lib64/libdl.so.2...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libdl.so.2
Reading symbols from /lib64/libm.so.6...(no debugging symbols found)...done.
Loaded symbols for /lib64/libm.so.6
Reading symbols from /lib64/libc.so.6...(no debugging symbols found)...done.
Loaded symbols for /lib64/libc.so.6
Reading symbols from /lib64/libpthread.so.0...(no debugging symbols
found)...done.
[Thread debugging using libthread_db enabled]
Loaded symbols for /lib64/libpthread.so.0
Reading symbols from /lib64/libgssapi_krb5.so.2...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libgssapi_krb5.so.2
Reading symbols from /lib64/libkrb5.so.3...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libkrb5.so.3
Reading symbols from /lib64/libcom_err.so.2...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libcom_err.so.2
Reading symbols from /lib64/libk5crypto.so.3...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libk5crypto.so.3
Reading symbols from /lib64/libz.so.1...(no debugging symbols found)...done.
Loaded symbols for /lib64/libz.so.1
Reading symbols from /lib64/ld-linux-x86-64.so.2...(no debugging symbols
found)...done.
Loaded symbols for /lib64/ld-linux-x86-64.so.2
Reading symbols from /lib64/libkrb5support.so.0...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libkrb5support.so.0
Reading symbols from /lib64/libkeyutils.so.1...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libkeyutils.so.1
Reading symbols from /lib64/libresolv.so.2...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libresolv.so.2
Reading symbols from /lib64/libselinux.so.1...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libselinux.so.1
Reading symbols from /lib64/libnss_files.so.2...(no debugging symbols
found)...done.
Loaded symbols for /lib64/libnss_files.so.2
Core was generated by `postgres: bgworker: parallel worker for PID
25433 '.
Program terminated with signal 11, Segmentation fault.
#0 0x00000000006f2fa6 in pagetable_destroy (tb=0x20775a0) at
../../../src/include/lib/simplehash.h:361
361 tb->alloc->HashFree(tb->data, tb->alloc->args);
Missing separate debuginfos, use: debuginfo-install
glibc-2.12-1.192.el6.x86_64 keyutils-libs-1.4-5.el6.x86_64
krb5-libs-1.10.3-57.el6.x86_64 libcom_err-1.41.12-22.el6.x86_64
libselinux-2.0.94-7.el6.x86_64 openssl-1.0.1e-48.el6_8.1.x86_64
zlib-1.2.3-29.el6.x86_64
(gdb) bt
#0 0x00000000006f2fa6 in pagetable_destroy (tb=0x20775a0) at
../../../src/include/lib/simplehash.h:361
#1 0x00000000006f3b52 in tbm_free (tbm=0x2075990) at tidbitmap.c:296
#2 0x00000000006ab29b in ExecEndBitmapHeapScan (node=0x207c110) at
nodeBitmapHeapscan.c:717
#3 0x0000000000691701 in ExecEndNode (node=0x207c110) at execProcnode.c:689
#4 0x00000000006a8f86 in ExecEndAgg (node=0x207c228) at nodeAgg.c:3563
#5 0x0000000000691800 in ExecEndNode (node=0x207c228) at execProcnode.c:755
#6 0x000000000068cb0f in ExecEndPlan (planstate=0x207c228,
estate=0x207b908) at execMain.c:1469
#7 0x000000000068afb7 in standard_ExecutorEnd (queryDesc=0x20594b8) at
execMain.c:470
#8 0x000000000068aef6 in ExecutorEnd (queryDesc=0x20594b8) at
execMain.c:441
#9 0x0000000000690ba5 in ParallelQueryMain (seg=0x1fe1e58,
toc=0x7fcbc3677000) at execParallel.c:819
#10 0x0000000000518620 in ParallelWorkerMain (main_arg=300704707) at
parallel.c:1109
#11 0x0000000000793db2 in StartBackgroundWorker () at bgworker.c:747
#12 0x00000000007a6265 in do_start_bgworker (rw=0x20064a0) at
postmaster.c:5560
#13 0x00000000007a659f in maybe_start_bgworker () at postmaster.c:5744
#14 0x00000000007a5706 in sigusr1_handler (postgres_signal_arg=10) at
postmaster.c:4998
#15 <signal handler called>
#16 0x0000003746ee1503 in __select_nocancel () from /lib64/libc.so.6
#17 0x00000000007a10ca in ServerLoop () at postmaster.c:1676
#18 0x00000000007a08bf in PostmasterMain (argc=3, argv=0x1fdfd20) at
postmaster.c:1320
#19 0x00000000006e7816 in main (argc=3, argv=0x1fdfd20) at main.c:228
(gdb)

--
regards,tushar
EnterpriseDB https://www.enterprisedb.com/
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#38Dilip Kumar
dilipbalaut@gmail.com
In reply to: tushar (#37)
3 attachment(s)
Re: Parallel bitmap heap scan

On Tue, Jan 10, 2017 at 2:19 PM, tushar <tushar.ahuja@enterprisedb.com> wrote:

We found a regression , earlier the testcase was working fine (against the
older patches of Parallel bitmap heap scan) but now getting a server crash
against v8 patches.

Testcase - (one of the table of TPC-H )

postgres=#explain analyze verbose
SELECT SUM(l_extendedprice) FROM lineitem
WHERE (l_shipdate >= '1995-01-01'::date)
AND (l_shipdate <='1996-03-31'::date);

While fixing some of the review comments in v7 and v8, I had allocated
new memory for pagetable, and missed to initialize it.

Thanks for reporting, I have fixed it in v9. After fix query is
running fine for me. Please try attached patch and confirm from your
side.

postgres=# explain analyze verbose
SELECT SUM(l_extendedprice) FROM lineitem
WHERE (l_shipdate >= '1995-01-01'::date)
AND (l_shipdate <='1996-03-31'::date);

QUERY PLAN

--------------------------------------------------------------------------------------------------------------------------------------------------------------
-----------
Finalize Aggregate (cost=798002.46..798002.47 rows=1 width=32)
(actual time=15501.245..15501.245 rows=1 loops=1)
Output: sum(l_extendedprice)
-> Gather (cost=798002.24..798002.45 rows=2 width=32) (actual
time=15494.358..15498.919 rows=3 loops=1)
Output: (PARTIAL sum(l_extendedprice))
Workers Planned: 2
Workers Launched: 2
-> Partial Aggregate (cost=797002.24..797002.25 rows=1
width=32) (actual time=15492.937..15492.937 rows=1 loops=3)
Output: PARTIAL sum(l_extendedprice)
Worker 0: actual time=15491.218..15491.219 rows=1 loops=1
Worker 1: actual time=15493.514..15493.514 rows=1 loops=1
-> Parallel Bitmap Heap Scan on public.lineitem
(cost=147461.75..791014.31 rows=2395170 width=8) (actual
time=8553.301..15061.333 rows=189294
7 loops=3)
Output: l_extendedprice
Recheck Cond: ((lineitem.l_shipdate >=
'1995-01-01'::date) AND (lineitem.l_shipdate <= '1996-03-31'::date))
Rows Removed by Index Recheck: 6451177
Heap Blocks: exact=27963 lossy=164938
Worker 0: actual time=8548.957..15054.511
rows=1887239 loops=1
Worker 1: actual time=8554.817..15050.317
rows=1902477 loops=1
-> Bitmap Index Scan on idx_lineitem_shipdate
(cost=0.00..146024.65 rows=5748409 width=0) (actual
time=8533.701..8533.701 rows=5678841
loops=1)
Index Cond: ((lineitem.l_shipdate >=
'1995-01-01'::date) AND (lineitem.l_shipdate <= '1996-03-31'::date))
Planning time: 2.742 ms
Execution time: 15509.696 ms
(21 rows)

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0001-opt-parallelcost-refactoring-v9.patchapplication/octet-stream; name=0001-opt-parallelcost-refactoring-v9.patchDownload
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 46d7d06..3cf9417 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -126,6 +126,7 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static int	compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 
 
 /*
@@ -678,49 +679,7 @@ create_plain_partial_paths(PlannerInfo *root, RelOptInfo *rel)
 {
 	int			parallel_workers;
 
-	/*
-	 * If the user has set the parallel_workers reloption, use that; otherwise
-	 * select a default number of workers.
-	 */
-	if (rel->rel_parallel_workers != -1)
-		parallel_workers = rel->rel_parallel_workers;
-	else
-	{
-		int			parallel_threshold;
-
-		/*
-		 * If this relation is too small to be worth a parallel scan, just
-		 * return without doing anything ... unless it's an inheritance child.
-		 * In that case, we want to generate a parallel path here anyway.  It
-		 * might not be worthwhile just for this relation, but when combined
-		 * with all of its inheritance siblings it may well pay off.
-		 */
-		if (rel->pages < (BlockNumber) min_parallel_relation_size &&
-			rel->reloptkind == RELOPT_BASEREL)
-			return;
-
-		/*
-		 * Select the number of workers based on the log of the size of the
-		 * relation.  This probably needs to be a good deal more
-		 * sophisticated, but we need something here for now.  Note that the
-		 * upper limit of the min_parallel_relation_size GUC is chosen to
-		 * prevent overflow here.
-		 */
-		parallel_workers = 1;
-		parallel_threshold = Max(min_parallel_relation_size, 1);
-		while (rel->pages >= (BlockNumber) (parallel_threshold * 3))
-		{
-			parallel_workers++;
-			parallel_threshold *= 3;
-			if (parallel_threshold > INT_MAX / 3)
-				break;			/* avoid overflow */
-		}
-	}
-
-	/*
-	 * In no case use more than max_parallel_workers_per_gather workers.
-	 */
-	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+	parallel_workers = compute_parallel_worker(rel, rel->pages);
 
 	/* If any limit was set to zero, the user doesn't want a parallel scan. */
 	if (parallel_workers <= 0)
@@ -2866,6 +2825,59 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+static int
+compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
+{
+	int			parallel_workers;
+
+	/*
+	 * If the user has set the parallel_workers reloption, use that; otherwise
+	 * select a default number of workers.
+	 */
+	if (rel->rel_parallel_workers != -1)
+		parallel_workers = rel->rel_parallel_workers;
+	else
+	{
+		int			parallel_threshold;
+
+		/*
+		 * If this relation is too small to be worth a parallel scan, just
+		 * return without doing anything ... unless it's an inheritance child.
+		 * In that case, we want to generate a parallel path here anyway.  It
+		 * might not be worthwhile just for this relation, but when combined
+		 * with all of its inheritance siblings it may well pay off.
+		 */
+		if (pages < (BlockNumber) min_parallel_relation_size &&
+			rel->reloptkind == RELOPT_BASEREL)
+			return 0;
+
+		/*
+		 * Select the number of workers based on the log of the size of the
+		 * relation.  This probably needs to be a good deal more
+		 * sophisticated, but we need something here for now.  Note that the
+		 * upper limit of the min_parallel_relation_size GUC is chosen to
+		 * prevent overflow here.
+		 */
+		parallel_workers = 1;
+		parallel_threshold = Max(min_parallel_relation_size, 1);
+		while (pages >= (BlockNumber) (parallel_threshold * 3))
+		{
+			parallel_workers++;
+			parallel_threshold *= 3;
+			if (parallel_threshold > INT_MAX / 3)
+				break;			/* avoid overflow */
+		}
+	}
+
+	/*
+	 * In no case use more than max_parallel_workers_per_gather workers.
+	 */
+	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+
+	return parallel_workers;
+}
+
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a52eb7e..deb973b 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -161,6 +161,7 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
 static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
 static double relation_byte_size(double tuples, int width);
 static double page_size(double tuples, int width);
+static Cost update_cost_for_parallelism(Path *path, Cost cpu_run_cost);
 
 
 /*
@@ -237,44 +238,7 @@ cost_seqscan(Path *path, PlannerInfo *root,
 
 	/* Adjust costing for parallelism, if used. */
 	if (path->parallel_workers > 0)
-	{
-		double		parallel_divisor = path->parallel_workers;
-		double		leader_contribution;
-
-		/*
-		 * Early experience with parallel query suggests that when there is
-		 * only one worker, the leader often makes a very substantial
-		 * contribution to executing the parallel portion of the plan, but as
-		 * more workers are added, it does less and less, because it's busy
-		 * reading tuples from the workers and doing whatever non-parallel
-		 * post-processing is needed.  By the time we reach 4 workers, the
-		 * leader no longer makes a meaningful contribution.  Thus, for now,
-		 * estimate that the leader spends 30% of its time servicing each
-		 * worker, and the remainder executing the parallel plan.
-		 */
-		leader_contribution = 1.0 - (0.3 * path->parallel_workers);
-		if (leader_contribution > 0)
-			parallel_divisor += leader_contribution;
-
-		/*
-		 * In the case of a parallel plan, the row count needs to represent
-		 * the number of tuples processed per worker.  Otherwise, higher-level
-		 * plan nodes that appear below the gather will be costed incorrectly,
-		 * because they'll anticipate receiving more rows than any given copy
-		 * will actually get.
-		 */
-		path->rows = clamp_row_est(path->rows / parallel_divisor);
-
-		/* The CPU cost is divided among all the workers. */
-		cpu_run_cost /= parallel_divisor;
-
-		/*
-		 * It may be possible to amortize some of the I/O cost, but probably
-		 * not very much, because most operating systems already do aggressive
-		 * prefetching.  For now, we assume that the disk run cost can't be
-		 * amortized at all.
-		 */
-	}
+		cpu_run_cost = update_cost_for_parallelism(path, cpu_run_cost);
 
 	path->startup_cost = startup_cost;
 	path->total_cost = startup_cost + cpu_run_cost + disk_run_cost;
@@ -831,7 +795,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
 	Cost		indexTotalCost;
-	Selectivity indexSelectivity;
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
@@ -855,13 +818,12 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	if (!enable_bitmapscan)
 		startup_cost += disable_cost;
 
-	/*
-	 * Fetch total cost of obtaining the bitmap, as well as its total
-	 * selectivity.
-	 */
-	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+	pages_fetched = compute_bitmap_pages(root, baserel, bitmapqual,
+										 loop_count, &indexTotalCost,
+										 &tuples_fetched);
 
 	startup_cost += indexTotalCost;
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
 
 	/* Fetch estimated page costs for tablespace containing table. */
 	get_tablespace_page_costs(baserel->reltablespace,
@@ -869,41 +831,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 							  &spc_seq_page_cost);
 
 	/*
-	 * Estimate number of main-table pages fetched.
-	 */
-	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
-
-	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
-
-	if (loop_count > 1)
-	{
-		/*
-		 * For repeated bitmap scans, scale up the number of tuples fetched in
-		 * the Mackert and Lohman formula by the number of scans, so that we
-		 * estimate the number of pages fetched by all the scans. Then
-		 * pro-rate for one scan.
-		 */
-		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
-											baserel->pages,
-											get_indexpath_pages(bitmapqual),
-											root);
-		pages_fetched /= loop_count;
-	}
-	else
-	{
-		/*
-		 * For a single scan, the number of heap pages that need to be fetched
-		 * is the same as the Mackert and Lohman formula for the case T <= b
-		 * (ie, no re-reads needed).
-		 */
-		pages_fetched = (2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
-	}
-	if (pages_fetched >= T)
-		pages_fetched = T;
-	else
-		pages_fetched = ceil(pages_fetched);
-
-	/*
 	 * For small numbers of pages we should charge spc_random_page_cost
 	 * apiece, while if nearly all the table's pages are being read, it's more
 	 * appropriate to charge spc_seq_page_cost apiece.  The effect is
@@ -944,6 +871,56 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 }
 
 /*
+ * update_cost_for_parallelism
+ *
+ * Adjust the cpu cost based on number of parallel workers, also update
+ * the number of rows processed at each worker.
+ */
+static Cost
+update_cost_for_parallelism(Path *path, Cost cpu_run_cost)
+{
+	double		parallel_divisor = path->parallel_workers;
+	double		leader_contribution;
+	Cost		cpu_cost = cpu_run_cost;
+
+	/*
+	 * Early experience with parallel query suggests that when there is only
+	 * one worker, the leader often makes a very substantial contribution to
+	 * executing the parallel portion of the plan, but as more workers are
+	 * added, it does less and less, because it's busy reading tuples from the
+	 * workers and doing whatever non-parallel post-processing is needed.  By
+	 * the time we reach 4 workers, the leader no longer makes a meaningful
+	 * contribution.  Thus, for now, estimate that the leader spends 30% of
+	 * its time servicing each worker, and the remainder executing the
+	 * parallel plan.
+	 */
+	leader_contribution = 1.0 - (0.3 * path->parallel_workers);
+	if (leader_contribution > 0)
+		parallel_divisor += leader_contribution;
+
+	/*
+	 * In the case of a parallel plan, the row count needs to represent the
+	 * number of tuples processed per worker.  Otherwise, higher-level plan
+	 * nodes that appear below the gather will be costed incorrectly, because
+	 * they'll anticipate receiving more rows than any given copy will
+	 * actually get.
+	 */
+	path->rows = clamp_row_est(path->rows / parallel_divisor);
+
+	/* The CPU cost is divided among all the workers. */
+	cpu_cost /= parallel_divisor;
+
+	/*
+	 * It may be possible to amortize some of the I/O cost, but probably not
+	 * very much, because most operating systems already do aggressive
+	 * prefetching.  For now, we assume that the disk run cost can't be
+	 * amortized at all.
+	 */
+
+	return cpu_cost;
+}
+
+/*
  * cost_bitmap_tree_node
  *		Extract cost and selectivity from a bitmap tree node (index/and/or)
  */
@@ -4798,3 +4775,69 @@ page_size(double tuples, int width)
 {
 	return ceil(relation_byte_size(tuples, width) / BLCKSZ);
 }
+
+/*
+ * compute_bitmap_pages
+ *
+ * compute number of pages fetched from heap in bitmap heap scan.
+ */
+double
+compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel, Path *bitmapqual,
+					 int loop_count, Cost *cost, double *tuple)
+{
+	Cost		indexTotalCost;
+	Selectivity indexSelectivity;
+	double		T;
+	double		pages_fetched;
+	double		tuples_fetched;
+
+	/*
+	 * Fetch total cost of obtaining the bitmap, as well as its total
+	 * selectivity.
+	 */
+	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+
+	/*
+	 * Estimate number of main-table pages fetched.
+	 */
+	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
+
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
+
+	if (loop_count > 1)
+	{
+		/*
+		 * For repeated bitmap scans, scale up the number of tuples fetched in
+		 * the Mackert and Lohman formula by the number of scans, so that we
+		 * estimate the number of pages fetched by all the scans. Then
+		 * pro-rate for one scan.
+		 */
+		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
+											baserel->pages,
+											get_indexpath_pages(bitmapqual),
+											root);
+		pages_fetched /= loop_count;
+	}
+	else
+	{
+		/*
+		 * For a single scan, the number of heap pages that need to be fetched
+		 * is the same as the Mackert and Lohman formula for the case T <= b
+		 * (ie, no re-reads needed).
+		 */
+		pages_fetched =
+			(2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
+	}
+
+	if (pages_fetched >= T)
+		pages_fetched = T;
+	else
+		pages_fetched = ceil(pages_fetched);
+
+	if (cost)
+		*cost = indexTotalCost;
+	if (tuple)
+		*tuple = tuples_fetched;
+
+	return pages_fetched;
+}
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 39376ec..0e68264 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -183,6 +183,8 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
 					   double cte_rows);
 extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
 extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern double compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel,
+				  Path *bitmapqual, int loop_count, Cost *cost, double *tuple);
 
 /*
  * prototypes for clausesel.c
0002-hash-support-alloc-free-v9.patchapplication/octet-stream; name=0002-hash-support-alloc-free-v9.patchDownload
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 47c9656..14f8c2d 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -330,7 +330,7 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
 	else
 		hashtable->hash_iv = 0;
 
-	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
+	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, NULL);
 	hashtable->hashtab->private_data = hashtable;
 
 	return hashtable;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 7b31948..0885812 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -244,7 +244,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index 12aedbc..46604fe 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -66,6 +66,7 @@
 #define SH_STATUS_EMPTY SH_MAKE_NAME(EMPTY)
 #define SH_STATUS_IN_USE SH_MAKE_NAME(IN_USE)
 #define SH_ITERATOR SH_MAKE_NAME(iterator)
+#define SH_ALLOCATOR SH_MAKE_NAME(alloc)
 
 /* function declarations */
 #define SH_CREATE SH_MAKE_NAME(create)
@@ -78,6 +79,7 @@
 #define SH_START_ITERATE_AT SH_MAKE_NAME(start_iterate_at)
 #define SH_ITERATE SH_MAKE_NAME(iterate)
 #define SH_STAT SH_MAKE_NAME(stat)
+#define SH_COPY SH_MAKE_NAME(copy)
 
 /* internal helper functions (no externally visible prototypes) */
 #define SH_COMPUTE_PARAMETERS SH_MAKE_NAME(compute_parameters)
@@ -90,6 +92,18 @@
 /* generate forward declarations necessary to use the hash table */
 #ifdef SH_DECLARE
 
+typedef struct SH_ALLOCATOR
+{
+	/* Allocation function */
+	void	   *(*HashAlloc) (Size size, void *args);
+
+	/* Free function */
+	void		(*HashFree) (void *pointer, void *args);
+
+	/* Arguments to be passed to alloc and free functions */
+	void	   *args;
+}	SH_ALLOCATOR;
+
 /* type definitions */
 typedef struct SH_TYPE
 {
@@ -112,6 +126,9 @@ typedef struct SH_TYPE
 	/* hash buckets */
 	SH_ELEMENT_TYPE *data;
 
+	/* hash allocator */
+	SH_ALLOCATOR *alloc;
+
 	/* memory context to use for allocations */
 	MemoryContext ctx;
 
@@ -133,7 +150,8 @@ typedef struct SH_ITERATOR
 } SH_ITERATOR;
 
 /* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
+							SH_ALLOCATOR *alloc);
 SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
 SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
 SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -143,6 +161,7 @@ SH_SCOPE void SH_START_ITERATE(SH_TYPE *tb, SH_ITERATOR *iter);
 SH_SCOPE void SH_START_ITERATE_AT(SH_TYPE *tb, SH_ITERATOR *iter, uint32 at);
 SH_SCOPE SH_ELEMENT_TYPE *SH_ITERATE(SH_TYPE *tb, SH_ITERATOR *iter);
 SH_SCOPE void SH_STAT(SH_TYPE *tb);
+SH_SCOPE void SH_COPY(SH_TYPE *src_tb, SH_TYPE *dst_tb, SH_ELEMENT_TYPE *data);
 
 #endif   /* SH_DECLARE */
 
@@ -276,15 +295,38 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
 #endif
 }
 
+/* default memory allocator function */
+static void *
+SH_DEFAULT_ALLOC(Size size, void *args)
+{
+	MemoryContext context = (MemoryContext) args;
+
+	return MemoryContextAllocExtended(context, size,
+									  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+}
+
+/* default memory free function */
+static void
+SH_DEFAULT_FREE(void *pointer, void *args)
+{
+	pfree(pointer);
+}
+
 /*
  * Create a hash table with enough space for `nelements` distinct members,
  * allocating required memory in the passed-in context.
+ *
+ * WARNING : alloc is an allocator handle which provides memory allocator
+ * function for storing hash elements. User needs to be aware that providing
+ * allocator does not store complete hash table in the allocator memory i.e
+ * hash table will still be allocated in local memory. However all the
+ * elements will be stored in the memory provided by the allocator.
  */
 SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements)
+SH_CREATE(MemoryContext ctx, uint32 nelements, SH_ALLOCATOR *alloc)
 {
-	SH_TYPE    *tb;
-	uint64		size;
+	SH_TYPE     *tb;
+	uint64		 size;
 
 	tb = MemoryContextAllocZero(ctx, sizeof(SH_TYPE));
 	tb->ctx = ctx;
@@ -294,9 +336,18 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 
 	SH_COMPUTE_PARAMETERS(tb, size);
 
-	tb->data = MemoryContextAllocExtended(tb->ctx,
-										  sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	if (!alloc)
+	{
+		tb->alloc = palloc(sizeof(SH_ALLOCATOR));
+		tb->alloc->HashAlloc = SH_DEFAULT_ALLOC;
+		tb->alloc->HashFree = SH_DEFAULT_FREE;
+		tb->alloc->args = tb->ctx;
+	}
+	else
+		tb->alloc = alloc;
+
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	return tb;
 }
@@ -305,7 +356,12 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 SH_SCOPE void
 SH_DESTROY(SH_TYPE *tb)
 {
-	pfree(tb->data);
+	if (tb->alloc)
+	{
+		tb->alloc->HashFree(tb->data, tb->alloc->args);
+		pfree(tb->alloc);
+	}
+
 	pfree(tb);
 }
 
@@ -333,9 +389,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 	/* compute parameters for new table */
 	SH_COMPUTE_PARAMETERS(tb, newsize);
 
-	tb->data = MemoryContextAllocExtended(
-								 tb->ctx, sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	newdata = tb->data;
 
@@ -421,7 +476,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 		}
 	}
 
-	pfree(olddata);
+	tb->alloc->HashFree(olddata, tb->alloc->args);
+
 }
 
 /*
@@ -826,6 +882,18 @@ SH_STAT(SH_TYPE *tb)
 		 total_collisions, max_collisions, avg_collisions);
 }
 
+/*
+ * Copy information from src_tb to dst_tb, it only copies those informations
+ * which are required by SH_ITERATE. Also store input data as hashtable data.
+ */
+SH_SCOPE void
+SH_COPY (SH_TYPE *src_tb, SH_TYPE *dst_tb, SH_ELEMENT_TYPE *data)
+{
+	dst_tb->size = src_tb->size;
+	dst_tb->sizemask = src_tb->sizemask;
+	dst_tb->data = data;
+}
+
 #endif   /* SH_DEFINE */
 
 
0003-parallel-bitmap-heap-scan-v9.patchapplication/octet-stream; name=0003-parallel-bitmap-heap-scan-v9.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1545f03..cf2dbbd 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1232,6 +1232,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1ce42ea..76a99a1 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 86d9fb5..404c7ac 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -203,6 +204,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -255,6 +260,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -731,6 +741,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index d5fd57a..9df6dfd 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,16 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+					  ParallelIterator *parallel_iterator);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -68,11 +74,18 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
 
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (pbminfo)
+		parallel_tbm = (ParallelTIDBitmap *) ((char *) pbminfo +
+											  pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,6 +100,35 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	prefetch_iterator = node->prefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	The first worker to see the state of ParallelBitmapInfo as PBM_INITIAL
+	 *	becomes leader and sets the state to PBM_INPROGRESS. All other workers
+	 *	see the state as PBM_INPROGRESS, and will wait for leader to finish
+	 *	building the TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *		ParallelBitmapInfo so that other worker can see those.
+	 *	  Set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader creates shared TIDBitmap.
+	 *	  Copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over page and chunk array
+	 *			and select heap pages one by one. If prefetch is enable then
+	 *			there will be two iterators.
+	 *	  b) Since multiple workers are iterating over same page and chunk
+	 *			array we need to have a shared iterator, so we grab a spin
+	 *			lock and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,10 +143,56 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.
+		 *
+		 * In parallel mode leader will immediately come out of the function,
+		 * but all other worker will be blocked until leader wake them up.
+		 */
+		if (pbminfo == NULL || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag in lower level bitmap index scan. Later
+			 * bitmap index node will use this flag to indicate tidbitmap that
+			 * it needs to create an shared page table.
+			 */
+			if (pbminfo)
+				pbms_set_parallel(outerPlanState(node));
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			/*
+			 * copy TBM local information in shared memory before waking up
+			 * other workers. Other workers will create there own TBM and copy
+			 * information from shared memory.
+			 */
+			if (pbminfo)
+			{
+				tbm_update_shared_members(tbm, parallel_tbm);
+
+				/* Change the state under a lock */
+				SpinLockAcquire(&pbminfo->state_mutex);
+				pbminfo->state = PBM_FINISHED;
+				SpinLockRelease(&pbminfo->state_mutex);
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+				/* Wake up all other workers. */
+				ConditionVariableBroadcast(&pbminfo->cv);
+			}
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM. Here we
+			 * need to create a local TBM and copy information from shared
+			 * location. We also need to attach to shared page table using
+			 * hash table handle stored in parallel_tbm (shared memory).
+			 */
+			tbm = tbm_attach(parallel_tbm, node->ss.ps.state->es_query_dsa);
+		}
 
 		node->tbm = tbm;
 		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
@@ -118,19 +206,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			node->prefetch_target = -1;
 		}
 #endif   /* USE_PREFETCH */
+
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+									 pbminfo ? &pbminfo->tbmiterator : NULL);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +229,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (pbminfo)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+							   pbminfo ? &pbminfo->prefetch_iterator : NULL);
+
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not ensure
+				 * that current blockno in main iterator and prefetch iterator
+				 * is same. It's possible that whatever blockno we are
+				 * prefetching is getting processed by some other worker.
+				 */
+				if ((pbminfo == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +300,39 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				int		   *prefetch_target;
+
+				if (pbminfo == NULL)
+					prefetch_target = &node->prefetch_target;
+				else
+				{
+					prefetch_target = &pbminfo->prefetch_target;
+					/* If we are in parallel mode then grab prefetch_mutex */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+				}
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +348,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (pbminfo == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+					pbminfo->prefetch_target++;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +387,53 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+			TBMIterator *iterator = node->prefetch_iterator;
+			ParallelIterator *parallel_iteartor = NULL;
+			ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (pbminfo)
+			{
+				parallel_iteartor = &pbminfo->prefetch_iterator;
+				prefetch_pages = &pbminfo->prefetch_pages;
+				prefetch_target = pbminfo->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
+
+			while (*prefetch_pages < prefetch_target)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+														  parallel_iteartor);
 
 				if (tbmpre == NULL)
 				{
 					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					tbm_end_iterate(iterator);
+					prefetch_iterator = node->prefetch_iterator = NULL;
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +648,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +766,7 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -653,3 +853,218 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : then we need to wait till leader creates BITMAP
+		 * PBM_FINISHED	  : BITMAP is ready so no need to wait.
+		 *---------------
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Wrapper over tbm_iterate, which take care of copying and restoring iterator
+ * variables from parallel iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator, ParallelIterator *piterator)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (piterator == NULL)
+		return tbm_iterate(iterator);
+
+	/* We are in parallel mode so grab parallel iterator mutex */
+	SpinLockAcquire(&piterator->mutex);
+
+	/*
+	 * Copy information from shared location to local iterator and call
+	 * iterate. Then restore back to shared iterator.
+	 */
+	iterator->spageptr = piterator->spageptr;
+	iterator->schunkptr = piterator->schunkptr;
+	iterator->schunkbit = piterator->schunkbit;
+	output = tbm_iterate(iterator);
+	piterator->spageptr = iterator->spageptr;
+	piterator->schunkptr = iterator->schunkptr;
+	piterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&piterator->mutex);
+
+	return output;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node as
+	 * parallel, because only first node will create the main bitmap other
+	 * bitmaps will be merged to the first bitmap so no need to create them in
+	 * shared memory.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState *) node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..56f8376 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate that we
+	 * need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_dsa);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..bbc199e 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -42,7 +42,14 @@
 
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
+#include "nodes/execnodes.h"
 #include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
+#include "storage/lwlock.h"
+#include "storage/shmem.h"
+#include "storage/spin.h"
+#include "utils/dsa.h"
+#include "utils/hsearch.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -136,27 +143,15 @@ struct TIDBitmap
 	bool		iterating;		/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
+	bool		is_shared;		/* need to build shared tbm if set */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	int			dsa_entries;	/* actual allocated entries */
+	dsa_area   *area;			/* reference to per-query shared memory area */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
 };
 
-/*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
- */
-struct TBMIterator
-{
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
-};
-
-
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
 static bool tbm_intersect_page(TIDBitmap *a, PagetableEntry *apage,
@@ -168,6 +163,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+void	   *tbm_alloc_shared(Size size, void *arg);
+void		tbm_free_shared(void *pointer, void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -198,6 +195,19 @@ hash_blockno(BlockNumber b)
 #define SH_DECLARE
 #include "lib/simplehash.h"
 
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+struct ParallelTIDBitmap
+{
+	dsa_pointer dsa_data;		/* dsa pointers for all kind of pages */
+	pagetable_hash pagetable;	/* hash table of PagetableEntry's */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	int			dsa_entries;	/* total item in dsa_pages */
+};
 
 /*
  * tbm_create - create an initially-empty bitmap
@@ -244,7 +254,18 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	if (tbm->is_shared)
+	{
+		pagetable_alloc *allocator = palloc(sizeof(pagetable_alloc));
+
+		allocator->args = tbm;
+		allocator->HashAlloc = tbm_alloc_shared;
+		allocator->HashFree = tbm_free_shared;
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, allocator);
+		tbm->dsa_entries = tbm->pagetable->size;
+	}
+	else
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -1061,3 +1082,136 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_update_shared_members
+ *
+ * Store leader's private tbm state to shared location. This must
+ * be called before waking up other workers.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before waking up the other
+	 * workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->dsa_data = tbm->dsa_data;
+	parallel_tbm->dsa_entries = tbm->dsa_entries;
+	pagetable_copy(tbm->pagetable, &parallel_tbm->pagetable, NULL);
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->is_shared = true;
+	tbm->area = (dsa_area *) area;
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ *
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size
+tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
+
+/*
+ * tbm_attach
+ *
+ * Create a local TIDBitmap for worker and Attach it to shared TID bitmap
+ * created by leader.
+ */
+TIDBitmap *
+tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area)
+{
+	TIDBitmap  *tbm = makeNode(TIDBitmap);
+	PagetableEntry *dsa_entry;
+
+	tbm->mcxt = CurrentMemoryContext;
+	tbm->status = TBM_HASH;
+	tbm->nchunks = parallel_tbm->nchunks;
+	tbm->nentries = parallel_tbm->nentries;
+	tbm->npages = parallel_tbm->npages;
+	tbm->maxentries = parallel_tbm->maxentries;
+	tbm->dsa_data = parallel_tbm->dsa_data;
+	tbm->dsa_entries = parallel_tbm->dsa_entries;
+	dsa_entry = dsa_get_address(area, tbm->dsa_data);
+	dsa_entry = (void *) (((char *) dsa_entry) + sizeof(dsa_pointer));
+	tbm->pagetable = palloc0(sizeof(struct pagetable_hash));
+	pagetable_copy(&parallel_tbm->pagetable, tbm->pagetable, dsa_entry);
+
+	/* Mark the tbm as parallel and also keep the DSA reference in it. */
+	tbm_set_parallel(tbm, area);
+
+	return tbm;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Memory allocation function for bitmap hash.
+ * It allocates memory from DSA and also stores dsa_pointer in the memory
+ * so that during free we can directly get the dsa_pointe and free it.
+ */
+void *
+tbm_alloc_shared(Size size, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->area, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	/* Keep track of actual number of entries */
+	if (tbm->pagetable)
+		tbm->dsa_entries = tbm->pagetable->size;
+
+	ptr = dsa_get_address(tbm->area, dsaptr);
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * tbm_free_shared
+ *
+ * Memory free function for tid bitmap
+ */
+void
+tbm_free_shared(void *pointer, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsa_data;
+
+	/*
+	 * If TBM is in iterating phase means pagetable was already created and we
+	 * have come here during tbm_free. So we need not to do anything because
+	 * by this time DSA would have been already freed.
+	 */
+	if (tbm->iterating)
+		return;
+
+	dsa_data = *((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+	dsa_free(tbm->area, dsa_data);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 3cf9417..bb262a6 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2825,6 +2825,31 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
 static int
 compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
 {
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index deb973b..4535016 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -798,6 +798,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -859,8 +860,13 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+		cpu_run_cost = update_cost_for_parallelism(path, cpu_run_cost);
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 7b43c4a..e74a717 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1557,6 +1561,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1599,6 +1608,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index c7bcd9b..25fe7b6 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2803,7 +2803,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3b7c56d..bcf85a2 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3192,7 +3193,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index f37a0bf..0f15d04 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3386,6 +3386,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index ee7e05a..d7ebac4 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -178,6 +178,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index ce13bf7..06b9883 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1416,6 +1418,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t		mutex;			/* mutual exclusion for below three fields */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+}	ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	ParallelIterator tbmiterator;
+	ParallelIterator prefetch_iterator;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	Size		ptbm_offset;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1430,6 +1498,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1446,6 +1516,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1461,7 +1532,9 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1476,6 +1549,8 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..77d0fc6 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+} TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -62,5 +79,12 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+						   ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+						  ParallelTIDBitmap *parallel_tbm);
+void		tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
+TIDBitmap *tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index d16f879..8209749 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 480f25f..0341b57 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 5b37894..5c2cc0c 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -784,6 +784,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#39tushar
tushar.ahuja@enterprisedb.com
In reply to: Dilip Kumar (#38)
Re: Parallel bitmap heap scan

On 01/10/2017 05:16 PM, Dilip Kumar wrote:

Please try attached patch and confirm from your
side.

Thanks,issue seems to be fixed now.

--
regards,tushar
EnterpriseDB https://www.enterprisedb.com/
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#40Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: tushar (#39)
Re: Parallel bitmap heap scan

On Wed, Jan 11, 2017 at 5:33 PM, tushar <tushar.ahuja@enterprisedb.com> wrote:

On 01/10/2017 05:16 PM, Dilip Kumar wrote:

Please try attached patch and confirm from your
side.

Thanks,issue seems to be fixed now.

--
regards,tushar
EnterpriseDB https://www.enterprisedb.com/
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

Hello Dilip,
I was trying to test the performance of all the parallel query related
patches on TPC-H queries, I found that when parallel hash [1 ]and your
patch are applied then Q10 and Q14 were hanged, however, without any
one of these patches, these queries are running fine. Following is the
stack trace,

At the master:

#0 0x00003fff8bef7de8 in __epoll_wait_nocancel () from /lib64/power8/libc.so.6
#1 0x00000000104ea184 in WaitEventSetWaitBlock (set=0x10039b2d360,
cur_timeout=-1, occurred_events=0x3fffd4a55a98, nevents=1) at
latch.c:998
#2 0x00000000104e9fc8 in WaitEventSetWait (set=0x10039b2d360,
timeout=-1, occurred_events=0x3fffd4a55a98, nevents=1,
wait_event_info=134217730) at latch.c:950
#3 0x00000000104e9484 in WaitLatchOrSocket (latch=0x3fff85128ab4,
wakeEvents=1, sock=-1, timeout=-1, wait_event_info=134217730) at
latch.c:350
#4 0x00000000104e931c in WaitLatch (latch=0x3fff85128ab4,
wakeEvents=1, timeout=0, wait_event_info=134217730) at latch.c:304
#5 0x000000001032a378 in gather_readnext (gatherstate=0x10039aeeb98)
at nodeGather.c:393
#6 0x000000001032a044 in gather_getnext (gatherstate=0x10039aeeb98)
at nodeGather.c:293
#7 0x0000000010329e94 in ExecGather (node=0x10039aeeb98) at nodeGather.c:234
#8 0x00000000103086f0 in ExecProcNode (node=0x10039aeeb98) at
execProcnode.c:521
#9 0x000000001031f550 in fetch_input_tuple (aggstate=0x10039aed220)
at nodeAgg.c:587
#10 0x00000000103229a4 in agg_retrieve_direct (aggstate=0x10039aed220)
at nodeAgg.c:2211

At one of the worker process,

#0 0x00000000103879f8 in pagetable_insert (tb=0x10039abcac0,
key=6491, found=0x3fffd4a54b88 "") at
../../../src/include/lib/simplehash.h:571
#1 0x0000000010389964 in tbm_get_pageentry (tbm=0x10039ab5778,
pageno=6491) at tidbitmap.c:876
#2 0x0000000010388608 in tbm_add_tuples (tbm=0x10039ab5778,
tids=0x10039a94c30, ntids=1, recheck=0 '\000') at tidbitmap.c:340
#3 0x00000000100f5e54 in btgetbitmap (scan=0x10039ab3c80,
tbm=0x10039ab5778) at nbtree.c:439
#4 0x00000000100eab7c in index_getbitmap (scan=0x10039ab3c80,
bitmap=0x10039ab5778) at indexam.c:687
#5 0x0000000010328acc in MultiExecBitmapIndexScan
(node=0x10039a98630) at nodeBitmapIndexscan.c:98
#6 0x00000000103088d8 in MultiExecProcNode (node=0x10039a98630) at
execProcnode.c:591
#7 0x0000000010326c70 in BitmapHeapNext (node=0x10039a98770) at
nodeBitmapHeapscan.c:164
#8 0x0000000010316440 in ExecScanFetch (node=0x10039a98770,
accessMtd=0x10326b70 <BitmapHeapNext>, recheckMtd=0x10327bb4
<BitmapHeapRecheck>) at execScan.c:95
#9 0x0000000010316590 in ExecScan (node=0x10039a98770,
accessMtd=0x10326b70 <BitmapHeapNext>, recheckMtd=0x10327bb4
<BitmapHeapRecheck>) at execScan.c:180
#10 0x0000000010327c84 in ExecBitmapHeapScan (node=0x10039a98770) at
nodeBitmapHeapscan.c:623

At other workers,

#0 0x00003fff8bef7de8 in __epoll_wait_nocancel () from /lib64/power8/libc.so.6
#1 0x00000000104ea184 in WaitEventSetWaitBlock (set=0x10039a24670,
cur_timeout=-1, occurred_events=0x3fffd4a55ed8, nevents=1) at
latch.c:998
#2 0x00000000104e9fc8 in WaitEventSetWait (set=0x10039a24670,
timeout=-1, occurred_events=0x3fffd4a55ed8, nevents=1,
wait_event_info=134217737) at latch.c:950
#3 0x000000001051a3dc in ConditionVariableSleep (cv=0x3fff7c2d01ac,
wait_event_info=134217737) at condition_variable.c:132
#4 0x00000000103283e0 in pbms_is_leader (pbminfo=0x3fff7c2d0178) at
nodeBitmapHeapscan.c:900
#5 0x0000000010326c38 in BitmapHeapNext (node=0x10039a95f50) at
nodeBitmapHeapscan.c:153
#6 0x0000000010316440 in ExecScanFetch (node=0x10039a95f50,
accessMtd=0x10326b70 <BitmapHeapNext>, recheckMtd=0x10327bb4
<BitmapHeapRecheck>) at execScan.c:95
#7 0x0000000010316590 in ExecScan (node=0x10039a95f50,
accessMtd=0x10326b70 <BitmapHeapNext>, recheckMtd=0x10327bb4
<BitmapHeapRecheck>) at execScan.c:180
#8 0x0000000010327c84 in ExecBitmapHeapScan (node=0x10039a95f50) at
nodeBitmapHeapscan.c:623
#9 0x0000000010308588 in ExecProcNode (node=0x10039a95f50) at
execProcnode.c:443
#10 0x00000000103310d8 in ExecHashJoinOuterGetTuple
(outerNode=0x10039a95f50, hjstate=0x10039a96808,
hashvalue=0x3fffd4a5639c) at nodeHashjoin.c:936

I was using TPC-H with scale factor 20, please let me know if there is
anything more you require in this regard.

[1]: /messages/by-id/CAEepm=1vGcv6LBrxZeqPb_rPxfraidWAF_8_4z2ZMQ+7DOjj9w@mail.gmail.com -- Regards, Rafia Sabih EnterpriseDB: http://www.enterprisedb.com/
--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#41Dilip Kumar
dilipbalaut@gmail.com
In reply to: Rafia Sabih (#40)
Re: Parallel bitmap heap scan

On Thu, Jan 12, 2017 at 5:22 PM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Hello Dilip,
I was trying to test the performance of all the parallel query related
patches on TPC-H queries, I found that when parallel hash [1 ]and your
patch are applied then Q10 and Q14 were hanged, however, without any
one of these patches, these queries are running fine. Following is the
stack trace,

Thanks for reporting, I will look into this.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#42tushar
tushar.ahuja@enterprisedb.com
In reply to: tushar (#39)
3 attachment(s)
Re: Parallel bitmap heap scan

On 01/11/2017 05:33 PM, tushar wrote:

On 01/10/2017 05:16 PM, Dilip Kumar wrote:

Please try attached patch and confirm from your
side.

Thanks,issue seems to be fixed now.

Prabhat and I have done some testing for this feature and written few
testcases. PFA the sql script(along with the expected .out files)

In addition we have generated the LCOV (code coverage) report and
compared the files which are changed for the "Parallel Bitmap Heap scan"
patch.
You can see the numbers for "with_patch" V/s "with_patch+TestCases"
(.pdf file is attached)

--
regards,tushar
EnterpriseDB https://www.enterprisedb.com/
The Enterprise PostgreSQL Company

Attachments:

pbhs.sqlapplication/octetstream; name=pbhs.sqlDownload
pbhs.outtext/plain; charset=UTF-8; name=pbhs.outDownload
pbhs.pdfapplication/pdf; name=pbhs.pdfDownload
%PDF-1.4
% ����
4
0
obj
<<
/Type
/Catalog
/Names
<<
/JavaScript
3
0
R
>>
/PageLabels
<<
/Nums
[
0
<<
/S
/D
/St
1
>>
]
>>
/Outlines
2
0
R
/Pages
1
0
R
>>
endobj
5
0
obj
<<
/Creator
(��Google Sheets)
/Title
()
>>
endobj
6
0
obj
<<
/Type
/Page
/Parent
1
0
R
/MediaBox
[
0
0
792
612
]
/Contents
7
0
R
/Resources
8
0
R
/Annots
10
0
R
/Group
<<
/S
/Transparency
/CS
/DeviceRGB
>>
>>
endobj
7
0
obj
<<
/Filter
/FlateDecode
/Length
9
0
R
>>
stream
x��[[�5.i^P=�H2&3+�����S���L�$��%3��ZM�@��s||�:�vuw�����j�;_�T����Ui�=tU7���e���Z�G��}u�.+�g�����gY������P
"���KyS����������z���`G��z��	�R������%/e�II�l�
�=G��E`'V^�4���D��%8��a�eLC��^��VCy��uD���%��/�I��U���@��'�KT#M�u����x��H&��o*��������E��YN?u�	�haDk�
�J��p5�5�#���v�" �1qpd522������*F@�HR+�+K��Q��(���TeB���q�v��]��b|$�y$D�+p#@�0���\����������'8�f���x���@r#d]��j��\�X~X|P��P�w�!��/��cx��=x������z���uP��xV�/��%�O��+]�P:��%��z����P���������o��}^<�������j�
��6������pCj\��^�����p��a���[n!���K���O1���.�����������mm��m�f%�>	��?��������F>dT^��������2�[6r���S�G�q�V3�d?�����w@��L��m�������x������Rv�+����6��m�)���	J�~o�za��w���%\/�v�p���t����Y�!"�\A�C�����lG<�	�z���X^�[1����	�>�;W�#~�����������n����s����Nq��������9,mw��<5���$������sI����p�����EK��a&�6�U������^)�
�a���5��^��N���1�q^�����a���H�����YhY��c�A��6��=iF(=�V��38 �����g�A����G�>�
�$�1�6���9!IR�3}N0f�#�q�p=��':�<7�;�*��)e����H��oi���d>�S-1H<�m�z4�f�O���;���(�i�'`�i�E�w�rQw{b�������6��:���v�h����F����@Z��qMz�KD���H�O_"���������4������zhr��������(�K_�=�g�e��zW$���q�+�\�d�4?l>a��'����t3�N�s���5}:���N��!����Xa@��I��`������WRyVx��^�+3X!���x��X+�������o��O}�L�N���S_`���s��i�%>>��_u��
Op�0��b�?w/�������p�C;����^���l������T�V���5��gY%6	�b�zO��
r�W���%x���q��'��f����#�+N���+C���<s�`/��S����V��S>��������?�����'�\��=>���;�����������;ft>�
�����PbUVH-}�7b��u%)�+a�&i$��7
�"�r�u X��"h���
�A�i��Q�(�+��TA��,hxPj)z�xX��&h1��JM T���a-�;I��o�;6��������+3C���IK[�E���AG`X���8;��wJ�{K�}�H�A���j�>E�A����pD.9�Rl�z�����j������(�b�D* �F�G:���(�]��oG�4��P)6�=2QM�D)>c�}�;����#oL?�x�I�b��Y��j�6)mX����qE���Vk�W�8A���@�� �*�<�O�G��XE�)�Y*��>�6e�(����N��g[c��IN��U���U������x���U����E����)s�}+����!��c CV� TS�=�C��tP�1��S_hY�4,z `�n�.�wG!9�i�������wRa�2�%�����1���v�d�
B5�������u45� |������=`���Cb�S�9�LC�!��6�O���(�9�#�t�"���,��v�!����������u�D��*Vz�����%����e �r���V��jTL�����)���_��^���-�*__�?����nK��
6|S��#)��A��Kbw
��w�E��,k��&C�]������U�jH�q(C
�"����y�������dd�*�hr����]y0����$����y��jHfq(7-��Y�����1>+,v>I4)8�f�	�?����Fr�Ff��i���%cK<]���b1&s@w������,�����H�^�2X4He�t���b��8����1�P��C�����PK���P����+�<�
��+D�w��X��#|2X�9�we<
endstream
endobj
9
0
obj
2368
endobj
10
0
obj
[
]
endobj
13
0
obj
<<
/CA
0.14901961
/ca
0.14901961
>>
endobj
8
0
obj
<<
/Font
<<
/Font0
11
0
R
/Font1
12
0
R
>>
/Pattern
<<
>>
/XObject
<<
>>
/ExtGState
<<
/Alpha2
13
0
R
>>
/ProcSet
[
/PDF
/Text
/ImageB
/ImageC
/ImageI
]
>>
endobj
11
0
obj
<<
/Type
/Font
/Subtype
/Type0
/BaseFont
/MUFUZY+ArialMT
/Encoding
/Identity-H
/DescendantFonts
[
14
0
R
]
/ToUnicode
15
0
R
>>
endobj
12
0
obj
<<
/Type
/Font
/Subtype
/Type0
/BaseFont
/MUFUZY+Arial-BoldMT
/Encoding
/Identity-H
/DescendantFonts
[
18
0
R
]
/ToUnicode
19
0
R
>>
endobj
15
0
obj
<<
/Filter
/FlateDecode
/Length
22
0
R
>>
stream
x�}R�n�0��>��c ����T�8���~������_�R�H�fvwf{�C�T�vd���U#kZ���)`5[D��V�g�o�I��\M�]i�>�s~��0��m����!����#�|*����?����ih������"m[j�o�i�9��&G���5V*p�!��_���*0�&��U7}��/�-��^�s�D��RB@�.H9����G�"M��XF����c�w�"!�nI�
�s{������sj�L!7�M��7����0��:FnB�$d��f��c0� ,<�LS
��e��J�3�nt��y/�N����y�����5pi�����/�z��
endstream
endobj
17
0
obj
<<
/Filter
/FlateDecode
/Length
23
0
R
>>
stream
x���	`T��8>3�~�o��}f���n�
H�<�R��� � 7��**HPDT�/<�xPHk�����z������k)U ������m�����ef>o��7�����!5!��oZ:���!�q�����M���-�>	�����2���3�!�5"4)8c��)G���A����L������u3\���h���sc���j�\?w�$��3]z�7\��3i�<w������||����R
���������^@!>�����p�������>M�W�tk> �	m�3�f�z��������P���s�
�h<���F�O����P�U��a�F����v��q0�%Z�������B*B���h.�_��M@��w�^�*t���r�r���=�~�vs���#
���;��V�S�/�<�z}�Tv ji��O�h������hA�m��t���}*������\sn���4m@{p
D�������:�[E��N���_���.��=�;�B�]�iA��m\�}y�/���T�j��\�k�:���u2W�=C�5����h��x�s�/r;��q��������:��M�W��xK��\�$��Pcw�MA3a����3x'��������Y� {$��I����u�AO�x!���N����1�7������I��k�t/z��qo<_�g�%x~?��C�8���&��w�n>�+�2����w
+�{���q�}��g����[�F>,��?�����F�����
������o����^���_�-P�!�7�%���%~"��)�_�, 7�����A�"���W�e������B�Vq�����+��9��z�)a�����pR�Kw�H~�������fQvuv}v[�%�W��9�(�:h�$����^����0va\�/�W��L���|�Fr�����K�F���;h�F���]I
�����d*�O��I����$��99W�
����"�n=����}���;���_�W�B��O�~?�������B� �+Uq��Rl�G�)]"
�FH
���N����7h��:��#�rn��G������D4�BS�&��,�-�XX,^D.�C�I>
c��<EN���!x0�f����D/�$u�o�	~/��w�����N��hF��|���g�w�a�3,�O�?�*��n8`���K�q(�=�~���K�2!����x(~	��h����G������D����	����a<����CUx	�=TQ&� ��>�6���!���"��cN����� ~G>F7����>�^��$����'��xP�R���-G���?����cQ�?�m	��O@����i;�����9A���/����G�O��A3���.�;�"�&�h����u����D�s��Gs��
�Q��rK����1t?��������P���*a 9(�u!k��dY���h�p}�_��%��h
�G4
����}�]
�Qt��^~5\�����P�57��������+�*���
C{�/$M�20�������T22�����	�p?���u#����~cF_j����������US]��{���]*2�e�%�Tq�(/�D#�P0��y=n��thv����(���b@r`c�9������_��^''A��N��q�xa��x#+���%����a�4:Jb=^���T�$���'��x��q��?Yo>��!^�`
�D���o����o��f@cx�V��/�o���mUm�j$�m��K0H`@���4�9��?�9��O[���L��<|���#�D}��f�or��f�����aEP?VM���Yb��g���{�[+���m��u�����I�5s��i����9p����Kx����U��F�5�3��r��U���#�u���q}=��%���kB�ka��Cm���q��.�2N{B{e�ojr�i�oV��%g���S^��F����sGPx@|��q�Ds�H�~R��V/Z3���!#��N�������g�k������Sh������E�+!������qI�SoM���L�
����S�S`Ff6+���}h>}�YH����"����o.����S�?)�t�����L���������x	���RqS+I&��qH`��p�I�}*a�	:����:�hn1�����"��Q��o&��N�u�7��i��t<��LnAT]�5���N��0�O3����j�<*9x��q�k�c;x�W�����P���8.B��p�. �����b���O�?�!��VI�d98>�Yo�����D��Pk�$}�%��7��O����.���y�54D������Q/��fVxE>�G��%�������k�����>�l���������#y��(vv��n������5�k&����K������
���y-�i���'�<pm=��������&��[
�z��q�u�V���`�������po��8B�%4�f��8�@�1tr�Y��n�&v�g�zr+F,O��0��J�<��#���y�����=�3�0����H0S���P��p� � t����s����8�F��{G|Jx�����$C]Z&3�p$�J6�`����3��q�%O!��<e��p(��H0�*������<�-e��%PZ��l�^�J�H�}p��W*q����W�f��
'N��OC���h?���u}!�c�����U]3K�}����b��Ue�L!��w/�]�U�g!
��B�R�����	�*J��nI��^��D��]��D(����7x�{V�i,&���H�y�U=z�T�9\���w������������_����C��<�W|�����*���f����/o�������E���p����Y(*?����������E��]Do�`-M�����Q�nP��2w� ��
*��(���*MH�N�t�~�a���u���:Ty����T�n)W�&��r%|	����������'���h�){�s)n%����
#4����<��D$,��!~�����
���r�����|����RR�[w�ut?�W�=���$uH%u��[��o��~�M}�����j�|{8@�K��Frh�n��>���%��O����a�)nG���Bi@u��qD��V�"T�o��\�N���V	]3
{pC&�����������xAcr_�.�
���8
�i��/���l��[s�
���Z�i��6
����������DE;�4�to%�L
5�oL�
<��t�10B���i����i�$<�:B�������I�o��E�����D�J�� �n�1��x��w����VL��kP3����#�P^\EV�V;�v�d���|W��EF{&�&�FFfK�m�=��f�#�����l�:W��H������G�G�?;�����#	��-t�����A
,T�D������+�u��`C�%�u�.6/.���i�k!vo���8�|��@2-�X	�.��=���
�S�a>��������a���W��nJ�~�O'@�%i�N)����E�8f��o����Y�?��-�~q��_�}��
�}���_��=�;��f���]�������3��og��X���Y�:6��f�Y��"�f/X���z6���O��Wx� Q �"`;���l(T:�S���;����H���C�(C/C/�s#D����-a�P-�w9���!�\��B&��7����E���:��o��j�x���&�.Q�jz��UE��\�����V�����%����Dhe���#��H��# b���(�P��0���������kA�G�Q�q�����:@1���b1z7u�����6�J^5�D
�����x!H������(��K�}=(���
�n7a��E�z�6����yi}�6x��Bq������R$����*3z^$\$�*�&�*�%������G;f��8nu������},|,r2l���C"zT/�c����I�#H�T��T]�w�ao4��a��r8�i1��<�}��Zqp��M���]�U�U������0�> /%V�*Y��H��
�kG_2��%�@��C�A	�+#�*�3�3z1����Mq�U���f*�A�X4�5��yJL��^�A=�|M�RI���%��K�z�@��
�mz��;��=?��������3b�7_Z7���}������k<?�j����>�zRwJ�cs��~��n���o�c �wb��P����"Li#c�\�TAZ8c�Z��E�XYT(���=���)���4{'OWR�{������o_L����~w��/��:�����h+5~��j�Mn��z}�w��F��Jm����/4U�s�lv��K��tN��*��U
�@g||py��C�V
�L�mM���bw/v/��'� %�x�d���RGQia���4Fi=M���v�����[q�m�����y�a����*Z��y��0�����2
l��(��:CW��vs���){�����.`��:@?(�H4F���m)|h��-�,�����-l]9k�ZoK��_.~g��)w������3����;�<�}�,^:��+�;���m�����~u_[���Sm!K��������;c�B�G�0.�X1�9���E+&kg�$3��b�	.Y6���,X,�Xa�)���3��/���WqWi?�97�7���$�6Ny�i�p���xNC����*y�`!l4T��P����d�.AP���j���U0�[�^�����IFQ�ZjJ�H����M�V#0W��#�a�Gw�g�G+^���*���8E��N�\gbC?Uw��U[��z>���t�t���i��k��~`��j��.�_PPG_Q�e����������t��(
i�Z�;�;��N(�P
�rU��.����������������<w�/�O+{�}6�T?L��>gx<l:�,�1�����3#@!��,���B���,D!S�,��|�Bl�/�B��������C��|Q7a6'����#)��)�`�
�q(UI��.C�L��	A'��o)XS�����7������'�(sJ7��m8��Q}n������xN
����
1��p>�o�.'���������sM������u^��a*A�T�� 	ZL%�.���k�?��{�c�b.�2��KCb���C�p�����P1���vB�O��3�&�+�F�|^�Z�c~�{��;Z6��zm������w
[�@��{����I_s��g6l��O����M������P�2��r��n7��'���Z�@�-�J,4Jv�Q��OX�`��J3�A5�4I��eZ���l����aM-&"����(����y��:���hf������oZ��pB��@q����/�����������i�<�=%�H�1z��2�����%�����<C��qF��!�%�d<Ao�h+�k���S����4���
�#e
Q���Tf���t�t`p��������������U�c�1k��FQ�������`�_��b�O�
�iK��x{���0�Xg�;a
5&�`��x���:s)�9��.a��(QI���N���^T�ao'4��nV\1{��c�#�����~���}����?i�5����{��[_�G9fu���o�2�1��?�9q;���_����O^�o}��-[`D'�l�/ 
�3�4��?"�
�h��+vm!�:�������S^�|��2M$\_H��e`�yb����uCN��������.X��5? 8�?'J��nw�I�����{:wsw��n����e��������o=N�Z��jB@5�D����Q�2���
K�<��s��u%��Xz^Y�N�/�,�"FI���0�.��{/�*��4���)�1N����2�����F]�n����|�Eb]�`���������Bi����F��v2�A�Iy�M�ehP��t9z�Jr�*�u�4W����>�F�i�y����j������A�b
���fC�2_�����i��"�I��O���������U��+�
�e+�j���������/�	J7�-�K�E�|������{���_)/������
����q����i�W��K<MyJ�,nG� �C36v��F�b#1���(aL4��5+{�t���X�2�&��i���81r�N�l��A�`����	.l�0�J��bv���,f7��,C<�3|����[�lc�94Nu����]���jTy^��Z���h~�T����t���lZ�3���������y��Z�W����7{������n��w�=m��>W��|��;���E�����g�=\V�o���������+��8y��s�!��=�t�K�,�������H%I��4c&Y�4)�
\����:���u�|�6KX�4�$S����-g_���2����;9v�
A9I�@59��;hJZ�4J)�]�]���
�*2����Q���/�13$5�%��T?*�68��}��LJ�`��O�Q����7;�q^�3����bD�_j����TU��"��O�*����urw�Nxz�������=�
��{�_�?���{.�?������MI	��@[�D\����9&R�0J���7�vmD�Ec��Z^��=����mt��m�u�F���B5q]���R��R�)�]t�2\��.s����P��:�DKR�������T���RS]B�4����b��4fDb����_2i���=q�W��<���������y�����_,Z���%K����[o�j�����cgc�O�a}C�����_d���5�����_��d7�&+�4[��m�y��B�:���"��:P�����e�k>A��	�\Ua��SS�� �>p�W����w/�O�}x��{�Q"�����p�#R����O�����VpH6W��KPS��u��Z����|�s�����]���1J��v��z��%� ����r@�k��X?X~�)�
���E�������t-�4���c���=���2�W7�3�+�gcV�����Xq��3Xb0v8uf�~��~`�O�{��`dBN`q��M�.�P���:�ma�����m�P�����[���?�8��k�������v�!��dX�F����<�q���[��h���qq������ �1�[�<CA��K�`B�`�\�����T�9�?��s�x��bl��������:;��k�)��2�I"���������Y
��'P�o]�D��0������5�U]�,5�������s���J0S@����Q��c#��k���,���3�Jg�x����_W�$��<	.]"J��=�����=�1��GE��=g����d<^���{����z�k��rQ���0SetM��&�&�%*+qf�Fa���v�p�M,�+\��<�/P,m��zg&�����+//+C���ta,�B2��,{"h�6*+��
��h�J�������rmQ��(��U"��K�J������9mx��Ri{�����w�)��7����N��JK��li �_8��kgZ���h��le�!s��`���������z���3��;z!��A���E�q����+U�D''��$q����M'�Q��Ey*�����wN�~��W7��6�3|���Wx���?�9�����3���������S�}��do�������.��!W�-;�Q���=p�-��;-��p��>��w;&�Y�s��z��H�8��MF����&��`� ���B���hoi������WAW.������A�X������������7p�:6D7���&��������<]��VZ�$���Z��0����xEO;�D�8��pa�M]�M]�M] �1�7j��i+�Pl�f[�t�)�� ok��@�-�����dQ1������T4��M�������;���c[p�7��+.z�������	s>_���������
8��l=�n��>����W�_��KG��/�3��Z�(�dY��S�P��
��q����FsW���F���+�d���X/��U���������KW�X��0��������q3<��{�����[!�����JV�-1��=T�3�����K�������$n#$l��U\	A�+���z�������5M����g���c��}8�K�����l��
4��C�p����|���n��%��B1���G����98����%�b�[����#�)�s�DI#�e=e�2�>���I�*�n�����!+`CV��� W���]e:�JO��t�7�������V:�U�6�j����-F>����('��z��.L�)>�L���;=	(���%�*R	����(�$����m���T:�R�r�0Q�r]@d��������n��������?{h���O�l�������$��9��7nN�^�������}�Y�p7��O��-�[�m^��+��~j_���I�����s�9Q��v�*o3�_5��u�zH%�@�M���2��2�N���I\�D��g��S� ����\�df2��I���	��	��	�����m�
M�_ ��G�Y�����p\�qm�����������������O>��m�����?�4!	�so�3o��.
{��'��$���@K_��[#��"#DX8�������#.h�����������
����]?���b������L�t3��23M��� f�����^���qa��ER��~�5#�h8��D�;��'��/tX������o������F6����w��&����aC��u��]�a�������T�����T9����0���	H�SU��`$$b#���c�����������|��;��L.Ny�v�� ���c3������C�4D'��T�l/��^�_��mv0�(4���"�VBw�%�4���3�Jf�h�_��B�)��a��A�uYc]�X���[���r,����U��&�15-K�1�D��@�:I$\S����'I��C�����������O6\�}E�na��=u��W����px��	w�4�W&�������������e�3�z�����%��0�5KI�[����6P]|~Y���x27�_�-��TI
W��]!]U0�����Q\�4�����=�$�.:�������X@�
�Y�R���:�)T���I1W����N�O
�����6K�������v�v�s�~c���Jn��nm��^���;Sj���}����.��;�+�2�F�,��{tO����Z�["wGH$����JR8%��U�����b~���H�S������W�0�K�����Yy��8U\y��D��
�b�����.L��i���x8n���:,�V�l8��*i���+�5�XT�t�?�4*�e���p��>e�w��{$LG���������%Z�m�����u��T������PB��������:p��4:EG4\jbR���{7D7%Y��]R�xz��_��K�������>o���="�OO��M����/�>����#fN����?��Ja�s���O����k�u��'����Q�G��{��la����I�^�N������-w�s��k�UU�.�h�M7\��KJY��;�a��O������b�I����I�?����G����Jn����.1�I%�1c��e�q�����gJ�[��ow:gq�,sm �Fy���91���
�SM�dD����;M=�~M6"h�7��`����|����^d�Q�hJyH��i���ZO_�H��+����Y����nk��e���_�|���V��NR���K�;C���,����R��S�=]�����15T�F��@a0SQ^]��V\�_^1V��L�gfn����m�Q�1��U���^Y\���'��-#e�JG_����9���c��;��p59�N�:������`�`�H��4%
�Z;Q.�J^2�`�<��F%��t4�D��le��I�r�"��E���'HdS�JS2���oL�����ZLq��Sl��)���v[^�J�1%��OwKoI��5����N1	��^�|��du���Z����m��l��s�R��J�*w�d����5��H
��"����&F	���E)1��P�X%2Y):�[�Yb���5C�����`�>OX&S�����c����=��9j���xv��h��F��Cl3���|�q�^�WS]b���06�����@2�����_(��M�=k��A/��}x:��z�-��������(�
\�o��sf�x&]p���/�5t�P�C���\\??8�����+�.>y���{�OJ�z�����v��T�Y	�T�DG�m�6,���B�0@�6����hU�����u�bO��.|���p����s6��
����f8o��n+��~8p8�7�7�oB/8R�+��Jg�����iW9��������v���E�"QQ��/��-� hI��2R����n3l��&_������-/�������5C:�6ks����sMs�Qt�m����
�]\U����g��p��@CUC:��@�"�
������I���x�0er�n0������f�3vS���0�"�mV�O����Df�:�
���Lgh���OYH7f�t�V������(���Z��'�����MNG�����E�./�,�z���F��_��M��<p�+����W�������ga���GO�����������]��s���u���Q��"��f#����b�S�w�T;�9����2q�wGmR0����'�t�$6`[w�{�!f�u�����������4H���h?O��(��@��1�y���=�?��ZH�Efr�����&�y�e���n��W��N8G�D�\�2'���w��1������#`)�����m�B�-�tR�d��,v��/E�D<eq�o����m�B��4��#���
����D��0�`�p9C�0C�+�>�n>�z>��|�%\(������I*}Lb�I2O��R�H��!k"H'�wA���	���.8E=�,����Ro8
��B�d��������mg�Pz(Nqu[������|y���n	-����V����u`�L�oy:2������w�����O��KT�k)�Z��	5����d�:�;*:�L�*�������=���y�y�|:�x���0�ip�r'�+)�����m0�
�^�Z�M���_���S������sl���.`:?�bL�������,�\N���HP�� ���\Xw�FW���\�;���%��8�|�%R<w1n�bN:�.�G5��B��D���"��Y��7���],1S�<!��tP�L�I<��a'��0�'�L�b8�	)�p&����P}�i���e�1E8N�Q����p���oNSCH63���������[����>��q}����+7���M�-~z��k�>������8�$���y}����Q�4$R8���I#P��>2�k�1���la�2�&��R����H
D�v0�����0���'�=z�{H��������$�����bq��49��;�@`���?�����u�F��:���C^�Tbq�6�M����B��]w�q���a��4a����Q����F_���W7kXR�M*]M�]T
*��~*&��U&3�����b�(.�����z��;�{����+�l��o�����!�<�G!p���N�����_��k�_�_����P�f��tT���R�m��	�M���S���/��a�_>�|������k����c�^�"x�����������=3�C+��x�b�e��W�	1T��4m6�[aKy��
��JA�����V$km=�W�z�J�l3lg��]�%�$/)��d]��
�g�gY������e���fJ���+�*�O|�������V���4���h��`jQ�����!0�Z�RC�Q�:�(jW���TU�#�[k�?%l�u*<�z�4�|�a���`�!��C��;w��Y�W&w�����<wP-��.��=��3�-r�**d�R�0���Fa�k�����9'_�����<��x����3Lq�Y��}Ei���^'��P�bQ������h3?oX��9c�N���G�;!��������n[b�v�\#PS�bvs��>�i[l=�-Z�:��75���
��w���O���_���K�l�|��M��#R=����|�����>�tn���������~��7TsY�GW�������T���H�5�n���,_ T�]v��0rF�kS���[��J��8�S�Q��:��6���tr�R{��*��r�-wL�T���B]Fl�/�h�#a�W5,��N�`��r��������~2�������y?�����)�����L/��:IOC�{� �����3F�6+����1�����^�3"	S���
vV$�g,/�����Z�5u�Z�_;q�)�����Ft	d9���^U�����\I�z��Z�r{�M��r������^���
�=�>�<���Q�-m��j��:��G~��tw����}!�,����y�x��3����"�%����j�nm!i������38�,�X�M������o��>����0e��QiV��������Be���<��:���P}R�	����.���1�WE)% �)~#����Gx��?����Cp����Ah�������*������������S���������L�AkJ�:{�.�c�`�W����_<x�����T
���@W������_�����5_���x���Xgf��������C8-r��S�3UP,�������nV�[�{�P�����j��xH��H�c��</�b/e/��.�8�f�F�0�wQz^�I1-��Z���W�����8�^Y��"<����$���%�(���*pODQR.YNI�W�D��S��U���eS)�=��T�;
E��s�H�W��:`^
)�����c6��I
�IX&a�$&-�}�0 )�v�;c
:�
�K?C�������>(d���4��`���M��t��?M]�`d v$��.���t�=�i���t�N��X�wk�\�����\��*���gzU�(�U���:�n�(����mq�lM�O��j

*�d�) 1��-QK�f��&�n�kE3aWv�l����~hU�Ox,{�P��[�"x��� }����8n�7}=���\��g��Xt�/}���_�4��2a����9{S�Rxk���;�z���!����]������8�������]��
��6l��Wos����L�u7�"s]�H��t
��S�g?��W(������\Q	g��&&r|U5�O!�V4�,w~8/w
:���A2�:������@��4����P�C�r��Fv��?:w��eGS��M&��v#O������L�e��E- ba(��F- ba�[k��8,�iK�-�m.�Xj�nnpY�fm%�-����!6�:���*
�
��$ ��J0W8.��>��IXL�C�z(���6�H*;R�\��3s6�LY�f���m�"�0����Z�vY��L[�`��r���Y��uaD:*��
"����D��a��eVL]��iU�	�5�"R�d�O2f�d�.�����R�(���+�7���������������e������Z����Aj����)���y
����������|���uu K��S��@�m'�����]��|��d�`�M����l�*���3���l:�XO�x~�M����/mON�d��[�M�jy>��������eg{	y���}z��a�m���h�8�Q���[
����Io���}�9����<��E�r����P�H0�������A���_S5��a!���8G~5�� ���L��1]��ti[�.mcl�V�J����P�����{1���y/�i��.S�m���)���zu�d��n6��|�#U>?�?�!?�?�w�[\������N�?Q�]��i>����O��������T�O1��s�$[4�9q^���.E�UI�D=��T�y��;��S!�#���	+V=s�'�O������/|�O?�e��!=��/$+o�s������y��q�f^C!��N;��1O%��;���
�nI
����c�zy�8S���>�>���}�{�@p�0A�7��#�s�9�}�{�J�f�SDA��-�V��_�M�����@��\����x-���OfG��a�#u|�Bb���w�Z&`@���y�H2;h3���nF�.�%N:��g��h�9�m�����n��1l.Fv] e�ssn�(������-;
���2&��K'�A��uS�M�S$�1A��i5��B���NB}tT�QF	�������+���D����M�������������,{b��U+�m�k�6��%����k�����1����{���w����3�`���zc�]��_������8)����=|=
.+�_���D�\����OL���g�g�s�#m����?	�;�=;���I>�g|5|} �>^?f�� ��\��KG�?��!G�B���!��tC��T�����6�|��E���L���Q�P���3���-��ZFR)n��������T����C�t�����C����iYH�`Y��O����30|�,T8�W_�.��,�9u��W�����������~��-)qq�f|�s}�����?�m��]]�����-���)�j��ks�<�={�U}��r��������G�Q�,a?��}j��`��I���������xQq���h��!N�66aHUJ��X.�{��]�A�=�����I�`�:�\��r�tb������A������pj�K�������^�X����`I���m�_���%3�^s�%�]v����~z��}^(��qA�������V�nHHs����X\��$�	�;<>e��5��Y@���nc�Fy��(W*���M-Z����(~��r�����@���C����	�e�:�6�>A�%�Rf��l�����tK��nq).�Y<^��MIO)]�\T�T�3�q���W<��9�E��%��nO����Z�f�$-�����.�V�D��"U�?5����rI����x������3�(T��VB}C�BC[BC�3T��,������|���koxiq�����X�lG�v���y�u����
�/ Q���{��s�����h�G��
�8\2<�����4BA3�Tb��
����8}*�l�����0�{�5H�}��y0���E;����q9��>_n}���<3"R�+6��t�}Ky�� QR^���������A&�Q�Ta����'���.�-��mq���b'�BN�vg�����fG���kz���&g�g�� �=���0H����3'��Od2��jB'��]k��O�g�'��F�d��c�u��g��.����H�t���9�H�G�R*E����.�d%5�\�Fpi���>�
������Y�T������;����k�@Bvd\�a��tIWRSM?���^��{$����6���-Y\����G�]����QK5��l_8s�,��2���������������S�_��z\�|��[J3��6=8r��^�h�G-��t���O]�
�����I��(
�B����u��cv���	�����i��j3F���5s��+�
2��9�"T����j�1;�I�e@�4Oj��I<�g��,�I�$Q����a�����
[�75�<��L5�T����.��ve*��2q���~b��`;��2=U�V���PsUU�o��c����J�T�z����#�D_Uw��+Vl����)�=��~��g���X�>{����
�S���#�[����(L��|�j�����+gnou���e���=~0���[���R>���?PC#����_n�����*��W��r	x��=�U�3k��/�\�p`h�Nl	5Z�'�d^xc�9��t�]�`Si��':)�6^\9�Qx��J� ��K�l5��������-(CC8`���7OL�������?���u����(��,�`����dWQ��|9�*�d~=������g�.��.���g������#���������YHl�~o�A#F�����=3�+>0�*����b�ecL�'2*#��L����d�I9i0R z�],AF�,bQE�"������'�OVR��6��F���U�R���j�{�e�t;�8���;����"�d*E�j�����v�Fq���2T���ZT�^��c�XR/�S��id�<SY�n�7�[�����*������V�k�'�#��+��Wh��U}��F�����g�)����_-Ei��:�"n��Sm}&�C���>l8����t,h�T���\"v��I���������aC���J��RT����#4>/���CY&���p�vl/�
�P���qd�!4	D�P���E���@��D8�����8��?���sv1�����p)$t�o~g�����QI�*�?����
�_f����Ta0����
|�}����o"��U!a`�[��!�L��-�MO;u�1?�`.��g\'�W�I�IZQ���>c����&0���!w������{�{>&�������S����t�����\��q����j}�<�4�������L�3&��y;/�>���>�~�����N����u*�����x�t�L�n�hkq�kVmF%�\�V]FvP�����������	��.5��4e��~�������=����y^�G��yq9_��+]���sq��������x�_U2��D�-0���]����hW<bD	���M�9d��<�W��[����r9��F5R�"Gn�hHC���~�A�+��8G�gKS���[�[�E�nq�s����Y���*E�Z���Y����F��7�+�G���/�Md��y��S���-����r�?���}J<�Dm"m����hn#6�9����vDu8y7r�����)5c�a{Jk�}d��2Ae�����Q���j�5��Np]�Z�Z�R]*K�����C���:��Js��~��Lm�E/' $AQU�Y�]t����r��z�1Mu:��qIr\r��A�
���yNi��9d���Qe/<��v�����t�k��(=[J���I��������A�&��Z������2���d��s���e.��W6]��l
���|�sz��CCN54A���5?�`?z�g~���&�W
���.L+W9�}�C����4n.5�E���do��:G�#w�us����t|��~ps��qT��*��P@Fb���*��O��*��\w�0�n���`����&u�o��z�=fM/�x.��s��lW�|���Z�s�>���E�������[���y�b~���� ���GY2���e�I�������y�/_����j.��%����e��Q�;���G�=@��=L��8wx����^�L�:�y����l�o�:/�}Nly��D�����sVfn�#��]N7v��j�1<T;���_/?���l��6�]��4��a�����^������lr��j�^���s<�Q��"�����q���>T~��Y?��-�����j����8(������^���1�#�4Q�$YQ�(*�����K�N���@$������$NU���+DO!����m����������(GD�(�v�sc����"�9ITn7T���q������p���I�0�+\K�����3�e��~�;yw�,����H����^���Jf�n��P
����������x�
j�E�Z������	R_-.J�*F��B�z��fk���zU�J�^t��+�N�"��_���Hm�c�|�'��d�$�8���n�U��������6P�_
��`�����8��0w+y[&n���;	~g(�Kbl_�����R�R����x (_��'��d�<^�_�'���,�6�H�M��%����O�HHN�29�������%��]����)���I��IE%���0H0�
�$d$QT'i���0c��JZ���� �J�AI�o���E�FF���hr�tLe(����z;�[����P���CN}Q�N9]3w��8��O��n�v�~��cli^%��{��1�	`�w���L�5���t,���]t�P�������jd�z'�|r|W�V�����|��5O��k�B�)�2}
����zV%|��������)����e��AN������������ /�>�Y�O9"������������<��}��d;��0{�=']�������3��(�������,�;�C3�#��_�|��
���X4�B��Kh	
\2�W�(�\_
��,��3u�B���@�a��������y�=,]����h�06����B� <	�3���&��������x�z�2��z�%��?�'C���������u���t/
���e��{��-�^G=����B_���WBX	u�t ��P��eV���j�V��)��_E�!�����{���}��b���0�C��	!����-�^H+��W�����A���'h�M��6���_AH���1H�Nm�i��'�
�
5A:B�r���B��Q��h����)���)h(\ch�(�m�����0��?�6r�Po�w���1��;����|���)���?�9�'���>LA�����V��������':6p��u$�u�R<?
� ��&���@��t������Z({�L��,@�)N�g����T�9��g���0�G �!�h���,��~xO��BW� <a6�>C(����1|�����pCx���p����l>M�y:�.ZOB|���}'��������)MQ��R����K�Iq�#���F�h
nY)�;h3���`L��t���gi�����56&@���S_�1��C(���;����tz��(^<e#��_�.�@��'Q�u�A��6���H�
U�\��G�>B��!�%�A?_���=c:����bAx9��������v�[�����{4���������@>^��r�+��\�� �	�k�
B�J!�&`�G���U�t�S���#����>��c�����{������qj"���M"���A]�#t'
������G��Oq�J-|�iJy~�
!��~�G��4�
�Q�@�3���!�4�5w�?�F���3�����:�=��$�.C�@����$A�!��A	���Z�P+j-*S�\��,u�hU�T[[��j�H�����I.�@������]�����Y��}����8��N�t��y��y��^��[��;��X�|?����q���3��[F��K��z���kT��u7�X��}��������Ud?a=mo2��MV��<������A�����T�Skg���U���o���b�&�gm���(w��"��TD~"����������X��P�y"��(�����<����w`/p���$��\����~SY��g:m�>�s*���T�e�������-�x_{�i��$t�S+��y�'D����`R!/�3��:{	t�}�E�%��xL��[��0��i��I�����z��t����0��IJh����n*��:�����k)����sH��2����s#�u`��G[(�F��^`�>v	�c3��X�f{�����Bk���
�I�{7�nG���������&���6�m���xC��
;�D�NA��G��KKQ��| �����l4���^�X.~�Rj]�D�)���M�����1t�y�5�Q�1m7����C[=N	Ta����ZZ�n�-�4"���a[��~�����p?De��_J��TelG�;@���(k����z�����^���������q{+��8���3�h�+�E!���J�^���2Ez����"���t��L��w�{�u�������Aw)&�+hX��M���Mv�����h�9X��3�������5�&����;���y�K,2k�����[�8���x�������;����}a�1n}���7����cP�2�bj�]H���OG8������4��WKi:��?*C����d�o��(`�������
�w1����P{Y���v�>9M;D���Aw_��:����W	W��������v�kKn��*�q�Ac}��F2f.�����K#�<{�t��D�Qoc5�	u���nk<�`�L��3�A�����#����4�����Oc���B&*_�p���s�|�rq��2_�1�A� 'C�udt�u�[����4��j�����O���/�����u$�0��\��a�\���q}�7�`#��)���F��^ �I��!��D'�����?(�u��Sve'��Pa}�}�e�/}��&����������?B>��?F��!#����{!x�O��
���^���}@[���2l��6��e���o+a��D:���y�{���S�-H�\�)��d���K�|���}�}������8�Dy6DcN�O��<��h�e�~���b�&�X�K���l;����3�������G�g*��E�����j_���(9:�����{P�����1�6`�����+c]=��/!_�;
�KgLs���������}�c�wSs.���p�b��������<��a����S�3�;�������]z����%;�\�n����.��q�9����9�Lg������+<�0�i���4��qc{Sn�Qa4�z�1t���v�e��E�o(���������)d%?�\����m��
���h�&tK�-�gw�e�\���3�����l0$���j��y����1��<�,��4_.�E9�~��;�����$��!z���![A�B�>�i��>i�$t�����4��5�A^��+��(�{��G�
c��Y���=�
y�^b�����1���1����x��=�y&������T����9k��>������:F�:�4�,��k�}����b�E��,�uw�zN�oEZm�5q/�_%�����G�6�=��������Tg\�1���?^��������m����8^����	�+�XzL�C�u�l��w�J�_a��z���Z�������c�{���&�0�s�d7B���;��cwZ���/'�d9�	����>���kPm����D>$�k��'m8����i���|�F����<
�������a0]�k�F�A[A4W�=����7�pW�}3k2�E�f#�f�;r�;���lN��b]�����wB^��"��^`'s9Mk�M{�)fO�n���P���pg�oWR�U�0��o���~Dx�3��5R��}���!���T���L�Z�/��������\�6�}J7���FU2Z����	$,uF��oA�L�E�2�v����4�����^.3��7k]�>a��O�����v�wl��Q@�~���G��T��"�uH�1��\ ���*�Q�t��S��
���v���(��n��2�
���I������3���?�=��L��p������
���!g��L7��<K:.q�K��@��yl����n����?�����[�c~a�V�?(����_���|�<���K �0��'6��2U��p��W��`����6��qT�
52��� _v�;�e|"n�{� 3�j�}KU����UM�
��E�H�&!�rr
�H�=���.��J�����o~����/���j�3�0V��n���	�}��S���D�m����o�X��C�|�����w{*16��ib?o�y�<�����0��c��Q�o�������f��XF�W)��x��FZ��ia�$B���W��T�9�RO	ioy��}/���t��'t��j��>CZ�l�W~��.��J����u5�y��]���!�����0?���%����	���N��{�^j��r}E�?�M'����31�n����yR)�s��sz�L���sc�������MK�Rh�������%��g�����O���
:sw����)�������v[o��b���R���y���i9��p�5��hS�5��5�{ y�l�~%��
����i{��}<�&��*Mbo7���g��4�k@n����4�s1��-M���d�(Ja���v��<F{���LA��E����-�Rm|>��������E�������Jp3�+��3{��?yT�_<�Y��D;�yN�Z�yO"�!�h;U����4��w����E�-���
���f���r��
;�=���6,��vKS�OY$��
Y>��/��������+g:�r;V�3G�z����)l<����>'�$m[��|��;�������?�krM�:m�-E����X����}~>�s�p��:p*�>O�F�{��(;��X+��n�2H�-�a�'�
��a.
�����)h�|]��(����^�������h��d!-�
��{Tq��m3N�;�����C|���4;�������������gK{4���#����l�Q>�J�Y���Va!_w�-`�b�����*�P�f����aN�+07e�[���,Xv������J���,Y��=d>�s;����#^��������	�^���s��q��y�y�}��gO��i{�I��<��H����&�	��/{���v�^�:i����9HK��LOF��m:�:��
<��m�c��4�U�7���{��;NP'q�4$��������N�'x{�����)��cl+>ss�:/�k�BJ��/�MQ������ ��p9��1��_���}O��4��;�g�L���;���Ef����;�����h���5H�4���N}����{�=���}
d(���4@��@=���0�~����~��%�\��wP%l�J�O�a$Uj�qU�x�t��XF{uo��W/���������������Z��"�-Xf�y��:�#a�Q�4FY���lG��-{
��t��g�\I�S4�s!����}p����|E{r��5��V��4��n6�'g���;k�V	��{�F���o8R����"|�T1���[o[�E&�O��&�\o��v\F�s����lP�=S�S�i>Ni���9Tr���0}C��+K>��uK��jo���y�=��*�	|^K��T�Km^�����#Q�O2���[�k�L�E-��h���i�s�Sq��h�,G�n��KK�e��l#a\��6��~���c�I�-�G����h��=s�1���Dy�4�W��\���)�Gm�7���y��{b=����.5R��AtV������o��M4������r�*�>��y��J�[��4P�F�A�Aq_$�%���|!���[�)������	�^�'��}�Ax_-�ge�ft���~@�A���q�|�{5<�sj>��������y���Q��)�� �LK���-���.G�d�0[T�����
���bL~�z����y�@q6�C1_��l�4���z�('�����������u��r���@����q������Za�'�@'�{��/T:y~�	������3�s�D��5���C�B��L������>��3${�qq�~�Ao�7��x����T'����|i����
;=��_/R�ou����b��z^g��
��S�������������l}�"xW�F�
��6�e�.#��b�J�<�O�z� ;���W�������\#��tvK%��Q�nD��(�9���&�0��y����M3�A�
����;%�^%�|y�$i�G�*��5���h\Pr1��K|������hzO�r�V�K.O�#F�1b��#F�1b��#F�1b��#F�1b��#F�1b��#F�1b��#��4��Vt�F���%�)��������&�;i���&���9����:+����a�U��j�����9	y}���-�\��@=0��H�"��Al��
`��O�Z��'F��Z�'��0:!l'�1��HG�
���
��
����c�k�bP>OBF���"�������r��r�,��p�L/����R��j����A��_��=�H��='��U|���F|d$�:\5����i��uF{����O�H�����7L�������nC��O��k���QJ&���S�D���MR�������
�C?��D��#��������G����{�yW��?R6�`-�G�W�#���a�-����@�������z��m�������W��rv���lu���n:vQ7�rj�7���B�
��Q�v�h$
4�Uw��5R�G�����������(t��b>@0��w�pw���^�Dj�� ����C�������F4����`�?�����uD����,����B��� �^�t�}����~�k���0�����x�����L��yIz=���k6��AX	,�^�V]�O�Kv�>A��>r#m�Q�J(x*`�/�a�����A=|�p�%x�}��K�����K��[p���Uq��`����K��w|	���;\j�5�f��?O�%�7 �n@.��\��L����&�������c�CY�z��uZ�9-<Io�����"-|���/��YZ8U�k�������k��Oq^J�����SZ�J�pw-�����P���z�@!
����Fy�H�>	zW�hW�����q�l�
A)�M*wJg���w�t��sm�h}�A1�������j�/��$��*�np���v7$|��&��
rAX�K$�(��Z��m"a�*������O7�t����S�G+S��tm|�����(9��T����W���Oqyq�=�JJCA������i�Z����.^{���n��iPP�y>U	�`J��D������i�P�����p������Z���o������Z�� |���H]���]���s�Z
�. Tw���j�P�VW�����i�(��T�`�|pY\���`�4�W�z�?T�w����^�!�s���HB������J�f��NZ��	��>�-�����x�x�z��4oo;_�/���w������|�O���]�}$�����%��#�J�F��O��*�����}:��H[�X/���Gv���+��&g�j�&�E<�Z$�����G��*����"C��#�	��l��{J����j4��V����.���Jv��%�����w�(-��sSr�G&]PT��e��F����S��"O.�lN+�����VZY59P^�S;�}VX�S��Ei�Nc�v�p�#JK�k�iB����C��\��00�|�Ro�������d��8�.����	=Sc��U���33�N�U	����h�}��������}Bg_�0�DF
��T���
�3�
�T��P�����T�5�,1Z�N���?�����Y�����f�gei5�Kg���(��Q8����pNJ$|E �}f)?D���+f�ay��Hi������������<.���3
�Sy������Y��C�3./(�5a��S�Z���	��l�l�5jh3����Q�P�k(�5*4J�E��O(����������[�B}���ki~���F��;�k��.u�V6Q����y��x�������GhS��
����E��v��6�G��N������PJ���_��k��py��:�?<+��./��OT�=�8�;��d����I�a�_�����n������0�o���)����������]5Z(]�OU�F$�x���`J������U���*-K�r���-��6�f�������J��R�dI�?�����������
endstream
endobj
14
0
obj
<<
/Type
/Font
/Subtype
/CIDFontType2
/BaseFont
/MUFUZY+ArialMT
/CIDSystemInfo
<<
/Registry
(Adobe)
/Ordering
(UCS)
/Supplement
0
>>
/FontDescriptor
16
0
R
/CIDToGIDMap
/Identity
/DW
556
/W
[
0
[
750
0
0
277
]
4
7
0
8
[
889
]
9
13
0
14
[
583
0
333
277
277
]
19
28
556
29
[
277
0
0
583
]
33
35
0
36
37
666
38
[
722
0
666
610
777
722
277
0
0
556
]
48
50
0
51
[
666
0
0
666
610
]
56
67
0
68
69
556
70
[
500
556
556
0
556
556
222
0
500
222
833
]
81
83
556
84
[
0
333
500
277
556
500
722
500
0
500
]
]
>>
endobj
16
0
obj
<<
/Type
/FontDescriptor
/FontName
/MUFUZY+ArialMT
/Flags
4
/FontBBox
[
-664
-324
2000
1005
]
/Ascent
728
/Descent
-210
/ItalicAngle
0
/CapHeight
716
/StemV
80
/FontFile2
17
0
R
>>
endobj
19
0
obj
<<
/Filter
/FlateDecode
/Length
24
0
R
>>
stream
x�}R�j�0��+tl���(#()��n?@�����������MZ�����K�������f�a�-L����c�@�p��pA���Jw�+O�����}����5��bq��Lo����[���������G���7QF���8�I�g��R��1�n�y{~��*�hFF�4�@j���c\��3u�]�a$�^���?UHs�8�1�$��w���u^$/�#�����r(7HSrA�J��mU�4��uy��@�l����9�S��D��+~]��NN;�W��@�����?}�h���P��X���X)-��d��}!�$3�c��up������o2��{
endstream
endobj
21
0
obj
<<
/Filter
/FlateDecode
/Length
25
0
R
>>
stream
x���	xTE�7~���K:Ig�t6��u��!�
��D%�&h$  ���,.3�� ��+*
�"*M���q�ad�ywyu�A������!����}���=����=��ZN�:u�������B��/�7��7����6e���m�pO!��i;�W�������D���Jg��6����]D
����%���!�7��/��X*��
�����McK�:��N�g�z���RF���nF��w��y�<�=��%����L��(S
P&�y8������L��"�~Z�~�+d~jc�(��eil�J*}����.���i"��)�<4�Ne*�i
��\l~D'����|�]inG�u�4}��CeTAg��$�I)�Q�y�i���<4�^��W��
t#������&�����j��|�<N�i��N;�x���^����s���*4_3��5��t?�d��)�M�U���)O#t�Ca����h�1�t*M���ZE��y�������o�H�$*D���Gl;�������t6u�����T�V���W�w�OP
=��l{\+�����y�� ���A���hg:�������_j.�ShZ��b~��_�i|	_��L1�f�vm�fd��G!���!z�%�v����/����/*�+��WT�����|�h!�K��i?��4�_��l.��������������]Z |(��y��y)�N��i)d{7��n��J_���5s�al6����!�)w�>���7�{����z�qu�:J�P����]���M���o
�~ �W�����8��:H�w��{�1z��No�;BP�6���V���F���+��$���G�Z�:�_9]�o�7���{��������M�Q�*�������P�W�j@�R��ST3S���M��i;�'���*}�>O��v�m���]������p(���C�.�$��-������!������(f!�e������zv;���f�+�
v=��ng[����m�{���	|����|-���=�9�?�����J�T)�*S����0���e9$�^������|�|�������E�����}�n��������E{L�������\O�3��}���M�
�5����b��>�e�����z��4��~|;OV��#H�b*�c�A����/�Z	c^�D>�����$��j��^��DKu��*�������O���U���������y6��5Z����l��U|2��{�m��������B��v�#l8��U���
�(�r�2�p�9���sB�w�:�~��U���Q�N5V�-�S;�����f�cL3?�uS`�������_E��5c�-�zL����"�f:�x�>R��>��G�h�(X��s�;�w�
�+���a�����b���<��������-)��n�)4����[o����������x�������*z����l5���??���	��N��yY>+�z8�-��i������� H{9��~���������
�cn�h������F�oR��,��a�����F��\	�m�z~k�s��s��t�q��������r���[1��gmH����>����0����fX�N��Mz�6e��.�����:�f�����va�JX�Z���ws�(���_VheQ��.�4 |�9��Q�c"}3v�:��G/�1�.JaciHx<��2S�{I��V>�\�\��@���b[-�Q3��yR�����*��.TZ2�x@�QaA ?/7'���������M��$'%&���b]1N���k��
�[��ZBj ��S�E<w��Hh	��T��2!�,��~I%g�*iDJ�%��_EU��cr������v6e\#�kks���#2|����X�������]����P���������]1����g:��.g�1�Rs��b�#���1�wq���S����1���Z����?f��P���1���M�Bl�y��C�;*�Eh�l&���d3�9b4���k@��5�n��t���1����2�I��D�����{ODQy���=s3�Uc�s�"�j�
�s\c��l�ljB���u-�����~�����C�*4�#���of���r�?���;{�-���U!Yvkz��a��1�Us�C��M�j3w%������������N�vW\�p������!Y\���wK����
�����'���0��9�V�7��������r�nY�.�H�w��W}E���#�~?e������"z��j���C�`��"���S�q��)��������@|��Nk^�gg�	^�n�tDB��5F�~���JFI�)�[DNg4'e��Y��fo��&�&�������?�nO����C��3�3#��r��Mi��Y�b��~��b��a�yV(�4�Q��V�g(2JyNwait��|���R�)PJ���u!w�)�g�3;�'y�m�L����K�lV/C������^�{�s�R�_5��'NY�����:�U��r�u�ZVMk7�M���sWu���}���i�Nh��guF�nM1�
��r�+����`+'Li�p�$�rbc+g|t���]y�k���b�T��*b~�zEo�v���a-���L�����4{4��y�<���i�)�#&_�/����YX���j#�45�������u-��},@8�^��_WuU��>ZuFWU#�>��������|<��~�����w�Wq�c�)I	h�8AN6����\�j;+j��1�������1r9p�i���x����S�jHU�4�v�o-7W�!������6�TJ���	�m�����J ��m�����j��1f�sW����B�X�1�!������xx�H����9:kr���M����L����B�"�u�wt��t{�7/-/]����J�������gn��'R�;��5(kq���uY�e����N���R�����,7����[�B�jK��S;���3W|;�l��\%.�2�����9z<������/�i�^~B����G�t=������
�?���W%$V����f,��2;[*EZ�%1�����]��@*���i��GOl4bi<#�	�Os��R�\?��Q�����C��
kb����YB�����C��rst[����e��u���6�u�����?��lj�m���O����g}��i��nd����>k��3/�M�����ym�k�6��8/{^����F�2���0����p��_+��������f�|U_S���#�J���Z�s,-�����R<��O3?���^���5����$_��D�g�(��|�8�������"�"m��j�nK���&����J�f+�n�nq<��)F�fc���w;���9M��K��eS�To�j�3���������,�H��c����b��Y�d��dU���,3�`����|�R�����-c�����?���u�G��y?�)?T���31-����)�M��TO�����.�V3����O�o�W��Y9��6�,���{�����o�g������M��
�]���������,$wV�rm���F�]Su-�������mWK����n'E��!s��V����l����1�1<Fu��X_��4!���!����@D.�����htW����`U'd�dK����gtm��|������ok����)4��4����\���/�Q��ZN��5h��C���Z�y����L���Bv��K��t���<��J��6Yp�e������1�:���j{���w@F�j���d�7�D�*�����n�J����:z?,�E(R��5;�V�i~w�7b����1�����Q�@��*L��c82��;��G�;��Cm5���T<t�����S��:��$���>g.��%���lu���%�R��v�C�=�����M�:�&���������;?���9>v�^����{�F��c�s�5�����;f;��T[-�Wk����g9�6��$��W�#��q6��T��p��t5�a����r:4��V�����O�no������e\Ll���+(�$m4�"c�\e��$;�&$�Z����q�rD$d��#�b��W�G��&+��c��2�#�Q��R9w9�����cq�,�M�_��aR��n��S����Z���-�3��K1���c\��3�G
��BT�s1��&v�"��������tW�����w��J����A$��G�A��v�60����V�
��>�4��n��f�Y�H=Rp�P����Y�z�'I�>~+�n����^��cu���-�"�}�:���%F��u$wx��5v�����c��(�-�t<�=;m�&��#�\�&���*�j����,K�r�������������_'���y��u*�q`���M��TbO)�������/�>��3/x��-;�>��!��=���v�h�����7��p���5
�c�h�T�Cu�:�r���7f����Z�'���S2O��������������\�>����p�I6����)=�S��4�]���[����u�c���yV^����yFp`y��S�GZV����y<�N���q��'e1������fe
`��@j<���I�FfBu������^���>��\��B��������0���~��"Gal������O&T�����>�����6\[�\�=5������NM�����
�S�\�4�if6�����@�G�����
FVdkI��t$��<�������y3�xs��p�8Xga��7��,:.�'EI��fct=7'0�|(�������,�$c;���i_zq_{����8�mSN����G'�~��No�[?��;��������v��w�vc�5����\uzfE����u��������qcF�_J,�T��\���	�L�6TC��?�q��M�M���s�x��^�v�����i����y?�?���Q��z���a)�%����6���l�+<^���������mK�������'g)�\���������~���'���a*91��	1d�((G��a��0X,d�SmL��l*������;����������}_l���G�b�o>�����E4�����d�VhB���t������7v�K/l�����G���y�<���l�����x���?>�L����Ox`!�i�2d�:�8�Ior6%N�L�6en��������oY?>\)w
O)O;M�u��R�v����m�b��������1�������o��S�u�X?w�=-���{����:�HW���FD|9x4��G7�s�9�9��<��s2�����!�����%��aG4L���f��G����h���u%��^>m���g��xv+`v��n������~���<r�&���-��$S&������F]L����bovo��s�u��mO����)�d��9���������q>�z�y�����������$��K(�Oy,��%EjC�jI�RA�Z���������e�����u��G\���
G�7SR#&e�8�������s�������Q6+I�(QI�������O��m7b��!p�"��K���H����^�_<0C^a����K�(�	�H�A�D�\	�-
S#�]�@�H��nM$��p�����jlX(XX�f�|�)��F�D�p�S�IV��"�&W���t��9����C�����
�$��MO���w�G;��\5�%�|�%�]�r��QS
�K'�SU������zh�[��`����W�>������G/���}�<�n�kf�U�����.�jv�1/O�$��DOB\R<����y������ps�D8u��a��yD�N��sT�'%;���c��(�BwI�����T#6.)����fO��{�N8\����K;���y78��q�>���n�a�b�����,?�-N,�.�MZ���)�pr�+o]t�����'
y����@���'�=��W���G�S�����_z�6����NpV�_�9:�94������	���HS�tCz�
5sV�9��m�+Bq�y�j��:�����t��t���frq0z��!r�Sfj��w������O�p����"�?x�|��:��g'��	:��+F��P����Xm��k��K�wo�e(`#:���h��J�=�r��^�-���c�c���.�RR4��R��hS�=�}������\��E���������]������/���-[��g	uiu�����6wBBAFff��	q���	��!-	l.��������@V&��f����;?(����@G���P��E��
�+(/0��T^R�b��J|��`Y�B����@-H+|�*�@D����tpwU}�uU�z~s0z#]W����Z\"@��:�`Rv��
R����I����n�2��0eu���K����hKaV����q#f��zh����j`�&N�4q�9�����;V����0�u�OP����������9��&�kKJM�b�mW�U��r��k�?rk�qV�-.Vw��`��,�!��W���)�]��''�|�w����~���y��p�	�����A�f�^YZ��S��%)�YD��	RR������{>A
�OQ�����<ua���~����>��������}j`������F�����n�Hc���b��!���M�7)�3�C8�iZ��7�''M�4yo��
�[\��W���7��>�>�?�u�g��Y���K[d�F_nW���
%���J[zK����M�sK�-���f
'�y>vk��,��s�*kn�I���$SnN^ ?��^=~U�����s�^�f��|�E7�t�E7��5L_~����\nn�s����m�]��z���_r�1pX�)I<�\���L*��UN�=5�6�_����n�k��2�X?��1�	��������R������$����n�����eh)-�:2N�[�0�����J��1<����g��O`1�b�p�����W3}��t0>��x�XL���Y������Wbjf���
~�nR�GJG��;����	
���
9�mF�-s��	�,m���$6������:�.��_�2]��5��]����f�~+\w���(u�s�����������;$��n
0E���el3�,-���eF6��=�����3"�$��,��b>��R'��������Y6�I��F$����Yh��k[��:���K��-��j`���?1�R��c���0:�=�^�����!qC�-7�w����y�����g����6���(vR����bnI�/��e��5�j����H\�|u����1z]�i�c���G��r�
�!�Cr�����N-����-p������r�\�&_���hQ��)����rS��9�sc���R�xo�����zj����-���r�����~�=�!��|�ox������X���t+���.u3�`���e��:�#��#�R��v!������fYq~v%x��;�!:c:��@�������)u���5N��\M����I�.^�>Uej]aLC:K�K��W�q!E����1_h+��ci�����~y"~�������������lhN]���s��y%G��q��j��C�fi��k�R������xN~��F�>b��`
Lma���L!�F��Q2����g������\�c�j��T���4�RQ�*N��F~�7>�'�j��t�;>�5�����|q��?���y��G�8����xR����Q�3�31����x@�>[�JvU�`���xWL�tY�a��=��;[]�>��������S�*���Cr)KO���_W�'�����%o���+��o�6�-�g��M�G?{����O*�,��$'���|��}��4r����/%s�i�W_�R��g~��kw`O�o���r�E���N�k�����W��PjbR2KM����8lN��+�O��SC�JHg��
���Fq���fg:W���YBT���J����R'�T'oJ����$/K^�| ��d�������d��K7G���P����J6;�5E<���U����="�(��a�.b���63���R��Bh�4!w��!�	����������{���1�������C��W33��?x��A7��|O��g-��5�`��zV��	7k�CO��xUB=�O�����P�9S����s���0�q�%D�����n���>�3������Ad���4G��2;2��CEP9s��s.�~:K���>���,m����n��o{�1v�a�	7���4��aF�V�^Rn]<������@���O^~��t%�nw�bRX
OT���*�y&���������IZL2���S��r������b]��Gj����trR����
��2�H�!��t88g:��Jqcbx3�cb}���F�����vV;�:g;/5bT^�V�cUE��K��-3�]C��aB��z
��&�+�=�H3v���3���}_��*���J�.��l�H���YvR���J�f���DV���T=��<�Cz]�<4�S\��EdJ�%���'��?5��+{�]~u���*�5�
#>v�x���y(�X��,"����4�M����MVz�Ol�n%�J��]y���(��eQ��Ma�9���dQ�z?MB�E�/��W�](?�����Y&5�y��:��z$}�.�����7����t�n�+i�HG[kPG.��"�};�Gx��DYI�d:
��A�l���dC!��t��^�4����#�G��"�$��V�2����cc�F�!�%H_�VgC>��|>��"�~9@]@�J��2U�$
������I����/����$�����G�~�@������a�*����1�#�~�������o�y1������Av2�����n��xP%������*G1��r�f���_S>����|���Q�Y�L���������>�=JG]-�h�����l?��������\�s �
���h�D���
���B;�h�'��G���>�b���<D(�� �����E��zfA������4	H����-@�(���(���
��)�C������	���1����Z3���@P��O�X(DY!��Bg�z��-tK�L�J��P���b�B�z��N'� ��nE�Xw��2A�����W��7}�R!�kb=�5a��c`����'u��QYt��6�9Y_=���T��3�?���e��c|H�x�Wa��4��IE������� `{�]���Sw@��R����U�i;��4b�i;���
��T�g�����M�A��X{�41�����}�J�"�X�������6��:�Q`�j�p��
����2 �|�O�O�vw��}Bk1_W�R(W��m���?�~�3z���t��.EiT_{S�3���j�iXw{����x�x
p���}��l4�6����n�|�6�^��^z���~�z�eo*�a����u�~���_�Ga���vN�}���i��`;�&��~�b��"�(A�,;���Q�������js����W���j��|V�mn��������-�)��
9�}1��j�e���dY�/�����~��4��Y��b*�� O�w���~��C���xeg$]�@g
��.F���"?FY'���_�b��m��S�n������/�����4m
��+Q��{�]�(�J��1�s�5�n_Fw�:�����;1�g$�]���m3����FP��`|� xp��[��Y��S��&�����SE��D�Mt��I��`���tl�lk�e7��U�_�����$Z�%�����i*���>��`�K�4��ki��O���G��R��`|�?�	t�^�X�Ak�N����^��}��\H�^��0�C�1��D�H'��O��^l���heD���v����h%lI���[/��	��0(_
,�D ����Q�"���g�Nd2��?`��N5�}��g�����%�B9zw{�BW��:�
�#T�|+����*d9���Am'�P[i�b"�n�>�Ok�)�y���E=�P�8�A_�p�y�('������Q�����(D�����M��������}���������|����rz����8��v����;i	�������)=���hTu	�	Z�1�*����:��B�����s��#l(�����	����������g\+�J>6}���d�;�\�v���#�,&��GE6;�w�>|��Z!���E����R�~�)�!G������/���/(�W����'�����8_�3
�:�!|r��$�G��C�1�s] �t)�;)!:OH_)�{�te��y�t�������Kq�;�'�z����(��c?0t�c~�n��0��!�:u"������J�<�7 �}*��O�"QV������w([@�����H�rmrU:��������^R��=:�y"���=�l�u�;-jK~�L��Q�Su������g�����=g]���e�!��w����%�ZI���D���a>��&!�'�E�P�F�Bx�^������+����j���|���|74���
�v��@��{�F��P>�&P���-G��>�+�#������q�1�K�'�X�#?r���?q�����5K���}��/��|��}����/��Y���|��z�}~�����Y8
|�^cv���I?������hQ�o�*�If�)J*|g��
�Y����9�E�g�.���_�}��m�h&���G�Q�%��s��x��_��\'`]t�����x����>�+���ga/sD���m�������4������=u��9�M�e�w~����{���_����{�O��=���i<��G�Ie6�6~����~)�K~����w�����|���D������^��I�z�����w�u:J�k�]��>�^������K��l��k�sW.������f?Ne��������6�l��}����7���_��M�/�6Zh�%}����?��!d&��sq�J�@"��u�\��/6��*�y�9W9l~����)_��(�y�������g���������N�_��]|�����3��6Y�y���	;?g��������G����Q�������q7d�L��1���&�����Iz
���!"w�3P�<q7��C�{
��k�bq?%�+�H�1q=���M�P�TZ���V���-�w�|'���h�x��N7:����c2�a��;����p�`w���^�W�}���������q�)���'��
�Zin��n��>�y7d����o�z*�b�(����w���{�e�~������N���"w���5�q��K��K�-����|��ob�Q��lWZ����d+m��j�t��Xqg�� �������S�e�3�;��H�]%��F��e*#�����^�dy'��|�w��%������'�.l����&����?��n���;���|�g>e�vu���4K73�o�^�&u&�z'�����AG�ZX ����{�JY}����r��nn��Q����gG���i����N���h��|��(y�!�Wa�c�{%����2v��WFG�2�9l�w�����^mp#�#�m})����w���v�m�'���"a����/-l��3���7����_�������x�[P�,�P=��>F�x-�
���Vzzo ]�Q���Q?���*�S����@o =������7�7���3���
�����)9����~�g���������c8�>$�O�������$������k����Z�f���1}jq��~l~
l���h�L�����B��=9���
���-a��j���?�������;��d��GxO��1oC��GX���w8t��E�(��;����c�p��a����">S�����H�]8'�i@�����]�gqN<��)�j;'T�Zis�Pr������-�����ZEY:|8��~����<	�/�����gH�ay��@���q����P�!��,���[����B����)���8O�&}8�t��Q��/�F?W|_"���l����!���h��NZc;�|���_���G���U��
��(�MM����r�t��������������~q���.tgm�������/hhF��)��p&Wh�|������XW������0�tp��6U��G�����������
�~O)������k��E��H�>@��nS�E]w�_�����i�����]|+�����Q���uG���5A���c����7"w
��,�nL���~G/��x�'����s��5���]���������h����h{:�m�R���l����_w8~*�n'�qs4��9�K����'[k�o����I�ki]�Y���D�E��4��,R���m�|��U�w4�"�6r�~+�=�|���H���o�������;4���7U?���7��GD�X�/E |9�/Q?�7����>�x���E���B�������~���g�i?�ZgQ���^z����r~��=���1�_�����a�����Xwr����{�s�^?���W������A�7\_�_�_�����������C�~D��/l������~�����3���zRJ@��E`><��h:f~!��������{�zy>������w��;��$�+n{����4����
��}���b/�Y���[�_�v��1�sL�/�r��1*��9_�>G������#�\�s��y1������]{��+��c-�i�K��|��:���ke����Y�oi��_X�r}���Q���Y�P��@;�No=����|cv���m��o�u �������6��+t�-������$|/��Y���ol�Y�/����Zy'�{�=�&���i��}>s�e��w����D��&*a:����6�5�}����s�G|?�O���7�"�=��~G{h������{q����'����$<a�u	c�����������8�3Z#��G�����$��q����/��	A6�2Q�^~3:A�#�������wJ��<����D�����
|�O�sN:�m��C���/R��aS�-8����n�I�
�oqF�>�?���z	p)����������P����4�p�V�P�:W����(~��t�<�-�r��N���D�}��D�(Y���6N'��a�G#�
�9��{�/��h�Q'�hG���>Up��G�1�����:Q|?d����8=b�QX��X�Nq�������-�=K���T�{�J-��q����7R!����zR�Lg���{b�-��j�����a�
t�?G>Y�K�w��(��N�{����	�mG�#��Oc�E�\[���_�.:��E������#\)���hT�*4��X`��^��	��|AQ�~�#��2+��
�$����=8Xb��4�{g��O��f����[��O����_�6�����A��|�����_�����?x����w���-��-K��W�?)@�M��]�F�����f������;��'�~%��J�[��y��sw��z�=���]g	�T�c>%��������=�jk��iG�8ikaS������&m�SBvF� �99vF�K.D<;b�D�/�j�����S�)S�3i�"6K�La��:��R�\�?����#���w5��#{Nx�������z�=$���G{��n}�����d��Q����_�Q��������m��?��g��;/����6Z,�+��`.3z���[w��r��������+�6z������-����}Z��i+5�6`�,���_#�w���=�.�:�E�rK�
1������[��>����=���h����9���\���W�N�p���z�:���}��?|�����I�����.u"�&�{q��
���P,;��A���2����_��2��x���{`<��'`����7"py�bu��D"wO�>�?Q"�/��D�8��l��s-Q���e�)���>��}�C���>��}�C���>��}�C���>��}�C���>��}�C���>��}�C���>�����������*��t����7{mU�	������&V��v���l2���l�eF;hb����`Y������2����e��4�#yG�$�����-�t��-$i�=�mK.������o�������Ao&�(��-�u>�p/*��IV���<_L@A���X�����������-�2�����X�^4���&��� oqe�rW����q*w�R�+�Q<��_����-esk[|R�Q�Vn��SH9�:�j��m=q�o-$EX���+s��jtz5:�Mn���������<�����'H�����GmnoY�p)1e�r��OY��<�,������4���e��^5�W+)T���Ce��J:e�b�Z�"�,j-�_��V��H�K��v��Z���U)��m��������G��%��2�J��?�81�N9��m���u5.e"�9b���R�HVtQ+*�IP�(��A��J���)�$�O���@�hd�:�*7H��E�h~dD�F����u�8���
)�b����k+���RH����"�T*�*�Va�Va�Va�V�S��}�\��kP�D���)��:`�B�RZ!��+,�P�/��Q2���9�D����I����WV���z�u���To���J9�m��0������0z��<�dBB0YJ��_����Pd1�<? ��_�����/".��o��D���DI�C5��=T6��E��|/�J��o�����=������{Z������6����X�,�5Xb|�V 5�
$z�j���q�D�}�wR�c�^�N���}����-��'T�?��a�m�q��V� ;[uAl�H�����?�wP:�>�HG���@�/~/�c�^��5��X��w�Fv�6�AA)�oi���k���u�u|���0��bc�R�_Z\�U��������7�d������ ?����_��V�j�0&1.N���,C-x��!������U��h,�Q�`)���x^��-p�LY,.�5��y���y�c8��c8�I�y��E��hG8Z��"9Z���p�H��p�H�p4��
��
�hG��hG8$�8�a���Cr�0�aH�Rp��������(G��(G)8J%�~p����~p����_r����_r����n���np�%�[��"@p�!p�!�q��q�$�!p�!~�.�@���r,�r@�����r,�r��B)�Y,�������o�������!p��GHr��G!�G!������%�fpl�fpl����.����������F;�Z��I��>�t	��
�%�oi����+%��*$����>I���Z}�5����T`.�	�	<�d�E�m��C�5�6�������M�i;d���X}��SL�v��t����������u����l"xV�P5/G����C�[����#������������u�Y����Ti��T��q�h�#}��@�HX�k�4���kg�"���~
��W@P�>������}@�
�E��Qb�����lk��b�!�)(����R����� ��L��8��T �"�fn��V�ad?!�����lk���4�9��`��&�M"�*X'Zt�-��V�d��+	�D��h(�E����[\y��r[}#@rZ}����
��3��e�4@P�
���5����������@�P����*����l����+��k|�5NQ��.��}��5������?���7�wmq��k��kd��+��|���[�+�-,>�[�;�7�7�����V�9�}�������}
��T�"��wr~��b��2��+�U��	���H�����,����~���I�,��o����v�m�m�-��c�g��%��n{��ew��v����������7Yw����������}�U1;��(�����	�X}��<���}=!��9�M	i��X(���'�

�������`}��pv�.��mBj��lg4����"���P�hd�Uk3:�����65�����[�82����G-�3x���3���~Bch{VS�L������&��i���<vLm����C���������&;,�A��P�
A1�(��b�'�D1�Q�\�(�-�9c) ������D�]�cjw���L>�AY�`>�(�o��@@����FQ�5��e��dE>��d�NV�c��P��"�V�!�E���v��/R&�0Z&�e��������m��%O���;�%w�L�%�z�loh�t���E"�R-���-����E�3kCKrk��=�#�O��A�����1w=i��md�;��������{m]��Vc��TV%*kmU��Hv���m���jD[�F�lk���
���4�i�9��c�������Q���B�;Fd{�d�Q�m��`S��;*�������u&���oey�������YYn$'����hI�
W��0�Q�J����s�@��l/��S�?�/��o����G����E�������P�	������
M��6!m`4MQd�.�cL���� :���D(���������7��m\��g��};�R�8~Ik�<>�K�r���ea[���qU����2��VVA�#�H(F`]���u��7o�����V$������d�B���@pa��n���j���
o�`�)�@����[�A�?3������u��~atB"��J0��E�YL2s�d�T�u?N� F���77
endstream
endobj
18
0
obj
<<
/Type
/Font
/Subtype
/CIDFontType2
/BaseFont
/MUFUZY+Arial-BoldMT
/CIDSystemInfo
<<
/Registry
(Adobe)
/Ordering
(UCS)
/Supplement
0
>>
/FontDescriptor
20
0
R
/CIDToGIDMap
/Identity
/DW
556
/W
[
0
[
750
0
0
277
]
4
19
0
20
21
556
22
28
0
29
[
333
]
30
37
0
38
[
722
0
0
610
0
0
277
0
0
610
833
0
0
666
]
52
67
0
68
[
556
0
556
610
556
333
0
0
277
0
0
277
889
]
81
83
610
84
[
0
389
556
333
610
556
]
]
>>
endobj
20
0
obj
<<
/Type
/FontDescriptor
/FontName
/MUFUZY+Arial-BoldMT
/Flags
4
/FontBBox
[
-627
-376
2000
1017
]
/Ascent
728
/Descent
-210
/ItalicAngle
0
/CapHeight
715
/StemV
80
/FontFile2
21
0
R
>>
endobj
22
0
obj
351
endobj
23
0
obj
25916
endobj
24
0
obj
335
endobj
25
0
obj
16845
endobj
1
0
obj
<<
/Type
/Pages
/Kids
[
6
0
R
]
/Count
1
>>
endobj
xref
0 26
0000000002 65535 f 
0000048568 00000 n 
0000000003 00000 f 
0000000000 00000 f 
0000000016 00000 n 
0000000160 00000 n 
0000000231 00000 n 
0000000397 00000 n 
0000002931 00000 n 
0000002839 00000 n 
0000002859 00000 n 
0000003102 00000 n 
0000003246 00000 n 
0000002879 00000 n 
0000029814 00000 n 
0000003395 00000 n 
0000030330 00000 n 
0000003822 00000 n 
0000047859 00000 n 
0000030527 00000 n 
0000048282 00000 n 
0000030938 00000 n 
0000048484 00000 n 
0000048504 00000 n 
0000048526 00000 n 
0000048546 00000 n 
trailer
<<
/Size
26
/Root
4
0
R
/Info
5
0
R
>>
startxref
48627
%%EOF
#43Dilip Kumar
dilipbalaut@gmail.com
In reply to: Dilip Kumar (#41)
1 attachment(s)
Re: Parallel bitmap heap scan

On Thu, Jan 12, 2017 at 5:47 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

Hello Dilip,
I was trying to test the performance of all the parallel query related
patches on TPC-H queries, I found that when parallel hash [1 ]and your
patch are applied then Q10 and Q14 were hanged, however, without any
one of these patches, these queries are running fine. Following is the
stack trace,

Thanks for reporting, I will look into this.

I had setup for TPCH scale factor 5, I could reproduce the hang
reported by you with Q10. I have fixed the issue in v10 (only 0003 is
changed other patches have no change). However, after fixing this
issue it's crashing in parallel shared hash patch and call stack is
same what you have reported on parallel shared hash thread[1]/messages/by-id/CAOGQiiOJji9GbLwfT0vBuixdkgAsxzZjTpPxsO6BiTLD--SzYg@mail.gmail.com

[1]: /messages/by-id/CAOGQiiOJji9GbLwfT0vBuixdkgAsxzZjTpPxsO6BiTLD--SzYg@mail.gmail.com

Please verify with the new patch.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0003-parallel-bitmap-heap-scan-v10.patchapplication/octet-stream; name=0003-parallel-bitmap-heap-scan-v10.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1545f03..cf2dbbd 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1232,6 +1232,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1ce42ea..76a99a1 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 86d9fb5..404c7ac 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -203,6 +204,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -255,6 +260,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -731,6 +741,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index d5fd57a..9df6dfd 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,16 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+					  ParallelIterator *parallel_iterator);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -68,11 +74,18 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
 
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (pbminfo)
+		parallel_tbm = (ParallelTIDBitmap *) ((char *) pbminfo +
+											  pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,6 +100,35 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	prefetch_iterator = node->prefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	The first worker to see the state of ParallelBitmapInfo as PBM_INITIAL
+	 *	becomes leader and sets the state to PBM_INPROGRESS. All other workers
+	 *	see the state as PBM_INPROGRESS, and will wait for leader to finish
+	 *	building the TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *		ParallelBitmapInfo so that other worker can see those.
+	 *	  Set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader creates shared TIDBitmap.
+	 *	  Copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over page and chunk array
+	 *			and select heap pages one by one. If prefetch is enable then
+	 *			there will be two iterators.
+	 *	  b) Since multiple workers are iterating over same page and chunk
+	 *			array we need to have a shared iterator, so we grab a spin
+	 *			lock and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,10 +143,56 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.
+		 *
+		 * In parallel mode leader will immediately come out of the function,
+		 * but all other worker will be blocked until leader wake them up.
+		 */
+		if (pbminfo == NULL || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag in lower level bitmap index scan. Later
+			 * bitmap index node will use this flag to indicate tidbitmap that
+			 * it needs to create an shared page table.
+			 */
+			if (pbminfo)
+				pbms_set_parallel(outerPlanState(node));
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			/*
+			 * copy TBM local information in shared memory before waking up
+			 * other workers. Other workers will create there own TBM and copy
+			 * information from shared memory.
+			 */
+			if (pbminfo)
+			{
+				tbm_update_shared_members(tbm, parallel_tbm);
+
+				/* Change the state under a lock */
+				SpinLockAcquire(&pbminfo->state_mutex);
+				pbminfo->state = PBM_FINISHED;
+				SpinLockRelease(&pbminfo->state_mutex);
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+				/* Wake up all other workers. */
+				ConditionVariableBroadcast(&pbminfo->cv);
+			}
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM. Here we
+			 * need to create a local TBM and copy information from shared
+			 * location. We also need to attach to shared page table using
+			 * hash table handle stored in parallel_tbm (shared memory).
+			 */
+			tbm = tbm_attach(parallel_tbm, node->ss.ps.state->es_query_dsa);
+		}
 
 		node->tbm = tbm;
 		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
@@ -118,19 +206,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			node->prefetch_target = -1;
 		}
 #endif   /* USE_PREFETCH */
+
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+									 pbminfo ? &pbminfo->tbmiterator : NULL);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +229,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (pbminfo)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+							   pbminfo ? &pbminfo->prefetch_iterator : NULL);
+
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not ensure
+				 * that current blockno in main iterator and prefetch iterator
+				 * is same. It's possible that whatever blockno we are
+				 * prefetching is getting processed by some other worker.
+				 */
+				if ((pbminfo == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +300,39 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				int		   *prefetch_target;
+
+				if (pbminfo == NULL)
+					prefetch_target = &node->prefetch_target;
+				else
+				{
+					prefetch_target = &pbminfo->prefetch_target;
+					/* If we are in parallel mode then grab prefetch_mutex */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+				}
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +348,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (pbminfo == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+					pbminfo->prefetch_target++;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +387,53 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+			TBMIterator *iterator = node->prefetch_iterator;
+			ParallelIterator *parallel_iteartor = NULL;
+			ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (pbminfo)
+			{
+				parallel_iteartor = &pbminfo->prefetch_iterator;
+				prefetch_pages = &pbminfo->prefetch_pages;
+				prefetch_target = pbminfo->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
+
+			while (*prefetch_pages < prefetch_target)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+														  parallel_iteartor);
 
 				if (tbmpre == NULL)
 				{
 					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					tbm_end_iterate(iterator);
+					prefetch_iterator = node->prefetch_iterator = NULL;
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +648,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +766,7 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -653,3 +853,218 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : then we need to wait till leader creates BITMAP
+		 * PBM_FINISHED	  : BITMAP is ready so no need to wait.
+		 *---------------
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Wrapper over tbm_iterate, which take care of copying and restoring iterator
+ * variables from parallel iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator, ParallelIterator *piterator)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (piterator == NULL)
+		return tbm_iterate(iterator);
+
+	/* We are in parallel mode so grab parallel iterator mutex */
+	SpinLockAcquire(&piterator->mutex);
+
+	/*
+	 * Copy information from shared location to local iterator and call
+	 * iterate. Then restore back to shared iterator.
+	 */
+	iterator->spageptr = piterator->spageptr;
+	iterator->schunkptr = piterator->schunkptr;
+	iterator->schunkbit = piterator->schunkbit;
+	output = tbm_iterate(iterator);
+	piterator->spageptr = iterator->spageptr;
+	piterator->schunkptr = iterator->schunkptr;
+	piterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&piterator->mutex);
+
+	return output;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node as
+	 * parallel, because only first node will create the main bitmap other
+	 * bitmaps will be merged to the first bitmap so no need to create them in
+	 * shared memory.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState *) node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..56f8376 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate that we
+	 * need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_dsa);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..40b2c84 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -42,7 +42,14 @@
 
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
+#include "nodes/execnodes.h"
 #include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
+#include "storage/lwlock.h"
+#include "storage/shmem.h"
+#include "storage/spin.h"
+#include "utils/dsa.h"
+#include "utils/hsearch.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -136,27 +143,15 @@ struct TIDBitmap
 	bool		iterating;		/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
+	bool		is_shared;		/* need to build shared tbm if set */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	int			dsa_entries;	/* actual allocated entries */
+	dsa_area   *area;			/* reference to per-query shared memory area */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
 };
 
-/*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
- */
-struct TBMIterator
-{
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
-};
-
-
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
 static bool tbm_intersect_page(TIDBitmap *a, PagetableEntry *apage,
@@ -168,6 +163,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+void	   *tbm_alloc_shared(Size size, void *arg);
+void		tbm_free_shared(void *pointer, void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -198,6 +195,19 @@ hash_blockno(BlockNumber b)
 #define SH_DECLARE
 #include "lib/simplehash.h"
 
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+struct ParallelTIDBitmap
+{
+	dsa_pointer dsa_data;		/* dsa pointers for all kind of pages */
+	pagetable_hash pagetable;	/* hash table of PagetableEntry's */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	int			dsa_entries;	/* total item in dsa_pages */
+};
 
 /*
  * tbm_create - create an initially-empty bitmap
@@ -244,7 +254,18 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	if (tbm->is_shared)
+	{
+		pagetable_alloc *allocator = palloc(sizeof(pagetable_alloc));
+
+		allocator->args = tbm;
+		allocator->HashAlloc = tbm_alloc_shared;
+		allocator->HashFree = tbm_free_shared;
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, allocator);
+		tbm->dsa_entries = tbm->pagetable->size;
+	}
+	else
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -1061,3 +1082,137 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_update_shared_members
+ *
+ * Store leader's private tbm state to shared location. This must
+ * be called before waking up other workers.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before waking up the other
+	 * workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->dsa_data = tbm->dsa_data;
+	parallel_tbm->dsa_entries = tbm->dsa_entries;
+	pagetable_copy(tbm->pagetable, &parallel_tbm->pagetable, NULL);
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->is_shared = true;
+	tbm->area = (dsa_area *) area;
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ *
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size
+tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
+
+/*
+ * tbm_attach
+ *
+ * Create a local TIDBitmap for worker and Attach it to shared TID bitmap
+ * created by leader.
+ */
+TIDBitmap *
+tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area)
+{
+	TIDBitmap  *tbm = makeNode(TIDBitmap);
+	PagetableEntry *dsa_entry;
+
+	tbm->mcxt = CurrentMemoryContext;
+	tbm->status = TBM_HASH;
+	tbm->nchunks = parallel_tbm->nchunks;
+	tbm->nentries = parallel_tbm->nentries;
+	tbm->npages = parallel_tbm->npages;
+	tbm->maxentries = parallel_tbm->maxentries;
+	tbm->dsa_data = parallel_tbm->dsa_data;
+	tbm->dsa_entries = parallel_tbm->dsa_entries;
+	dsa_entry = dsa_get_address(area, tbm->dsa_data);
+	dsa_entry = (void *) (((char *) dsa_entry) + sizeof(dsa_pointer));
+	tbm->pagetable = palloc0(sizeof(struct pagetable_hash));
+	pagetable_copy(&parallel_tbm->pagetable, tbm->pagetable, dsa_entry);
+
+	/* Mark the tbm as parallel and also keep the DSA reference in it. */
+	tbm_set_parallel(tbm, area);
+
+	return tbm;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Memory allocation function for bitmap hash.
+ * It allocates memory from DSA and also stores dsa_pointer in the memory
+ * so that during free we can directly get the dsa_pointe and free it.
+ */
+void *
+tbm_alloc_shared(Size size, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->area, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	/* Keep track of actual number of entries */
+	if (tbm->pagetable)
+		tbm->dsa_entries = tbm->pagetable->size;
+
+	ptr = dsa_get_address(tbm->area, dsaptr);
+	memset(ptr, 0, size + sizeof(dsa_pointer));
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * tbm_free_shared
+ *
+ * Memory free function for tid bitmap
+ */
+void
+tbm_free_shared(void *pointer, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsa_data;
+
+	/*
+	 * If TBM is in iterating phase means pagetable was already created and we
+	 * have come here during tbm_free. So we need not to do anything because
+	 * by this time DSA would have been already freed.
+	 */
+	if (tbm->iterating)
+		return;
+
+	dsa_data = *((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+	dsa_free(tbm->area, dsa_data);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 3cf9417..bb262a6 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2825,6 +2825,31 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
 static int
 compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
 {
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index deb973b..4535016 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -798,6 +798,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -859,8 +860,13 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+		cpu_run_cost = update_cost_for_parallelism(path, cpu_run_cost);
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 7b43c4a..e74a717 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1557,6 +1561,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1599,6 +1608,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index c7bcd9b..25fe7b6 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2803,7 +2803,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3b7c56d..bcf85a2 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3192,7 +3193,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index f37a0bf..0f15d04 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3386,6 +3386,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index ee7e05a..d7ebac4 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -178,6 +178,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index ce13bf7..06b9883 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1416,6 +1418,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t		mutex;			/* mutual exclusion for below three fields */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+}	ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	ParallelIterator tbmiterator;
+	ParallelIterator prefetch_iterator;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	Size		ptbm_offset;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1430,6 +1498,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1446,6 +1516,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1461,7 +1532,9 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1476,6 +1549,8 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..77d0fc6 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+} TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -62,5 +79,12 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+						   ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+						  ParallelTIDBitmap *parallel_tbm);
+void		tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
+TIDBitmap *tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index d16f879..8209749 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 480f25f..0341b57 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 5b37894..5c2cc0c 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -784,6 +784,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#44Dilip Kumar
dilipbalaut@gmail.com
In reply to: Dilip Kumar (#43)
3 attachment(s)
Re: Parallel bitmap heap scan

On Fri, Jan 13, 2017 at 6:36 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

Please verify with the new patch.

Patch 0001 and 0003 required to rebase on the latest head. 0002 is
still the same.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0001-opt-parallelcost-refactoring-v11.patchapplication/octet-stream; name=0001-opt-parallelcost-refactoring-v11.patchDownload
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 46d7d06..3cf9417 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -126,6 +126,7 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static int	compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 
 
 /*
@@ -678,49 +679,7 @@ create_plain_partial_paths(PlannerInfo *root, RelOptInfo *rel)
 {
 	int			parallel_workers;
 
-	/*
-	 * If the user has set the parallel_workers reloption, use that; otherwise
-	 * select a default number of workers.
-	 */
-	if (rel->rel_parallel_workers != -1)
-		parallel_workers = rel->rel_parallel_workers;
-	else
-	{
-		int			parallel_threshold;
-
-		/*
-		 * If this relation is too small to be worth a parallel scan, just
-		 * return without doing anything ... unless it's an inheritance child.
-		 * In that case, we want to generate a parallel path here anyway.  It
-		 * might not be worthwhile just for this relation, but when combined
-		 * with all of its inheritance siblings it may well pay off.
-		 */
-		if (rel->pages < (BlockNumber) min_parallel_relation_size &&
-			rel->reloptkind == RELOPT_BASEREL)
-			return;
-
-		/*
-		 * Select the number of workers based on the log of the size of the
-		 * relation.  This probably needs to be a good deal more
-		 * sophisticated, but we need something here for now.  Note that the
-		 * upper limit of the min_parallel_relation_size GUC is chosen to
-		 * prevent overflow here.
-		 */
-		parallel_workers = 1;
-		parallel_threshold = Max(min_parallel_relation_size, 1);
-		while (rel->pages >= (BlockNumber) (parallel_threshold * 3))
-		{
-			parallel_workers++;
-			parallel_threshold *= 3;
-			if (parallel_threshold > INT_MAX / 3)
-				break;			/* avoid overflow */
-		}
-	}
-
-	/*
-	 * In no case use more than max_parallel_workers_per_gather workers.
-	 */
-	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+	parallel_workers = compute_parallel_worker(rel, rel->pages);
 
 	/* If any limit was set to zero, the user doesn't want a parallel scan. */
 	if (parallel_workers <= 0)
@@ -2866,6 +2825,59 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+static int
+compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
+{
+	int			parallel_workers;
+
+	/*
+	 * If the user has set the parallel_workers reloption, use that; otherwise
+	 * select a default number of workers.
+	 */
+	if (rel->rel_parallel_workers != -1)
+		parallel_workers = rel->rel_parallel_workers;
+	else
+	{
+		int			parallel_threshold;
+
+		/*
+		 * If this relation is too small to be worth a parallel scan, just
+		 * return without doing anything ... unless it's an inheritance child.
+		 * In that case, we want to generate a parallel path here anyway.  It
+		 * might not be worthwhile just for this relation, but when combined
+		 * with all of its inheritance siblings it may well pay off.
+		 */
+		if (pages < (BlockNumber) min_parallel_relation_size &&
+			rel->reloptkind == RELOPT_BASEREL)
+			return 0;
+
+		/*
+		 * Select the number of workers based on the log of the size of the
+		 * relation.  This probably needs to be a good deal more
+		 * sophisticated, but we need something here for now.  Note that the
+		 * upper limit of the min_parallel_relation_size GUC is chosen to
+		 * prevent overflow here.
+		 */
+		parallel_workers = 1;
+		parallel_threshold = Max(min_parallel_relation_size, 1);
+		while (pages >= (BlockNumber) (parallel_threshold * 3))
+		{
+			parallel_workers++;
+			parallel_threshold *= 3;
+			if (parallel_threshold > INT_MAX / 3)
+				break;			/* avoid overflow */
+		}
+	}
+
+	/*
+	 * In no case use more than max_parallel_workers_per_gather workers.
+	 */
+	parallel_workers = Min(parallel_workers, max_parallel_workers_per_gather);
+
+	return parallel_workers;
+}
+
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 458f139..a43daa7 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -813,7 +813,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
 	Cost		indexTotalCost;
-	Selectivity indexSelectivity;
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
@@ -837,13 +836,12 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	if (!enable_bitmapscan)
 		startup_cost += disable_cost;
 
-	/*
-	 * Fetch total cost of obtaining the bitmap, as well as its total
-	 * selectivity.
-	 */
-	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+	pages_fetched = compute_bitmap_pages(root, baserel, bitmapqual,
+										 loop_count, &indexTotalCost,
+										 &tuples_fetched);
 
 	startup_cost += indexTotalCost;
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
 
 	/* Fetch estimated page costs for tablespace containing table. */
 	get_tablespace_page_costs(baserel->reltablespace,
@@ -851,41 +849,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 							  &spc_seq_page_cost);
 
 	/*
-	 * Estimate number of main-table pages fetched.
-	 */
-	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
-
-	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
-
-	if (loop_count > 1)
-	{
-		/*
-		 * For repeated bitmap scans, scale up the number of tuples fetched in
-		 * the Mackert and Lohman formula by the number of scans, so that we
-		 * estimate the number of pages fetched by all the scans. Then
-		 * pro-rate for one scan.
-		 */
-		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
-											baserel->pages,
-											get_indexpath_pages(bitmapqual),
-											root);
-		pages_fetched /= loop_count;
-	}
-	else
-	{
-		/*
-		 * For a single scan, the number of heap pages that need to be fetched
-		 * is the same as the Mackert and Lohman formula for the case T <= b
-		 * (ie, no re-reads needed).
-		 */
-		pages_fetched = (2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
-	}
-	if (pages_fetched >= T)
-		pages_fetched = T;
-	else
-		pages_fetched = ceil(pages_fetched);
-
-	/*
 	 * For small numbers of pages we should charge spc_random_page_cost
 	 * apiece, while if nearly all the table's pages are being read, it's more
 	 * appropriate to charge spc_seq_page_cost apiece.  The effect is
@@ -4820,3 +4783,69 @@ get_parallel_divisor(Path *path)
 
 	return parallel_divisor;
 }
+
+/*
+ * compute_bitmap_pages
+ *
+ * compute number of pages fetched from heap in bitmap heap scan.
+ */
+double
+compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel, Path *bitmapqual,
+					 int loop_count, Cost *cost, double *tuple)
+{
+	Cost		indexTotalCost;
+	Selectivity indexSelectivity;
+	double		T;
+	double		pages_fetched;
+	double		tuples_fetched;
+
+	/*
+	 * Fetch total cost of obtaining the bitmap, as well as its total
+	 * selectivity.
+	 */
+	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+
+	/*
+	 * Estimate number of main-table pages fetched.
+	 */
+	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
+
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
+
+	if (loop_count > 1)
+	{
+		/*
+		 * For repeated bitmap scans, scale up the number of tuples fetched in
+		 * the Mackert and Lohman formula by the number of scans, so that we
+		 * estimate the number of pages fetched by all the scans. Then
+		 * pro-rate for one scan.
+		 */
+		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
+											baserel->pages,
+											get_indexpath_pages(bitmapqual),
+											root);
+		pages_fetched /= loop_count;
+	}
+	else
+	{
+		/*
+		 * For a single scan, the number of heap pages that need to be fetched
+		 * is the same as the Mackert and Lohman formula for the case T <= b
+		 * (ie, no re-reads needed).
+		 */
+		pages_fetched =
+			(2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
+	}
+
+	if (pages_fetched >= T)
+		pages_fetched = T;
+	else
+		pages_fetched = ceil(pages_fetched);
+
+	if (cost)
+		*cost = indexTotalCost;
+	if (tuple)
+		*tuple = tuples_fetched;
+
+	return pages_fetched;
+}
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 39376ec..0e68264 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -183,6 +183,8 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
 					   double cte_rows);
 extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
 extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern double compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel,
+				  Path *bitmapqual, int loop_count, Cost *cost, double *tuple);
 
 /*
  * prototypes for clausesel.c
0002-hash-support-alloc-free-v11.patchapplication/octet-stream; name=0002-hash-support-alloc-free-v11.patchDownload
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 47c9656..14f8c2d 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -330,7 +330,7 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
 	else
 		hashtable->hash_iv = 0;
 
-	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
+	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, NULL);
 	hashtable->hashtab->private_data = hashtable;
 
 	return hashtable;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 7b31948..0885812 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -244,7 +244,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index 12aedbc..46604fe 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -66,6 +66,7 @@
 #define SH_STATUS_EMPTY SH_MAKE_NAME(EMPTY)
 #define SH_STATUS_IN_USE SH_MAKE_NAME(IN_USE)
 #define SH_ITERATOR SH_MAKE_NAME(iterator)
+#define SH_ALLOCATOR SH_MAKE_NAME(alloc)
 
 /* function declarations */
 #define SH_CREATE SH_MAKE_NAME(create)
@@ -78,6 +79,7 @@
 #define SH_START_ITERATE_AT SH_MAKE_NAME(start_iterate_at)
 #define SH_ITERATE SH_MAKE_NAME(iterate)
 #define SH_STAT SH_MAKE_NAME(stat)
+#define SH_COPY SH_MAKE_NAME(copy)
 
 /* internal helper functions (no externally visible prototypes) */
 #define SH_COMPUTE_PARAMETERS SH_MAKE_NAME(compute_parameters)
@@ -90,6 +92,18 @@
 /* generate forward declarations necessary to use the hash table */
 #ifdef SH_DECLARE
 
+typedef struct SH_ALLOCATOR
+{
+	/* Allocation function */
+	void	   *(*HashAlloc) (Size size, void *args);
+
+	/* Free function */
+	void		(*HashFree) (void *pointer, void *args);
+
+	/* Arguments to be passed to alloc and free functions */
+	void	   *args;
+}	SH_ALLOCATOR;
+
 /* type definitions */
 typedef struct SH_TYPE
 {
@@ -112,6 +126,9 @@ typedef struct SH_TYPE
 	/* hash buckets */
 	SH_ELEMENT_TYPE *data;
 
+	/* hash allocator */
+	SH_ALLOCATOR *alloc;
+
 	/* memory context to use for allocations */
 	MemoryContext ctx;
 
@@ -133,7 +150,8 @@ typedef struct SH_ITERATOR
 } SH_ITERATOR;
 
 /* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
+							SH_ALLOCATOR *alloc);
 SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
 SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
 SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -143,6 +161,7 @@ SH_SCOPE void SH_START_ITERATE(SH_TYPE *tb, SH_ITERATOR *iter);
 SH_SCOPE void SH_START_ITERATE_AT(SH_TYPE *tb, SH_ITERATOR *iter, uint32 at);
 SH_SCOPE SH_ELEMENT_TYPE *SH_ITERATE(SH_TYPE *tb, SH_ITERATOR *iter);
 SH_SCOPE void SH_STAT(SH_TYPE *tb);
+SH_SCOPE void SH_COPY(SH_TYPE *src_tb, SH_TYPE *dst_tb, SH_ELEMENT_TYPE *data);
 
 #endif   /* SH_DECLARE */
 
@@ -276,15 +295,38 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
 #endif
 }
 
+/* default memory allocator function */
+static void *
+SH_DEFAULT_ALLOC(Size size, void *args)
+{
+	MemoryContext context = (MemoryContext) args;
+
+	return MemoryContextAllocExtended(context, size,
+									  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+}
+
+/* default memory free function */
+static void
+SH_DEFAULT_FREE(void *pointer, void *args)
+{
+	pfree(pointer);
+}
+
 /*
  * Create a hash table with enough space for `nelements` distinct members,
  * allocating required memory in the passed-in context.
+ *
+ * WARNING : alloc is an allocator handle which provides memory allocator
+ * function for storing hash elements. User needs to be aware that providing
+ * allocator does not store complete hash table in the allocator memory i.e
+ * hash table will still be allocated in local memory. However all the
+ * elements will be stored in the memory provided by the allocator.
  */
 SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements)
+SH_CREATE(MemoryContext ctx, uint32 nelements, SH_ALLOCATOR *alloc)
 {
-	SH_TYPE    *tb;
-	uint64		size;
+	SH_TYPE     *tb;
+	uint64		 size;
 
 	tb = MemoryContextAllocZero(ctx, sizeof(SH_TYPE));
 	tb->ctx = ctx;
@@ -294,9 +336,18 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 
 	SH_COMPUTE_PARAMETERS(tb, size);
 
-	tb->data = MemoryContextAllocExtended(tb->ctx,
-										  sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	if (!alloc)
+	{
+		tb->alloc = palloc(sizeof(SH_ALLOCATOR));
+		tb->alloc->HashAlloc = SH_DEFAULT_ALLOC;
+		tb->alloc->HashFree = SH_DEFAULT_FREE;
+		tb->alloc->args = tb->ctx;
+	}
+	else
+		tb->alloc = alloc;
+
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	return tb;
 }
@@ -305,7 +356,12 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 SH_SCOPE void
 SH_DESTROY(SH_TYPE *tb)
 {
-	pfree(tb->data);
+	if (tb->alloc)
+	{
+		tb->alloc->HashFree(tb->data, tb->alloc->args);
+		pfree(tb->alloc);
+	}
+
 	pfree(tb);
 }
 
@@ -333,9 +389,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 	/* compute parameters for new table */
 	SH_COMPUTE_PARAMETERS(tb, newsize);
 
-	tb->data = MemoryContextAllocExtended(
-								 tb->ctx, sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	newdata = tb->data;
 
@@ -421,7 +476,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 		}
 	}
 
-	pfree(olddata);
+	tb->alloc->HashFree(olddata, tb->alloc->args);
+
 }
 
 /*
@@ -826,6 +882,18 @@ SH_STAT(SH_TYPE *tb)
 		 total_collisions, max_collisions, avg_collisions);
 }
 
+/*
+ * Copy information from src_tb to dst_tb, it only copies those informations
+ * which are required by SH_ITERATE. Also store input data as hashtable data.
+ */
+SH_SCOPE void
+SH_COPY (SH_TYPE *src_tb, SH_TYPE *dst_tb, SH_ELEMENT_TYPE *data)
+{
+	dst_tb->size = src_tb->size;
+	dst_tb->sizemask = src_tb->sizemask;
+	dst_tb->data = data;
+}
+
 #endif   /* SH_DEFINE */
 
 
0003-parallel-bitmap-heap-scan-v11.patchapplication/octet-stream; name=0003-parallel-bitmap-heap-scan-v11.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1545f03..cf2dbbd 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1232,6 +1232,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1ce42ea..76a99a1 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index e01fe6d..090aaf3 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -205,6 +206,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -257,6 +262,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -733,6 +743,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index d5fd57a..9df6dfd 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,16 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+					  ParallelIterator *parallel_iterator);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -68,11 +74,18 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
 
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (pbminfo)
+		parallel_tbm = (ParallelTIDBitmap *) ((char *) pbminfo +
+											  pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,6 +100,35 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	prefetch_iterator = node->prefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	The first worker to see the state of ParallelBitmapInfo as PBM_INITIAL
+	 *	becomes leader and sets the state to PBM_INPROGRESS. All other workers
+	 *	see the state as PBM_INPROGRESS, and will wait for leader to finish
+	 *	building the TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *		ParallelBitmapInfo so that other worker can see those.
+	 *	  Set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader creates shared TIDBitmap.
+	 *	  Copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over page and chunk array
+	 *			and select heap pages one by one. If prefetch is enable then
+	 *			there will be two iterators.
+	 *	  b) Since multiple workers are iterating over same page and chunk
+	 *			array we need to have a shared iterator, so we grab a spin
+	 *			lock and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,10 +143,56 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.
+		 *
+		 * In parallel mode leader will immediately come out of the function,
+		 * but all other worker will be blocked until leader wake them up.
+		 */
+		if (pbminfo == NULL || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag in lower level bitmap index scan. Later
+			 * bitmap index node will use this flag to indicate tidbitmap that
+			 * it needs to create an shared page table.
+			 */
+			if (pbminfo)
+				pbms_set_parallel(outerPlanState(node));
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			/*
+			 * copy TBM local information in shared memory before waking up
+			 * other workers. Other workers will create there own TBM and copy
+			 * information from shared memory.
+			 */
+			if (pbminfo)
+			{
+				tbm_update_shared_members(tbm, parallel_tbm);
+
+				/* Change the state under a lock */
+				SpinLockAcquire(&pbminfo->state_mutex);
+				pbminfo->state = PBM_FINISHED;
+				SpinLockRelease(&pbminfo->state_mutex);
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+				/* Wake up all other workers. */
+				ConditionVariableBroadcast(&pbminfo->cv);
+			}
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM. Here we
+			 * need to create a local TBM and copy information from shared
+			 * location. We also need to attach to shared page table using
+			 * hash table handle stored in parallel_tbm (shared memory).
+			 */
+			tbm = tbm_attach(parallel_tbm, node->ss.ps.state->es_query_dsa);
+		}
 
 		node->tbm = tbm;
 		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
@@ -118,19 +206,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			node->prefetch_target = -1;
 		}
 #endif   /* USE_PREFETCH */
+
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+									 pbminfo ? &pbminfo->tbmiterator : NULL);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +229,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (pbminfo)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+							   pbminfo ? &pbminfo->prefetch_iterator : NULL);
+
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not ensure
+				 * that current blockno in main iterator and prefetch iterator
+				 * is same. It's possible that whatever blockno we are
+				 * prefetching is getting processed by some other worker.
+				 */
+				if ((pbminfo == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +300,39 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				int		   *prefetch_target;
+
+				if (pbminfo == NULL)
+					prefetch_target = &node->prefetch_target;
+				else
+				{
+					prefetch_target = &pbminfo->prefetch_target;
+					/* If we are in parallel mode then grab prefetch_mutex */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+				}
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +348,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (pbminfo == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+					pbminfo->prefetch_target++;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +387,53 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+			TBMIterator *iterator = node->prefetch_iterator;
+			ParallelIterator *parallel_iteartor = NULL;
+			ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (pbminfo)
+			{
+				parallel_iteartor = &pbminfo->prefetch_iterator;
+				prefetch_pages = &pbminfo->prefetch_pages;
+				prefetch_target = pbminfo->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
+
+			while (*prefetch_pages < prefetch_target)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+														  parallel_iteartor);
 
 				if (tbmpre == NULL)
 				{
 					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					tbm_end_iterate(iterator);
+					prefetch_iterator = node->prefetch_iterator = NULL;
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +648,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +766,7 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -653,3 +853,218 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : then we need to wait till leader creates BITMAP
+		 * PBM_FINISHED	  : BITMAP is ready so no need to wait.
+		 *---------------
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Wrapper over tbm_iterate, which take care of copying and restoring iterator
+ * variables from parallel iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator, ParallelIterator *piterator)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (piterator == NULL)
+		return tbm_iterate(iterator);
+
+	/* We are in parallel mode so grab parallel iterator mutex */
+	SpinLockAcquire(&piterator->mutex);
+
+	/*
+	 * Copy information from shared location to local iterator and call
+	 * iterate. Then restore back to shared iterator.
+	 */
+	iterator->spageptr = piterator->spageptr;
+	iterator->schunkptr = piterator->schunkptr;
+	iterator->schunkbit = piterator->schunkbit;
+	output = tbm_iterate(iterator);
+	piterator->spageptr = iterator->spageptr;
+	piterator->schunkptr = iterator->schunkptr;
+	piterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&piterator->mutex);
+
+	return output;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node as
+	 * parallel, because only first node will create the main bitmap other
+	 * bitmaps will be merged to the first bitmap so no need to create them in
+	 * shared memory.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState *) node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..56f8376 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate that we
+	 * need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_dsa);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..40b2c84 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -42,7 +42,14 @@
 
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
+#include "nodes/execnodes.h"
 #include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
+#include "storage/lwlock.h"
+#include "storage/shmem.h"
+#include "storage/spin.h"
+#include "utils/dsa.h"
+#include "utils/hsearch.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -136,27 +143,15 @@ struct TIDBitmap
 	bool		iterating;		/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
+	bool		is_shared;		/* need to build shared tbm if set */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	int			dsa_entries;	/* actual allocated entries */
+	dsa_area   *area;			/* reference to per-query shared memory area */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
 };
 
-/*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
- */
-struct TBMIterator
-{
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
-};
-
-
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
 static bool tbm_intersect_page(TIDBitmap *a, PagetableEntry *apage,
@@ -168,6 +163,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+void	   *tbm_alloc_shared(Size size, void *arg);
+void		tbm_free_shared(void *pointer, void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -198,6 +195,19 @@ hash_blockno(BlockNumber b)
 #define SH_DECLARE
 #include "lib/simplehash.h"
 
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+struct ParallelTIDBitmap
+{
+	dsa_pointer dsa_data;		/* dsa pointers for all kind of pages */
+	pagetable_hash pagetable;	/* hash table of PagetableEntry's */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	int			dsa_entries;	/* total item in dsa_pages */
+};
 
 /*
  * tbm_create - create an initially-empty bitmap
@@ -244,7 +254,18 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	if (tbm->is_shared)
+	{
+		pagetable_alloc *allocator = palloc(sizeof(pagetable_alloc));
+
+		allocator->args = tbm;
+		allocator->HashAlloc = tbm_alloc_shared;
+		allocator->HashFree = tbm_free_shared;
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, allocator);
+		tbm->dsa_entries = tbm->pagetable->size;
+	}
+	else
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -1061,3 +1082,137 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_update_shared_members
+ *
+ * Store leader's private tbm state to shared location. This must
+ * be called before waking up other workers.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before waking up the other
+	 * workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->dsa_data = tbm->dsa_data;
+	parallel_tbm->dsa_entries = tbm->dsa_entries;
+	pagetable_copy(tbm->pagetable, &parallel_tbm->pagetable, NULL);
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->is_shared = true;
+	tbm->area = (dsa_area *) area;
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ *
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size
+tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
+
+/*
+ * tbm_attach
+ *
+ * Create a local TIDBitmap for worker and Attach it to shared TID bitmap
+ * created by leader.
+ */
+TIDBitmap *
+tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area)
+{
+	TIDBitmap  *tbm = makeNode(TIDBitmap);
+	PagetableEntry *dsa_entry;
+
+	tbm->mcxt = CurrentMemoryContext;
+	tbm->status = TBM_HASH;
+	tbm->nchunks = parallel_tbm->nchunks;
+	tbm->nentries = parallel_tbm->nentries;
+	tbm->npages = parallel_tbm->npages;
+	tbm->maxentries = parallel_tbm->maxentries;
+	tbm->dsa_data = parallel_tbm->dsa_data;
+	tbm->dsa_entries = parallel_tbm->dsa_entries;
+	dsa_entry = dsa_get_address(area, tbm->dsa_data);
+	dsa_entry = (void *) (((char *) dsa_entry) + sizeof(dsa_pointer));
+	tbm->pagetable = palloc0(sizeof(struct pagetable_hash));
+	pagetable_copy(&parallel_tbm->pagetable, tbm->pagetable, dsa_entry);
+
+	/* Mark the tbm as parallel and also keep the DSA reference in it. */
+	tbm_set_parallel(tbm, area);
+
+	return tbm;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Memory allocation function for bitmap hash.
+ * It allocates memory from DSA and also stores dsa_pointer in the memory
+ * so that during free we can directly get the dsa_pointe and free it.
+ */
+void *
+tbm_alloc_shared(Size size, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->area, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	/* Keep track of actual number of entries */
+	if (tbm->pagetable)
+		tbm->dsa_entries = tbm->pagetable->size;
+
+	ptr = dsa_get_address(tbm->area, dsaptr);
+	memset(ptr, 0, size + sizeof(dsa_pointer));
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * tbm_free_shared
+ *
+ * Memory free function for tid bitmap
+ */
+void
+tbm_free_shared(void *pointer, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsa_data;
+
+	/*
+	 * If TBM is in iterating phase means pagetable was already created and we
+	 * have come here during tbm_free. So we need not to do anything because
+	 * by this time DSA would have been already freed.
+	 */
+	if (tbm->iterating)
+		return;
+
+	dsa_data = *((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+	dsa_free(tbm->area, dsa_data);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 3cf9417..bb262a6 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2825,6 +2825,31 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
 static int
 compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
 {
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a43daa7..f81e469 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -816,6 +816,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -877,8 +878,32 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
+
+		/*
+		 * It may be possible to amortize some of the I/O cost, but probably
+		 * not very much, because most operating systems already do aggressive
+		 * prefetching.  For now, we assume that the disk run cost can't be
+		 * amortized at all.
+		 */
+
+		/*
+		 * In the case of a parallel plan, the row count needs to represent
+		 * the number of tuples processed per worker.
+		 */
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 0a5c050..70b7c17 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1557,6 +1561,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1599,6 +1608,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index c7bcd9b..25fe7b6 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2803,7 +2803,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3b7c56d..bcf85a2 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3192,7 +3193,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index f37a0bf..0f15d04 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3386,6 +3386,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index ee7e05a..d7ebac4 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -178,6 +178,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index ce13bf7..06b9883 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1416,6 +1418,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t		mutex;			/* mutual exclusion for below three fields */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+}	ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	ParallelIterator tbmiterator;
+	ParallelIterator prefetch_iterator;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	Size		ptbm_offset;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1430,6 +1498,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1446,6 +1516,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1461,7 +1532,9 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1476,6 +1549,8 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..77d0fc6 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+} TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -62,5 +79,12 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+						   ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+						  ParallelTIDBitmap *parallel_tbm);
+void		tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
+TIDBitmap *tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index d16f879..8209749 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 81a9be7..968d39a 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 5b37894..5c2cc0c 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -784,6 +784,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#45Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#44)
Re: Parallel bitmap heap scan

On Wed, Jan 18, 2017 at 12:14 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Fri, Jan 13, 2017 at 6:36 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

Please verify with the new patch.

Patch 0001 and 0003 required to rebase on the latest head. 0002 is
still the same.

I've committed the first half of 0001.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#46Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#45)
3 attachment(s)
Re: Parallel bitmap heap scan

On Thu, Jan 19, 2017 at 12:26 AM, Robert Haas <robertmhaas@gmail.com> wrote:

Patch 0001 and 0003 required to rebase on the latest head. 0002 is
still the same.

I've committed the first half of 0001.

Thanks. 0001 and 0003 required rebasing after this commit.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0001-opt-parallelcost-refactoring-v12.patchapplication/octet-stream; name=0001-opt-parallelcost-refactoring-v12.patchDownload
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 458f139..a43daa7 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -813,7 +813,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
 	Cost		indexTotalCost;
-	Selectivity indexSelectivity;
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
@@ -837,13 +836,12 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	if (!enable_bitmapscan)
 		startup_cost += disable_cost;
 
-	/*
-	 * Fetch total cost of obtaining the bitmap, as well as its total
-	 * selectivity.
-	 */
-	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+	pages_fetched = compute_bitmap_pages(root, baserel, bitmapqual,
+										 loop_count, &indexTotalCost,
+										 &tuples_fetched);
 
 	startup_cost += indexTotalCost;
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
 
 	/* Fetch estimated page costs for tablespace containing table. */
 	get_tablespace_page_costs(baserel->reltablespace,
@@ -851,41 +849,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 							  &spc_seq_page_cost);
 
 	/*
-	 * Estimate number of main-table pages fetched.
-	 */
-	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
-
-	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
-
-	if (loop_count > 1)
-	{
-		/*
-		 * For repeated bitmap scans, scale up the number of tuples fetched in
-		 * the Mackert and Lohman formula by the number of scans, so that we
-		 * estimate the number of pages fetched by all the scans. Then
-		 * pro-rate for one scan.
-		 */
-		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
-											baserel->pages,
-											get_indexpath_pages(bitmapqual),
-											root);
-		pages_fetched /= loop_count;
-	}
-	else
-	{
-		/*
-		 * For a single scan, the number of heap pages that need to be fetched
-		 * is the same as the Mackert and Lohman formula for the case T <= b
-		 * (ie, no re-reads needed).
-		 */
-		pages_fetched = (2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
-	}
-	if (pages_fetched >= T)
-		pages_fetched = T;
-	else
-		pages_fetched = ceil(pages_fetched);
-
-	/*
 	 * For small numbers of pages we should charge spc_random_page_cost
 	 * apiece, while if nearly all the table's pages are being read, it's more
 	 * appropriate to charge spc_seq_page_cost apiece.  The effect is
@@ -4820,3 +4783,69 @@ get_parallel_divisor(Path *path)
 
 	return parallel_divisor;
 }
+
+/*
+ * compute_bitmap_pages
+ *
+ * compute number of pages fetched from heap in bitmap heap scan.
+ */
+double
+compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel, Path *bitmapqual,
+					 int loop_count, Cost *cost, double *tuple)
+{
+	Cost		indexTotalCost;
+	Selectivity indexSelectivity;
+	double		T;
+	double		pages_fetched;
+	double		tuples_fetched;
+
+	/*
+	 * Fetch total cost of obtaining the bitmap, as well as its total
+	 * selectivity.
+	 */
+	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+
+	/*
+	 * Estimate number of main-table pages fetched.
+	 */
+	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
+
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
+
+	if (loop_count > 1)
+	{
+		/*
+		 * For repeated bitmap scans, scale up the number of tuples fetched in
+		 * the Mackert and Lohman formula by the number of scans, so that we
+		 * estimate the number of pages fetched by all the scans. Then
+		 * pro-rate for one scan.
+		 */
+		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
+											baserel->pages,
+											get_indexpath_pages(bitmapqual),
+											root);
+		pages_fetched /= loop_count;
+	}
+	else
+	{
+		/*
+		 * For a single scan, the number of heap pages that need to be fetched
+		 * is the same as the Mackert and Lohman formula for the case T <= b
+		 * (ie, no re-reads needed).
+		 */
+		pages_fetched =
+			(2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
+	}
+
+	if (pages_fetched >= T)
+		pages_fetched = T;
+	else
+		pages_fetched = ceil(pages_fetched);
+
+	if (cost)
+		*cost = indexTotalCost;
+	if (tuple)
+		*tuple = tuples_fetched;
+
+	return pages_fetched;
+}
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 39376ec..0e68264 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -183,6 +183,8 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
 					   double cte_rows);
 extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
 extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern double compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel,
+				  Path *bitmapqual, int loop_count, Cost *cost, double *tuple);
 
 /*
  * prototypes for clausesel.c
0002-hash-support-alloc-free-v12.patchapplication/octet-stream; name=0002-hash-support-alloc-free-v12.patchDownload
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 47c9656..14f8c2d 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -330,7 +330,7 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
 	else
 		hashtable->hash_iv = 0;
 
-	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
+	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, NULL);
 	hashtable->hashtab->private_data = hashtable;
 
 	return hashtable;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 7b31948..0885812 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -244,7 +244,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index 12aedbc..46604fe 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -66,6 +66,7 @@
 #define SH_STATUS_EMPTY SH_MAKE_NAME(EMPTY)
 #define SH_STATUS_IN_USE SH_MAKE_NAME(IN_USE)
 #define SH_ITERATOR SH_MAKE_NAME(iterator)
+#define SH_ALLOCATOR SH_MAKE_NAME(alloc)
 
 /* function declarations */
 #define SH_CREATE SH_MAKE_NAME(create)
@@ -78,6 +79,7 @@
 #define SH_START_ITERATE_AT SH_MAKE_NAME(start_iterate_at)
 #define SH_ITERATE SH_MAKE_NAME(iterate)
 #define SH_STAT SH_MAKE_NAME(stat)
+#define SH_COPY SH_MAKE_NAME(copy)
 
 /* internal helper functions (no externally visible prototypes) */
 #define SH_COMPUTE_PARAMETERS SH_MAKE_NAME(compute_parameters)
@@ -90,6 +92,18 @@
 /* generate forward declarations necessary to use the hash table */
 #ifdef SH_DECLARE
 
+typedef struct SH_ALLOCATOR
+{
+	/* Allocation function */
+	void	   *(*HashAlloc) (Size size, void *args);
+
+	/* Free function */
+	void		(*HashFree) (void *pointer, void *args);
+
+	/* Arguments to be passed to alloc and free functions */
+	void	   *args;
+}	SH_ALLOCATOR;
+
 /* type definitions */
 typedef struct SH_TYPE
 {
@@ -112,6 +126,9 @@ typedef struct SH_TYPE
 	/* hash buckets */
 	SH_ELEMENT_TYPE *data;
 
+	/* hash allocator */
+	SH_ALLOCATOR *alloc;
+
 	/* memory context to use for allocations */
 	MemoryContext ctx;
 
@@ -133,7 +150,8 @@ typedef struct SH_ITERATOR
 } SH_ITERATOR;
 
 /* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
+							SH_ALLOCATOR *alloc);
 SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
 SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
 SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -143,6 +161,7 @@ SH_SCOPE void SH_START_ITERATE(SH_TYPE *tb, SH_ITERATOR *iter);
 SH_SCOPE void SH_START_ITERATE_AT(SH_TYPE *tb, SH_ITERATOR *iter, uint32 at);
 SH_SCOPE SH_ELEMENT_TYPE *SH_ITERATE(SH_TYPE *tb, SH_ITERATOR *iter);
 SH_SCOPE void SH_STAT(SH_TYPE *tb);
+SH_SCOPE void SH_COPY(SH_TYPE *src_tb, SH_TYPE *dst_tb, SH_ELEMENT_TYPE *data);
 
 #endif   /* SH_DECLARE */
 
@@ -276,15 +295,38 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
 #endif
 }
 
+/* default memory allocator function */
+static void *
+SH_DEFAULT_ALLOC(Size size, void *args)
+{
+	MemoryContext context = (MemoryContext) args;
+
+	return MemoryContextAllocExtended(context, size,
+									  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+}
+
+/* default memory free function */
+static void
+SH_DEFAULT_FREE(void *pointer, void *args)
+{
+	pfree(pointer);
+}
+
 /*
  * Create a hash table with enough space for `nelements` distinct members,
  * allocating required memory in the passed-in context.
+ *
+ * WARNING : alloc is an allocator handle which provides memory allocator
+ * function for storing hash elements. User needs to be aware that providing
+ * allocator does not store complete hash table in the allocator memory i.e
+ * hash table will still be allocated in local memory. However all the
+ * elements will be stored in the memory provided by the allocator.
  */
 SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements)
+SH_CREATE(MemoryContext ctx, uint32 nelements, SH_ALLOCATOR *alloc)
 {
-	SH_TYPE    *tb;
-	uint64		size;
+	SH_TYPE     *tb;
+	uint64		 size;
 
 	tb = MemoryContextAllocZero(ctx, sizeof(SH_TYPE));
 	tb->ctx = ctx;
@@ -294,9 +336,18 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 
 	SH_COMPUTE_PARAMETERS(tb, size);
 
-	tb->data = MemoryContextAllocExtended(tb->ctx,
-										  sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	if (!alloc)
+	{
+		tb->alloc = palloc(sizeof(SH_ALLOCATOR));
+		tb->alloc->HashAlloc = SH_DEFAULT_ALLOC;
+		tb->alloc->HashFree = SH_DEFAULT_FREE;
+		tb->alloc->args = tb->ctx;
+	}
+	else
+		tb->alloc = alloc;
+
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	return tb;
 }
@@ -305,7 +356,12 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 SH_SCOPE void
 SH_DESTROY(SH_TYPE *tb)
 {
-	pfree(tb->data);
+	if (tb->alloc)
+	{
+		tb->alloc->HashFree(tb->data, tb->alloc->args);
+		pfree(tb->alloc);
+	}
+
 	pfree(tb);
 }
 
@@ -333,9 +389,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 	/* compute parameters for new table */
 	SH_COMPUTE_PARAMETERS(tb, newsize);
 
-	tb->data = MemoryContextAllocExtended(
-								 tb->ctx, sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	newdata = tb->data;
 
@@ -421,7 +476,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 		}
 	}
 
-	pfree(olddata);
+	tb->alloc->HashFree(olddata, tb->alloc->args);
+
 }
 
 /*
@@ -826,6 +882,18 @@ SH_STAT(SH_TYPE *tb)
 		 total_collisions, max_collisions, avg_collisions);
 }
 
+/*
+ * Copy information from src_tb to dst_tb, it only copies those informations
+ * which are required by SH_ITERATE. Also store input data as hashtable data.
+ */
+SH_SCOPE void
+SH_COPY (SH_TYPE *src_tb, SH_TYPE *dst_tb, SH_ELEMENT_TYPE *data)
+{
+	dst_tb->size = src_tb->size;
+	dst_tb->sizemask = src_tb->sizemask;
+	dst_tb->data = data;
+}
+
 #endif   /* SH_DEFINE */
 
 
0003-parallel-bitmap-heap-scan-v12.patchapplication/octet-stream; name=0003-parallel-bitmap-heap-scan-v12.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 01fad38..2af778a 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1240,6 +1240,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1ce42ea..76a99a1 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index e01fe6d..090aaf3 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -205,6 +206,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -257,6 +262,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -733,6 +743,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index f18827d..48d4186 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,16 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+					  ParallelIterator *parallel_iterator);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -68,11 +74,18 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
 
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (pbminfo)
+		parallel_tbm = (ParallelTIDBitmap *) ((char *) pbminfo +
+											  pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,6 +100,35 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	prefetch_iterator = node->prefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	The first worker to see the state of ParallelBitmapInfo as PBM_INITIAL
+	 *	becomes leader and sets the state to PBM_INPROGRESS. All other workers
+	 *	see the state as PBM_INPROGRESS, and will wait for leader to finish
+	 *	building the TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *		ParallelBitmapInfo so that other worker can see those.
+	 *	  Set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader creates shared TIDBitmap.
+	 *	  Copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over page and chunk array
+	 *			and select heap pages one by one. If prefetch is enable then
+	 *			there will be two iterators.
+	 *	  b) Since multiple workers are iterating over same page and chunk
+	 *			array we need to have a shared iterator, so we grab a spin
+	 *			lock and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,10 +143,56 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.
+		 *
+		 * In parallel mode leader will immediately come out of the function,
+		 * but all other worker will be blocked until leader wake them up.
+		 */
+		if (pbminfo == NULL || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag in lower level bitmap index scan. Later
+			 * bitmap index node will use this flag to indicate tidbitmap that
+			 * it needs to create an shared page table.
+			 */
+			if (pbminfo)
+				pbms_set_parallel(outerPlanState(node));
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			/*
+			 * copy TBM local information in shared memory before waking up
+			 * other workers. Other workers will create there own TBM and copy
+			 * information from shared memory.
+			 */
+			if (pbminfo)
+			{
+				tbm_update_shared_members(tbm, parallel_tbm);
+
+				/* Change the state under a lock */
+				SpinLockAcquire(&pbminfo->state_mutex);
+				pbminfo->state = PBM_FINISHED;
+				SpinLockRelease(&pbminfo->state_mutex);
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+				/* Wake up all other workers. */
+				ConditionVariableBroadcast(&pbminfo->cv);
+			}
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM. Here we
+			 * need to create a local TBM and copy information from shared
+			 * location. We also need to attach to shared page table using
+			 * hash table handle stored in parallel_tbm (shared memory).
+			 */
+			tbm = tbm_attach(parallel_tbm, node->ss.ps.state->es_query_dsa);
+		}
 
 		node->tbm = tbm;
 		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
@@ -118,19 +206,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			node->prefetch_target = -1;
 		}
 #endif   /* USE_PREFETCH */
+
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+									 pbminfo ? &pbminfo->tbmiterator : NULL);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +229,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (pbminfo)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+							   pbminfo ? &pbminfo->prefetch_iterator : NULL);
+
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not ensure
+				 * that current blockno in main iterator and prefetch iterator
+				 * is same. It's possible that whatever blockno we are
+				 * prefetching is getting processed by some other worker.
+				 */
+				if ((pbminfo == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +300,39 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				int		   *prefetch_target;
+
+				if (pbminfo == NULL)
+					prefetch_target = &node->prefetch_target;
+				else
+				{
+					prefetch_target = &pbminfo->prefetch_target;
+					/* If we are in parallel mode then grab prefetch_mutex */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+				}
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +348,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (pbminfo == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+					pbminfo->prefetch_target++;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +387,53 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+			TBMIterator *iterator = node->prefetch_iterator;
+			ParallelIterator *parallel_iteartor = NULL;
+			ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (pbminfo)
+			{
+				parallel_iteartor = &pbminfo->prefetch_iterator;
+				prefetch_pages = &pbminfo->prefetch_pages;
+				prefetch_target = pbminfo->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
+
+			while (*prefetch_pages < prefetch_target)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+														  parallel_iteartor);
 
 				if (tbmpre == NULL)
 				{
 					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					tbm_end_iterate(iterator);
+					prefetch_iterator = node->prefetch_iterator = NULL;
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +648,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +766,7 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -651,3 +851,218 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : then we need to wait till leader creates BITMAP
+		 * PBM_FINISHED	  : BITMAP is ready so no need to wait.
+		 *---------------
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Wrapper over tbm_iterate, which take care of copying and restoring iterator
+ * variables from parallel iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator, ParallelIterator *piterator)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (piterator == NULL)
+		return tbm_iterate(iterator);
+
+	/* We are in parallel mode so grab parallel iterator mutex */
+	SpinLockAcquire(&piterator->mutex);
+
+	/*
+	 * Copy information from shared location to local iterator and call
+	 * iterate. Then restore back to shared iterator.
+	 */
+	iterator->spageptr = piterator->spageptr;
+	iterator->schunkptr = piterator->schunkptr;
+	iterator->schunkbit = piterator->schunkbit;
+	output = tbm_iterate(iterator);
+	piterator->spageptr = iterator->spageptr;
+	piterator->schunkptr = iterator->schunkptr;
+	piterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&piterator->mutex);
+
+	return output;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node as
+	 * parallel, because only first node will create the main bitmap other
+	 * bitmaps will be merged to the first bitmap so no need to create them in
+	 * shared memory.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState *) node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..56f8376 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate that we
+	 * need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_dsa);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..40b2c84 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -42,7 +42,14 @@
 
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
+#include "nodes/execnodes.h"
 #include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
+#include "storage/lwlock.h"
+#include "storage/shmem.h"
+#include "storage/spin.h"
+#include "utils/dsa.h"
+#include "utils/hsearch.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -136,27 +143,15 @@ struct TIDBitmap
 	bool		iterating;		/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
+	bool		is_shared;		/* need to build shared tbm if set */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	int			dsa_entries;	/* actual allocated entries */
+	dsa_area   *area;			/* reference to per-query shared memory area */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
 };
 
-/*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
- */
-struct TBMIterator
-{
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
-};
-
-
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
 static bool tbm_intersect_page(TIDBitmap *a, PagetableEntry *apage,
@@ -168,6 +163,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+void	   *tbm_alloc_shared(Size size, void *arg);
+void		tbm_free_shared(void *pointer, void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -198,6 +195,19 @@ hash_blockno(BlockNumber b)
 #define SH_DECLARE
 #include "lib/simplehash.h"
 
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+struct ParallelTIDBitmap
+{
+	dsa_pointer dsa_data;		/* dsa pointers for all kind of pages */
+	pagetable_hash pagetable;	/* hash table of PagetableEntry's */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	int			dsa_entries;	/* total item in dsa_pages */
+};
 
 /*
  * tbm_create - create an initially-empty bitmap
@@ -244,7 +254,18 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	if (tbm->is_shared)
+	{
+		pagetable_alloc *allocator = palloc(sizeof(pagetable_alloc));
+
+		allocator->args = tbm;
+		allocator->HashAlloc = tbm_alloc_shared;
+		allocator->HashFree = tbm_free_shared;
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, allocator);
+		tbm->dsa_entries = tbm->pagetable->size;
+	}
+	else
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -1061,3 +1082,137 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_update_shared_members
+ *
+ * Store leader's private tbm state to shared location. This must
+ * be called before waking up other workers.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before waking up the other
+	 * workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->dsa_data = tbm->dsa_data;
+	parallel_tbm->dsa_entries = tbm->dsa_entries;
+	pagetable_copy(tbm->pagetable, &parallel_tbm->pagetable, NULL);
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->is_shared = true;
+	tbm->area = (dsa_area *) area;
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ *
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size
+tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
+
+/*
+ * tbm_attach
+ *
+ * Create a local TIDBitmap for worker and Attach it to shared TID bitmap
+ * created by leader.
+ */
+TIDBitmap *
+tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area)
+{
+	TIDBitmap  *tbm = makeNode(TIDBitmap);
+	PagetableEntry *dsa_entry;
+
+	tbm->mcxt = CurrentMemoryContext;
+	tbm->status = TBM_HASH;
+	tbm->nchunks = parallel_tbm->nchunks;
+	tbm->nentries = parallel_tbm->nentries;
+	tbm->npages = parallel_tbm->npages;
+	tbm->maxentries = parallel_tbm->maxentries;
+	tbm->dsa_data = parallel_tbm->dsa_data;
+	tbm->dsa_entries = parallel_tbm->dsa_entries;
+	dsa_entry = dsa_get_address(area, tbm->dsa_data);
+	dsa_entry = (void *) (((char *) dsa_entry) + sizeof(dsa_pointer));
+	tbm->pagetable = palloc0(sizeof(struct pagetable_hash));
+	pagetable_copy(&parallel_tbm->pagetable, tbm->pagetable, dsa_entry);
+
+	/* Mark the tbm as parallel and also keep the DSA reference in it. */
+	tbm_set_parallel(tbm, area);
+
+	return tbm;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Memory allocation function for bitmap hash.
+ * It allocates memory from DSA and also stores dsa_pointer in the memory
+ * so that during free we can directly get the dsa_pointe and free it.
+ */
+void *
+tbm_alloc_shared(Size size, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->area, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	/* Keep track of actual number of entries */
+	if (tbm->pagetable)
+		tbm->dsa_entries = tbm->pagetable->size;
+
+	ptr = dsa_get_address(tbm->area, dsaptr);
+	memset(ptr, 0, size + sizeof(dsa_pointer));
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * tbm_free_shared
+ *
+ * Memory free function for tid bitmap
+ */
+void
+tbm_free_shared(void *pointer, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsa_data;
+
+	/*
+	 * If TBM is in iterating phase means pagetable was already created and we
+	 * have come here during tbm_free. So we need not to do anything because
+	 * by this time DSA would have been already freed.
+	 */
+	if (tbm->iterating)
+		return;
+
+	dsa_data = *((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+	dsa_free(tbm->area, dsa_data);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 5c18987..a2f6fbb 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2875,6 +2875,31 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 }
 
 /*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+/*
  * Compute the number of parallel workers that should be used to scan a
  * relation.  "pages" is the number of pages from the relation that we
  * expect to scan.
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a43daa7..f81e469 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -816,6 +816,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -877,8 +878,32 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
+
+		/*
+		 * It may be possible to amortize some of the I/O cost, but probably
+		 * not very much, because most operating systems already do aggressive
+		 * prefetching.  For now, we assume that the disk run cost can't be
+		 * amortized at all.
+		 */
+
+		/*
+		 * In the case of a parallel plan, the row count needs to represent
+		 * the number of tuples processed per worker.
+		 */
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 5283468..630c501 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1557,6 +1561,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1599,6 +1608,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index fae1f67..221b5f1 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2834,7 +2834,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f440875..20b9a59 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3258,7 +3259,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 7176cf1..f8eef0b 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3392,6 +3392,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index ee7e05a..d7ebac4 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -178,6 +178,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index f9bcdd6..384a724 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1423,6 +1425,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t		mutex;			/* mutual exclusion for below three fields */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+}	ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	ParallelIterator tbmiterator;
+	ParallelIterator prefetch_iterator;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	Size		ptbm_offset;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1437,6 +1505,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1453,6 +1523,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1468,7 +1539,9 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1483,6 +1556,8 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..77d0fc6 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+} TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -62,5 +79,12 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+						   ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+						  ParallelTIDBitmap *parallel_tbm);
+void		tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
+TIDBitmap *tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 7b41317..cfef818 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 81a9be7..968d39a 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index de8225b..c52c864 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -786,6 +786,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#47Haribabu Kommi
kommi.haribabu@gmail.com
In reply to: Dilip Kumar (#46)
Re: Parallel bitmap heap scan

On Mon, Jan 23, 2017 at 3:42 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Thu, Jan 19, 2017 at 12:26 AM, Robert Haas <robertmhaas@gmail.com>
wrote:

Patch 0001 and 0003 required to rebase on the latest head. 0002 is
still the same.

I've committed the first half of 0001.

Thanks. 0001 and 0003 required rebasing after this commit.

I reviewed 0002-hash-support-alloc-free-v12.patch, some minor comments.

- SH_TYPE    *tb;
- uint64 size;
+ SH_TYPE     *tb;
+ uint64 size;

The above change may not be required.

+ if (tb->alloc)
+ {
+ tb->alloc->HashFree(tb->data, tb->alloc->args);
+ pfree(tb->alloc);
+ }

The above code tries to free the tb->alloc memory. In case if the user
has provide the alloc structure to SH_CREATE function and the same
pointer is stored in the tb structure. And in free function freeing that
memory may cause problem.

So either explicitly mentioning that the input must a palloc'ed data or
by default allocate memory and copy the input data into allocated
memory.

Regards,
Hari Babu
Fujitsu Australia

#48Dilip Kumar
dilipbalaut@gmail.com
In reply to: Haribabu Kommi (#47)
3 attachment(s)
Re: Parallel bitmap heap scan

On Mon, Jan 23, 2017 at 1:52 PM, Haribabu Kommi
<kommi.haribabu@gmail.com> wrote:

I reviewed 0002-hash-support-alloc-free-v12.patch, some minor comments.

- SH_TYPE    *tb;
- uint64 size;
+ SH_TYPE     *tb;
+ uint64 size;

The above change may not be required.

+ if (tb->alloc)
+ {
+ tb->alloc->HashFree(tb->data, tb->alloc->args);
+ pfree(tb->alloc);
+ }

The above code tries to free the tb->alloc memory. In case if the user
has provide the alloc structure to SH_CREATE function and the same
pointer is stored in the tb structure. And in free function freeing that
memory may cause problem.

So either explicitly mentioning that the input must a palloc'ed data or
by default allocate memory and copy the input data into allocated
memory.

I have changed as per the comments. 0002 and 0003 are changed, 0001 is
still the same.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0001-opt-parallelcost-refactoring-v13.patchapplication/octet-stream; name=0001-opt-parallelcost-refactoring-v13.patchDownload
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 458f139..a43daa7 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -813,7 +813,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
 	Cost		indexTotalCost;
-	Selectivity indexSelectivity;
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
@@ -837,13 +836,12 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	if (!enable_bitmapscan)
 		startup_cost += disable_cost;
 
-	/*
-	 * Fetch total cost of obtaining the bitmap, as well as its total
-	 * selectivity.
-	 */
-	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+	pages_fetched = compute_bitmap_pages(root, baserel, bitmapqual,
+										 loop_count, &indexTotalCost,
+										 &tuples_fetched);
 
 	startup_cost += indexTotalCost;
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
 
 	/* Fetch estimated page costs for tablespace containing table. */
 	get_tablespace_page_costs(baserel->reltablespace,
@@ -851,41 +849,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 							  &spc_seq_page_cost);
 
 	/*
-	 * Estimate number of main-table pages fetched.
-	 */
-	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
-
-	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
-
-	if (loop_count > 1)
-	{
-		/*
-		 * For repeated bitmap scans, scale up the number of tuples fetched in
-		 * the Mackert and Lohman formula by the number of scans, so that we
-		 * estimate the number of pages fetched by all the scans. Then
-		 * pro-rate for one scan.
-		 */
-		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
-											baserel->pages,
-											get_indexpath_pages(bitmapqual),
-											root);
-		pages_fetched /= loop_count;
-	}
-	else
-	{
-		/*
-		 * For a single scan, the number of heap pages that need to be fetched
-		 * is the same as the Mackert and Lohman formula for the case T <= b
-		 * (ie, no re-reads needed).
-		 */
-		pages_fetched = (2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
-	}
-	if (pages_fetched >= T)
-		pages_fetched = T;
-	else
-		pages_fetched = ceil(pages_fetched);
-
-	/*
 	 * For small numbers of pages we should charge spc_random_page_cost
 	 * apiece, while if nearly all the table's pages are being read, it's more
 	 * appropriate to charge spc_seq_page_cost apiece.  The effect is
@@ -4820,3 +4783,69 @@ get_parallel_divisor(Path *path)
 
 	return parallel_divisor;
 }
+
+/*
+ * compute_bitmap_pages
+ *
+ * compute number of pages fetched from heap in bitmap heap scan.
+ */
+double
+compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel, Path *bitmapqual,
+					 int loop_count, Cost *cost, double *tuple)
+{
+	Cost		indexTotalCost;
+	Selectivity indexSelectivity;
+	double		T;
+	double		pages_fetched;
+	double		tuples_fetched;
+
+	/*
+	 * Fetch total cost of obtaining the bitmap, as well as its total
+	 * selectivity.
+	 */
+	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+
+	/*
+	 * Estimate number of main-table pages fetched.
+	 */
+	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
+
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
+
+	if (loop_count > 1)
+	{
+		/*
+		 * For repeated bitmap scans, scale up the number of tuples fetched in
+		 * the Mackert and Lohman formula by the number of scans, so that we
+		 * estimate the number of pages fetched by all the scans. Then
+		 * pro-rate for one scan.
+		 */
+		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
+											baserel->pages,
+											get_indexpath_pages(bitmapqual),
+											root);
+		pages_fetched /= loop_count;
+	}
+	else
+	{
+		/*
+		 * For a single scan, the number of heap pages that need to be fetched
+		 * is the same as the Mackert and Lohman formula for the case T <= b
+		 * (ie, no re-reads needed).
+		 */
+		pages_fetched =
+			(2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
+	}
+
+	if (pages_fetched >= T)
+		pages_fetched = T;
+	else
+		pages_fetched = ceil(pages_fetched);
+
+	if (cost)
+		*cost = indexTotalCost;
+	if (tuple)
+		*tuple = tuples_fetched;
+
+	return pages_fetched;
+}
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 39376ec..0e68264 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -183,6 +183,8 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
 					   double cte_rows);
 extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
 extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern double compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel,
+				  Path *bitmapqual, int loop_count, Cost *cost, double *tuple);
 
 /*
  * prototypes for clausesel.c
0002-hash-support-alloc-free-v13.patchapplication/octet-stream; name=0002-hash-support-alloc-free-v13.patchDownload
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 47c9656..14f8c2d 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -330,7 +330,7 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
 	else
 		hashtable->hash_iv = 0;
 
-	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
+	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, NULL);
 	hashtable->hashtab->private_data = hashtable;
 
 	return hashtable;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 7b31948..0885812 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -244,7 +244,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index 12aedbc..e2c8f44 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -66,6 +66,7 @@
 #define SH_STATUS_EMPTY SH_MAKE_NAME(EMPTY)
 #define SH_STATUS_IN_USE SH_MAKE_NAME(IN_USE)
 #define SH_ITERATOR SH_MAKE_NAME(iterator)
+#define SH_ALLOCATOR SH_MAKE_NAME(alloc)
 
 /* function declarations */
 #define SH_CREATE SH_MAKE_NAME(create)
@@ -78,6 +79,7 @@
 #define SH_START_ITERATE_AT SH_MAKE_NAME(start_iterate_at)
 #define SH_ITERATE SH_MAKE_NAME(iterate)
 #define SH_STAT SH_MAKE_NAME(stat)
+#define SH_COPY SH_MAKE_NAME(copy)
 
 /* internal helper functions (no externally visible prototypes) */
 #define SH_COMPUTE_PARAMETERS SH_MAKE_NAME(compute_parameters)
@@ -90,6 +92,18 @@
 /* generate forward declarations necessary to use the hash table */
 #ifdef SH_DECLARE
 
+typedef struct SH_ALLOCATOR
+{
+	/* Allocation function */
+	void	   *(*HashAlloc) (Size size, void *args);
+
+	/* Free function */
+	void		(*HashFree) (void *pointer, void *args);
+
+	/* Arguments to be passed to alloc and free functions */
+	void	   *args;
+}	SH_ALLOCATOR;
+
 /* type definitions */
 typedef struct SH_TYPE
 {
@@ -112,6 +126,9 @@ typedef struct SH_TYPE
 	/* hash buckets */
 	SH_ELEMENT_TYPE *data;
 
+	/* hash allocator */
+	SH_ALLOCATOR *alloc;
+
 	/* memory context to use for allocations */
 	MemoryContext ctx;
 
@@ -133,7 +150,8 @@ typedef struct SH_ITERATOR
 } SH_ITERATOR;
 
 /* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
+							SH_ALLOCATOR *alloc);
 SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
 SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
 SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -143,6 +161,7 @@ SH_SCOPE void SH_START_ITERATE(SH_TYPE *tb, SH_ITERATOR *iter);
 SH_SCOPE void SH_START_ITERATE_AT(SH_TYPE *tb, SH_ITERATOR *iter, uint32 at);
 SH_SCOPE SH_ELEMENT_TYPE *SH_ITERATE(SH_TYPE *tb, SH_ITERATOR *iter);
 SH_SCOPE void SH_STAT(SH_TYPE *tb);
+SH_SCOPE void SH_COPY(SH_TYPE *src_tb, SH_TYPE *dst_tb, SH_ELEMENT_TYPE *data);
 
 #endif   /* SH_DECLARE */
 
@@ -276,15 +295,38 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
 #endif
 }
 
+/* default memory allocator function */
+static void *
+SH_DEFAULT_ALLOC(Size size, void *args)
+{
+	MemoryContext context = (MemoryContext) args;
+
+	return MemoryContextAllocExtended(context, size,
+									  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+}
+
+/* default memory free function */
+static void
+SH_DEFAULT_FREE(void *pointer, void *args)
+{
+	pfree(pointer);
+}
+
 /*
  * Create a hash table with enough space for `nelements` distinct members,
  * allocating required memory in the passed-in context.
+ *
+ * WARNING : alloc is an allocator handle which provides memory allocator
+ * function for storing hash elements. User needs to be aware that providing
+ * allocator does not store complete hash table in the allocator memory i.e
+ * hash table will still be allocated in local memory. However all the
+ * elements will be stored in the memory provided by the allocator.
  */
 SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements)
+SH_CREATE(MemoryContext ctx, uint32 nelements, SH_ALLOCATOR *alloc)
 {
-	SH_TYPE    *tb;
-	uint64		size;
+	SH_TYPE     *tb;
+	uint64		 size;
 
 	tb = MemoryContextAllocZero(ctx, sizeof(SH_TYPE));
 	tb->ctx = ctx;
@@ -294,9 +336,19 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 
 	SH_COMPUTE_PARAMETERS(tb, size);
 
-	tb->data = MemoryContextAllocExtended(tb->ctx,
-										  sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	tb->alloc = palloc(sizeof(SH_ALLOCATOR));
+
+	if (!alloc)
+	{
+		tb->alloc->HashAlloc = SH_DEFAULT_ALLOC;
+		tb->alloc->HashFree = SH_DEFAULT_FREE;
+		tb->alloc->args = tb->ctx;
+	}
+	else
+		memcpy(tb->alloc, alloc, sizeof(SH_ALLOCATOR));
+
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	return tb;
 }
@@ -305,7 +357,12 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 SH_SCOPE void
 SH_DESTROY(SH_TYPE *tb)
 {
-	pfree(tb->data);
+	if (tb->alloc)
+	{
+		tb->alloc->HashFree(tb->data, tb->alloc->args);
+		pfree(tb->alloc);
+	}
+
 	pfree(tb);
 }
 
@@ -333,9 +390,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 	/* compute parameters for new table */
 	SH_COMPUTE_PARAMETERS(tb, newsize);
 
-	tb->data = MemoryContextAllocExtended(
-								 tb->ctx, sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	newdata = tb->data;
 
@@ -421,7 +477,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 		}
 	}
 
-	pfree(olddata);
+	tb->alloc->HashFree(olddata, tb->alloc->args);
+
 }
 
 /*
@@ -826,6 +883,18 @@ SH_STAT(SH_TYPE *tb)
 		 total_collisions, max_collisions, avg_collisions);
 }
 
+/*
+ * Copy information from src_tb to dst_tb, it only copies those informations
+ * which are required by SH_ITERATE. Also store input data as hashtable data.
+ */
+SH_SCOPE void
+SH_COPY (SH_TYPE *src_tb, SH_TYPE *dst_tb, SH_ELEMENT_TYPE *data)
+{
+	dst_tb->size = src_tb->size;
+	dst_tb->sizemask = src_tb->sizemask;
+	dst_tb->data = data;
+}
+
 #endif   /* SH_DEFINE */
 
 
0003-parallel-bitmap-heap-scan-v13.patchapplication/octet-stream; name=0003-parallel-bitmap-heap-scan-v13.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 01fad38..2af778a 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1240,6 +1240,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1ce42ea..76a99a1 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index e01fe6d..090aaf3 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -205,6 +206,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -257,6 +262,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -733,6 +743,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index f18827d..48d4186 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,16 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+					  ParallelIterator *parallel_iterator);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -68,11 +74,18 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
 
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (pbminfo)
+		parallel_tbm = (ParallelTIDBitmap *) ((char *) pbminfo +
+											  pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,6 +100,35 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	prefetch_iterator = node->prefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	The first worker to see the state of ParallelBitmapInfo as PBM_INITIAL
+	 *	becomes leader and sets the state to PBM_INPROGRESS. All other workers
+	 *	see the state as PBM_INPROGRESS, and will wait for leader to finish
+	 *	building the TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *		ParallelBitmapInfo so that other worker can see those.
+	 *	  Set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader creates shared TIDBitmap.
+	 *	  Copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over page and chunk array
+	 *			and select heap pages one by one. If prefetch is enable then
+	 *			there will be two iterators.
+	 *	  b) Since multiple workers are iterating over same page and chunk
+	 *			array we need to have a shared iterator, so we grab a spin
+	 *			lock and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,10 +143,56 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.
+		 *
+		 * In parallel mode leader will immediately come out of the function,
+		 * but all other worker will be blocked until leader wake them up.
+		 */
+		if (pbminfo == NULL || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag in lower level bitmap index scan. Later
+			 * bitmap index node will use this flag to indicate tidbitmap that
+			 * it needs to create an shared page table.
+			 */
+			if (pbminfo)
+				pbms_set_parallel(outerPlanState(node));
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			/*
+			 * copy TBM local information in shared memory before waking up
+			 * other workers. Other workers will create there own TBM and copy
+			 * information from shared memory.
+			 */
+			if (pbminfo)
+			{
+				tbm_update_shared_members(tbm, parallel_tbm);
+
+				/* Change the state under a lock */
+				SpinLockAcquire(&pbminfo->state_mutex);
+				pbminfo->state = PBM_FINISHED;
+				SpinLockRelease(&pbminfo->state_mutex);
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+				/* Wake up all other workers. */
+				ConditionVariableBroadcast(&pbminfo->cv);
+			}
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM. Here we
+			 * need to create a local TBM and copy information from shared
+			 * location. We also need to attach to shared page table using
+			 * hash table handle stored in parallel_tbm (shared memory).
+			 */
+			tbm = tbm_attach(parallel_tbm, node->ss.ps.state->es_query_dsa);
+		}
 
 		node->tbm = tbm;
 		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
@@ -118,19 +206,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			node->prefetch_target = -1;
 		}
 #endif   /* USE_PREFETCH */
+
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+									 pbminfo ? &pbminfo->tbmiterator : NULL);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +229,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (pbminfo)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+							   pbminfo ? &pbminfo->prefetch_iterator : NULL);
+
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not ensure
+				 * that current blockno in main iterator and prefetch iterator
+				 * is same. It's possible that whatever blockno we are
+				 * prefetching is getting processed by some other worker.
+				 */
+				if ((pbminfo == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +300,39 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				int		   *prefetch_target;
+
+				if (pbminfo == NULL)
+					prefetch_target = &node->prefetch_target;
+				else
+				{
+					prefetch_target = &pbminfo->prefetch_target;
+					/* If we are in parallel mode then grab prefetch_mutex */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+				}
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +348,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (pbminfo == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+					pbminfo->prefetch_target++;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +387,53 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+			TBMIterator *iterator = node->prefetch_iterator;
+			ParallelIterator *parallel_iteartor = NULL;
+			ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (pbminfo)
+			{
+				parallel_iteartor = &pbminfo->prefetch_iterator;
+				prefetch_pages = &pbminfo->prefetch_pages;
+				prefetch_target = pbminfo->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
+
+			while (*prefetch_pages < prefetch_target)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+														  parallel_iteartor);
 
 				if (tbmpre == NULL)
 				{
 					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					tbm_end_iterate(iterator);
+					prefetch_iterator = node->prefetch_iterator = NULL;
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +648,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +766,7 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -651,3 +851,218 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : then we need to wait till leader creates BITMAP
+		 * PBM_FINISHED	  : BITMAP is ready so no need to wait.
+		 *---------------
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Wrapper over tbm_iterate, which take care of copying and restoring iterator
+ * variables from parallel iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator, ParallelIterator *piterator)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (piterator == NULL)
+		return tbm_iterate(iterator);
+
+	/* We are in parallel mode so grab parallel iterator mutex */
+	SpinLockAcquire(&piterator->mutex);
+
+	/*
+	 * Copy information from shared location to local iterator and call
+	 * iterate. Then restore back to shared iterator.
+	 */
+	iterator->spageptr = piterator->spageptr;
+	iterator->schunkptr = piterator->schunkptr;
+	iterator->schunkbit = piterator->schunkbit;
+	output = tbm_iterate(iterator);
+	piterator->spageptr = iterator->spageptr;
+	piterator->schunkptr = iterator->schunkptr;
+	piterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&piterator->mutex);
+
+	return output;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node as
+	 * parallel, because only first node will create the main bitmap other
+	 * bitmaps will be merged to the first bitmap so no need to create them in
+	 * shared memory.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState *) node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..56f8376 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate that we
+	 * need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_dsa);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..cf8d6d7 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -42,7 +42,14 @@
 
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
+#include "nodes/execnodes.h"
 #include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
+#include "storage/lwlock.h"
+#include "storage/shmem.h"
+#include "storage/spin.h"
+#include "utils/dsa.h"
+#include "utils/hsearch.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -136,27 +143,15 @@ struct TIDBitmap
 	bool		iterating;		/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
+	bool		is_shared;		/* need to build shared tbm if set */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	int			dsa_entries;	/* actual allocated entries */
+	dsa_area   *area;			/* reference to per-query shared memory area */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
 };
 
-/*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
- */
-struct TBMIterator
-{
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
-};
-
-
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
 static bool tbm_intersect_page(TIDBitmap *a, PagetableEntry *apage,
@@ -168,6 +163,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+void	   *tbm_alloc_shared(Size size, void *arg);
+void		tbm_free_shared(void *pointer, void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -198,6 +195,19 @@ hash_blockno(BlockNumber b)
 #define SH_DECLARE
 #include "lib/simplehash.h"
 
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+struct ParallelTIDBitmap
+{
+	dsa_pointer dsa_data;		/* dsa pointers for all kind of pages */
+	pagetable_hash pagetable;	/* hash table of PagetableEntry's */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	int			dsa_entries;	/* total item in dsa_pages */
+};
 
 /*
  * tbm_create - create an initially-empty bitmap
@@ -244,7 +254,18 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	if (tbm->is_shared)
+	{
+		pagetable_alloc allocator;
+
+		allocator.args = tbm;
+		allocator.HashAlloc = tbm_alloc_shared;
+		allocator.HashFree = tbm_free_shared;
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, &allocator);
+		tbm->dsa_entries = tbm->pagetable->size;
+	}
+	else
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -1061,3 +1082,137 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_update_shared_members
+ *
+ * Store leader's private tbm state to shared location. This must
+ * be called before waking up other workers.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before waking up the other
+	 * workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->dsa_data = tbm->dsa_data;
+	parallel_tbm->dsa_entries = tbm->dsa_entries;
+	pagetable_copy(tbm->pagetable, &parallel_tbm->pagetable, NULL);
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->is_shared = true;
+	tbm->area = (dsa_area *) area;
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ *
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size
+tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
+
+/*
+ * tbm_attach
+ *
+ * Create a local TIDBitmap for worker and Attach it to shared TID bitmap
+ * created by leader.
+ */
+TIDBitmap *
+tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area)
+{
+	TIDBitmap  *tbm = makeNode(TIDBitmap);
+	PagetableEntry *dsa_entry;
+
+	tbm->mcxt = CurrentMemoryContext;
+	tbm->status = TBM_HASH;
+	tbm->nchunks = parallel_tbm->nchunks;
+	tbm->nentries = parallel_tbm->nentries;
+	tbm->npages = parallel_tbm->npages;
+	tbm->maxentries = parallel_tbm->maxentries;
+	tbm->dsa_data = parallel_tbm->dsa_data;
+	tbm->dsa_entries = parallel_tbm->dsa_entries;
+	dsa_entry = dsa_get_address(area, tbm->dsa_data);
+	dsa_entry = (void *) (((char *) dsa_entry) + sizeof(dsa_pointer));
+	tbm->pagetable = palloc0(sizeof(struct pagetable_hash));
+	pagetable_copy(&parallel_tbm->pagetable, tbm->pagetable, dsa_entry);
+
+	/* Mark the tbm as parallel and also keep the DSA reference in it. */
+	tbm_set_parallel(tbm, area);
+
+	return tbm;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Memory allocation function for bitmap hash.
+ * It allocates memory from DSA and also stores dsa_pointer in the memory
+ * so that during free we can directly get the dsa_pointe and free it.
+ */
+void *
+tbm_alloc_shared(Size size, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->area, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	/* Keep track of actual number of entries */
+	if (tbm->pagetable)
+		tbm->dsa_entries = tbm->pagetable->size;
+
+	ptr = dsa_get_address(tbm->area, dsaptr);
+	memset(ptr, 0, size + sizeof(dsa_pointer));
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * tbm_free_shared
+ *
+ * Memory free function for tid bitmap
+ */
+void
+tbm_free_shared(void *pointer, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsa_data;
+
+	/*
+	 * If TBM is in iterating phase means pagetable was already created and we
+	 * have come here during tbm_free. So we need not to do anything because
+	 * by this time DSA would have been already freed.
+	 */
+	if (tbm->iterating)
+		return;
+
+	dsa_data = *((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+	dsa_free(tbm->area, dsa_data);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 5c18987..a2f6fbb 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2875,6 +2875,31 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 }
 
 /*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+/*
  * Compute the number of parallel workers that should be used to scan a
  * relation.  "pages" is the number of pages from the relation that we
  * expect to scan.
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a43daa7..f81e469 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -816,6 +816,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -877,8 +878,32 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
+
+		/*
+		 * It may be possible to amortize some of the I/O cost, but probably
+		 * not very much, because most operating systems already do aggressive
+		 * prefetching.  For now, we assume that the disk run cost can't be
+		 * amortized at all.
+		 */
+
+		/*
+		 * In the case of a parallel plan, the row count needs to represent
+		 * the number of tuples processed per worker.
+		 */
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 5283468..630c501 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1557,6 +1561,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1599,6 +1608,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index fae1f67..221b5f1 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2834,7 +2834,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f440875..20b9a59 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3258,7 +3259,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 7176cf1..f8eef0b 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3392,6 +3392,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index ee7e05a..d7ebac4 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -178,6 +178,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index f9bcdd6..384a724 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1423,6 +1425,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t		mutex;			/* mutual exclusion for below three fields */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+}	ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	ParallelIterator tbmiterator;
+	ParallelIterator prefetch_iterator;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	Size		ptbm_offset;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1437,6 +1505,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1453,6 +1523,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1468,7 +1539,9 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1483,6 +1556,8 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..77d0fc6 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+} TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -62,5 +79,12 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+						   ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+						  ParallelTIDBitmap *parallel_tbm);
+void		tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
+TIDBitmap *tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 7b41317..cfef818 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 81a9be7..968d39a 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index de8225b..c52c864 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -786,6 +786,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#49Dilip Kumar
dilipbalaut@gmail.com
In reply to: Dilip Kumar (#48)
3 attachment(s)
Re: Parallel bitmap heap scan

On Tue, Jan 24, 2017 at 10:18 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

I have changed as per the comments. 0002 and 0003 are changed, 0001 is
still the same.

2 days back my colleague Rafia, reported one issue (offlist) that
parallel bitmap node is not scaling as good as other nodes e.g
parallel sequence scan and parallel index scan.

After some perf analysis, I found that there was one unconditional
spin lock in parallel bitmap patch which we were taking for checking
the prefetch target. Basically, we were always taking the lock and
checking the prefetch_target is reached to prefetch_maximum. So even
after it will reach to prefetch_maximum we will acquire the lock and
check after every tuple. I have changed that logic, now I will check
the condition first if we need to increase the target then will take
the lock and recheck the condition.

There is just one line change in 0003 compared to older version, all
other patches are the same.

Some performance data to show that new parallel bitmap patch performs
way better than the previous version.
TPCH scale 20, work_mem 64MB, shared buffers 8GB (4 workers)
machine intel 8 socket machine

v13(time in ms) v14 (time in ms)
Q6 37065.841 18202.903

Q14 15229.569 11341.121

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0001-opt-parallelcost-refactoring-v14.patchapplication/octet-stream; name=0001-opt-parallelcost-refactoring-v14.patchDownload
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 458f139..a43daa7 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -813,7 +813,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
 	Cost		indexTotalCost;
-	Selectivity indexSelectivity;
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
@@ -837,13 +836,12 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	if (!enable_bitmapscan)
 		startup_cost += disable_cost;
 
-	/*
-	 * Fetch total cost of obtaining the bitmap, as well as its total
-	 * selectivity.
-	 */
-	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+	pages_fetched = compute_bitmap_pages(root, baserel, bitmapqual,
+										 loop_count, &indexTotalCost,
+										 &tuples_fetched);
 
 	startup_cost += indexTotalCost;
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
 
 	/* Fetch estimated page costs for tablespace containing table. */
 	get_tablespace_page_costs(baserel->reltablespace,
@@ -851,41 +849,6 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 							  &spc_seq_page_cost);
 
 	/*
-	 * Estimate number of main-table pages fetched.
-	 */
-	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
-
-	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
-
-	if (loop_count > 1)
-	{
-		/*
-		 * For repeated bitmap scans, scale up the number of tuples fetched in
-		 * the Mackert and Lohman formula by the number of scans, so that we
-		 * estimate the number of pages fetched by all the scans. Then
-		 * pro-rate for one scan.
-		 */
-		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
-											baserel->pages,
-											get_indexpath_pages(bitmapqual),
-											root);
-		pages_fetched /= loop_count;
-	}
-	else
-	{
-		/*
-		 * For a single scan, the number of heap pages that need to be fetched
-		 * is the same as the Mackert and Lohman formula for the case T <= b
-		 * (ie, no re-reads needed).
-		 */
-		pages_fetched = (2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
-	}
-	if (pages_fetched >= T)
-		pages_fetched = T;
-	else
-		pages_fetched = ceil(pages_fetched);
-
-	/*
 	 * For small numbers of pages we should charge spc_random_page_cost
 	 * apiece, while if nearly all the table's pages are being read, it's more
 	 * appropriate to charge spc_seq_page_cost apiece.  The effect is
@@ -4820,3 +4783,69 @@ get_parallel_divisor(Path *path)
 
 	return parallel_divisor;
 }
+
+/*
+ * compute_bitmap_pages
+ *
+ * compute number of pages fetched from heap in bitmap heap scan.
+ */
+double
+compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel, Path *bitmapqual,
+					 int loop_count, Cost *cost, double *tuple)
+{
+	Cost		indexTotalCost;
+	Selectivity indexSelectivity;
+	double		T;
+	double		pages_fetched;
+	double		tuples_fetched;
+
+	/*
+	 * Fetch total cost of obtaining the bitmap, as well as its total
+	 * selectivity.
+	 */
+	cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
+
+	/*
+	 * Estimate number of main-table pages fetched.
+	 */
+	tuples_fetched = clamp_row_est(indexSelectivity * baserel->tuples);
+
+	T = (baserel->pages > 1) ? (double) baserel->pages : 1.0;
+
+	if (loop_count > 1)
+	{
+		/*
+		 * For repeated bitmap scans, scale up the number of tuples fetched in
+		 * the Mackert and Lohman formula by the number of scans, so that we
+		 * estimate the number of pages fetched by all the scans. Then
+		 * pro-rate for one scan.
+		 */
+		pages_fetched = index_pages_fetched(tuples_fetched * loop_count,
+											baserel->pages,
+											get_indexpath_pages(bitmapqual),
+											root);
+		pages_fetched /= loop_count;
+	}
+	else
+	{
+		/*
+		 * For a single scan, the number of heap pages that need to be fetched
+		 * is the same as the Mackert and Lohman formula for the case T <= b
+		 * (ie, no re-reads needed).
+		 */
+		pages_fetched =
+			(2.0 * T * tuples_fetched) / (2.0 * T + tuples_fetched);
+	}
+
+	if (pages_fetched >= T)
+		pages_fetched = T;
+	else
+		pages_fetched = ceil(pages_fetched);
+
+	if (cost)
+		*cost = indexTotalCost;
+	if (tuple)
+		*tuple = tuples_fetched;
+
+	return pages_fetched;
+}
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 39376ec..0e68264 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -183,6 +183,8 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
 					   double cte_rows);
 extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
 extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern double compute_bitmap_pages(PlannerInfo *root, RelOptInfo *baserel,
+				  Path *bitmapqual, int loop_count, Cost *cost, double *tuple);
 
 /*
  * prototypes for clausesel.c
0002-hash-support-alloc-free-v14.patchapplication/octet-stream; name=0002-hash-support-alloc-free-v14.patchDownload
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 47c9656..14f8c2d 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -330,7 +330,7 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
 	else
 		hashtable->hash_iv = 0;
 
-	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
+	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, NULL);
 	hashtable->hashtab->private_data = hashtable;
 
 	return hashtable;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 7b31948..0885812 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -244,7 +244,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index 12aedbc..e2c8f44 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -66,6 +66,7 @@
 #define SH_STATUS_EMPTY SH_MAKE_NAME(EMPTY)
 #define SH_STATUS_IN_USE SH_MAKE_NAME(IN_USE)
 #define SH_ITERATOR SH_MAKE_NAME(iterator)
+#define SH_ALLOCATOR SH_MAKE_NAME(alloc)
 
 /* function declarations */
 #define SH_CREATE SH_MAKE_NAME(create)
@@ -78,6 +79,7 @@
 #define SH_START_ITERATE_AT SH_MAKE_NAME(start_iterate_at)
 #define SH_ITERATE SH_MAKE_NAME(iterate)
 #define SH_STAT SH_MAKE_NAME(stat)
+#define SH_COPY SH_MAKE_NAME(copy)
 
 /* internal helper functions (no externally visible prototypes) */
 #define SH_COMPUTE_PARAMETERS SH_MAKE_NAME(compute_parameters)
@@ -90,6 +92,18 @@
 /* generate forward declarations necessary to use the hash table */
 #ifdef SH_DECLARE
 
+typedef struct SH_ALLOCATOR
+{
+	/* Allocation function */
+	void	   *(*HashAlloc) (Size size, void *args);
+
+	/* Free function */
+	void		(*HashFree) (void *pointer, void *args);
+
+	/* Arguments to be passed to alloc and free functions */
+	void	   *args;
+}	SH_ALLOCATOR;
+
 /* type definitions */
 typedef struct SH_TYPE
 {
@@ -112,6 +126,9 @@ typedef struct SH_TYPE
 	/* hash buckets */
 	SH_ELEMENT_TYPE *data;
 
+	/* hash allocator */
+	SH_ALLOCATOR *alloc;
+
 	/* memory context to use for allocations */
 	MemoryContext ctx;
 
@@ -133,7 +150,8 @@ typedef struct SH_ITERATOR
 } SH_ITERATOR;
 
 /* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
+							SH_ALLOCATOR *alloc);
 SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
 SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
 SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -143,6 +161,7 @@ SH_SCOPE void SH_START_ITERATE(SH_TYPE *tb, SH_ITERATOR *iter);
 SH_SCOPE void SH_START_ITERATE_AT(SH_TYPE *tb, SH_ITERATOR *iter, uint32 at);
 SH_SCOPE SH_ELEMENT_TYPE *SH_ITERATE(SH_TYPE *tb, SH_ITERATOR *iter);
 SH_SCOPE void SH_STAT(SH_TYPE *tb);
+SH_SCOPE void SH_COPY(SH_TYPE *src_tb, SH_TYPE *dst_tb, SH_ELEMENT_TYPE *data);
 
 #endif   /* SH_DECLARE */
 
@@ -276,15 +295,38 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
 #endif
 }
 
+/* default memory allocator function */
+static void *
+SH_DEFAULT_ALLOC(Size size, void *args)
+{
+	MemoryContext context = (MemoryContext) args;
+
+	return MemoryContextAllocExtended(context, size,
+									  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+}
+
+/* default memory free function */
+static void
+SH_DEFAULT_FREE(void *pointer, void *args)
+{
+	pfree(pointer);
+}
+
 /*
  * Create a hash table with enough space for `nelements` distinct members,
  * allocating required memory in the passed-in context.
+ *
+ * WARNING : alloc is an allocator handle which provides memory allocator
+ * function for storing hash elements. User needs to be aware that providing
+ * allocator does not store complete hash table in the allocator memory i.e
+ * hash table will still be allocated in local memory. However all the
+ * elements will be stored in the memory provided by the allocator.
  */
 SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements)
+SH_CREATE(MemoryContext ctx, uint32 nelements, SH_ALLOCATOR *alloc)
 {
-	SH_TYPE    *tb;
-	uint64		size;
+	SH_TYPE     *tb;
+	uint64		 size;
 
 	tb = MemoryContextAllocZero(ctx, sizeof(SH_TYPE));
 	tb->ctx = ctx;
@@ -294,9 +336,19 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 
 	SH_COMPUTE_PARAMETERS(tb, size);
 
-	tb->data = MemoryContextAllocExtended(tb->ctx,
-										  sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	tb->alloc = palloc(sizeof(SH_ALLOCATOR));
+
+	if (!alloc)
+	{
+		tb->alloc->HashAlloc = SH_DEFAULT_ALLOC;
+		tb->alloc->HashFree = SH_DEFAULT_FREE;
+		tb->alloc->args = tb->ctx;
+	}
+	else
+		memcpy(tb->alloc, alloc, sizeof(SH_ALLOCATOR));
+
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	return tb;
 }
@@ -305,7 +357,12 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 SH_SCOPE void
 SH_DESTROY(SH_TYPE *tb)
 {
-	pfree(tb->data);
+	if (tb->alloc)
+	{
+		tb->alloc->HashFree(tb->data, tb->alloc->args);
+		pfree(tb->alloc);
+	}
+
 	pfree(tb);
 }
 
@@ -333,9 +390,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 	/* compute parameters for new table */
 	SH_COMPUTE_PARAMETERS(tb, newsize);
 
-	tb->data = MemoryContextAllocExtended(
-								 tb->ctx, sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	tb->data = tb->alloc->HashAlloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
+									tb->alloc->args);
 
 	newdata = tb->data;
 
@@ -421,7 +477,8 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 		}
 	}
 
-	pfree(olddata);
+	tb->alloc->HashFree(olddata, tb->alloc->args);
+
 }
 
 /*
@@ -826,6 +883,18 @@ SH_STAT(SH_TYPE *tb)
 		 total_collisions, max_collisions, avg_collisions);
 }
 
+/*
+ * Copy information from src_tb to dst_tb, it only copies those informations
+ * which are required by SH_ITERATE. Also store input data as hashtable data.
+ */
+SH_SCOPE void
+SH_COPY (SH_TYPE *src_tb, SH_TYPE *dst_tb, SH_ELEMENT_TYPE *data)
+{
+	dst_tb->size = src_tb->size;
+	dst_tb->sizemask = src_tb->sizemask;
+	dst_tb->data = data;
+}
+
 #endif   /* SH_DEFINE */
 
 
0003-parallel-bitmap-heap-scan-v14.patchapplication/octet-stream; name=0003-parallel-bitmap-heap-scan-v14.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 01fad38..2af778a 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1240,6 +1240,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1ce42ea..76a99a1 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index e01fe6d..090aaf3 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -205,6 +206,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -257,6 +262,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -733,6 +743,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index f18827d..9d34b53 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,16 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+					  ParallelIterator *parallel_iterator);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -68,11 +74,18 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
 
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (pbminfo)
+		parallel_tbm = (ParallelTIDBitmap *) ((char *) pbminfo +
+											  pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,6 +100,35 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	prefetch_iterator = node->prefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	The first worker to see the state of ParallelBitmapInfo as PBM_INITIAL
+	 *	becomes leader and sets the state to PBM_INPROGRESS. All other workers
+	 *	see the state as PBM_INPROGRESS, and will wait for leader to finish
+	 *	building the TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *		ParallelBitmapInfo so that other worker can see those.
+	 *	  Set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader creates shared TIDBitmap.
+	 *	  Copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over page and chunk array
+	 *			and select heap pages one by one. If prefetch is enable then
+	 *			there will be two iterators.
+	 *	  b) Since multiple workers are iterating over same page and chunk
+	 *			array we need to have a shared iterator, so we grab a spin
+	 *			lock and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,10 +143,56 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.
+		 *
+		 * In parallel mode leader will immediately come out of the function,
+		 * but all other worker will be blocked until leader wake them up.
+		 */
+		if (pbminfo == NULL || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag in lower level bitmap index scan. Later
+			 * bitmap index node will use this flag to indicate tidbitmap that
+			 * it needs to create an shared page table.
+			 */
+			if (pbminfo)
+				pbms_set_parallel(outerPlanState(node));
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			/*
+			 * copy TBM local information in shared memory before waking up
+			 * other workers. Other workers will create there own TBM and copy
+			 * information from shared memory.
+			 */
+			if (pbminfo)
+			{
+				tbm_update_shared_members(tbm, parallel_tbm);
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+				/* Change the state under a lock */
+				SpinLockAcquire(&pbminfo->state_mutex);
+				pbminfo->state = PBM_FINISHED;
+				SpinLockRelease(&pbminfo->state_mutex);
+
+				/* Wake up all other workers. */
+				ConditionVariableBroadcast(&pbminfo->cv);
+			}
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM. Here we
+			 * need to create a local TBM and copy information from shared
+			 * location. We also need to attach to shared page table using
+			 * hash table handle stored in parallel_tbm (shared memory).
+			 */
+			tbm = tbm_attach(parallel_tbm, node->ss.ps.state->es_query_dsa);
+		}
 
 		node->tbm = tbm;
 		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
@@ -118,19 +206,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			node->prefetch_target = -1;
 		}
 #endif   /* USE_PREFETCH */
+
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+									 pbminfo ? &pbminfo->tbmiterator : NULL);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +229,44 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (pbminfo)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				tbmpre = pbms_parallel_iterate(prefetch_iterator,
+							   pbminfo ? &pbminfo->prefetch_iterator : NULL);
+
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not ensure
+				 * that current blockno in main iterator and prefetch iterator
+				 * is same. It's possible that whatever blockno we are
+				 * prefetching is getting processed by some other worker.
+				 */
+				if ((pbminfo == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -182,20 +300,39 @@ BitmapHeapNext(BitmapHeapScanState *node)
 
 #ifdef USE_PREFETCH
 
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Increase prefetch target if it's not yet at the max. */
+			if (node->prefetch_target < node->prefetch_maximum)
+			{
+				int		   *prefetch_target;
+
+				if (pbminfo == NULL)
+					prefetch_target = &node->prefetch_target;
+				else
+				{
+					prefetch_target = &pbminfo->prefetch_target;
+					/* If we are in parallel mode then grab prefetch_mutex */
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+				}
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +348,25 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (pbminfo == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+				{
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+					if (pbminfo->prefetch_target < node->prefetch_maximum)
+						pbminfo->prefetch_target++;
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+				}
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +390,53 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+			TBMIterator *iterator = node->prefetch_iterator;
+			ParallelIterator *parallel_iteartor = NULL;
+			ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (pbminfo)
+			{
+				parallel_iteartor = &pbminfo->prefetch_iterator;
+				prefetch_pages = &pbminfo->prefetch_pages;
+				prefetch_target = pbminfo->prefetch_target;
+			}
+			else
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
+
+			while (*prefetch_pages < prefetch_target)
+			{
+				TBMIterateResult *tbmpre = pbms_parallel_iterate(iterator,
+														  parallel_iteartor);
 
 				if (tbmpre == NULL)
 				{
 					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					tbm_end_iterate(iterator);
+					prefetch_iterator = node->prefetch_iterator = NULL;
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +651,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +769,7 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -651,3 +854,218 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : then we need to wait till leader creates BITMAP
+		 * PBM_FINISHED	  : BITMAP is ready so no need to wait.
+		 *---------------
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Wrapper over tbm_iterate, which take care of copying and restoring iterator
+ * variables from parallel iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator, ParallelIterator *piterator)
+{
+	TBMIterateResult *output;
+
+	/* If not running in parallel mode then directly call tbm_iterate. */
+	if (piterator == NULL)
+		return tbm_iterate(iterator);
+
+	/* We are in parallel mode so grab parallel iterator mutex */
+	SpinLockAcquire(&piterator->mutex);
+
+	/*
+	 * Copy information from shared location to local iterator and call
+	 * iterate. Then restore back to shared iterator.
+	 */
+	iterator->spageptr = piterator->spageptr;
+	iterator->schunkptr = piterator->schunkptr;
+	iterator->schunkbit = piterator->schunkbit;
+	output = tbm_iterate(iterator);
+	piterator->spageptr = iterator->spageptr;
+	piterator->schunkptr = iterator->schunkptr;
+	piterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&piterator->mutex);
+
+	return output;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node as
+	 * parallel, because only first node will create the main bitmap other
+	 * bitmaps will be merged to the first bitmap so no need to create them in
+	 * shared memory.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState *) node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..56f8376 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate that we
+	 * need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_dsa);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..cf8d6d7 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -42,7 +42,14 @@
 
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
+#include "nodes/execnodes.h"
 #include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
+#include "storage/lwlock.h"
+#include "storage/shmem.h"
+#include "storage/spin.h"
+#include "utils/dsa.h"
+#include "utils/hsearch.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -136,27 +143,15 @@ struct TIDBitmap
 	bool		iterating;		/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
+	bool		is_shared;		/* need to build shared tbm if set */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	int			dsa_entries;	/* actual allocated entries */
+	dsa_area   *area;			/* reference to per-query shared memory area */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
 };
 
-/*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
- */
-struct TBMIterator
-{
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
-};
-
-
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
 static bool tbm_intersect_page(TIDBitmap *a, PagetableEntry *apage,
@@ -168,6 +163,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+void	   *tbm_alloc_shared(Size size, void *arg);
+void		tbm_free_shared(void *pointer, void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -198,6 +195,19 @@ hash_blockno(BlockNumber b)
 #define SH_DECLARE
 #include "lib/simplehash.h"
 
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+struct ParallelTIDBitmap
+{
+	dsa_pointer dsa_data;		/* dsa pointers for all kind of pages */
+	pagetable_hash pagetable;	/* hash table of PagetableEntry's */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	int			dsa_entries;	/* total item in dsa_pages */
+};
 
 /*
  * tbm_create - create an initially-empty bitmap
@@ -244,7 +254,18 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	if (tbm->is_shared)
+	{
+		pagetable_alloc allocator;
+
+		allocator.args = tbm;
+		allocator.HashAlloc = tbm_alloc_shared;
+		allocator.HashFree = tbm_free_shared;
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, &allocator);
+		tbm->dsa_entries = tbm->pagetable->size;
+	}
+	else
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -1061,3 +1082,137 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_update_shared_members
+ *
+ * Store leader's private tbm state to shared location. This must
+ * be called before waking up other workers.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before waking up the other
+	 * workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->dsa_data = tbm->dsa_data;
+	parallel_tbm->dsa_entries = tbm->dsa_entries;
+	pagetable_copy(tbm->pagetable, &parallel_tbm->pagetable, NULL);
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for
+ * creating bitmap using dynamic shared memory hash table.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->is_shared = true;
+	tbm->area = (dsa_area *) area;
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ *
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size
+tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
+
+/*
+ * tbm_attach
+ *
+ * Create a local TIDBitmap for worker and Attach it to shared TID bitmap
+ * created by leader.
+ */
+TIDBitmap *
+tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area)
+{
+	TIDBitmap  *tbm = makeNode(TIDBitmap);
+	PagetableEntry *dsa_entry;
+
+	tbm->mcxt = CurrentMemoryContext;
+	tbm->status = TBM_HASH;
+	tbm->nchunks = parallel_tbm->nchunks;
+	tbm->nentries = parallel_tbm->nentries;
+	tbm->npages = parallel_tbm->npages;
+	tbm->maxentries = parallel_tbm->maxentries;
+	tbm->dsa_data = parallel_tbm->dsa_data;
+	tbm->dsa_entries = parallel_tbm->dsa_entries;
+	dsa_entry = dsa_get_address(area, tbm->dsa_data);
+	dsa_entry = (void *) (((char *) dsa_entry) + sizeof(dsa_pointer));
+	tbm->pagetable = palloc0(sizeof(struct pagetable_hash));
+	pagetable_copy(&parallel_tbm->pagetable, tbm->pagetable, dsa_entry);
+
+	/* Mark the tbm as parallel and also keep the DSA reference in it. */
+	tbm_set_parallel(tbm, area);
+
+	return tbm;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Memory allocation function for bitmap hash.
+ * It allocates memory from DSA and also stores dsa_pointer in the memory
+ * so that during free we can directly get the dsa_pointe and free it.
+ */
+void *
+tbm_alloc_shared(Size size, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->area, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	/* Keep track of actual number of entries */
+	if (tbm->pagetable)
+		tbm->dsa_entries = tbm->pagetable->size;
+
+	ptr = dsa_get_address(tbm->area, dsaptr);
+	memset(ptr, 0, size + sizeof(dsa_pointer));
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * tbm_free_shared
+ *
+ * Memory free function for tid bitmap
+ */
+void
+tbm_free_shared(void *pointer, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsa_data;
+
+	/*
+	 * If TBM is in iterating phase means pagetable was already created and we
+	 * have come here during tbm_free. So we need not to do anything because
+	 * by this time DSA would have been already freed.
+	 */
+	if (tbm->iterating)
+		return;
+
+	dsa_data = *((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+	dsa_free(tbm->area, dsa_data);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 5c18987..a2f6fbb 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2875,6 +2875,31 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 }
 
 /*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+/*
  * Compute the number of parallel workers that should be used to scan a
  * relation.  "pages" is the number of pages from the relation that we
  * expect to scan.
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a43daa7..f81e469 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -816,6 +816,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -877,8 +878,32 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
+
+		/*
+		 * It may be possible to amortize some of the I/O cost, but probably
+		 * not very much, because most operating systems already do aggressive
+		 * prefetching.  For now, we assume that the disk run cost can't be
+		 * amortized at all.
+		 */
+
+		/*
+		 * In the case of a parallel plan, the row count needs to represent
+		 * the number of tuples processed per worker.
+		 */
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 5283468..630c501 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1557,6 +1561,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1599,6 +1608,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index fae1f67..221b5f1 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2834,7 +2834,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f440875..20b9a59 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3258,7 +3259,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 7176cf1..f8eef0b 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3392,6 +3392,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index ee7e05a..d7ebac4 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -178,6 +178,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index f9bcdd6..384a724 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1423,6 +1425,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t		mutex;			/* mutual exclusion for below three fields */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+}	ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	ParallelIterator tbmiterator;
+	ParallelIterator prefetch_iterator;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	Size		ptbm_offset;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1437,6 +1505,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1453,6 +1523,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1468,7 +1539,9 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1483,6 +1556,8 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..77d0fc6 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+} TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -62,5 +79,12 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+						   ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+						  ParallelTIDBitmap *parallel_tbm);
+void		tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
+TIDBitmap *tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 7b41317..cfef818 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 81a9be7..968d39a 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index de8225b..c52c864 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -786,6 +786,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#50Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#49)
Re: Parallel bitmap heap scan

On Fri, Jan 27, 2017 at 1:32 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

There is just one line change in 0003 compared to older version, all
other patches are the same.

I spent some time looking at 0001 (and how those changes are used in
0003) and I thought it looked good, so I committed 0001.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#51Haribabu Kommi
kommi.haribabu@gmail.com
In reply to: Dilip Kumar (#49)
Re: Parallel bitmap heap scan

On Fri, Jan 27, 2017 at 5:32 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Tue, Jan 24, 2017 at 10:18 AM, Dilip Kumar <dilipbalaut@gmail.com>
wrote:

I have changed as per the comments. 0002 and 0003 are changed, 0001 is
still the same.

2 days back my colleague Rafia, reported one issue (offlist) that
parallel bitmap node is not scaling as good as other nodes e.g
parallel sequence scan and parallel index scan.

After some perf analysis, I found that there was one unconditional
spin lock in parallel bitmap patch which we were taking for checking
the prefetch target. Basically, we were always taking the lock and
checking the prefetch_target is reached to prefetch_maximum. So even
after it will reach to prefetch_maximum we will acquire the lock and
check after every tuple. I have changed that logic, now I will check
the condition first if we need to increase the target then will take
the lock and recheck the condition.

There is just one line change in 0003 compared to older version, all
other patches are the same.

Some performance data to show that new parallel bitmap patch performs
way better than the previous version.
TPCH scale 20, work_mem 64MB, shared buffers 8GB (4 workers)
machine intel 8 socket machine

v13(time in ms) v14 (time in ms)
Q6 37065.841 18202.903

Q14 15229.569 11341.121

Thanks for the update. I have some comments

0002-hash-support-alloc-free-v14.patch:

+ if (tb->alloc)
+ {

The memory for tb->alloc is allocated always, is the if check still
required?

0003-parallel-bitmap-heap-scan-v14.patch:

+ * and set parallel flag in lower level bitmap index scan. Later
+ * bitmap index node will use this flag to indicate tidbitmap that
+ * it needs to create an shared page table.
+ */

I feel better to mention, where this flag is used, so that it will be more
clear.

+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for

The flag of shared is set in tidbitmap structure itself, but the
comment is mentioned that tidbitmpa should be created in shared memory.
I think that is the page table that needs to be created in shared memory.
Providing some more information here will be helpful.

- node->tbmres = tbmres = tbm_iterate(tbmiterator);
+ node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+ pbminfo ? &pbminfo->tbmiterator : NULL);

Instead Passing both normal and paralle iterators to the functions
and checking inside again for NULL, How about just adding check
in the caller itself? Or if you prefer the current method, then try to
explain the details of input in the function header and more description.

+ /* Increase prefetch target if it's not yet at the max. */
+ if (node->prefetch_target < node->prefetch_maximum)

I didn't evaluate all scenarios, but the above code may have a problem,
In case of parallel mode the the prefetch_target is fetched from node
and not from the pbminfo. Later it gets from the pminfo and update that.
May be this code needs to rewrite.

+ TBMIterator *iterator = node->prefetch_iterator;

Why another variable? Why can't we use the prefetch_iterator itself?
Currently node->tbmiterator and node->prefetch_iterator are initialized
irrespective of whether is it a parallel one or not. But while using, there
is a check to use the parallel one in case of parallel. if it is the case,
why can't we avoid the initialization itself?

+ else
+ needWait = false;

By default needWait is false. Just set that to true only for the
case of PBM_INPROGRESS

+ * so that during free we can directly get the dsa_pointe and free it.

dsa_pointe -> dsa_pointer

+typedef struct
+{
+ TIDBitmap  *tbm; /* TIDBitmap we're iterating over */
+ int spageptr; /* next spages index */
+ int schunkptr; /* next schunks index */
+ int schunkbit; /* next bit to check in current schunk */
+ TBMIterateResult output; /* MUST BE LAST (because variable-size) */
+} TBMIterator;

I didn't find the need of moving this structure. Can you point me where it
is used.

Regards,
Hari Babu
Fujitsu Australia

#52Michael Paquier
michael.paquier@gmail.com
In reply to: Haribabu Kommi (#51)
Re: Parallel bitmap heap scan

On Tue, Jan 31, 2017 at 11:17 AM, Haribabu Kommi
<kommi.haribabu@gmail.com> wrote:

On Fri, Jan 27, 2017 at 5:32 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Tue, Jan 24, 2017 at 10:18 AM, Dilip Kumar <dilipbalaut@gmail.com>
wrote:

I have changed as per the comments. 0002 and 0003 are changed, 0001 is
still the same.

There is just one line change in 0003 compared to older version, all
other patches are the same.

Thanks for the update. I have some comments

This review is too fresh to be addressed, so I have moved this patch
to the next CF.
--
Michael

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#53Dilip Kumar
dilipbalaut@gmail.com
In reply to: Haribabu Kommi (#51)
1 attachment(s)
Re: Parallel bitmap heap scan

On Tue, Jan 31, 2017 at 7:47 AM, Haribabu Kommi
<kommi.haribabu@gmail.com> wrote:

Thanks for the update. I have some comments

Thanks for the review.

0002-hash-support-alloc-free-v14.patch:

+ if (tb->alloc)
+ {

The memory for tb->alloc is allocated always, is the if check still
required?

In parallel case, only first worker will call SH_CREATE, other worker
will only do palloc for pagetable and copy the reference from main
worker, so they will not have allocator.

0003-parallel-bitmap-heap-scan-v14.patch:

+ * and set parallel flag in lower level bitmap index scan. Later
+ * bitmap index node will use this flag to indicate tidbitmap that
+ * it needs to create an shared page table.
+ */

I feel better to mention, where this flag is used, so that it will be more
clear.

Done

+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that this tidbitmap
+ * should be created in shared memory. DSA area will be used for

The flag of shared is set in tidbitmap structure itself, but the
comment is mentioned that tidbitmpa should be created in shared memory.
I think that is the page table that needs to be created in shared memory.
Providing some more information here will be helpful.

Done

- node->tbmres = tbmres = tbm_iterate(tbmiterator);
+ node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+ pbminfo ? &pbminfo->tbmiterator : NULL);

Instead Passing both normal and paralle iterators to the functions
and checking inside again for NULL, How about just adding check
in the caller itself? Or if you prefer the current method, then try to
explain the details of input in the function header and more description.

Done as you said.

+ /* Increase prefetch target if it's not yet at the max. */
+ if (node->prefetch_target < node->prefetch_maximum)

I didn't evaluate all scenarios, but the above code may have a problem,
In case of parallel mode the the prefetch_target is fetched from node
and not from the pbminfo. Later it gets from the pminfo and update that.
May be this code needs to rewrite.

Fixed it.

+ TBMIterator *iterator = node->prefetch_iterator;

Why another variable? Why can't we use the prefetch_iterator itself?
Currently node->tbmiterator and node->prefetch_iterator are initialized
irrespective of whether is it a parallel one or not. But while using, there
is a check to use the parallel one in case of parallel. if it is the case,
why can't we avoid the initialization itself?

Fixed

+ else
+ needWait = false;

By default needWait is false. Just set that to true only for the
case of PBM_INPROGRESS

Actually inside the while loop, suppose first we set needWait = true,
if PBM_INPROGRESS and got for ConditionalSleep, After it come out of
sleep, we need to check that PBM_FINISHED is set or we need to sleep
again, So in such case we need to reset it to needWait=false. However
this can be done by directly returning if it's PBM_FINISHED. But I
want to keep below the logic common.
+ SpinLockRelease(&pbminfo->state_mutex);
+
+ /* If we are leader or leader has already created a TIDBITMAP */
+ if (leader || !needWait)
+ break;

+ * so that during free we can directly get the dsa_pointe and free it.

dsa_pointe -> dsa_pointer

Done

+typedef struct
+{
+ TIDBitmap  *tbm; /* TIDBitmap we're iterating over */
+ int spageptr; /* next spages index */
+ int schunkptr; /* next schunks index */
+ int schunkbit; /* next bit to check in current schunk */
+ TBMIterateResult output; /* MUST BE LAST (because variable-size) */
+} TBMIterator;

I didn't find the need of moving this structure. Can you point me where it
is used.

Because pbms_parallel_iterate need to access this structure so I
moved it to tidbitmap.h

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0003-parallel-bitmap-heap-scan-v15.patchapplication/octet-stream; name=0003-parallel-bitmap-heap-scan-v15.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 01fad38..2af778a 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1240,6 +1240,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1ce42ea..76a99a1 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index e01fe6d..090aaf3 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -205,6 +206,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -257,6 +262,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -733,6 +743,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index f18827d..cecba16 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,16 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static TBMIterateResult *pbms_parallel_iterate(TBMIterator *iterator,
+					  ParallelIterator *parallel_iterator);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -68,11 +74,18 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
 
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTIDBitmap *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (pbminfo)
+		parallel_tbm = (ParallelTIDBitmap *) ((char *) pbminfo +
+											  pbminfo->ptbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,6 +100,35 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	prefetch_iterator = node->prefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	The first worker to see the state of ParallelBitmapInfo as PBM_INITIAL
+	 *	becomes leader and sets the state to PBM_INPROGRESS. All other workers
+	 *	see the state as PBM_INPROGRESS, and will wait for leader to finish
+	 *	building the TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *		ParallelBitmapInfo so that other worker can see those.
+	 *	  Set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader creates shared TIDBitmap.
+	 *	  Copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over page and chunk array
+	 *			and select heap pages one by one. If prefetch is enable then
+	 *			there will be two iterators.
+	 *	  b) Since multiple workers are iterating over same page and chunk
+	 *			array we need to have a shared iterator, so we grab a spin
+	 *			lock and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,10 +143,56 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.
+		 *
+		 * In parallel mode leader will immediately come out of the function,
+		 * but all other worker will be blocked until leader wake them up.
+		 */
+		if (pbminfo == NULL || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag in lower level bitmap index scan. This
+			 * flag will be used by bitmap index node to identify whether
+			 * it needs to create a shared pagetable or local pagetable.
+			 */
+			if (pbminfo)
+				pbms_set_parallel(outerPlanState(node));
+
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			/*
+			 * copy TBM local information in shared memory before waking up
+			 * other workers. Other workers will create there own TBM and copy
+			 * information from shared memory.
+			 */
+			if (pbminfo)
+			{
+				tbm_update_shared_members(tbm, parallel_tbm);
+
+				/* Change the state under a lock */
+				SpinLockAcquire(&pbminfo->state_mutex);
+				pbminfo->state = PBM_FINISHED;
+				SpinLockRelease(&pbminfo->state_mutex);
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+				/* Wake up all other workers. */
+				ConditionVariableBroadcast(&pbminfo->cv);
+			}
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM. Here we
+			 * need to create a local TBM and copy information from shared
+			 * location. We also need to attach to shared page table using
+			 * hash table handle stored in parallel_tbm (shared memory).
+			 */
+			tbm = tbm_attach(parallel_tbm, node->ss.ps.state->es_query_dsa);
+		}
 
 		node->tbm = tbm;
 		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
@@ -118,19 +206,26 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			node->prefetch_target = -1;
 		}
 #endif   /* USE_PREFETCH */
+
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
+		int		   *prefetch_target;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			if (!pbminfo)
+				node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			else
+				node->tbmres = tbmres = pbms_parallel_iterate(tbmiterator,
+													   &pbminfo->tbmiterator);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +233,47 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			if (pbminfo)
+			{
+				/*
+				 * If we are in parallel mode then acquire prefetch_mutex and
+				 * check prefetch_pages from shared location.
+				 */
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+			else if (node->prefetch_pages > 0)
 			{
 				/* The main iterator has closed the distance by one page */
 				node->prefetch_pages--;
 			}
-			else if (prefetch_iterator)
+			else
+				need_prefetch = true;
+
+			if (prefetch_iterator && need_prefetch)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				if (!pbminfo)
+					tbmpre = tbm_iterate(prefetch_iterator);
+				else
+					tbmpre = pbms_parallel_iterate(prefetch_iterator,
+												  &pbminfo->prefetch_iterator);
+
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind main iterator, but we can not ensure
+				 * that current blockno in main iterator and prefetch iterator
+				 * is same. It's possible that whatever blockno we are
+				 * prefetching is getting processed by some other worker.
+				 */
+				if ((pbminfo == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -183,19 +308,40 @@ BitmapHeapNext(BitmapHeapScanState *node)
 #ifdef USE_PREFETCH
 
 			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
+			 * If we are in shared mode then use prefetch_target from shared
+			 * location i.e pbminfo otherwise directly from node.
 			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
+			if (pbminfo == NULL)
+				prefetch_target = &node->prefetch_target;
 			else
-				node->prefetch_target++;
+				prefetch_target = &pbminfo->prefetch_target;
+
+			/* Increase prefetch target if it's not yet at the max. */
+			if (*prefetch_target < node->prefetch_maximum)
+			{
+				/* If we are in parallel mode then grab prefetch_mutex */
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +357,25 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (pbminfo == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+				{
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+					if (pbminfo->prefetch_target < node->prefetch_maximum)
+						pbminfo->prefetch_target++;
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+				}
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,9 +399,38 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (pbminfo)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				prefetch_pages = &pbminfo->prefetch_pages;
+				prefetch_target = pbminfo->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
+
+			/*
+			 *  We are checking the prefetch_pages without mutex. Henceforth
+			 *  in case of parallel mode there can be some extra prefetch.
+			 *  Should we acquire mutex and recheck before iterating?
+			 */
+			while (*prefetch_pages < prefetch_target)
+			{
+				TBMIterateResult *tbmpre;
+
+				if (!pbminfo)
+					tbmpre = tbm_iterate(prefetch_iterator);
+				else
+					tbmpre = pbms_parallel_iterate(prefetch_iterator,
+												  &pbminfo->prefetch_iterator);
 
 				if (tbmpre == NULL)
 				{
@@ -247,10 +439,19 @@ BitmapHeapNext(BitmapHeapScanState *node)
 					node->prefetch_iterator = prefetch_iterator = NULL;
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +666,22 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->tbmiterator.schunkbit = 0;
+		pbminfo->tbmiterator.spageptr = 0;
+		pbminfo->tbmiterator.schunkptr = 0;
+		pbminfo->prefetch_iterator.schunkbit = 0;
+		pbminfo->prefetch_iterator.spageptr = 0;
+		pbminfo->prefetch_iterator.schunkptr = 0;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +784,7 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -651,3 +869,218 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : then we need to wait till leader creates BITMAP
+		 * PBM_FINISHED	  : BITMAP is ready so no need to wait.
+		 *---------------
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------
+ *	pbms_iterator_init - initialize parallel bitmap scan iterator
+ *
+ * ----------------
+ */
+static void
+pbms_iterator_init(ParallelIterator *target)
+{
+	SpinLockInit(&target->mutex);
+
+	target->schunkbit = 0;
+	target->schunkptr = 0;
+	target->spageptr = 0;
+}
+
+/*
+ * pbms_parallel_iterate
+ *
+ * Wrapper over tbm_iterate, which take care of copying and restoring iterator
+ * variables from parallel iterator.
+ *
+ * iterator - local iterator for the worker
+ * piterator - shared iterator.
+ */
+static TBMIterateResult *
+pbms_parallel_iterate(TBMIterator *iterator, ParallelIterator *piterator)
+{
+	TBMIterateResult *output;
+
+
+	/* We are in parallel mode so grab parallel iterator mutex */
+	SpinLockAcquire(&piterator->mutex);
+
+	/*
+	 * Copy information from shared location to local iterator and call
+	 * iterate. Then restore back to shared iterator.
+	 */
+	iterator->spageptr = piterator->spageptr;
+	iterator->schunkptr = piterator->schunkptr;
+	iterator->schunkbit = piterator->schunkbit;
+	output = tbm_iterate(iterator);
+	piterator->spageptr = iterator->spageptr;
+	piterator->schunkptr = iterator->schunkptr;
+	piterator->schunkbit = iterator->schunkbit;
+
+	SpinLockRelease(&piterator->mutex);
+
+	return output;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node as
+	 * parallel, because only first node will create the main bitmap other
+	 * bitmaps will be merged to the first bitmap so no need to create them in
+	 * shared memory.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState *) node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tidbitmap(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->ptbm_offset = offset;
+
+	/* Initialize shared tbmiterator and prefetch_iterator */
+	pbms_iterator_init(&pbminfo->tbmiterator);
+	pbms_iterator_init(&pbminfo->prefetch_iterator);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..56f8376 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate that we
+	 * need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_dsa);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..ebd2950 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -42,7 +42,14 @@
 
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
+#include "nodes/execnodes.h"
 #include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
+#include "storage/lwlock.h"
+#include "storage/shmem.h"
+#include "storage/spin.h"
+#include "utils/dsa.h"
+#include "utils/hsearch.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -136,27 +143,15 @@ struct TIDBitmap
 	bool		iterating;		/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
+	bool		is_shared;		/* need to build shared tbm if set */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	int			dsa_entries;	/* actual allocated entries */
+	dsa_area   *area;			/* reference to per-query shared memory area */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
 };
 
-/*
- * When iterating over a bitmap in sorted order, a TBMIterator is used to
- * track our progress.  There can be several iterators scanning the same
- * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
- * any iterator is created.
- */
-struct TBMIterator
-{
-	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
-	int			spageptr;		/* next spages index */
-	int			schunkptr;		/* next schunks index */
-	int			schunkbit;		/* next bit to check in current schunk */
-	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
-};
-
-
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
 static bool tbm_intersect_page(TIDBitmap *a, PagetableEntry *apage,
@@ -168,6 +163,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+void	   *tbm_alloc_shared(Size size, void *arg);
+void		tbm_free_shared(void *pointer, void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -198,6 +195,19 @@ hash_blockno(BlockNumber b)
 #define SH_DECLARE
 #include "lib/simplehash.h"
 
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+struct ParallelTIDBitmap
+{
+	dsa_pointer dsa_data;		/* dsa pointers for all kind of pages */
+	pagetable_hash pagetable;	/* hash table of PagetableEntry's */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	int			dsa_entries;	/* total item in dsa_pages */
+};
 
 /*
  * tbm_create - create an initially-empty bitmap
@@ -244,7 +254,18 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	if (tbm->is_shared)
+	{
+		pagetable_alloc allocator;
+
+		allocator.args = tbm;
+		allocator.HashAlloc = tbm_alloc_shared;
+		allocator.HashFree = tbm_free_shared;
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, &allocator);
+		tbm->dsa_entries = tbm->pagetable->size;
+	}
+	else
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -1061,3 +1082,136 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * tbm_update_shared_members
+ *
+ * Store leader's private tbm state to shared location. This must
+ * be called before waking up other workers.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTIDBitmap *parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before waking up the other
+	 * workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->dsa_data = tbm->dsa_data;
+	parallel_tbm->dsa_entries = tbm->dsa_entries;
+	pagetable_copy(tbm->pagetable, &parallel_tbm->pagetable, NULL);
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that it should create pagetable
+ * in shared memory using DSA.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->is_shared = true;
+	tbm->area = (dsa_area *) area;
+}
+
+/*
+ * tbm_estimate_parallel_tidbitmap
+ *
+ * Estimate size of shared TIDBitmap related info.
+ */
+Size
+tbm_estimate_parallel_tidbitmap(Size size)
+{
+	return add_size(size, sizeof(ParallelTIDBitmap));
+}
+
+/*
+ * tbm_attach
+ *
+ * Create a local TIDBitmap for worker and Attach it to shared TID bitmap
+ * created by leader.
+ */
+TIDBitmap *
+tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area)
+{
+	TIDBitmap  *tbm = makeNode(TIDBitmap);
+	PagetableEntry *dsa_entry;
+
+	tbm->mcxt = CurrentMemoryContext;
+	tbm->status = TBM_HASH;
+	tbm->nchunks = parallel_tbm->nchunks;
+	tbm->nentries = parallel_tbm->nentries;
+	tbm->npages = parallel_tbm->npages;
+	tbm->maxentries = parallel_tbm->maxentries;
+	tbm->dsa_data = parallel_tbm->dsa_data;
+	tbm->dsa_entries = parallel_tbm->dsa_entries;
+	dsa_entry = dsa_get_address(area, tbm->dsa_data);
+	dsa_entry = (void *) (((char *) dsa_entry) + sizeof(dsa_pointer));
+	tbm->pagetable = palloc0(sizeof(struct pagetable_hash));
+	pagetable_copy(&parallel_tbm->pagetable, tbm->pagetable, dsa_entry);
+
+	/* Mark the tbm as parallel and also keep the DSA reference in it. */
+	tbm_set_parallel(tbm, area);
+
+	return tbm;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Memory allocation function for bitmap hash.
+ * It allocates memory from DSA and also stores dsa_pointer in the memory
+ * so that during free we can directly get the dsa_pointer and free it.
+ */
+void *
+tbm_alloc_shared(Size size, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->area, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	/* Keep track of actual number of entries */
+	if (tbm->pagetable)
+		tbm->dsa_entries = tbm->pagetable->size;
+
+	ptr = dsa_get_address(tbm->area, dsaptr);
+	memset(ptr, 0, size + sizeof(dsa_pointer));
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * tbm_free_shared
+ *
+ * Memory free function for tid bitmap
+ */
+void
+tbm_free_shared(void *pointer, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsa_data;
+
+	/*
+	 * If TBM is in iterating phase means pagetable was already created and we
+	 * have come here during tbm_free. So we need not to do anything because
+	 * by this time DSA would have been already freed.
+	 */
+	if (tbm->iterating)
+		return;
+
+	dsa_data = *((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+	dsa_free(tbm->area, dsa_data);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 5c18987..a2f6fbb 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2875,6 +2875,31 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 }
 
 /*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+/*
  * Compute the number of parallel workers that should be used to scan a
  * relation.  "pages" is the number of pages from the relation that we
  * expect to scan.
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a43daa7..f81e469 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -816,6 +816,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -877,8 +878,32 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
+
+		/*
+		 * It may be possible to amortize some of the I/O cost, but probably
+		 * not very much, because most operating systems already do aggressive
+		 * prefetching.  For now, we assume that the disk run cost can't be
+		 * amortized at all.
+		 */
+
+		/*
+		 * In the case of a parallel plan, the row count needs to represent
+		 * the number of tuples processed per worker.
+		 */
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 5283468..630c501 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1557,6 +1561,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1599,6 +1608,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index fae1f67..221b5f1 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2834,7 +2834,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f440875..20b9a59 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3258,7 +3259,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 7176cf1..f8eef0b 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3392,6 +3392,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index ee7e05a..d7ebac4 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -178,6 +178,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index f9bcdd6..384a724 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1423,6 +1425,72 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+/*
+ *	Stores the information about current position of the
+ *	shared iterator used for parallel bitmap heap scan.
+ */
+typedef struct
+{
+	slock_t		mutex;			/* mutual exclusion for below three fields */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+}	ParallelIterator;
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	ParallelIterator tbmiterator;
+	ParallelIterator prefetch_iterator;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	Size		ptbm_offset;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1437,6 +1505,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1453,6 +1523,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1468,7 +1539,9 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1483,6 +1556,8 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..77d0fc6 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -31,9 +31,6 @@
  */
 typedef struct TIDBitmap TIDBitmap;
 
-/* Likewise, TBMIterator is private */
-typedef struct TBMIterator TBMIterator;
-
 /* Result structure for tbm_iterate */
 typedef struct
 {
@@ -44,6 +41,26 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * When iterating over a bitmap in sorted order, a TBMIterator is used to
+ * track our progress.  There can be several iterators scanning the same
+ * bitmap concurrently.  Note that the bitmap becomes read-only as soon as
+ * any iterator is created.
+ */
+typedef struct
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+} TBMIterator;
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTIDBitmap ParallelTIDBitmap;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -62,5 +79,12 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+						   ParallelTIDBitmap *stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+						  ParallelTIDBitmap *parallel_tbm);
+void		tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tidbitmap(Size size);
+TIDBitmap *tbm_attach(ParallelTIDBitmap *parallel_tbm, void *area);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 7b41317..cfef818 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 81a9be7..968d39a 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index de8225b..c52c864 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -786,6 +786,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#54Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#53)
Re: Parallel bitmap heap scan

On Tue, Jan 31, 2017 at 6:05 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

0002-hash-support-alloc-free-v14.patch:

+ if (tb->alloc)
+ {

The memory for tb->alloc is allocated always, is the if check still
required?

In parallel case, only first worker will call SH_CREATE, other worker
will only do palloc for pagetable and copy the reference from main
worker, so they will not have allocator.

When Andres wrote in
/messages/by-id/20161017201558.cr34stc6zvxy3mst@alap3.anarazel.de
that you should try to share only the iteration arrays, I believe that
he meant the results of tbm_begin_iterate() -- that is,
iterator->spageptr, chunkptr, schunkbit, spages, and schunks. I'm
perhaps putting words into his mouth, but what he said was that we
could avoid sharing "the whole underlying hash". But the patch set
you've implemented here doesn't behave that way. Instead, you've got
the array containing the hash table elements shared (which is good)
plus there's a sort of dummy hash table in every worker which copies
some but not all of the members of the original hash table, leading to
the otherwise-unnecessary if-test that Haribabu is complaining about.
So the hash table is kinda-shared-kinda-not-shared, which I don't
*think* is really what Andres had in mind.

In short, I think SH_COPY (implemented here as pagetable_copy) needs
to go away. The work of tbm_begin_iterate() should be done before we
begin the shared scan and the results of that work should be shared.
What happens right now (it appears) is that every backend does that
work based on the same hash table and we just assume they all get the
same answer. And we really do assume that, because
pbms_parallel_iterate() assumes it can shuttle private state back and
forth between iterator in different backends and nothing will break;
but those backends aren't actually using the same iteration arrays.
They are using different iteration arrays that should have the same
contents because they were all derived from the same semi-shared hash
table. That's pretty fragile, and a waste of CPU cycles if the hash
table is large (every backend does its own sort).

On a related note, I think it's unacceptable to make the internal
details of TBMIterator public. You've moved it from tidbitmap.c to
tidbitmap.h so that nodeBitmapHeapScan.c can tinker with the guts of
the TBMIterator, but that's not OK. Those details need to remain
private to tidbitmap.c. pbms_parallel_iterate() is a nasty kludge; we
need some better solution. The knowledge of how a shared iterator
should iterate needs to be private to tidbitmap.c, not off in the
executor someplace. And I think the entries need to actually be
updated in shared memory directly, not copied back and forth between a
backend-private iterator and a shared iterator.

Also, pbms_parallel_iterate() can't hold a spinlock around a call to
tbm_iterate(). Note the coding rules for spinlocks mentioned in
spin.h and src/backend/storage/lmgr/README. I think the right thing
to do here is to use an LWLock in a new tranche (add an entry to
BuiltinTrancheIds).

In 0002, you have this:

+ tb->alloc = palloc(sizeof(SH_ALLOCATOR));

This should be using MemoryContextAlloc(ctx, ...) rather than palloc.
Otherwise the allocator object is in a different context from
everything else in the hash table. But TBH, I don't really see why we
want this to be a separate object. Why not just put
HashAlloc/HashFree/args into SH_TYPE directly? That avoids some
pointer chasing and doesn't really seem to cost anything (except that
SH_CREATE will grow a slightly longer argument sequence).

I think maybe we should rename the functions to element_allocate,
element_free, and element_allocator_ctx, or something like that. The
current names aren't capitalized consistently with other things in
this module, and putting the word "element" in there would make it
more clear what the purpose of this thing is.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#55Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#54)
Re: Parallel bitmap heap scan

On Wed, Feb 1, 2017 at 11:09 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Hi Robert,

Thanks for the review.

When Andres wrote in
/messages/by-id/20161017201558.cr34stc6zvxy3mst@alap3.anarazel.de
that you should try to share only the iteration arrays, I believe that
he meant the results of tbm_begin_iterate() -- that is,
iterator->spageptr, chunkptr, schunkbit, spages, and schunks. I'm
perhaps putting words into his mouth, but what he said was that we
could avoid sharing "the whole underlying hash". But the patch set
you've implemented here doesn't behave that way. Instead, you've got
the array containing the hash table elements shared (which is good)

Here is my analysis why we selected this instead of just sharing the iterator:
----------------------------------------------------------------------------------------------------
The problem is that each entry of iteration array is just a pointer to
hash entry. So if we only shared iterator then iterator element will
be pointing to local memory of other workers. I thought of one more
option that during tbm_begin_iterate instead of keeping pointer to
hash entry, we can make a copy of that in DSA area, but that will have
2 copies of hash table element for short duration which is not good.
And, I think what we are doing currently is better than that.

The work of tbm_begin_iterate() should be done before we

begin the shared scan and the results of that work should be shared.
What happens right now (it appears) is that every backend does that
work based on the same hash table and we just assume they all get the
same answer.

Actually, tbm_begin_iterate is processing the each element of the hash
table and converting to chunk and page array and currently, it’s done
by each worker and it’s pretty cheap. Suppose I try to do this only by
the first worker then implementation will look something like this.

1. First, we need to create a tbm->spages array and tbm->schunks array
and both should be the array of dsa_pointers.
2. Then each worker will process these array’s and will convert them
to the array of their local pointers.
3. With the current solution where all hash elements are stored in one
large dsa chunk, then how we are going to divide them into multiple
dsa pointers.

I will work on remaining comments.

And we really do assume that, because

pbms_parallel_iterate() assumes it can shuttle private state back and
forth between iterator in different backends and nothing will break;
but those backends aren't actually using the same iteration arrays.
They are using different iteration arrays that should have the same
contents because they were all derived from the same semi-shared hash
table. That's pretty fragile, and a waste of CPU cycles if the hash
table is large (every backend does its own sort).

On a related note, I think it's unacceptable to make the internal
details of TBMIterator public. You've moved it from tidbitmap.c to
tidbitmap.h so that nodeBitmapHeapScan.c can tinker with the guts of
the TBMIterator, but that's not OK. Those details need to remain
private to tidbitmap.c. pbms_parallel_iterate() is a nasty kludge; we
need some better solution. The knowledge of how a shared iterator
should iterate needs to be private to tidbitmap.c, not off in the
executor someplace. And I think the entries need to actually be
updated in shared memory directly, not copied back and forth between a
backend-private iterator and a shared iterator.

Also, pbms_parallel_iterate() can't hold a spinlock around a call to
tbm_iterate(). Note the coding rules for spinlocks mentioned in
spin.h and src/backend/storage/lmgr/README. I think the right thing
to do here is to use an LWLock in a new tranche (add an entry to
BuiltinTrancheIds).

In 0002, you have this:

+ tb->alloc = palloc(sizeof(SH_ALLOCATOR));

This should be using MemoryContextAlloc(ctx, ...) rather than palloc.
Otherwise the allocator object is in a different context from
everything else in the hash table. But TBH, I don't really see why we
want this to be a separate object. Why not just put
HashAlloc/HashFree/args into SH_TYPE directly? That avoids some
pointer chasing and doesn't really seem to cost anything (except that
SH_CREATE will grow a slightly longer argument sequence).

I think maybe we should rename the functions to element_allocate,
element_free, and element_allocator_ctx, or something like that. The
current names aren't capitalized consistently with other things in
this module, and putting the word "element" in there would make it
more clear what the purpose of this thing is.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#56Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#54)
2 attachment(s)
Re: Parallel bitmap heap scan

On Wed, Feb 1, 2017 at 11:09 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Here are the latest version of the patch, which address all the
comments given by Robert.

/messages/by-id/20161017201558.cr34stc6zvxy3mst@alap3.anarazel.de
that you should try to share only the iteration arrays, I believe that
he meant the results of tbm_begin_iterate() -- that is,
iterator->spageptr, chunkptr, schunkbit, spages, and schunks. I'm
perhaps putting words into his mouth, but what he said was that we
could avoid sharing "the whole underlying hash". But the patch set
you've implemented here doesn't behave that way. Instead, you've got
the array containing the hash table elements shared (which is good)
plus there's a sort of dummy hash table in every worker which copies
some but not all of the members of the original hash table, leading to
the otherwise-unnecessary if-test that Haribabu is complaining about.
So the hash table is kinda-shared-kinda-not-shared, which I don't
*think* is really what Andres had in mind.

In short, I think SH_COPY (implemented here as pagetable_copy) needs
to go away. The work of tbm_begin_iterate() should be done before we
begin the shared scan and the results of that work should be shared.

I have removed the SH_COPY and now leader performs the
tbm_begin_shared_iterate before waking up the worker. Basically, the
leader will create the page and chunk array and that is the array of
the "relptr" (offlist, suggested by Robert).

What happens right now (it appears) is that every backend does that
work based on the same hash table and we just assume they all get the
same answer. And we really do assume that, because
pbms_parallel_iterate() assumes it can shuttle private state back and
forth between iterator in different backends and nothing will break;
but those backends aren't actually using the same iteration arrays.
They are using different iteration arrays that should have the same
contents because they were all derived from the same semi-shared hash
table. That's pretty fragile, and a waste of CPU cycles if the hash
table is large (every backend does its own sort).

On a related note, I think it's unacceptable to make the internal
details of TBMIterator public. You've moved it from tidbitmap.c to
tidbitmap.h so that nodeBitmapHeapScan.c can tinker with the guts of
the TBMIterator, but that's not OK. Those details need to remain
private to tidbitmap.c.

Fixed

pbms_parallel_iterate() is a nasty kludge; we

need some better solution. The knowledge of how a shared iterator
should iterate needs to be private to tidbitmap.c, not off in the
executor someplace. And I think the entries need to actually be
updated in shared memory directly, not copied back and forth between a
backend-private iterator and a shared iterator.

I have fixed this, now there is new function called tbm_shared_iterate
which will directly iterate using shared iterator. So now no need to
copy member back and forth between local and shared iterator.

Also, pbms_parallel_iterate() can't hold a spinlock around a call to
tbm_iterate(). Note the coding rules for spinlocks mentioned in
spin.h and src/backend/storage/lmgr/README. I think the right thing
to do here is to use an LWLock in a new tranche (add an entry to
BuiltinTrancheIds).

Done that way.

In 0002, you have this:

+ tb->alloc = palloc(sizeof(SH_ALLOCATOR));

This should be using MemoryContextAlloc(ctx, ...) rather than palloc.
Otherwise the allocator object is in a different context from
everything else in the hash table. But TBH, I don't really see why we
want this to be a separate object. Why not just put
HashAlloc/HashFree/args into SH_TYPE directly? That avoids some
pointer chasing and doesn't really seem to cost anything (except that
SH_CREATE will grow a slightly longer argument sequence).

Done as suggested

I think maybe we should rename the functions to element_allocate,
element_free, and element_allocator_ctx, or something like that. The
current names aren't capitalized consistently with other things in
this module, and putting the word "element" in there would make it
more clear what the purpose of this thing is.

Fixed as per the suggestion

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0002-hash-support-alloc-free-v16.patchapplication/octet-stream; name=0002-hash-support-alloc-free-v16.patchDownload
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 47c9656..2d9ce93 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -330,7 +330,8 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
 	else
 		hashtable->hash_iv = 0;
 
-	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
+	hashtable->hashtab =
+			tuplehash_create(tablecxt, nbuckets, NULL, NULL, NULL);
 	hashtable->hashtab->private_data = hashtable;
 
 	return hashtable;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 7b31948..36102b5 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -244,7 +244,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL, NULL, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index 12aedbc..8c3e726 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -90,6 +90,12 @@
 /* generate forward declarations necessary to use the hash table */
 #ifdef SH_DECLARE
 
+/* Allocation function for hash table elements */
+typedef void *(*element_allocate) (Size size, void *args);
+
+/* Free function */
+typedef void (*element_free) (void *pointer, void *args);
+
 /* type definitions */
 typedef struct SH_TYPE
 {
@@ -112,8 +118,17 @@ typedef struct SH_TYPE
 	/* hash buckets */
 	SH_ELEMENT_TYPE *data;
 
+	/* Allocation function for hash table elements */
+	element_allocate alloc;
+
+	/* Free function */
+	element_free free;
+
+	/* Arguments to be passed to allocation and free functions */
+	void	   *args;
+
 	/* memory context to use for allocations */
-	MemoryContext ctx;
+	MemoryContext element_allocator_ctx;
 
 	/* user defined data, useful for callbacks */
 	void	   *private_data;
@@ -133,7 +148,8 @@ typedef struct SH_ITERATOR
 } SH_ITERATOR;
 
 /* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
+		  element_allocate alloc, element_free free, void *args);
 SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
 SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
 SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -276,27 +292,62 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
 #endif
 }
 
+/* default memory allocator function */
+static void *
+SH_DEFAULT_ALLOC(Size size, void *args)
+{
+	MemoryContext context = (MemoryContext) args;
+
+	return MemoryContextAllocExtended(context, size,
+									  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+}
+
+/* default memory free function */
+static void
+SH_DEFAULT_FREE(void *pointer, void *args)
+{
+	pfree(pointer);
+}
+
 /*
  * Create a hash table with enough space for `nelements` distinct members,
  * allocating required memory in the passed-in context.
+ *
+ * WARNING : alloc is an allocator handle which provides memory allocator
+ * function for storing hash elements. User needs to be aware that providing
+ * allocator does not store complete hash table in the allocator memory i.e
+ * hash table will still be allocated in local memory. However all the
+ * elements will be stored in the memory provided by the allocator.
  */
 SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements)
+SH_CREATE(MemoryContext ctx, uint32 nelements, element_allocate alloc,
+		  element_free free, void *args)
 {
-	SH_TYPE    *tb;
-	uint64		size;
+	SH_TYPE     *tb;
+	uint64		 size;
 
 	tb = MemoryContextAllocZero(ctx, sizeof(SH_TYPE));
-	tb->ctx = ctx;
+	tb->element_allocator_ctx = ctx;
 
 	/* increase nelements by fillfactor, want to store nelements elements */
 	size = Min((double) SH_MAX_SIZE, ((double) nelements) / SH_FILLFACTOR);
 
 	SH_COMPUTE_PARAMETERS(tb, size);
 
-	tb->data = MemoryContextAllocExtended(tb->ctx,
-										  sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	if (!alloc)
+	{
+		tb->alloc = (element_allocate) SH_DEFAULT_ALLOC;
+		tb->free = (element_free) SH_DEFAULT_FREE;
+		tb->args = (void *) ctx;
+	}
+	else
+	{
+		tb->alloc = alloc;
+		tb->free = free;
+		tb->args = args;
+	}
+
+	tb->data = tb->alloc(sizeof(SH_ELEMENT_TYPE) * tb->size, tb->args);
 
 	return tb;
 }
@@ -305,7 +356,7 @@ SH_CREATE(MemoryContext ctx, uint32 nelements)
 SH_SCOPE void
 SH_DESTROY(SH_TYPE *tb)
 {
-	pfree(tb->data);
+	tb->free(tb->data, tb->args);
 	pfree(tb);
 }
 
@@ -333,9 +384,7 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 	/* compute parameters for new table */
 	SH_COMPUTE_PARAMETERS(tb, newsize);
 
-	tb->data = MemoryContextAllocExtended(
-								 tb->ctx, sizeof(SH_ELEMENT_TYPE) * tb->size,
-										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+	tb->data = tb->alloc(sizeof(SH_ELEMENT_TYPE) * tb->size, tb->args);
 
 	newdata = tb->data;
 
@@ -421,7 +470,7 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 		}
 	}
 
-	pfree(olddata);
+	tb->free(olddata, tb->args);
 }
 
 /*
0003-parallel-bitmap-heap-scan-v16.patchapplication/octet-stream; name=0003-parallel-bitmap-heap-scan-v16.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 01fad38..2af778a 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1240,6 +1240,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1ce42ea..76a99a1 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1753,6 +1753,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index e01fe6d..090aaf3 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -29,6 +29,7 @@
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
 #include "executor/tqueue.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
@@ -205,6 +206,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -257,6 +262,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -733,6 +743,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index f18827d..b806cb5 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,14 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo * pbminfo);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -68,11 +72,18 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	TBMIterator *tbmiterator;
 	TBMIterateResult *tbmres;
 
+	ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelTBMInfo *parallel_tbm = NULL;
+
+	/* Get the parallel TBM address in shared memory using offset */
+	if (pbminfo)
+		parallel_tbm = (ParallelTBMInfo *) ((char *) pbminfo +
+											pbminfo->tbm_offset);
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,6 +98,35 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	prefetch_iterator = node->prefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	The first worker to see the state of ParallelBitmapInfo as PBM_INITIAL
+	 *	becomes leader and sets the state to PBM_INPROGRESS. All other workers
+	 *	see the state as PBM_INPROGRESS, and will wait for leader to finish
+	 *	building the TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *		ParallelBitmapInfo so that other worker can see those.
+	 *	  Convert the TIDBitmap into shared chunk and page array
+	 *	  Set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader creates shared TIDBitmap.
+	 *	  Copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over shared page and chunk
+	 *			array and select heap pages one by one.
+	 *	  b) Since multiple workers are iterating over same page and chunk
+	 *			array we need to have a shared iterator, so we grab a LWLock
+	 *			and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,36 +141,111 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.
+		 *
+		 * In parallel mode leader will immediately come out of the function,
+		 * but all other worker will be blocked until leader wake them up.
+		 */
+		if (pbminfo == NULL || pbms_is_leader(pbminfo))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag in lower level bitmap index scan. This
+			 * flag will be used by bitmap index node to identify whether it
+			 * needs to create a shared pagetable or local pagetable.
+			 */
+			if (pbminfo)
+				pbms_set_parallel(outerPlanState(node));
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			/*
+			 * Copy the local TIDBitmap information to the shared location so
+			 * that other workers can use them.
+			 */
+			if (pbminfo)
+				tbm_update_shared_members(tbm, parallel_tbm);
+		}
+		else
+		{
+			/*
+			 * By this time leader has already created the shared TBM. Here we
+			 * need to create a local TBM and copy information from shared
+			 * location.
+			 */
+			tbm = tbm_attach(parallel_tbm, node->ss.ps.state->es_query_dsa);
+		}
 
 		node->tbm = tbm;
-		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+
+		/*
+		 * If we are running in parallel mode then initialize a shared
+		 * iterator otherwise a local iterator.
+		 */
+		if (pbminfo)
+			node->tbmiterator = tbmiterator =
+				tbm_begin_shared_iterate(tbm, parallel_tbm, false);
+		else
+			node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+
 		node->tbmres = tbmres = NULL;
 
 #ifdef USE_PREFETCH
 		if (node->prefetch_maximum > 0)
 		{
-			node->prefetch_iterator = prefetch_iterator = tbm_begin_iterate(tbm);
+			if (pbminfo)
+				node->prefetch_iterator = prefetch_iterator =
+					tbm_begin_shared_iterate(tbm, parallel_tbm, true);
+			else
+				node->prefetch_iterator = prefetch_iterator =
+					tbm_begin_iterate(tbm);
+
 			node->prefetch_pages = 0;
 			node->prefetch_target = -1;
 		}
 #endif   /* USE_PREFETCH */
+
+		/*
+		 * copy local TBM information in shared memory before waking up the
+		 * other workers. Other workers will create there own TBM and copy
+		 * information from shared memory.
+		 */
+		if (pbminfo)
+		{
+			/* Change the state under a lock */
+			SpinLockAcquire(&pbminfo->state_mutex);
+			pbminfo->state = PBM_FINISHED;
+			SpinLockRelease(&pbminfo->state_mutex);
+
+			/* Wake up all other workers. */
+			ConditionVariableBroadcast(&pbminfo->cv);
+		}
 	}
 
 	for (;;)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
+		int		   *prefetch_target;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			/*
+			 * If we are in parallel mode perform shared iterate otherwise
+			 * local iterate.
+			 */
+			if (!pbminfo)
+				node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			else
+				node->tbmres = tbmres = tbm_shared_iterate(tbmiterator);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +253,50 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			/*
+			 * If we are in parallel mode then acquire prefetch_mutex and
+			 * check prefetch_pages from shared location.
+			 */
+			if (pbminfo)
 			{
+				SpinLockAcquire(&pbminfo->prefetch_mutex);
+
 				/* The main iterator has closed the distance by one page */
-				node->prefetch_pages--;
+				if (pbminfo->prefetch_pages > 0)
+					pbminfo->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&pbminfo->prefetch_mutex);
 			}
-			else if (prefetch_iterator)
+			else
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				/* The main iterator has closed the distance by one page */
+				 if (node->prefetch_pages > 0)
+					 node->prefetch_pages--;
+				else
+					need_prefetch = true;
+			}
+
+			if (prefetch_iterator && need_prefetch)
+			{
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				if (!pbminfo)
+					tbmpre = tbm_iterate(prefetch_iterator);
+				else
+					tbmpre = tbm_shared_iterate(prefetch_iterator);
+
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind the main iterator, but we can not
+				 * ensure that the current blockno in the main iterator and
+				 * in the prefetch iterator is same. It's possible that
+				 * whatever blockno we are prefetching is getting processed
+				 * by some other worker.
+				 */
+				if ((pbminfo == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -183,19 +331,40 @@ BitmapHeapNext(BitmapHeapScanState *node)
 #ifdef USE_PREFETCH
 
 			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
+			 * If we are in shared mode then use prefetch_target from shared
+			 * location i.e pbminfo otherwise directly from node.
 			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
+			if (pbminfo == NULL)
+				prefetch_target = &node->prefetch_target;
 			else
-				node->prefetch_target++;
+				prefetch_target = &pbminfo->prefetch_target;
+
+			/* Increase prefetch target if it's not yet at the max. */
+			if (*prefetch_target < node->prefetch_maximum)
+			{
+				/* If we are in parallel mode then grab prefetch_mutex */
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +380,25 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (pbminfo == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				if (pbminfo->prefetch_target < node->prefetch_maximum)
+				{
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+					if (pbminfo->prefetch_target < node->prefetch_maximum)
+						pbminfo->prefetch_target++;
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+				}
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +422,59 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (pbminfo)
+			{
+				prefetch_pages = &pbminfo->prefetch_pages;
+				prefetch_target = pbminfo->prefetch_target;
+			}
+			else
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
+
+			/*
+			 * We are checking the prefetch_pages without mutex. Henceforth in
+			 * case of parallel mode there can be some extra prefetch. Should
+			 * we acquire mutex and recheck before iterating?
+			 */
+			while (*prefetch_pages < prefetch_target)
+			{
+				TBMIterateResult *tbmpre;
+
+				if (!pbminfo)
+					tbmpre = tbm_iterate(prefetch_iterator);
+				else
+					tbmpre = tbm_shared_iterate(prefetch_iterator);
 
 				if (tbmpre == NULL)
 				{
 					/* No more pages to prefetch */
 					tbm_end_iterate(prefetch_iterator);
+
 					node->prefetch_iterator = prefetch_iterator = NULL;
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (pbminfo != NULL)
+					SpinLockAcquire(&pbminfo->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (pbminfo != NULL)
+					SpinLockRelease(&pbminfo->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -465,6 +689,16 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -567,6 +801,7 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -651,3 +886,168 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ * pbms_is_leader
+ *
+ * Check if we are the first one to come here, if yes then
+ * we become leader, otherwise we need to wait until leader
+ * create the shared TID bitmap and wake us up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo * pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : then we need to wait till leader creates BITMAP
+		 * PBM_FINISHED   : BITMAP is ready so no need to wait.
+		 *---------------
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/*-------------------
+ * pbms_set_parallel
+ *
+ * Parallel bitmap heap scan set below index scan node as parallel
+ * so that it can create shared TIDBitmap.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapOr or BitmapAnd set first bitmap index scan node as
+	 * parallel, because only first node will create the main bitmap other
+	 * bitmaps will be merged to the first bitmap so no need to create them in
+	 * shared memory.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			pbms_set_parallel(((BitmapOrState *) node)->bitmapplans[0]);
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	/* Estimate the size for sharing parallel TBM info. */
+	node->pscan_len = tbm_estimate_parallel_tbminfo(offset);
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	ParallelTBMInfo *parallel_tbm;
+	EState	   *estate = node->ss.ps.state;
+	Size		offset = add_size(offsetof(ParallelBitmapInfo,
+										   phs_snapshot_data),
+								  EstimateSnapshotSpace(estate->es_snapshot));
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Offset to parallel TBM info. */
+	pbminfo->tbm_offset = offset;
+
+	parallel_tbm = (ParallelTBMInfo *) (((char *) pbminfo) + offset);
+	tbm_init_shared_iterator(parallel_tbm);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..56f8376 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -82,6 +82,13 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	}
 
 	/*
+	 * If parallel flag is set then set flag in TIDBitmap to indicate that we
+	 * need a shared page table.
+	 */
+	if (node->biss_Parallel)
+		tbm_set_parallel(tbm, node->ss.ps.state->es_query_dsa);
+
+	/*
 	 * Get TIDs from index and insert into bitmap
 	 */
 	while (doscan)
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 36102b5..bb321bd 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -42,7 +42,15 @@
 
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
+#include "nodes/execnodes.h"
 #include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
+#include "storage/lwlock.h"
+#include "storage/shmem.h"
+#include "storage/spin.h"
+#include "utils/dsa.h"
+#include "utils/hsearch.h"
+#include "utils/relptr.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -102,6 +110,8 @@ typedef struct PagetableEntry
 	bitmapword	words[Max(WORDS_PER_PAGE, WORDS_PER_CHUNK)];
 } PagetableEntry;
 
+relptr_declare(PagetableEntry, RelptrPagetableEntry);
+
 /*
  * We want to avoid the overhead of creating the hashtable, which is
  * comparatively large, when not necessary. Particularly when we are using a
@@ -136,9 +146,17 @@ struct TIDBitmap
 	bool		iterating;		/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
+	bool		is_shared;		/* need to build shared tbm if set */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	int			dsa_entries;	/* actual allocated entries */
+	dsa_area   *area;			/* reference to per-query shared memory area */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	char	   *base;			/* base pointer of the element array */
+	RelptrPagetableEntry *relpages;		/* page array of relptr */
+	RelptrPagetableEntry *relchunks;	/* chunk array of relptr */
+	ParallelTBMInfo *parallel_tbm;		/* reference to parallel TBM */
 };
 
 /*
@@ -153,9 +171,43 @@ struct TBMIterator
 	int			spageptr;		/* next spages index */
 	int			schunkptr;		/* next schunks index */
 	int			schunkbit;		/* next bit to check in current schunk */
+	TBMSharedIterator *shareditr;		/* reference to shared iterator info */
 	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
 };
 
+/*
+ * same as TBMIterator but, it will be allocated in shared memory. It also has
+ * LWLock to protect the shared members. However it don't have TBMIterateResult
+ * because results should be local for each worker. So for using shared
+ * iterator we need to use local TBMIterator and that will be wrapper around
+ * the shared iterator. Henceforth, shared member will be accessed from
+ * shared iterator and result will be in local iterator which is wrapper
+ * around shared iterator.
+ */
+struct TBMSharedIterator
+{
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	LWLock		lock;			/* lock to protect the shared access */
+};
+
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+struct ParallelTBMInfo
+{
+	dsa_pointer dsa_data;		/* dsa pointers for all kind of pages */
+	dsa_pointer spages;
+	dsa_pointer schunks;
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	int			dsa_entries;	/* total item in dsa_pages */
+	TBMSharedIterator tbmiterator;
+	TBMSharedIterator prefetch_iterator;
+};
 
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
@@ -168,6 +220,10 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+static int tbm_shared_comparator(const void *left, const void *right,
+					  void *arg);
+static void *tbm_alloc_shared(Size size, void *arg);
+static void tbm_free_shared(void *pointer, void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -198,7 +254,6 @@ hash_blockno(BlockNumber b)
 #define SH_DECLARE
 #include "lib/simplehash.h"
 
-
 /*
  * tbm_create - create an initially-empty bitmap
  *
@@ -244,7 +299,11 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL, NULL, NULL);
+	if (tbm->is_shared)
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, tbm_alloc_shared,
+										  tbm_free_shared, tbm);
+	else
+		tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL, NULL, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -664,6 +723,87 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	return iterator;
 }
 
+TBMIterator *
+tbm_begin_shared_iterate(TIDBitmap *tbm, ParallelTBMInfo * parallel_tbm,
+						 bool prefetch)
+{
+	TBMIterator *iterator;
+
+	/*
+	 * Create the TBMIterator struct, with enough trailing space to serve the
+	 * needs of the TBMIterateResult sub-struct.
+	 */
+	iterator = (TBMIterator *) palloc(sizeof(TBMIterator) +
+								 MAX_TUPLES_PER_PAGE * sizeof(OffsetNumber));
+	iterator->tbm = tbm;
+
+	if (prefetch)
+		iterator->shareditr = &parallel_tbm->prefetch_iterator;
+	else
+		iterator->shareditr = &parallel_tbm->tbmiterator;
+
+	/*
+	 * If we have a hashtable, create and fill the sorted page lists, unless
+	 * we already did that for a previous iterator.  Note that the lists are
+	 * attached to the bitmap not the iterator, so they can be used by more
+	 * than one iterator. This list is created in shared memory so that
+	 * multiple workers can access it and perform shared iterate.
+	 */
+	if (tbm->status == TBM_HASH && !tbm->iterating)
+	{
+		pagetable_iterator i;
+		PagetableEntry *page;
+		RelptrPagetableEntry *relpages;
+		RelptrPagetableEntry *relchunks;
+		int			npages;
+		int			nchunks;
+
+		/*
+		 * Create page list and chunk list using relptr so that we can share
+		 * this information across multiple workers.
+		 */
+		if (tbm->npages)
+			parallel_tbm->spages = dsa_allocate(tbm->area,
+							   tbm->npages * (sizeof(RelptrPagetableEntry)));
+		if (tbm->nchunks)
+			parallel_tbm->schunks = dsa_allocate(tbm->area,
+							  tbm->nchunks * (sizeof(RelptrPagetableEntry)));
+
+		relpages = dsa_get_address(tbm->area, parallel_tbm->spages);
+		relchunks = dsa_get_address(tbm->area, parallel_tbm->schunks);
+
+		npages = nchunks = 0;
+		pagetable_start_iterate(tbm->pagetable, &i);
+		while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+		{
+			if (page->ischunk)
+				relptr_store(tbm->base, relchunks[nchunks++], page);
+			else
+				relptr_store(tbm->base, relpages[npages++], page);
+		}
+
+		Assert(npages == tbm->npages);
+		Assert(nchunks == tbm->nchunks);
+		if (npages > 1)
+			qsort_arg(relpages, npages, sizeof(RelptrPagetableEntry *),
+					  tbm_shared_comparator, (void *) tbm->base);
+		if (nchunks > 1)
+			qsort_arg(relchunks, nchunks, sizeof(RelptrPagetableEntry *),
+					  tbm_shared_comparator, (void *) tbm->base);
+
+		iterator->schunkbit = 0;
+		iterator->schunkptr = 0;
+		iterator->spageptr = 0;
+	}
+
+	tbm->relpages = dsa_get_address(tbm->area, parallel_tbm->spages);
+	tbm->relchunks = dsa_get_address(tbm->area, parallel_tbm->schunks);
+
+	tbm->iterating = true;
+
+	return iterator;
+}
+
 /*
  * tbm_iterate - scan through next page of a TIDBitmap
  *
@@ -777,6 +917,125 @@ tbm_iterate(TBMIterator *iterator)
 }
 
 /*
+ *  tbm_shared_iterate - same as tbm_iterate. Only difference is it's in
+ *  shared memory and multiple worker operate on the shared iterator
+ *  under a lock.
+ */
+TBMIterateResult *
+tbm_shared_iterate(TBMIterator *tbmiterator)
+{
+	TBMSharedIterator *iterator = tbmiterator->shareditr;
+	TBMIterateResult *output = &(tbmiterator->output);
+	TIDBitmap  *tbm = tbmiterator->tbm;
+
+	Assert(tbm->iterating);
+
+	LWLockAcquire(&iterator->lock, LW_EXCLUSIVE);
+
+	/*
+	 * If lossy chunk pages remain, make sure we've advanced schunkptr/
+	 * schunkbit to the next set bit.
+	 */
+	while (iterator->schunkptr < tbm->nchunks)
+	{
+		PagetableEntry *chunk =
+				relptr_access(tbm->base, tbm->relchunks[iterator->schunkptr]);
+		int			schunkbit = iterator->schunkbit;
+
+		while (schunkbit < PAGES_PER_CHUNK)
+		{
+			int			wordnum = WORDNUM(schunkbit);
+			int			bitnum = BITNUM(schunkbit);
+
+			if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
+				break;
+			schunkbit++;
+		}
+		if (schunkbit < PAGES_PER_CHUNK)
+		{
+			iterator->schunkbit = schunkbit;
+			break;
+		}
+		/* advance to next chunk */
+		iterator->schunkptr++;
+		iterator->schunkbit = 0;
+	}
+
+	/*
+	 * If both chunk and per-page data remain, must output the numerically
+	 * earlier page.
+	 */
+	if (iterator->schunkptr < tbm->nchunks)
+	{
+		PagetableEntry *chunk =
+				relptr_access(tbm->base, tbm->relchunks[iterator->schunkptr]);
+		PagetableEntry *page =
+				relptr_access(tbm->base, tbm->relpages[iterator->spageptr]);
+
+		BlockNumber chunk_blockno;
+
+		chunk_blockno = chunk->blockno + iterator->schunkbit;
+		if (iterator->spageptr >= tbm->npages || chunk_blockno < page->blockno)
+		{
+			/* Return a lossy page indicator from the chunk */
+			output->blockno = chunk_blockno;
+			output->ntuples = -1;
+			output->recheck = true;
+			iterator->schunkbit++;
+
+			LWLockRelease(&iterator->lock);
+			return output;
+		}
+	}
+
+	if (iterator->spageptr < tbm->npages)
+	{
+		PagetableEntry *page;
+		int			ntuples;
+		int			wordnum;
+
+		/* In ONE_PAGE state, we don't allocate an spages[] array */
+		if (tbm->status == TBM_ONE_PAGE)
+			page = &tbm->entry1;
+		else
+			page = relptr_access(tbm->base, tbm->relpages[iterator->spageptr]);
+
+		/* scan bitmap to extract individual offset numbers */
+		ntuples = 0;
+		for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
+		{
+			bitmapword	w = page->words[wordnum];
+
+			if (w != 0)
+			{
+				int			off = wordnum * BITS_PER_BITMAPWORD + 1;
+
+				while (w != 0)
+				{
+					if (w & 1)
+						output->offsets[ntuples++] = (OffsetNumber) off;
+					off++;
+					w >>= 1;
+				}
+			}
+		}
+		output->blockno = page->blockno;
+		output->ntuples = ntuples;
+		output->recheck = page->recheck;
+
+		iterator->spageptr++;
+
+		LWLockRelease(&iterator->lock);
+		return output;
+	}
+
+	LWLockRelease(&iterator->lock);
+
+	/* Nothing more in the bitmap */
+	return NULL;
+}
+
+/*
  * tbm_end_iterate - finish an iteration over a TIDBitmap
  *
  * Currently this is just a pfree, but it might do more someday.  (For
@@ -1061,3 +1320,169 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * same as tbm_comparator, Only difference is that this will get relptr
+ * of PagetableEntry and it need to get actual PagetableEntry using relptr.
+ */
+static int
+tbm_shared_comparator(const void *left, const void *right, void *arg)
+{
+	PagetableEntry *lpage;
+	PagetableEntry *rpage;
+
+	lpage = relptr_access((char *) arg, *((RelptrPagetableEntry *) left));
+	rpage = relptr_access((char *) arg, *((RelptrPagetableEntry *) right));
+
+	if (lpage->blockno < rpage->blockno)
+		return -1;
+	else if (lpage->blockno > rpage->blockno)
+		return 1;
+	return 0;
+}
+
+/*
+ * tbm_update_shared_members
+ *
+ * Store leader's private tbm state to shared location. This must
+ * be called before waking up other workers.
+ */
+void
+tbm_update_shared_members(TIDBitmap *tbm, ParallelTBMInfo * parallel_tbm)
+{
+	/*
+	 * Copy private information to shared location before waking up the other
+	 * workers.
+	 */
+	parallel_tbm->maxentries = tbm->maxentries;
+	parallel_tbm->nchunks = tbm->nchunks;
+	parallel_tbm->nentries = tbm->nentries;
+	parallel_tbm->npages = tbm->npages;
+	parallel_tbm->dsa_data = tbm->dsa_data;
+	parallel_tbm->dsa_entries = tbm->dsa_entries;
+
+	tbm->base = dsa_get_address(tbm->area, tbm->dsa_data);
+}
+
+/*
+ * tbm_set_parallel
+ *
+ * Mark tidbitmap as shared and also store DSA area in it.
+ * marking tidbitmap as shared is indication that it should create pagetable
+ * in shared memory using DSA.
+ */
+void
+tbm_set_parallel(TIDBitmap *tbm, void *area)
+{
+	tbm->is_shared = true;
+	tbm->area = (dsa_area *) area;
+}
+
+/*
+ * tbm_estimate_parallel_tbminfo
+ *
+ * Estimate size of ParallelTBMInfo.
+ */
+Size
+tbm_estimate_parallel_tbminfo(Size size)
+{
+	return add_size(size, sizeof(ParallelTBMInfo));
+}
+
+/*
+ *	tbm_init_shared_iterator
+ *
+ *  Initializes the lwlocks for the shared iterators.
+ */
+void
+tbm_init_shared_iterator(ParallelTBMInfo * tbminfo)
+{
+	LWLockInitialize(&tbminfo->tbmiterator.lock,
+					 LWTRANCHE_PARALLEL_BITMAP_ITERATOR);
+
+	LWLockInitialize(&tbminfo->prefetch_iterator.lock,
+					 LWTRANCHE_PARALLEL_BITMAP_ITERATOR);
+}
+
+/*
+ * tbm_attach
+ *
+ * Create a local TIDBitmap for worker and Attach it to shared TID bitmap
+ * created by leader.
+ */
+TIDBitmap *
+tbm_attach(ParallelTBMInfo * parallel_tbm, void *area)
+{
+	TIDBitmap  *tbm = makeNode(TIDBitmap);
+
+	tbm->mcxt = CurrentMemoryContext;
+	tbm->status = TBM_HASH;
+	tbm->nchunks = parallel_tbm->nchunks;
+	tbm->nentries = parallel_tbm->nentries;
+	tbm->npages = parallel_tbm->npages;
+	tbm->maxentries = parallel_tbm->maxentries;
+	tbm->dsa_data = parallel_tbm->dsa_data;
+	tbm->dsa_entries = parallel_tbm->dsa_entries;
+	tbm->base = dsa_get_address(area, tbm->dsa_data);
+	tbm->iterating = true;
+
+	/* Mark the tbm as parallel and also keep the DSA reference in it. */
+	tbm_set_parallel(tbm, area);
+
+	return tbm;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Memory allocation function for bitmap hash.
+ * It allocates memory from DSA and also stores dsa_pointer in the memory
+ * so that during free we can directly get the dsa_pointer and free it.
+ */
+static void *
+tbm_alloc_shared(Size size, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->area, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	/* Keep track of actual number of entries */
+	if (tbm->pagetable)
+		tbm->dsa_entries = tbm->pagetable->size;
+
+	ptr = dsa_get_address(tbm->area, dsaptr);
+	memset(ptr, 0, size + sizeof(dsa_pointer));
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * tbm_free_shared
+ *
+ * Memory free function for tid bitmap
+ */
+static void
+tbm_free_shared(void *pointer, void *arg)
+{
+	TIDBitmap  *tbm = arg;
+	dsa_pointer dsa_data;
+
+	/*
+	 * If TBM is in iterating phase means pagetable was already created and we
+	 * have come here during tbm_free. So we need not to do anything because
+	 * by this time DSA would have been already freed.
+	 */
+	if (tbm->iterating)
+		return;
+
+	dsa_data = *((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+	dsa_free(tbm->area, dsa_data);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 5c18987..a2f6fbb 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2875,6 +2875,31 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 }
 
 /*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+/*
  * Compute the number of parallel workers that should be used to scan a
  * relation.  "pages" is the number of pages from the relation that we
  * expect to scan.
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a43daa7..f81e469 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -816,6 +816,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -877,8 +878,32 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
+
+		/*
+		 * It may be possible to amortize some of the I/O cost, but probably
+		 * not very much, because most operating systems already do aggressive
+		 * prefetching.  For now, we assume that the disk run cost can't be
+		 * amortized at all.
+		 */
+
+		/*
+		 * In the case of a parallel plan, the row count needs to represent
+		 * the number of tuples processed per worker.
+		 */
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 5283468..630c501 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1557,6 +1561,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1599,6 +1608,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index fae1f67..221b5f1 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2834,7 +2834,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f440875..20b9a59 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3258,7 +3259,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 7176cf1..f8eef0b 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3392,6 +3392,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index ee7e05a..d7ebac4 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -178,6 +178,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index f9bcdd6..59a3971 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1423,6 +1425,61 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		tbmiterator				main iterator
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_iterator		iterator for scanning ahead of current pages
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		ptbm_offset				offset in bytes of ParallelTIDBitmap.
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	Size		tbm_offset;
+	Size		itr_offset;
+	Size		prefetchitr_offset;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1437,6 +1494,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1453,6 +1512,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1468,7 +1528,9 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1483,6 +1545,8 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..647d7f4 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -33,6 +33,7 @@ typedef struct TIDBitmap TIDBitmap;
 
 /* Likewise, TBMIterator is private */
 typedef struct TBMIterator TBMIterator;
+typedef struct TBMSharedIterator TBMSharedIterator;
 
 /* Result structure for tbm_iterate */
 typedef struct
@@ -44,6 +45,11 @@ typedef struct
 	OffsetNumber offsets[FLEXIBLE_ARRAY_MEMBER];
 } TBMIterateResult;
 
+/*
+ * Holds shared members of TIDBitmap for parallel bitmap scan.
+ */
+typedef struct ParallelTBMInfo ParallelTBMInfo;
+
 /* function prototypes in nodes/tidbitmap.c */
 
 extern TIDBitmap *tbm_create(long maxbytes);
@@ -62,5 +68,16 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+						   ParallelTBMInfo * stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+						  ParallelTBMInfo * parallel_tbm);
+void		tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tbminfo(Size size);
+TIDBitmap  *tbm_attach(ParallelTBMInfo * parallel_tbm, void *area);
+TBMIterator *tbm_begin_shared_iterate(TIDBitmap *tbm,
+						 ParallelTBMInfo * parallel_tbm, bool prefetch);
+TBMIterateResult *tbm_shared_iterate(TBMIterator *iterator);
+void		tbm_init_shared_iterator(ParallelTBMInfo * tbminfo);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 7b41317..cfef818 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 81a9be7..968d39a 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index de8225b..c52c864 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -786,6 +786,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 8bd93c3..f1a2734 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -212,6 +212,7 @@ typedef enum BuiltinTrancheIds
 	LWTRANCHE_LOCK_MANAGER,
 	LWTRANCHE_PREDICATE_LOCK_MANAGER,
 	LWTRANCHE_PARALLEL_QUERY_DSA,
+	LWTRANCHE_PARALLEL_BITMAP_ITERATOR,
 	LWTRANCHE_FIRST_USER_DEFINED
 }	BuiltinTrancheIds;
 
#57Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#56)
Re: Parallel bitmap heap scan

On Sun, Feb 5, 2017 at 9:48 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

I think maybe we should rename the functions to element_allocate,
element_free, and element_allocator_ctx, or something like that. The
current names aren't capitalized consistently with other things in
this module, and putting the word "element" in there would make it
more clear what the purpose of this thing is.

Fixed as per the suggestion

Eh, not really. You changed the memory context to be called
element_allocator_ctx, rather than changing the args passed to the
element allocator to have that name, which is what I had in mind.

I did some assorted renaming and other cosmetic improvements and committed 0002.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#58Jeff Janes
jeff.janes@gmail.com
In reply to: Robert Haas (#57)
Re: Parallel bitmap heap scan

On Tue, Feb 7, 2017 at 1:03 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Sun, Feb 5, 2017 at 9:48 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

I think maybe we should rename the functions to element_allocate,
element_free, and element_allocator_ctx, or something like that. The
current names aren't capitalized consistently with other things in
this module, and putting the word "element" in there would make it
more clear what the purpose of this thing is.

Fixed as per the suggestion

Eh, not really. You changed the memory context to be called
element_allocator_ctx, rather than changing the args passed to the
element allocator to have that name, which is what I had in mind.

I did some assorted renaming and other cosmetic improvements and committed
0002.

I'm getting compiler errors:

In file included from execGrouping.c:47:
../../../src/include/lib/simplehash.h:91: error: redefinition of typedef
'simplehash_allocate'
../../../src/include/lib/simplehash.h:91: note: previous declaration of
'simplehash_allocate' was here
../../../src/include/lib/simplehash.h:92: error: redefinition of typedef
'simplehash_free'
../../../src/include/lib/simplehash.h:92: note: previous declaration of
'simplehash_free' was here

gcc version 4.4.7 20120313 (Red Hat 4.4.7-17) (GCC)

Cheers,

Jeff

#59Andres Freund
andres@anarazel.de
In reply to: Robert Haas (#57)
Re: Parallel bitmap heap scan

On 2017-02-07 16:03:43 -0500, Robert Haas wrote:

On Sun, Feb 5, 2017 at 9:48 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

I think maybe we should rename the functions to element_allocate,
element_free, and element_allocator_ctx, or something like that. The
current names aren't capitalized consistently with other things in
this module, and putting the word "element" in there would make it
more clear what the purpose of this thing is.

Fixed as per the suggestion

Eh, not really. You changed the memory context to be called
element_allocator_ctx, rather than changing the args passed to the
element allocator to have that name, which is what I had in mind.

I did some assorted renaming and other cosmetic improvements and committed 0002.

FWIW, I think it'd have been better to not add the new callbacks as
parameters to *_create(), but rather have them be "templatized" like the
rest of simplehash. That'd require that callback to check the context,
to know whether it should use shared memory or not, but that seems fine
to me. Right now this pushes the head of simplehash above a
cacheline...

I'm also doubtful about the naming of the default callbacks:
+/* default memory allocator function */
+static void *
+SH_DEFAULT_ALLOC(Size size, void *args)
+{
+   MemoryContext context = (MemoryContext) args;
+
+   return MemoryContextAllocExtended(context, size,
+                                     MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+}

SH_DEFAULT_ALLOC sounds like it's a name that's one of the prefixed
names (like SH_CREATE actually becomes pagetable_create and such) - but
afaics it's not. Which afaics means that this'll generate symbol
conflicts if one translation unit uses multiple simplehash.h style
hashes. Afaics these should either be prefixed, or static inline
functions.

- Andres

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#60Andres Freund
andres@anarazel.de
In reply to: Jeff Janes (#58)
Re: Parallel bitmap heap scan

On 2017-02-07 13:13:43 -0800, Jeff Janes wrote:

On Tue, Feb 7, 2017 at 1:03 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Sun, Feb 5, 2017 at 9:48 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

I think maybe we should rename the functions to element_allocate,
element_free, and element_allocator_ctx, or something like that. The
current names aren't capitalized consistently with other things in
this module, and putting the word "element" in there would make it
more clear what the purpose of this thing is.

Fixed as per the suggestion

Eh, not really. You changed the memory context to be called
element_allocator_ctx, rather than changing the args passed to the
element allocator to have that name, which is what I had in mind.

I did some assorted renaming and other cosmetic improvements and committed
0002.

I'm getting compiler errors:

In file included from execGrouping.c:47:
../../../src/include/lib/simplehash.h:91: error: redefinition of typedef
'simplehash_allocate'
../../../src/include/lib/simplehash.h:91: note: previous declaration of
'simplehash_allocate' was here
../../../src/include/lib/simplehash.h:92: error: redefinition of typedef
'simplehash_free'
../../../src/include/lib/simplehash.h:92: note: previous declaration of
'simplehash_free' was here

Oh yea, that too - you can't just redefine a typedef like that according
to C89 :(

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#61Robert Haas
robertmhaas@gmail.com
In reply to: Jeff Janes (#58)
Re: Parallel bitmap heap scan

On Tue, Feb 7, 2017 at 4:13 PM, Jeff Janes <jeff.janes@gmail.com> wrote:

I'm getting compiler errors:

In file included from execGrouping.c:47:
../../../src/include/lib/simplehash.h:91: error: redefinition of typedef
'simplehash_allocate'
../../../src/include/lib/simplehash.h:91: note: previous declaration of
'simplehash_allocate' was here
../../../src/include/lib/simplehash.h:92: error: redefinition of typedef
'simplehash_free'
../../../src/include/lib/simplehash.h:92: note: previous declaration of
'simplehash_free' was here

Thanks, I'll stick an #ifdef guard around that.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#62Robert Haas
robertmhaas@gmail.com
In reply to: Andres Freund (#59)
1 attachment(s)
Re: Parallel bitmap heap scan

On Tue, Feb 7, 2017 at 4:15 PM, Andres Freund <andres@anarazel.de> wrote:

FWIW, I think it'd have been better to not add the new callbacks as
parameters to *_create(), but rather have them be "templatized" like the
rest of simplehash. That'd require that callback to check the context,
to know whether it should use shared memory or not, but that seems fine
to me. Right now this pushes the head of simplehash above a
cacheline...

Something like the attached?

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

Attachments:

rework-simplehash-allocator.patchapplication/octet-stream; name=rework-simplehash-allocator.patchDownload
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 2d9ce93..47c9656 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -330,8 +330,7 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
 	else
 		hashtable->hash_iv = 0;
 
-	hashtable->hashtab =
-			tuplehash_create(tablecxt, nbuckets, NULL, NULL, NULL);
+	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
 	hashtable->hashtab->private_data = hashtable;
 
 	return hashtable;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 36102b5..7b31948 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -244,7 +244,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL, NULL, NULL);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index 359c7b8..eadf663 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -78,6 +78,8 @@
 #define SH_START_ITERATE_AT SH_MAKE_NAME(start_iterate_at)
 #define SH_ITERATE SH_MAKE_NAME(iterate)
 #define SH_STAT SH_MAKE_NAME(stat)
+#define SH_ALLOCATE SH_MAKE_NAME(allocate)
+#define SH_FREE SH_MAKE_NAME(free)
 
 /* internal helper functions (no externally visible prototypes) */
 #define SH_COMPUTE_PARAMETERS SH_MAKE_NAME(compute_parameters)
@@ -87,13 +89,6 @@
 #define SH_INITIAL_BUCKET SH_MAKE_NAME(initial_bucket)
 #define SH_ENTRY_HASH SH_MAKE_NAME(entry_hash)
 
-/* Allocation function for hash table elements */
-#ifndef SIMPLEHASH_TYPEDEFS
-#define SIMPLEHASH_TYPEDEFS
-typedef void *(*simplehash_allocate) (Size size, void *args);
-typedef void (*simplehash_free) (void *pointer, void *args);
-#endif
-
 /* generate forward declarations necessary to use the hash table */
 #ifdef SH_DECLARE
 
@@ -119,11 +114,6 @@ typedef struct SH_TYPE
 	/* hash buckets */
 	SH_ELEMENT_TYPE *data;
 
-	/* Allocation and free functions, and the associated context. */
-	simplehash_allocate element_alloc;
-	simplehash_free element_free;
-	void	   *element_args;
-
 	/* memory context to use for allocations */
 	MemoryContext ctx;
 
@@ -145,8 +135,7 @@ typedef struct SH_ITERATOR
 } SH_ITERATOR;
 
 /* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
-		simplehash_allocate allocfunc, simplehash_free freefunc, void *args);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
 SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
 SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
 SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -289,23 +278,25 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
 #endif
 }
 
+#ifndef SH_USE_NONDEFAULT_ALLOCATOR
+
 /* default memory allocator function */
-static void *
-SH_DEFAULT_ALLOC(Size size, void *args)
+static inline void *
+SH_ALLOCATE(SH_TYPE *type, Size size)
 {
-	MemoryContext context = (MemoryContext) args;
-
-	return MemoryContextAllocExtended(context, size,
+	return MemoryContextAllocExtended(type->ctx, size,
 									  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
 }
 
 /* default memory free function */
-static void
-SH_DEFAULT_FREE(void *pointer, void *args)
+static inline void
+SH_FREE(SH_TYPE *type, void *pointer)
 {
 	pfree(pointer);
 }
 
+#endif
+
 /*
  * Create a hash table with enough space for `nelements` distinct members.
  * Memory for the hash table is allocated from the passed-in context.  If
@@ -316,8 +307,7 @@ SH_DEFAULT_FREE(void *pointer, void *args)
  * the passed-in context.
  */
 SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements, simplehash_allocate allocfunc,
-		  simplehash_free freefunc, void *args)
+SH_CREATE(MemoryContext ctx, uint32 nelements)
 {
 	SH_TYPE    *tb;
 	uint64		size;
@@ -330,22 +320,7 @@ SH_CREATE(MemoryContext ctx, uint32 nelements, simplehash_allocate allocfunc,
 
 	SH_COMPUTE_PARAMETERS(tb, size);
 
-	if (allocfunc == NULL)
-	{
-		tb->element_alloc = SH_DEFAULT_ALLOC;
-		tb->element_free = SH_DEFAULT_FREE;
-		tb->element_args = ctx;
-	}
-	else
-	{
-		tb->element_alloc = allocfunc;
-		tb->element_free = freefunc;
-
-		tb->element_args = args;
-	}
-
-	tb->data = tb->element_alloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
-								 tb->element_args);
+	tb->data = SH_ALLOCATE(tb, sizeof(SH_ELEMENT_TYPE) * tb->size);
 
 	return tb;
 }
@@ -354,7 +329,7 @@ SH_CREATE(MemoryContext ctx, uint32 nelements, simplehash_allocate allocfunc,
 SH_SCOPE void
 SH_DESTROY(SH_TYPE *tb)
 {
-	tb->element_free(tb->data, tb->element_args);
+	SH_FREE(tb, tb->data);
 	pfree(tb);
 }
 
@@ -382,8 +357,7 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 	/* compute parameters for new table */
 	SH_COMPUTE_PARAMETERS(tb, newsize);
 
-	tb->data = tb->element_alloc(sizeof(SH_ELEMENT_TYPE) * tb->size,
-								 tb->element_args);
+	tb->data = SH_ALLOCATE(tb, sizeof(SH_ELEMENT_TYPE) * tb->size);
 
 	newdata = tb->data;
 
@@ -469,7 +443,7 @@ SH_GROW(SH_TYPE *tb, uint32 newsize)
 		}
 	}
 
-	tb->element_free(olddata, tb->element_args);
+	SH_FREE(tb, olddata);
 }
 
 /*
#63Andres Freund
andres@anarazel.de
In reply to: Robert Haas (#62)
Re: Parallel bitmap heap scan

On 2017-02-07 16:36:55 -0500, Robert Haas wrote:

On Tue, Feb 7, 2017 at 4:15 PM, Andres Freund <andres@anarazel.de> wrote:

FWIW, I think it'd have been better to not add the new callbacks as
parameters to *_create(), but rather have them be "templatized" like the
rest of simplehash. That'd require that callback to check the context,
to know whether it should use shared memory or not, but that seems fine
to me. Right now this pushes the head of simplehash above a
cacheline...

Something like the attached?

Yes.

+#ifndef SH_USE_NONDEFAULT_ALLOCATOR
+

That should probably be documented in the file header.

Thanks!

Andres

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#64Robert Haas
robertmhaas@gmail.com
In reply to: Andres Freund (#63)
Re: Parallel bitmap heap scan

On Tue, Feb 7, 2017 at 4:45 PM, Andres Freund <andres@anarazel.de> wrote:

On 2017-02-07 16:36:55 -0500, Robert Haas wrote:

On Tue, Feb 7, 2017 at 4:15 PM, Andres Freund <andres@anarazel.de> wrote:

FWIW, I think it'd have been better to not add the new callbacks as
parameters to *_create(), but rather have them be "templatized" like the
rest of simplehash. That'd require that callback to check the context,
to know whether it should use shared memory or not, but that seems fine
to me. Right now this pushes the head of simplehash above a
cacheline...

Something like the attached?

Yes.

+#ifndef SH_USE_NONDEFAULT_ALLOCATOR
+

That should probably be documented in the file header.

Right. OK, did that and a few other cleanups, and committed.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#65Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#56)
Re: Parallel bitmap heap scan

On Sun, Feb 5, 2017 at 9:48 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

Here are the latest version of the patch, which address all the
comments given by Robert.

I think it would be useful to break the remaining patch into two
parts, one that enhances the tidbitmap.c API and another that uses
that to implement Parallel Bitmap Heap Scan. In this review I'm going
to focus on the changes to tidbitmap.c and tidbitmap.h.

+extern void tbm_restore_shared_members(TIDBitmap *tbm,
+                           ParallelTBMInfo * stbm);
+extern void tbm_update_shared_members(TIDBitmap *tbm,
+                          ParallelTBMInfo * parallel_tbm);
+void        tbm_set_parallel(TIDBitmap *tbm, void *area);
+extern Size tbm_estimate_parallel_tbminfo(Size size);
+TIDBitmap  *tbm_attach(ParallelTBMInfo * parallel_tbm, void *area);
+TBMIterator *tbm_begin_shared_iterate(TIDBitmap *tbm,
+                         ParallelTBMInfo * parallel_tbm, bool prefetch);
+TBMIterateResult *tbm_shared_iterate(TBMIterator *iterator);
+void        tbm_init_shared_iterator(ParallelTBMInfo * tbminfo);

There are several cosmetic problems here. You may have noticed that
all function prototypes in PostgreSQL header files are explicitly
declared extern; yours should be, too. Also, there is extra
whitespace before some of the variable names here, like
"ParallelTBMInfo * tbminfo" instead of "ParallelTBMInfo *tbminfo". If
that is due to pgindent, the solution is to add the typedef names to
your local typedef list. Also, tbm_restore_shared_members doesn't
actually exist.

More broadly, this seems like an extremely complicated API. Even
ignoring the function that doesn't exist, that's still 7 different
functions just for shared iteration, which seems like a lot. I
suggest the following API:

1. Add an additional argument to tbm_create(), dsa_area *dsa. If it's
NULL, we allocate a backend-private memory for the hash entries as
normal. If it's true, we use the dsa_area to store the hash entries,
using the infrastructure added by your 0002 and revised in
c3c4f6e1740be256178cd7551d5b8a7677159b74. (You can use a flag in the
BitmapIndexScan and BitmapOr to decide whether to pass NULL or
es_query_dsa.)

2. Add a new function dsa_pointer tbm_prepare_shared_iterate(TIDBitmap
*tbm) which allocates shared iteration arrays using the DSA passed to
tbm_create() and returns a dsa_pointer to the result. Arrange this so
that if it's called more than once, each call returns a separate
iterator, so that you can call it once to get the main iterator and a
second time for the prefetch iterator, but have both of those point to
the same underlying iteration arrays.

3. Add a new function TBMSharedIterator
*tbm_attach_shared_iterate(dsa_area *dsa, dsa_pointer) which is called
once per backend and gets passed the dsa_pointer from the previous
step.

4. Add a new function TBMIterateResult
*tbm_shared_iterate(TBMSharedIterator *) to fetch the next result.

As compared with your proposal, this involves only 3 functions instead
of 7 (plus one new argument to another function), and I think it's a
lot clearer what those functions are actually doing. You don't need
tbm_estimate_parallel_tbminfo() any more because the data being passed
from one backend to another is precisely a dsa_pointer, and the bitmap
scan can just leave space for that in its own estimator function. You
don't need tbm_update_shared_members() separately from
tbm_begin_shared_iterate() separately from tbm_init_shared_iterator()
because tbm_prepare_shared_iterate() can do all that stuff. You don't
need tbm_set_parallel() because the additional argument to
tbm_create() takes care of that need.

The way you've got ParallelTBMInfo set up right now doesn't respect
the separation of concerns between nodeBitmapHeapscan.c and
tidbitmap.c properly. tidbitmap.c shouldn't know that the caller
intends on creating two iterators, one of which is for prefetching.
The design above hopefully addresses that: each call to
tbm_prepare_shared_iterate returns a dsa_pointer to a separate chunk
of shared iterator state, but tidbitmap.c does not need to know how
many times that will be called.

Apart from the above, this patch will need a rebase over today's
commits, and please make sure all functions you add have high-quality
header comments.

Thanks,

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#66Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#65)
Re: Parallel bitmap heap scan

On Wed, Feb 8, 2017 at 8:07 AM, Robert Haas <robertmhaas@gmail.com> wrote:

.Thanks for your input, I have few queries about these comments.

2. Add a new function dsa_pointer tbm_prepare_shared_iterate(TIDBitmap
*tbm) which allocates shared iteration arrays using the DSA passed to
tbm_create() and returns a dsa_pointer to the result. Arrange this so
that if it's called more than once, each call returns a separate
iterator, so that you can call it once to get the main iterator and a
second time for the prefetch iterator, but have both of those point to
the same underlying iteration arrays.

3. Add a new function TBMSharedIterator
*tbm_attach_shared_iterate(dsa_area *dsa, dsa_pointer) which is called
once per backend and gets passed the dsa_pointer from the previous
step.

IIUC, tbm_prepare_shared_iterate will be called only by the leader,
for tbmiterator as well as for the prefetch_iterator. And,
tbm_attach_shared_iterate will be called by each backend and for both
the iterators.

IMHO, tbm_attach_shared_iterate should return TBMIterator with
reference to TBMSharedIterator inside it. The reason behind same is
that we can not keep TBMIterateResult inside TBMSharedIterator
otherwise, results will also go in shared memory but we want to have
local result memory for each worker so that other worker doesn't
disturb it.

Another option can be that we change the tbm_shared_iterate as explained below

TBMIterateResult * tbm_shared_iterate(TBMSharedIterator *,
TBMIterateResult *result).

Now, if result passed to this API is NULL then it will allocate the
memory for the result and that way we will have local result memory,
and if it's not NULL we will use this memory to store our results.
BitmapHeapNode already having a reference to the TBMIterateResult so
we should not have any problem in passing this reference to the
tbm_shared_iterate. I think this one looks better than what I
explained above.

Please suggest.

4. Add a new function TBMIterateResult
*tbm_shared_iterate(TBMSharedIterator *) to fetch the next result.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#67Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#64)
Re: Parallel bitmap heap scan

On Wed, Feb 8, 2017 at 3:44 AM, Robert Haas <robertmhaas@gmail.com> wrote:

+#ifndef SH_USE_NONDEFAULT_ALLOCATOR
+

That should probably be documented in the file header.

Right. OK, did that and a few other cleanups, and committed.

The new SH_CREATE(MemoryContext ctx, uint32 nelements) don't have any
option to supply arguments to it. Our callback functions need access
to TBM.

Is it expected that if the user of SH_CREATE who doesn't want to pass
a "MemoryContext" then we can pass arguments instead of ctx?

something like this ?
if (!tbm->dsa)
tbm->pagetable = pagetable_create(tbm->mcxt, 128);
else
tbm->pagetable = pagetable_create((MemoryContext)tbm, 128);

And, In allocation function, we can access this context and typecast to tbm?

As shown below.
static void *
pagetable_allocate(pagetable_hash *pagetable, Size size)
{
TIDBitmap *tbm = pagetable->ctx;

So Is it expected to do like I explained above, or we missed to have
an arg parameter to SH_CREATE as well as in SH_TYPE structure or there
is some other way you have in mind?

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#68Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#64)
1 attachment(s)
Re: Parallel bitmap heap scan

On Wed, Feb 8, 2017 at 3:44 AM, Robert Haas <robertmhaas@gmail.com> wrote:

+#ifndef SH_USE_NONDEFAULT_ALLOCATOR
+

That should probably be documented in the file header.

Right. OK, did that and a few other cleanups, and committed.

I think we need to have prototype for the default allocator outside of
#ifndef SH_USE_NONDEFAULT_ALLOCATOR. Because the file e.g. tidbitmap.c
who wants to use SH_USE_NONDEFAULT_ALLOCATOR will provide the
allocator function definition but it can not have the declaration of
those function as that function take SH_TYPE as input and that will be
only defined once we include the simplehash.h.

So basically we can not declare prototype before including
simplehash.h for allocator. And, if we don't declare we will get
"implicit declaration warning" because simplehash itself is using
those functions.

The solution is simplehash.h, should always declare it, and provide
the definitions only if SH_USE_NONDEFAULT_ALLOCATOR is not defined.
Attached patch does that.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

hash_allocate_fix.patchapplication/octet-stream; name=hash_allocate_fix.patchDownload
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index ff50633..6e85de8 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -280,6 +280,10 @@ SH_ENTRY_HASH(SH_TYPE *tb, SH_ELEMENT_TYPE * entry)
 #endif
 }
 
+/* default memory allocator function */
+static inline void *SH_ALLOCATE(SH_TYPE *type, Size size);
+static inline void SH_FREE(SH_TYPE *type, void *pointer);
+
 #ifndef SH_USE_NONDEFAULT_ALLOCATOR
 
 /* default memory allocator function */
#69Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#68)
Re: Parallel bitmap heap scan

On Wed, Feb 8, 2017 at 5:21 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Wed, Feb 8, 2017 at 3:44 AM, Robert Haas <robertmhaas@gmail.com> wrote:

+#ifndef SH_USE_NONDEFAULT_ALLOCATOR
+

That should probably be documented in the file header.

Right. OK, did that and a few other cleanups, and committed.

I think we need to have prototype for the default allocator outside of
#ifndef SH_USE_NONDEFAULT_ALLOCATOR. Because the file e.g. tidbitmap.c
who wants to use SH_USE_NONDEFAULT_ALLOCATOR will provide the
allocator function definition but it can not have the declaration of
those function as that function take SH_TYPE as input and that will be
only defined once we include the simplehash.h.

So basically we can not declare prototype before including
simplehash.h for allocator. And, if we don't declare we will get
"implicit declaration warning" because simplehash itself is using
those functions.

The solution is simplehash.h, should always declare it, and provide
the definitions only if SH_USE_NONDEFAULT_ALLOCATOR is not defined.
Attached patch does that.

Makes sense, will commit.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#70Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#67)
Re: Parallel bitmap heap scan

On Wed, Feb 8, 2017 at 4:20 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

The new SH_CREATE(MemoryContext ctx, uint32 nelements) don't have any
option to supply arguments to it. Our callback functions need access
to TBM.

Is it expected that if the user of SH_CREATE who doesn't want to pass
a "MemoryContext" then we can pass arguments instead of ctx?

You can store whatever you want in SH_TYPE's private_data member.
SH_ALLOCATE and SH_FREE both get a pointer to the SH_TYPE, so they
have access to that. Hmm, but there's no way to get that set in
SH_CREATE before SH_ALLOCATE is called. Maybe we need to add a
private_data argument to SH_CREATE. execGrouping.c could use that
instead of frobbing private_data directly:

-    hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
-    hashtable->hashtab->private_data = hashtable;
+    hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, hashtable);

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#71Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#66)
Re: Parallel bitmap heap scan

On Wed, Feb 8, 2017 at 1:59 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

IIUC, tbm_prepare_shared_iterate will be called only by the leader,
for tbmiterator as well as for the prefetch_iterator. And,
tbm_attach_shared_iterate will be called by each backend and for both
the iterators.

That's what I had in mind.

IMHO, tbm_attach_shared_iterate should return TBMIterator with
reference to TBMSharedIterator inside it. The reason behind same is
that we can not keep TBMIterateResult inside TBMSharedIterator
otherwise, results will also go in shared memory but we want to have
local result memory for each worker so that other worker doesn't
disturb it.

No, I don't agree. I think TBMSharedIterator should be an unshared
structure created by tbm_attach_shared_iterate, which can internally
contain backend-private state like a TBMIterateResult, and which can
also contain a pointer to the shared-memory structure previously
created by tbm_prepare_shared_iterate. That thing needs to be called
something other than a TBMSharedIterator, like TBMSharedIterationState
or something.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#72Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#70)
1 attachment(s)
Re: Parallel bitmap heap scan

On Wed, Feb 8, 2017 at 7:01 PM, Robert Haas <robertmhaas@gmail.com> wrote:

You can store whatever you want in SH_TYPE's private_data member.
SH_ALLOCATE and SH_FREE both get a pointer to the SH_TYPE, so they
have access to that. Hmm, but there's no way to get that set in
SH_CREATE before SH_ALLOCATE is called. Maybe we need to add a
private_data argument to SH_CREATE. execGrouping.c could use that
instead of frobbing private_data directly:

-    hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
-    hashtable->hashtab->private_data = hashtable;
+    hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, hashtable);

Okay, will go ahead as you suggested. Patch attached for the same.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

hash_create_fix.patchapplication/octet-stream; name=hash_create_fix.patchDownload
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 47c9656..4b1f634 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -330,8 +330,7 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
 	else
 		hashtable->hash_iv = 0;
 
-	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
-	hashtable->hashtab->private_data = hashtable;
+	hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, hashtable);
 
 	return hashtable;
 }
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 7b31948..0885812 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -244,7 +244,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
diff --git a/src/include/lib/simplehash.h b/src/include/lib/simplehash.h
index ff50633..0209c8e 100644
--- a/src/include/lib/simplehash.h
+++ b/src/include/lib/simplehash.h
@@ -137,7 +137,8 @@ typedef struct SH_ITERATOR
 } SH_ITERATOR;
 
 /* externally visible function prototypes */
-SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements);
+SH_SCOPE SH_TYPE *SH_CREATE(MemoryContext ctx, uint32 nelements,
+							void *private_data);
 SH_SCOPE void SH_DESTROY(SH_TYPE *tb);
 SH_SCOPE void SH_GROW(SH_TYPE *tb, uint32 newsize);
 SH_SCOPE SH_ELEMENT_TYPE *SH_INSERT(SH_TYPE *tb, SH_KEY_TYPE key, bool *found);
@@ -309,13 +310,14 @@ SH_FREE(SH_TYPE *type, void *pointer)
  * the passed-in context.
  */
 SH_SCOPE SH_TYPE *
-SH_CREATE(MemoryContext ctx, uint32 nelements)
+SH_CREATE(MemoryContext ctx, uint32 nelements, void *private_data)
 {
 	SH_TYPE    *tb;
 	uint64		size;
 
 	tb = MemoryContextAllocZero(ctx, sizeof(SH_TYPE));
 	tb->ctx = ctx;
+	tb->private_data = private_data;
 
 	/* increase nelements by fillfactor, want to store nelements elements */
 	size = Min((double) SH_MAX_SIZE, ((double) nelements) / SH_FILLFACTOR);
#73Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#72)
Re: Parallel bitmap heap scan

On Wed, Feb 8, 2017 at 8:58 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Wed, Feb 8, 2017 at 7:01 PM, Robert Haas <robertmhaas@gmail.com> wrote:

You can store whatever you want in SH_TYPE's private_data member.
SH_ALLOCATE and SH_FREE both get a pointer to the SH_TYPE, so they
have access to that. Hmm, but there's no way to get that set in
SH_CREATE before SH_ALLOCATE is called. Maybe we need to add a
private_data argument to SH_CREATE. execGrouping.c could use that
instead of frobbing private_data directly:

-    hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
-    hashtable->hashtab->private_data = hashtable;
+    hashtable->hashtab = tuplehash_create(tablecxt, nbuckets, hashtable);

Okay, will go ahead as you suggested. Patch attached for the same.

Looks good to me. If nobody has further ideas here, I'll push this
and your previous patch tomorrow.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#74Robert Haas
robertmhaas@gmail.com
In reply to: Robert Haas (#73)
Re: Parallel bitmap heap scan

On Wed, Feb 8, 2017 at 10:59 AM, Robert Haas <robertmhaas@gmail.com> wrote:

Looks good to me. If nobody has further ideas here, I'll push this
and your previous patch tomorrow.

Done.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#75Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#65)
3 attachment(s)
Re: Parallel bitmap heap scan

On Wed, Feb 8, 2017 at 8:07 AM, Robert Haas <robertmhaas@gmail.com> wrote:

Thanks for the detailed review and your valuable feedback.

I think it would be useful to break the remaining patch into two
parts, one that enhances the tidbitmap.c API and another that uses
that to implement Parallel Bitmap Heap Scan.

I have done that.

There are several cosmetic problems here. You may have noticed that
all function prototypes in PostgreSQL header files are explicitly
declared extern; yours should be, too. Also, there is extra
whitespace before some of the variable names here, like
"ParallelTBMInfo * tbminfo" instead of "ParallelTBMInfo *tbminfo". If
that is due to pgindent, the solution is to add the typedef names to
your local typedef list. Also, tbm_restore_shared_members doesn't
actually exist.

Fixed

1. Add an additional argument to tbm_create(), dsa_area *dsa. If it's
NULL, we allocate a backend-private memory for the hash entries as
normal. If it's true, we use the dsa_area to store the hash entries,
using the infrastructure added by your 0002 and revised in
c3c4f6e1740be256178cd7551d5b8a7677159b74. (You can use a flag in the
BitmapIndexScan and BitmapOr to decide whether to pass NULL or
es_query_dsa.)

Done

2. Add a new function dsa_pointer tbm_prepare_shared_iterate(TIDBitmap
*tbm) which allocates shared iteration arrays using the DSA passed to
tbm_create() and returns a dsa_pointer to the result. Arrange this so
that if it's called more than once, each call returns a separate
iterator, so that you can call it once to get the main iterator and a
second time for the prefetch iterator, but have both of those point to
the same underlying iteration arrays.

3. Add a new function TBMSharedIterator
*tbm_attach_shared_iterate(dsa_area *dsa, dsa_pointer) which is called
once per backend and gets passed the dsa_pointer from the previous
step.

4. Add a new function TBMIterateResult
*tbm_shared_iterate(TBMSharedIterator *) to fetch the next result.

I have tried to get these three API's as you explained with one
change. In tbm_attach_shared_iterate I need to pass TBM also because
each worker will create their own copy of TBM. Those workers need to
get the TBM-related information from the shared location. Even though
I try to access some of the information like chunk, npages from
directly shared location, but some other information like base pointer
to dsa element, RelptrPagetableEntry etc. should be local to each
worker (after conversion from dsa pointer to local pointer).

As compared with your proposal, this involves only 3 functions instead
of 7 (plus one new argument to another function), and I think it's a
lot clearer what those functions are actually doing. You don't need
tbm_estimate_parallel_tbminfo() any more because the data being passed
from one backend to another is precisely a dsa_pointer, and the bitmap
scan can just leave space for that in its own estimator function. You
don't need tbm_update_shared_members() separately from
tbm_begin_shared_iterate() separately from tbm_init_shared_iterator()
because tbm_prepare_shared_iterate() can do all that stuff. You don't
need tbm_set_parallel() because the additional argument to
tbm_create() takes care of that need.

Right

The way you've got ParallelTBMInfo set up right now doesn't respect
the separation of concerns between nodeBitmapHeapscan.c and
tidbitmap.c properly. tidbitmap.c shouldn't know that the caller
intends on creating two iterators, one of which is for prefetching.
The design above hopefully addresses that: each call to
tbm_prepare_shared_iterate returns a dsa_pointer to a separate chunk
of shared iterator state, but tidbitmap.c does not need to know how
many times that will be called.

Done

Apart from the above, this patch will need a rebase over today's
commits,

Done
and please make sure all functions you add have high-quality

header comments.

I tried my best, please check the latest patch (0001).

Apart from those, there are some more changes.
1. I have moved the dsa_pointer and dsa_area declaration from dsa.h to
postgres .h, an independent patch is attached for the same. Because we
need to declare function headers with dsa_pointer in tidbitmap.h, but
tidbitmap.h also used in FRONTEND, therefore, this can not include
dsa.h (as it contains atomics.h).

2. I noticed there was one defect in my code related to handling the
TBM_ONE_PAGE, In initial version, there was no problem, but it got
introduced somewhere in intermediate versions.

I have fixed the same. There were two option to do that

1. Don’t switch to TBM_ONE_PAGE in case of parallel mode (ideally
this can not happen only worst estimation can get us there) and
directly got to TBM_HASH
2. In shared information keep space for sharing a PagetableEntry.

I have implemented 2nd (in the initial versions I implemented with 1st).

Note: Patch is also rebased on top of guc_parallel_index_scan_v1.patch
from Parallel Index Scan thread[1]/messages/by-id/CAA4eK1+TnM4pXQbvn7OXqam+k_HZqb0ROZUMxOiL6DWJYCyYow@mail.gmail.com

[1]: /messages/by-id/CAA4eK1+TnM4pXQbvn7OXqam+k_HZqb0ROZUMxOiL6DWJYCyYow@mail.gmail.com
/messages/by-id/CAA4eK1+TnM4pXQbvn7OXqam+k_HZqb0ROZUMxOiL6DWJYCyYow@mail.gmail.com

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

move-dsa-to-postgres-h.patchapplication/octet-stream; name=move-dsa-to-postgres-h.patchDownload
diff --git a/src/include/postgres.h b/src/include/postgres.h
index ff2c5c0..387d9f4 100644
--- a/src/include/postgres.h
+++ b/src/include/postgres.h
@@ -790,6 +790,35 @@ extern Datum Float8GetDatum(float8 X);
 #define Float4GetDatumFast(X) PointerGetDatum(&(X))
 #endif
 
+typedef struct dsa_area dsa_area;
+
+/*
+ * If this system only uses a 32-bit value for Size, then use the 32-bit
+ * implementation of DSA.  This limits the amount of DSA that can be created
+ * to something significantly less than the entire 4GB address space because
+ * the DSA pointer must encode both a segment identifier and an offset, but
+ * that shouldn't be a significant limitation in practice.
+ *
+ * If this system doesn't support atomic operations on 64-bit values, then
+ * we fall back to 32-bit dsa_pointer for lack of other options.
+ *
+ * For testing purposes, USE_SMALL_DSA_POINTER can be defined to force the use
+ * of 32-bit dsa_pointer even on systems capable of supporting a 64-bit
+ * dsa_pointer.
+ */
+#if SIZEOF_SIZE_T == 4 || !defined(PG_HAVE_ATOMIC_U64_SUPPORT) || \
+	defined(USE_SMALL_DSA_POINTER)
+#define SIZEOF_DSA_POINTER 4
+#else
+#define SIZEOF_DSA_POINTER 8
+#endif
+
+#if SIZEOF_DSA_POINTER == 4
+typedef uint32 dsa_pointer;
+#else
+typedef uint64 dsa_pointer;
+#endif
+
 
 /* ----------------------------------------------------------------
  *				Section 3:	exception handling backend support
diff --git a/src/include/utils/dsa.h b/src/include/utils/dsa.h
index bb634e7..d9b4366 100644
--- a/src/include/utils/dsa.h
+++ b/src/include/utils/dsa.h
@@ -21,28 +21,7 @@
 
 /* The opaque type used for an area. */
 struct dsa_area;
-typedef struct dsa_area dsa_area;
 
-/*
- * If this system only uses a 32-bit value for Size, then use the 32-bit
- * implementation of DSA.  This limits the amount of DSA that can be created
- * to something significantly less than the entire 4GB address space because
- * the DSA pointer must encode both a segment identifier and an offset, but
- * that shouldn't be a significant limitation in practice.
- *
- * If this system doesn't support atomic operations on 64-bit values, then
- * we fall back to 32-bit dsa_pointer for lack of other options.
- *
- * For testing purposes, USE_SMALL_DSA_POINTER can be defined to force the use
- * of 32-bit dsa_pointer even on systems capable of supporting a 64-bit
- * dsa_pointer.
- */
-#if SIZEOF_SIZE_T == 4 || !defined(PG_HAVE_ATOMIC_U64_SUPPORT) || \
-	defined(USE_SMALL_DSA_POINTER)
-#define SIZEOF_DSA_POINTER 4
-#else
-#define SIZEOF_DSA_POINTER 8
-#endif
 
 /*
  * The type of 'relative pointers' to memory allocated by a dynamic shared
@@ -52,7 +31,6 @@ typedef struct dsa_area dsa_area;
  * operations.
  */
 #if SIZEOF_DSA_POINTER == 4
-typedef uint32 dsa_pointer;
 typedef pg_atomic_uint32 dsa_pointer_atomic;
 #define dsa_pointer_atomic_init pg_atomic_init_u32
 #define dsa_pointer_atomic_read pg_atomic_read_u32
@@ -61,7 +39,6 @@ typedef pg_atomic_uint32 dsa_pointer_atomic;
 #define dsa_pointer_atomic_compare_exchange pg_atomic_compare_exchange_u32
 #define DSA_POINTER_FORMAT "%08x"
 #else
-typedef uint64 dsa_pointer;
 typedef pg_atomic_uint64 dsa_pointer_atomic;
 #define dsa_pointer_atomic_init pg_atomic_init_u64
 #define dsa_pointer_atomic_read pg_atomic_read_u64
0001-tidbitmap-support-shared-v1.patchapplication/octet-stream; name=0001-tidbitmap-support-shared-v1.patchDownload
diff --git a/src/backend/access/gin/ginget.c b/src/backend/access/gin/ginget.c
index 60f005c..87cd9ea 100644
--- a/src/backend/access/gin/ginget.c
+++ b/src/backend/access/gin/ginget.c
@@ -120,7 +120,7 @@ collectMatchBitmap(GinBtreeData *btree, GinBtreeStack *stack,
 	Form_pg_attribute attr;
 
 	/* Initialize empty bitmap result */
-	scanEntry->matchBitmap = tbm_create(work_mem * 1024L);
+	scanEntry->matchBitmap = tbm_create(work_mem * 1024L, NULL);
 
 	/* Null query cannot partial-match anything */
 	if (scanEntry->isPartialMatch &&
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..94bb289 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,7 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L);
+		tbm = tbm_create(work_mem * 1024L, NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 3c6155b..1d280be 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,7 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L);
+				result = tbm_create(work_mem * 1024L, NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..81b742b 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -43,6 +43,9 @@
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
 #include "nodes/tidbitmap.h"
+#include "storage/lwlock.h"
+#include "utils/dsa.h"
+#include "utils/relptr.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -102,6 +105,9 @@ typedef struct PagetableEntry
 	bitmapword	words[Max(WORDS_PER_PAGE, WORDS_PER_CHUNK)];
 } PagetableEntry;
 
+/* Relative pointer of PagetableEntry to share across workers. */
+relptr_declare(PagetableEntry, RelptrPagetableEntry);
+
 /*
  * We want to avoid the overhead of creating the hashtable, which is
  * comparatively large, when not necessary. Particularly when we are using a
@@ -139,6 +145,13 @@ struct TIDBitmap
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	dsa_area   *dsa;			/* reference to per-query dsa area */
+	char	   *base;			/* pointer to the element array */
+	dsa_pointer dsapages;		/* dsa_pointer to the page array */
+	dsa_pointer dsachunks;		/* dsa_pointer to the chunk array */
+	RelptrPagetableEntry *relpages;		/* page array of relptr */
+	RelptrPagetableEntry *relchunks;	/* chunk array of relptr */
 };
 
 /*
@@ -156,6 +169,45 @@ struct TBMIterator
 	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
 };
 
+/* Shared TBMInfo to shared across multiple workers */
+typedef struct TBMSharedInfo
+{
+	dsa_pointer dsa_data;		/* dsa pointers to head of pagetable data */
+	dsa_pointer spages;			/* dsa pointer to page array */
+	dsa_pointer schunks;		/* dsa pointer to chunk array */
+	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
+	TBMStatus	status;			/* status of TIDBitmap */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+}	TBMSharedInfo;
+
+/*
+ * This stores the shared members of TBMSharedIterator so that multiple
+ * workers can operate on the same state. It also stores the TBMSharedInfo,
+ * in order to share relptrs of the chunk and the pages arrays and other
+ * TBM related information with other workers.
+ */
+typedef struct TBMSharedIteratorState
+{
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	LWLock		lock;			/* lock to protect above members */
+	TBMSharedInfo tbminfo;		/* TBM info to shared across workers */
+}	TBMSharedIteratorState;
+
+/*
+ * same as TBMIterator except that it holds a reference to the shared
+ * memory state so that multiple workers could operate on the same state.
+ */
+struct TBMSharedIterator
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	TBMSharedIteratorState *state;
+	TBMIterateResult output;
+};
 
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
@@ -168,6 +220,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+static int tbm_shared_comparator(const void *left, const void *right,
+					  void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -187,6 +241,7 @@ hash_blockno(BlockNumber b)
 }
 
 /* define hashtable mapping block numbers to PagetableEntry's */
+#define SH_USE_NONDEFAULT_ALLOCATOR
 #define SH_PREFIX pagetable
 #define SH_ELEMENT_TYPE PagetableEntry
 #define SH_KEY_TYPE BlockNumber
@@ -204,10 +259,12 @@ hash_blockno(BlockNumber b)
  *
  * The bitmap will live in the memory context that is CurrentMemoryContext
  * at the time of this call.  It will be limited to (approximately) maxbytes
- * total memory consumption.
+ * total memory consumption.  If dsa passed to this function is not NULL
+ * then the memory for storing elements of the underlying page table will
+ * be allocated from the DSA.
  */
 TIDBitmap *
-tbm_create(long maxbytes)
+tbm_create(long maxbytes, dsa_area *dsa)
 {
 	TIDBitmap  *tbm;
 	long		nbuckets;
@@ -230,6 +287,7 @@ tbm_create(long maxbytes)
 	nbuckets = Max(nbuckets, 16);		/* sanity limit */
 	tbm->maxentries = (int) nbuckets;
 	tbm->lossify_start = 0;
+	tbm->dsa = dsa;
 
 	return tbm;
 }
@@ -244,7 +302,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, tbm);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -777,6 +835,129 @@ tbm_iterate(TBMIterator *iterator)
 }
 
 /*
+ *	tbm_shared_iterate - scan through next page of a TIDBitmap
+ *
+ *	As above, but this will iterate using shared iterator which is shared
+ *	across multiple workers.  We need to acquire the iterator LWLock, before
+ *	accessing the shared members.
+ */
+TBMIterateResult *
+tbm_shared_iterate(TBMSharedIterator *iterator)
+{
+	TIDBitmap  *tbm = iterator->tbm;
+	TBMIterateResult *output = &iterator->output;
+	TBMSharedIteratorState *state = iterator->state;
+
+	Assert(tbm->iterating);
+
+	/* Acquire the LWLock before accessing the shared members */
+	LWLockAcquire(&state->lock, LW_EXCLUSIVE);
+
+	/*
+	 * If lossy chunk pages remain, make sure we've advanced schunkptr/
+	 * schunkbit to the next set bit.
+	 */
+	while (state->schunkptr < tbm->nchunks)
+	{
+		PagetableEntry *chunk = relptr_access(tbm->base,
+										   tbm->relchunks[state->schunkptr]);
+		int			schunkbit = state->schunkbit;
+
+		while (schunkbit < PAGES_PER_CHUNK)
+		{
+			int			wordnum = WORDNUM(schunkbit);
+			int			bitnum = BITNUM(schunkbit);
+
+			if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
+				break;
+			schunkbit++;
+		}
+		if (schunkbit < PAGES_PER_CHUNK)
+		{
+			state->schunkbit = schunkbit;
+			break;
+		}
+		/* advance to next chunk */
+		state->schunkptr++;
+		state->schunkbit = 0;
+	}
+
+	/*
+	 * If both chunk and per-page data remain, must output the numerically
+	 * earlier page.
+	 */
+	if (state->schunkptr < tbm->nchunks)
+	{
+		PagetableEntry *chunk;
+		PagetableEntry *page;
+		BlockNumber chunk_blockno;
+
+		chunk = relptr_access(tbm->base, tbm->relchunks[state->schunkptr]);
+		page = relptr_access(tbm->base, tbm->relpages[state->spageptr]);
+		chunk_blockno = chunk->blockno + state->schunkbit;
+
+		if (state->spageptr >= tbm->npages || chunk_blockno < page->blockno)
+		{
+			/* Return a lossy page indicator from the chunk */
+			output->blockno = chunk_blockno;
+			output->ntuples = -1;
+			output->recheck = true;
+			state->schunkbit++;
+
+			LWLockRelease(&state->lock);
+			return output;
+		}
+	}
+
+	if (state->spageptr < tbm->npages)
+	{
+		PagetableEntry *page;
+		int			ntuples;
+		int			wordnum;
+
+		/* In ONE_PAGE state, we don't allocate an spages[] array */
+		if (tbm->status == TBM_ONE_PAGE)
+			page = &tbm->entry1;
+		else
+			page = relptr_access(tbm->base, tbm->relpages[state->spageptr]);
+
+		/* scan bitmap to extract individual offset numbers */
+		ntuples = 0;
+		for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
+		{
+			bitmapword	w = page->words[wordnum];
+
+			if (w != 0)
+			{
+				int			off = wordnum * BITS_PER_BITMAPWORD + 1;
+
+				while (w != 0)
+				{
+					if (w & 1)
+						output->offsets[ntuples++] = (OffsetNumber) off;
+					off++;
+					w >>= 1;
+				}
+			}
+		}
+
+		output->blockno = page->blockno;
+		output->ntuples = ntuples;
+		output->recheck = page->recheck;
+		state->spageptr++;
+
+		LWLockRelease(&state->lock);
+
+		return output;
+	}
+
+	LWLockRelease(&state->lock);
+
+	/* Nothing more in the bitmap */
+	return NULL;
+}
+
+/*
  * tbm_end_iterate - finish an iteration over a TIDBitmap
  *
  * Currently this is just a pfree, but it might do more someday.  (For
@@ -790,6 +971,17 @@ tbm_end_iterate(TBMIterator *iterator)
 }
 
 /*
+ * tbm_end_shared_iterate - finish an iteration over a TIDBitmap
+ *
+ * As above, but it frees the memory of TBMSharedIterator.
+ */
+void
+tbm_end_shared_iterate(TBMSharedIterator *iterator)
+{
+	pfree(iterator);
+}
+
+/*
  * tbm_find_pageentry - find a PagetableEntry for the pageno
  *
  * Returns NULL if there is no non-lossy entry for the pageno.
@@ -1061,3 +1253,246 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * As above, but this will get relptrs (relative pointer) of PagetableEntry
+ * and it needs to convert it to actual PagetableEntry before comparing the
+ * blockno.
+ */
+static int
+tbm_shared_comparator(const void *left, const void *right, void *arg)
+{
+	PagetableEntry *lpage;
+	PagetableEntry *rpage;
+
+	lpage = relptr_access((char *) arg, *((RelptrPagetableEntry *) left));
+	rpage = relptr_access((char *) arg, *((RelptrPagetableEntry *) right));
+
+	if (lpage->blockno < rpage->blockno)
+		return -1;
+	else if (lpage->blockno > rpage->blockno)
+		return 1;
+	return 0;
+}
+
+/*
+ * tbm_prepare_shared_iterate - prepare to iterator through a TIDBitmap
+ * by multiple workers using shared iterator.
+ *
+ * The TBMSharedIteratorState will be allocated from DSA so that multiple
+ * worker can attach to it and iterate jointly.
+ *
+ * This will convert the pagetable hash into page and chunk array of
+ * reptr (relative pointer) so that these arrays can be shared across
+ * multiple workers.
+ */
+dsa_pointer
+tbm_prepare_shared_iterate(TIDBitmap *tbm)
+{
+	dsa_pointer iterator;
+	TBMSharedIteratorState *iterater_state;
+	TBMSharedInfo *tbminfo;
+
+	/*
+	 * Create the TBMIterator struct, with enough trailing space to serve the
+	 * needs of the TBMIterateResult sub-struct.
+	 */
+	iterator = dsa_allocate(tbm->dsa, sizeof(TBMSharedIteratorState));
+	iterater_state = dsa_get_address(tbm->dsa, iterator);
+	tbminfo = &iterater_state->tbminfo;
+
+	/*
+	 * If we have a hashtable, create and fill the sorted page lists, unless
+	 * we already did that for a previous iterator.  Note that the lists are
+	 * attached to the bitmap not the iterator, so they can be used by more
+	 * than one iterator.  However, we keep dsa_pointer to these in the shared
+	 * iterator so that other workers can have access to these and store in
+	 * their local TBM.
+	 */
+	if (tbm->status == TBM_HASH && !tbm->iterating)
+	{
+		pagetable_iterator i;
+		PagetableEntry *page;
+		int			npages;
+		int			nchunks;
+
+		/*
+		 * Create page list and chunk list using relptr so that we can share
+		 * this information across multiple workers.
+		 */
+		if (tbm->npages)
+		{
+			tbm->dsapages = dsa_allocate(tbm->dsa,
+							   tbm->npages * (sizeof(RelptrPagetableEntry)));
+			tbm->relpages = dsa_get_address(tbm->dsa, tbm->dsapages);
+		}
+		if (tbm->nchunks)
+		{
+			tbm->dsachunks = dsa_allocate(tbm->dsa,
+							  tbm->nchunks * (sizeof(RelptrPagetableEntry)));
+			tbm->relchunks = dsa_get_address(tbm->dsa, tbm->dsachunks);
+		}
+
+		tbm->base = dsa_get_address(tbm->dsa, tbm->dsa_data);
+
+		npages = nchunks = 0;
+		pagetable_start_iterate(tbm->pagetable, &i);
+		while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+		{
+			if (page->ischunk)
+				relptr_store(tbm->base, tbm->relchunks[nchunks++], page);
+			else
+				relptr_store(tbm->base, tbm->relpages[npages++], page);
+		}
+
+		Assert(npages == tbm->npages);
+		Assert(nchunks == tbm->nchunks);
+		if (npages > 1)
+			qsort_arg(tbm->relpages, npages, sizeof(RelptrPagetableEntry *),
+					  tbm_shared_comparator, (void *) tbm->base);
+		if (nchunks > 1)
+			qsort_arg(tbm->relchunks, nchunks, sizeof(RelptrPagetableEntry *),
+					  tbm_shared_comparator, (void *) tbm->base);
+	}
+
+	/*
+	 * Store the TBM member in the shared state.  This is done to shared it
+	 * across multiple workers.
+	 */
+	tbminfo->maxentries = tbm->maxentries;
+	tbminfo->nchunks = tbm->nchunks;
+	tbminfo->nentries = tbm->nentries;
+	tbminfo->npages = tbm->npages;
+	tbminfo->dsa_data = tbm->dsa_data;
+	tbminfo->spages = tbm->dsapages;
+	tbminfo->schunks = tbm->dsachunks;
+	tbminfo->status = tbm->status;
+
+	if (tbm->status == TBM_ONE_PAGE)
+		memcpy(&tbminfo->entry1, &tbm->entry1, sizeof(PagetableEntry));
+
+	/* Initialize the shared iterator state. */
+	iterater_state->schunkbit = 0;
+	iterater_state->schunkptr = 0;
+	iterater_state->spageptr = 0;
+
+
+	LWLockInitialize(&iterater_state->lock, LWTRANCHE_PARALLEL_TBM_ITERATOR);
+
+	tbm->iterating = true;
+
+	return iterator;
+}
+
+/*
+ *	tbm_attach_shared_iterate
+ *
+ *	Allocate an iterator and attach shared iterator state to it so that
+ *	multiple workers can access the same memory. We also need to copy
+ *	some of the TBM related information from shared state to TBM because
+ *	workers (other than the leader) would have created a local TBM therefore
+ *	they have to get these information from shared location.
+ */
+TBMSharedIterator *
+tbm_attach_shared_iterate(TIDBitmap *tbm, dsa_area *dsa, dsa_pointer iterator)
+{
+	TBMSharedIterator *shared_iterator;
+	TBMSharedInfo *tbminfo;
+
+	shared_iterator = (TBMSharedIterator *) palloc(sizeof(TBMIterator) +
+								 MAX_TUPLES_PER_PAGE * sizeof(OffsetNumber));
+
+	shared_iterator->tbm = tbm;
+	shared_iterator->state =
+		(TBMSharedIteratorState *) dsa_get_address(dsa, iterator);
+	tbminfo = &shared_iterator->state->tbminfo;
+
+	/*
+	 * If we have come here from worker other than the leader then we need to
+	 * get TBM related information from shared iterator status. This needs to
+	 * be done only once per worker.
+	 */
+	if (!tbm->iterating)
+	{
+		tbm->status = tbminfo->status;
+		tbm->nchunks = tbminfo->nchunks;
+		tbm->nentries = tbminfo->nentries;
+		tbm->npages = tbminfo->npages;
+		tbm->maxentries = tbminfo->maxentries;
+		if (tbm->status == TBM_HASH)
+		{
+			tbm->dsa_data = tbminfo->dsa_data;
+			tbm->base = dsa_get_address(dsa, tbm->dsa_data);
+
+			/* Convert dsa pointer to the local pointer */
+			if (tbm->npages)
+				tbm->relpages = dsa_get_address(tbm->dsa, tbminfo->spages);
+
+			if (tbm->nchunks)
+				tbm->relchunks = dsa_get_address(tbm->dsa, tbminfo->schunks);
+		}
+		else
+			memcpy(&tbm->entry1, &tbminfo->entry1, sizeof(PagetableEntry));
+
+		tbm->iterating = true;
+	}
+
+	return shared_iterator;
+}
+
+/*
+ * tbm_alloc_shared
+ *
+ * Callback function for allocating the memory for hashtable elements.
+ * It allocates memory from DSA if tbm holds a reference to a dsa.
+ */
+static inline void *
+pagetable_allocate(pagetable_hash *pagetable, Size size)
+{
+	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	if (tbm->dsa == NULL)
+		return MemoryContextAllocExtended(pagetable->ctx, size,
+										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->dsa, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	ptr = dsa_get_address(tbm->dsa, dsaptr);
+	memset(ptr, 0, size + sizeof(dsa_pointer));
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * pagetable_free
+ *
+ * Callback function for freeing hash table elements.
+ */
+static inline void
+pagetable_free(pagetable_hash *pagetable, void *pointer)
+{
+	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
+	dsa_pointer dsa_data;
+
+	if (tbm->dsa == NULL)
+		return pfree(pointer);
+
+	/*
+	 * If TBM is in iterating phase that means pagetable is already created
+	 * and we have come here during tbm_free. By this time we are already
+	 * detached from the DSA because the GatherNode would have been shutdown.
+	 */
+	if (tbm->iterating)
+		return;
+
+	dsa_data = *((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+	dsa_free(tbm->dsa, dsa_data);
+}
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..3889a42 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -24,7 +24,6 @@
 
 #include "storage/itemptr.h"
 
-
 /*
  * Actual bitmap representation is private to tidbitmap.c.  Callers can
  * do IsA(x, TIDBitmap) on it, but nothing else.
@@ -33,6 +32,7 @@ typedef struct TIDBitmap TIDBitmap;
 
 /* Likewise, TBMIterator is private */
 typedef struct TBMIterator TBMIterator;
+typedef struct TBMSharedIterator TBMSharedIterator;
 
 /* Result structure for tbm_iterate */
 typedef struct
@@ -46,7 +46,7 @@ typedef struct
 
 /* function prototypes in nodes/tidbitmap.c */
 
-extern TIDBitmap *tbm_create(long maxbytes);
+extern TIDBitmap *tbm_create(long maxbytes, dsa_area *dsa);
 extern void tbm_free(TIDBitmap *tbm);
 
 extern void tbm_add_tuples(TIDBitmap *tbm,
@@ -62,5 +62,10 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_end_shared_iterate(TBMSharedIterator *iterator);
+extern dsa_pointer tbm_prepare_shared_iterate(TIDBitmap *tbm);
+extern TBMSharedIterator *tbm_attach_shared_iterate(TIDBitmap *tbm,
+						  dsa_area *dsa, dsa_pointer iterator);
+extern TBMIterateResult *tbm_shared_iterate(TBMSharedIterator *iterator);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 8bd93c3..4d392e0 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -212,6 +212,7 @@ typedef enum BuiltinTrancheIds
 	LWTRANCHE_LOCK_MANAGER,
 	LWTRANCHE_PREDICATE_LOCK_MANAGER,
 	LWTRANCHE_PARALLEL_QUERY_DSA,
+	LWTRANCHE_PARALLEL_TBM_ITERATOR,
 	LWTRANCHE_FIRST_USER_DEFINED
 }	BuiltinTrancheIds;
 
0002-parallel-bitmap-heapscan-v1.patchapplication/octet-stream; name=0002-parallel-bitmap-heapscan-v1.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 5b67def..82c7c87 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1240,6 +1240,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 0be48fb..82805d6 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1754,6 +1754,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index fe87c9a..a2593be 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
 
 #include "executor/execParallel.h"
 #include "executor/executor.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "executor/nodeCustom.h"
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
@@ -205,6 +206,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -257,6 +262,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -733,6 +743,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index f18827d..91e032e 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,14 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -66,13 +70,16 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	HeapScanDesc scan;
 	TIDBitmap  *tbm;
 	TBMIterator *tbmiterator;
+	TBMSharedIterator *stbmiterator;
 	TBMIterateResult *tbmres;
-
+	ParallelBitmapInfo *shared_info = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
+	TBMSharedIterator *sprefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	dsa_area *dsa = node->ss.ps.state->es_query_dsa;
 
 	/*
 	 * extract necessary information from index scan node
@@ -81,12 +88,45 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	slot = node->ss.ss_ScanTupleSlot;
 	scan = node->ss.ss_currentScanDesc;
 	tbm = node->tbm;
+
 	tbmiterator = node->tbmiterator;
+	stbmiterator = node->stbmiterator;
+
 	tbmres = node->tbmres;
 #ifdef USE_PREFETCH
 	prefetch_iterator = node->prefetch_iterator;
+	sprefetch_iterator = node->sprefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	The first worker to see the state of ParallelBitmapInfo as PBM_INITIAL
+	 *	becomes leader and sets the state to PBM_INPROGRESS. All other workers
+	 *	see the state as PBM_INPROGRESS, and will wait for leader to finish
+	 *	building the TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *		ParallelBitmapInfo so that other worker can see those.
+	 *	  Convert the TIDBitmap into shared chunk and page array
+	 *	  Set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader creates shared TIDBitmap.
+	 *	  Copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over shared page and chunk
+	 *			array and select heap pages one by one.
+	 *	  b) Since multiple workers are iterating over same page and chunk
+	 *			array we need to have a shared iterator, so we grab a LWLock
+	 *			and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,21 +141,90 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.  In parallel mode leader will
+		 * immediately come out of the function, but all other workers will
+		 * be blocked until leader wake them up.
+		 */
+		if (shared_info == NULL || pbms_is_leader(shared_info))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag. This flag will be used to indicate the
+			 * underlying tidbitmap layer to allocate pagetable elements from
+			 * DSA.
+			 */
+			if (shared_info)
+				pbms_set_parallel(outerPlanState(node));
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			/*
+			 * In parallel mode, prepare the iterate over the TIDBitmap. This
+			 * will return the dsa_pointer to the shared iterator state. Which
+			 * will be used by multiple workers to iterate jointly.
+			 */
+			if (shared_info)
+			{
+				shared_info->tbmiterator = tbm_prepare_shared_iterate(tbm);
+#ifdef USE_PREFETCH
+				if (node->prefetch_maximum > 0)
+				{
+					shared_info->prefetch_iterator =
+										tbm_prepare_shared_iterate(tbm);
+				}
+
+				/*
+				 * By this time we have already created the shared iterator so it's
+				 * time to wake up other workers.
+				 */
+				SpinLockAcquire(&shared_info->state_mutex);
+				shared_info->state = PBM_FINISHED;
+				SpinLockRelease(&shared_info->state_mutex);
+
+				/* Wake up all other workers. */
+				ConditionVariableBroadcast(&shared_info->cv);
+#endif
+			}
+		}
+		else
+			tbm = tbm_create(work_mem * 1024L, dsa);
 
 		node->tbm = tbm;
-		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+
+		if (shared_info)
+		{
+			/*
+			 * Allocate worker specific iterator and attach the shared
+			 * iterator state to it.
+			 */
+			node->stbmiterator = stbmiterator = tbm_attach_shared_iterate(
+										   tbm, dsa, shared_info->tbmiterator);
+		}
+		else
+			node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+
 		node->tbmres = tbmres = NULL;
 
 #ifdef USE_PREFETCH
 		if (node->prefetch_maximum > 0)
 		{
-			node->prefetch_iterator = prefetch_iterator = tbm_begin_iterate(tbm);
-			node->prefetch_pages = 0;
-			node->prefetch_target = -1;
+			if (shared_info)
+			{
+				node->sprefetch_iterator = tbm_attach_shared_iterate(tbm, dsa,
+											   shared_info->prefetch_iterator);
+				sprefetch_iterator = node->sprefetch_iterator;
+			}
+			else
+			{
+				node->prefetch_iterator = prefetch_iterator =
+												tbm_begin_iterate(tbm);
+				node->prefetch_pages = 0;
+				node->prefetch_target = -1;
+			}
 		}
 #endif   /* USE_PREFETCH */
 	}
@@ -124,13 +233,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
+		int		   *prefetch_target;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			/*
+			 * If we are in parallel mode perform shared iterate otherwise
+			 * local iterate.
+			 */
+			if (!shared_info)
+				node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			else
+				node->tbmres = tbmres = tbm_shared_iterate(stbmiterator);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +256,50 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			/*
+			 * If we are in parallel mode then acquire prefetch_mutex and
+			 * check prefetch_pages from shared location.
+			 */
+			if (shared_info)
 			{
+				SpinLockAcquire(&shared_info->prefetch_mutex);
+
 				/* The main iterator has closed the distance by one page */
-				node->prefetch_pages--;
+				if (shared_info->prefetch_pages > 0)
+					shared_info->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&shared_info->prefetch_mutex);
 			}
-			else if (prefetch_iterator)
+			else
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				/* The main iterator has closed the distance by one page */
+				 if (node->prefetch_pages > 0)
+					 node->prefetch_pages--;
+				else
+					need_prefetch = true;
+			}
+
+			if (prefetch_iterator && need_prefetch)
+			{
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				if (!shared_info)
+					tbmpre = tbm_iterate(prefetch_iterator);
+				else
+					tbmpre = tbm_shared_iterate(sprefetch_iterator);
+
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind the main iterator, but we can not
+				 * ensure that the current blockno in the main iterator and
+				 * in the prefetch iterator is same. It's possible that
+				 * whatever blockno we are prefetching will be processed
+				 * by another worker.
+				 */
+				if ((shared_info == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -183,19 +334,40 @@ BitmapHeapNext(BitmapHeapScanState *node)
 #ifdef USE_PREFETCH
 
 			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
+			 * If we are in shared mode then use prefetch_target from shared
+			 * location i.e pbminfo otherwise directly from node.
 			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
+			if (shared_info == NULL)
+				prefetch_target = &node->prefetch_target;
 			else
-				node->prefetch_target++;
+				prefetch_target = &shared_info->prefetch_target;
+
+			/* Increase prefetch target if it's not yet at the max. */
+			if (*prefetch_target < node->prefetch_maximum)
+			{
+				/* If we are in parallel mode then grab prefetch_mutex */
+				if (shared_info != NULL)
+					SpinLockAcquire(&shared_info->prefetch_mutex);
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (shared_info != NULL)
+					SpinLockRelease(&shared_info->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +383,25 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (shared_info == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				if (shared_info->prefetch_target < node->prefetch_maximum)
+				{
+					SpinLockAcquire(&shared_info->prefetch_mutex);
+					if (shared_info->prefetch_target < node->prefetch_maximum)
+						shared_info->prefetch_target++;
+					SpinLockRelease(&shared_info->prefetch_mutex);
+				}
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +425,67 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (shared_info)
+			{
+				prefetch_pages = &shared_info->prefetch_pages;
+				prefetch_target = shared_info->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
+
+			/*
+			 * We are checking the prefetch_pages without mutex.  Henceforth,
+			 * in parallel mode there can be some extra prefetch.  Should
+			 * we acquire mutex and recheck before iterating?
+			 */
+			while (*prefetch_pages < prefetch_target)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
+
+				if (!shared_info)
+					tbmpre = tbm_iterate(prefetch_iterator);
+				else
+					tbmpre = tbm_shared_iterate(sprefetch_iterator);
 
 				if (tbmpre == NULL)
 				{
-					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					if (!shared_info)
+					{
+						/* No more pages to prefetch */
+						tbm_end_iterate(prefetch_iterator);
+						node->prefetch_iterator = prefetch_iterator = NULL;
+					}
+					else
+					{
+						/* No more pages to prefetch */
+						tbm_end_shared_iterate(sprefetch_iterator);
+						node->sprefetch_iterator = sprefetch_iterator = NULL;
+					}
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (shared_info != NULL)
+					SpinLockAcquire(&shared_info->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (shared_info != NULL)
+					SpinLockRelease(&shared_info->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -458,6 +693,10 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->tbmiterator);
 	if (node->prefetch_iterator)
 		tbm_end_iterate(node->prefetch_iterator);
+	if (node->stbmiterator)
+		tbm_end_shared_iterate(node->stbmiterator);
+	if (node->sprefetch_iterator)
+		tbm_end_shared_iterate(node->sprefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
 	node->tbm = NULL;
@@ -465,6 +704,16 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -514,6 +763,10 @@ ExecEndBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->tbmiterator);
 	if (node->prefetch_iterator)
 		tbm_end_iterate(node->prefetch_iterator);
+	if (node->stbmiterator)
+		tbm_end_shared_iterate(node->stbmiterator);
+	if (node->sprefetch_iterator)
+		tbm_end_shared_iterate(node->sprefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
 
@@ -567,6 +820,7 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -651,3 +905,160 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ *		pbms_is_leader
+ *
+ * 		If PBMState is PBM_INITIAL then we become the leader and set the state
+ *		to PBM_INPROGRESS.  Otherwise we become the worker therefore we need to
+ *		wait for the leader to wake us up and also PBMState should be
+ *		PBM_FINISHED.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : then we need to wait till leader creates BITMAP
+		 * PBM_FINISHED   : BITMAP is ready so no need to wait.
+		 *---------------
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/*-------------------
+ *		pbms_set_parallel
+ *
+ *		Recursively process the node and set the parallel flag. This flag
+ *		will be used to indicate the underlying tidbitmap layer to allocate
+ *		pagetable elements from DSA.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapAnd, set first bitmap index scan node as parallel
+	 * because only first node will create the main bitmap other node's bitmap
+	 * will be merged to the first bitmap thus no need to create them in shared
+	 * memory.  BitmapOr and BitmapIndex will create the TidBitmap so set the
+	 * parallel flag true.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			((BitmapOrState *) node)->is_parallel = true;;
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+
+	/* Estimate the size for sharing parallel Bitmap info. */
+	node->pscan_len = add_size(offsetof(ParallelBitmapInfo,
+							   phs_snapshot_data),
+							   EstimateSnapshotSpace(estate->es_snapshot));
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 94bb289..2f01cad 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,8 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L, NULL);
+		tbm = tbm_create(work_mem * 1024L,
+				node->biss_Parallel ? node->ss.ps.state->es_query_dsa: NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 1d280be..e8f110b 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,8 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L, NULL);
+				result = tbm_create(work_mem * 1024L, node->is_parallel ?
+									node->ps.state->es_query_dsa: NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 85505c5..b7f7ce7 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2877,6 +2877,31 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 }
 
 /*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched, 0);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+/*
  * Compute the number of parallel workers that should be used to scan a
  * relation.  We compute the parallel workers based on the size of the heap to
  * be scanned and the size of the index to be scanned, then choose a minimum
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a43daa7..f81e469 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -816,6 +816,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -877,8 +878,32 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
+
+		/*
+		 * It may be possible to amortize some of the I/O cost, but probably
+		 * not very much, because most operating systems already do aggressive
+		 * prefetching.  For now, we assume that the disk run cost can't be
+		 * amortized at all.
+		 */
+
+		/*
+		 * In the case of a parallel plan, the row count needs to represent
+		 * the number of tuples processed per worker.
+		 */
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 5283468..630c501 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1557,6 +1561,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1599,6 +1608,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 997bdcf..6d152f7 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2834,7 +2834,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f440875..20b9a59 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3258,7 +3259,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 7176cf1..f8eef0b 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3392,6 +3392,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index a864f78..7e85510 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -179,6 +179,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 42c6c58..56f4436 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1261,6 +1263,7 @@ typedef struct BitmapOrState
 	PlanState	ps;				/* its first field is NodeTag */
 	PlanState **bitmapplans;	/* array of PlanStates for my inputs */
 	int			nplans;			/* number of input plans */
+	bool		is_parallel;	/* create shared tbm if it's set */
 } BitmapOrState;
 
 /* ----------------------------------------------------------------
@@ -1427,6 +1430,59 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		tbmiterator				iterator for scanning current pages
+ *		prefetch_iterator		iterator for prefetching ahead of current page
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	dsa_pointer	tbmiterator;
+	dsa_pointer	prefetch_iterator;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1441,6 +1497,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1457,6 +1515,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1472,7 +1531,9 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1487,6 +1548,10 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	TBMSharedIterator *stbmiterator;
+	TBMSharedIterator *sprefetch_iterator;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 7b41317..cfef818 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 81e7a42..51d6c09 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -54,6 +54,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index de8225b..c52c864 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -786,6 +786,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#76Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#75)
Re: Parallel bitmap heap scan

On Sat, Feb 11, 2017 at 1:41 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

I tried my best, please check the latest patch (0001).

I don't think it's acceptable (or necessary) to move the DSA
definitions into postgres.h. Why do you think you need to do that,
vs. just including dsa.h in a few more places?

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#77Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#76)
Re: Parallel bitmap heap scan

On Mon, Feb 13, 2017 at 6:24 PM, Robert Haas <robertmhaas@gmail.com> wrote:

I don't think it's acceptable (or necessary) to move the DSA
definitions into postgres.h. Why do you think you need to do that,
vs. just including dsa.h in a few more places?

I need to access dsa_pointer in tidbitmap.h, which is included from
FRONTEND as well. Now, problem is that dsa.h is including #include
"port/atomics.h", but atomic.h can not be included if FRONTEND is
defined.

#ifndef ATOMICS_H
#define ATOMICS_H
#ifdef FRONTEND
#error "atomics.h may not be included from frontend code"
#endif

Is there any other solution to this ?

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#78Haribabu Kommi
kommi.haribabu@gmail.com
In reply to: Dilip Kumar (#77)
Re: Parallel bitmap heap scan

On Tue, Feb 14, 2017 at 12:48 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Mon, Feb 13, 2017 at 6:24 PM, Robert Haas <robertmhaas@gmail.com>
wrote:

I don't think it's acceptable (or necessary) to move the DSA
definitions into postgres.h. Why do you think you need to do that,
vs. just including dsa.h in a few more places?

I need to access dsa_pointer in tidbitmap.h, which is included from
FRONTEND as well. Now, problem is that dsa.h is including #include
"port/atomics.h", but atomic.h can not be included if FRONTEND is
defined.

#ifndef ATOMICS_H
#define ATOMICS_H
#ifdef FRONTEND
#error "atomics.h may not be included from frontend code"
#endif

Is there any other solution to this ?

How about creating another header file with the parallel changes
and include it only in necessary places?

Following are my observations, while going through the patch.

+#if SIZEOF_DSA_POINTER == 4
+typedef uint32 dsa_pointer;
+#else
+typedef uint64 dsa_pointer;
+#endif

I feel the declaration of the above typdef can be moved into the
section above if we going with the current move into postgres.h
file.

+/*
+ * tbm_alloc_shared
+ *
+ * Callback function for allocating the memory for hashtable elements.
+ * It allocates memory from DSA if tbm holds a reference to a dsa.
+ */
+static inline void *
+pagetable_allocate(pagetable_hash *pagetable, Size size)

Function name and comments mismatch?

Regards,
Hari Babu
Fujitsu Australia

#79Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#77)
Re: Parallel bitmap heap scan

On Mon, Feb 13, 2017 at 8:48 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Mon, Feb 13, 2017 at 6:24 PM, Robert Haas <robertmhaas@gmail.com> wrote:

I don't think it's acceptable (or necessary) to move the DSA
definitions into postgres.h. Why do you think you need to do that,
vs. just including dsa.h in a few more places?

I need to access dsa_pointer in tidbitmap.h, which is included from
FRONTEND as well. Now, problem is that dsa.h is including #include
"port/atomics.h", but atomic.h can not be included if FRONTEND is
defined.

#ifndef ATOMICS_H
#define ATOMICS_H
#ifdef FRONTEND
#error "atomics.h may not be included from frontend code"
#endif

Is there any other solution to this ?

Well, any problem like this generally has a bunch of solutions, so
I'll say yes. I spent a good chunk of today studying the issue and
started a new thread devoted specifically to it:

/messages/by-id/CA+TgmoZ=F=GkxV0YEv-A8tb+AEGy_Qa7GSiJ8deBKFATnzfEug@mail.gmail.com

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#80Dilip Kumar
dilipbalaut@gmail.com
In reply to: Haribabu Kommi (#78)
2 attachment(s)
Re: Parallel bitmap heap scan

On Tue, Feb 14, 2017 at 6:51 AM, Haribabu Kommi
<kommi.haribabu@gmail.com> wrote:

+#if SIZEOF_DSA_POINTER == 4
+typedef uint32 dsa_pointer;
+#else
+typedef uint64 dsa_pointer;
+#endif

I feel the declaration of the above typdef can be moved into the
section above if we going with the current move into postgres.h
file.

Robert has already given the patch[1]/messages/by-id/CA+TgmoZ=F=GkxV0YEv-A8tb+AEGy_Qa7GSiJ8deBKFATnzfEug@mail.gmail.com for the same so now don't need
to do anything for this. Therefore, I included the dsa.h in
tidbitmap.h.

[1]: /messages/by-id/CA+TgmoZ=F=GkxV0YEv-A8tb+AEGy_Qa7GSiJ8deBKFATnzfEug@mail.gmail.com

+/*
+ * tbm_alloc_shared
+ *
+ * Callback function for allocating the memory for hashtable elements.
+ * It allocates memory from DSA if tbm holds a reference to a dsa.
+ */
+static inline void *
+pagetable_allocate(pagetable_hash *pagetable, Size size)

Function name and comments mismatch?

Fixed.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0001-tidbitmap-support-shared-v2.patchapplication/octet-stream; name=0001-tidbitmap-support-shared-v2.patchDownload
diff --git a/src/backend/access/gin/ginget.c b/src/backend/access/gin/ginget.c
index 60f005c..87cd9ea 100644
--- a/src/backend/access/gin/ginget.c
+++ b/src/backend/access/gin/ginget.c
@@ -120,7 +120,7 @@ collectMatchBitmap(GinBtreeData *btree, GinBtreeStack *stack,
 	Form_pg_attribute attr;
 
 	/* Initialize empty bitmap result */
-	scanEntry->matchBitmap = tbm_create(work_mem * 1024L);
+	scanEntry->matchBitmap = tbm_create(work_mem * 1024L, NULL);
 
 	/* Null query cannot partial-match anything */
 	if (scanEntry->isPartialMatch &&
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..94bb289 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,7 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L);
+		tbm = tbm_create(work_mem * 1024L, NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 3c6155b..1d280be 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,7 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L);
+				result = tbm_create(work_mem * 1024L, NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..488c641 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -43,6 +43,9 @@
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
 #include "nodes/tidbitmap.h"
+#include "storage/lwlock.h"
+#include "utils/dsa.h"
+#include "utils/relptr.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -102,6 +105,9 @@ typedef struct PagetableEntry
 	bitmapword	words[Max(WORDS_PER_PAGE, WORDS_PER_CHUNK)];
 } PagetableEntry;
 
+/* Relative pointer of PagetableEntry to share across workers. */
+relptr_declare(PagetableEntry, RelptrPagetableEntry);
+
 /*
  * We want to avoid the overhead of creating the hashtable, which is
  * comparatively large, when not necessary. Particularly when we are using a
@@ -139,6 +145,13 @@ struct TIDBitmap
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	dsa_pointer dsa_data;		/* dsa_pointer to the element array */
+	dsa_area   *dsa;			/* reference to per-query dsa area */
+	char	   *base;			/* pointer to the element array */
+	dsa_pointer dsapages;		/* dsa_pointer to the page array */
+	dsa_pointer dsachunks;		/* dsa_pointer to the chunk array */
+	RelptrPagetableEntry *relpages;		/* page array of relptr */
+	RelptrPagetableEntry *relchunks;	/* chunk array of relptr */
 };
 
 /*
@@ -156,6 +169,45 @@ struct TBMIterator
 	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
 };
 
+/* Shared TBMInfo to shared across multiple workers */
+typedef struct TBMSharedInfo
+{
+	dsa_pointer dsa_data;		/* dsa pointers to head of pagetable data */
+	dsa_pointer spages;			/* dsa pointer to page array */
+	dsa_pointer schunks;		/* dsa pointer to chunk array */
+	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
+	TBMStatus	status;			/* status of TIDBitmap */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+}	TBMSharedInfo;
+
+/*
+ * This stores the shared members of TBMSharedIterator so that multiple
+ * workers can operate on the same state. It also stores the TBMSharedInfo,
+ * in order to share relptrs of the chunk and the pages arrays and other
+ * TBM related information with other workers.
+ */
+typedef struct TBMSharedIteratorState
+{
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	LWLock		lock;			/* lock to protect above members */
+	TBMSharedInfo tbminfo;		/* TBM info to shared across workers */
+}	TBMSharedIteratorState;
+
+/*
+ * same as TBMIterator except that it holds a reference to the shared
+ * memory state so that multiple workers could operate on the same state.
+ */
+struct TBMSharedIterator
+{
+	TIDBitmap  *tbm;			/* TIDBitmap we're iterating over */
+	TBMSharedIteratorState *state;
+	TBMIterateResult output;
+};
 
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
@@ -168,6 +220,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+static int tbm_shared_comparator(const void *left, const void *right,
+					  void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -187,6 +241,7 @@ hash_blockno(BlockNumber b)
 }
 
 /* define hashtable mapping block numbers to PagetableEntry's */
+#define SH_USE_NONDEFAULT_ALLOCATOR
 #define SH_PREFIX pagetable
 #define SH_ELEMENT_TYPE PagetableEntry
 #define SH_KEY_TYPE BlockNumber
@@ -204,10 +259,12 @@ hash_blockno(BlockNumber b)
  *
  * The bitmap will live in the memory context that is CurrentMemoryContext
  * at the time of this call.  It will be limited to (approximately) maxbytes
- * total memory consumption.
+ * total memory consumption.  If dsa passed to this function is not NULL
+ * then the memory for storing elements of the underlying page table will
+ * be allocated from the DSA.
  */
 TIDBitmap *
-tbm_create(long maxbytes)
+tbm_create(long maxbytes, dsa_area *dsa)
 {
 	TIDBitmap  *tbm;
 	long		nbuckets;
@@ -230,6 +287,7 @@ tbm_create(long maxbytes)
 	nbuckets = Max(nbuckets, 16);		/* sanity limit */
 	tbm->maxentries = (int) nbuckets;
 	tbm->lossify_start = 0;
+	tbm->dsa = dsa;
 
 	return tbm;
 }
@@ -244,7 +302,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, tbm);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -777,6 +835,129 @@ tbm_iterate(TBMIterator *iterator)
 }
 
 /*
+ *	tbm_shared_iterate - scan through next page of a TIDBitmap
+ *
+ *	As above, but this will iterate using shared iterator which is shared
+ *	across multiple workers.  We need to acquire the iterator LWLock, before
+ *	accessing the shared members.
+ */
+TBMIterateResult *
+tbm_shared_iterate(TBMSharedIterator *iterator)
+{
+	TIDBitmap  *tbm = iterator->tbm;
+	TBMIterateResult *output = &iterator->output;
+	TBMSharedIteratorState *state = iterator->state;
+
+	Assert(tbm->iterating);
+
+	/* Acquire the LWLock before accessing the shared members */
+	LWLockAcquire(&state->lock, LW_EXCLUSIVE);
+
+	/*
+	 * If lossy chunk pages remain, make sure we've advanced schunkptr/
+	 * schunkbit to the next set bit.
+	 */
+	while (state->schunkptr < tbm->nchunks)
+	{
+		PagetableEntry *chunk = relptr_access(tbm->base,
+										   tbm->relchunks[state->schunkptr]);
+		int			schunkbit = state->schunkbit;
+
+		while (schunkbit < PAGES_PER_CHUNK)
+		{
+			int			wordnum = WORDNUM(schunkbit);
+			int			bitnum = BITNUM(schunkbit);
+
+			if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
+				break;
+			schunkbit++;
+		}
+		if (schunkbit < PAGES_PER_CHUNK)
+		{
+			state->schunkbit = schunkbit;
+			break;
+		}
+		/* advance to next chunk */
+		state->schunkptr++;
+		state->schunkbit = 0;
+	}
+
+	/*
+	 * If both chunk and per-page data remain, must output the numerically
+	 * earlier page.
+	 */
+	if (state->schunkptr < tbm->nchunks)
+	{
+		PagetableEntry *chunk;
+		PagetableEntry *page;
+		BlockNumber chunk_blockno;
+
+		chunk = relptr_access(tbm->base, tbm->relchunks[state->schunkptr]);
+		page = relptr_access(tbm->base, tbm->relpages[state->spageptr]);
+		chunk_blockno = chunk->blockno + state->schunkbit;
+
+		if (state->spageptr >= tbm->npages || chunk_blockno < page->blockno)
+		{
+			/* Return a lossy page indicator from the chunk */
+			output->blockno = chunk_blockno;
+			output->ntuples = -1;
+			output->recheck = true;
+			state->schunkbit++;
+
+			LWLockRelease(&state->lock);
+			return output;
+		}
+	}
+
+	if (state->spageptr < tbm->npages)
+	{
+		PagetableEntry *page;
+		int			ntuples;
+		int			wordnum;
+
+		/* In ONE_PAGE state, we don't allocate an spages[] array */
+		if (tbm->status == TBM_ONE_PAGE)
+			page = &tbm->entry1;
+		else
+			page = relptr_access(tbm->base, tbm->relpages[state->spageptr]);
+
+		/* scan bitmap to extract individual offset numbers */
+		ntuples = 0;
+		for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
+		{
+			bitmapword	w = page->words[wordnum];
+
+			if (w != 0)
+			{
+				int			off = wordnum * BITS_PER_BITMAPWORD + 1;
+
+				while (w != 0)
+				{
+					if (w & 1)
+						output->offsets[ntuples++] = (OffsetNumber) off;
+					off++;
+					w >>= 1;
+				}
+			}
+		}
+
+		output->blockno = page->blockno;
+		output->ntuples = ntuples;
+		output->recheck = page->recheck;
+		state->spageptr++;
+
+		LWLockRelease(&state->lock);
+
+		return output;
+	}
+
+	LWLockRelease(&state->lock);
+
+	/* Nothing more in the bitmap */
+	return NULL;
+}
+
+/*
  * tbm_end_iterate - finish an iteration over a TIDBitmap
  *
  * Currently this is just a pfree, but it might do more someday.  (For
@@ -790,6 +971,17 @@ tbm_end_iterate(TBMIterator *iterator)
 }
 
 /*
+ * tbm_end_shared_iterate - finish an iteration over a TIDBitmap
+ *
+ * As above, but it frees the memory of TBMSharedIterator.
+ */
+void
+tbm_end_shared_iterate(TBMSharedIterator *iterator)
+{
+	pfree(iterator);
+}
+
+/*
  * tbm_find_pageentry - find a PagetableEntry for the pageno
  *
  * Returns NULL if there is no non-lossy entry for the pageno.
@@ -1061,3 +1253,246 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * As above, but this will get relptrs (relative pointer) of PagetableEntry
+ * and it needs to convert it to actual PagetableEntry before comparing the
+ * blockno.
+ */
+static int
+tbm_shared_comparator(const void *left, const void *right, void *arg)
+{
+	PagetableEntry *lpage;
+	PagetableEntry *rpage;
+
+	lpage = relptr_access((char *) arg, *((RelptrPagetableEntry *) left));
+	rpage = relptr_access((char *) arg, *((RelptrPagetableEntry *) right));
+
+	if (lpage->blockno < rpage->blockno)
+		return -1;
+	else if (lpage->blockno > rpage->blockno)
+		return 1;
+	return 0;
+}
+
+/*
+ * tbm_prepare_shared_iterate - prepare to iterator through a TIDBitmap
+ * by multiple workers using shared iterator.
+ *
+ * The TBMSharedIteratorState will be allocated from DSA so that multiple
+ * worker can attach to it and iterate jointly.
+ *
+ * This will convert the pagetable hash into page and chunk array of
+ * reptr (relative pointer) so that these arrays can be shared across
+ * multiple workers.
+ */
+dsa_pointer
+tbm_prepare_shared_iterate(TIDBitmap *tbm)
+{
+	dsa_pointer iterator;
+	TBMSharedIteratorState *iterater_state;
+	TBMSharedInfo *tbminfo;
+
+	/*
+	 * Create the TBMIterator struct, with enough trailing space to serve the
+	 * needs of the TBMIterateResult sub-struct.
+	 */
+	iterator = dsa_allocate(tbm->dsa, sizeof(TBMSharedIteratorState));
+	iterater_state = dsa_get_address(tbm->dsa, iterator);
+	tbminfo = &iterater_state->tbminfo;
+
+	/*
+	 * If we have a hashtable, create and fill the sorted page lists, unless
+	 * we already did that for a previous iterator.  Note that the lists are
+	 * attached to the bitmap not the iterator, so they can be used by more
+	 * than one iterator.  However, we keep dsa_pointer to these in the shared
+	 * iterator so that other workers can have access to these and store in
+	 * their local TBM.
+	 */
+	if (tbm->status == TBM_HASH && !tbm->iterating)
+	{
+		pagetable_iterator i;
+		PagetableEntry *page;
+		int			npages;
+		int			nchunks;
+
+		/*
+		 * Create page list and chunk list using relptr so that we can share
+		 * this information across multiple workers.
+		 */
+		if (tbm->npages)
+		{
+			tbm->dsapages = dsa_allocate(tbm->dsa,
+							   tbm->npages * (sizeof(RelptrPagetableEntry)));
+			tbm->relpages = dsa_get_address(tbm->dsa, tbm->dsapages);
+		}
+		if (tbm->nchunks)
+		{
+			tbm->dsachunks = dsa_allocate(tbm->dsa,
+							  tbm->nchunks * (sizeof(RelptrPagetableEntry)));
+			tbm->relchunks = dsa_get_address(tbm->dsa, tbm->dsachunks);
+		}
+
+		tbm->base = dsa_get_address(tbm->dsa, tbm->dsa_data);
+
+		npages = nchunks = 0;
+		pagetable_start_iterate(tbm->pagetable, &i);
+		while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+		{
+			if (page->ischunk)
+				relptr_store(tbm->base, tbm->relchunks[nchunks++], page);
+			else
+				relptr_store(tbm->base, tbm->relpages[npages++], page);
+		}
+
+		Assert(npages == tbm->npages);
+		Assert(nchunks == tbm->nchunks);
+		if (npages > 1)
+			qsort_arg(tbm->relpages, npages, sizeof(RelptrPagetableEntry *),
+					  tbm_shared_comparator, (void *) tbm->base);
+		if (nchunks > 1)
+			qsort_arg(tbm->relchunks, nchunks, sizeof(RelptrPagetableEntry *),
+					  tbm_shared_comparator, (void *) tbm->base);
+	}
+
+	/*
+	 * Store the TBM member in the shared state.  This is done to shared it
+	 * across multiple workers.
+	 */
+	tbminfo->maxentries = tbm->maxentries;
+	tbminfo->nchunks = tbm->nchunks;
+	tbminfo->nentries = tbm->nentries;
+	tbminfo->npages = tbm->npages;
+	tbminfo->dsa_data = tbm->dsa_data;
+	tbminfo->spages = tbm->dsapages;
+	tbminfo->schunks = tbm->dsachunks;
+	tbminfo->status = tbm->status;
+
+	if (tbm->status == TBM_ONE_PAGE)
+		memcpy(&tbminfo->entry1, &tbm->entry1, sizeof(PagetableEntry));
+
+	/* Initialize the shared iterator state. */
+	iterater_state->schunkbit = 0;
+	iterater_state->schunkptr = 0;
+	iterater_state->spageptr = 0;
+
+
+	LWLockInitialize(&iterater_state->lock, LWTRANCHE_PARALLEL_TBM_ITERATOR);
+
+	tbm->iterating = true;
+
+	return iterator;
+}
+
+/*
+ *	tbm_attach_shared_iterate
+ *
+ *	Allocate an iterator and attach shared iterator state to it so that
+ *	multiple workers can access the same memory. We also need to copy
+ *	some of the TBM related information from shared state to TBM because
+ *	workers (other than the leader) would have created a local TBM therefore
+ *	they have to get these information from shared location.
+ */
+TBMSharedIterator *
+tbm_attach_shared_iterate(TIDBitmap *tbm, dsa_area *dsa, dsa_pointer iterator)
+{
+	TBMSharedIterator *shared_iterator;
+	TBMSharedInfo *tbminfo;
+
+	shared_iterator = (TBMSharedIterator *) palloc(sizeof(TBMIterator) +
+								 MAX_TUPLES_PER_PAGE * sizeof(OffsetNumber));
+
+	shared_iterator->tbm = tbm;
+	shared_iterator->state =
+		(TBMSharedIteratorState *) dsa_get_address(dsa, iterator);
+	tbminfo = &shared_iterator->state->tbminfo;
+
+	/*
+	 * If we have come here from worker other than the leader then we need to
+	 * get TBM related information from shared iterator status. This needs to
+	 * be done only once per worker.
+	 */
+	if (!tbm->iterating)
+	{
+		tbm->status = tbminfo->status;
+		tbm->nchunks = tbminfo->nchunks;
+		tbm->nentries = tbminfo->nentries;
+		tbm->npages = tbminfo->npages;
+		tbm->maxentries = tbminfo->maxentries;
+		if (tbm->status == TBM_HASH)
+		{
+			tbm->dsa_data = tbminfo->dsa_data;
+			tbm->base = dsa_get_address(dsa, tbm->dsa_data);
+
+			/* Convert dsa pointer to the local pointer */
+			if (tbm->npages)
+				tbm->relpages = dsa_get_address(tbm->dsa, tbminfo->spages);
+
+			if (tbm->nchunks)
+				tbm->relchunks = dsa_get_address(tbm->dsa, tbminfo->schunks);
+		}
+		else
+			memcpy(&tbm->entry1, &tbminfo->entry1, sizeof(PagetableEntry));
+
+		tbm->iterating = true;
+	}
+
+	return shared_iterator;
+}
+
+/*
+ * pagetable_allocate
+ *
+ * Callback function for allocating the memory for hashtable elements.
+ * It allocates memory from DSA if tbm holds a reference to a dsa.
+ */
+static inline void *
+pagetable_allocate(pagetable_hash *pagetable, Size size)
+{
+	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
+	dsa_pointer dsaptr;
+	char	   *ptr;
+
+	if (tbm->dsa == NULL)
+		return MemoryContextAllocExtended(pagetable->ctx, size,
+										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+
+	/* Add the size for storing dsa_pointer */
+	dsaptr = dsa_allocate(tbm->dsa, size + sizeof(dsa_pointer));
+
+	tbm->dsa_data = dsaptr;
+
+	ptr = dsa_get_address(tbm->dsa, dsaptr);
+	memset(ptr, 0, size + sizeof(dsa_pointer));
+
+	/* Store dsa_pointer */
+	*((dsa_pointer *) ptr) = dsaptr;
+
+	return (ptr + sizeof(dsa_pointer));
+}
+
+/*
+ * pagetable_free
+ *
+ * Callback function for freeing hash table elements.
+ */
+static inline void
+pagetable_free(pagetable_hash *pagetable, void *pointer)
+{
+	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
+	dsa_pointer dsa_data;
+
+	if (tbm->dsa == NULL)
+		return pfree(pointer);
+
+	/*
+	 * If TBM is in iterating phase that means pagetable is already created
+	 * and we have come here during tbm_free. By this time we are already
+	 * detached from the DSA because the GatherNode would have been shutdown.
+	 */
+	if (tbm->iterating)
+		return;
+
+	dsa_data = *((dsa_pointer *) ((char *) pointer - sizeof(dsa_pointer)));
+	dsa_free(tbm->dsa, dsa_data);
+}
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..17ed89d 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -23,6 +23,7 @@
 #define TIDBITMAP_H
 
 #include "storage/itemptr.h"
+#include "utils/dsa.h"
 
 
 /*
@@ -33,6 +34,7 @@ typedef struct TIDBitmap TIDBitmap;
 
 /* Likewise, TBMIterator is private */
 typedef struct TBMIterator TBMIterator;
+typedef struct TBMSharedIterator TBMSharedIterator;
 
 /* Result structure for tbm_iterate */
 typedef struct
@@ -46,7 +48,7 @@ typedef struct
 
 /* function prototypes in nodes/tidbitmap.c */
 
-extern TIDBitmap *tbm_create(long maxbytes);
+extern TIDBitmap *tbm_create(long maxbytes, dsa_area *dsa);
 extern void tbm_free(TIDBitmap *tbm);
 
 extern void tbm_add_tuples(TIDBitmap *tbm,
@@ -62,5 +64,10 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_end_shared_iterate(TBMSharedIterator *iterator);
+extern dsa_pointer tbm_prepare_shared_iterate(TIDBitmap *tbm);
+extern TBMSharedIterator *tbm_attach_shared_iterate(TIDBitmap *tbm,
+						  dsa_area *dsa, dsa_pointer iterator);
+extern TBMIterateResult *tbm_shared_iterate(TBMSharedIterator *iterator);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 8bd93c3..4d392e0 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -212,6 +212,7 @@ typedef enum BuiltinTrancheIds
 	LWTRANCHE_LOCK_MANAGER,
 	LWTRANCHE_PREDICATE_LOCK_MANAGER,
 	LWTRANCHE_PARALLEL_QUERY_DSA,
+	LWTRANCHE_PARALLEL_TBM_ITERATOR,
 	LWTRANCHE_FIRST_USER_DEFINED
 }	BuiltinTrancheIds;
 
0002-parallel-bitmap-heapscan-v2.patchapplication/octet-stream; name=0002-parallel-bitmap-heapscan-v2.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 5b67def..82c7c87 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1240,6 +1240,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 0be48fb..82805d6 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1754,6 +1754,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index fe87c9a..a2593be 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
 
 #include "executor/execParallel.h"
 #include "executor/executor.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "executor/nodeCustom.h"
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
@@ -205,6 +206,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -257,6 +262,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -733,6 +743,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index f18827d..91e032e 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,14 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -66,13 +70,16 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	HeapScanDesc scan;
 	TIDBitmap  *tbm;
 	TBMIterator *tbmiterator;
+	TBMSharedIterator *stbmiterator;
 	TBMIterateResult *tbmres;
-
+	ParallelBitmapInfo *shared_info = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
+	TBMSharedIterator *sprefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	dsa_area *dsa = node->ss.ps.state->es_query_dsa;
 
 	/*
 	 * extract necessary information from index scan node
@@ -81,12 +88,45 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	slot = node->ss.ss_ScanTupleSlot;
 	scan = node->ss.ss_currentScanDesc;
 	tbm = node->tbm;
+
 	tbmiterator = node->tbmiterator;
+	stbmiterator = node->stbmiterator;
+
 	tbmres = node->tbmres;
 #ifdef USE_PREFETCH
 	prefetch_iterator = node->prefetch_iterator;
+	sprefetch_iterator = node->sprefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	The first worker to see the state of ParallelBitmapInfo as PBM_INITIAL
+	 *	becomes leader and sets the state to PBM_INPROGRESS. All other workers
+	 *	see the state as PBM_INPROGRESS, and will wait for leader to finish
+	 *	building the TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *		ParallelBitmapInfo so that other worker can see those.
+	 *	  Convert the TIDBitmap into shared chunk and page array
+	 *	  Set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader creates shared TIDBitmap.
+	 *	  Copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over shared page and chunk
+	 *			array and select heap pages one by one.
+	 *	  b) Since multiple workers are iterating over same page and chunk
+	 *			array we need to have a shared iterator, so we grab a LWLock
+	 *			and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,21 +141,90 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.  In parallel mode leader will
+		 * immediately come out of the function, but all other workers will
+		 * be blocked until leader wake them up.
+		 */
+		if (shared_info == NULL || pbms_is_leader(shared_info))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag. This flag will be used to indicate the
+			 * underlying tidbitmap layer to allocate pagetable elements from
+			 * DSA.
+			 */
+			if (shared_info)
+				pbms_set_parallel(outerPlanState(node));
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			/*
+			 * In parallel mode, prepare the iterate over the TIDBitmap. This
+			 * will return the dsa_pointer to the shared iterator state. Which
+			 * will be used by multiple workers to iterate jointly.
+			 */
+			if (shared_info)
+			{
+				shared_info->tbmiterator = tbm_prepare_shared_iterate(tbm);
+#ifdef USE_PREFETCH
+				if (node->prefetch_maximum > 0)
+				{
+					shared_info->prefetch_iterator =
+										tbm_prepare_shared_iterate(tbm);
+				}
+
+				/*
+				 * By this time we have already created the shared iterator so it's
+				 * time to wake up other workers.
+				 */
+				SpinLockAcquire(&shared_info->state_mutex);
+				shared_info->state = PBM_FINISHED;
+				SpinLockRelease(&shared_info->state_mutex);
+
+				/* Wake up all other workers. */
+				ConditionVariableBroadcast(&shared_info->cv);
+#endif
+			}
+		}
+		else
+			tbm = tbm_create(work_mem * 1024L, dsa);
 
 		node->tbm = tbm;
-		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+
+		if (shared_info)
+		{
+			/*
+			 * Allocate worker specific iterator and attach the shared
+			 * iterator state to it.
+			 */
+			node->stbmiterator = stbmiterator = tbm_attach_shared_iterate(
+										   tbm, dsa, shared_info->tbmiterator);
+		}
+		else
+			node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+
 		node->tbmres = tbmres = NULL;
 
 #ifdef USE_PREFETCH
 		if (node->prefetch_maximum > 0)
 		{
-			node->prefetch_iterator = prefetch_iterator = tbm_begin_iterate(tbm);
-			node->prefetch_pages = 0;
-			node->prefetch_target = -1;
+			if (shared_info)
+			{
+				node->sprefetch_iterator = tbm_attach_shared_iterate(tbm, dsa,
+											   shared_info->prefetch_iterator);
+				sprefetch_iterator = node->sprefetch_iterator;
+			}
+			else
+			{
+				node->prefetch_iterator = prefetch_iterator =
+												tbm_begin_iterate(tbm);
+				node->prefetch_pages = 0;
+				node->prefetch_target = -1;
+			}
 		}
 #endif   /* USE_PREFETCH */
 	}
@@ -124,13 +233,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
+		int		   *prefetch_target;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			/*
+			 * If we are in parallel mode perform shared iterate otherwise
+			 * local iterate.
+			 */
+			if (!shared_info)
+				node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			else
+				node->tbmres = tbmres = tbm_shared_iterate(stbmiterator);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +256,50 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+			/*
+			 * If we are in parallel mode then acquire prefetch_mutex and
+			 * check prefetch_pages from shared location.
+			 */
+			if (shared_info)
 			{
+				SpinLockAcquire(&shared_info->prefetch_mutex);
+
 				/* The main iterator has closed the distance by one page */
-				node->prefetch_pages--;
+				if (shared_info->prefetch_pages > 0)
+					shared_info->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&shared_info->prefetch_mutex);
 			}
-			else if (prefetch_iterator)
+			else
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				/* The main iterator has closed the distance by one page */
+				 if (node->prefetch_pages > 0)
+					 node->prefetch_pages--;
+				else
+					need_prefetch = true;
+			}
+
+			if (prefetch_iterator && need_prefetch)
+			{
+				TBMIterateResult *tbmpre;
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				/* Do not let the prefetch iterator get behind the main one */
+				if (!shared_info)
+					tbmpre = tbm_iterate(prefetch_iterator);
+				else
+					tbmpre = tbm_shared_iterate(sprefetch_iterator);
+
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind the main iterator, but we can not
+				 * ensure that the current blockno in the main iterator and
+				 * in the prefetch iterator is same. It's possible that
+				 * whatever blockno we are prefetching will be processed
+				 * by another worker.
+				 */
+				if ((shared_info == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -183,19 +334,40 @@ BitmapHeapNext(BitmapHeapScanState *node)
 #ifdef USE_PREFETCH
 
 			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
+			 * If we are in shared mode then use prefetch_target from shared
+			 * location i.e pbminfo otherwise directly from node.
 			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
+			if (shared_info == NULL)
+				prefetch_target = &node->prefetch_target;
 			else
-				node->prefetch_target++;
+				prefetch_target = &shared_info->prefetch_target;
+
+			/* Increase prefetch target if it's not yet at the max. */
+			if (*prefetch_target < node->prefetch_maximum)
+			{
+				/* If we are in parallel mode then grab prefetch_mutex */
+				if (shared_info != NULL)
+					SpinLockAcquire(&shared_info->prefetch_mutex);
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (shared_info != NULL)
+					SpinLockRelease(&shared_info->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +383,25 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (shared_info == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				if (shared_info->prefetch_target < node->prefetch_maximum)
+				{
+					SpinLockAcquire(&shared_info->prefetch_mutex);
+					if (shared_info->prefetch_target < node->prefetch_maximum)
+						shared_info->prefetch_target++;
+					SpinLockRelease(&shared_info->prefetch_mutex);
+				}
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +425,67 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (shared_info)
+			{
+				prefetch_pages = &shared_info->prefetch_pages;
+				prefetch_target = shared_info->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
+
+			/*
+			 * We are checking the prefetch_pages without mutex.  Henceforth,
+			 * in parallel mode there can be some extra prefetch.  Should
+			 * we acquire mutex and recheck before iterating?
+			 */
+			while (*prefetch_pages < prefetch_target)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
+
+				if (!shared_info)
+					tbmpre = tbm_iterate(prefetch_iterator);
+				else
+					tbmpre = tbm_shared_iterate(sprefetch_iterator);
 
 				if (tbmpre == NULL)
 				{
-					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					if (!shared_info)
+					{
+						/* No more pages to prefetch */
+						tbm_end_iterate(prefetch_iterator);
+						node->prefetch_iterator = prefetch_iterator = NULL;
+					}
+					else
+					{
+						/* No more pages to prefetch */
+						tbm_end_shared_iterate(sprefetch_iterator);
+						node->sprefetch_iterator = sprefetch_iterator = NULL;
+					}
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (shared_info != NULL)
+					SpinLockAcquire(&shared_info->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (shared_info != NULL)
+					SpinLockRelease(&shared_info->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -458,6 +693,10 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->tbmiterator);
 	if (node->prefetch_iterator)
 		tbm_end_iterate(node->prefetch_iterator);
+	if (node->stbmiterator)
+		tbm_end_shared_iterate(node->stbmiterator);
+	if (node->sprefetch_iterator)
+		tbm_end_shared_iterate(node->sprefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
 	node->tbm = NULL;
@@ -465,6 +704,16 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -514,6 +763,10 @@ ExecEndBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->tbmiterator);
 	if (node->prefetch_iterator)
 		tbm_end_iterate(node->prefetch_iterator);
+	if (node->stbmiterator)
+		tbm_end_shared_iterate(node->stbmiterator);
+	if (node->sprefetch_iterator)
+		tbm_end_shared_iterate(node->sprefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
 
@@ -567,6 +820,7 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -651,3 +905,160 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ *		pbms_is_leader
+ *
+ * 		If PBMState is PBM_INITIAL then we become the leader and set the state
+ *		to PBM_INPROGRESS.  Otherwise we become the worker therefore we need to
+ *		wait for the leader to wake us up and also PBMState should be
+ *		PBM_FINISHED.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : then we need to wait till leader creates BITMAP
+		 * PBM_FINISHED   : BITMAP is ready so no need to wait.
+		 *---------------
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/*-------------------
+ *		pbms_set_parallel
+ *
+ *		Recursively process the node and set the parallel flag. This flag
+ *		will be used to indicate the underlying tidbitmap layer to allocate
+ *		pagetable elements from DSA.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapAnd, set first bitmap index scan node as parallel
+	 * because only first node will create the main bitmap other node's bitmap
+	 * will be merged to the first bitmap thus no need to create them in shared
+	 * memory.  BitmapOr and BitmapIndex will create the TidBitmap so set the
+	 * parallel flag true.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			((BitmapOrState *) node)->is_parallel = true;;
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+
+	/* Estimate the size for sharing parallel Bitmap info. */
+	node->pscan_len = add_size(offsetof(ParallelBitmapInfo,
+							   phs_snapshot_data),
+							   EstimateSnapshotSpace(estate->es_snapshot));
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 94bb289..2f01cad 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,8 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L, NULL);
+		tbm = tbm_create(work_mem * 1024L,
+				node->biss_Parallel ? node->ss.ps.state->es_query_dsa: NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 1d280be..e8f110b 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,8 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L, NULL);
+				result = tbm_create(work_mem * 1024L, node->is_parallel ?
+									node->ps.state->es_query_dsa: NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 85505c5..b7f7ce7 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2877,6 +2877,31 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 }
 
 /*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched, 0);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+/*
  * Compute the number of parallel workers that should be used to scan a
  * relation.  We compute the parallel workers based on the size of the heap to
  * be scanned and the size of the index to be scanned, then choose a minimum
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a43daa7..f81e469 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -816,6 +816,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -877,8 +878,32 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
+
+		/*
+		 * It may be possible to amortize some of the I/O cost, but probably
+		 * not very much, because most operating systems already do aggressive
+		 * prefetching.  For now, we assume that the disk run cost can't be
+		 * amortized at all.
+		 */
+
+		/*
+		 * In the case of a parallel plan, the row count needs to represent
+		 * the number of tuples processed per worker.
+		 */
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 5283468..630c501 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1557,6 +1561,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1599,6 +1608,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 997bdcf..6d152f7 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2834,7 +2834,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f440875..20b9a59 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1071,7 +1071,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1080,9 +1081,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3258,7 +3259,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 7176cf1..f8eef0b 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3392,6 +3392,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index a864f78..7e85510 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -179,6 +179,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 42c6c58..56f4436 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1261,6 +1263,7 @@ typedef struct BitmapOrState
 	PlanState	ps;				/* its first field is NodeTag */
 	PlanState **bitmapplans;	/* array of PlanStates for my inputs */
 	int			nplans;			/* number of input plans */
+	bool		is_parallel;	/* create shared tbm if it's set */
 } BitmapOrState;
 
 /* ----------------------------------------------------------------
@@ -1427,6 +1430,59 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		tbmiterator				iterator for scanning current pages
+ *		prefetch_iterator		iterator for prefetching ahead of current page
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	dsa_pointer	tbmiterator;
+	dsa_pointer	prefetch_iterator;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1441,6 +1497,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1457,6 +1515,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1472,7 +1531,9 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1487,6 +1548,10 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	TBMSharedIterator *stbmiterator;
+	TBMSharedIterator *sprefetch_iterator;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 7b41317..cfef818 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -52,7 +52,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 81e7a42..51d6c09 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -54,6 +54,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index de8225b..c52c864 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -786,6 +786,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#81Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#80)
Re: Parallel bitmap heap scan

On Tue, Feb 14, 2017 at 12:18 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

Fixed.

Thanks, the external interface to this looks much cleaner now.
Scrutinizing the internals:

What is the point of having a TBMSharedIterator contain a TIDBitmap
pointer? All the values in that TIDBitmap are populated from the
TBMSharedInfo, but it seems to me that the fields that are copied over
unchanged (like status and nchunks) could just be used directly from
the TBMSharedInfo, and the fields that are computed (like relpages and
relchunks) could be stored directly in the TBMSharedIterator.
tbm_shared_iterate() is already separate code from tbm_iterate(), so
it can be changed to refer to whichever data structure contains the
data it needs.

Why is TBMSharedInfo separate from TBMSharedIteratorState? It seems
like you could merge those two into a single structure.

I think we can simplify things here a bit by having shared iterators
not support single-page mode. In the backend-private case,
tbm_begin_iterate() really doesn't need to do anything with the
pagetable in the TBM_ONE_PAGE case, but for a shared iterator, we've
got to anyway copy the pagetable into shared memory. So it seems to
me that it would be simpler just to transform it into a standard
iteration array while we're at it, instead of copying it into entry1.
In other words, I suggest removing both "entry1" and "status" from
TBMSharedInfo and making tbm_prepare_shared_iterate smarter to
compensate.

I think "dsa_data" is poorly named; it should be something like
"dsapagetable" in TIDBitmap and just "pagetable" in TBMSharedInfo. I
think you should should move the "base", "relpages", and "relchunks"
into TBMSharedIterator and give them better names, like "ptbase",
"ptpages" and "ptchunks". "base" isn't clear that we're talking about
the pagetable's base as opposed to anything else, and "relpages" and
"relchunks" are named based on the fact that the pointers are relative
rather than named based on what data they point at, which doesn't seem
like the right choice.

I suggest putting the parallel functions next to each other in the
file: tbm_begin_iterate(), tbm_prepare_shared_iterate(),
tbm_iterate(), tbm_shared_iterate(), tbm_end_iterate(),
tbm_end_shared_iterate().

+ if (tbm->dsa == NULL)
+ return pfree(pointer);

Don't try to return a value of type void. The correct spelling of
this is { pfree(pointer); return; }. Formatted appropriately across 4
lines, of course.

+    /*
+     * If TBM is in iterating phase that means pagetable is already created
+     * and we have come here during tbm_free. By this time we are already
+     * detached from the DSA because the GatherNode would have been shutdown.
+     */
+    if (tbm->iterating)
+        return;

This seems like something of a kludge, although not a real easy one to
fix. The problem here is that tidbitmap.c ideally shouldn't have to
know that it's being used by the executor or that there's a Gather
node involved, and certainly not the order of operations around
shutdown. It should really be the problem of 0002 to handle this kind
of problem, without 0001 having to know anything about it. It strikes
me that it would probably be a good idea to have Gather clean up its
children before it gets cleaned up itself. So in ExecShutdownNode, we
could do something like this:

diff --git a/src/backend/executor/execProcnode.c
b/src/backend/executor/execProcnode.c
index 0dd95c6..5ccc2e8 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -815,6 +815,8 @@ ExecShutdownNode(PlanState *node)
     if (node == NULL)
         return false;
+    planstate_tree_walker(node, ExecShutdownNode, NULL);
+
     switch (nodeTag(node))
     {
         case T_GatherState:
@@ -824,5 +826,5 @@ ExecShutdownNode(PlanState *node)
             break;
     }
-    return planstate_tree_walker(node, ExecShutdownNode, NULL);
+    return false;
 }

Also, in ExecEndGather, something like this:

diff --git a/src/backend/executor/nodeGather.c
b/src/backend/executor/nodeGather.c
index a1a3561..32c97d3 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -229,10 +229,10 @@ ExecGather(GatherState *node)
 void
 ExecEndGather(GatherState *node)
 {
+    ExecEndNode(outerPlanState(node));      /* let children clean up first */
     ExecShutdownGather(node);
     ExecFreeExprContext(&node->ps);
     ExecClearTuple(node->ps.ps_ResultTupleSlot);
-    ExecEndNode(outerPlanState(node));
 }

/*

With those changes and an ExecShutdownBitmapHeapScan() called from
ExecShutdownNode(), it should be possible (I think) for us to always
have the bitmap heap scan node shut down before the Gather node shuts
down, which I think would let you avoid having a special case for this
inside the TBM code.

+    char       *ptr;
+    dsaptr = dsa_allocate(tbm->dsa, size + sizeof(dsa_pointer));
+    tbm->dsa_data = dsaptr;
+    ptr = dsa_get_address(tbm->dsa, dsaptr);
+    memset(ptr, 0, size + sizeof(dsa_pointer));
+    *((dsa_pointer *) ptr) = dsaptr;

Hmm. Apparently, we need a dsa_allocate_and_zero() or dsa_allocate0()
function. This pattern seems likely to come up a lot, and I don't
think we should require every caller to deal with it.

I don't see why you think you need to add sizeof(dsa_pointer) to the
allocation and store an extra copy of the dsa_pointer in the
additional space. You are already storing it in tbm->dsa_data and
that seems good enough. pagetable_free() needs the value, but it has
a pointer to the TIDBitmap and could just pass tbm->dsa_data directly
instead of fishing the pointer out of the extra space. Also, this
kind of thing is in general not very wise because it tends to cause
alignment faults on some picky machine. dsa_allocate() will return a
MAXALIGN'd pointer but ptr + sizeof(dsa_pointer) will only be
MAXALIGN'd if (sizeof(dsa_pointer) % MAXIMUM_ALIGNOF) == 0, which I
suspect might not be true everywhere and which is a shaky assumption
for the future even if it happens to be true today. For example,
consider a 32-bit platform where doubles need to be 8-byte aligned.

It seems shaky to me that tbm->iterating can be set when we've created
either a shared or a backend-private iterator. For example,
tbm_iterate() asserts that tbm->iterating is set as a way of checking
that spages and schunks will be set. But that's not guaranteed any
more with these changes, because we might've built the shared version
of the iteration arrays rather than the backend-private version.
Maybe you ought to change it from a bool to an enum:
TBM_NOT_ITERATING, TBM_ITERATING_PRIVATE, TBM_ITERATING_SHARED. And
then update all of the asserts and tests to check for the specific
state they care about.

+        while (schunkbit < PAGES_PER_CHUNK)
+        {
+            int            wordnum = WORDNUM(schunkbit);
+            int            bitnum = BITNUM(schunkbit);
+
+            if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
+                break;
+            schunkbit++;
+        }

How about refactoring this chunk of code into a static inline function
and having both tbm_iterate() and tbm_shared_iterate() call it?
Probably something like static inline void
tbm_advance_schunkbit(PageTableEntry *chunk, int *schunkbit).

+        /* scan bitmap to extract individual offset numbers */
+        ntuples = 0;
+        for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
+        {
+            bitmapword    w = page->words[wordnum];
+
+            if (w != 0)
+            {
+                int            off = wordnum * BITS_PER_BITMAPWORD + 1;
+
+                while (w != 0)
+                {
+                    if (w & 1)
+                        output->offsets[ntuples++] = (OffsetNumber) off;
+                    off++;
+                    w >>= 1;
+                }
+            }
+        }

Similarly, this looks like it could be refactored into a shared static
inline function as well, instead of duplicating it.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#82Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#81)
Re: Parallel bitmap heap scan

On Tue, Feb 14, 2017 at 10:18 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Thanks, the external interface to this looks much cleaner now.
Scrutinizing the internals:

Thanks for the comments, I am working on these. I have few doubts for
some of the comments.

I suggest removing both "entry1" and "status" from
TBMSharedInfo and making tbm_prepare_shared_iterate smarter to
compensate.

This is easily doable and it will look much cleaner. While doing this
I am facing one problem related to
relptr_store and relptr_access. The problem is that when I call
"relptr_store (base, rp, val)" with both base and val as a same
address then it will store offset as 0 which is fine, but when we use
relptr_access with 0 offset it is returning NULL instead of giving the
actual address. I expect it should return directly base when offset is
0. (I am facing this problem because in this case (TBM_ONE_PAGE),
there will be only one page and address of that will be same as
base.).

The question can be why we did not face this issue while converting
the pagetable to page array, because at least one entry will be there
which should have the same address as the base. But coincidently we
did not get that problem because in that case as of now we were
storing dsa_pointer in the head of the element array, therefore first
pagetable element was not same as the base.

There can be multiple solutions to this but none of them looks cleaner to me.
sol1: If relptr_access return NULL then directly use the base address
as our PagetableEntry.
sol2: Instead of using base as start of the element array, just try to
use some modified base as e.g base=base - some number.
sol3: change relptr_access to not return NULL in this case. But, this
will change the current behaviour of this interface and need to
analyze the side effects.

I don't see why you think you need to add sizeof(dsa_pointer) to the
allocation and store an extra copy of the dsa_pointer in the
additional space. You are already storing it in tbm->dsa_data and
that seems good enough. pagetable_free() needs the value, but it has
a pointer to the TIDBitmap and could just pass tbm->dsa_data directly
instead of fishing the pointer out of the extra space.

If you see the code of SH_GROW, first it needs to allocate the bigger
chunk copy data from smaller chunk to the bigger chunk and then free
the smaller chunk.

So by the time it call the pagetable_free, it would have already
called the pagetable_allocate for the newer bigger chunk of memory so
now, tbm->dsa_data points to the latest memory, but pagetable_free
wants to free older one.

One solution to this could be that I keep two dsa_pointer in TBM, one
point to old memory and another to new. (After this here we will get
the same problem of relptr_access because now we will have first entry
pointer is same as base pointer)

Please provide your thought.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#83Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#82)
Re: Parallel bitmap heap scan

On Wed, Feb 15, 2017 at 7:17 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

This is easily doable and it will look much cleaner. While doing this
I am facing one problem related to
relptr_store and relptr_access. The problem is that when I call
"relptr_store (base, rp, val)" with both base and val as a same
address then it will store offset as 0 which is fine, but when we use
relptr_access with 0 offset it is returning NULL instead of giving the
actual address. I expect it should return directly base when offset is
0. (I am facing this problem because in this case (TBM_ONE_PAGE),
there will be only one page and address of that will be same as
base.).

There can be multiple solutions to this but none of them looks cleaner to me.
sol1: If relptr_access return NULL then directly use the base address
as our PagetableEntry.
sol2: Instead of using base as start of the element array, just try to
use some modified base as e.g base=base - some number.
sol3: change relptr_access to not return NULL in this case. But, this
will change the current behaviour of this interface and need to
analyze the side effects.

Hmm, yeah, that's a problem. How about not using relative pointers
here, and instead just using array indexes?

I don't see why you think you need to add sizeof(dsa_pointer) to the
allocation and store an extra copy of the dsa_pointer in the
additional space. You are already storing it in tbm->dsa_data and
that seems good enough. pagetable_free() needs the value, but it has
a pointer to the TIDBitmap and could just pass tbm->dsa_data directly
instead of fishing the pointer out of the extra space.

If you see the code of SH_GROW, first it needs to allocate the bigger
chunk copy data from smaller chunk to the bigger chunk and then free
the smaller chunk.

Oh, I see.

So by the time it call the pagetable_free, it would have already
called the pagetable_allocate for the newer bigger chunk of memory so
now, tbm->dsa_data points to the latest memory, but pagetable_free
wants to free older one.

One solution to this could be that I keep two dsa_pointer in TBM, one
point to old memory and another to new. (After this here we will get
the same problem of relptr_access because now we will have first entry
pointer is same as base pointer)

Yes, two dsa_pointers seems fine. Maybe that's not totally beautiful,
but it seems better than what you had before.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#84Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#81)
4 attachment(s)
Re: Parallel bitmap heap scan

On Tue, Feb 14, 2017 at 10:18 PM, Robert Haas <robertmhaas@gmail.com> wrote:

What is the point of having a TBMSharedIterator contain a TIDBitmap
pointer? All the values in that TIDBitmap are populated from the
TBMSharedInfo, but it seems to me that the fields that are copied over
unchanged (like status and nchunks) could just be used directly from
the TBMSharedInfo, and the fields that are computed (like relpages and
relchunks) could be stored directly in the TBMSharedIterator.
tbm_shared_iterate() is already separate code from tbm_iterate(), so
it can be changed to refer to whichever data structure contains the
data it needs.

Done

Why is TBMSharedInfo separate from TBMSharedIteratorState? It seems
like you could merge those two into a single structure.

Done

I think we can simplify things here a bit by having shared iterators
not support single-page mode. In the backend-private case,
tbm_begin_iterate() really doesn't need to do anything with the
pagetable in the TBM_ONE_PAGE case, but for a shared iterator, we've
got to anyway copy the pagetable into shared memory. So it seems to
me that it would be simpler just to transform it into a standard
iteration array while we're at it, instead of copying it into entry1.
In other words, I suggest removing both "entry1" and "status" from
TBMSharedInfo and making tbm_prepare_shared_iterate smarter to
compensate.

Done

I think "dsa_data" is poorly named; it should be something like
"dsapagetable" in TIDBitmap and just "pagetable" in TBMSharedInfo. I
think you should should move the "base", "relpages", and "relchunks"
into TBMSharedIterator and give them better names, like "ptbase",
"ptpages" and "ptchunks". "base" isn't clear that we're talking about
the pagetable's base as opposed to anything else, and "relpages" and
"relchunks" are named based on the fact that the pointers are relative
rather than named based on what data they point at, which doesn't seem
like the right choice.

Done

I suggest putting the parallel functions next to each other in the
file: tbm_begin_iterate(), tbm_prepare_shared_iterate(),
tbm_iterate(), tbm_shared_iterate(), tbm_end_iterate(),
tbm_end_shared_iterate().

Done

+ if (tbm->dsa == NULL)
+ return pfree(pointer);

Don't try to return a value of type void. The correct spelling of
this is { pfree(pointer); return; }. Formatted appropriately across 4
lines, of course.

Fixed

+    /*
+     * If TBM is in iterating phase that means pagetable is already created
+     * and we have come here during tbm_free. By this time we are already
+     * detached from the DSA because the GatherNode would have been shutdown.
+     */
+    if (tbm->iterating)
+        return;

This seems like something of a kludge, although not a real easy one to
fix. The problem here is that tidbitmap.c ideally shouldn't have to
know that it's being used by the executor or that there's a Gather
node involved, and certainly not the order of operations around
shutdown. It should really be the problem of 0002 to handle this kind
of problem, without 0001 having to know anything about it. It strikes
me that it would probably be a good idea to have Gather clean up its
children before it gets cleaned up itself. So in ExecShutdownNode, we
could do something like this:

diff --git a/src/backend/executor/execProcnode.c
b/src/backend/executor/execProcnode.c
index 0dd95c6..5ccc2e8 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -815,6 +815,8 @@ ExecShutdownNode(PlanState *node)
if (node == NULL)
return false;
+    planstate_tree_walker(node, ExecShutdownNode, NULL);
+
switch (nodeTag(node))
{
case T_GatherState:
@@ -824,5 +826,5 @@ ExecShutdownNode(PlanState *node)
break;
}
-    return planstate_tree_walker(node, ExecShutdownNode, NULL);
+    return false;
}

Also, in ExecEndGather, something like this:

diff --git a/src/backend/executor/nodeGather.c
b/src/backend/executor/nodeGather.c
index a1a3561..32c97d3 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -229,10 +229,10 @@ ExecGather(GatherState *node)
void
ExecEndGather(GatherState *node)
{
+    ExecEndNode(outerPlanState(node));      /* let children clean up first */
ExecShutdownGather(node);
ExecFreeExprContext(&node->ps);
ExecClearTuple(node->ps.ps_ResultTupleSlot);
-    ExecEndNode(outerPlanState(node));
}

/*

With those changes and an ExecShutdownBitmapHeapScan() called from
ExecShutdownNode(), it should be possible (I think) for us to always
have the bitmap heap scan node shut down before the Gather node shuts
down, which I think would let you avoid having a special case for this
inside the TBM code.

Done
(gather_shutdown_children_first.patch does what you have mentioned
above and 0002 implement the BitmapHeapScanShutdown function).

+    char       *ptr;
+    dsaptr = dsa_allocate(tbm->dsa, size + sizeof(dsa_pointer));
+    tbm->dsa_data = dsaptr;
+    ptr = dsa_get_address(tbm->dsa, dsaptr);
+    memset(ptr, 0, size + sizeof(dsa_pointer));
+    *((dsa_pointer *) ptr) = dsaptr;

Hmm. Apparently, we need a dsa_allocate_and_zero() or dsa_allocate0()
function. This pattern seems likely to come up a lot, and I don't
think we should require every caller to deal with it.

Done

I don't see why you think you need to add sizeof(dsa_pointer) to the
allocation and store an extra copy of the dsa_pointer in the
additional space. You are already storing it in tbm->dsa_data and

Done with the help of two pointers as discussed in another mail [1]/messages/by-id/CA+TgmoaWsCJ5L2du9i1RQaegaNLgOYF2KgRFu+7sUAeQc_xBFg@mail.gmail.com.

[1]: /messages/by-id/CA+TgmoaWsCJ5L2du9i1RQaegaNLgOYF2KgRFu+7sUAeQc_xBFg@mail.gmail.com

It seems shaky to me that tbm->iterating can be set when we've created
either a shared or a backend-private iterator. For example,
tbm_iterate() asserts that tbm->iterating is set as a way of checking
that spages and schunks will be set. But that's not guaranteed any
more with these changes, because we might've built the shared version
of the iteration arrays rather than the backend-private version.
Maybe you ought to change it from a bool to an enum:
TBM_NOT_ITERATING, TBM_ITERATING_PRIVATE, TBM_ITERATING_SHARED. And
then update all of the asserts and tests to check for the specific
state they care about.

Done

+        while (schunkbit < PAGES_PER_CHUNK)
+        {
+            int            wordnum = WORDNUM(schunkbit);
+            int            bitnum = BITNUM(schunkbit);
+
+            if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
+                break;
+            schunkbit++;
+        }

How about refactoring this chunk of code into a static inline function
and having both tbm_iterate() and tbm_shared_iterate() call it?
Probably something like static inline void
tbm_advance_schunkbit(PageTableEntry *chunk, int *schunkbit).

Good idea, done this way

+        /* scan bitmap to extract individual offset numbers */
+        ntuples = 0;
+        for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
+        {
+            bitmapword    w = page->words[wordnum];
+
+            if (w != 0)
+            {
+                int            off = wordnum * BITS_PER_BITMAPWORD + 1;
+
+                while (w != 0)
+                {
+                    if (w & 1)
+                        output->offsets[ntuples++] = (OffsetNumber) off;
+                    off++;
+                    w >>= 1;
+                }
+            }
+        }

Similarly, this looks like it could be refactored into a shared static
inline function as well, instead of duplicating it.

Done.

Attached patches:

interface_dsa_allocate0.patch : Provides new interface dsa_allocate0,
which is used by 0001
gather_shutdown_childeren_first.patch : Processing the child node
before shuting down the gather, 0002 will use that part to shutdown
BitmapNode before gather.

0001: tidbimap change to provide the interfaces for shared iterator.
0002: actual parallel bitmap heap scan by using interfaces of 0001.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

interface_dsa_allocate0.patchapplication/octet-stream; name=interface_dsa_allocate0.patchDownload
diff --git a/src/backend/utils/mmgr/dsa.c b/src/backend/utils/mmgr/dsa.c
index 7dc43f1..f0b2a27 100644
--- a/src/backend/utils/mmgr/dsa.c
+++ b/src/backend/utils/mmgr/dsa.c
@@ -756,6 +756,22 @@ dsa_allocate(dsa_area *area, Size size)
 }
 
 /*
+ * As above, but clears allocated memory
+ */
+dsa_pointer
+dsa_allocate0(dsa_area *area, Size size)
+{
+	dsa_pointer dp;
+	char	   *object;
+
+	dp = dsa_allocate(area, size);
+	object = dsa_get_address(area, dp);
+	memset(object, 0, size);
+
+	return dp;
+}
+
+/*
  * Free memory obtained with dsa_allocate.
  */
 void
diff --git a/src/include/utils/dsa.h b/src/include/utils/dsa.h
index bb634e7..03d6a56 100644
--- a/src/include/utils/dsa.h
+++ b/src/include/utils/dsa.h
@@ -110,5 +110,6 @@ extern void dsa_free(dsa_area *area, dsa_pointer dp);
 extern void *dsa_get_address(dsa_area *area, dsa_pointer dp);
 extern void dsa_trim(dsa_area *area);
 extern void dsa_dump(dsa_area *area);
+extern dsa_pointer dsa_allocate0(dsa_area *area, Size size);
 
 #endif   /* DSA_H */
gather_shutdown_children_first.patchapplication/octet-stream; name=gather_shutdown_children_first.patchDownload
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 0dd95c6..5ccc2e8 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -815,6 +815,8 @@ ExecShutdownNode(PlanState *node)
 	if (node == NULL)
 		return false;
 
+	planstate_tree_walker(node, ExecShutdownNode, NULL);
+
 	switch (nodeTag(node))
 	{
 		case T_GatherState:
@@ -824,5 +826,5 @@ ExecShutdownNode(PlanState *node)
 			break;
 	}
 
-	return planstate_tree_walker(node, ExecShutdownNode, NULL);
+	return false;
 }
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index a1a3561..32c97d3 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -229,10 +229,10 @@ ExecGather(GatherState *node)
 void
 ExecEndGather(GatherState *node)
 {
+	ExecEndNode(outerPlanState(node));		/* let children clean up first */
 	ExecShutdownGather(node);
 	ExecFreeExprContext(&node->ps);
 	ExecClearTuple(node->ps.ps_ResultTupleSlot);
-	ExecEndNode(outerPlanState(node));
 }
 
 /*
0001-tidbitmap-support-shared-v3.patchapplication/octet-stream; name=0001-tidbitmap-support-shared-v3.patchDownload
diff --git a/src/backend/access/gin/ginget.c b/src/backend/access/gin/ginget.c
index 60f005c..87cd9ea 100644
--- a/src/backend/access/gin/ginget.c
+++ b/src/backend/access/gin/ginget.c
@@ -120,7 +120,7 @@ collectMatchBitmap(GinBtreeData *btree, GinBtreeStack *stack,
 	Form_pg_attribute attr;
 
 	/* Initialize empty bitmap result */
-	scanEntry->matchBitmap = tbm_create(work_mem * 1024L);
+	scanEntry->matchBitmap = tbm_create(work_mem * 1024L, NULL);
 
 	/* Null query cannot partial-match anything */
 	if (scanEntry->isPartialMatch &&
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..94bb289 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,7 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L);
+		tbm = tbm_create(work_mem * 1024L, NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 3c6155b..1d280be 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,7 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L);
+				result = tbm_create(work_mem * 1024L, NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..ac6bb7b 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -43,6 +43,8 @@
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
 #include "nodes/tidbitmap.h"
+#include "storage/lwlock.h"
+#include "utils/dsa.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -121,6 +123,16 @@ typedef enum
 } TBMStatus;
 
 /*
+ * Current iterating state of the TBM.
+ */
+typedef enum
+{
+	TBM_NOT_ITERATING,			/* not yet converted to page and chunk array */
+	TBM_ITERATING_PRIVATE,		/* converted to local page and chunk array */
+	TBM_ITERATING_SHARED		/* converted to shared page and chunk array */
+} TBMIteratingState;
+
+/*
  * Here is the representation for a whole TIDBitMap:
  */
 struct TIDBitmap
@@ -133,12 +145,17 @@ struct TIDBitmap
 	int			maxentries;		/* limit on same to meet maxbytes */
 	int			npages;			/* number of exact entries in pagetable */
 	int			nchunks;		/* number of lossy entries in pagetable */
-	bool		iterating;		/* tbm_begin_iterate called? */
+	TBMIteratingState iterating;	/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	dsa_pointer dsapagetable;	/* dsa_pointer to the element array */
+	dsa_pointer dsapagetableold;	/* dsa_pointer to the old element array */
+	dsa_area   *dsa;			/* reference to per-query dsa area */
+	dsa_pointer ptpages;		/* dsa_pointer to the page array */
+	dsa_pointer ptchunks;		/* dsa_pointer to the chunk array */
 };
 
 /*
@@ -156,6 +173,40 @@ struct TBMIterator
 	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
 };
 
+/*
+ * This stores the shared members of TBMSharedIterator so that multiple
+ * workers can operate on the same state. It also stores the TBMSharedInfo,
+ * in order to share relptrs of the chunk and the pages arrays and other
+ * TBM related information with other workers.
+ */
+typedef struct TBMSharedIteratorState
+{
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+	LWLock		lock;			/* lock to protect above members */
+	dsa_pointer pagetable;		/* dsa pointers to head of pagetable data */
+	dsa_pointer spages;			/* dsa pointer to page array */
+	dsa_pointer schunks;		/* dsa pointer to chunk array */
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+} TBMSharedIteratorState;
+
+/*
+ * same as TBMIterator except that it holds a reference to the shared
+ * memory state so that multiple workers could operate on the same state.
+ */
+struct TBMSharedIterator
+{
+	PagetableEntry *ptbase;		/* pointer to the pagetable element array */
+	dsa_area   *dsa;			/* reference to per-query dsa area */
+	int		   *ptpages;		/* sorted exact page index list */
+	int		   *ptchunks;		/* sorted lossy page index list */
+	TBMSharedIteratorState *state;		/* shared members */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+};
 
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
@@ -168,6 +219,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+static int tbm_shared_comparator(const void *left, const void *right,
+					  void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -187,6 +240,7 @@ hash_blockno(BlockNumber b)
 }
 
 /* define hashtable mapping block numbers to PagetableEntry's */
+#define SH_USE_NONDEFAULT_ALLOCATOR
 #define SH_PREFIX pagetable
 #define SH_ELEMENT_TYPE PagetableEntry
 #define SH_KEY_TYPE BlockNumber
@@ -204,10 +258,12 @@ hash_blockno(BlockNumber b)
  *
  * The bitmap will live in the memory context that is CurrentMemoryContext
  * at the time of this call.  It will be limited to (approximately) maxbytes
- * total memory consumption.
+ * total memory consumption.  If dsa passed to this function is not NULL
+ * then the memory for storing elements of the underlying page table will
+ * be allocated from the DSA.
  */
 TIDBitmap *
-tbm_create(long maxbytes)
+tbm_create(long maxbytes, dsa_area *dsa)
 {
 	TIDBitmap  *tbm;
 	long		nbuckets;
@@ -230,6 +286,7 @@ tbm_create(long maxbytes)
 	nbuckets = Max(nbuckets, 16);		/* sanity limit */
 	tbm->maxentries = (int) nbuckets;
 	tbm->lossify_start = 0;
+	tbm->dsa = dsa;
 
 	return tbm;
 }
@@ -244,7 +301,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, tbm);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -294,7 +351,7 @@ tbm_add_tuples(TIDBitmap *tbm, const ItemPointer tids, int ntids,
 	PagetableEntry *page = NULL;	/* only valid when currblk is valid */
 	int			i;
 
-	Assert(!tbm->iterating);
+	Assert(tbm->iterating == TBM_NOT_ITERATING);
 	for (i = 0; i < ntids; i++)
 	{
 		BlockNumber blk = ItemPointerGetBlockNumber(tids + i);
@@ -624,7 +681,7 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	 * attached to the bitmap not the iterator, so they can be used by more
 	 * than one iterator.
 	 */
-	if (tbm->status == TBM_HASH && !tbm->iterating)
+	if (tbm->status == TBM_HASH && (tbm->iterating == TBM_NOT_ITERATING))
 	{
 		pagetable_iterator i;
 		PagetableEntry *page;
@@ -659,12 +716,189 @@ tbm_begin_iterate(TIDBitmap *tbm)
 				  tbm_comparator);
 	}
 
-	tbm->iterating = true;
+	tbm->iterating = TBM_ITERATING_PRIVATE;
 
 	return iterator;
 }
 
 /*
+ * tbm_prepare_shared_iterate - prepare to iterator through a TIDBitmap
+ * by multiple workers using shared iterator.
+ *
+ * The TBMSharedIteratorState will be allocated from DSA so that multiple
+ * worker can attach to it and iterate jointly.
+ *
+ * This will convert the pagetable hash into page and chunk array of the index
+ * into pagetable array so that multiple workers can use it to get the actual
+ * page entry.
+ */
+dsa_pointer
+tbm_prepare_shared_iterate(TIDBitmap *tbm)
+{
+	dsa_pointer iterator;
+	TBMSharedIteratorState *iterator_state;
+
+	/*
+	 * Allocate TBMSharedIteratorState from DSA to hold the shared members and
+	 * lock, this will also be used by multiple worker for shared iterate.
+	 */
+	iterator = dsa_allocate(tbm->dsa, sizeof(TBMSharedIteratorState));
+	iterator_state = dsa_get_address(tbm->dsa, iterator);
+
+	/*
+	 * If we have a hashtable, create and fill the sorted page lists, unless
+	 * we already did that for a previous iterator.  Note that the lists are
+	 * attached to the bitmap not the iterator, so they can be used by more
+	 * than one iterator.  However, we keep dsa_pointer to these in the shared
+	 * iterator so that other workers can access them directly.
+	 */
+	if (tbm->iterating == TBM_NOT_ITERATING)
+	{
+		pagetable_iterator i;
+		PagetableEntry *ptbase;
+		PagetableEntry *page;
+		int			idx;
+		int			npages;
+		int			nchunks;
+		int		   *ptpages;
+		int		   *ptchunks;
+
+		/*
+		 * Create page list and chunk list using relptr so that we can share
+		 * this information across multiple workers.
+		 */
+		if (tbm->npages)
+		{
+			tbm->ptpages = dsa_allocate(tbm->dsa, tbm->npages * (sizeof(int)));
+			ptpages = dsa_get_address(tbm->dsa, tbm->ptpages);
+		}
+		if (tbm->nchunks)
+		{
+			tbm->ptchunks = dsa_allocate(tbm->dsa,
+										 tbm->nchunks * (sizeof(int)));
+			ptchunks = dsa_get_address(tbm->dsa, tbm->ptchunks);
+		}
+
+		/*
+		 * If TBM status is TBM_HASH then iterate over the pagetable and
+		 * convert it to page and chunk arrays.  But if it's in TBM_ONE_PAGE
+		 * mode then directly allocate the space for one entry from the DSA.
+		 */
+		npages = nchunks = 0;
+		if (tbm->status == TBM_HASH)
+		{
+			ptbase = dsa_get_address(tbm->dsa, tbm->dsapagetable);
+
+			pagetable_start_iterate(tbm->pagetable, &i);
+			while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+			{
+				idx = page - ptbase;
+				if (page->ischunk)
+					ptchunks[nchunks++] = idx;
+				else
+					ptpages[npages++] = idx;
+			}
+
+			Assert(npages == tbm->npages);
+			Assert(nchunks == tbm->nchunks);
+		}
+		else
+		{
+			/*
+			 * In one page mode allocate the space for one pagetable entry and
+			 * directly store it's index i.e. 0 in page array
+			 */
+			tbm->dsapagetable = dsa_allocate(tbm->dsa, sizeof(PagetableEntry));
+			ptbase = dsa_get_address(tbm->dsa, tbm->dsapagetable);
+			ptpages[0] = 0;
+		}
+
+		if (npages > 1)
+			qsort_arg((void *) ptpages, npages, sizeof(int),
+					  tbm_shared_comparator, (void *) ptbase);
+		if (nchunks > 1)
+			qsort_arg((void *) ptchunks, nchunks, sizeof(int),
+					  tbm_shared_comparator, (void *) ptbase);
+	}
+
+	/*
+	 * Store the TBM member in the shared state so that we can share them
+	 * across multiple workers.
+	 */
+	iterator_state->maxentries = tbm->maxentries;
+	iterator_state->nchunks = tbm->nchunks;
+	iterator_state->nentries = tbm->nentries;
+	iterator_state->npages = tbm->npages;
+	iterator_state->pagetable = tbm->dsapagetable;
+	iterator_state->spages = tbm->ptpages;
+	iterator_state->schunks = tbm->ptchunks;
+
+	/* Initialize the shared iterator state */
+	iterator_state->schunkbit = 0;
+	iterator_state->schunkptr = 0;
+	iterator_state->spageptr = 0;
+
+	/* Initialize the iterator lock */
+	LWLockInitialize(&iterator_state->lock, LWTRANCHE_PARALLEL_TBM_ITERATOR);
+
+	tbm->iterating = TBM_ITERATING_SHARED;
+
+	return iterator;
+}
+
+/*
+ * tbm_extract_page_tuple - extract the tuple offsets from the page
+ *
+ * Process the page bits to extract the tuple offsets and store them into
+ * TBMIterateResult.
+ */
+static inline int
+tbm_extract_page_tuple(PagetableEntry *page, TBMIterateResult *output)
+{
+	int			wordnum;
+	int			ntuples = 0;
+
+	for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
+	{
+		bitmapword	w = page->words[wordnum];
+
+		if (w != 0)
+		{
+			int			off = wordnum * BITS_PER_BITMAPWORD + 1;
+
+			while (w != 0)
+			{
+				if (w & 1)
+					output->offsets[ntuples++] = (OffsetNumber) off;
+				off++;
+				w >>= 1;
+			}
+		}
+	}
+
+	return ntuples;
+}
+
+/*
+ *	tbm_advance_schunkbit
+ *
+ *	Advance the chunkbit to get the next page entry from the chunk
+ */
+static inline void
+tbm_advance_schunkbit(PagetableEntry *chunk, int *schunkbit)
+{
+	while (*schunkbit < PAGES_PER_CHUNK)
+	{
+		int			wordnum = WORDNUM(*schunkbit);
+		int			bitnum = BITNUM(*schunkbit);
+
+		if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
+			break;
+		(*schunkbit)++;
+	}
+}
+
+/*
  * tbm_iterate - scan through next page of a TIDBitmap
  *
  * Returns a TBMIterateResult representing one page, or NULL if there are
@@ -682,7 +916,7 @@ tbm_iterate(TBMIterator *iterator)
 	TIDBitmap  *tbm = iterator->tbm;
 	TBMIterateResult *output = &(iterator->output);
 
-	Assert(tbm->iterating);
+	Assert(tbm->iterating == TBM_ITERATING_PRIVATE);
 
 	/*
 	 * If lossy chunk pages remain, make sure we've advanced schunkptr/
@@ -693,15 +927,7 @@ tbm_iterate(TBMIterator *iterator)
 		PagetableEntry *chunk = tbm->schunks[iterator->schunkptr];
 		int			schunkbit = iterator->schunkbit;
 
-		while (schunkbit < PAGES_PER_CHUNK)
-		{
-			int			wordnum = WORDNUM(schunkbit);
-			int			bitnum = BITNUM(schunkbit);
-
-			if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
-				break;
-			schunkbit++;
-		}
+		tbm_advance_schunkbit(chunk, &schunkbit);
 		if (schunkbit < PAGES_PER_CHUNK)
 		{
 			iterator->schunkbit = schunkbit;
@@ -738,7 +964,6 @@ tbm_iterate(TBMIterator *iterator)
 	{
 		PagetableEntry *page;
 		int			ntuples;
-		int			wordnum;
 
 		/* In ONE_PAGE state, we don't allocate an spages[] array */
 		if (tbm->status == TBM_ONE_PAGE)
@@ -747,24 +972,7 @@ tbm_iterate(TBMIterator *iterator)
 			page = tbm->spages[iterator->spageptr];
 
 		/* scan bitmap to extract individual offset numbers */
-		ntuples = 0;
-		for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
-		{
-			bitmapword	w = page->words[wordnum];
-
-			if (w != 0)
-			{
-				int			off = wordnum * BITS_PER_BITMAPWORD + 1;
-
-				while (w != 0)
-				{
-					if (w & 1)
-						output->offsets[ntuples++] = (OffsetNumber) off;
-					off++;
-					w >>= 1;
-				}
-			}
-		}
+		ntuples = tbm_extract_page_tuple(page, output);
 		output->blockno = page->blockno;
 		output->ntuples = ntuples;
 		output->recheck = page->recheck;
@@ -777,6 +985,95 @@ tbm_iterate(TBMIterator *iterator)
 }
 
 /*
+ *------------------------
+ *	tbm_shared_iterate
+ *
+ *	As above, but this will iterate using shared iterator which is shared
+ *	across multiple workers.  We need to acquire the iterator LWLock, before
+ *	accessing the shared members.
+ *----------------------
+ */
+TBMIterateResult *
+tbm_shared_iterate(TBMSharedIterator *iterator)
+{
+	TBMIterateResult *output = &iterator->output;
+	TBMSharedIteratorState *state = iterator->state;
+	PagetableEntry *ptbase = iterator->ptbase;
+	int		   *ptpages = iterator->ptpages;
+	int		   *ptchunks = iterator->ptchunks;
+
+	/* Acquire the LWLock before accessing the shared members */
+	LWLockAcquire(&state->lock, LW_EXCLUSIVE);
+
+	/*
+	 * If lossy chunk pages remain, make sure we've advanced schunkptr/
+	 * schunkbit to the next set bit.
+	 */
+	while (state->schunkptr < state->nchunks)
+	{
+		PagetableEntry *chunk = ptbase + ptchunks[state->schunkptr];
+		int			schunkbit = state->schunkbit;
+
+		tbm_advance_schunkbit(chunk, &schunkbit);
+		if (schunkbit < PAGES_PER_CHUNK)
+		{
+			state->schunkbit = schunkbit;
+			break;
+		}
+		/* advance to next chunk */
+		state->schunkptr++;
+		state->schunkbit = 0;
+	}
+
+	/*
+	 * If both chunk and per-page data remain, must output the numerically
+	 * earlier page.
+	 */
+	if (state->schunkptr < state->nchunks)
+	{
+		PagetableEntry *chunk = ptbase + ptchunks[state->schunkptr];
+		PagetableEntry *page = ptbase + ptpages[state->spageptr];
+		BlockNumber chunk_blockno;
+
+		chunk_blockno = chunk->blockno + state->schunkbit;
+
+		if (state->spageptr >= state->npages || chunk_blockno < page->blockno)
+		{
+			/* Return a lossy page indicator from the chunk */
+			output->blockno = chunk_blockno;
+			output->ntuples = -1;
+			output->recheck = true;
+			state->schunkbit++;
+
+			LWLockRelease(&state->lock);
+			return output;
+		}
+	}
+
+	if (state->spageptr < state->npages)
+	{
+		PagetableEntry *page = ptbase + ptpages[state->spageptr];
+		int			ntuples;
+
+		/* scan bitmap to extract individual offset numbers */
+		ntuples = tbm_extract_page_tuple(page, output);
+		output->blockno = page->blockno;
+		output->ntuples = ntuples;
+		output->recheck = page->recheck;
+		state->spageptr++;
+
+		LWLockRelease(&state->lock);
+
+		return output;
+	}
+
+	LWLockRelease(&state->lock);
+
+	/* Nothing more in the bitmap */
+	return NULL;
+}
+
+/*
  * tbm_end_iterate - finish an iteration over a TIDBitmap
  *
  * Currently this is just a pfree, but it might do more someday.  (For
@@ -790,6 +1087,17 @@ tbm_end_iterate(TBMIterator *iterator)
 }
 
 /*
+ * tbm_end_shared_iterate - finish an iteration over a TIDBitmap
+ *
+ * As above, but it frees the memory of TBMSharedIterator.
+ */
+void
+tbm_end_shared_iterate(TBMSharedIterator *iterator)
+{
+	pfree(iterator);
+}
+
+/*
  * tbm_find_pageentry - find a PagetableEntry for the pageno
  *
  * Returns NULL if there is no non-lossy entry for the pageno.
@@ -995,7 +1303,7 @@ tbm_lossify(TIDBitmap *tbm)
 	 * push nentries down to significantly less than maxentries, or else we'll
 	 * just end up doing this again very soon.  We shoot for maxentries/2.
 	 */
-	Assert(!tbm->iterating);
+	Assert(tbm->iterating == TBM_NOT_ITERATING);
 	Assert(tbm->status == TBM_HASH);
 
 	pagetable_start_iterate_at(tbm->pagetable, &i, tbm->lossify_start);
@@ -1061,3 +1369,113 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * As above, but this will get index into PagetableEntry array. Therefore, it
+ * needs to get actual PagetableEntry using the index before comparing the
+ * blockno.
+ */
+static int
+tbm_shared_comparator(const void *left, const void *right, void *arg)
+{
+	PagetableEntry *lpage = ((PagetableEntry *) arg) + *((int *) left);
+	PagetableEntry *rpage = ((PagetableEntry *) arg) + *((int *) right);
+
+	if (lpage->blockno < rpage->blockno)
+		return -1;
+	else if (lpage->blockno > rpage->blockno)
+		return 1;
+	return 0;
+}
+
+/*
+ *	tbm_attach_shared_iterate
+ *
+ *	Allocate an iterator and attach shared iterator state to it so that
+ *	multiple workers can access the same memory. We also need to convert
+ *	some of the dsa_ponter from shared state to local pointers so that
+ *	shared_iterate can operate on those.
+ */
+TBMSharedIterator *
+tbm_attach_shared_iterate(dsa_area *dsa, dsa_pointer iterator)
+{
+	TBMSharedIterator *shared_iterator;
+	TBMSharedIteratorState *shared_state;
+
+	/*
+	 * Create the TBMSharedIterator struct, with enough trailing space to
+	 * serve the needs of the TBMIterateResult sub-struct.
+	 */
+	shared_iterator = (TBMSharedIterator *) palloc(sizeof(TBMIterator) +
+								 MAX_TUPLES_PER_PAGE * sizeof(OffsetNumber));
+
+	shared_state = shared_iterator->state =
+		(TBMSharedIteratorState *) dsa_get_address(dsa, iterator);
+
+	shared_iterator->ptbase = dsa_get_address(dsa, shared_state->pagetable);
+
+	/* Convert dsa pointers to the local pointers */
+	if (shared_state->npages)
+		shared_iterator->ptpages = dsa_get_address(dsa, shared_state->spages);
+	if (shared_state->nchunks)
+		shared_iterator->ptchunks = dsa_get_address(dsa,
+													shared_state->schunks);
+
+	shared_iterator->dsa = dsa;
+
+	return shared_iterator;
+}
+
+/*
+ * pagetable_allocate
+ *
+ * Callback function for allocating the memory for hashtable elements.
+ * It allocates memory from DSA if tbm holds a reference to a dsa.
+ */
+static inline void *
+pagetable_allocate(pagetable_hash *pagetable, Size size)
+{
+	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
+	dsa_pointer dsaptr;
+
+	if (tbm->dsa == NULL)
+		return MemoryContextAllocExtended(pagetable->ctx, size,
+										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+
+	dsaptr = dsa_allocate0(tbm->dsa, size);
+
+	/*
+	 * If we already have the dsapagetable then save this reference in
+	 * dsapagetableold so that pagetable_free can free the old entry.
+	 */
+	if (tbm->dsapagetable)
+		tbm->dsapagetableold = tbm->dsapagetable;
+
+	tbm->dsapagetable = dsaptr;
+
+	return dsa_get_address(tbm->dsa, dsaptr);
+}
+
+/*
+ * pagetable_free
+ *
+ * Callback function for freeing hash table elements.
+ */
+static inline void
+pagetable_free(pagetable_hash *pagetable, void *pointer)
+{
+	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
+
+	/*
+	 * If dsa pointer is NULL then we have to pfree the local pointer
+	 * otherwise we need to free the dsa pointer.
+	 */
+	if (tbm->dsa == NULL)
+		pfree(pointer);
+	else
+	{
+		/* Free the dsapagetableold and make it same as dsapagetable */
+		dsa_free(tbm->dsa, tbm->dsapagetableold);
+		tbm->dsapagetableold = tbm->dsapagetable;
+	}
+}
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..ef598d0 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -23,6 +23,7 @@
 #define TIDBITMAP_H
 
 #include "storage/itemptr.h"
+#include "utils/dsa.h"
 
 
 /*
@@ -33,6 +34,7 @@ typedef struct TIDBitmap TIDBitmap;
 
 /* Likewise, TBMIterator is private */
 typedef struct TBMIterator TBMIterator;
+typedef struct TBMSharedIterator TBMSharedIterator;
 
 /* Result structure for tbm_iterate */
 typedef struct
@@ -46,7 +48,7 @@ typedef struct
 
 /* function prototypes in nodes/tidbitmap.c */
 
-extern TIDBitmap *tbm_create(long maxbytes);
+extern TIDBitmap *tbm_create(long maxbytes, dsa_area *dsa);
 extern void tbm_free(TIDBitmap *tbm);
 
 extern void tbm_add_tuples(TIDBitmap *tbm,
@@ -62,5 +64,10 @@ extern bool tbm_is_empty(const TIDBitmap *tbm);
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_end_shared_iterate(TBMSharedIterator *iterator);
+extern dsa_pointer tbm_prepare_shared_iterate(TIDBitmap *tbm);
+extern TBMSharedIterator *tbm_attach_shared_iterate(dsa_area *dsa,
+						  dsa_pointer iterator);
+extern TBMIterateResult *tbm_shared_iterate(TBMSharedIterator *iterator);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 8bd93c3..4d392e0 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -212,6 +212,7 @@ typedef enum BuiltinTrancheIds
 	LWTRANCHE_LOCK_MANAGER,
 	LWTRANCHE_PREDICATE_LOCK_MANAGER,
 	LWTRANCHE_PARALLEL_QUERY_DSA,
+	LWTRANCHE_PARALLEL_TBM_ITERATOR,
 	LWTRANCHE_FIRST_USER_DEFINED
 }	BuiltinTrancheIds;
 
0002-parallel-bitmap-heapscan-v3.patchapplication/octet-stream; name=0002-parallel-bitmap-heapscan-v3.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index fad5cb0..e5a5e91 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1244,6 +1244,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapScan</></entry>
+         <entry>Waiting for leader backend to complete the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index af25836..bffc971 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1754,6 +1754,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 98d4f1e..8d61fcb 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
 
 #include "executor/execParallel.h"
 #include "executor/executor.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "executor/nodeCustom.h"
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
@@ -210,6 +211,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -266,6 +271,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -745,6 +755,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 0dd95c6..6970d37 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -820,6 +820,9 @@ ExecShutdownNode(PlanState *node)
 		case T_GatherState:
 			ExecShutdownGather((GatherState *) node);
 			break;
+		case T_BitmapHeapScanState:
+			ExecShutdownBitmapHeapScan((BitmapHeapScanState *) node);
+			break;
 		default:
 			break;
 	}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index f18827d..8cd54d5 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,14 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
+#include "miscadmin.h"
+#include "pgstat.h"
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapInfo *pbminfo);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -66,13 +70,16 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	HeapScanDesc scan;
 	TIDBitmap  *tbm;
 	TBMIterator *tbmiterator;
+	TBMSharedIterator *stbmiterator;
 	TBMIterateResult *tbmres;
-
+	ParallelBitmapInfo *shared_info = node->parallel_bitmap;
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
+	TBMSharedIterator *sprefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	dsa_area   *dsa = node->ss.ps.state->es_query_dsa;
 
 	/*
 	 * extract necessary information from index scan node
@@ -81,12 +88,45 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	slot = node->ss.ss_ScanTupleSlot;
 	scan = node->ss.ss_currentScanDesc;
 	tbm = node->tbm;
+
 	tbmiterator = node->tbmiterator;
+	stbmiterator = node->stbmiterator;
+
 	tbmres = node->tbmres;
 #ifdef USE_PREFETCH
 	prefetch_iterator = node->prefetch_iterator;
+	sprefetch_iterator = node->sprefetch_iterator;
 #endif
 
+	/* --------------------------------------------------------------------
+	 *	Parallel Bitmap Heap Scan Algorithm
+	 *
+	 *	The first worker to see the state of ParallelBitmapInfo as PBM_INITIAL
+	 *	becomes leader and sets the state to PBM_INPROGRESS. All other workers
+	 *	see the state as PBM_INPROGRESS, and will wait for leader to finish
+	 *	building the TIDBitmap.
+	 *
+	 *	Leader Processing:
+	 *	  Create TIDBitmap using DSA memory.
+	 *	  Restore local TIDBitmap variable information into
+	 *		ParallelBitmapInfo so that other worker can see those.
+	 *	  Convert the TIDBitmap into shared chunk and page array
+	 *	  Set state to PBM_FINISHED.
+	 *	  Wake up other workers.
+	 *
+	 *	Other Worker Processing:
+	 *	  Wait until leader creates shared TIDBitmap.
+	 *	  Copy TIDBitmap from info from ParallelBitmapInfo to local TIDBitmap.
+	 *
+	 *	Iterate and process the pages.
+	 *	  a) In this phase each worker will iterate over shared page and chunk
+	 *			array and select heap pages one by one.
+	 *	  b) Since multiple workers are iterating over same page and chunk
+	 *			array we need to have a shared iterator, so we grab a LWLock
+	 *			and iterate within a lock.
+	 * ----------------------------------------------------------------
+	 */
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -101,21 +141,90 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 */
 	if (tbm == NULL)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		/*
+		 * Process the lower level node only if either we are running in non
+		 * parallel mode or we are leader.  In parallel mode leader will
+		 * immediately come out of the function, but all other workers will be
+		 * blocked until leader wake them up.
+		 */
+		if (shared_info == NULL || pbms_is_leader(shared_info))
+		{
+			/*
+			 * If we are in parallel mode recursively process the outer node
+			 * and set parallel flag. This flag will be used to indicate the
+			 * underlying tidbitmap layer to allocate pagetable elements from
+			 * DSA.
+			 */
+			if (shared_info)
+				pbms_set_parallel(outerPlanState(node));
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			/*
+			 * In parallel mode, prepare the iterate over the TIDBitmap. This
+			 * will return the dsa_pointer to the shared iterator state. Which
+			 * will be used by multiple workers to iterate jointly.
+			 */
+			if (shared_info)
+			{
+				shared_info->tbmiterator = tbm_prepare_shared_iterate(tbm);
+#ifdef USE_PREFETCH
+				if (node->prefetch_maximum > 0)
+				{
+					shared_info->prefetch_iterator =
+						tbm_prepare_shared_iterate(tbm);
+				}
+
+				/*
+				 * By this time we have already created the shared iterator so
+				 * it's time to wake up other workers.
+				 */
+				SpinLockAcquire(&shared_info->state_mutex);
+				shared_info->state = PBM_FINISHED;
+				SpinLockRelease(&shared_info->state_mutex);
+
+				/* Wake up all other workers. */
+				ConditionVariableBroadcast(&shared_info->cv);
+#endif
+			}
+		}
+		else
+			tbm = tbm_create(work_mem * 1024L, dsa);
 
 		node->tbm = tbm;
-		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+
+		if (shared_info)
+		{
+			/*
+			 * Allocate worker specific iterator and attach the shared
+			 * iterator state to it.
+			 */
+			node->stbmiterator = stbmiterator = tbm_attach_shared_iterate(
+											  dsa, shared_info->tbmiterator);
+		}
+		else
+			node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+
 		node->tbmres = tbmres = NULL;
 
 #ifdef USE_PREFETCH
 		if (node->prefetch_maximum > 0)
 		{
-			node->prefetch_iterator = prefetch_iterator = tbm_begin_iterate(tbm);
-			node->prefetch_pages = 0;
-			node->prefetch_target = -1;
+			if (shared_info)
+			{
+				node->sprefetch_iterator = tbm_attach_shared_iterate(dsa,
+											 shared_info->prefetch_iterator);
+				sprefetch_iterator = node->sprefetch_iterator;
+			}
+			else
+			{
+				node->prefetch_iterator = prefetch_iterator =
+					tbm_begin_iterate(tbm);
+				node->prefetch_pages = 0;
+				node->prefetch_target = -1;
+			}
 		}
 #endif   /* USE_PREFETCH */
 	}
@@ -124,13 +233,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	{
 		Page		dp;
 		ItemId		lp;
+		bool		need_prefetch = false;
+		int		   *prefetch_target;
 
 		/*
 		 * Get next page of results if needed
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			/*
+			 * If we are in parallel mode perform shared iterate otherwise
+			 * local iterate.
+			 */
+			if (!shared_info)
+				node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			else
+				node->tbmres = tbmres = tbm_shared_iterate(stbmiterator);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,17 +256,51 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+
+			/*
+			 * If we are in parallel mode then acquire prefetch_mutex and
+			 * check prefetch_pages from shared location.
+			 */
+			if (shared_info)
 			{
+				SpinLockAcquire(&shared_info->prefetch_mutex);
+
 				/* The main iterator has closed the distance by one page */
-				node->prefetch_pages--;
+				if (shared_info->prefetch_pages > 0)
+					shared_info->prefetch_pages--;
+				else
+					need_prefetch = true;
+				SpinLockRelease(&shared_info->prefetch_mutex);
 			}
-			else if (prefetch_iterator)
+			else
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				/* The main iterator has closed the distance by one page */
+				if (node->prefetch_pages > 0)
+					node->prefetch_pages--;
+				else
+					need_prefetch = true;
+			}
 
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+			if (prefetch_iterator && need_prefetch)
+			{
+				TBMIterateResult *tbmpre;
+
+				/* Do not let the prefetch iterator get behind the main one */
+				if (!shared_info)
+					tbmpre = tbm_iterate(prefetch_iterator);
+				else
+					tbmpre = tbm_shared_iterate(sprefetch_iterator);
+
+				/*
+				 * In case of parallel mode we can only ensure that prefetch
+				 * iterator is not behind the main iterator, but we can not
+				 * ensure that the current blockno in the main iterator and in
+				 * the prefetch iterator is same. It's possible that whatever
+				 * blockno we are prefetching will be processed by another
+				 * worker.
+				 */
+				if ((shared_info == NULL) &&
+					(tbmpre == NULL || tbmpre->blockno != tbmres->blockno))
 					elog(ERROR, "prefetch and main iterators are out of sync");
 			}
 #endif   /* USE_PREFETCH */
@@ -183,19 +335,40 @@ BitmapHeapNext(BitmapHeapScanState *node)
 #ifdef USE_PREFETCH
 
 			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
+			 * If we are in shared mode then use prefetch_target from shared
+			 * location i.e pbminfo otherwise directly from node.
 			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
+			if (shared_info == NULL)
+				prefetch_target = &node->prefetch_target;
 			else
-				node->prefetch_target++;
+				prefetch_target = &shared_info->prefetch_target;
+
+			/* Increase prefetch target if it's not yet at the max. */
+			if (*prefetch_target < node->prefetch_maximum)
+			{
+				/* If we are in parallel mode then grab prefetch_mutex */
+				if (shared_info != NULL)
+					SpinLockAcquire(&shared_info->prefetch_mutex);
+
+				/*
+				 * Increase prefetch target if it's not yet at the max.  Note
+				 * that we will increase it to zero after fetching the very
+				 * first page/tuple, then to one after the second tuple is
+				 * fetched, then it doubles as later pages are fetched.
+				 */
+				if (*prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (*prefetch_target >= node->prefetch_maximum / 2)
+					*prefetch_target = node->prefetch_maximum;
+				else if (*prefetch_target > 0)
+					*prefetch_target *= 2;
+				else
+					(*prefetch_target)++;
+
+				if (shared_info != NULL)
+					SpinLockRelease(&shared_info->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +384,25 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (shared_info == NULL)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				if (shared_info->prefetch_target < node->prefetch_maximum)
+				{
+					SpinLockAcquire(&shared_info->prefetch_mutex);
+					if (shared_info->prefetch_target < node->prefetch_maximum)
+						shared_info->prefetch_target++;
+					SpinLockRelease(&shared_info->prefetch_mutex);
+				}
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +426,67 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			int		   *prefetch_pages;
+			int			prefetch_target;
+
+			/*
+			 * If parallel bitmap info available means we are running in
+			 * parallel mode. So use parallel iterator for prefetching.
+			 */
+			if (shared_info)
+			{
+				prefetch_pages = &shared_info->prefetch_pages;
+				prefetch_target = shared_info->prefetch_target;
+			}
+			else
+			{
+				prefetch_pages = &node->prefetch_pages;
+				prefetch_target = node->prefetch_target;
+			}
+
+			/*
+			 * We are checking the prefetch_pages without mutex.  Henceforth,
+			 * in parallel mode there can be some extra prefetch.  Should we
+			 * acquire mutex and recheck before iterating?
+			 */
+			while (*prefetch_pages < prefetch_target)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				TBMIterateResult *tbmpre;
+
+				if (!shared_info)
+					tbmpre = tbm_iterate(prefetch_iterator);
+				else
+					tbmpre = tbm_shared_iterate(sprefetch_iterator);
 
 				if (tbmpre == NULL)
 				{
-					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
+					if (!shared_info)
+					{
+						/* No more pages to prefetch */
+						tbm_end_iterate(prefetch_iterator);
+						node->prefetch_iterator = prefetch_iterator = NULL;
+					}
+					else
+					{
+						/* No more pages to prefetch */
+						tbm_end_shared_iterate(sprefetch_iterator);
+						node->sprefetch_iterator = sprefetch_iterator = NULL;
+					}
 					break;
 				}
-				node->prefetch_pages++;
+
+				if (shared_info != NULL)
+					SpinLockAcquire(&shared_info->prefetch_mutex);
+
+				(*prefetch_pages)++;
+
+				if (shared_info != NULL)
+					SpinLockRelease(&shared_info->prefetch_mutex);
+
 				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -458,6 +694,10 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->tbmiterator);
 	if (node->prefetch_iterator)
 		tbm_end_iterate(node->prefetch_iterator);
+	if (node->stbmiterator)
+		tbm_end_shared_iterate(node->stbmiterator);
+	if (node->sprefetch_iterator)
+		tbm_end_shared_iterate(node->sprefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
 	node->tbm = NULL;
@@ -465,6 +705,16 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
 
+	/* reset parallel bitmap scan info, if present */
+	if (node->parallel_bitmap)
+	{
+		ParallelBitmapInfo *pbminfo = node->parallel_bitmap;
+
+		pbminfo->state = PBM_INITIAL;
+		pbminfo->prefetch_pages = 0;
+		pbminfo->prefetch_target = -1;
+	}
+
 	ExecScanReScan(&node->ss);
 
 	/*
@@ -514,8 +764,12 @@ ExecEndBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->tbmiterator);
 	if (node->prefetch_iterator)
 		tbm_end_iterate(node->prefetch_iterator);
-	if (node->tbm)
-		tbm_free(node->tbm);
+	if (node->stbmiterator)
+		tbm_end_shared_iterate(node->stbmiterator);
+	if (node->sprefetch_iterator)
+		tbm_end_shared_iterate(node->sprefetch_iterator);
+
+	ExecShutdownBitmapHeapScan(node);
 
 	/*
 	 * close heap scan
@@ -567,6 +821,7 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->parallel_bitmap = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -651,3 +906,173 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/* ----------------------------------------------------------------
+ *		ExecShutdownBitmapHeapScan
+ * ----------------------------------------------------------------
+ */
+void
+ExecShutdownBitmapHeapScan(BitmapHeapScanState *node)
+{
+	if (node->tbm)
+		tbm_free(node->tbm);
+
+	node->tbm = NULL;
+}
+
+/*----------------
+ *		pbms_is_leader
+ *
+ *		If PBMState is PBM_INITIAL then we become the leader and set the state
+ *		to PBM_INPROGRESS.  Otherwise we become the worker therefore we need to
+ *		wait for the leader to wake us up and also PBMState should be
+ *		PBM_FINISHED.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapInfo *pbminfo)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : then we become leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : then we need to wait till leader creates BITMAP
+		 * PBM_FINISHED   : BITMAP is ready so no need to wait.
+		 *---------------
+		 */
+		SpinLockAcquire(&pbminfo->state_mutex);
+
+		if (pbminfo->state == PBM_INITIAL)
+		{
+			pbminfo->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pbminfo->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pbminfo->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pbminfo->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/*-------------------
+ *		pbms_set_parallel
+ *
+ *		Recursively process the node and set the parallel flag. This flag
+ *		will be used to indicate the underlying tidbitmap layer to allocate
+ *		pagetable elements from DSA.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * In case of BitmapAnd, set first bitmap index scan node as parallel
+	 * because only first node will create the main bitmap other node's bitmap
+	 * will be merged to the first bitmap thus no need to create them in
+	 * shared memory.  BitmapOr and BitmapIndex will create the TidBitmap so
+	 * set the parallel flag true.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			((BitmapOrState *) node)->is_parallel = true;;
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+
+	/* Estimate the size for sharing parallel Bitmap info. */
+	node->pscan_len = add_size(offsetof(ParallelBitmapInfo,
+										phs_snapshot_data),
+							   EstimateSnapshotSpace(estate->es_snapshot));
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapInfo *pbminfo;
+	EState	   *estate = node->ss.ps.state;
+
+	pbminfo = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pbminfo->prefetch_mutex);
+	pbminfo->prefetch_pages = 0;
+	pbminfo->prefetch_target = -1;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pbminfo->state_mutex);
+	pbminfo->state = PBM_INITIAL;
+
+	ConditionVariableInit(&pbminfo->cv);
+	SerializeSnapshot(estate->es_snapshot, pbminfo->phs_snapshot_data);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pbminfo);
+
+	node->parallel_bitmap = pbminfo;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapInfo *pbminfo;
+	Snapshot	snapshot;
+
+	pbminfo = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->parallel_bitmap = pbminfo;
+
+	snapshot = RestoreSnapshot(pbminfo->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 94bb289..2f01cad 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,8 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L, NULL);
+		tbm = tbm_create(work_mem * 1024L,
+				node->biss_Parallel ? node->ss.ps.state->es_query_dsa: NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 1d280be..e8f110b 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,8 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L, NULL);
+				result = tbm_create(work_mem * 1024L, node->is_parallel ?
+									node->ps.state->es_query_dsa: NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index eeacf81..b32af89 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2875,6 +2875,31 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 }
 
 /*
+ * create_partial_bitmap_paths
+ *	  Build partial access paths for parallel scan of a plain relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched, 0);
+
+	/* If any limit was set to zero, the user doesn't want a parallel scan. */
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+/*
  * Compute the number of parallel workers that should be used to scan a
  * relation.  We compute the parallel workers based on the size of the heap to
  * be scanned and the size of the index to be scanned, then choose a minimum
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index d01630f..b92c432 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -861,6 +861,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -922,8 +923,32 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
+
+		/*
+		 * It may be possible to amortize some of the I/O cost, but probably
+		 * not very much, because most operating systems already do aggressive
+		 * prefetching.  For now, we assume that the disk run cost can't be
+		 * amortized at all.
+		 */
+
+		/*
+		 * In the case of a parallel plan, the row count needs to represent
+		 * the number of tuples processed per worker.
+		 */
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 56eccaf..097318f 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1618,6 +1622,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1660,6 +1669,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 997bdcf..6d152f7 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2834,7 +2834,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 		plan->plan_rows =
 			clamp_row_est(ipath->indexselectivity * ipath->path.parent->tuples);
 		plan->plan_width = 0;	/* meaningless */
-		plan->parallel_aware = false;
+		plan->parallel_aware = bitmapqual->parallel_aware;
 		*qual = get_actual_clauses(ipath->indexclauses);
 		*indexqual = get_actual_clauses(ipath->indexquals);
 		foreach(l, ipath->indexinfo->indpred)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3248296..6e70808 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1068,7 +1068,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1077,9 +1078,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3255,7 +3256,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index ada374c..9b191df 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3395,6 +3395,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index a864f78..7e85510 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -179,6 +179,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..34ec484 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,18 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecShutdownBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 9f41bab..c013464 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1261,6 +1263,7 @@ typedef struct BitmapOrState
 	PlanState	ps;				/* its first field is NodeTag */
 	PlanState **bitmapplans;	/* array of PlanStates for my inputs */
 	int			nplans;			/* number of input plans */
+	bool		is_parallel;	/* create shared tbm if it's set */
 } BitmapOrState;
 
 /* ----------------------------------------------------------------
@@ -1429,6 +1432,59 @@ typedef struct IndexOnlyScanState
 	long		ioss_HeapFetches;
 } IndexOnlyScanState;
 
+
+/* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will become
+ *							leader and will create TIDbitmap. This will
+ *							also set the state to PBM_INPROGRESS.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							so workers need to sleep until leader set the
+ *							state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker
+ *							can proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+}	PBMState;
+
+/* ----------------
+ *	 ParallelBitmapInfo information
+ *		relid					OID of relation to scan
+ *		prefetch_mutex			mutual exclusion for prefetch members
+ *								(prefetch_iterator, prefetch_pages and
+ *								 prefetch_target)
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state_mutex				mutual exclusion for state field
+ *		cv						conditional wait variable
+ *		state					current state of the TIDBitmap
+ *		tbmiterator				iterator for scanning current pages
+ *		prefetch_iterator		iterator for prefetching ahead of current page
+ *		phs_snapshot_data		snapshot data shared to worker
+ * ----------------
+ */
+typedef struct ParallelBitmapInfo
+{
+	Oid			relid;
+	slock_t		prefetch_mutex;
+	slock_t		state_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	ConditionVariable cv;
+	PBMState	state;
+	dsa_pointer	tbmiterator;
+	dsa_pointer	prefetch_iterator;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+}	ParallelBitmapInfo;
+
 /* ----------------
  *	 BitmapIndexScanState information
  *
@@ -1443,6 +1499,8 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Under parallel Bitmap heap so need to create shared
+ *						   TIDBitmap
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1459,6 +1517,7 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
@@ -1474,7 +1533,9 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		parallel_bitmap    shared memory for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1489,6 +1550,10 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	TBMSharedIterator *stbmiterator;
+	TBMSharedIterator *sprefetch_iterator;
+	ParallelBitmapInfo *parallel_bitmap;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 53cad24..d22b039 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -53,7 +53,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index ebda308..3ec2d9d 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -56,6 +56,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
 extern int compute_parallel_worker(RelOptInfo *rel, BlockNumber heap_pages,
 						BlockNumber index_pages);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+										Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 8b710ec..e31389f 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -787,6 +787,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#85Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#84)
Re: Parallel bitmap heap scan

On Thu, Feb 16, 2017 at 10:54 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

interface_dsa_allocate0.patch : Provides new interface dsa_allocate0,
which is used by 0001

Committed. I moved the function prototype for dsa_allocate0() next to
the existing prototype for dsa_allocate(). Please try to think about
things like this when you add functions or prototypes or structure
members: put them in a natural place where they are close to related
things, not just wherever happens to be convenient.

gather_shutdown_childeren_first.patch : Processing the child node

This same problem came up on the Parallel Hash thread. I mentioned
this proposed fix over there; let's see what he (or anyone else) has
to say about it.

0001: tidbimap change to provide the interfaces for shared iterator.

+ * in order to share relptrs of the chunk and the pages arrays and other
+ * TBM related information with other workers.

No relptrs any more.

+    dsa_pointer dsapagetable;    /* dsa_pointer to the element array */
+    dsa_pointer dsapagetableold;    /* dsa_pointer to the old element array */
+    dsa_area   *dsa;            /* reference to per-query dsa area */
+    dsa_pointer ptpages;        /* dsa_pointer to the page array */
+    dsa_pointer ptchunks;        /* dsa_pointer to the chunk array */

Let's put the DSA pointer first and then the other stuff after it.
That seems more logical.

+typedef struct TBMSharedIteratorState
+{
+    int            spageptr;        /* next spages index */
+    int            schunkptr;        /* next schunks index */
+    int            schunkbit;        /* next bit to check in current schunk */
+    LWLock        lock;            /* lock to protect above members */
+    dsa_pointer pagetable;        /* dsa pointers to head of pagetable data */
+    dsa_pointer spages;            /* dsa pointer to page array */
+    dsa_pointer schunks;        /* dsa pointer to chunk array */
+    int            nentries;        /* number of entries in pagetable */
+    int            maxentries;        /* limit on same to meet maxbytes */
+    int            npages;            /* number of exact entries in
pagetable */
+    int            nchunks;        /* number of lossy entries in pagetable */
+} TBMSharedIteratorState;

I think you've got this largely backwards from the most logical order.
Let's order it like this: nentries, maxentries, npages, nchunks,
pagetable, spages, schunks, lock (to protect BELOW members), spageptr,
chunkptr, schunkbit.

+struct TBMSharedIterator
+{
+    PagetableEntry *ptbase;        /* pointer to the pagetable element array */
+    dsa_area   *dsa;            /* reference to per-query dsa area */
+    int           *ptpages;        /* sorted exact page index list */
+    int           *ptchunks;        /* sorted lossy page index list */
+    TBMSharedIteratorState *state;        /* shared members */
+    TBMIterateResult output;    /* MUST BE LAST (because variable-size) */
+};

Do we really need "dsa" here when it's already present in the shared
state? It doesn't seem like we even use it for anything. It's needed
to create each backend's TBMSharedIterator, but not afterwards, I
think.

In terms of ordering, I'd move "state" to the top of the structure,
just like "tbm" comes first in a TBMIterator.

+ * total memory consumption.  If dsa passed to this function is not NULL
+ * then the memory for storing elements of the underlying page table will
+ * be allocated from the DSA.

Notice that you capitalized "DSA" in two different ways in the same
sentence; I'd go for the all-caps version. Also, you need the word
"the" before the first one.

+ if (tbm->status == TBM_HASH && (tbm->iterating == TBM_NOT_ITERATING))

Excess parentheses.

+ * tbm_prepare_shared_iterate - prepare to iterator through a TIDBitmap
+ * by multiple workers using shared iterator.

Prepare to iterate, not prepare to iterator. I suggest rephrasing
this as "prepare shared iteration state for a TIDBitmap".

+ * The TBMSharedIteratorState will be allocated from DSA so that multiple
+ * worker can attach to it and iterate jointly.

Maybe change to "The necessary shared state will be allocated from the
DSA passed to tbm_create, so that multiple workers can attach to it
and iterate jointly".

+ * This will convert the pagetable hash into page and chunk array of the index
+ * into pagetable array so that multiple workers can use it to get the actual
+ * page entry.

I think you can leave off everything starting from "so that". It's
basically redundant with what you already said.

+    dsa_pointer iterator;
+    TBMSharedIteratorState *iterator_state;

These aren't really great variable names, because the latter isn't the
state associated with the former. They're both the same object.
Maybe just "dp" and "istate".

I think this function should Assert(tbm->dsa != NULL) and
Assert(tbm->iterating != TBM_ITERATING_PRIVATE), and similarly
tbm_begin_iterate should Assert(tbm->iterating != TBM_ITERATE_SHARED).

+    /*
+     * If we have a hashtable, create and fill the sorted page lists, unless
+     * we already did that for a previous iterator.  Note that the lists are
+     * attached to the bitmap not the iterator, so they can be used by more
+     * than one iterator.  However, we keep dsa_pointer to these in the shared
+     * iterator so that other workers can access them directly.
+     */

This is mostly cut-and-pasted from tbm_begin_iterate() but it's not
really correct here now because (1) we're no longer trying to fake up
a TIDBitmap proper in every backend and (2) this code runs even if we
don't have a hashtable. I think the comment should be something like
"If we're not already iterating, create and fill the sorted page
lists. (If we are, the sorted page lists are already stored in the
TIDBitmap, and we can just reuse them.)"

+         * Create page list and chunk list using relptr so that we can share
+         * this information across multiple workers.

No relptrs any more.

+ tbm->ptpages = dsa_allocate(tbm->dsa, tbm->npages * (sizeof(int)));

Extra parentheses.

+ tbm->nchunks * (sizeof(int)));

Extra parentheses.

+ * If TBM status is TBM_HASH then iterate over the pagetable and

"If the TBM status is"...

+ * directly store it's index i.e. 0 in page array

s/it's/its/

Don't you also need some code here to handle the TBM_EMPTY case?

+    /*
+     * Store the TBM member in the shared state so that we can share them
+     * across multiple workers.
+     */
+    iterator_state->maxentries = tbm->maxentries;
+    iterator_state->nchunks = tbm->nchunks;
+    iterator_state->nentries = tbm->nentries;
+    iterator_state->npages = tbm->npages;
+    iterator_state->pagetable = tbm->dsapagetable;
+    iterator_state->spages = tbm->ptpages;
+    iterator_state->schunks = tbm->ptchunks;
+
+    /* Initialize the shared iterator state */
+    iterator_state->schunkbit = 0;
+    iterator_state->schunkptr = 0;
+    iterator_state->spageptr = 0;
+
+    /* Initialize the iterator lock */
+    LWLockInitialize(&iterator_state->lock, LWTRANCHE_PARALLEL_TBM_ITERATOR);

Set all of the structure members in the same order that you declare them.

+ * tbm_extract_page_tuple - extract the tuple offsets from the page

s/the page/a page/

+ * Process the page bits to extract the tuple offsets and store them into
+ * TBMIterateResult.

This duplicates the preceding, a bit. Maybe just "The extracted
offsets are stored into the TBMIterateResult".

+/*
+ *    tbm_advance_schunkbit
+ *
+ *    Advance the chunkbit to get the next page entry from the chunk
+ */

The formatting of this header comment is randomly different from the
preceding and following header comments.

I would change the argument to schunkbitp, declare a local variable
named schunkbit, and do int schunkbit = *schunkbitp at the top and
*schunkbitp = schunkbit at the bottom.

+ *    As above, but this will iterate using shared iterator which is shared
+ *    across multiple workers.  We need to acquire the iterator LWLock, before
+ *    accessing the shared members.

"using shared iterator which is shared" -> "using an iterator which is shared"
"multiple workers" -> "multiple processes"

+ PagetableEntry *chunk = ptbase + ptchunks[state->schunkptr];

Maybe &ptbase[ptchunks[state->schunkptr]] would look a little nicer.

+        PagetableEntry *chunk = ptbase + ptchunks[state->schunkptr];
+        PagetableEntry *page = ptbase + ptpages[state->spageptr];
+        PagetableEntry *page = ptbase + ptpages[state->spageptr];

Similarly.

+ * tbm_end_shared_iterate - finish an iteration over a TIDBitmap

Maybe s/an iteration/a shared iteration/

+ * As above, but it frees the memory of TBMSharedIterator.

Instead of this, I'd write "This doesn't free any of the shared state
associated with the iterator, just our backend-private state."

+    PagetableEntry *lpage = ((PagetableEntry *) arg) + *((int *) left);
+    PagetableEntry *rpage = ((PagetableEntry *) arg) + *((int *) right);

Again, try to use array dereference notation rather than pointer
arithmetic where you can. Maybe:

PageTableEntry *base = (PageTableEntry *) arg;
PageTableEntry *lpage = &arg[*(int *) left];
etc.

+tbm_attach_shared_iterate(dsa_area *dsa, dsa_pointer iterator)
+{
+    TBMSharedIterator *shared_iterator;
+    TBMSharedIteratorState *shared_state;

Again, variable naming issues. Let's go with dsa_pointer dp,
TBMSharedIterator *iterator, TBMSharedIteratorState *istate. Please
try to go through and always use the same variable name for the same
kind of object. It's confusing when "iterator" sometimes means a
TBMSharedIterator * and sometimes a TBMSharedIteratorState * and
sometimes a dsa_pointer. Make it all consistent (and hopefully
logical).

+    shared_iterator = (TBMSharedIterator *) palloc(sizeof(TBMIterator) +
+                                 MAX_TUPLES_PER_PAGE * sizeof(OffsetNumber));

Wrong sizeof. (This isn't just a style issue - it's a bug!)

+ * Callback function for allocating the memory for hashtable elements.
+ * It allocates memory from DSA if tbm holds a reference to a dsa.

Inconsistent capitalization of "DSA" again.

Maybe change the whole comment to "Allocate memory for hashtable
elements, using DSA if available."

+ if (tbm->dsapagetable)

Test against InvalidDsaPointer. Or maybe just get rid of the "if" and
do it unconditionally; if dsapagetable is InvalidDsaPointer then
dsapagetableold will be too, so no harm done.

+ LWTRANCHE_PARALLEL_TBM_ITERATOR,

Change to LWLTRANCHE_TBM, add a call to LWLockRegisterTranche in
lwlock.c, and patch the docs with the new tranche name.

Despite this fairly extensive list of mostly-trivial cosmetic
problems, I think this is in fairly good shape now. I think the basic
design is right, so once we get these loose ends tied up this should
be good to go in.

0002: actual parallel bitmap heap scan by using interfaces of 0001.

This needs a lot more extensive review than I quite have the time and
energy for right now, but here are a few comments.

+ <entry><literal>ParallelBitmapScan</></entry>

ParallelBitmapPopulate, maybe? You're not waiting for the scan;
you're waiting for the bitmap to be populated.

+ <entry>Waiting for leader backend to complete the TidBitmap.</entry>

Likewise, complete -> populate?

+ /* If any limit was set to zero, the user doesn't want a parallel scan. */

If we got zero? There aren't multiple limits mentioned anywhere
nearby, so this reads oddly.

+        /*
+         * It may be possible to amortize some of the I/O cost, but probably
+         * not very much, because most operating systems already do aggressive
+         * prefetching.  For now, we assume that the disk run cost can't be
+         * amortized at all.
+         */

This looks like it was cut-and-pasted from the sequential scan case,
but it doesn't really apply here. This presupposes we're relying on
OS prefetching, but in the case of a bitmap heap scan we do our own
prefetching. Maybe change to "Since we do prefetching in both the
parallel and non-parallel cases, we don't expect to get substantially
more I/O parallelism by spreading the scan across multiple backends.
So, assume the disk run cost can't be amortized at all."

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#86Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#85)
3 attachment(s)
Re: Parallel bitmap heap scan

On Fri, Feb 17, 2017 at 2:01 AM, Robert Haas <robertmhaas@gmail.com> wrote:

+ * in order to share relptrs of the chunk and the pages arrays and other
+ * TBM related information with other workers.

No relptrs any more.

Fixed

+    dsa_pointer dsapagetable;    /* dsa_pointer to the element array */
+    dsa_pointer dsapagetableold;    /* dsa_pointer to the old element array */
+    dsa_area   *dsa;            /* reference to per-query dsa area */
+    dsa_pointer ptpages;        /* dsa_pointer to the page array */
+    dsa_pointer ptchunks;        /* dsa_pointer to the chunk array */

Let's put the DSA pointer first and then the other stuff after it.
That seems more logical.

Done that way

+typedef struct TBMSharedIteratorState
+{
+    int            spageptr;        /* next spages index */
+    int            schunkptr;        /* next schunks index */
+    int            schunkbit;        /* next bit to check in current schunk */
+    LWLock        lock;            /* lock to protect above members */
+    dsa_pointer pagetable;        /* dsa pointers to head of pagetable data */
+    dsa_pointer spages;            /* dsa pointer to page array */
+    dsa_pointer schunks;        /* dsa pointer to chunk array */
+    int            nentries;        /* number of entries in pagetable */
+    int            maxentries;        /* limit on same to meet maxbytes */
+    int            npages;            /* number of exact entries in
pagetable */
+    int            nchunks;        /* number of lossy entries in pagetable */
+} TBMSharedIteratorState;

I think you've got this largely backwards from the most logical order.
Let's order it like this: nentries, maxentries, npages, nchunks,
pagetable, spages, schunks, lock (to protect BELOW members), spageptr,
chunkptr, schunkbit.

Done

+struct TBMSharedIterator
+{
+    PagetableEntry *ptbase;        /* pointer to the pagetable element array */
+    dsa_area   *dsa;            /* reference to per-query dsa area */
+    int           *ptpages;        /* sorted exact page index list */
+    int           *ptchunks;        /* sorted lossy page index list */
+    TBMSharedIteratorState *state;        /* shared members */
+    TBMIterateResult output;    /* MUST BE LAST (because variable-size) */
+};

Do we really need "dsa" here when it's already present in the shared
state? It doesn't seem like we even use it for anything. It's needed
to create each backend's TBMSharedIterator, but not afterwards, I
think.

Right, removed.

In terms of ordering, I'd move "state" to the top of the structure,
just like "tbm" comes first in a TBMIterator.

Yeah, that looks better. Done that way.

+ * total memory consumption.  If dsa passed to this function is not NULL
+ * then the memory for storing elements of the underlying page table will
+ * be allocated from the DSA.

Notice that you capitalized "DSA" in two different ways in the same
sentence; I'd go for the all-caps version. Also, you need the word
"the" before the first one.

Fixed, all such instances.

+ if (tbm->status == TBM_HASH && (tbm->iterating == TBM_NOT_ITERATING))

Excess parentheses.

Fixed

+ * tbm_prepare_shared_iterate - prepare to iterator through a TIDBitmap
+ * by multiple workers using shared iterator.

Prepare to iterate, not prepare to iterator. I suggest rephrasing
this as "prepare shared iteration state for a TIDBitmap".

Fixed.

+ * The TBMSharedIteratorState will be allocated from DSA so that multiple
+ * worker can attach to it and iterate jointly.

Maybe change to "The necessary shared state will be allocated from the
DSA passed to tbm_create, so that multiple workers can attach to it
and iterate jointly".

Done.

+ * This will convert the pagetable hash into page and chunk array of the index
+ * into pagetable array so that multiple workers can use it to get the actual
+ * page entry.

I think you can leave off everything starting from "so that". It's
basically redundant with what you already said.

Done

+    dsa_pointer iterator;
+    TBMSharedIteratorState *iterator_state;

These aren't really great variable names, because the latter isn't the
state associated with the former. They're both the same object.
Maybe just "dp" and "istate".

Done

I think this function should Assert(tbm->dsa != NULL) and
Assert(tbm->iterating != TBM_ITERATING_PRIVATE), and similarly
tbm_begin_iterate should Assert(tbm->iterating != TBM_ITERATE_SHARED).

Done

+    /*
+     * If we have a hashtable, create and fill the sorted page lists, unless
+     * we already did that for a previous iterator.  Note that the lists are
+     * attached to the bitmap not the iterator, so they can be used by more
+     * than one iterator.  However, we keep dsa_pointer to these in the shared
+     * iterator so that other workers can access them directly.
+     */

This is mostly cut-and-pasted from tbm_begin_iterate() but it's not
really correct here now because (1) we're no longer trying to fake up
a TIDBitmap proper in every backend and (2) this code runs even if we
don't have a hashtable. I think the comment should be something like
"If we're not already iterating, create and fill the sorted page
lists. (If we are, the sorted page lists are already stored in the
TIDBitmap, and we can just reuse them.)"

Done

+         * Create page list and chunk list using relptr so that we can share
+         * this information across multiple workers.

No relptrs any more.

Done

+ tbm->ptpages = dsa_allocate(tbm->dsa, tbm->npages * (sizeof(int)));

Extra parentheses.

+ tbm->nchunks * (sizeof(int)));

Extra parentheses.

Fixed

+ * If TBM status is TBM_HASH then iterate over the pagetable and

"If the TBM status is"...

+ * directly store it's index i.e. 0 in page array

s/it's/its/

Done

Don't you also need some code here to handle the TBM_EMPTY case?,

IMHO, we don't need to do anything for TBM_EMPTY because npages and
nchunks will be zero so iterator will handle such cases (same as it's
done for non-parallel case.)

+    /*
+     * Store the TBM member in the shared state so that we can share them
+     * across multiple workers.
+     */
+    iterator_state->maxentries = tbm->maxentries;
+    iterator_state->nchunks = tbm->nchunks;
+    iterator_state->nentries = tbm->nentries;
+    iterator_state->npages = tbm->npages;
+    iterator_state->pagetable = tbm->dsapagetable;
+    iterator_state->spages = tbm->ptpages;
+    iterator_state->schunks = tbm->ptchunks;
+
+    /* Initialize the shared iterator state */
+    iterator_state->schunkbit = 0;
+    iterator_state->schunkptr = 0;
+    iterator_state->spageptr = 0;
+
+    /* Initialize the iterator lock */
+    LWLockInitialize(&iterator_state->lock, LWTRANCHE_PARALLEL_TBM_ITERATOR);

Set all of the structure members in the same order that you declare them.

Done.

+ * tbm_extract_page_tuple - extract the tuple offsets from the page

s/the page/a page/

+ * Process the page bits to extract the tuple offsets and store them into
+ * TBMIterateResult.

This duplicates the preceding, a bit. Maybe just "The extracted
offsets are stored into the TBMIterateResult".

Done

+/*
+ *    tbm_advance_schunkbit
+ *
+ *    Advance the chunkbit to get the next page entry from the chunk
+ */

The formatting of this header comment is randomly different from the
preceding and following header comments.

Seems like different function in the file are using different
formatting. But done as near by functions.

I would change the argument to schunkbitp, declare a local variable
named schunkbit, and do int schunkbit = *schunkbitp at the top and
*schunkbitp = schunkbit at the bottom.

Fixed

+ *    As above, but this will iterate using shared iterator which is shared
+ *    across multiple workers.  We need to acquire the iterator LWLock, before
+ *    accessing the shared members.

"using shared iterator which is shared" -> "using an iterator which is shared"
"multiple workers" -> "multiple processes"

Done

+ PagetableEntry *chunk = ptbase + ptchunks[state->schunkptr];

Maybe &ptbase[ptchunks[state->schunkptr]] would look a little nicer.

+        PagetableEntry *chunk = ptbase + ptchunks[state->schunkptr];
+        PagetableEntry *page = ptbase + ptpages[state->spageptr];
+        PagetableEntry *page = ptbase + ptpages[state->spageptr];

Similarly.

Done

+ * tbm_end_shared_iterate - finish an iteration over a TIDBitmap

Maybe s/an iteration/a shared iteration/

+ * As above, but it frees the memory of TBMSharedIterator.

Instead of this, I'd write "This doesn't free any of the shared state
associated with the iterator, just our backend-private state."

Done

+    PagetableEntry *lpage = ((PagetableEntry *) arg) + *((int *) left);
+    PagetableEntry *rpage = ((PagetableEntry *) arg) + *((int *) right);

Again, try to use array dereference notation rather than pointer
arithmetic where you can. Maybe:

PageTableEntry *base = (PageTableEntry *) arg;
PageTableEntry *lpage = &arg[*(int *) left];
etc.

Done

+tbm_attach_shared_iterate(dsa_area *dsa, dsa_pointer iterator)
+{
+    TBMSharedIterator *shared_iterator;
+    TBMSharedIteratorState *shared_state;

Again, variable naming issues. Let's go with dsa_pointer dp,
TBMSharedIterator *iterator, TBMSharedIteratorState *istate. Please
try to go through and always use the same variable name for the same
kind of object. It's confusing when "iterator" sometimes means a
TBMSharedIterator * and sometimes a TBMSharedIteratorState * and
sometimes a dsa_pointer. Make it all consistent (and hopefully
logical).

Done

+    shared_iterator = (TBMSharedIterator *) palloc(sizeof(TBMIterator) +
+                                 MAX_TUPLES_PER_PAGE * sizeof(OffsetNumber));

Wrong sizeof. (This isn't just a style issue - it's a bug!)

Right, it's a bug. Fixed.

+ * Callback function for allocating the memory for hashtable elements.
+ * It allocates memory from DSA if tbm holds a reference to a dsa.

Inconsistent capitalization of "DSA" again.

Maybe change the whole comment to "Allocate memory for hashtable
elements, using DSA if available."

Fixe

+ if (tbm->dsapagetable)

Test against InvalidDsaPointer. Or maybe just get rid of the "if" and
do it unconditionally; if dsapagetable is InvalidDsaPointer then
dsapagetableold will be too, so no harm done.

Right

+ LWTRANCHE_PARALLEL_TBM_ITERATOR,

Change to LWLTRANCHE_TBM, add a call to LWLockRegisterTranche in
lwlock.c, and patch the docs with the new tranche name.

Fixed

Despite this fairly extensive list of mostly-trivial cosmetic
problems, I think this is in fairly good shape now. I think the basic
design is right, so once we get these loose ends tied up this should
be good to go in.

0002: actual parallel bitmap heap scan by using interfaces of 0001.

This needs a lot more extensive review than I quite have the time and
energy for right now, but here are a few comments.

+ <entry><literal>ParallelBitmapScan</></entry>

ParallelBitmapPopulate, maybe? You're not waiting for the scan;
you're waiting for the bitmap to be populated.

Done

+ <entry>Waiting for leader backend to complete the TidBitmap.</entry>

Likewise, complete -> populate?

Done

+ /* If any limit was set to zero, the user doesn't want a parallel scan. */

If we got zero? There aren't multiple limits mentioned anywhere
nearby, so this reads oddly.

Removed

+        /*
+         * It may be possible to amortize some of the I/O cost, but probably
+         * not very much, because most operating systems already do aggressive
+         * prefetching.  For now, we assume that the disk run cost can't be
+         * amortized at all.
+         */

This looks like it was cut-and-pasted from the sequential scan case,
but it doesn't really apply here. This presupposes we're relying on
OS prefetching, but in the case of a bitmap heap scan we do our own
prefetching. Maybe change to "Since we do prefetching in both the
parallel and non-parallel cases, we don't expect to get substantially
more I/O parallelism by spreading the scan across multiple backends.
So, assume the disk run cost can't be amortized at all."

Removed this comment.

Apart from these, there are some more changes.
in 0001:
- Improved the comments for TBMSharedIteratorState and TBMSharedIterator.
- Added error handling for return pointer from dsa_allocate.
- Moved function declaration in tidbitmap.h (functionaly related are
kept together)

in 0002:
- Improved comments.
- Code refactoring in BitmapHeapNext.
- Removed local tbm creation in BitmapHeapNext : as per new tidbitmap
it's of no use.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

gather_shutdown_children_first.patchapplication/octet-stream; name=gather_shutdown_children_first.patchDownload
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 0dd95c6..5ccc2e8 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -815,6 +815,8 @@ ExecShutdownNode(PlanState *node)
 	if (node == NULL)
 		return false;
 
+	planstate_tree_walker(node, ExecShutdownNode, NULL);
+
 	switch (nodeTag(node))
 	{
 		case T_GatherState:
@@ -824,5 +826,5 @@ ExecShutdownNode(PlanState *node)
 			break;
 	}
 
-	return planstate_tree_walker(node, ExecShutdownNode, NULL);
+	return false;
 }
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index a1a3561..32c97d3 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -229,10 +229,10 @@ ExecGather(GatherState *node)
 void
 ExecEndGather(GatherState *node)
 {
+	ExecEndNode(outerPlanState(node));		/* let children clean up first */
 	ExecShutdownGather(node);
 	ExecFreeExprContext(&node->ps);
 	ExecClearTuple(node->ps.ps_ResultTupleSlot);
-	ExecEndNode(outerPlanState(node));
 }
 
 /*
0001-tidbitmap-support-shared-v4.patchapplication/octet-stream; name=0001-tidbitmap-support-shared-v4.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index fad5cb0..1d70fae 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1082,6 +1082,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel query dynamic shared memory allocation lock.</entry>
         </row>
         <row>
+         <entry><literal>tbm</></entry>
+         <entry>Waiting for TBM shared iterator lock.</entry>
+        </row>
+        <row>
          <entry morerows="9"><literal>Lock</></entry>
          <entry><literal>relation</></entry>
          <entry>Waiting to acquire a lock on a relation.</entry>
diff --git a/src/backend/access/gin/ginget.c b/src/backend/access/gin/ginget.c
index 60f005c..87cd9ea 100644
--- a/src/backend/access/gin/ginget.c
+++ b/src/backend/access/gin/ginget.c
@@ -120,7 +120,7 @@ collectMatchBitmap(GinBtreeData *btree, GinBtreeStack *stack,
 	Form_pg_attribute attr;
 
 	/* Initialize empty bitmap result */
-	scanEntry->matchBitmap = tbm_create(work_mem * 1024L);
+	scanEntry->matchBitmap = tbm_create(work_mem * 1024L, NULL);
 
 	/* Null query cannot partial-match anything */
 	if (scanEntry->isPartialMatch &&
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..94bb289 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,7 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L);
+		tbm = tbm_create(work_mem * 1024L, NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 3c6155b..1d280be 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,7 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L);
+				result = tbm_create(work_mem * 1024L, NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..127e312 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -43,6 +43,8 @@
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
 #include "nodes/tidbitmap.h"
+#include "storage/lwlock.h"
+#include "utils/dsa.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -121,6 +123,16 @@ typedef enum
 } TBMStatus;
 
 /*
+ * Current iterating state of the TBM.
+ */
+typedef enum
+{
+	TBM_NOT_ITERATING,			/* not yet converted to page and chunk array */
+	TBM_ITERATING_PRIVATE,		/* converted to local page and chunk array */
+	TBM_ITERATING_SHARED		/* converted to shared page and chunk array */
+} TBMIteratingState;
+
+/*
  * Here is the representation for a whole TIDBitMap:
  */
 struct TIDBitmap
@@ -133,12 +145,17 @@ struct TIDBitmap
 	int			maxentries;		/* limit on same to meet maxbytes */
 	int			npages;			/* number of exact entries in pagetable */
 	int			nchunks;		/* number of lossy entries in pagetable */
-	bool		iterating;		/* tbm_begin_iterate called? */
+	TBMIteratingState iterating;	/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	dsa_pointer dsapagetable;	/* dsa_pointer to the element array */
+	dsa_pointer dsapagetableold;	/* dsa_pointer to the old element array */
+	dsa_pointer ptpages;		/* dsa_pointer to the page array */
+	dsa_pointer ptchunks;		/* dsa_pointer to the chunk array */
+	dsa_area   *dsa;			/* reference to per-query dsa area */
 };
 
 /*
@@ -156,6 +173,37 @@ struct TBMIterator
 	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
 };
 
+/*
+ * Holds the shared members of the iterator so that multiple processes
+ * can jointly iterate.
+ */
+typedef struct TBMSharedIteratorState
+{
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	dsa_pointer pagetable;		/* dsa pointers to head of pagetable data */
+	dsa_pointer spages;			/* dsa pointer to page array */
+	dsa_pointer schunks;		/* dsa pointer to chunk array */
+	LWLock		lock;			/* lock to protect below members */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+} TBMSharedIteratorState;
+
+/*
+ * same as TBMIterator, but it is used for joint iteration, therefore this
+ * also holds a reference to the shared state.
+ */
+struct TBMSharedIterator
+{
+	TBMSharedIteratorState *state;		/* shared state */
+	PagetableEntry *ptbase;		/* pointer to the pagetable element array */
+	int		   *ptpages;		/* sorted exact page index list */
+	int		   *ptchunks;		/* sorted lossy page index list */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+};
 
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
@@ -168,6 +216,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+static int tbm_shared_comparator(const void *left, const void *right,
+					  void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -187,6 +237,7 @@ hash_blockno(BlockNumber b)
 }
 
 /* define hashtable mapping block numbers to PagetableEntry's */
+#define SH_USE_NONDEFAULT_ALLOCATOR
 #define SH_PREFIX pagetable
 #define SH_ELEMENT_TYPE PagetableEntry
 #define SH_KEY_TYPE BlockNumber
@@ -204,10 +255,12 @@ hash_blockno(BlockNumber b)
  *
  * The bitmap will live in the memory context that is CurrentMemoryContext
  * at the time of this call.  It will be limited to (approximately) maxbytes
- * total memory consumption.
+ * total memory consumption.  If the DSA passed to this function is not NULL
+ * then the memory for storing elements of the underlying page table will
+ * be allocated from the DSA.
  */
 TIDBitmap *
-tbm_create(long maxbytes)
+tbm_create(long maxbytes, dsa_area *dsa)
 {
 	TIDBitmap  *tbm;
 	long		nbuckets;
@@ -230,6 +283,7 @@ tbm_create(long maxbytes)
 	nbuckets = Max(nbuckets, 16);		/* sanity limit */
 	tbm->maxentries = (int) nbuckets;
 	tbm->lossify_start = 0;
+	tbm->dsa = dsa;
 
 	return tbm;
 }
@@ -244,7 +298,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, tbm);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -294,7 +348,7 @@ tbm_add_tuples(TIDBitmap *tbm, const ItemPointer tids, int ntids,
 	PagetableEntry *page = NULL;	/* only valid when currblk is valid */
 	int			i;
 
-	Assert(!tbm->iterating);
+	Assert(tbm->iterating == TBM_NOT_ITERATING);
 	for (i = 0; i < ntids; i++)
 	{
 		BlockNumber blk = ItemPointerGetBlockNumber(tids + i);
@@ -603,6 +657,8 @@ tbm_begin_iterate(TIDBitmap *tbm)
 {
 	TBMIterator *iterator;
 
+	Assert(tbm->iterating != TBM_ITERATING_SHARED);
+
 	/*
 	 * Create the TBMIterator struct, with enough trailing space to serve the
 	 * needs of the TBMIterateResult sub-struct.
@@ -624,7 +680,7 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	 * attached to the bitmap not the iterator, so they can be used by more
 	 * than one iterator.
 	 */
-	if (tbm->status == TBM_HASH && !tbm->iterating)
+	if (tbm->status == TBM_HASH && tbm->iterating == TBM_NOT_ITERATING)
 	{
 		pagetable_iterator i;
 		PagetableEntry *page;
@@ -659,12 +715,207 @@ tbm_begin_iterate(TIDBitmap *tbm)
 				  tbm_comparator);
 	}
 
-	tbm->iterating = true;
+	tbm->iterating = TBM_ITERATING_PRIVATE;
 
 	return iterator;
 }
 
 /*
+ * tbm_prepare_shared_iterate - prepare shared iteration state for a TIDBitmap.
+ *
+ * The necessary shared state will be allocated from the DSA passed to
+ * tbm_create, so that multiple processes can attach to it and iterate jointly.
+ *
+ * This will convert the pagetable hash into page and chunk array of the index
+ * into pagetable array.
+ */
+dsa_pointer
+tbm_prepare_shared_iterate(TIDBitmap *tbm)
+{
+	dsa_pointer dp;
+	TBMSharedIteratorState *istate;
+
+	Assert(tbm->dsa != NULL);
+	Assert(tbm->iterating != TBM_ITERATING_PRIVATE);
+
+	/*
+	 * Allocate TBMSharedIteratorState from DSA to hold the shared members and
+	 * lock, this will also be used by multiple worker for shared iterate.
+	 */
+	dp = dsa_allocate(tbm->dsa, sizeof(TBMSharedIteratorState));
+	if (!DsaPointerIsValid(dp))
+		ereport(ERROR,
+				(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+				 errmsg("out of memory")));
+
+	istate = dsa_get_address(tbm->dsa, dp);
+
+	/*
+	 * If we're not already iterating, create and fill the sorted page lists.
+	 * (If we are, the sorted page lists are already stored in the TIDBitmap,
+	 * and we can just reuse them.)
+	 */
+	if (tbm->iterating == TBM_NOT_ITERATING)
+	{
+		pagetable_iterator i;
+		PagetableEntry *ptbase;
+		PagetableEntry *page;
+		int			idx;
+		int			npages;
+		int			nchunks;
+		int		   *ptpages;
+		int		   *ptchunks;
+
+		/*
+		 * Allocate the page and chunk array memory from the DSA to share
+		 * across multiple processes.
+		 */
+		if (tbm->npages)
+		{
+			tbm->ptpages = dsa_allocate(tbm->dsa, tbm->npages * sizeof(int));
+			if (!DsaPointerIsValid(tbm->ptpages))
+				ereport(ERROR,
+						(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+						 errmsg("out of memory")));
+			ptpages = dsa_get_address(tbm->dsa, tbm->ptpages);
+		}
+		if (tbm->nchunks)
+		{
+			tbm->ptchunks = dsa_allocate(tbm->dsa,
+										 tbm->nchunks * sizeof(int));
+			if (!DsaPointerIsValid(tbm->ptchunks))
+				ereport(ERROR,
+						(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+						 errmsg("out of memory")));
+			ptchunks = dsa_get_address(tbm->dsa, tbm->ptchunks);
+		}
+
+		/*
+		 * If TBM status is TBM_HASH then iterate over the pagetable and
+		 * convert it to page and chunk arrays.  But if it's in the
+		 * TBM_ONE_PAGE mode then directly allocate the space for one entry
+		 * from the DSA.
+		 */
+		npages = nchunks = 0;
+		if (tbm->status == TBM_HASH)
+		{
+			ptbase = dsa_get_address(tbm->dsa, tbm->dsapagetable);
+
+			pagetable_start_iterate(tbm->pagetable, &i);
+			while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+			{
+				idx = page - ptbase;
+				if (page->ischunk)
+					ptchunks[nchunks++] = idx;
+				else
+					ptpages[npages++] = idx;
+			}
+
+			Assert(npages == tbm->npages);
+			Assert(nchunks == tbm->nchunks);
+		}
+		else
+		{
+			/*
+			 * In one page mode allocate the space for one pagetable entry and
+			 * directly store its index i.e. 0 in page array
+			 */
+			tbm->dsapagetable = dsa_allocate(tbm->dsa, sizeof(PagetableEntry));
+			if (!DsaPointerIsValid(tbm->dsapagetable))
+				ereport(ERROR,
+						(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+						 errmsg("out of memory")));
+			ptbase = dsa_get_address(tbm->dsa, tbm->dsapagetable);
+			ptpages[0] = 0;
+		}
+
+		if (npages > 1)
+			qsort_arg((void *) ptpages, npages, sizeof(int),
+					  tbm_shared_comparator, (void *) ptbase);
+		if (nchunks > 1)
+			qsort_arg((void *) ptchunks, nchunks, sizeof(int),
+					  tbm_shared_comparator, (void *) ptbase);
+	}
+
+	/*
+	 * Store the TBM members in the shared state so that we can share them
+	 * across multiple processes.
+	 */
+	istate->nentries = tbm->nentries;
+	istate->maxentries = tbm->maxentries;
+	istate->npages = tbm->npages;
+	istate->nchunks = tbm->nchunks;
+	istate->pagetable = tbm->dsapagetable;
+	istate->spages = tbm->ptpages;
+	istate->schunks = tbm->ptchunks;
+
+	/* Initialize the iterator lock */
+	LWLockInitialize(&istate->lock, LWTRANCHE_TBM);
+
+	/* Initialize the shared iterator state */
+	istate->schunkbit = 0;
+	istate->schunkptr = 0;
+	istate->spageptr = 0;
+
+	tbm->iterating = TBM_ITERATING_SHARED;
+
+	return dp;
+}
+
+/*
+ * tbm_extract_page_tuple - extract the tuple offsets from a page
+ *
+ * The extracted offsets are stored into TBMIterateResult.
+ */
+static inline int
+tbm_extract_page_tuple(PagetableEntry *page, TBMIterateResult *output)
+{
+	int			wordnum;
+	int			ntuples = 0;
+
+	for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
+	{
+		bitmapword	w = page->words[wordnum];
+
+		if (w != 0)
+		{
+			int			off = wordnum * BITS_PER_BITMAPWORD + 1;
+
+			while (w != 0)
+			{
+				if (w & 1)
+					output->offsets[ntuples++] = (OffsetNumber) off;
+				off++;
+				w >>= 1;
+			}
+		}
+	}
+
+	return ntuples;
+}
+
+/*
+ *	tbm_advance_schunkbit - Advance the chunkbit
+ */
+static inline void
+tbm_advance_schunkbit(PagetableEntry *chunk, int *schunkbitp)
+{
+	int			schunkbit = *schunkbitp;
+
+	while (schunkbit < PAGES_PER_CHUNK)
+	{
+		int			wordnum = WORDNUM(schunkbit);
+		int			bitnum = BITNUM(schunkbit);
+
+		if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
+			break;
+		schunkbit++;
+	}
+
+	*schunkbitp = schunkbit;
+}
+
+/*
  * tbm_iterate - scan through next page of a TIDBitmap
  *
  * Returns a TBMIterateResult representing one page, or NULL if there are
@@ -682,7 +933,7 @@ tbm_iterate(TBMIterator *iterator)
 	TIDBitmap  *tbm = iterator->tbm;
 	TBMIterateResult *output = &(iterator->output);
 
-	Assert(tbm->iterating);
+	Assert(tbm->iterating == TBM_ITERATING_PRIVATE);
 
 	/*
 	 * If lossy chunk pages remain, make sure we've advanced schunkptr/
@@ -693,15 +944,7 @@ tbm_iterate(TBMIterator *iterator)
 		PagetableEntry *chunk = tbm->schunks[iterator->schunkptr];
 		int			schunkbit = iterator->schunkbit;
 
-		while (schunkbit < PAGES_PER_CHUNK)
-		{
-			int			wordnum = WORDNUM(schunkbit);
-			int			bitnum = BITNUM(schunkbit);
-
-			if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
-				break;
-			schunkbit++;
-		}
+		tbm_advance_schunkbit(chunk, &schunkbit);
 		if (schunkbit < PAGES_PER_CHUNK)
 		{
 			iterator->schunkbit = schunkbit;
@@ -738,7 +981,6 @@ tbm_iterate(TBMIterator *iterator)
 	{
 		PagetableEntry *page;
 		int			ntuples;
-		int			wordnum;
 
 		/* In ONE_PAGE state, we don't allocate an spages[] array */
 		if (tbm->status == TBM_ONE_PAGE)
@@ -747,24 +989,7 @@ tbm_iterate(TBMIterator *iterator)
 			page = tbm->spages[iterator->spageptr];
 
 		/* scan bitmap to extract individual offset numbers */
-		ntuples = 0;
-		for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
-		{
-			bitmapword	w = page->words[wordnum];
-
-			if (w != 0)
-			{
-				int			off = wordnum * BITS_PER_BITMAPWORD + 1;
-
-				while (w != 0)
-				{
-					if (w & 1)
-						output->offsets[ntuples++] = (OffsetNumber) off;
-					off++;
-					w >>= 1;
-				}
-			}
-		}
+		ntuples = tbm_extract_page_tuple(page, output);
 		output->blockno = page->blockno;
 		output->ntuples = ntuples;
 		output->recheck = page->recheck;
@@ -777,6 +1002,94 @@ tbm_iterate(TBMIterator *iterator)
 }
 
 /*
+ *	tbm_shared_iterate - scan through next page of a TIDBitmap
+ *
+ *	As above, but this will iterate using an iterator which is shared
+ *	across multiple processes.  We need to acquire the iterator LWLock,
+ *	before accessing the shared members.
+ */
+TBMIterateResult *
+tbm_shared_iterate(TBMSharedIterator *iterator)
+{
+	TBMIterateResult *output = &iterator->output;
+	TBMSharedIteratorState *istate = iterator->state;
+	PagetableEntry *ptbase = iterator->ptbase;
+	int		   *ptpages = iterator->ptpages;
+	int		   *ptchunks = iterator->ptchunks;
+
+	/* Acquire the LWLock before accessing the shared members */
+	LWLockAcquire(&istate->lock, LW_EXCLUSIVE);
+
+	/*
+	 * If lossy chunk pages remain, make sure we've advanced schunkptr/
+	 * schunkbit to the next set bit.
+	 */
+	while (istate->schunkptr < istate->nchunks)
+	{
+		PagetableEntry *chunk = &ptbase[ptchunks[istate->schunkptr]];
+		int			schunkbit = istate->schunkbit;
+
+		tbm_advance_schunkbit(chunk, &schunkbit);
+		if (schunkbit < PAGES_PER_CHUNK)
+		{
+			istate->schunkbit = schunkbit;
+			break;
+		}
+		/* advance to next chunk */
+		istate->schunkptr++;
+		istate->schunkbit = 0;
+	}
+
+	/*
+	 * If both chunk and per-page data remain, must output the numerically
+	 * earlier page.
+	 */
+	if (istate->schunkptr < istate->nchunks)
+	{
+		PagetableEntry *chunk = &ptbase[ptchunks[istate->schunkptr]];
+		PagetableEntry *page = &ptbase[ptpages[istate->spageptr]];
+		BlockNumber chunk_blockno;
+
+		chunk_blockno = chunk->blockno + istate->schunkbit;
+
+		if (istate->spageptr >= istate->npages ||
+			chunk_blockno < page->blockno)
+		{
+			/* Return a lossy page indicator from the chunk */
+			output->blockno = chunk_blockno;
+			output->ntuples = -1;
+			output->recheck = true;
+			istate->schunkbit++;
+
+			LWLockRelease(&istate->lock);
+			return output;
+		}
+	}
+
+	if (istate->spageptr < istate->npages)
+	{
+		PagetableEntry *page = &ptbase[ptpages[istate->spageptr]];
+		int			ntuples;
+
+		/* scan bitmap to extract individual offset numbers */
+		ntuples = tbm_extract_page_tuple(page, output);
+		output->blockno = page->blockno;
+		output->ntuples = ntuples;
+		output->recheck = page->recheck;
+		istate->spageptr++;
+
+		LWLockRelease(&istate->lock);
+
+		return output;
+	}
+
+	LWLockRelease(&istate->lock);
+
+	/* Nothing more in the bitmap */
+	return NULL;
+}
+
+/*
  * tbm_end_iterate - finish an iteration over a TIDBitmap
  *
  * Currently this is just a pfree, but it might do more someday.  (For
@@ -790,6 +1103,18 @@ tbm_end_iterate(TBMIterator *iterator)
 }
 
 /*
+ * tbm_end_shared_iterate - finish a shared iteration over a TIDBitmap
+ *
+ * This doesn't free any of the shared state associated with the iterator,
+ * just our backend-private state.
+ */
+void
+tbm_end_shared_iterate(TBMSharedIterator *iterator)
+{
+	pfree(iterator);
+}
+
+/*
  * tbm_find_pageentry - find a PagetableEntry for the pageno
  *
  * Returns NULL if there is no non-lossy entry for the pageno.
@@ -995,7 +1320,7 @@ tbm_lossify(TIDBitmap *tbm)
 	 * push nentries down to significantly less than maxentries, or else we'll
 	 * just end up doing this again very soon.  We shoot for maxentries/2.
 	 */
-	Assert(!tbm->iterating);
+	Assert(tbm->iterating == TBM_NOT_ITERATING);
 	Assert(tbm->status == TBM_HASH);
 
 	pagetable_start_iterate_at(tbm->pagetable, &i, tbm->lossify_start);
@@ -1061,3 +1386,106 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * As above, but this will get index into PagetableEntry array.  Therefore,
+ * it needs to get actual PagetableEntry using the index before comparing the
+ * blockno.
+ */
+static int
+tbm_shared_comparator(const void *left, const void *right, void *arg)
+{
+	PagetableEntry *base = (PagetableEntry *) arg;
+	PagetableEntry *lpage = &base[*(int *) left];
+	PagetableEntry *rpage = &base[*(int *) right];
+
+	if (lpage->blockno < rpage->blockno)
+		return -1;
+	else if (lpage->blockno > rpage->blockno)
+		return 1;
+	return 0;
+}
+
+/*
+ *	tbm_attach_shared_iterate
+ *
+ *	Allocate a backend-private iterator and attach the shared iterator state
+ *	to it so that multiple processed can iterate jointly.
+ *
+ *	We also converts the DSA pointers to local pointers and store them into
+ *	our private iterator.
+ */
+TBMSharedIterator *
+tbm_attach_shared_iterate(dsa_area *dsa, dsa_pointer dp)
+{
+	TBMSharedIterator *iterator;
+	TBMSharedIteratorState *istate;
+
+	/*
+	 * Create the TBMSharedIterator struct, with enough trailing space to
+	 * serve the needs of the TBMIterateResult sub-struct.
+	 */
+	iterator = (TBMSharedIterator *) palloc(sizeof(TBMSharedIterator) +
+								 MAX_TUPLES_PER_PAGE * sizeof(OffsetNumber));
+
+	istate = (TBMSharedIteratorState *) dsa_get_address(dsa, dp);
+
+	iterator->state = istate;
+
+	iterator->ptbase = dsa_get_address(dsa, istate->pagetable);
+	if (istate->npages)
+		iterator->ptpages = dsa_get_address(dsa, istate->spages);
+	if (istate->nchunks)
+		iterator->ptchunks = dsa_get_address(dsa, istate->schunks);
+
+	return iterator;
+}
+
+/*
+ * pagetable_allocate
+ *
+ * Callback function for allocating the memory for hashtable elements.
+ * Allocate memory for hashtable elements, using DSA if available.
+ */
+static inline void *
+pagetable_allocate(pagetable_hash *pagetable, Size size)
+{
+	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
+
+	if (tbm->dsa == NULL)
+		return MemoryContextAllocExtended(pagetable->ctx, size,
+										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+
+	/*
+	 * Save the dsapagetable reference in dsapagetableold before allocating
+	 * new memory so that pagetable_free can free the old entry.
+	 */
+	tbm->dsapagetableold = tbm->dsapagetable;
+	tbm->dsapagetable = dsa_allocate0(tbm->dsa, size);
+	if (!DsaPointerIsValid(tbm->dsapagetable))
+		ereport(ERROR,
+				(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+				 errmsg("out of memory")));
+	return dsa_get_address(tbm->dsa, tbm->dsapagetable);
+}
+
+/*
+ * pagetable_free
+ *
+ * Callback function for freeing hash table elements.
+ */
+static inline void
+pagetable_free(pagetable_hash *pagetable, void *pointer)
+{
+	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
+
+	/* pfree the input pointer if DSA is not available */
+	if (tbm->dsa == NULL)
+		pfree(pointer);
+	else
+	{
+		/* Free the dsapagetableold and make it same as dsapagetable */
+		dsa_free(tbm->dsa, tbm->dsapagetableold);
+		tbm->dsapagetableold = tbm->dsapagetable;
+	}
+}
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index ab81d94..3e13394 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -510,6 +510,7 @@ RegisterLWLockTranches(void)
 						  "predicate_lock_manager");
 	LWLockRegisterTranche(LWTRANCHE_PARALLEL_QUERY_DSA,
 						  "parallel_query_dsa");
+	LWLockRegisterTranche(LWTRANCHE_TBM, "tbm");
 
 	/* Register named tranches. */
 	for (i = 0; i < NamedLWLockTrancheRequests; i++)
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..cff5683 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -23,6 +23,7 @@
 #define TIDBITMAP_H
 
 #include "storage/itemptr.h"
+#include "utils/dsa.h"
 
 
 /*
@@ -33,6 +34,7 @@ typedef struct TIDBitmap TIDBitmap;
 
 /* Likewise, TBMIterator is private */
 typedef struct TBMIterator TBMIterator;
+typedef struct TBMSharedIterator TBMSharedIterator;
 
 /* Result structure for tbm_iterate */
 typedef struct
@@ -46,7 +48,7 @@ typedef struct
 
 /* function prototypes in nodes/tidbitmap.c */
 
-extern TIDBitmap *tbm_create(long maxbytes);
+extern TIDBitmap *tbm_create(long maxbytes, dsa_area *dsa);
 extern void tbm_free(TIDBitmap *tbm);
 
 extern void tbm_add_tuples(TIDBitmap *tbm,
@@ -60,7 +62,12 @@ extern void tbm_intersect(TIDBitmap *a, const TIDBitmap *b);
 extern bool tbm_is_empty(const TIDBitmap *tbm);
 
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
+extern dsa_pointer tbm_prepare_shared_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
+extern TBMIterateResult *tbm_shared_iterate(TBMSharedIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_end_shared_iterate(TBMSharedIterator *iterator);
+extern TBMSharedIterator *tbm_attach_shared_iterate(dsa_area *dsa,
+						  dsa_pointer dp);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 8bd93c3..0cd45bb 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -212,6 +212,7 @@ typedef enum BuiltinTrancheIds
 	LWTRANCHE_LOCK_MANAGER,
 	LWTRANCHE_PREDICATE_LOCK_MANAGER,
 	LWTRANCHE_PARALLEL_QUERY_DSA,
+	LWTRANCHE_TBM,
 	LWTRANCHE_FIRST_USER_DEFINED
 }	BuiltinTrancheIds;
 
0002-parallel-bitmap-heapscan-v4.patchapplication/octet-stream; name=0002-parallel-bitmap-heapscan-v4.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1d70fae..6a3481f 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1248,6 +1248,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapPopulate</></entry>
+         <entry>Waiting for the leader to populate the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index af25836..bffc971 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1754,6 +1754,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 98d4f1e..8d61fcb 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
 
 #include "executor/execParallel.h"
 #include "executor/executor.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "executor/nodeCustom.h"
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
@@ -210,6 +211,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -266,6 +271,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -745,6 +755,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 5ccc2e8..704dcfa 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -822,6 +822,9 @@ ExecShutdownNode(PlanState *node)
 		case T_GatherState:
 			ExecShutdownGather((GatherState *) node);
 			break;
+		case T_BitmapHeapScanState:
+			ExecShutdownBitmapHeapScan((BitmapHeapScanState *) node);
+			break;
 		default:
 			break;
 	}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index f18827d..537db45 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,11 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
-
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapState *pbminfo);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -66,13 +67,17 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	HeapScanDesc scan;
 	TIDBitmap  *tbm;
 	TBMIterator *tbmiterator;
+	TBMSharedIterator *stbmiterator;
 	TBMIterateResult *tbmres;
 
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
+	TBMSharedIterator *sprefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelBitmapState *pstate = node->pstate;
+	dsa_area   *dsa = node->ss.ps.state->es_query_dsa;
 
 	/*
 	 * extract necessary information from index scan node
@@ -82,11 +87,16 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	scan = node->ss.ss_currentScanDesc;
 	tbm = node->tbm;
 	tbmiterator = node->tbmiterator;
+	stbmiterator = node->stbmiterator;
 	tbmres = node->tbmres;
 #ifdef USE_PREFETCH
 	prefetch_iterator = node->prefetch_iterator;
+	sprefetch_iterator = node->sprefetch_iterator;
 #endif
 
+	if (node->inited)
+		goto start_iterate;
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -99,7 +109,7 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 * node->prefetch_maximum.  This is to avoid doing a lot of prefetching in
 	 * a scan that stops after a few tuples because of a LIMIT.
 	 */
-	if (tbm == NULL)
+	if (!pstate)
 	{
 		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
 
@@ -119,7 +129,69 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		}
 #endif   /* USE_PREFETCH */
 	}
+	else
+	{
+		/*
+		 * The leader will immediately come out of the function, but others
+		 * will be blocked until leader populates the TBM and wakes them up.
+		 */
+		if (pbms_is_leader(pstate))
+		{
+			/*
+			 * Set the flag in the lower nodes to indicate that we need a
+			 * shared TBM.
+			 */
+			pbms_set_parallel(outerPlanState(node));
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			/*
+			 * Prepare to iterate over the TBM. This will return the
+			 * dsa_pointer of the iterator state which will be used by
+			 * multiple processes to iterate jointly.
+			 */
+			pstate->tbmiterator = tbm_prepare_shared_iterate(tbm);
+#ifdef USE_PREFETCH
+			if (node->prefetch_maximum > 0)
+			{
+				pstate->prefetch_iterator = tbm_prepare_shared_iterate(tbm);
+
+				/* We don't need mutex here as we haven't yet woke up others */
+				pstate->prefetch_pages = 0;
+				pstate->prefetch_target = -1;
+			}
+#endif
 
+			/*
+			 * By this time we have already populated the TBM and initialized
+			 * the shared iterators so set the state to PBM_FINISHED and wake
+			 * up others.
+			 */
+			SpinLockAcquire(&pstate->state_mutex);
+			pstate->state = PBM_FINISHED;
+			SpinLockRelease(&pstate->state_mutex);
+			ConditionVariableBroadcast(&pstate->cv);
+		}
+
+		/* Allocate a private iterator and attach the shared state to it */
+		stbmiterator = tbm_attach_shared_iterate(dsa, pstate->tbmiterator);
+		node->stbmiterator = stbmiterator;
+		node->tbmres = tbmres = NULL;
+
+#ifdef USE_PREFETCH
+		if (node->prefetch_maximum > 0)
+		{
+			sprefetch_iterator =
+				tbm_attach_shared_iterate(dsa, pstate->prefetch_iterator);
+			node->sprefetch_iterator = sprefetch_iterator;
+		}
+#endif   /* USE_PREFETCH */
+	}
+
+	node->inited = true;
+
+start_iterate:
 	for (;;)
 	{
 		Page		dp;
@@ -130,7 +202,14 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			/*
+			 * If we are in parallel mode perform shared iterate otherwise
+			 * local iterate.
+			 */
+			if (!pstate)
+				node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			else
+				node->tbmres = tbmres = tbm_shared_iterate(stbmiterator);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,19 +217,50 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+
+			if (!pstate)
 			{
-				/* The main iterator has closed the distance by one page */
-				node->prefetch_pages--;
+				if (node->prefetch_pages > 0)
+					/* The main iterator has closed the distance by one page */
+					node->prefetch_pages--;
+				else if (prefetch_iterator)
+				{
+					/*
+					 * Do not let the prefetch iterator get behind the main
+					 * one.
+					 */
+					TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+
+					if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+						elog(ERROR, "prefetch and main iterators are out of sync");
+				}
 			}
-			else if (prefetch_iterator)
+			else if (node->prefetch_maximum > 0)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
-
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
-					elog(ERROR, "prefetch and main iterators are out of sync");
+				SpinLockAcquire(&pstate->prefetch_mutex);
+				if (pstate->prefetch_pages > 0)
+				{
+					node->prefetch_pages--;
+					SpinLockRelease(&pstate->prefetch_mutex);
+				}
+				else
+				{
+					/* Release the mutex before iterating */
+					SpinLockRelease(&pstate->prefetch_mutex);
+
+					/*
+					 * In case of shared mode, we can not ensure that the
+					 * current blockno of the main iterator and that of the
+					 * prefetch iterator are same.  It's possible that
+					 * whatever blockno we are prefetching will be processed
+					 * by another process.  Therefore we don't validate the
+					 * blockno like we do in non-parallel case.
+					 */
+					if (sprefetch_iterator)
+						tbm_shared_iterate(sprefetch_iterator);
+				}
 			}
+
 #endif   /* USE_PREFETCH */
 
 			/*
@@ -188,14 +298,32 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * page/tuple, then to one after the second tuple is fetched, then
 			 * it doubles as later pages are fetched.
 			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			if (!pstate)
+			{
+				if (node->prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (node->prefetch_target >= node->prefetch_maximum / 2)
+					node->prefetch_target = node->prefetch_maximum;
+				else if (node->prefetch_target > 0)
+					node->prefetch_target *= 2;
+				else
+					node->prefetch_target++;
+			}
+			else if (pstate->prefetch_target < node->prefetch_maximum)
+			{
+				SpinLockAcquire(&pstate->prefetch_mutex);
+				if (pstate->prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (pstate->prefetch_target >=
+						 node->prefetch_maximum / 2)
+					pstate->prefetch_target = node->prefetch_maximum;
+				else if (pstate->prefetch_target > 0)
+					pstate->prefetch_target *= 2;
+				else
+					pstate->prefetch_target++;
+				SpinLockRelease(&pstate->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +339,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (!pstate)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else if (pstate->prefetch_target < node->prefetch_maximum)
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pstate->prefetch_mutex);
+				if (pstate->prefetch_target < node->prefetch_maximum)
+					pstate->prefetch_target++;
+				SpinLockRelease(&pstate->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +378,71 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			if (!pstate)
+			{
+				while (node->prefetch_pages < node->prefetch_target)
+				{
+					TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+
+					if (tbmpre == NULL)
+					{
+						/* No more pages to prefetch */
+						tbm_end_iterate(prefetch_iterator);
+						node->prefetch_iterator = prefetch_iterator = NULL;
+						break;
+					}
+					node->prefetch_pages++;
+					PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+				}
+			}
+			else if (pstate->prefetch_pages < pstate->prefetch_target)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				SpinLockAcquire(&pstate->prefetch_mutex);
 
-				if (tbmpre == NULL)
+				/* Recheck whether we still need to prefetch under the mutex */
+				if (!pstate->prefetching &&
+					pstate->prefetch_pages < pstate->prefetch_target)
 				{
-					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
-					break;
+					/* Allow only one process to prefetch */
+					pstate->prefetching = true;
+					SpinLockRelease(&pstate->prefetch_mutex);
+
+					do
+					{
+						TBMIterateResult *tbmpre;
+
+						tbmpre = tbm_shared_iterate(sprefetch_iterator);
+						if (tbmpre == NULL)
+						{
+							/* No more pages to prefetch */
+							tbm_end_shared_iterate(sprefetch_iterator);
+							node->sprefetch_iterator =
+								sprefetch_iterator = NULL;
+							break;
+						}
+
+						SpinLockAcquire(&pstate->prefetch_mutex);
+						pstate->prefetch_pages++;
+						if (pstate->prefetch_pages >= pstate->prefetch_target)
+						{
+							SpinLockRelease(&pstate->prefetch_mutex);
+							break;
+						}
+						SpinLockRelease(&pstate->prefetch_mutex);
+
+						PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM,
+									   tbmpre->blockno);
+					} while (true);
+
+					SpinLockAcquire(&pstate->prefetch_mutex);
+					pstate->prefetching = false;
+					SpinLockRelease(&pstate->prefetch_mutex);
 				}
-				node->prefetch_pages++;
-				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+				else
+					SpinLockRelease(&pstate->prefetch_mutex);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -458,12 +650,23 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->tbmiterator);
 	if (node->prefetch_iterator)
 		tbm_end_iterate(node->prefetch_iterator);
+	if (node->stbmiterator)
+		tbm_end_shared_iterate(node->stbmiterator);
+	if (node->sprefetch_iterator)
+		tbm_end_shared_iterate(node->sprefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
 	node->tbm = NULL;
 	node->tbmiterator = NULL;
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
+	node->inited = false;
+	node->stbmiterator = NULL;
+	node->sprefetch_iterator = NULL;
+
+	/* Reset parallel bitmap state, if present */
+	if (node->pstate)
+		node->pstate->state = PBM_INITIAL;
 
 	ExecScanReScan(&node->ss);
 
@@ -514,8 +717,12 @@ ExecEndBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->tbmiterator);
 	if (node->prefetch_iterator)
 		tbm_end_iterate(node->prefetch_iterator);
-	if (node->tbm)
-		tbm_free(node->tbm);
+	if (node->stbmiterator)
+		tbm_end_shared_iterate(node->stbmiterator);
+	if (node->sprefetch_iterator)
+		tbm_end_shared_iterate(node->sprefetch_iterator);
+
+	ExecShutdownBitmapHeapScan(node);
 
 	/*
 	 * close heap scan
@@ -567,6 +774,11 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->pscan_len = 0;
+	scanstate->inited = false;
+	scanstate->stbmiterator = NULL;
+	scanstate->sprefetch_iterator = NULL;
+	scanstate->pstate = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -651,3 +863,174 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/* ----------------------------------------------------------------
+ *		ExecShutdownBitmapHeapScan
+ * ----------------------------------------------------------------
+ */
+void
+ExecShutdownBitmapHeapScan(BitmapHeapScanState *node)
+{
+	if (node->tbm)
+		tbm_free(node->tbm);
+
+	node->tbm = NULL;
+}
+
+/*----------------
+ *		pbms_is_leader
+ *
+ *		The first process to come here and see the state to the PBM_INITIAL
+ *		will become the leader for the parallel bitmap scan and will be
+ *		responsible for populating the TIDBitmap.  The other processes will
+ *		be blocked by the condition variable until the leader wakes them up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapState *pstate)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : We become the leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : We need to wait till leader creates bitmap
+		 * PBM_FINISHED   : bitmap is ready so no need to wait
+		 *---------------
+		 */
+		SpinLockAcquire(&pstate->state_mutex);
+
+		if (pstate->state == PBM_INITIAL)
+		{
+			pstate->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pstate->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pstate->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pstate->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/*-------------------
+ *		pbms_set_parallel
+ *
+ *		Recursively process the node and set the parallel flag. This flag
+ *		will be used to indicate the underlying layer to allocate the
+ *		pagetable elements from the DSA.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * For BitmapAnd node, set parallel flag only for the first child because
+	 * only that node will create the main TBM and other TBMs will be merged
+	 * to the main one.  Only BitmapOr and BitmapIndex nodes create TBM,
+	 * therefore set the parallel flag only for these types.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			((BitmapOrState *) node)->is_parallel = true;;
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+
+	/* Estimate the size for sharing parallel Bitmap info. */
+	node->pscan_len = add_size(offsetof(ParallelBitmapState,
+										phs_snapshot_data),
+							   EstimateSnapshotSpace(estate->es_snapshot));
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapState *pstate;
+	EState	   *estate = node->ss.ps.state;
+
+	pstate = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pstate->prefetch_mutex);
+	pstate->tbmiterator = 0;
+	pstate->prefetch_iterator = 0;
+	pstate->prefetch_pages = 0;
+	pstate->prefetch_target = 0;
+	pstate->prefetching = false;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pstate->state_mutex);
+	pstate->state = PBM_INITIAL;
+	ConditionVariableInit(&pstate->cv);
+	SerializeSnapshot(estate->es_snapshot, pstate->phs_snapshot_data);
+
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pstate);
+	node->pstate = pstate;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapState *pstate;
+	Snapshot	snapshot;
+
+	pstate = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->pstate = pstate;
+
+	snapshot = RestoreSnapshot(pstate->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 94bb289..2f01cad 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,8 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L, NULL);
+		tbm = tbm_create(work_mem * 1024L,
+				node->biss_Parallel ? node->ss.ps.state->es_query_dsa: NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 1d280be..e8f110b 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,8 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L, NULL);
+				result = tbm_create(work_mem * 1024L, node->is_parallel ?
+									node->ps.state->es_query_dsa: NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index eeacf81..f671f0a 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2875,6 +2875,30 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 }
 
 /*
+ * create_partial_bitmap_paths
+ *	  Build partial bitmap heap path for the relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched, 0);
+
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+/*
  * Compute the number of parallel workers that should be used to scan a
  * relation.  We compute the parallel workers based on the size of the heap to
  * be scanned and the size of the index to be scanned, then choose a minimum
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index d01630f..6240819 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -861,6 +861,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -922,8 +923,21 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
+
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 56eccaf..097318f 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1618,6 +1622,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1660,6 +1669,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3248296..6e70808 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1068,7 +1068,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1077,9 +1078,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3255,7 +3256,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index ada374c..9b191df 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3395,6 +3395,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index a864f78..7e85510 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -179,6 +179,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..34ec484 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,18 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecShutdownBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 9f41bab..58b60cb 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1261,6 +1263,7 @@ typedef struct BitmapOrState
 	PlanState	ps;				/* its first field is NodeTag */
 	PlanState **bitmapplans;	/* array of PlanStates for my inputs */
 	int			nplans;			/* number of input plans */
+	bool		is_parallel;	/* create shared TBM pagetable if it's set */
 } BitmapOrState;
 
 /* ----------------------------------------------------------------
@@ -1443,6 +1446,7 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Create shared TBM pagetable if it's set
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1459,9 +1463,60 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will set the
+ *							state to PBM_INPROGRESS and that process will
+ *							be responsible for creating TIDBitmap.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							therefore workers need to sleep until leader set
+ *							the state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker can
+ *							proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+} PBMState;
+
+/* ----------------
+ *	 ParallelBitmapState : Shared state for BitmapHeapScanState
+ *		tbmiterator				iterator for scanning current pages
+ *		prefetch_iterator		iterator for prefetching ahead of current page
+ *		prefetch_mutex			mutual exclusion for the prefetching variables
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		prefetching				set true if prefetching is in progress
+ *		state_mutex				mutual exclusion for state
+ *		state					current state of the TIDBitmap
+ *		cv						conditional wait variable
+ *		phs_snapshot_data		snapshot data shared to workers
+ * ----------------
+ */
+typedef struct ParallelBitmapState
+{
+	dsa_pointer	tbmiterator;
+	dsa_pointer	prefetch_iterator;
+	slock_t		prefetch_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	bool		prefetching;
+	slock_t		state_mutex;
+	PBMState	state;
+	ConditionVariable cv;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+} ParallelBitmapState;
+
+/* ----------------
  *	 BitmapHeapScanState information
  *
  *		bitmapqualorig	   execution state for bitmapqualorig expressions
@@ -1474,7 +1529,12 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		inited			   is node is ready to iterate
+ *		stbmiterator	   shared iterator
+ *		sprefetch_iterator shared iterator for prefetching
+ *		pstate			   shared state for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1489,6 +1549,11 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	bool				 inited;
+	TBMSharedIterator	*stbmiterator;
+	TBMSharedIterator	*sprefetch_iterator;
+	ParallelBitmapState *pstate;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 53cad24..d22b039 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -53,7 +53,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index ebda308..3ec2d9d 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -56,6 +56,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
 extern int compute_parallel_worker(RelOptInfo *rel, BlockNumber heap_pages,
 						BlockNumber index_pages);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+										Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 8b710ec..e31389f 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -787,6 +787,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#87Dilip Kumar
dilipbalaut@gmail.com
In reply to: Dilip Kumar (#86)
Re: Parallel bitmap heap scan

On Sat, Feb 18, 2017 at 10:45 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

in 0002:
- Improved comments.
- Code refactoring in BitmapHeapNext.
- Removed local tbm creation in BitmapHeapNext : as per new tidbitmap
it's of no use.

I have observed one problem with 0002 and I though of sharing that
before fixing the same because we might have encountered the same
problem in some other patches i.e parallel shared hash and there might
be already a way to handle that.

The problem is that In ExecEndBitmapHeapScan we do tbm_free. Wherein,
it frees local pagetable memory and the shared memory using callback
routine and if other than the Backend (actual backend for the client
which is executing gather node) node any other worker become the
leader (worker which is responsible for creating the shared TBM) then
it will finish it work and free the shared memory by calling
ExecEndBitmapHeapScan, and it's possible that other worker are still
operating on the shared memory.

So far I never observed this issue in our test may be because either
Backend become the leader or by the time leader (non backend) free the
TBM others also finishes there work.

Solution:
- One solution to this problem can be that leader should not complete
the scan unless all other worker have finished the work.
- We can also think of that we don't make anyone wait but we make a
arrangement that last worker to finish the bitmapscan only free the
memory, but one problem with this solution is that last worker can be
non-leader also, which will have access to shared memory but how to
free the pagetable local memory (it's local to the leader).

Any suggestion on this ?

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#88Amit Kapila
amit.kapila16@gmail.com
In reply to: Dilip Kumar (#87)
Re: Parallel bitmap heap scan

On Sun, Feb 19, 2017 at 7:18 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Sat, Feb 18, 2017 at 10:45 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

in 0002:
- Improved comments.
- Code refactoring in BitmapHeapNext.
- Removed local tbm creation in BitmapHeapNext : as per new tidbitmap
it's of no use.

I have observed one problem with 0002 and I though of sharing that
before fixing the same because we might have encountered the same
problem in some other patches i.e parallel shared hash and there might
be already a way to handle that.

The problem is that In ExecEndBitmapHeapScan we do tbm_free. Wherein,
it frees local pagetable memory and the shared memory using callback
routine and if other than the Backend (actual backend for the client
which is executing gather node) node any other worker become the
leader (worker which is responsible for creating the shared TBM) then
it will finish it work and free the shared memory by calling
ExecEndBitmapHeapScan, and it's possible that other worker are still
operating on the shared memory.
So far I never observed this issue in our test may be because either
Backend become the leader or by the time leader (non backend) free the
TBM others also finishes there work.

Solution:
- One solution to this problem can be that leader should not complete
the scan unless all other worker have finished the work.

For Parallel Seq Scan, we do wait for parallel workers to finish
before freeing the shared memory. Why the case is different here?

--
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#89Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#87)
Re: Parallel bitmap heap scan

On Sun, Feb 19, 2017 at 7:18 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

I have observed one problem with 0002 and I though of sharing that
before fixing the same because we might have encountered the same
problem in some other patches i.e parallel shared hash and there might
be already a way to handle that.

The problem is that In ExecEndBitmapHeapScan we do tbm_free. Wherein,
it frees local pagetable memory and the shared memory using callback
routine and if other than the Backend (actual backend for the client
which is executing gather node) node any other worker become the
leader (worker which is responsible for creating the shared TBM) then
it will finish it work and free the shared memory by calling
ExecEndBitmapHeapScan, and it's possible that other worker are still
operating on the shared memory.

So far I never observed this issue in our test may be because either
Backend become the leader or by the time leader (non backend) free the
TBM others also finishes there work.

Solution:
- One solution to this problem can be that leader should not complete
the scan unless all other worker have finished the work.
- We can also think of that we don't make anyone wait but we make a
arrangement that last worker to finish the bitmapscan only free the
memory, but one problem with this solution is that last worker can be
non-leader also, which will have access to shared memory but how to
free the pagetable local memory (it's local to the leader).

Any suggestion on this ?

It's probably OK if tbm_free() doesn't free the memory allocated from
DSA, and we just let cleanup at end of query do it. However, that
could cause some trouble if the Parallel Bitmap Heap Scan gets
executed over and over and keeps allocating more and more memory from
DSA. I think the way to fix that would be to maintain a reference
count that starts at 1 when the iterator arrays are created and gets
incremented every time a TBMSharedIteratorState is created. It gets
decremented when the TIDBitmap is destroyed that has iterator arrays
is destroyed, and each time a TBMSharedIteratorState is destroyed.
When it reaches 0, the process that reduces the reference count to 0
calls dsa_free on the DSA pointers for pagetable, spages, and schunks.
(Also, if a TIDBitmap is freed before iteration begins, it frees the
DSA pointer for the pagetable only; the others won't have values yet.)

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#90Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#89)
Re: Parallel bitmap heap scan

On Sun, Feb 19, 2017 at 7:44 PM, Robert Haas <robertmhaas@gmail.com> wrote:

It's probably OK if tbm_free() doesn't free the memory allocated from
DSA, and we just let cleanup at end of query do it. However, that
could cause some trouble if the Parallel Bitmap Heap Scan gets
executed over and over and keeps allocating more and more memory from
DSA.

Is it possible that Parallel Bitmap Heap Scan will be executed
multiple time without shutting down the Workers?

I can imagine it can get executed over and over if plan is something like below.

NestLoopJoin
-> SeqScan
-> Gather
-> Parallel Bitmap Heap Scan

But in such case every time the Inner node of the NLJ will be
rescanned i.e. Gather will be rescanned which in turn shutdown
workers.

So basically, what I want to propose is that Only during
ExecReScanBitmapHeapScan we can free all the DSA pointers because at
that time we can be sure that all the workers have completed there
task and we are safe to free. (And we don't free any DSA memory at
ExecEndBitmapHeapScan).

I think the way to fix that would be to maintain a reference

count that starts at 1 when the iterator arrays are created and gets
incremented every time a TBMSharedIteratorState is created. It gets
decremented when the TIDBitmap is destroyed that has iterator arrays
is destroyed, and each time a TBMSharedIteratorState is destroyed.
When it reaches 0, the process that reduces the reference count to 0
calls dsa_free on the DSA pointers for pagetable, spages, and schunks.
(Also, if a TIDBitmap is freed before iteration begins, it frees the
DSA pointer for the pagetable only; the others won't have values yet.)

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#91Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#90)
Re: Parallel bitmap heap scan

On Sun, Feb 19, 2017 at 9:59 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

I can imagine it can get executed over and over if plan is something like below.

NestLoopJoin
-> SeqScan
-> Gather
-> Parallel Bitmap Heap Scan

But in such case every time the Inner node of the NLJ will be
rescanned i.e. Gather will be rescanned which in turn shutdown
workers.

Yeah, but it looks like ExecReScanGather gets rid of the workers, but
reuses the existing DSM. I'm not quite sure what happens to the DSA.
It looks like it probably just hangs around from the previous
iteration, which means that any allocations will also hang around.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#92Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Robert Haas (#91)
Re: Parallel bitmap heap scan

On Sun, Feb 19, 2017 at 10:34 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Sun, Feb 19, 2017 at 9:59 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

I can imagine it can get executed over and over if plan is something like below.

NestLoopJoin
-> SeqScan
-> Gather
-> Parallel Bitmap Heap Scan

But in such case every time the Inner node of the NLJ will be
rescanned i.e. Gather will be rescanned which in turn shutdown
workers.

Yeah, but it looks like ExecReScanGather gets rid of the workers, but
reuses the existing DSM. I'm not quite sure what happens to the DSA.
It looks like it probably just hangs around from the previous
iteration, which means that any allocations will also hang around.

Yes, it hangs around. Being able to reuse state in a rescan is a
feature: you might be able to reuse a hash table or a bitmap.

In the Parallel Shared Hash patch, the last participant to detach from
the shared hash table barrier gets a different return code and runs
some cleanup code. The alternative was to make the leader wait for
the workers to finish accessing the hash table so that it could do
that. I had it that way in an early version, but my goal is to
minimise synchronisation points so now it's just 'last to leave turns
out the lights' with no waiting.

One practical problem that came up was the need for executor nodes to
get a chance to do that kind of cleanup before the DSM segment is
detached. In my patch series I introduced a new node API
ExecNodeDetach to allow for that. Andres objected that the need for
that is evidence that the existing protocol is broken and should be
fixed instead. I'm looking into that.

On Sun, Feb 19, 2017 at 9:59 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

So basically, what I want to propose is that Only during
ExecReScanBitmapHeapScan we can free all the DSA pointers because at
that time we can be sure that all the workers have completed there
task and we are safe to free. (And we don't free any DSA memory at
ExecEndBitmapHeapScan).

I think this works.

I also grappled a bit with the question of whether it's actually worth
trying to free DSA memory when you're finished with it, eating
precious CPU cycles at end of a join, or just letting the the
executor's DSA area get nuked at end of parallel execution. As you
say, there is a special case for rescans to avoid leaks. I described
this as a potential approach in a TODO note in my v5 patch, but
currently my code just does the clean-up every time on the grounds
that it's simple and hasn't shown up as a performance problem yet.

Some hand-wavy thoughts on this topic in the context of hash joins:

The argument for cleaning up sooner rather than later would be that it
could reduce the total peak memory usage of large execution plans. Is
that a reasonable goal and can we achieve it? I suspect the answer is
yes in theory but no in practice, and we don't even try to achieve it
in non-parallel queries as far as I know.

My understanding is that PostgreSQL's planner can generate left-deep,
bushy and right-deep hash join plans:

N nested left-deep hash joins: Each hash join is on the outer side of
its parent, supplying tuples to probe the parent hash table. Their
probe phases overlap, so all N hash tables must exist and be fully
loaded at the same time.

N nested right-deep hash joins: Each hash join is on the inner side
of its parent, supplying tuples to build the hash table of its parent.
Theoretically you only need two full hash tables in memory at peak,
because you'll finish probing each one while build its parent's hash
table and then not need the child's hash table again (unless you need
to rescan).

N nested bushy hash joins: Somewhere in between.

But we don't actually take advantage of that opportunity to reduce
peak memory today. We always have N live hash tables and don't free
them until standard_ExecutorEnd runs ExecProcEnd on the top of the
plan. Perhaps we could teach hash tables to free themselves ASAP at
the end of their probe phase unless they know a rescan is possible.
But that just opens a whole can of worms: if we care about total peak
memory usage, should it become a planner goal that might favour
right-deep hash joins? I guess similar questions must arise for
bitmap heap scan and anything else holding memory that it doesn't
technically need anymore, and parallel query doesn't really change
anything about the situation, except maybe that Gather nodes provide a
point of scoping somewhere in between 'eager destruction' and 'hog all
the space until end of plan' which makes things a bit better. I don't
know anywhere near enough about query planners to say whether such
ideas about planning are reasonable, and am quite aware that it's
difficult terrain, and I have other fish to fry, so I'm going to put
down the can opener and back away.

--
Thomas Munro
http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#93Tom Lane
tgl@sss.pgh.pa.us
In reply to: Thomas Munro (#92)
Re: Parallel bitmap heap scan

Thomas Munro <thomas.munro@enterprisedb.com> writes:

One practical problem that came up was the need for executor nodes to
get a chance to do that kind of cleanup before the DSM segment is
detached. In my patch series I introduced a new node API
ExecNodeDetach to allow for that. Andres objected that the need for
that is evidence that the existing protocol is broken and should be
fixed instead. I'm looking into that.

The thing that you really have to worry about for this kind of proposal
is "what if the query errors out and we never get to ExecEndNode"?
It's particularly nasty if you're talking about parallel queries where
maybe only one or some of the processes involved detect an error.

regards, tom lane

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#94Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#91)
Re: Parallel bitmap heap scan

On Sun, Feb 19, 2017 at 10:34 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Yeah, but it looks like ExecReScanGather gets rid of the workers, but
reuses the existing DSM. I'm not quite sure what happens to the DSA.
It looks like it probably just hangs around from the previous
iteration, which means that any allocations will also hang around.

Yes right, they hang around. But, during rescan
(ExecReScanBitmapHeapScan) we can free all these DSA pointers ? That
mean before reallocating the DSA pointers we would have already got
rid of the old ones.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#95Robert Haas
robertmhaas@gmail.com
In reply to: Thomas Munro (#92)
Re: Parallel bitmap heap scan

On Mon, Feb 20, 2017 at 5:55 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Sun, Feb 19, 2017 at 9:59 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

So basically, what I want to propose is that Only during
ExecReScanBitmapHeapScan we can free all the DSA pointers because at
that time we can be sure that all the workers have completed there
task and we are safe to free. (And we don't free any DSA memory at
ExecEndBitmapHeapScan).

I think this works.

OK.

Some hand-wavy thoughts on this topic in the context of hash joins:

The argument for cleaning up sooner rather than later would be that it
could reduce the total peak memory usage of large execution plans. Is
that a reasonable goal and can we achieve it? I suspect the answer is
yes in theory but no in practice, and we don't even try to achieve it
in non-parallel queries as far as I know.

We're pretty stupid about causing nodes to stop eating up resources as
early as we could; for example, when a Limit is filled, we don't make
any attempt to have scans underneath it release pins or memory or
anything else. But we don't usually let the same node consume memory
multiple times. ExecReScanBitmapHeapScan frees all of the memory used
for the previous bitmap in the non-parallel case, so it should
probably do that in the parallel case also.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#96Robert Haas
robertmhaas@gmail.com
In reply to: Tom Lane (#93)
Re: Parallel bitmap heap scan

On Mon, Feb 20, 2017 at 6:19 AM, Tom Lane <tgl@sss.pgh.pa.us> wrote:

Thomas Munro <thomas.munro@enterprisedb.com> writes:

One practical problem that came up was the need for executor nodes to
get a chance to do that kind of cleanup before the DSM segment is
detached. In my patch series I introduced a new node API
ExecNodeDetach to allow for that. Andres objected that the need for
that is evidence that the existing protocol is broken and should be
fixed instead. I'm looking into that.

The thing that you really have to worry about for this kind of proposal
is "what if the query errors out and we never get to ExecEndNode"?
It's particularly nasty if you're talking about parallel queries where
maybe only one or some of the processes involved detect an error.

I think that's not actually a problem, because we've already got code
to make sure that all DSM resources associated with the query get
blown away in that case. Of course, that code might have bugs, but if
it does, I think it's better to try to fix those bugs than to insert
some belt-and-suspenders mechanism for reclaiming every possible chunk
of memory in retail fashion, just like we blow up es_query_cxt rather
than trying to pfree allocations individually.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#97Tom Lane
tgl@sss.pgh.pa.us
In reply to: Robert Haas (#96)
Re: Parallel bitmap heap scan

Robert Haas <robertmhaas@gmail.com> writes:

On Mon, Feb 20, 2017 at 6:19 AM, Tom Lane <tgl@sss.pgh.pa.us> wrote:

The thing that you really have to worry about for this kind of proposal
is "what if the query errors out and we never get to ExecEndNode"?
It's particularly nasty if you're talking about parallel queries where
maybe only one or some of the processes involved detect an error.

I think that's not actually a problem, because we've already got code
to make sure that all DSM resources associated with the query get
blown away in that case. Of course, that code might have bugs, but if
it does, I think it's better to try to fix those bugs than to insert
some belt-and-suspenders mechanism for reclaiming every possible chunk
of memory in retail fashion, just like we blow up es_query_cxt rather
than trying to pfree allocations individually.

Actually, I think we're saying the same thing: rely on the general DSM
cleanup mechanism, don't insert extra stuff that you expect will get
done by executor shutdown.

regards, tom lane

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#98Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#95)
2 attachment(s)
Re: Parallel bitmap heap scan

On Mon, Feb 20, 2017 at 11:18 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Feb 20, 2017 at 5:55 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Sun, Feb 19, 2017 at 9:59 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

So basically, what I want to propose is that Only during
ExecReScanBitmapHeapScan we can free all the DSA pointers because at
that time we can be sure that all the workers have completed there
task and we are safe to free. (And we don't free any DSA memory at
ExecEndBitmapHeapScan).

I think this works.

OK.

In my latest version of the patch, I have fixed it as described above
i.e only cleanup in ExecReScanBitmapHeapScan.

For getting this there is some change in both the patches.

0001:
1. I have got a new interface, "tbm_free_shared_area(dsa_area *dsa,
dsa_pointer dp)" which will be responsible for freeing all the shared
members (pagetable, ptpage and ptchunk). Actually, we can not do this
in tbm_free itself because the only leader will have a local tbm with
reference to all these pointers and our parallel bitmap leader may not
necessarily be the actual backend.

If we want to get this done using tbm, then we need to create a local
tbm in each worker and get the shared member reference copied into tbm
using tbm_attach_shared_iterate like we were doing in the earlier
version.

2. Now tbm_free is not freeing any of the shared members which can be
accessed by other worker so tbm_free is safe to call from
ExecEndBitmapHeapScan without any safety check or ref count.

0002:
1. We don't need ExecShutdownBitmapHeapScan anymore because now we are
not freeing any shared member in ExecEndBitmapHeapScan.

2. In ExecReScanBitmapHeapScan we will call tbm_free_shared_area to
free the shared members of the TBM.
3. After that, we will free TBMSharedIteratorState what we allocated
using tbm_prepare_shared_iterate.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0001-tidbitmap-support-shared-v5.patchapplication/octet-stream; name=0001-tidbitmap-support-shared-v5.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index fad5cb0..1d70fae 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1082,6 +1082,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel query dynamic shared memory allocation lock.</entry>
         </row>
         <row>
+         <entry><literal>tbm</></entry>
+         <entry>Waiting for TBM shared iterator lock.</entry>
+        </row>
+        <row>
          <entry morerows="9"><literal>Lock</></entry>
          <entry><literal>relation</></entry>
          <entry>Waiting to acquire a lock on a relation.</entry>
diff --git a/src/backend/access/gin/ginget.c b/src/backend/access/gin/ginget.c
index 60f005c..87cd9ea 100644
--- a/src/backend/access/gin/ginget.c
+++ b/src/backend/access/gin/ginget.c
@@ -120,7 +120,7 @@ collectMatchBitmap(GinBtreeData *btree, GinBtreeStack *stack,
 	Form_pg_attribute attr;
 
 	/* Initialize empty bitmap result */
-	scanEntry->matchBitmap = tbm_create(work_mem * 1024L);
+	scanEntry->matchBitmap = tbm_create(work_mem * 1024L, NULL);
 
 	/* Null query cannot partial-match anything */
 	if (scanEntry->isPartialMatch &&
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..94bb289 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,7 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L);
+		tbm = tbm_create(work_mem * 1024L, NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 3c6155b..1d280be 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,7 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L);
+				result = tbm_create(work_mem * 1024L, NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..4e9af62 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -43,6 +43,8 @@
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
 #include "nodes/tidbitmap.h"
+#include "storage/lwlock.h"
+#include "utils/dsa.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -121,6 +123,16 @@ typedef enum
 } TBMStatus;
 
 /*
+ * Current iterating state of the TBM.
+ */
+typedef enum
+{
+	TBM_NOT_ITERATING,			/* not yet converted to page and chunk array */
+	TBM_ITERATING_PRIVATE,		/* converted to local page and chunk array */
+	TBM_ITERATING_SHARED		/* converted to shared page and chunk array */
+} TBMIteratingState;
+
+/*
  * Here is the representation for a whole TIDBitMap:
  */
 struct TIDBitmap
@@ -133,12 +145,17 @@ struct TIDBitmap
 	int			maxentries;		/* limit on same to meet maxbytes */
 	int			npages;			/* number of exact entries in pagetable */
 	int			nchunks;		/* number of lossy entries in pagetable */
-	bool		iterating;		/* tbm_begin_iterate called? */
+	TBMIteratingState iterating;	/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	dsa_pointer dsapagetable;	/* dsa_pointer to the element array */
+	dsa_pointer dsapagetableold;	/* dsa_pointer to the old element array */
+	dsa_pointer ptpages;		/* dsa_pointer to the page array */
+	dsa_pointer ptchunks;		/* dsa_pointer to the chunk array */
+	dsa_area   *dsa;			/* reference to per-query dsa area */
 };
 
 /*
@@ -156,6 +173,37 @@ struct TBMIterator
 	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
 };
 
+/*
+ * Holds the shared members of the iterator so that multiple processes
+ * can jointly iterate.
+ */
+typedef struct TBMSharedIteratorState
+{
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	dsa_pointer pagetable;		/* dsa pointers to head of pagetable data */
+	dsa_pointer spages;			/* dsa pointer to page array */
+	dsa_pointer schunks;		/* dsa pointer to chunk array */
+	LWLock		lock;			/* lock to protect below members */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+} TBMSharedIteratorState;
+
+/*
+ * same as TBMIterator, but it is used for joint iteration, therefore this
+ * also holds a reference to the shared state.
+ */
+struct TBMSharedIterator
+{
+	TBMSharedIteratorState *state;		/* shared state */
+	PagetableEntry *ptbase;		/* pointer to the pagetable element array */
+	int		   *ptpages;		/* sorted exact page index list */
+	int		   *ptchunks;		/* sorted lossy page index list */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+};
 
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
@@ -168,6 +216,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+static int tbm_shared_comparator(const void *left, const void *right,
+					  void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -187,6 +237,7 @@ hash_blockno(BlockNumber b)
 }
 
 /* define hashtable mapping block numbers to PagetableEntry's */
+#define SH_USE_NONDEFAULT_ALLOCATOR
 #define SH_PREFIX pagetable
 #define SH_ELEMENT_TYPE PagetableEntry
 #define SH_KEY_TYPE BlockNumber
@@ -204,10 +255,12 @@ hash_blockno(BlockNumber b)
  *
  * The bitmap will live in the memory context that is CurrentMemoryContext
  * at the time of this call.  It will be limited to (approximately) maxbytes
- * total memory consumption.
+ * total memory consumption.  If the DSA passed to this function is not NULL
+ * then the memory for storing elements of the underlying page table will
+ * be allocated from the DSA.
  */
 TIDBitmap *
-tbm_create(long maxbytes)
+tbm_create(long maxbytes, dsa_area *dsa)
 {
 	TIDBitmap  *tbm;
 	long		nbuckets;
@@ -230,6 +283,7 @@ tbm_create(long maxbytes)
 	nbuckets = Max(nbuckets, 16);		/* sanity limit */
 	tbm->maxentries = (int) nbuckets;
 	tbm->lossify_start = 0;
+	tbm->dsa = dsa;
 
 	return tbm;
 }
@@ -244,7 +298,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, tbm);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -281,6 +335,26 @@ tbm_free(TIDBitmap *tbm)
 }
 
 /*
+ * tbm_free_shared_area - free shared members of the TIDBitmap
+ */
+void
+tbm_free_shared_area(dsa_area *dsa, dsa_pointer dp)
+{
+	TBMSharedIteratorState *istate = dsa_get_address(dsa, dp);
+
+	if (DsaPointerIsValid(istate->pagetable))
+		dsa_free(dsa, istate->pagetable);
+	if (DsaPointerIsValid(istate->pagetable))
+		dsa_free(dsa, istate->spages);
+	if (DsaPointerIsValid(istate->schunks))
+		dsa_free(dsa, istate->schunks);
+
+	istate->pagetable = InvalidDsaPointer;
+	istate->spages = InvalidDsaPointer;
+	istate->schunks = InvalidDsaPointer;
+}
+
+/*
  * tbm_add_tuples - add some tuple IDs to a TIDBitmap
  *
  * If recheck is true, then the recheck flag will be set in the
@@ -294,7 +368,7 @@ tbm_add_tuples(TIDBitmap *tbm, const ItemPointer tids, int ntids,
 	PagetableEntry *page = NULL;	/* only valid when currblk is valid */
 	int			i;
 
-	Assert(!tbm->iterating);
+	Assert(tbm->iterating == TBM_NOT_ITERATING);
 	for (i = 0; i < ntids; i++)
 	{
 		BlockNumber blk = ItemPointerGetBlockNumber(tids + i);
@@ -603,6 +677,8 @@ tbm_begin_iterate(TIDBitmap *tbm)
 {
 	TBMIterator *iterator;
 
+	Assert(tbm->iterating != TBM_ITERATING_SHARED);
+
 	/*
 	 * Create the TBMIterator struct, with enough trailing space to serve the
 	 * needs of the TBMIterateResult sub-struct.
@@ -624,7 +700,7 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	 * attached to the bitmap not the iterator, so they can be used by more
 	 * than one iterator.
 	 */
-	if (tbm->status == TBM_HASH && !tbm->iterating)
+	if (tbm->status == TBM_HASH && tbm->iterating == TBM_NOT_ITERATING)
 	{
 		pagetable_iterator i;
 		PagetableEntry *page;
@@ -659,12 +735,190 @@ tbm_begin_iterate(TIDBitmap *tbm)
 				  tbm_comparator);
 	}
 
-	tbm->iterating = true;
+	tbm->iterating = TBM_ITERATING_PRIVATE;
 
 	return iterator;
 }
 
 /*
+ * tbm_prepare_shared_iterate - prepare shared iteration state for a TIDBitmap.
+ *
+ * The necessary shared state will be allocated from the DSA passed to
+ * tbm_create, so that multiple processes can attach to it and iterate jointly.
+ *
+ * This will convert the pagetable hash into page and chunk array of the index
+ * into pagetable array.
+ */
+dsa_pointer
+tbm_prepare_shared_iterate(TIDBitmap *tbm)
+{
+	dsa_pointer dp;
+	TBMSharedIteratorState *istate;
+
+	Assert(tbm->dsa != NULL);
+	Assert(tbm->iterating != TBM_ITERATING_PRIVATE);
+
+	/*
+	 * Allocate TBMSharedIteratorState from DSA to hold the shared members and
+	 * lock, this will also be used by multiple worker for shared iterate.
+	 */
+	dp = dsa_allocate(tbm->dsa, sizeof(TBMSharedIteratorState));
+	istate = dsa_get_address(tbm->dsa, dp);
+
+	/*
+	 * If we're not already iterating, create and fill the sorted page lists.
+	 * (If we are, the sorted page lists are already stored in the TIDBitmap,
+	 * and we can just reuse them.)
+	 */
+	if (tbm->iterating == TBM_NOT_ITERATING)
+	{
+		pagetable_iterator i;
+		PagetableEntry *ptbase;
+		PagetableEntry *page;
+		int			idx;
+		int			npages;
+		int			nchunks;
+		int		   *ptpages;
+		int		   *ptchunks;
+
+		/*
+		 * Allocate the page and chunk array memory from the DSA to share
+		 * across multiple processes.
+		 */
+		if (tbm->npages)
+		{
+			tbm->ptpages = dsa_allocate(tbm->dsa, tbm->npages * sizeof(int));
+			ptpages = dsa_get_address(tbm->dsa, tbm->ptpages);
+		}
+		if (tbm->nchunks)
+		{
+			tbm->ptchunks = dsa_allocate(tbm->dsa,
+										 tbm->nchunks * sizeof(int));
+			ptchunks = dsa_get_address(tbm->dsa, tbm->ptchunks);
+		}
+
+		/*
+		 * If TBM status is TBM_HASH then iterate over the pagetable and
+		 * convert it to page and chunk arrays.  But if it's in the
+		 * TBM_ONE_PAGE mode then directly allocate the space for one entry
+		 * from the DSA.
+		 */
+		npages = nchunks = 0;
+		if (tbm->status == TBM_HASH)
+		{
+			ptbase = dsa_get_address(tbm->dsa, tbm->dsapagetable);
+
+			pagetable_start_iterate(tbm->pagetable, &i);
+			while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+			{
+				idx = page - ptbase;
+				if (page->ischunk)
+					ptchunks[nchunks++] = idx;
+				else
+					ptpages[npages++] = idx;
+			}
+
+			Assert(npages == tbm->npages);
+			Assert(nchunks == tbm->nchunks);
+		}
+		else
+		{
+			/*
+			 * In one page mode allocate the space for one pagetable entry and
+			 * directly store its index i.e. 0 in page array
+			 */
+			tbm->dsapagetable = dsa_allocate(tbm->dsa, sizeof(PagetableEntry));
+			ptbase = dsa_get_address(tbm->dsa, tbm->dsapagetable);
+			ptpages[0] = 0;
+		}
+
+		if (npages > 1)
+			qsort_arg((void *) ptpages, npages, sizeof(int),
+					  tbm_shared_comparator, (void *) ptbase);
+		if (nchunks > 1)
+			qsort_arg((void *) ptchunks, nchunks, sizeof(int),
+					  tbm_shared_comparator, (void *) ptbase);
+	}
+
+	/*
+	 * Store the TBM members in the shared state so that we can share them
+	 * across multiple processes.
+	 */
+	istate->nentries = tbm->nentries;
+	istate->maxentries = tbm->maxentries;
+	istate->npages = tbm->npages;
+	istate->nchunks = tbm->nchunks;
+	istate->pagetable = tbm->dsapagetable;
+	istate->spages = tbm->ptpages;
+	istate->schunks = tbm->ptchunks;
+
+	/* Initialize the iterator lock */
+	LWLockInitialize(&istate->lock, LWTRANCHE_TBM);
+
+	/* Initialize the shared iterator state */
+	istate->schunkbit = 0;
+	istate->schunkptr = 0;
+	istate->spageptr = 0;
+
+	tbm->iterating = TBM_ITERATING_SHARED;
+
+	return dp;
+}
+
+/*
+ * tbm_extract_page_tuple - extract the tuple offsets from a page
+ *
+ * The extracted offsets are stored into TBMIterateResult.
+ */
+static inline int
+tbm_extract_page_tuple(PagetableEntry *page, TBMIterateResult *output)
+{
+	int			wordnum;
+	int			ntuples = 0;
+
+	for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
+	{
+		bitmapword	w = page->words[wordnum];
+
+		if (w != 0)
+		{
+			int			off = wordnum * BITS_PER_BITMAPWORD + 1;
+
+			while (w != 0)
+			{
+				if (w & 1)
+					output->offsets[ntuples++] = (OffsetNumber) off;
+				off++;
+				w >>= 1;
+			}
+		}
+	}
+
+	return ntuples;
+}
+
+/*
+ *	tbm_advance_schunkbit - Advance the chunkbit
+ */
+static inline void
+tbm_advance_schunkbit(PagetableEntry *chunk, int *schunkbitp)
+{
+	int			schunkbit = *schunkbitp;
+
+	while (schunkbit < PAGES_PER_CHUNK)
+	{
+		int			wordnum = WORDNUM(schunkbit);
+		int			bitnum = BITNUM(schunkbit);
+
+		if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
+			break;
+		schunkbit++;
+	}
+
+	*schunkbitp = schunkbit;
+}
+
+/*
  * tbm_iterate - scan through next page of a TIDBitmap
  *
  * Returns a TBMIterateResult representing one page, or NULL if there are
@@ -682,7 +936,7 @@ tbm_iterate(TBMIterator *iterator)
 	TIDBitmap  *tbm = iterator->tbm;
 	TBMIterateResult *output = &(iterator->output);
 
-	Assert(tbm->iterating);
+	Assert(tbm->iterating == TBM_ITERATING_PRIVATE);
 
 	/*
 	 * If lossy chunk pages remain, make sure we've advanced schunkptr/
@@ -693,15 +947,7 @@ tbm_iterate(TBMIterator *iterator)
 		PagetableEntry *chunk = tbm->schunks[iterator->schunkptr];
 		int			schunkbit = iterator->schunkbit;
 
-		while (schunkbit < PAGES_PER_CHUNK)
-		{
-			int			wordnum = WORDNUM(schunkbit);
-			int			bitnum = BITNUM(schunkbit);
-
-			if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
-				break;
-			schunkbit++;
-		}
+		tbm_advance_schunkbit(chunk, &schunkbit);
 		if (schunkbit < PAGES_PER_CHUNK)
 		{
 			iterator->schunkbit = schunkbit;
@@ -738,7 +984,6 @@ tbm_iterate(TBMIterator *iterator)
 	{
 		PagetableEntry *page;
 		int			ntuples;
-		int			wordnum;
 
 		/* In ONE_PAGE state, we don't allocate an spages[] array */
 		if (tbm->status == TBM_ONE_PAGE)
@@ -747,24 +992,7 @@ tbm_iterate(TBMIterator *iterator)
 			page = tbm->spages[iterator->spageptr];
 
 		/* scan bitmap to extract individual offset numbers */
-		ntuples = 0;
-		for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
-		{
-			bitmapword	w = page->words[wordnum];
-
-			if (w != 0)
-			{
-				int			off = wordnum * BITS_PER_BITMAPWORD + 1;
-
-				while (w != 0)
-				{
-					if (w & 1)
-						output->offsets[ntuples++] = (OffsetNumber) off;
-					off++;
-					w >>= 1;
-				}
-			}
-		}
+		ntuples = tbm_extract_page_tuple(page, output);
 		output->blockno = page->blockno;
 		output->ntuples = ntuples;
 		output->recheck = page->recheck;
@@ -777,6 +1005,94 @@ tbm_iterate(TBMIterator *iterator)
 }
 
 /*
+ *	tbm_shared_iterate - scan through next page of a TIDBitmap
+ *
+ *	As above, but this will iterate using an iterator which is shared
+ *	across multiple processes.  We need to acquire the iterator LWLock,
+ *	before accessing the shared members.
+ */
+TBMIterateResult *
+tbm_shared_iterate(TBMSharedIterator *iterator)
+{
+	TBMIterateResult *output = &iterator->output;
+	TBMSharedIteratorState *istate = iterator->state;
+	PagetableEntry *ptbase = iterator->ptbase;
+	int		   *ptpages = iterator->ptpages;
+	int		   *ptchunks = iterator->ptchunks;
+
+	/* Acquire the LWLock before accessing the shared members */
+	LWLockAcquire(&istate->lock, LW_EXCLUSIVE);
+
+	/*
+	 * If lossy chunk pages remain, make sure we've advanced schunkptr/
+	 * schunkbit to the next set bit.
+	 */
+	while (istate->schunkptr < istate->nchunks)
+	{
+		PagetableEntry *chunk = &ptbase[ptchunks[istate->schunkptr]];
+		int			schunkbit = istate->schunkbit;
+
+		tbm_advance_schunkbit(chunk, &schunkbit);
+		if (schunkbit < PAGES_PER_CHUNK)
+		{
+			istate->schunkbit = schunkbit;
+			break;
+		}
+		/* advance to next chunk */
+		istate->schunkptr++;
+		istate->schunkbit = 0;
+	}
+
+	/*
+	 * If both chunk and per-page data remain, must output the numerically
+	 * earlier page.
+	 */
+	if (istate->schunkptr < istate->nchunks)
+	{
+		PagetableEntry *chunk = &ptbase[ptchunks[istate->schunkptr]];
+		PagetableEntry *page = &ptbase[ptpages[istate->spageptr]];
+		BlockNumber chunk_blockno;
+
+		chunk_blockno = chunk->blockno + istate->schunkbit;
+
+		if (istate->spageptr >= istate->npages ||
+			chunk_blockno < page->blockno)
+		{
+			/* Return a lossy page indicator from the chunk */
+			output->blockno = chunk_blockno;
+			output->ntuples = -1;
+			output->recheck = true;
+			istate->schunkbit++;
+
+			LWLockRelease(&istate->lock);
+			return output;
+		}
+	}
+
+	if (istate->spageptr < istate->npages)
+	{
+		PagetableEntry *page = &ptbase[ptpages[istate->spageptr]];
+		int			ntuples;
+
+		/* scan bitmap to extract individual offset numbers */
+		ntuples = tbm_extract_page_tuple(page, output);
+		output->blockno = page->blockno;
+		output->ntuples = ntuples;
+		output->recheck = page->recheck;
+		istate->spageptr++;
+
+		LWLockRelease(&istate->lock);
+
+		return output;
+	}
+
+	LWLockRelease(&istate->lock);
+
+	/* Nothing more in the bitmap */
+	return NULL;
+}
+
+/*
  * tbm_end_iterate - finish an iteration over a TIDBitmap
  *
  * Currently this is just a pfree, but it might do more someday.  (For
@@ -790,6 +1106,18 @@ tbm_end_iterate(TBMIterator *iterator)
 }
 
 /*
+ * tbm_end_shared_iterate - finish a shared iteration over a TIDBitmap
+ *
+ * This doesn't free any of the shared state associated with the iterator,
+ * just our backend-private state.
+ */
+void
+tbm_end_shared_iterate(TBMSharedIterator *iterator)
+{
+	pfree(iterator);
+}
+
+/*
  * tbm_find_pageentry - find a PagetableEntry for the pageno
  *
  * Returns NULL if there is no non-lossy entry for the pageno.
@@ -995,7 +1323,7 @@ tbm_lossify(TIDBitmap *tbm)
 	 * push nentries down to significantly less than maxentries, or else we'll
 	 * just end up doing this again very soon.  We shoot for maxentries/2.
 	 */
-	Assert(!tbm->iterating);
+	Assert(tbm->iterating == TBM_NOT_ITERATING);
 	Assert(tbm->status == TBM_HASH);
 
 	pagetable_start_iterate_at(tbm->pagetable, &i, tbm->lossify_start);
@@ -1061,3 +1389,102 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * As above, but this will get index into PagetableEntry array.  Therefore,
+ * it needs to get actual PagetableEntry using the index before comparing the
+ * blockno.
+ */
+static int
+tbm_shared_comparator(const void *left, const void *right, void *arg)
+{
+	PagetableEntry *base = (PagetableEntry *) arg;
+	PagetableEntry *lpage = &base[*(int *) left];
+	PagetableEntry *rpage = &base[*(int *) right];
+
+	if (lpage->blockno < rpage->blockno)
+		return -1;
+	else if (lpage->blockno > rpage->blockno)
+		return 1;
+	return 0;
+}
+
+/*
+ *	tbm_attach_shared_iterate
+ *
+ *	Allocate a backend-private iterator and attach the shared iterator state
+ *	to it so that multiple processed can iterate jointly.
+ *
+ *	We also converts the DSA pointers to local pointers and store them into
+ *	our private iterator.
+ */
+TBMSharedIterator *
+tbm_attach_shared_iterate(dsa_area *dsa, dsa_pointer dp)
+{
+	TBMSharedIterator *iterator;
+	TBMSharedIteratorState *istate;
+
+	/*
+	 * Create the TBMSharedIterator struct, with enough trailing space to
+	 * serve the needs of the TBMIterateResult sub-struct.
+	 */
+	iterator = (TBMSharedIterator *) palloc(sizeof(TBMSharedIterator) +
+								 MAX_TUPLES_PER_PAGE * sizeof(OffsetNumber));
+
+	istate = (TBMSharedIteratorState *) dsa_get_address(dsa, dp);
+
+	iterator->state = istate;
+
+	iterator->ptbase = dsa_get_address(dsa, istate->pagetable);
+	if (istate->npages)
+		iterator->ptpages = dsa_get_address(dsa, istate->spages);
+	if (istate->nchunks)
+		iterator->ptchunks = dsa_get_address(dsa, istate->schunks);
+
+	return iterator;
+}
+
+/*
+ * pagetable_allocate
+ *
+ * Callback function for allocating the memory for hashtable elements.
+ * Allocate memory for hashtable elements, using DSA if available.
+ */
+static inline void *
+pagetable_allocate(pagetable_hash *pagetable, Size size)
+{
+	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
+
+	if (tbm->dsa == NULL)
+		return MemoryContextAllocExtended(pagetable->ctx, size,
+										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+
+	/*
+	 * Save the dsapagetable reference in dsapagetableold before allocating
+	 * new memory so that pagetable_free can free the old entry.
+	 */
+	tbm->dsapagetableold = tbm->dsapagetable;
+	tbm->dsapagetable = dsa_allocate0(tbm->dsa, size);
+
+	return dsa_get_address(tbm->dsa, tbm->dsapagetable);
+}
+
+/*
+ * pagetable_free
+ *
+ * Callback function for freeing hash table elements.
+ */
+static inline void
+pagetable_free(pagetable_hash *pagetable, void *pointer)
+{
+	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
+
+	/* pfree the input pointer if DSA is not available */
+	if (tbm->dsa == NULL)
+		pfree(pointer);
+	else if (DsaPointerIsValid(tbm->dsapagetableold))
+	{
+		dsa_free(tbm->dsa, tbm->dsapagetableold);
+		tbm->dsapagetableold = InvalidDsaPointer;
+	}
+}
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index ab81d94..3e13394 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -510,6 +510,7 @@ RegisterLWLockTranches(void)
 						  "predicate_lock_manager");
 	LWLockRegisterTranche(LWTRANCHE_PARALLEL_QUERY_DSA,
 						  "parallel_query_dsa");
+	LWLockRegisterTranche(LWTRANCHE_TBM, "tbm");
 
 	/* Register named tranches. */
 	for (i = 0; i < NamedLWLockTrancheRequests; i++)
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..87f4bb7 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -23,6 +23,7 @@
 #define TIDBITMAP_H
 
 #include "storage/itemptr.h"
+#include "utils/dsa.h"
 
 
 /*
@@ -33,6 +34,7 @@ typedef struct TIDBitmap TIDBitmap;
 
 /* Likewise, TBMIterator is private */
 typedef struct TBMIterator TBMIterator;
+typedef struct TBMSharedIterator TBMSharedIterator;
 
 /* Result structure for tbm_iterate */
 typedef struct
@@ -46,8 +48,9 @@ typedef struct
 
 /* function prototypes in nodes/tidbitmap.c */
 
-extern TIDBitmap *tbm_create(long maxbytes);
+extern TIDBitmap *tbm_create(long maxbytes, dsa_area *dsa);
 extern void tbm_free(TIDBitmap *tbm);
+extern void tbm_free_shared_area(dsa_area *dsa, dsa_pointer dp);
 
 extern void tbm_add_tuples(TIDBitmap *tbm,
 			   const ItemPointer tids, int ntids,
@@ -60,7 +63,12 @@ extern void tbm_intersect(TIDBitmap *a, const TIDBitmap *b);
 extern bool tbm_is_empty(const TIDBitmap *tbm);
 
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
+extern dsa_pointer tbm_prepare_shared_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
+extern TBMIterateResult *tbm_shared_iterate(TBMSharedIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_end_shared_iterate(TBMSharedIterator *iterator);
+extern TBMSharedIterator *tbm_attach_shared_iterate(dsa_area *dsa,
+						  dsa_pointer dp);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 8bd93c3..0cd45bb 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -212,6 +212,7 @@ typedef enum BuiltinTrancheIds
 	LWTRANCHE_LOCK_MANAGER,
 	LWTRANCHE_PREDICATE_LOCK_MANAGER,
 	LWTRANCHE_PARALLEL_QUERY_DSA,
+	LWTRANCHE_TBM,
 	LWTRANCHE_FIRST_USER_DEFINED
 }	BuiltinTrancheIds;
 
0002-parallel-bitmap-heapscan-v5.patchapplication/octet-stream; name=0002-parallel-bitmap-heapscan-v5.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1d70fae..6a3481f 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1248,6 +1248,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapPopulate</></entry>
+         <entry>Waiting for the leader to populate the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index af25836..bffc971 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1754,6 +1754,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 646df08..e5263f2 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
 
 #include "executor/execParallel.h"
 #include "executor/executor.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "executor/nodeCustom.h"
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
@@ -215,6 +216,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -275,6 +280,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -757,6 +767,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index f18827d..1d92a25 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -48,10 +48,11 @@
 #include "utils/snapmgr.h"
 #include "utils/tqual.h"
 
-
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
 
+static bool pbms_is_leader(ParallelBitmapState *pbminfo);
+static void pbms_set_parallel(PlanState *node);
 
 /* ----------------------------------------------------------------
  *		BitmapHeapNext
@@ -66,13 +67,17 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	HeapScanDesc scan;
 	TIDBitmap  *tbm;
 	TBMIterator *tbmiterator;
+	TBMSharedIterator *stbmiterator;
 	TBMIterateResult *tbmres;
 
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
+	TBMSharedIterator *sprefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelBitmapState *pstate = node->pstate;
+	dsa_area   *dsa = node->ss.ps.state->es_query_dsa;
 
 	/*
 	 * extract necessary information from index scan node
@@ -82,11 +87,16 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	scan = node->ss.ss_currentScanDesc;
 	tbm = node->tbm;
 	tbmiterator = node->tbmiterator;
+	stbmiterator = node->stbmiterator;
 	tbmres = node->tbmres;
 #ifdef USE_PREFETCH
 	prefetch_iterator = node->prefetch_iterator;
+	sprefetch_iterator = node->sprefetch_iterator;
 #endif
 
+	if (node->inited)
+		goto start_iterate;
+
 	/*
 	 * If we haven't yet performed the underlying index scan, do it, and begin
 	 * the iteration over the bitmap.
@@ -99,7 +109,7 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 * node->prefetch_maximum.  This is to avoid doing a lot of prefetching in
 	 * a scan that stops after a few tuples because of a LIMIT.
 	 */
-	if (tbm == NULL)
+	if (!pstate)
 	{
 		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
 
@@ -119,7 +129,71 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		}
 #endif   /* USE_PREFETCH */
 	}
+	else
+	{
+		/*
+		 * The leader will immediately come out of the function, but others
+		 * will be blocked until leader populates the TBM and wakes them up.
+		 */
+		if (pbms_is_leader(pstate))
+		{
+			/*
+			 * Set the flag in the lower nodes to indicate that we need a
+			 * shared TBM.
+			 */
+			pbms_set_parallel(outerPlanState(node));
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
+
+			node->tbm = tbm;
+
+			/*
+			 * Prepare to iterate over the TBM. This will return the
+			 * dsa_pointer of the iterator state which will be used by
+			 * multiple processes to iterate jointly.
+			 */
+			pstate->tbmiterator = tbm_prepare_shared_iterate(tbm);
+#ifdef USE_PREFETCH
+			if (node->prefetch_maximum > 0)
+			{
+				pstate->prefetch_iterator = tbm_prepare_shared_iterate(tbm);
+
+				/* We don't need mutex here as we haven't yet woke up others */
+				pstate->prefetch_pages = 0;
+				pstate->prefetch_target = -1;
+			}
+#endif
+
+			/*
+			 * By this time we have already populated the TBM and initialized
+			 * the shared iterators so set the state to PBM_FINISHED and wake
+			 * up others.
+			 */
+			SpinLockAcquire(&pstate->state_mutex);
+			pstate->state = PBM_FINISHED;
+			SpinLockRelease(&pstate->state_mutex);
+			ConditionVariableBroadcast(&pstate->cv);
+		}
+
+		/* Allocate a private iterator and attach the shared state to it */
+		stbmiterator = tbm_attach_shared_iterate(dsa, pstate->tbmiterator);
+		node->stbmiterator = stbmiterator;
+		node->tbmres = tbmres = NULL;
+
+#ifdef USE_PREFETCH
+		if (node->prefetch_maximum > 0)
+		{
+			sprefetch_iterator =
+				tbm_attach_shared_iterate(dsa, pstate->prefetch_iterator);
+			node->sprefetch_iterator = sprefetch_iterator;
+		}
+#endif   /* USE_PREFETCH */
+	}
+
+	node->inited = true;
 
+start_iterate:
 	for (;;)
 	{
 		Page		dp;
@@ -130,7 +204,14 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			/*
+			 * If we are in parallel mode perform shared iterate otherwise
+			 * local iterate.
+			 */
+			if (!pstate)
+				node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			else
+				node->tbmres = tbmres = tbm_shared_iterate(stbmiterator);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -138,19 +219,50 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
+
+			if (!pstate)
 			{
-				/* The main iterator has closed the distance by one page */
-				node->prefetch_pages--;
+				if (node->prefetch_pages > 0)
+					/* The main iterator has closed the distance by one page */
+					node->prefetch_pages--;
+				else if (prefetch_iterator)
+				{
+					/*
+					 * Do not let the prefetch iterator get behind the main
+					 * one.
+					 */
+					TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+
+					if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+						elog(ERROR, "prefetch and main iterators are out of sync");
+				}
 			}
-			else if (prefetch_iterator)
+			else if (node->prefetch_maximum > 0)
 			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
-
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
-					elog(ERROR, "prefetch and main iterators are out of sync");
+				SpinLockAcquire(&pstate->prefetch_mutex);
+				if (pstate->prefetch_pages > 0)
+				{
+					node->prefetch_pages--;
+					SpinLockRelease(&pstate->prefetch_mutex);
+				}
+				else
+				{
+					/* Release the mutex before iterating */
+					SpinLockRelease(&pstate->prefetch_mutex);
+
+					/*
+					 * In case of shared mode, we can not ensure that the
+					 * current blockno of the main iterator and that of the
+					 * prefetch iterator are same.  It's possible that
+					 * whatever blockno we are prefetching will be processed
+					 * by another process.  Therefore we don't validate the
+					 * blockno like we do in non-parallel case.
+					 */
+					if (sprefetch_iterator)
+						tbm_shared_iterate(sprefetch_iterator);
+				}
 			}
+
 #endif   /* USE_PREFETCH */
 
 			/*
@@ -188,14 +300,32 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * page/tuple, then to one after the second tuple is fetched, then
 			 * it doubles as later pages are fetched.
 			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			if (!pstate)
+			{
+				if (node->prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (node->prefetch_target >= node->prefetch_maximum / 2)
+					node->prefetch_target = node->prefetch_maximum;
+				else if (node->prefetch_target > 0)
+					node->prefetch_target *= 2;
+				else
+					node->prefetch_target++;
+			}
+			else if (pstate->prefetch_target < node->prefetch_maximum)
+			{
+				SpinLockAcquire(&pstate->prefetch_mutex);
+				if (pstate->prefetch_target >= node->prefetch_maximum)
+					 /* don't increase any further */ ;
+				else if (pstate->prefetch_target >=
+						 node->prefetch_maximum / 2)
+					pstate->prefetch_target = node->prefetch_maximum;
+				else if (pstate->prefetch_target > 0)
+					pstate->prefetch_target *= 2;
+				else
+					pstate->prefetch_target++;
+				SpinLockRelease(&pstate->prefetch_mutex);
+			}
+
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -211,8 +341,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (!pstate)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else if (pstate->prefetch_target < node->prefetch_maximum)
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pstate->prefetch_mutex);
+				if (pstate->prefetch_target < node->prefetch_maximum)
+					pstate->prefetch_target++;
+				SpinLockRelease(&pstate->prefetch_mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -236,21 +380,71 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
+			if (!pstate)
 			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+				while (node->prefetch_pages < node->prefetch_target)
+				{
+					TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+
+					if (tbmpre == NULL)
+					{
+						/* No more pages to prefetch */
+						tbm_end_iterate(prefetch_iterator);
+						node->prefetch_iterator = prefetch_iterator = NULL;
+						break;
+					}
+					node->prefetch_pages++;
+					PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+				}
+			}
+			else if (pstate->prefetch_pages < pstate->prefetch_target)
+			{
+				SpinLockAcquire(&pstate->prefetch_mutex);
 
-				if (tbmpre == NULL)
+				/* Recheck whether we still need to prefetch under the mutex */
+				if (!pstate->prefetching &&
+					pstate->prefetch_pages < pstate->prefetch_target)
 				{
-					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
-					break;
+					/* Allow only one process to prefetch */
+					pstate->prefetching = true;
+					SpinLockRelease(&pstate->prefetch_mutex);
+
+					do
+					{
+						TBMIterateResult *tbmpre;
+
+						tbmpre = tbm_shared_iterate(sprefetch_iterator);
+						if (tbmpre == NULL)
+						{
+							/* No more pages to prefetch */
+							tbm_end_shared_iterate(sprefetch_iterator);
+							node->sprefetch_iterator =
+								sprefetch_iterator = NULL;
+							break;
+						}
+
+						SpinLockAcquire(&pstate->prefetch_mutex);
+						pstate->prefetch_pages++;
+						if (pstate->prefetch_pages >= pstate->prefetch_target)
+						{
+							SpinLockRelease(&pstate->prefetch_mutex);
+							break;
+						}
+						SpinLockRelease(&pstate->prefetch_mutex);
+
+						PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM,
+									   tbmpre->blockno);
+					} while (true);
+
+					SpinLockAcquire(&pstate->prefetch_mutex);
+					pstate->prefetching = false;
+					SpinLockRelease(&pstate->prefetch_mutex);
 				}
-				node->prefetch_pages++;
-				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+				else
+					SpinLockRelease(&pstate->prefetch_mutex);
 			}
 		}
+
 #endif   /* USE_PREFETCH */
 
 		/*
@@ -458,12 +652,40 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->tbmiterator);
 	if (node->prefetch_iterator)
 		tbm_end_iterate(node->prefetch_iterator);
+	if (node->stbmiterator)
+		tbm_end_shared_iterate(node->stbmiterator);
+	if (node->sprefetch_iterator)
+		tbm_end_shared_iterate(node->sprefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
 	node->tbm = NULL;
 	node->tbmiterator = NULL;
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
+	node->inited = false;
+	node->stbmiterator = NULL;
+	node->sprefetch_iterator = NULL;
+
+	/* Reset parallel bitmap state, if present */
+	if (node->pstate)
+	{
+		dsa_area   *dsa = node->ss.ps.state->es_query_dsa;
+
+		node->pstate->state = PBM_INITIAL;
+
+		if (DsaPointerIsValid(node->pstate->tbmiterator))
+		{
+			/* First we free the shared TBM members using the shared state */
+			tbm_free_shared_area(dsa, node->pstate->tbmiterator);
+			dsa_free(dsa, node->pstate->tbmiterator);
+		}
+
+		if (DsaPointerIsValid(node->pstate->prefetch_iterator))
+			dsa_free(dsa, node->pstate->prefetch_iterator);
+
+		node->pstate->tbmiterator = InvalidDsaPointer;
+		node->pstate->prefetch_iterator = InvalidDsaPointer;
+	}
 
 	ExecScanReScan(&node->ss);
 
@@ -516,6 +738,10 @@ ExecEndBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->prefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
+	if (node->stbmiterator)
+		tbm_end_shared_iterate(node->stbmiterator);
+	if (node->sprefetch_iterator)
+		tbm_end_shared_iterate(node->sprefetch_iterator);
 
 	/*
 	 * close heap scan
@@ -567,6 +793,11 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->pscan_len = 0;
+	scanstate->inited = false;
+	scanstate->stbmiterator = NULL;
+	scanstate->sprefetch_iterator = NULL;
+	scanstate->pstate = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -651,3 +882,161 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ *		pbms_is_leader
+ *
+ *		The first process to come here and see the state to the PBM_INITIAL
+ *		will become the leader for the parallel bitmap scan and will be
+ *		responsible for populating the TIDBitmap.  The other processes will
+ *		be blocked by the condition variable until the leader wakes them up.
+ * ---------------
+ */
+static bool
+pbms_is_leader(ParallelBitmapState *pstate)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * PBM_INITIAL	  : We become the leader and set it to PBM_INPROGRESS
+		 * PBM_INPROGRESS : We need to wait till leader creates bitmap
+		 * PBM_FINISHED   : bitmap is ready so no need to wait
+		 *---------------
+		 */
+		SpinLockAcquire(&pstate->state_mutex);
+
+		if (pstate->state == PBM_INITIAL)
+		{
+			pstate->state = PBM_INPROGRESS;
+			leader = true;
+		}
+		else if (pstate->state == PBM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pstate->state_mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pstate->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/*-------------------
+ *		pbms_set_parallel
+ *
+ *		Recursively process the node and set the parallel flag. This flag
+ *		will be used to indicate the underlying layer to allocate the
+ *		pagetable elements from the DSA.
+ * -----------------
+ */
+static void
+pbms_set_parallel(PlanState *node)
+{
+	/*
+	 * For BitmapAnd node, set parallel flag only for the first child because
+	 * only that node will create the main TBM and other TBMs will be merged
+	 * to the main one.  Only BitmapOr and BitmapIndex nodes create TBM,
+	 * therefore set the parallel flag only for these types.
+	 */
+	switch (node->type)
+	{
+		case T_BitmapIndexScanState:
+			((BitmapIndexScanState *) node)->biss_Parallel = true;
+			break;
+		case T_BitmapOrState:
+			((BitmapOrState *) node)->is_parallel = true;;
+			break;
+		case T_BitmapAndState:
+			pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]);
+			break;
+		default:
+			break;
+	}
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+
+	/* Estimate the size for sharing parallel Bitmap info. */
+	node->pscan_len = add_size(offsetof(ParallelBitmapState,
+										phs_snapshot_data),
+							   EstimateSnapshotSpace(estate->es_snapshot));
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapState *pstate;
+	EState	   *estate = node->ss.ps.state;
+
+	pstate = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	/* Initialize mutex to protect prefetch pages and target */
+	SpinLockInit(&pstate->prefetch_mutex);
+	pstate->tbmiterator = 0;
+	pstate->prefetch_iterator = 0;
+	pstate->prefetch_pages = 0;
+	pstate->prefetch_target = 0;
+	pstate->prefetching = false;
+
+	/* Initialize mutex to protect current state */
+	SpinLockInit(&pstate->state_mutex);
+	pstate->state = PBM_INITIAL;
+	ConditionVariableInit(&pstate->cv);
+	SerializeSnapshot(estate->es_snapshot, pstate->phs_snapshot_data);
+
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pstate);
+	node->pstate = pstate;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapState *pstate;
+	Snapshot	snapshot;
+
+	pstate = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->pstate = pstate;
+
+	snapshot = RestoreSnapshot(pstate->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 94bb289..2f01cad 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,8 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L, NULL);
+		tbm = tbm_create(work_mem * 1024L,
+				node->biss_Parallel ? node->ss.ps.state->es_query_dsa: NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 1d280be..e8f110b 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,8 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L, NULL);
+				result = tbm_create(work_mem * 1024L, node->is_parallel ?
+									node->ps.state->es_query_dsa: NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index eeacf81..f671f0a 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2875,6 +2875,30 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 }
 
 /*
+ * create_partial_bitmap_paths
+ *	  Build partial bitmap heap path for the relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched, 0);
+
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+/*
  * Compute the number of parallel workers that should be used to scan a
  * relation.  We compute the parallel workers based on the size of the heap to
  * be scanned and the size of the index to be scanned, then choose a minimum
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index d01630f..6240819 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -861,6 +861,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -922,8 +923,21 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
+
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index d92826b..c411962 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1618,6 +1622,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1660,6 +1669,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3248296..6e70808 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1068,7 +1068,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1077,9 +1078,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3255,7 +3256,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index ada374c..9b191df 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3395,6 +3395,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index a864f78..7e85510 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -179,6 +179,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 1c1cb80..912a4cd 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1261,6 +1263,7 @@ typedef struct BitmapOrState
 	PlanState	ps;				/* its first field is NodeTag */
 	PlanState **bitmapplans;	/* array of PlanStates for my inputs */
 	int			nplans;			/* number of input plans */
+	bool		is_parallel;	/* create shared TBM pagetable if it's set */
 } BitmapOrState;
 
 /* ----------------------------------------------------------------
@@ -1445,6 +1448,7 @@ typedef struct IndexOnlyScanState
  *		RuntimeContext	   expr context for evaling runtime Skeys
  *		RelationDesc	   index relation descriptor
  *		ScanDesc		   index scan descriptor
+ *		Parallel		   Create shared TBM pagetable if it's set
  * ----------------
  */
 typedef struct BitmapIndexScanState
@@ -1461,9 +1465,60 @@ typedef struct BitmapIndexScanState
 	ExprContext *biss_RuntimeContext;
 	Relation	biss_RelationDesc;
 	IndexScanDesc biss_ScanDesc;
+	bool		biss_Parallel;
 } BitmapIndexScanState;
 
 /* ----------------
+ *	 PBMState information : Current status of the TIDBitmap creation during
+ *							parallel bitmap heap scan.
+ *
+ *		PBM_INITIAL			TIDBitmap creation is not yet started, so
+ *							first worker to see this state will set the
+ *							state to PBM_INPROGRESS and that process will
+ *							be responsible for creating TIDBitmap.
+ *		PBM_INPROGRESS		TIDBitmap creation is already in progress,
+ *							therefore workers need to sleep until leader set
+ *							the state to PBM_FINISHED and wake us up.
+ *		PBM_FINISHED		TIDBitmap creation is done, so now all worker can
+ *							proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	PBM_INITIAL,
+	PBM_INPROGRESS,
+	PBM_FINISHED
+} PBMState;
+
+/* ----------------
+ *	 ParallelBitmapState : Shared state for BitmapHeapScanState
+ *		tbmiterator				iterator for scanning current pages
+ *		prefetch_iterator		iterator for prefetching ahead of current page
+ *		prefetch_mutex			mutual exclusion for the prefetching variables
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		prefetching				set true if prefetching is in progress
+ *		state_mutex				mutual exclusion for state
+ *		state					current state of the TIDBitmap
+ *		cv						conditional wait variable
+ *		phs_snapshot_data		snapshot data shared to workers
+ * ----------------
+ */
+typedef struct ParallelBitmapState
+{
+	dsa_pointer	tbmiterator;
+	dsa_pointer	prefetch_iterator;
+	slock_t		prefetch_mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	bool		prefetching;
+	slock_t		state_mutex;
+	PBMState	state;
+	ConditionVariable cv;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+} ParallelBitmapState;
+
+/* ----------------
  *	 BitmapHeapScanState information
  *
  *		bitmapqualorig	   execution state for bitmapqualorig expressions
@@ -1476,7 +1531,12 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
- * ----------------
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		inited			   is node is ready to iterate
+ *		stbmiterator	   shared iterator
+ *		sprefetch_iterator shared iterator for prefetching
+ *		pstate			   shared state for parallel bitmap scan
+ *----------------
  */
 typedef struct BitmapHeapScanState
 {
@@ -1491,6 +1551,11 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	bool				 inited;
+	TBMSharedIterator	*stbmiterator;
+	TBMSharedIterator	*sprefetch_iterator;
+	ParallelBitmapState *pstate;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 53cad24..d22b039 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -53,7 +53,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index ebda308..3ec2d9d 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -56,6 +56,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
 extern int compute_parallel_worker(RelOptInfo *rel, BlockNumber heap_pages,
 						BlockNumber index_pages);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+										Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 8b710ec..e31389f 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -787,6 +787,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#99Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#98)
Re: Parallel bitmap heap scan

On Tue, Feb 21, 2017 at 10:20 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

0001:
1. I have got a new interface, "tbm_free_shared_area(dsa_area *dsa,
dsa_pointer dp)" which will be responsible for freeing all the shared
members (pagetable, ptpage and ptchunk). Actually, we can not do this
in tbm_free itself because the only leader will have a local tbm with
reference to all these pointers and our parallel bitmap leader may not
necessarily be the actual backend.

I'm not entirely sure about the calling convention for
tbm_free_shared_area() but the rest seems OK.

2. Now tbm_free is not freeing any of the shared members which can be
accessed by other worker so tbm_free is safe to call from
ExecEndBitmapHeapScan without any safety check or ref count.

That also seems fine. We ended up with something very similar in the
Parallel Index Scan patch.

0002:
1. We don't need ExecShutdownBitmapHeapScan anymore because now we are
not freeing any shared member in ExecEndBitmapHeapScan.
2. In ExecReScanBitmapHeapScan we will call tbm_free_shared_area to
free the shared members of the TBM.
3. After that, we will free TBMSharedIteratorState what we allocated
using tbm_prepare_shared_iterate.

Check. But I think tbm_free_shared_area() should also free the object
itself, instead of making the caller do that separately.

+        if (DsaPointerIsValid(node->pstate->tbmiterator))
+        {
+            /* First we free the shared TBM members using the shared state */
+            tbm_free_shared_area(dsa, node->pstate->tbmiterator);
+            dsa_free(dsa, node->pstate->tbmiterator);
+        }
+
+        if (DsaPointerIsValid(node->pstate->prefetch_iterator))
+            dsa_free(dsa, node->pstate->prefetch_iterator);

The fact that these cases aren't symmetric suggests that your
abstraction is leaky. I'm guessing that you can't call
tbm_free_shared_area because the two iterators share one copy of the
underlying iteration arrays, and the TBM code isn't smart enough to
avoid freeing them twice. You're going to have to come up with a
better solution to that problem; nodeBitmapHeapScan.c shouldn't know
about the way the underlying storage details are managed. (Maybe you
need to reference-count the iterator arrays?)

+ if (node->inited)
+ goto start_iterate;

My first programming teacher told me not to use goto. I've
occasionally violated that rule, but I need a better reason than you
have here. It looks very easy to avoid.

+ pbms_set_parallel(outerPlanState(node));

I think this should be a flag in the plan, and the planner should set
it correctly, instead of having it be a flag in the executor that the
executor sets. Also, the flag itself should really be called
something that involves the word "shared" rather than "parallel",
because the bitmap will not be created in parallel, but it will be
shared.

Have you checked whether this patch causes any regression in the
non-parallel case? It adds a bunch of "if" statements, so it might.
Hopefully not, but it needs to be carefully tested.

@@ -48,10 +48,11 @@
#include "utils/snapmgr.h"
#include "utils/tqual.h"

-
static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);

Unnecessary.

+static bool pbms_is_leader(ParallelBitmapState *pbminfo);
+static void pbms_set_parallel(PlanState *node);

I don't think this "pbms" terminology is very good. It's dissimilar
to the way other functions in this file are named. Maybe
BitmapShouldInitializeSharedState().

I think that some of the bits that this function makes conditional on
pstate should be factored out into inline functions. Like:

-            if (node->prefetch_target >= node->prefetch_maximum)
-                 /* don't increase any further */ ;
-            else if (node->prefetch_target >= node->prefetch_maximum / 2)
-                node->prefetch_target = node->prefetch_maximum;
-            else if (node->prefetch_target > 0)
-                node->prefetch_target *= 2;
-            else
-                node->prefetch_target++;
+            if (!pstate)
+            {
+                if (node->prefetch_target >= node->prefetch_maximum)
+                     /* don't increase any further */ ;
+                else if (node->prefetch_target >= node->prefetch_maximum / 2)
+                    node->prefetch_target = node->prefetch_maximum;
+                else if (node->prefetch_target > 0)
+                    node->prefetch_target *= 2;
+                else
+                    node->prefetch_target++;
+            }
+            else if (pstate->prefetch_target < node->prefetch_maximum)
+            {
+                SpinLockAcquire(&pstate->prefetch_mutex);
+                if (pstate->prefetch_target >= node->prefetch_maximum)
+                     /* don't increase any further */ ;
+                else if (pstate->prefetch_target >=
+                         node->prefetch_maximum / 2)
+                    pstate->prefetch_target = node->prefetch_maximum;
+                else if (pstate->prefetch_target > 0)
+                    pstate->prefetch_target *= 2;
+                else
+                    pstate->prefetch_target++;
+                SpinLockRelease(&pstate->prefetch_mutex);
+            }

I suggest creating an inline function like BitmapAdjustPrefetch() for
this logic, and letting the code call that. The function can look
something like this: if (pstate == NULL) { non-parallel stuff; return;
} parallel stuff follows...

And similarly for the other cases where you've made the logic
conditional. This will make it more clear what's happening
post-patch, I think, and will also help keep the level of indentation
from getting out-of-control in certain places. In fact, maybe you
should submit a preliminary refactoring patch that moves these chunks
of logic into functions and then the main patch can apply over top of
that.

+ bool inited;

Suggest: initialized

- * ----------------
+ *        pscan_len           size of the shared memory for parallel bitmap
+ *        inited               is node is ready to iterate
+ *        stbmiterator       shared iterator
+ *        sprefetch_iterator shared iterator for prefetching
+ *        pstate               shared state for parallel bitmap scan
+ *----------------

No need to change number of dashes.

+ *     PBMState information : Current status of the TIDBitmap creation during
+ *                            parallel bitmap heap scan.

If you look for existing places where comments are formatted like
this, I bet you won't find many. Copy the surrounding style more.

+    dsa_pointer    tbmiterator;
+    dsa_pointer    prefetch_iterator;
+    slock_t        prefetch_mutex;
+    int            prefetch_pages;
+    int            prefetch_target;
+    bool        prefetching;
+    slock_t        state_mutex;
+    PBMState    state;
+    ConditionVariable cv;
+    char        phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];

I think it is probably not a good idea to have two separate mutexes
here. They'll be in the same cache line, so it won't be much faster
than having one mutex, and the state mutex won't be acquired very
often so you won't really gain anything anyway. I think you can just
merge the mutexes into one called 'mutex'.

+ /* Allow only one process to prefetch */

If this is a good idea, there should be a comment explaining why.

pbms_is_leader() looks horrifically inefficient. Every worker will
reacquire the spinlock for every tuple. You should only have to enter
this spinlock-acquiring loop for the first tuple. After that, either
you became the leader, did the initialization, and set the state to
PBM_FINISHED, or you waited until the pre-existing leader did the
same. You should have a backend-local flag that keeps you from
touching the spinlock for every tuple. I wouldn't be surprised if
fixing this nets a noticeable performance increase for this patch at
high worker counts.

+    TBMSharedIterator    *stbmiterator;
+    TBMSharedIterator    *sprefetch_iterator;

Maybe shared_iterator and shared_prefetch_iterator.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#100Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#99)
Re: Parallel bitmap heap scan

On Sun, Feb 26, 2017 at 9:26 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Thanks for the review, I will work on these. There are some comments I
need suggestions.

tbm_free_shared_area because the two iterators share one copy of the
underlying iteration arrays, and the TBM code isn't smart enough to
avoid freeing them twice. You're going to have to come up with a
better solution to that problem; nodeBitmapHeapScan.c shouldn't know
about the way the underlying storage details are managed. (Maybe you
need to reference-count the iterator arrays?)

Yeah, I also think current way doesn't look so clean, currently, these
arrays are just integers array, may be we can use a first slot of the
array for reference-count? or convert to the structure which has space
for reference-count and an integers array. What do you suggest?

+ if (node->inited)
+ goto start_iterate;

My first programming teacher told me not to use goto. I've
occasionally violated that rule, but I need a better reason than you
have here. It looks very easy to avoid.

Yes, this can be avoided, I was just trying to get rid of multi-level
if nesting and end up with the "goto".

+ pbms_set_parallel(outerPlanState(node));

I think this should be a flag in the plan, and the planner should set
it correctly, instead of having it be a flag in the executor that the
executor sets. Also, the flag itself should really be called
something that involves the word "shared" rather than "parallel",
because the bitmap will not be created in parallel, but it will be
shared.

Earlier, I thought that it will be not a good idea to set that flag in
BitmapIndexScan path because the same path can be used either under
ParallelBitmapHeapPath or under normal BitmapHeapPath. But, now after
putting some thought, I realised that we can do that in create_plan.
Therefore, I will change this.

Have you checked whether this patch causes any regression in the
non-parallel case? It adds a bunch of "if" statements, so it might.
Hopefully not, but it needs to be carefully tested.

During the initial patch development, I have tested this aspect also
but never published any results for the same. I will do that along
with my next patch and post the results.

pbms_is_leader() looks horrifically inefficient. Every worker will
reacquire the spinlock for every tuple. You should only have to enter
this spinlock-acquiring loop for the first tuple. After that, either
you became the leader, did the initialization, and set the state to
PBM_FINISHED, or you waited until the pre-existing leader did the
same. You should have a backend-local flag that keeps you from
touching the spinlock for every tuple. I wouldn't be surprised if
fixing this nets a noticeable performance increase for this patch at
high worker counts.

I think there is some confusion, if you notice, below code will avoid
calling pbms_is_leader for every tuple.
It will be called only first time. And, after that node->inited will
be true and it will directly jump to start_iterate for subsequent
calls. Am I missing something?

+ if (node->inited)
+ goto start_iterate;

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#101Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#100)
Re: Parallel bitmap heap scan

On Mon, Feb 27, 2017 at 10:30 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Sun, Feb 26, 2017 at 9:26 PM, Robert Haas <robertmhaas@gmail.com> wrote:

tbm_free_shared_area because the two iterators share one copy of the
underlying iteration arrays, and the TBM code isn't smart enough to
avoid freeing them twice. You're going to have to come up with a
better solution to that problem; nodeBitmapHeapScan.c shouldn't know
about the way the underlying storage details are managed. (Maybe you
need to reference-count the iterator arrays?)

Yeah, I also think current way doesn't look so clean, currently, these
arrays are just integers array, may be we can use a first slot of the
array for reference-count? or convert to the structure which has space
for reference-count and an integers array. What do you suggest?

Maybe something like typedef struct { int refcnt; SomeType
somename[FLEXIBLE_ARRAY_MEMBER]; } SomeOtherType; would be a good
approach.

+ if (node->inited)
+ goto start_iterate;

My first programming teacher told me not to use goto. I've
occasionally violated that rule, but I need a better reason than you
have here. It looks very easy to avoid.

Yes, this can be avoided, I was just trying to get rid of multi-level
if nesting and end up with the "goto".

That's what I figured.

+ pbms_set_parallel(outerPlanState(node));

I think this should be a flag in the plan, and the planner should set
it correctly, instead of having it be a flag in the executor that the
executor sets. Also, the flag itself should really be called
something that involves the word "shared" rather than "parallel",
because the bitmap will not be created in parallel, but it will be
shared.

Earlier, I thought that it will be not a good idea to set that flag in
BitmapIndexScan path because the same path can be used either under
ParallelBitmapHeapPath or under normal BitmapHeapPath. But, now after
putting some thought, I realised that we can do that in create_plan.
Therefore, I will change this.

Cool.

pbms_is_leader() looks horrifically inefficient. Every worker will
reacquire the spinlock for every tuple. You should only have to enter
this spinlock-acquiring loop for the first tuple. After that, either
you became the leader, did the initialization, and set the state to
PBM_FINISHED, or you waited until the pre-existing leader did the
same. You should have a backend-local flag that keeps you from
touching the spinlock for every tuple. I wouldn't be surprised if
fixing this nets a noticeable performance increase for this patch at
high worker counts.

I think there is some confusion, if you notice, below code will avoid
calling pbms_is_leader for every tuple.
It will be called only first time. And, after that node->inited will
be true and it will directly jump to start_iterate for subsequent
calls. Am I missing something?

+ if (node->inited)
+ goto start_iterate;

Oh, OK. I guess I was just confused, then.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#102Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#99)
4 attachment(s)
Re: Parallel bitmap heap scan

On Sun, Feb 26, 2017 at 9:26 PM, Robert Haas <robertmhaas@gmail.com> wrote:

I'm not entirely sure about the calling convention for
tbm_free_shared_area() but the rest seems OK.

Changed

2. Now tbm_free is not freeing any of the shared members which can be
accessed by other worker so tbm_free is safe to call from
ExecEndBitmapHeapScan without any safety check or ref count.

That also seems fine. We ended up with something very similar in the
Parallel Index Scan patch.

0002:
1. We don't need ExecShutdownBitmapHeapScan anymore because now we are
not freeing any shared member in ExecEndBitmapHeapScan.
2. In ExecReScanBitmapHeapScan we will call tbm_free_shared_area to
free the shared members of the TBM.
3. After that, we will free TBMSharedIteratorState what we allocated
using tbm_prepare_shared_iterate.

Check. But I think tbm_free_shared_area() should also free the object
itself, instead of making the caller do that separately.

Right, done that way.

+        if (DsaPointerIsValid(node->pstate->tbmiterator))
+        {
+            /* First we free the shared TBM members using the shared state */
+            tbm_free_shared_area(dsa, node->pstate->tbmiterator);
+            dsa_free(dsa, node->pstate->tbmiterator);
+        }
+
+        if (DsaPointerIsValid(node->pstate->prefetch_iterator))
+            dsa_free(dsa, node->pstate->prefetch_iterator);

The fact that these cases aren't symmetric suggests that your
abstraction is leaky. I'm guessing that you can't call
tbm_free_shared_area because the two iterators share one copy of the
underlying iteration arrays, and the TBM code isn't smart enough to
avoid freeing them twice. You're going to have to come up with a
better solution to that problem; nodeBitmapHeapScan.c shouldn't know
about the way the underlying storage details are managed. (Maybe you
need to reference-count the iterator arrays?)

Converted iterator arrays to structure and maintained the refcount. I
had to do the same thing for pagetable also because that is also
shared across iterator.

+ if (node->inited)
+ goto start_iterate;

My first programming teacher told me not to use goto. I've
occasionally violated that rule, but I need a better reason than you
have here. It looks very easy to avoid.

Changed

+ pbms_set_parallel(outerPlanState(node));

I think this should be a flag in the plan, and the planner should set
it correctly, instead of having it be a flag in the executor that the
executor sets. Also, the flag itself should really be called
something that involves the word "shared" rather than "parallel",
because the bitmap will not be created in parallel, but it will be
shared.

Done

Have you checked whether this patch causes any regression in the
non-parallel case? It adds a bunch of "if" statements, so it might.
Hopefully not, but it needs to be carefully tested.

With new patch I tested in my local machine, perform multiple
executions and it doesn't show any regression. Attached file
perf_result contains the explain analyze output for one of the query
(head, patch with 0 workers and patch with 2 workers). I will perform
the test with all the TPC-H queries which using bitmap plan on the
bigger machine and will post the results next week.

@@ -48,10 +48,11 @@
#include "utils/snapmgr.h"
#include "utils/tqual.h"

-
static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);

Unnecessary.

Fixed

+static bool pbms_is_leader(ParallelBitmapState *pbminfo);
+static void pbms_set_parallel(PlanState *node);

I don't think this "pbms" terminology is very good. It's dissimilar
to the way other functions in this file are named. Maybe
BitmapShouldInitializeSharedState().

Changed

I think that some of the bits that this function makes conditional on
pstate should be factored out into inline functions. Like:

-            if (node->prefetch_target >= node->prefetch_maximum)
-                 /* don't increase any further */ ;
-            else if (node->prefetch_target >= node->prefetch_maximum / 2)
-                node->prefetch_target = node->prefetch_maximum;
-            else if (node->prefetch_target > 0)
-                node->prefetch_target *= 2;
-            else
-                node->prefetch_target++;
+            if (!pstate)
+            {
+                if (node->prefetch_target >= node->prefetch_maximum)
+                     /* don't increase any further */ ;
+                else if (node->prefetch_target >= node->prefetch_maximum / 2)
+                    node->prefetch_target = node->prefetch_maximum;
+                else if (node->prefetch_target > 0)
+                    node->prefetch_target *= 2;
+                else
+                    node->prefetch_target++;
+            }
+            else if (pstate->prefetch_target < node->prefetch_maximum)
+            {
+                SpinLockAcquire(&pstate->prefetch_mutex);
+                if (pstate->prefetch_target >= node->prefetch_maximum)
+                     /* don't increase any further */ ;
+                else if (pstate->prefetch_target >=
+                         node->prefetch_maximum / 2)
+                    pstate->prefetch_target = node->prefetch_maximum;
+                else if (pstate->prefetch_target > 0)
+                    pstate->prefetch_target *= 2;
+                else
+                    pstate->prefetch_target++;
+                SpinLockRelease(&pstate->prefetch_mutex);
+            }

I suggest creating an inline function like BitmapAdjustPrefetch() for
this logic, and letting the code call that. The function can look
something like this: if (pstate == NULL) { non-parallel stuff; return;
} parallel stuff follows...

And similarly for the other cases where you've made the logic
conditional. This will make it more clear what's happening
post-patch, I think, and will also help keep the level of indentation
from getting out-of-control in certain places. In fact, maybe you
should submit a preliminary refactoring patch that moves these chunks
of logic into functions and then the main patch can apply over top of
that.

Done.

+ bool inited;

Suggest: initialized

Done

- * ----------------
+ *        pscan_len           size of the shared memory for parallel bitmap
+ *        inited               is node is ready to iterate
+ *        stbmiterator       shared iterator
+ *        sprefetch_iterator shared iterator for prefetching
+ *        pstate               shared state for parallel bitmap scan
+ *----------------

No need to change number of dashes.

Fixed

+ *     PBMState information : Current status of the TIDBitmap creation during
+ *                            parallel bitmap heap scan.

If you look for existing places where comments are formatted like
this, I bet you won't find many. Copy the surrounding style more.

Done as surrounding structure, also changed the name to
SharedBitmapState, I think that is better name for the purpose.

+    dsa_pointer    tbmiterator;
+    dsa_pointer    prefetch_iterator;
+    slock_t        prefetch_mutex;
+    int            prefetch_pages;
+    int            prefetch_target;
+    bool        prefetching;
+    slock_t        state_mutex;
+    PBMState    state;
+    ConditionVariable cv;
+    char        phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];

I think it is probably not a good idea to have two separate mutexes
here. They'll be in the same cache line, so it won't be much faster
than having one mutex, and the state mutex won't be acquired very
often so you won't really gain anything anyway. I think you can just
merge the mutexes into one called 'mutex'.

Done.

+ /* Allow only one process to prefetch */

If this is a good idea, there should be a comment explaining why.

Done

+    TBMSharedIterator    *stbmiterator;
+    TBMSharedIterator    *sprefetch_iterator;

Maybe shared_iterator and shared_prefetch_iterator.

Done

0001- same as previous with some changes for freeing the shared memory stuff.
0002- nodeBitmapHeapScan refactoring, this applies independently
0003- actual parallel bitmap stuff applies on top of 0001 and 0002

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0001-tidbitmap-support-shared-v6.patchapplication/octet-stream; name=0001-tidbitmap-support-shared-v6.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index fad5cb0..1d70fae 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1082,6 +1082,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel query dynamic shared memory allocation lock.</entry>
         </row>
         <row>
+         <entry><literal>tbm</></entry>
+         <entry>Waiting for TBM shared iterator lock.</entry>
+        </row>
+        <row>
          <entry morerows="9"><literal>Lock</></entry>
          <entry><literal>relation</></entry>
          <entry>Waiting to acquire a lock on a relation.</entry>
diff --git a/src/backend/access/gin/ginget.c b/src/backend/access/gin/ginget.c
index 60f005c..87cd9ea 100644
--- a/src/backend/access/gin/ginget.c
+++ b/src/backend/access/gin/ginget.c
@@ -120,7 +120,7 @@ collectMatchBitmap(GinBtreeData *btree, GinBtreeStack *stack,
 	Form_pg_attribute attr;
 
 	/* Initialize empty bitmap result */
-	scanEntry->matchBitmap = tbm_create(work_mem * 1024L);
+	scanEntry->matchBitmap = tbm_create(work_mem * 1024L, NULL);
 
 	/* Null query cannot partial-match anything */
 	if (scanEntry->isPartialMatch &&
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..94bb289 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,7 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L);
+		tbm = tbm_create(work_mem * 1024L, NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 3c6155b..1d280be 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,7 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L);
+				result = tbm_create(work_mem * 1024L, NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..a1b2a45 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -43,6 +43,8 @@
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
 #include "nodes/tidbitmap.h"
+#include "storage/lwlock.h"
+#include "utils/dsa.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -103,6 +105,15 @@ typedef struct PagetableEntry
 } PagetableEntry;
 
 /*
+ * Holds array of pagetable entries.
+ */
+typedef struct PTEntryArray
+{
+	int			refcount;
+	PagetableEntry ptentry[FLEXIBLE_ARRAY_MEMBER];
+} PTEntryArray;
+
+/*
  * We want to avoid the overhead of creating the hashtable, which is
  * comparatively large, when not necessary. Particularly when we are using a
  * bitmap scan on the inside of a nestloop join: a bitmap may well live only
@@ -121,6 +132,16 @@ typedef enum
 } TBMStatus;
 
 /*
+ * Current iterating state of the TBM.
+ */
+typedef enum
+{
+	TBM_NOT_ITERATING,			/* not yet converted to page and chunk array */
+	TBM_ITERATING_PRIVATE,		/* converted to local page and chunk array */
+	TBM_ITERATING_SHARED		/* converted to shared page and chunk array */
+} TBMIteratingState;
+
+/*
  * Here is the representation for a whole TIDBitMap:
  */
 struct TIDBitmap
@@ -133,12 +154,17 @@ struct TIDBitmap
 	int			maxentries;		/* limit on same to meet maxbytes */
 	int			npages;			/* number of exact entries in pagetable */
 	int			nchunks;		/* number of lossy entries in pagetable */
-	bool		iterating;		/* tbm_begin_iterate called? */
+	TBMIteratingState iterating;	/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	dsa_pointer dsapagetable;	/* dsa_pointer to the element array */
+	dsa_pointer dsapagetableold;	/* dsa_pointer to the old element array */
+	dsa_pointer ptpages;		/* dsa_pointer to the page array */
+	dsa_pointer ptchunks;		/* dsa_pointer to the chunk array */
+	dsa_area   *dsa;			/* reference to per-query dsa area */
 };
 
 /*
@@ -156,6 +182,46 @@ struct TBMIterator
 	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
 };
 
+/*
+ * Holds the shared members of the iterator so that multiple processes
+ * can jointly iterate.
+ */
+typedef struct TBMSharedIteratorState
+{
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	dsa_pointer pagetable;		/* dsa pointers to head of pagetable data */
+	dsa_pointer spages;			/* dsa pointer to page array */
+	dsa_pointer schunks;		/* dsa pointer to chunk array */
+	LWLock		lock;			/* lock to protect below members */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+} TBMSharedIteratorState;
+
+/*
+ * pagetable iteration array.
+ */
+typedef struct PTIterationArray
+{
+	int			refcount;		/* no. of shared iterator referring */
+	int			index[FLEXIBLE_ARRAY_MEMBER];	/* index array */
+} PTIterationArray;
+
+/*
+ * same as TBMIterator, but it is used for joint iteration, therefore this
+ * also holds a reference to the shared state.
+ */
+struct TBMSharedIterator
+{
+	TBMSharedIteratorState *state;		/* shared state */
+	PTEntryArray *ptbase;		/* pagetable element array */
+	PTIterationArray *ptpages;	/* sorted exact page index list */
+	PTIterationArray *ptchunks; /* sorted lossy page index list */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+};
 
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
@@ -168,6 +234,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+static int tbm_shared_comparator(const void *left, const void *right,
+					  void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -187,6 +255,7 @@ hash_blockno(BlockNumber b)
 }
 
 /* define hashtable mapping block numbers to PagetableEntry's */
+#define SH_USE_NONDEFAULT_ALLOCATOR
 #define SH_PREFIX pagetable
 #define SH_ELEMENT_TYPE PagetableEntry
 #define SH_KEY_TYPE BlockNumber
@@ -204,10 +273,12 @@ hash_blockno(BlockNumber b)
  *
  * The bitmap will live in the memory context that is CurrentMemoryContext
  * at the time of this call.  It will be limited to (approximately) maxbytes
- * total memory consumption.
+ * total memory consumption.  If the DSA passed to this function is not NULL
+ * then the memory for storing elements of the underlying page table will
+ * be allocated from the DSA.
  */
 TIDBitmap *
-tbm_create(long maxbytes)
+tbm_create(long maxbytes, dsa_area *dsa)
 {
 	TIDBitmap  *tbm;
 	long		nbuckets;
@@ -230,6 +301,7 @@ tbm_create(long maxbytes)
 	nbuckets = Max(nbuckets, 16);		/* sanity limit */
 	tbm->maxentries = (int) nbuckets;
 	tbm->lossify_start = 0;
+	tbm->dsa = dsa;
 
 	return tbm;
 }
@@ -244,7 +316,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, tbm);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -281,6 +353,40 @@ tbm_free(TIDBitmap *tbm)
 }
 
 /*
+ * tbm_free_shared_area - free shared state
+ *
+ * Free shared iterator state, Also free shared pagetable and iterator arrays
+ * memory if they are not referred by any of the shared iterator i.e recount
+ * is becomes 0.
+ */
+void
+tbm_free_shared_area(dsa_area *dsa, dsa_pointer dp)
+{
+	TBMSharedIteratorState *istate = dsa_get_address(dsa, dp);
+	PTEntryArray *ptbase = dsa_get_address(dsa, istate->pagetable);
+	PTIterationArray *ptpages;
+	PTIterationArray *ptchunks;
+
+	if (--ptbase->refcount == 0)
+		dsa_free(dsa, istate->pagetable);
+
+	if (istate->spages)
+	{
+		ptpages = dsa_get_address(dsa, istate->spages);
+		if (--ptpages->refcount == 0)
+			dsa_free(dsa, istate->spages);
+	}
+	if (istate->schunks)
+	{
+		ptchunks = dsa_get_address(dsa, istate->schunks);
+		if (--ptchunks->refcount == 0)
+			dsa_free(dsa, istate->schunks);
+	}
+
+	dsa_free(dsa, dp);
+}
+
+/*
  * tbm_add_tuples - add some tuple IDs to a TIDBitmap
  *
  * If recheck is true, then the recheck flag will be set in the
@@ -294,7 +400,7 @@ tbm_add_tuples(TIDBitmap *tbm, const ItemPointer tids, int ntids,
 	PagetableEntry *page = NULL;	/* only valid when currblk is valid */
 	int			i;
 
-	Assert(!tbm->iterating);
+	Assert(tbm->iterating == TBM_NOT_ITERATING);
 	for (i = 0; i < ntids; i++)
 	{
 		BlockNumber blk = ItemPointerGetBlockNumber(tids + i);
@@ -603,6 +709,8 @@ tbm_begin_iterate(TIDBitmap *tbm)
 {
 	TBMIterator *iterator;
 
+	Assert(tbm->iterating != TBM_ITERATING_SHARED);
+
 	/*
 	 * Create the TBMIterator struct, with enough trailing space to serve the
 	 * needs of the TBMIterateResult sub-struct.
@@ -624,7 +732,7 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	 * attached to the bitmap not the iterator, so they can be used by more
 	 * than one iterator.
 	 */
-	if (tbm->status == TBM_HASH && !tbm->iterating)
+	if (tbm->status == TBM_HASH && tbm->iterating == TBM_NOT_ITERATING)
 	{
 		pagetable_iterator i;
 		PagetableEntry *page;
@@ -659,12 +767,192 @@ tbm_begin_iterate(TIDBitmap *tbm)
 				  tbm_comparator);
 	}
 
-	tbm->iterating = true;
+	tbm->iterating = TBM_ITERATING_PRIVATE;
 
 	return iterator;
 }
 
 /*
+ * tbm_prepare_shared_iterate - prepare shared iteration state for a TIDBitmap.
+ *
+ * The necessary shared state will be allocated from the DSA passed to
+ * tbm_create, so that multiple processes can attach to it and iterate jointly.
+ *
+ * This will convert the pagetable hash into page and chunk array of the index
+ * into pagetable array.
+ */
+dsa_pointer
+tbm_prepare_shared_iterate(TIDBitmap *tbm)
+{
+	dsa_pointer dp;
+	TBMSharedIteratorState *istate;
+
+	Assert(tbm->dsa != NULL);
+	Assert(tbm->iterating != TBM_ITERATING_PRIVATE);
+
+	/*
+	 * Allocate TBMSharedIteratorState from DSA to hold the shared members and
+	 * lock, this will also be used by multiple worker for shared iterate.
+	 */
+	dp = dsa_allocate(tbm->dsa, sizeof(TBMSharedIteratorState));
+	istate = dsa_get_address(tbm->dsa, dp);
+
+	/*
+	 * If we're not already iterating, create and fill the sorted page lists.
+	 * (If we are, the sorted page lists are already stored in the TIDBitmap,
+	 * and we can just reuse them.)
+	 */
+	if (tbm->iterating == TBM_NOT_ITERATING)
+	{
+		pagetable_iterator i;
+		PagetableEntry *page;
+		PTEntryArray *ptbase;
+		PTIterationArray *ptpages;
+		PTIterationArray *ptchunks;
+		int			idx;
+		int			npages;
+		int			nchunks;
+
+		/*
+		 * Allocate the page and chunk array memory from the DSA to share
+		 * across multiple processes.
+		 */
+		if (tbm->npages)
+		{
+			tbm->ptpages = dsa_allocate(tbm->dsa, sizeof(PTIterationArray) +
+										tbm->npages * sizeof(int));
+			ptpages = dsa_get_address(tbm->dsa, tbm->ptpages);
+		}
+		if (tbm->nchunks)
+		{
+			tbm->ptchunks = dsa_allocate(tbm->dsa, sizeof(PTIterationArray) +
+										 tbm->nchunks * sizeof(int));
+			ptchunks = dsa_get_address(tbm->dsa, tbm->ptchunks);
+		}
+
+		/*
+		 * If TBM status is TBM_HASH then iterate over the pagetable and
+		 * convert it to page and chunk arrays.  But if it's in the
+		 * TBM_ONE_PAGE mode then directly allocate the space for one entry
+		 * from the DSA.
+		 */
+		npages = nchunks = 0;
+		if (tbm->status == TBM_HASH)
+		{
+			ptbase = dsa_get_address(tbm->dsa, tbm->dsapagetable);
+
+			pagetable_start_iterate(tbm->pagetable, &i);
+			while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+			{
+				idx = page - ptbase->ptentry;
+				if (page->ischunk)
+					ptchunks->index[nchunks++] = idx;
+				else
+					ptpages->index[npages++] = idx;
+			}
+
+			Assert(npages == tbm->npages);
+			Assert(nchunks == tbm->nchunks);
+		}
+		else
+		{
+			/*
+			 * In one page mode allocate the space for one pagetable entry and
+			 * directly store its index i.e. 0 in page array
+			 */
+			tbm->dsapagetable = dsa_allocate(tbm->dsa, sizeof(PTEntryArray) +
+											 sizeof(PagetableEntry));
+			ptbase = dsa_get_address(tbm->dsa, tbm->dsapagetable);
+			ptpages->index[0] = 0;
+		}
+
+		if (npages > 1)
+			qsort_arg((void *) (ptpages->index), npages, sizeof(int),
+					  tbm_shared_comparator, (void *) ptbase->ptentry);
+		if (nchunks > 1)
+			qsort_arg((void *) (ptchunks->index), nchunks, sizeof(int),
+					  tbm_shared_comparator, (void *) ptbase->ptentry);
+	}
+
+	/*
+	 * Store the TBM members in the shared state so that we can share them
+	 * across multiple processes.
+	 */
+	istate->nentries = tbm->nentries;
+	istate->maxentries = tbm->maxentries;
+	istate->npages = tbm->npages;
+	istate->nchunks = tbm->nchunks;
+	istate->pagetable = tbm->dsapagetable;
+	istate->spages = tbm->ptpages;
+	istate->schunks = tbm->ptchunks;
+
+	/* Initialize the iterator lock */
+	LWLockInitialize(&istate->lock, LWTRANCHE_TBM);
+
+	/* Initialize the shared iterator state */
+	istate->schunkbit = 0;
+	istate->schunkptr = 0;
+	istate->spageptr = 0;
+
+	tbm->iterating = TBM_ITERATING_SHARED;
+
+	return dp;
+}
+
+/*
+ * tbm_extract_page_tuple - extract the tuple offsets from a page
+ *
+ * The extracted offsets are stored into TBMIterateResult.
+ */
+static inline int
+tbm_extract_page_tuple(PagetableEntry *page, TBMIterateResult *output)
+{
+	int			wordnum;
+	int			ntuples = 0;
+
+	for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
+	{
+		bitmapword	w = page->words[wordnum];
+
+		if (w != 0)
+		{
+			int			off = wordnum * BITS_PER_BITMAPWORD + 1;
+
+			while (w != 0)
+			{
+				if (w & 1)
+					output->offsets[ntuples++] = (OffsetNumber) off;
+				off++;
+				w >>= 1;
+			}
+		}
+	}
+
+	return ntuples;
+}
+
+/*
+ *	tbm_advance_schunkbit - Advance the chunkbit
+ */
+static inline void
+tbm_advance_schunkbit(PagetableEntry *chunk, int *schunkbitp)
+{
+	int			schunkbit = *schunkbitp;
+
+	while (schunkbit < PAGES_PER_CHUNK)
+	{
+		int			wordnum = WORDNUM(schunkbit);
+		int			bitnum = BITNUM(schunkbit);
+
+		if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
+			break;
+		schunkbit++;
+	}
+
+	*schunkbitp = schunkbit;
+}
+
+/*
  * tbm_iterate - scan through next page of a TIDBitmap
  *
  * Returns a TBMIterateResult representing one page, or NULL if there are
@@ -682,7 +970,7 @@ tbm_iterate(TBMIterator *iterator)
 	TIDBitmap  *tbm = iterator->tbm;
 	TBMIterateResult *output = &(iterator->output);
 
-	Assert(tbm->iterating);
+	Assert(tbm->iterating == TBM_ITERATING_PRIVATE);
 
 	/*
 	 * If lossy chunk pages remain, make sure we've advanced schunkptr/
@@ -693,15 +981,7 @@ tbm_iterate(TBMIterator *iterator)
 		PagetableEntry *chunk = tbm->schunks[iterator->schunkptr];
 		int			schunkbit = iterator->schunkbit;
 
-		while (schunkbit < PAGES_PER_CHUNK)
-		{
-			int			wordnum = WORDNUM(schunkbit);
-			int			bitnum = BITNUM(schunkbit);
-
-			if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
-				break;
-			schunkbit++;
-		}
+		tbm_advance_schunkbit(chunk, &schunkbit);
 		if (schunkbit < PAGES_PER_CHUNK)
 		{
 			iterator->schunkbit = schunkbit;
@@ -738,7 +1018,6 @@ tbm_iterate(TBMIterator *iterator)
 	{
 		PagetableEntry *page;
 		int			ntuples;
-		int			wordnum;
 
 		/* In ONE_PAGE state, we don't allocate an spages[] array */
 		if (tbm->status == TBM_ONE_PAGE)
@@ -747,24 +1026,7 @@ tbm_iterate(TBMIterator *iterator)
 			page = tbm->spages[iterator->spageptr];
 
 		/* scan bitmap to extract individual offset numbers */
-		ntuples = 0;
-		for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
-		{
-			bitmapword	w = page->words[wordnum];
-
-			if (w != 0)
-			{
-				int			off = wordnum * BITS_PER_BITMAPWORD + 1;
-
-				while (w != 0)
-				{
-					if (w & 1)
-						output->offsets[ntuples++] = (OffsetNumber) off;
-					off++;
-					w >>= 1;
-				}
-			}
-		}
+		ntuples = tbm_extract_page_tuple(page, output);
 		output->blockno = page->blockno;
 		output->ntuples = ntuples;
 		output->recheck = page->recheck;
@@ -777,6 +1039,94 @@ tbm_iterate(TBMIterator *iterator)
 }
 
 /*
+ *	tbm_shared_iterate - scan through next page of a TIDBitmap
+ *
+ *	As above, but this will iterate using an iterator which is shared
+ *	across multiple processes.  We need to acquire the iterator LWLock,
+ *	before accessing the shared members.
+ */
+TBMIterateResult *
+tbm_shared_iterate(TBMSharedIterator *iterator)
+{
+	TBMIterateResult *output = &iterator->output;
+	TBMSharedIteratorState *istate = iterator->state;
+	PagetableEntry *ptbase = iterator->ptbase->ptentry;
+	int		   *idxpages = iterator->ptpages->index;
+	int		   *idxchunks = iterator->ptchunks->index;
+
+	/* Acquire the LWLock before accessing the shared members */
+	LWLockAcquire(&istate->lock, LW_EXCLUSIVE);
+
+	/*
+	 * If lossy chunk pages remain, make sure we've advanced schunkptr/
+	 * schunkbit to the next set bit.
+	 */
+	while (istate->schunkptr < istate->nchunks)
+	{
+		PagetableEntry *chunk = &ptbase[idxchunks[istate->schunkptr]];
+		int			schunkbit = istate->schunkbit;
+
+		tbm_advance_schunkbit(chunk, &schunkbit);
+		if (schunkbit < PAGES_PER_CHUNK)
+		{
+			istate->schunkbit = schunkbit;
+			break;
+		}
+		/* advance to next chunk */
+		istate->schunkptr++;
+		istate->schunkbit = 0;
+	}
+
+	/*
+	 * If both chunk and per-page data remain, must output the numerically
+	 * earlier page.
+	 */
+	if (istate->schunkptr < istate->nchunks)
+	{
+		PagetableEntry *chunk = &ptbase[idxchunks[istate->schunkptr]];
+		PagetableEntry *page = &ptbase[idxpages[istate->spageptr]];
+		BlockNumber chunk_blockno;
+
+		chunk_blockno = chunk->blockno + istate->schunkbit;
+
+		if (istate->spageptr >= istate->npages ||
+			chunk_blockno < page->blockno)
+		{
+			/* Return a lossy page indicator from the chunk */
+			output->blockno = chunk_blockno;
+			output->ntuples = -1;
+			output->recheck = true;
+			istate->schunkbit++;
+
+			LWLockRelease(&istate->lock);
+			return output;
+		}
+	}
+
+	if (istate->spageptr < istate->npages)
+	{
+		PagetableEntry *page = &ptbase[idxpages[istate->spageptr]];
+		int			ntuples;
+
+		/* scan bitmap to extract individual offset numbers */
+		ntuples = tbm_extract_page_tuple(page, output);
+		output->blockno = page->blockno;
+		output->ntuples = ntuples;
+		output->recheck = page->recheck;
+		istate->spageptr++;
+
+		LWLockRelease(&istate->lock);
+
+		return output;
+	}
+
+	LWLockRelease(&istate->lock);
+
+	/* Nothing more in the bitmap */
+	return NULL;
+}
+
+/*
  * tbm_end_iterate - finish an iteration over a TIDBitmap
  *
  * Currently this is just a pfree, but it might do more someday.  (For
@@ -790,6 +1140,18 @@ tbm_end_iterate(TBMIterator *iterator)
 }
 
 /*
+ * tbm_end_shared_iterate - finish a shared iteration over a TIDBitmap
+ *
+ * This doesn't free any of the shared state associated with the iterator,
+ * just our backend-private state.
+ */
+void
+tbm_end_shared_iterate(TBMSharedIterator *iterator)
+{
+	pfree(iterator);
+}
+
+/*
  * tbm_find_pageentry - find a PagetableEntry for the pageno
  *
  * Returns NULL if there is no non-lossy entry for the pageno.
@@ -995,7 +1357,7 @@ tbm_lossify(TIDBitmap *tbm)
 	 * push nentries down to significantly less than maxentries, or else we'll
 	 * just end up doing this again very soon.  We shoot for maxentries/2.
 	 */
-	Assert(!tbm->iterating);
+	Assert(tbm->iterating == TBM_NOT_ITERATING);
 	Assert(tbm->status == TBM_HASH);
 
 	pagetable_start_iterate_at(tbm->pagetable, &i, tbm->lossify_start);
@@ -1061,3 +1423,111 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * As above, but this will get index into PagetableEntry array.  Therefore,
+ * it needs to get actual PagetableEntry using the index before comparing the
+ * blockno.
+ */
+static int
+tbm_shared_comparator(const void *left, const void *right, void *arg)
+{
+	PagetableEntry *base = (PagetableEntry *) arg;
+	PagetableEntry *lpage = &base[*(int *) left];
+	PagetableEntry *rpage = &base[*(int *) right];
+
+	if (lpage->blockno < rpage->blockno)
+		return -1;
+	else if (lpage->blockno > rpage->blockno)
+		return 1;
+	return 0;
+}
+
+/*
+ *	tbm_attach_shared_iterate
+ *
+ *	Allocate a backend-private iterator and attach the shared iterator state
+ *	to it so that multiple processed can iterate jointly.
+ *
+ *	We also converts the DSA pointers to local pointers and store them into
+ *	our private iterator.
+ */
+TBMSharedIterator *
+tbm_attach_shared_iterate(dsa_area *dsa, dsa_pointer dp)
+{
+	TBMSharedIterator *iterator;
+	TBMSharedIteratorState *istate;
+
+	/*
+	 * Create the TBMSharedIterator struct, with enough trailing space to
+	 * serve the needs of the TBMIterateResult sub-struct.
+	 */
+	iterator = (TBMSharedIterator *) palloc(sizeof(TBMSharedIterator) +
+								 MAX_TUPLES_PER_PAGE * sizeof(OffsetNumber));
+
+	istate = (TBMSharedIteratorState *) dsa_get_address(dsa, dp);
+
+	iterator->state = istate;
+
+	iterator->ptbase = dsa_get_address(dsa, istate->pagetable);
+	iterator->ptbase->refcount++;
+	if (istate->npages)
+	{
+		iterator->ptpages = dsa_get_address(dsa, istate->spages);
+		iterator->ptpages->refcount++;
+	}
+	if (istate->nchunks)
+	{
+		iterator->ptchunks = dsa_get_address(dsa, istate->schunks);
+		iterator->ptchunks->refcount++;
+	}
+
+	return iterator;
+}
+
+/*
+ * pagetable_allocate
+ *
+ * Callback function for allocating the memory for hashtable elements.
+ * Allocate memory for hashtable elements, using DSA if available.
+ */
+static inline void *
+pagetable_allocate(pagetable_hash *pagetable, Size size)
+{
+	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
+	PTEntryArray *ptbase;
+
+	if (tbm->dsa == NULL)
+		return MemoryContextAllocExtended(pagetable->ctx, size,
+										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+
+	/*
+	 * Save the dsapagetable reference in dsapagetableold before allocating
+	 * new memory so that pagetable_free can free the old entry.
+	 */
+	tbm->dsapagetableold = tbm->dsapagetable;
+	tbm->dsapagetable = dsa_allocate0(tbm->dsa, sizeof(PTEntryArray) + size);
+
+	ptbase = dsa_get_address(tbm->dsa, tbm->dsapagetable);
+	return ptbase->ptentry;
+}
+
+/*
+ * pagetable_free
+ *
+ * Callback function for freeing hash table elements.
+ */
+static inline void
+pagetable_free(pagetable_hash *pagetable, void *pointer)
+{
+	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
+
+	/* pfree the input pointer if DSA is not available */
+	if (tbm->dsa == NULL)
+		pfree(pointer);
+	else if (DsaPointerIsValid(tbm->dsapagetableold))
+	{
+		dsa_free(tbm->dsa, tbm->dsapagetableold);
+		tbm->dsapagetableold = InvalidDsaPointer;
+	}
+}
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index ab81d94..3e13394 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -510,6 +510,7 @@ RegisterLWLockTranches(void)
 						  "predicate_lock_manager");
 	LWLockRegisterTranche(LWTRANCHE_PARALLEL_QUERY_DSA,
 						  "parallel_query_dsa");
+	LWLockRegisterTranche(LWTRANCHE_TBM, "tbm");
 
 	/* Register named tranches. */
 	for (i = 0; i < NamedLWLockTrancheRequests; i++)
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..87f4bb7 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -23,6 +23,7 @@
 #define TIDBITMAP_H
 
 #include "storage/itemptr.h"
+#include "utils/dsa.h"
 
 
 /*
@@ -33,6 +34,7 @@ typedef struct TIDBitmap TIDBitmap;
 
 /* Likewise, TBMIterator is private */
 typedef struct TBMIterator TBMIterator;
+typedef struct TBMSharedIterator TBMSharedIterator;
 
 /* Result structure for tbm_iterate */
 typedef struct
@@ -46,8 +48,9 @@ typedef struct
 
 /* function prototypes in nodes/tidbitmap.c */
 
-extern TIDBitmap *tbm_create(long maxbytes);
+extern TIDBitmap *tbm_create(long maxbytes, dsa_area *dsa);
 extern void tbm_free(TIDBitmap *tbm);
+extern void tbm_free_shared_area(dsa_area *dsa, dsa_pointer dp);
 
 extern void tbm_add_tuples(TIDBitmap *tbm,
 			   const ItemPointer tids, int ntids,
@@ -60,7 +63,12 @@ extern void tbm_intersect(TIDBitmap *a, const TIDBitmap *b);
 extern bool tbm_is_empty(const TIDBitmap *tbm);
 
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
+extern dsa_pointer tbm_prepare_shared_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
+extern TBMIterateResult *tbm_shared_iterate(TBMSharedIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_end_shared_iterate(TBMSharedIterator *iterator);
+extern TBMSharedIterator *tbm_attach_shared_iterate(dsa_area *dsa,
+						  dsa_pointer dp);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 8bd93c3..0cd45bb 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -212,6 +212,7 @@ typedef enum BuiltinTrancheIds
 	LWTRANCHE_LOCK_MANAGER,
 	LWTRANCHE_PREDICATE_LOCK_MANAGER,
 	LWTRANCHE_PARALLEL_QUERY_DSA,
+	LWTRANCHE_TBM,
 	LWTRANCHE_FIRST_USER_DEFINED
 }	BuiltinTrancheIds;
 
0002-bitmap-heapscan-refactoring_v6.patchapplication/octet-stream; name=0002-bitmap-heapscan-refactoring_v6.patchDownload
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index f18827d..ed437f2 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -51,6 +51,11 @@
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
+static inline void BitmapAdjustPrefetchIterator(BitmapHeapScanState *node,
+							 TBMIterateResult *tbmres);
+static inline void BitmapAdjustPrefetchTarget(BitmapHeapScanState *node);
+static inline void BitmapPrefetch(BitmapHeapScanState *node,
+			   HeapScanDesc scan);
 
 
 /* ----------------------------------------------------------------
@@ -138,19 +143,7 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			}
 
 #ifdef USE_PREFETCH
-			if (node->prefetch_pages > 0)
-			{
-				/* The main iterator has closed the distance by one page */
-				node->prefetch_pages--;
-			}
-			else if (prefetch_iterator)
-			{
-				/* Do not let the prefetch iterator get behind the main one */
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
-
-				if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
-					elog(ERROR, "prefetch and main iterators are out of sync");
-			}
+			BitmapAdjustPrefetchIterator(node, tbmres);
 #endif   /* USE_PREFETCH */
 
 			/*
@@ -181,21 +174,8 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			scan->rs_cindex = 0;
 
 #ifdef USE_PREFETCH
-
-			/*
-			 * Increase prefetch target if it's not yet at the max.  Note that
-			 * we will increase it to zero after fetching the very first
-			 * page/tuple, then to one after the second tuple is fetched, then
-			 * it doubles as later pages are fetched.
-			 */
-			if (node->prefetch_target >= node->prefetch_maximum)
-				 /* don't increase any further */ ;
-			else if (node->prefetch_target >= node->prefetch_maximum / 2)
-				node->prefetch_target = node->prefetch_maximum;
-			else if (node->prefetch_target > 0)
-				node->prefetch_target *= 2;
-			else
-				node->prefetch_target++;
+			/* Adjust the prefetch target */
+			BitmapAdjustPrefetchTarget(node);
 #endif   /* USE_PREFETCH */
 		}
 		else
@@ -236,20 +216,8 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (prefetch_iterator)
 		{
-			while (node->prefetch_pages < node->prefetch_target)
-			{
-				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
-
-				if (tbmpre == NULL)
-				{
-					/* No more pages to prefetch */
-					tbm_end_iterate(prefetch_iterator);
-					node->prefetch_iterator = prefetch_iterator = NULL;
-					break;
-				}
-				node->prefetch_pages++;
-				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
-			}
+			BitmapPrefetch(node, scan);
+			prefetch_iterator = node->prefetch_iterator;
 		}
 #endif   /* USE_PREFETCH */
 
@@ -410,6 +378,75 @@ bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres)
 }
 
 /*
+ *	BitmapAdjustPrefetchIterator - Adjust the prefetch iterator
+ */
+static inline void
+BitmapAdjustPrefetchIterator(BitmapHeapScanState *node,
+							 TBMIterateResult *tbmres)
+{
+	TBMIterator *prefetch_iterator = node->prefetch_iterator;
+
+	if (node->prefetch_pages > 0)
+	{
+		/* The main iterator has closed the distance by one page */
+		node->prefetch_pages--;
+	}
+	else if (prefetch_iterator)
+	{
+		/* Do not let the prefetch iterator get behind the main one */
+		TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+
+		if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+			elog(ERROR, "prefetch and main iterators are out of sync");
+	}
+}
+
+/*
+ * BitmapAdjustPrefetchTarget - Adjust the prefetch target
+ *
+ * Increase prefetch target if it's not yet at the max.  Note that
+ * we will increase it to zero after fetching the very first
+ * page/tuple, then to one after the second tuple is fetched, then
+ * it doubles as later pages are fetched.
+ */
+static inline void
+BitmapAdjustPrefetchTarget(BitmapHeapScanState *node)
+{
+	if (node->prefetch_target >= node->prefetch_maximum)
+		 /* don't increase any further */ ;
+	else if (node->prefetch_target >= node->prefetch_maximum / 2)
+		node->prefetch_target = node->prefetch_maximum;
+	else if (node->prefetch_target > 0)
+		node->prefetch_target *= 2;
+	else
+		node->prefetch_target++;
+}
+
+/*
+ * BitmapPrefetch - Prefetch, if prefetch_pages are behind prefetch_target
+ */
+static inline void
+BitmapPrefetch(BitmapHeapScanState *node, HeapScanDesc scan)
+{
+	TBMIterator *prefetch_iterator = node->prefetch_iterator;
+
+	while (node->prefetch_pages < node->prefetch_target)
+	{
+		TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+
+		if (tbmpre == NULL)
+		{
+			/* No more pages to prefetch */
+			tbm_end_iterate(prefetch_iterator);
+			node->prefetch_iterator = NULL;
+			break;
+		}
+		node->prefetch_pages++;
+		PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+	}
+}
+
+/*
  * BitmapHeapRecheck -- access method routine to recheck a tuple in EvalPlanQual
  */
 static bool
0003-parallel-bitmap-heapscan-v6.patchapplication/octet-stream; name=0003-parallel-bitmap-heapscan-v6.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1d70fae..6a3481f 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1248,6 +1248,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapPopulate</></entry>
+         <entry>Waiting for the leader to populate the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index af25836..bffc971 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1754,6 +1754,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 646df08..e5263f2 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
 
 #include "executor/execParallel.h"
 #include "executor/executor.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "executor/nodeCustom.h"
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
@@ -215,6 +216,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -275,6 +280,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -757,6 +767,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index 4f3b934..5cf122b 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -56,6 +56,7 @@ static inline void BitmapPrefetch(BitmapHeapScanState *node,
 static inline void BitmapAdjustPrefetchIterator(BitmapHeapScanState *node,
 							 TBMIterateResult *tbmres);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
+static bool BitmapShouldInitializeSharedState(ParallelBitmapHeapState *pbminfo);
 
 
 /* ----------------------------------------------------------------
@@ -71,13 +72,17 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	HeapScanDesc scan;
 	TIDBitmap  *tbm;
 	TBMIterator *tbmiterator;
+	TBMSharedIterator *shared_tbmiterator;
 	TBMIterateResult *tbmres;
 
 #ifdef USE_PREFETCH
 	TBMIterator *prefetch_iterator;
+	TBMSharedIterator *shared_prefetch_iterator;
 #endif
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelBitmapHeapState *pstate = node->pstate;
+	dsa_area   *dsa = node->ss.ps.state->es_query_dsa;
 
 	/*
 	 * extract necessary information from index scan node
@@ -87,9 +92,11 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	scan = node->ss.ss_currentScanDesc;
 	tbm = node->tbm;
 	tbmiterator = node->tbmiterator;
+	shared_tbmiterator = node->shared_tbmiterator;
 	tbmres = node->tbmres;
 #ifdef USE_PREFETCH
 	prefetch_iterator = node->prefetch_iterator;
+	shared_prefetch_iterator = node->shared_prefetch_iterator;
 #endif
 
 	/*
@@ -104,25 +111,90 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 * node->prefetch_maximum.  This is to avoid doing a lot of prefetching in
 	 * a scan that stops after a few tuples because of a LIMIT.
 	 */
-	if (tbm == NULL)
+	if (!node->initialized)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		if (!pstate)
+		{
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
 
-		node->tbm = tbm;
-		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
-		node->tbmres = tbmres = NULL;
+			node->tbm = tbm;
+			node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+			node->tbmres = tbmres = NULL;
 
 #ifdef USE_PREFETCH
-		if (node->prefetch_maximum > 0)
-		{
-			node->prefetch_iterator = prefetch_iterator = tbm_begin_iterate(tbm);
-			node->prefetch_pages = 0;
-			node->prefetch_target = -1;
+			if (node->prefetch_maximum > 0)
+			{
+				node->prefetch_iterator =
+					prefetch_iterator = tbm_begin_iterate(tbm);
+				node->prefetch_pages = 0;
+				node->prefetch_target = -1;
+			}
+#endif   /* USE_PREFETCH */
 		}
+		else
+		{
+			/*
+			 * The leader will immediately come out of the function, but
+			 * others will be blocked until leader populates the TBM and wakes
+			 * them up.
+			 */
+			if (BitmapShouldInitializeSharedState(pstate))
+			{
+				tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+				if (!tbm || !IsA(tbm, TIDBitmap))
+					elog(ERROR, "unrecognized result from subplan");
+
+				node->tbm = tbm;
+
+				/*
+				 * Prepare to iterate over the TBM. This will return the
+				 * dsa_pointer of the iterator state which will be used by
+				 * multiple processes to iterate jointly.
+				 */
+				pstate->tbmiterator = tbm_prepare_shared_iterate(tbm);
+#ifdef USE_PREFETCH
+				if (node->prefetch_maximum > 0)
+				{
+					pstate->prefetch_iterator =
+						tbm_prepare_shared_iterate(tbm);
+
+					/*
+					 * We don't need mutex here as we haven't yet woke up
+					 * others
+					 */
+					pstate->prefetch_pages = 0;
+					pstate->prefetch_target = -1;
+				}
+#endif
+
+				/*
+				 * By this time we have already populated the TBM and
+				 * initialized the shared iterators so set the state to
+				 * BM_FINISHED and wake up others.
+				 */
+				SpinLockAcquire(&pstate->mutex);
+				pstate->state = BM_FINISHED;
+				SpinLockRelease(&pstate->mutex);
+				ConditionVariableBroadcast(&pstate->cv);
+			}
+
+			/* Allocate a private iterator and attach the shared state to it */
+			node->shared_tbmiterator = shared_tbmiterator =
+				tbm_attach_shared_iterate(dsa, pstate->tbmiterator);
+			node->tbmres = tbmres = NULL;
+
+#ifdef USE_PREFETCH
+			if (node->prefetch_maximum > 0)
+			{
+				node->shared_prefetch_iterator = shared_prefetch_iterator =
+					tbm_attach_shared_iterate(dsa, pstate->prefetch_iterator);
+			}
 #endif   /* USE_PREFETCH */
+		}
+		node->initialized = true;
 	}
 
 	for (;;)
@@ -135,7 +207,14 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			/*
+			 * If we are in parallel mode perform shared iterate otherwise
+			 * local iterate.
+			 */
+			if (!pstate)
+				node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			else
+				node->tbmres = tbmres = tbm_shared_iterate(shared_tbmiterator);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -191,8 +270,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (!pstate)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else if (pstate->prefetch_target < node->prefetch_maximum)
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pstate->mutex);
+				if (pstate->prefetch_target < node->prefetch_maximum)
+					pstate->prefetch_target++;
+				SpinLockRelease(&pstate->mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -217,7 +310,11 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		if (prefetch_iterator)
 		{
 			BitmapPrefetch(node, scan);
-			prefetch_iterator = node->prefetch_iterator;
+
+			if (node->pstate == NULL)
+				prefetch_iterator = node->prefetch_iterator;
+			else
+				shared_prefetch_iterator = node->shared_prefetch_iterator;
 		}
 #endif   /* USE_PREFETCH */
 
@@ -384,20 +481,54 @@ static inline void
 BitmapAdjustPrefetchIterator(BitmapHeapScanState *node,
 							 TBMIterateResult *tbmres)
 {
-	TBMIterator *prefetch_iterator = node->prefetch_iterator;
+	ParallelBitmapHeapState *pstate = node->pstate;
 
-	if (node->prefetch_pages > 0)
+	if (pstate == NULL)
 	{
-		/* The main iterator has closed the distance by one page */
-		node->prefetch_pages--;
+		TBMIterator *prefetch_iterator = node->prefetch_iterator;
+
+		if (node->prefetch_pages > 0)
+		{
+			/* The main iterator has closed the distance by one page */
+			node->prefetch_pages--;
+		}
+		else if (prefetch_iterator)
+		{
+			/* Do not let the prefetch iterator get behind the main one */
+			TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+
+			if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				elog(ERROR, "prefetch and main iterators are out of sync");
+		}
+
+		return;
 	}
-	else if (prefetch_iterator)
+
+	if (node->prefetch_maximum > 0)
 	{
-		/* Do not let the prefetch iterator get behind the main one */
-		TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+		TBMSharedIterator *prefetch_iterator = node->shared_prefetch_iterator;
 
-		if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
-			elog(ERROR, "prefetch and main iterators are out of sync");
+		SpinLockAcquire(&pstate->mutex);
+		if (pstate->prefetch_pages > 0)
+		{
+			node->prefetch_pages--;
+			SpinLockRelease(&pstate->mutex);
+		}
+		else
+		{
+			/* Release the mutex before iterating */
+			SpinLockRelease(&pstate->mutex);
+
+			/*
+			 * In case of shared mode, we can not ensure that the current
+			 * blockno of the main iterator and that of the prefetch iterator
+			 * are same.  It's possible that whatever blockno we are
+			 * prefetching will be processed by another process.  Therefore we
+			 * don't validate the blockno unlike we do in non-parallel case.
+			 */
+			if (prefetch_iterator)
+				tbm_shared_iterate(prefetch_iterator);
+		}
 	}
 }
 
@@ -412,14 +543,38 @@ BitmapAdjustPrefetchIterator(BitmapHeapScanState *node,
 static inline void
 BitmapAdjustPrefetchTarget(BitmapHeapScanState *node)
 {
-	if (node->prefetch_target >= node->prefetch_maximum)
-		 /* don't increase any further */ ;
-	else if (node->prefetch_target >= node->prefetch_maximum / 2)
-		node->prefetch_target = node->prefetch_maximum;
-	else if (node->prefetch_target > 0)
-		node->prefetch_target *= 2;
-	else
-		node->prefetch_target++;
+	ParallelBitmapHeapState *pstate = node->pstate;
+
+	if (pstate == NULL)
+	{
+		if (node->prefetch_target >= node->prefetch_maximum)
+			 /* don't increase any further */ ;
+		else if (node->prefetch_target >= node->prefetch_maximum / 2)
+			node->prefetch_target = node->prefetch_maximum;
+		else if (node->prefetch_target > 0)
+			node->prefetch_target *= 2;
+		else
+			node->prefetch_target++;
+		return;
+	}
+
+	/*
+	 * Check before acquiring the mutex so that we can avoid acquiring the
+	 * mutex if target has already reached to its max value.
+	 */
+	if (pstate->prefetch_target < node->prefetch_maximum)
+	{
+		SpinLockAcquire(&pstate->mutex);
+		if (pstate->prefetch_target >= node->prefetch_maximum)
+			 /* don't increase any further */ ;
+		else if (pstate->prefetch_target >= node->prefetch_maximum / 2)
+			pstate->prefetch_target = node->prefetch_maximum;
+		else if (pstate->prefetch_target > 0)
+			pstate->prefetch_target *= 2;
+		else
+			pstate->prefetch_target++;
+		SpinLockRelease(&pstate->mutex);
+	}
 }
 
 /*
@@ -428,21 +583,77 @@ BitmapAdjustPrefetchTarget(BitmapHeapScanState *node)
 static inline void
 BitmapPrefetch(BitmapHeapScanState *node, HeapScanDesc scan)
 {
-	TBMIterator *prefetch_iterator = node->prefetch_iterator;
+	ParallelBitmapHeapState *pstate = node->pstate;
 
-	while (node->prefetch_pages < node->prefetch_target)
+	if (pstate == NULL)
 	{
-		TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+		TBMIterator *prefetch_iterator = node->prefetch_iterator;
 
-		if (tbmpre == NULL)
+		while (node->prefetch_pages < node->prefetch_target)
 		{
-			/* No more pages to prefetch */
-			tbm_end_iterate(prefetch_iterator);
-			node->prefetch_iterator = NULL;
-			break;
+			TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+
+			if (tbmpre == NULL)
+			{
+				/* No more pages to prefetch */
+				tbm_end_iterate(prefetch_iterator);
+				node->prefetch_iterator = NULL;
+				break;
+			}
+			node->prefetch_pages++;
+			PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+		}
+
+		return;
+	}
+
+	if (pstate->prefetch_pages < pstate->prefetch_target)
+	{
+		TBMSharedIterator *prefetch_iterator = node->shared_prefetch_iterator;
+
+		SpinLockAcquire(&pstate->mutex);
+
+		/* Recheck under the mutex */
+		if (pstate->prefetching &&
+			pstate->prefetch_pages < pstate->prefetch_target)
+		{
+			/*
+			 * If one of the process has already identified that we need to do
+			 * prefetch then let it perform the prefetch and allow others to
+			 * proceed with the work in hand.  Another option could be that we
+			 * allow all of them to participate in prefetching.  But, most of
+			 * this work done under mutex or LWLock so ultimately we may end up
+			 * in prefetching sequentially.
+			 */
+			pstate->prefetching = true;
+			SpinLockRelease(&pstate->mutex);
+			do
+			{
+				TBMIterateResult *tbmpre;
+
+				tbmpre = tbm_shared_iterate(prefetch_iterator);
+				if (tbmpre == NULL)
+				{
+					/* No more pages to prefetch */
+					tbm_end_shared_iterate(prefetch_iterator);
+					node->shared_prefetch_iterator = prefetch_iterator = NULL;
+					break;
+				}
+
+				SpinLockAcquire(&pstate->mutex);
+				pstate->prefetch_pages++;
+				if (pstate->prefetch_pages >= pstate->prefetch_target)
+				{
+					SpinLockRelease(&pstate->mutex);
+					break;
+				}
+				SpinLockRelease(&pstate->mutex);
+
+				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+			} while (true);
 		}
-		node->prefetch_pages++;
-		PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+		else
+			SpinLockRelease(&pstate->mutex);
 	}
 }
 
@@ -495,12 +706,36 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->tbmiterator);
 	if (node->prefetch_iterator)
 		tbm_end_iterate(node->prefetch_iterator);
+	if (node->shared_tbmiterator)
+		tbm_end_shared_iterate(node->shared_tbmiterator);
+	if (node->shared_prefetch_iterator)
+		tbm_end_shared_iterate(node->shared_prefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
 	node->tbm = NULL;
 	node->tbmiterator = NULL;
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
+	node->initialized = false;
+	node->shared_tbmiterator = NULL;
+	node->shared_prefetch_iterator = NULL;
+
+	/* Reset parallel bitmap state, if present */
+	if (node->pstate)
+	{
+		dsa_area   *dsa = node->ss.ps.state->es_query_dsa;
+
+		node->pstate->state = BM_INITIAL;
+
+		if (DsaPointerIsValid(node->pstate->tbmiterator))
+			tbm_free_shared_area(dsa, node->pstate->tbmiterator);
+
+		if (DsaPointerIsValid(node->pstate->prefetch_iterator))
+			dsa_free(dsa, node->pstate->prefetch_iterator);
+
+		node->pstate->tbmiterator = InvalidDsaPointer;
+		node->pstate->prefetch_iterator = InvalidDsaPointer;
+	}
 
 	ExecScanReScan(&node->ss);
 
@@ -553,6 +788,10 @@ ExecEndBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->prefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
+	if (node->shared_tbmiterator)
+		tbm_end_shared_iterate(node->shared_tbmiterator);
+	if (node->shared_prefetch_iterator)
+		tbm_end_shared_iterate(node->shared_prefetch_iterator);
 
 	/*
 	 * close heap scan
@@ -604,6 +843,10 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->pscan_len = 0;
+	scanstate->initialized = false;
+	scanstate->shared_tbmiterator = NULL;
+	scanstate->pstate = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -688,3 +931,127 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ *		BitmapShouldInitializeSharedState
+ *
+ *		The first process to come here and see the state to the BM_INITIAL
+ *		will become the leader for the parallel bitmap scan and will be
+ *		responsible for populating the TIDBitmap.  The other processes will
+ *		be blocked by the condition variable until the leader wakes them up.
+ * ---------------
+ */
+static bool
+BitmapShouldInitializeSharedState(ParallelBitmapHeapState *pstate)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * BM_INITIAL	  : We become the leader and set it to BM_INPROGRESS
+		 * BM_INPROGRESS : We need to wait till leader creates bitmap
+		 * BM_FINISHED   : bitmap is ready so no need to wait
+		 *---------------
+		 */
+		SpinLockAcquire(&pstate->mutex);
+
+		if (pstate->state == BM_INITIAL)
+		{
+			pstate->state = BM_INPROGRESS;
+			leader = true;
+		}
+		else if (pstate->state == BM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pstate->mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pstate->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+
+	/* Estimate the size for sharing parallel Bitmap info. */
+	node->pscan_len = add_size(offsetof(ParallelBitmapHeapState,
+										phs_snapshot_data),
+							   EstimateSnapshotSpace(estate->es_snapshot));
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapHeapState *pstate;
+	EState	   *estate = node->ss.ps.state;
+
+	pstate = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	pstate->tbmiterator = 0;
+	pstate->prefetch_iterator = 0;
+
+	/* Initialize the mutex */
+	SpinLockInit(&pstate->mutex);
+	pstate->prefetch_pages = 0;
+	pstate->prefetch_target = 0;
+	pstate->prefetching = false;
+	pstate->state = BM_INITIAL;
+
+	ConditionVariableInit(&pstate->cv);
+	SerializeSnapshot(estate->es_snapshot, pstate->phs_snapshot_data);
+
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pstate);
+	node->pstate = pstate;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapHeapState *pstate;
+	Snapshot	snapshot;
+
+	pstate = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->pstate = pstate;
+
+	snapshot = RestoreSnapshot(pstate->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 94bb289..ce2f321 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,9 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L, NULL);
+		tbm = tbm_create(work_mem * 1024L,
+						 ((BitmapIndexScan *) node->ss.ps.plan)->isshared ?
+						 node->ss.ps.state->es_query_dsa : NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 1d280be..c0f2614 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,9 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L, NULL);
+				result = tbm_create(work_mem * 1024L,
+									((BitmapOr *) node->ps.plan)->isshared ?
+									node->ps.state->es_query_dsa : NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 05d8538..89afc7a 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -331,6 +331,7 @@ _copyBitmapOr(const BitmapOr *from)
 	/*
 	 * copy remainder of node
 	 */
+	COPY_SCALAR_FIELD(isshared);
 	COPY_NODE_FIELD(bitmapplans);
 
 	return newnode;
@@ -496,6 +497,7 @@ _copyBitmapIndexScan(const BitmapIndexScan *from)
 	 * copy remainder of node
 	 */
 	COPY_SCALAR_FIELD(indexid);
+	COPY_SCALAR_FIELD(isshared);
 	COPY_NODE_FIELD(indexqual);
 	COPY_NODE_FIELD(indexqualorig);
 
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b3802b4..0ec4e1b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -441,6 +441,7 @@ _outBitmapOr(StringInfo str, const BitmapOr *node)
 
 	_outPlanInfo(str, (const Plan *) node);
 
+	WRITE_BOOL_FIELD(isshared);
 	WRITE_NODE_FIELD(bitmapplans);
 }
 
@@ -520,6 +521,7 @@ _outBitmapIndexScan(StringInfo str, const BitmapIndexScan *node)
 	_outScanInfo(str, (const Scan *) node);
 
 	WRITE_OID_FIELD(indexid);
+	WRITE_BOOL_FIELD(isshared);
 	WRITE_NODE_FIELD(indexqual);
 	WRITE_NODE_FIELD(indexqualorig);
 }
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index d2f69fe..6bf6d2d 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1607,6 +1607,7 @@ _readBitmapOr(void)
 
 	ReadCommonPlan(&local_node->plan);
 
+	READ_BOOL_FIELD(isshared);
 	READ_NODE_FIELD(bitmapplans);
 
 	READ_DONE();
@@ -1718,6 +1719,7 @@ _readBitmapIndexScan(void)
 	ReadCommonScan(&local_node->scan);
 
 	READ_OID_FIELD(indexid);
+	READ_BOOL_FIELD(isshared);
 	READ_NODE_FIELD(indexqual);
 	READ_NODE_FIELD(indexqualorig);
 
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index eeacf81..f671f0a 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2875,6 +2875,30 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 }
 
 /*
+ * create_partial_bitmap_paths
+ *	  Build partial bitmap heap path for the relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched, 0);
+
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+/*
  * Compute the number of parallel workers that should be used to scan a
  * relation.  We compute the parallel workers based on the size of the heap to
  * be scanned and the size of the index to be scanned, then choose a minimum
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index d01630f..6240819 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -861,6 +861,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -922,8 +923,21 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
+
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index d92826b..c411962 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1618,6 +1622,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1660,6 +1669,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 997bdcf..19e5f0a 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -125,6 +125,7 @@ static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
 						List *tlist, List *scan_clauses);
 static Plan *create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 					  List **qual, List **indexqual, List **indexECs);
+static void bitmap_subplan_mark_shared(Plan *plan);
 static TidScan *create_tidscan_plan(PlannerInfo *root, TidPath *best_path,
 					List *tlist, List *scan_clauses);
 static SubqueryScan *create_subqueryscan_plan(PlannerInfo *root,
@@ -2579,6 +2580,9 @@ create_bitmap_scan_plan(PlannerInfo *root,
 										   &bitmapqualorig, &indexquals,
 										   &indexECs);
 
+	if (best_path->path.parallel_aware)
+		bitmap_subplan_mark_shared(bitmapqualplan);
+
 	/*
 	 * The qpqual list must contain all restrictions not automatically handled
 	 * by the index, other than pseudoconstant clauses which will be handled
@@ -2715,6 +2719,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 			subplan = create_bitmap_subplan(root, (Path *) lfirst(l),
 											&subqual, &subindexqual,
 											&subindexEC);
+
 			subplans = lappend(subplans, subplan);
 			subquals = list_concat_unique(subquals, subqual);
 			subindexquals = list_concat_unique(subindexquals, subindexqual);
@@ -4706,6 +4711,24 @@ label_sort_with_costsize(PlannerInfo *root, Sort *plan, double limit_tuples)
 	plan->plan.parallel_aware = false;
 }
 
+/*
+ * bitmap_subplan_mark_shared
+ *	 Mark a shared flag in bitmap subplan so that it can create underlying
+ *	 bitmap in shared memory which is accecible by the multiple processes.
+ */
+static void
+bitmap_subplan_mark_shared(Plan *plan)
+{
+	if (IsA(plan, BitmapAnd))
+		bitmap_subplan_mark_shared(
+								linitial(((BitmapAnd *) plan)->bitmapplans));
+	else if (IsA(plan, BitmapOr))
+		((BitmapOr *) plan)->isshared = true;
+	else if (IsA(plan, BitmapIndexScan))
+		((BitmapIndexScan *) plan)->isshared = true;
+	else
+		elog(ERROR, "unrecognized node type: %d", nodeTag(plan));
+}
 
 /*****************************************************************************
  *
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3248296..6e70808 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1068,7 +1068,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1077,9 +1078,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3255,7 +3256,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index ada374c..9b191df 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3395,6 +3395,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index a864f78..7e85510 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -179,6 +179,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 1c1cb80..c3e6b2a 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1464,6 +1466,54 @@ typedef struct BitmapIndexScanState
 } BitmapIndexScanState;
 
 /* ----------------
+ *	 SharedBitmapState information
+ *
+ *		BM_INITIAL		TIDBitmap creation is not yet started, so first worker
+ *						to see this state will set the state to BM_INPROGRESS
+ *						and that process will be responsible for creating
+ *						TIDBitmap.
+ *		BM_INPROGRESS	TIDBitmap creation is already in progress, therefore
+ *						workers need to sleep until leader set the state to
+ *						BM_FINISHED and wake us up.
+ *		BM_FINISHED		TIDBitmap creation is done, so now all worker can
+ *						proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	BM_INITIAL,
+	BM_INPROGRESS,
+	BM_FINISHED
+} SharedBitmapState;
+
+/* ----------------
+ *	 ParallelBitmapHeapState information
+ *		tbmiterator				iterator for scanning current pages
+ *		prefetch_iterator		iterator for prefetching ahead of current page
+ *		mutex					mutual exclusion for the prefetching variable
+ *								and state
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		prefetching				set true if prefetching is in progress
+ *		state					current state of the TIDBitmap
+ *		cv						conditional wait variable
+ *		phs_snapshot_data		snapshot data shared to workers
+ * ----------------
+ */
+typedef struct ParallelBitmapHeapState
+{
+	dsa_pointer tbmiterator;
+	dsa_pointer prefetch_iterator;
+	slock_t		mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	bool		prefetching;
+	SharedBitmapState state;
+	ConditionVariable cv;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+} ParallelBitmapHeapState;
+
+/* ----------------
  *	 BitmapHeapScanState information
  *
  *		bitmapqualorig	   execution state for bitmapqualorig expressions
@@ -1476,6 +1526,11 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		initialized		   is node is ready to iterate
+ *		shared_tbmiterator	   shared iterator
+ *		shared_prefetch_iterator shared iterator for prefetching
+ *		pstate			   shared state for parallel bitmap scan
  * ----------------
  */
 typedef struct BitmapHeapScanState
@@ -1491,6 +1546,11 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	bool		initialized;
+	TBMSharedIterator *shared_tbmiterator;
+	TBMSharedIterator *shared_prefetch_iterator;
+	ParallelBitmapHeapState *pstate;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f72f7a8..7f38ca6 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -292,6 +292,7 @@ typedef struct BitmapAnd
 typedef struct BitmapOr
 {
 	Plan		plan;
+	bool		isshared;
 	List	   *bitmapplans;
 } BitmapOr;
 
@@ -420,6 +421,7 @@ typedef struct BitmapIndexScan
 {
 	Scan		scan;
 	Oid			indexid;		/* OID of index to scan */
+	bool		isshared;		/* Create shared bitmap if set */
 	List	   *indexqual;		/* list of index quals (OpExprs) */
 	List	   *indexqualorig;	/* the same in original form */
 } BitmapIndexScan;
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 53cad24..d22b039 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -53,7 +53,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index ebda308..3ec2d9d 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -56,6 +56,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
 extern int compute_parallel_worker(RelOptInfo *rel, BlockNumber heap_pages,
 						BlockNumber index_pages);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+										Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 8b710ec..e31389f 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -787,6 +787,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
perf_resultapplication/octet-stream; name=perf_resultDownload
#103Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#102)
Re: Parallel bitmap heap scan

On Tue, Feb 28, 2017 at 9:18 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

0001- same as previous with some changes for freeing the shared memory stuff.
0002- nodeBitmapHeapScan refactoring, this applies independently
0003- actual parallel bitmap stuff applies on top of 0001 and 0002

0002 wasn't quite careful enough about the placement of #ifdef
USE_PREFETCH, but otherwise looks OK. Committed after changing that
and getting rid of the local variable prefetch_iterator, which seemed
to be adding rather than removing complexity after this refactoring.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#104Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#103)
1 attachment(s)
Re: Parallel bitmap heap scan

On Thu, Mar 2, 2017 at 6:52 PM, Robert Haas <robertmhaas@gmail.com> wrote:

0002 wasn't quite careful enough about the placement of #ifdef
USE_PREFETCH, but otherwise looks OK. Committed after changing that
and getting rid of the local variable prefetch_iterator, which seemed
to be adding rather than removing complexity after this refactoring.

0003 is rebased after this commit.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0003-parallel-bitmap-heapscan-v7.patchapplication/octet-stream; name=0003-parallel-bitmap-heapscan-v7.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1d70fae..6a3481f 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1248,6 +1248,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapPopulate</></entry>
+         <entry>Waiting for the leader to populate the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index af25836..bffc971 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1754,6 +1754,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index de0e2ba..a1289e5 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
 
 #include "executor/execParallel.h"
 #include "executor/executor.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "executor/nodeCustom.h"
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
@@ -217,6 +218,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -277,6 +282,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -775,6 +785,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index c1aa9f1..4864691 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -58,6 +58,8 @@ static inline void BitmapAdjustPrefetchIterator(BitmapHeapScanState *node,
 static inline void BitmapAdjustPrefetchTarget(BitmapHeapScanState *node);
 static inline void BitmapPrefetch(BitmapHeapScanState *node,
 			   HeapScanDesc scan);
+static bool BitmapShouldInitializeSharedState(
+								  ParallelBitmapHeapState *pbminfo);
 
 
 /* ----------------------------------------------------------------
@@ -73,9 +75,12 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	HeapScanDesc scan;
 	TIDBitmap  *tbm;
 	TBMIterator *tbmiterator;
+	TBMSharedIterator *shared_tbmiterator;
 	TBMIterateResult *tbmres;
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelBitmapHeapState *pstate = node->pstate;
+	dsa_area   *dsa = node->ss.ps.state->es_query_dsa;
 
 	/*
 	 * extract necessary information from index scan node
@@ -84,7 +89,10 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	slot = node->ss.ss_ScanTupleSlot;
 	scan = node->ss.ss_currentScanDesc;
 	tbm = node->tbm;
-	tbmiterator = node->tbmiterator;
+	if (pstate == NULL)
+		tbmiterator = node->tbmiterator;
+	else
+		shared_tbmiterator = node->shared_tbmiterator;
 	tbmres = node->tbmres;
 
 	/*
@@ -99,25 +107,89 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 * node->prefetch_maximum.  This is to avoid doing a lot of prefetching in
 	 * a scan that stops after a few tuples because of a LIMIT.
 	 */
-	if (tbm == NULL)
+	if (!node->initialized)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		if (!pstate)
+		{
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
 
-		node->tbm = tbm;
-		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
-		node->tbmres = tbmres = NULL;
+			node->tbm = tbm;
+			node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+			node->tbmres = tbmres = NULL;
 
 #ifdef USE_PREFETCH
-		if (node->prefetch_maximum > 0)
-		{
-			node->prefetch_iterator = tbm_begin_iterate(tbm);
-			node->prefetch_pages = 0;
-			node->prefetch_target = -1;
+			if (node->prefetch_maximum > 0)
+			{
+				node->prefetch_iterator = tbm_begin_iterate(tbm);
+				node->prefetch_pages = 0;
+				node->prefetch_target = -1;
+			}
+#endif   /* USE_PREFETCH */
 		}
+		else
+		{
+			/*
+			 * The leader will immediately come out of the function, but
+			 * others will be blocked until leader populates the TBM and wakes
+			 * them up.
+			 */
+			if (BitmapShouldInitializeSharedState(pstate))
+			{
+				tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+				if (!tbm || !IsA(tbm, TIDBitmap))
+					elog(ERROR, "unrecognized result from subplan");
+
+				node->tbm = tbm;
+
+				/*
+				 * Prepare to iterate over the TBM. This will return the
+				 * dsa_pointer of the iterator state which will be used by
+				 * multiple processes to iterate jointly.
+				 */
+				pstate->tbmiterator = tbm_prepare_shared_iterate(tbm);
+#ifdef USE_PREFETCH
+				if (node->prefetch_maximum > 0)
+				{
+					pstate->prefetch_iterator =
+						tbm_prepare_shared_iterate(tbm);
+
+					/*
+					 * We don't need mutex here as we haven't yet woke up
+					 * others
+					 */
+					pstate->prefetch_pages = 0;
+					pstate->prefetch_target = -1;
+				}
+#endif
+
+				/*
+				 * By this time we have already populated the TBM and
+				 * initialized the shared iterators so set the state to
+				 * BM_FINISHED and wake up others.
+				 */
+				SpinLockAcquire(&pstate->mutex);
+				pstate->state = BM_FINISHED;
+				SpinLockRelease(&pstate->mutex);
+				ConditionVariableBroadcast(&pstate->cv);
+			}
+
+			/* Allocate a private iterator and attach the shared state to it */
+			node->shared_tbmiterator = shared_tbmiterator =
+				tbm_attach_shared_iterate(dsa, pstate->tbmiterator);
+			node->tbmres = tbmres = NULL;
+
+#ifdef USE_PREFETCH
+			if (node->prefetch_maximum > 0)
+			{
+				node->shared_prefetch_iterator =
+					tbm_attach_shared_iterate(dsa, pstate->prefetch_iterator);
+			}
 #endif   /* USE_PREFETCH */
+		}
+		node->initialized = true;
 	}
 
 	for (;;)
@@ -130,7 +202,14 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			/*
+			 * If we are in parallel mode perform shared iterate otherwise
+			 * local iterate.
+			 */
+			if (!pstate)
+				node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			else
+				node->tbmres = tbmres = tbm_shared_iterate(shared_tbmiterator);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -182,8 +261,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (!pstate)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else if (pstate->prefetch_target < node->prefetch_maximum)
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pstate->mutex);
+				if (pstate->prefetch_target < node->prefetch_maximum)
+					pstate->prefetch_target++;
+				SpinLockRelease(&pstate->mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -369,20 +462,53 @@ BitmapAdjustPrefetchIterator(BitmapHeapScanState *node,
 							 TBMIterateResult *tbmres)
 {
 #ifdef USE_PREFETCH
-	TBMIterator *prefetch_iterator = node->prefetch_iterator;
+	ParallelBitmapHeapState *pstate = node->pstate;
 
-	if (node->prefetch_pages > 0)
+	if (pstate == NULL)
 	{
-		/* The main iterator has closed the distance by one page */
-		node->prefetch_pages--;
+		TBMIterator *prefetch_iterator = node->prefetch_iterator;
+
+		if (node->prefetch_pages > 0)
+		{
+			/* The main iterator has closed the distance by one page */
+			node->prefetch_pages--;
+		}
+		else if (prefetch_iterator)
+		{
+			/* Do not let the prefetch iterator get behind the main one */
+			TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+
+			if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				elog(ERROR, "prefetch and main iterators are out of sync");
+		}
+		return;
 	}
-	else if (prefetch_iterator)
+
+	if (node->prefetch_maximum > 0)
 	{
-		/* Do not let the prefetch iterator get behind the main one */
-		TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+		TBMSharedIterator *prefetch_iterator = node->shared_prefetch_iterator;
+
+		SpinLockAcquire(&pstate->mutex);
+		if (pstate->prefetch_pages > 0)
+		{
+			node->prefetch_pages--;
+			SpinLockRelease(&pstate->mutex);
+		}
+		else
+		{
+			/* Release the mutex before iterating */
+			SpinLockRelease(&pstate->mutex);
 
-		if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
-			elog(ERROR, "prefetch and main iterators are out of sync");
+			/*
+			 * In case of shared mode, we can not ensure that the current
+			 * blockno of the main iterator and that of the prefetch iterator
+			 * are same.  It's possible that whatever blockno we are
+			 * prefetching will be processed by another process.  Therefore we
+			 * don't validate the blockno unlike we do in non-parallel case.
+			 */
+			if (prefetch_iterator)
+				tbm_shared_iterate(prefetch_iterator);
+		}
 	}
 #endif   /* USE_PREFETCH */
 }
@@ -399,14 +525,38 @@ static inline void
 BitmapAdjustPrefetchTarget(BitmapHeapScanState *node)
 {
 #ifdef USE_PREFETCH
-	if (node->prefetch_target >= node->prefetch_maximum)
-		 /* don't increase any further */ ;
-	else if (node->prefetch_target >= node->prefetch_maximum / 2)
-		node->prefetch_target = node->prefetch_maximum;
-	else if (node->prefetch_target > 0)
-		node->prefetch_target *= 2;
-	else
-		node->prefetch_target++;
+	ParallelBitmapHeapState *pstate = node->pstate;
+
+	if (pstate == NULL)
+	{
+		if (node->prefetch_target >= node->prefetch_maximum)
+			 /* don't increase any further */ ;
+		else if (node->prefetch_target >= node->prefetch_maximum / 2)
+			node->prefetch_target = node->prefetch_maximum;
+		else if (node->prefetch_target > 0)
+			node->prefetch_target *= 2;
+		else
+			node->prefetch_target++;
+		return;
+	}
+
+	/*
+	 * Check before acquiring the mutex so that we can avoid acquiring the
+	 * mutex if target has already reached to its max value.
+	 */
+	if (pstate->prefetch_target < node->prefetch_maximum)
+	{
+		SpinLockAcquire(&pstate->mutex);
+		if (pstate->prefetch_target >= node->prefetch_maximum)
+			 /* don't increase any further */ ;
+		else if (pstate->prefetch_target >= node->prefetch_maximum / 2)
+			pstate->prefetch_target = node->prefetch_maximum;
+		else if (pstate->prefetch_target > 0)
+			pstate->prefetch_target *= 2;
+		else
+			pstate->prefetch_target++;
+		SpinLockRelease(&pstate->mutex);
+	}
 #endif   /* USE_PREFETCH */
 }
 
@@ -417,23 +567,83 @@ static inline void
 BitmapPrefetch(BitmapHeapScanState *node, HeapScanDesc scan)
 {
 #ifdef USE_PREFETCH
-	TBMIterator *prefetch_iterator = node->prefetch_iterator;
+	ParallelBitmapHeapState *pstate = node->pstate;
 
-	if (prefetch_iterator)
+	if (pstate == NULL)
 	{
-		while (node->prefetch_pages < node->prefetch_target)
+		TBMIterator *prefetch_iterator = node->prefetch_iterator;
+
+		if (prefetch_iterator)
 		{
-			TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+			while (node->prefetch_pages < node->prefetch_target)
+			{
+				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+
+				if (tbmpre == NULL)
+				{
+					/* No more pages to prefetch */
+					tbm_end_iterate(prefetch_iterator);
+					node->prefetch_iterator = NULL;
+					break;
+				}
+				node->prefetch_pages++;
+				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+			}
+		}
+
+		return;
+	}
+
+	if (pstate->prefetch_pages < pstate->prefetch_target)
+	{
+		TBMSharedIterator *prefetch_iterator = node->shared_prefetch_iterator;
 
-			if (tbmpre == NULL)
+		if (prefetch_iterator)
+		{
+			SpinLockAcquire(&pstate->mutex);
+
+			/* Recheck under the mutex */
+			if (pstate->prefetching &&
+				pstate->prefetch_pages < pstate->prefetch_target)
 			{
-				/* No more pages to prefetch */
-				tbm_end_iterate(prefetch_iterator);
-				node->prefetch_iterator = NULL;
-				break;
+				/*
+				 * If one of the process has already identified that we need
+				 * to do prefetch then let it perform the prefetch and allow
+				 * others to proceed with the work in hand.  Another option
+				 * could be that we allow all of them to participate in
+				 * prefetching.  But, most of this work done under mutex or
+				 * LWLock so ultimately we may end up in prefetching
+				 * sequentially.
+				 */
+				pstate->prefetching = true;
+				SpinLockRelease(&pstate->mutex);
+				do
+				{
+					TBMIterateResult *tbmpre;
+
+					tbmpre = tbm_shared_iterate(prefetch_iterator);
+					if (tbmpre == NULL)
+					{
+						/* No more pages to prefetch */
+						tbm_end_shared_iterate(prefetch_iterator);
+						node->shared_prefetch_iterator = NULL;
+						break;
+					}
+
+					SpinLockAcquire(&pstate->mutex);
+					pstate->prefetch_pages++;
+					if (pstate->prefetch_pages >= pstate->prefetch_target)
+					{
+						SpinLockRelease(&pstate->mutex);
+						break;
+					}
+					SpinLockRelease(&pstate->mutex);
+
+					PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+				} while (true);
 			}
-			node->prefetch_pages++;
-			PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+			else
+				SpinLockRelease(&pstate->mutex);
 		}
 	}
 #endif   /* USE_PREFETCH */
@@ -488,12 +698,36 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->tbmiterator);
 	if (node->prefetch_iterator)
 		tbm_end_iterate(node->prefetch_iterator);
+	if (node->shared_tbmiterator)
+		tbm_end_shared_iterate(node->shared_tbmiterator);
+	if (node->shared_prefetch_iterator)
+		tbm_end_shared_iterate(node->shared_prefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
 	node->tbm = NULL;
 	node->tbmiterator = NULL;
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
+	node->initialized = false;
+	node->shared_tbmiterator = NULL;
+	node->shared_prefetch_iterator = NULL;
+
+	/* Reset parallel bitmap state, if present */
+	if (node->pstate)
+	{
+		dsa_area   *dsa = node->ss.ps.state->es_query_dsa;
+
+		node->pstate->state = BM_INITIAL;
+
+		if (DsaPointerIsValid(node->pstate->tbmiterator))
+			tbm_free_shared_area(dsa, node->pstate->tbmiterator);
+
+		if (DsaPointerIsValid(node->pstate->prefetch_iterator))
+			dsa_free(dsa, node->pstate->prefetch_iterator);
+
+		node->pstate->tbmiterator = InvalidDsaPointer;
+		node->pstate->prefetch_iterator = InvalidDsaPointer;
+	}
 
 	ExecScanReScan(&node->ss);
 
@@ -546,6 +780,10 @@ ExecEndBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->prefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
+	if (node->shared_tbmiterator)
+		tbm_end_shared_iterate(node->shared_tbmiterator);
+	if (node->shared_prefetch_iterator)
+		tbm_end_shared_iterate(node->shared_prefetch_iterator);
 
 	/*
 	 * close heap scan
@@ -597,6 +835,10 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->pscan_len = 0;
+	scanstate->initialized = false;
+	scanstate->shared_tbmiterator = NULL;
+	scanstate->pstate = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -681,3 +923,127 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ *		BitmapShouldInitializeSharedState
+ *
+ *		The first process to come here and see the state to the BM_INITIAL
+ *		will become the leader for the parallel bitmap scan and will be
+ *		responsible for populating the TIDBitmap.  The other processes will
+ *		be blocked by the condition variable until the leader wakes them up.
+ * ---------------
+ */
+static bool
+BitmapShouldInitializeSharedState(ParallelBitmapHeapState *pstate)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * BM_INITIAL	  : We become the leader and set it to BM_INPROGRESS
+		 * BM_INPROGRESS : We need to wait till leader creates bitmap
+		 * BM_FINISHED	 : bitmap is ready so no need to wait
+		 *---------------
+		 */
+		SpinLockAcquire(&pstate->mutex);
+
+		if (pstate->state == BM_INITIAL)
+		{
+			pstate->state = BM_INPROGRESS;
+			leader = true;
+		}
+		else if (pstate->state == BM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pstate->mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pstate->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+
+	/* Estimate the size for sharing parallel Bitmap info. */
+	node->pscan_len = add_size(offsetof(ParallelBitmapHeapState,
+										phs_snapshot_data),
+							   EstimateSnapshotSpace(estate->es_snapshot));
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapHeapState *pstate;
+	EState	   *estate = node->ss.ps.state;
+
+	pstate = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	pstate->tbmiterator = 0;
+	pstate->prefetch_iterator = 0;
+
+	/* Initialize the mutex */
+	SpinLockInit(&pstate->mutex);
+	pstate->prefetch_pages = 0;
+	pstate->prefetch_target = 0;
+	pstate->prefetching = false;
+	pstate->state = BM_INITIAL;
+
+	ConditionVariableInit(&pstate->cv);
+	SerializeSnapshot(estate->es_snapshot, pstate->phs_snapshot_data);
+
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pstate);
+	node->pstate = pstate;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapHeapState *pstate;
+	Snapshot	snapshot;
+
+	pstate = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->pstate = pstate;
+
+	snapshot = RestoreSnapshot(pstate->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 94bb289..ce2f321 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,9 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L, NULL);
+		tbm = tbm_create(work_mem * 1024L,
+						 ((BitmapIndexScan *) node->ss.ps.plan)->isshared ?
+						 node->ss.ps.state->es_query_dsa : NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 1d280be..c0f2614 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,9 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L, NULL);
+				result = tbm_create(work_mem * 1024L,
+									((BitmapOr *) node->ps.plan)->isshared ?
+									node->ps.state->es_query_dsa : NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 05d8538..89afc7a 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -331,6 +331,7 @@ _copyBitmapOr(const BitmapOr *from)
 	/*
 	 * copy remainder of node
 	 */
+	COPY_SCALAR_FIELD(isshared);
 	COPY_NODE_FIELD(bitmapplans);
 
 	return newnode;
@@ -496,6 +497,7 @@ _copyBitmapIndexScan(const BitmapIndexScan *from)
 	 * copy remainder of node
 	 */
 	COPY_SCALAR_FIELD(indexid);
+	COPY_SCALAR_FIELD(isshared);
 	COPY_NODE_FIELD(indexqual);
 	COPY_NODE_FIELD(indexqualorig);
 
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b3802b4..0ec4e1b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -441,6 +441,7 @@ _outBitmapOr(StringInfo str, const BitmapOr *node)
 
 	_outPlanInfo(str, (const Plan *) node);
 
+	WRITE_BOOL_FIELD(isshared);
 	WRITE_NODE_FIELD(bitmapplans);
 }
 
@@ -520,6 +521,7 @@ _outBitmapIndexScan(StringInfo str, const BitmapIndexScan *node)
 	_outScanInfo(str, (const Scan *) node);
 
 	WRITE_OID_FIELD(indexid);
+	WRITE_BOOL_FIELD(isshared);
 	WRITE_NODE_FIELD(indexqual);
 	WRITE_NODE_FIELD(indexqualorig);
 }
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 05bf2e9..d0d68cc 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1605,6 +1605,7 @@ _readBitmapOr(void)
 
 	ReadCommonPlan(&local_node->plan);
 
+	READ_BOOL_FIELD(isshared);
 	READ_NODE_FIELD(bitmapplans);
 
 	READ_DONE();
@@ -1716,6 +1717,7 @@ _readBitmapIndexScan(void)
 	ReadCommonScan(&local_node->scan);
 
 	READ_OID_FIELD(indexid);
+	READ_BOOL_FIELD(isshared);
 	READ_NODE_FIELD(indexqual);
 	READ_NODE_FIELD(indexqualorig);
 
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 633b5c1..1fef1bd 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2875,6 +2875,30 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 }
 
 /*
+ * create_partial_bitmap_paths
+ *	  Build partial bitmap heap path for the relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched, 0);
+
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+/*
  * Compute the number of parallel workers that should be used to scan a
  * relation.  We compute the parallel workers based on the size of the heap to
  * be scanned and the size of the index to be scanned, then choose a minimum
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index c138f57..b8cde32 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -860,6 +860,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -921,8 +922,21 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
+
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index d8e5b81..c2b72d4 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1617,6 +1621,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1659,6 +1668,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 1e953b4..eaf85da 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -125,6 +125,7 @@ static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
 						List *tlist, List *scan_clauses);
 static Plan *create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 					  List **qual, List **indexqual, List **indexECs);
+static void bitmap_subplan_mark_shared(Plan *plan);
 static TidScan *create_tidscan_plan(PlannerInfo *root, TidPath *best_path,
 					List *tlist, List *scan_clauses);
 static SubqueryScan *create_subqueryscan_plan(PlannerInfo *root,
@@ -2577,6 +2578,9 @@ create_bitmap_scan_plan(PlannerInfo *root,
 										   &bitmapqualorig, &indexquals,
 										   &indexECs);
 
+	if (best_path->path.parallel_aware)
+		bitmap_subplan_mark_shared(bitmapqualplan);
+
 	/*
 	 * The qpqual list must contain all restrictions not automatically handled
 	 * by the index, other than pseudoconstant clauses which will be handled
@@ -2712,6 +2716,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 			subplan = create_bitmap_subplan(root, (Path *) lfirst(l),
 											&subqual, &subindexqual,
 											&subindexEC);
+
 			subplans = lappend(subplans, subplan);
 			subquals = list_concat_unique(subquals, subqual);
 			subindexquals = list_concat_unique(subindexquals, subindexqual);
@@ -4700,6 +4705,24 @@ label_sort_with_costsize(PlannerInfo *root, Sort *plan, double limit_tuples)
 	plan->plan.parallel_aware = false;
 }
 
+/*
+ * bitmap_subplan_mark_shared
+ *	 Mark a shared flag in bitmap subplan so that it can create underlying
+ *	 bitmap in shared memory which is accecible by the multiple processes.
+ */
+static void
+bitmap_subplan_mark_shared(Plan *plan)
+{
+	if (IsA(plan, BitmapAnd))
+		bitmap_subplan_mark_shared(
+								linitial(((BitmapAnd *) plan)->bitmapplans));
+	else if (IsA(plan, BitmapOr))
+		((BitmapOr *) plan)->isshared = true;
+	else if (IsA(plan, BitmapIndexScan))
+		((BitmapIndexScan *) plan)->isshared = true;
+	else
+		elog(ERROR, "unrecognized node type: %d", nodeTag(plan));
+}
 
 /*****************************************************************************
  *
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3248296..6e70808 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1068,7 +1068,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1077,9 +1078,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3255,7 +3256,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 2fb9a8b..7cacb1e 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3395,6 +3395,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index a864f78..7e85510 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -179,6 +179,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 6332ea0..fc18f38 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1465,6 +1467,54 @@ typedef struct BitmapIndexScanState
 } BitmapIndexScanState;
 
 /* ----------------
+ *	 SharedBitmapState information
+ *
+ *		BM_INITIAL		TIDBitmap creation is not yet started, so first worker
+ *						to see this state will set the state to BM_INPROGRESS
+ *						and that process will be responsible for creating
+ *						TIDBitmap.
+ *		BM_INPROGRESS	TIDBitmap creation is already in progress, therefore
+ *						workers need to sleep until leader set the state to
+ *						BM_FINISHED and wake us up.
+ *		BM_FINISHED		TIDBitmap creation is done, so now all worker can
+ *						proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	BM_INITIAL,
+	BM_INPROGRESS,
+	BM_FINISHED
+} SharedBitmapState;
+
+/* ----------------
+ *	 ParallelBitmapHeapState information
+ *		tbmiterator				iterator for scanning current pages
+ *		prefetch_iterator		iterator for prefetching ahead of current page
+ *		mutex					mutual exclusion for the prefetching variable
+ *								and state
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		prefetching				set true if prefetching is in progress
+ *		state					current state of the TIDBitmap
+ *		cv						conditional wait variable
+ *		phs_snapshot_data		snapshot data shared to workers
+ * ----------------
+ */
+typedef struct ParallelBitmapHeapState
+{
+	dsa_pointer tbmiterator;
+	dsa_pointer prefetch_iterator;
+	slock_t		mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	bool		prefetching;
+	SharedBitmapState state;
+	ConditionVariable cv;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+} ParallelBitmapHeapState;
+
+/* ----------------
  *	 BitmapHeapScanState information
  *
  *		bitmapqualorig	   execution state for bitmapqualorig expressions
@@ -1477,6 +1527,11 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		initialized		   is node is ready to iterate
+ *		shared_tbmiterator	   shared iterator
+ *		shared_prefetch_iterator shared iterator for prefetching
+ *		pstate			   shared state for parallel bitmap scan
  * ----------------
  */
 typedef struct BitmapHeapScanState
@@ -1492,6 +1547,11 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	bool		initialized;
+	TBMSharedIterator *shared_tbmiterator;
+	TBMSharedIterator *shared_prefetch_iterator;
+	ParallelBitmapHeapState *pstate;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f72f7a8..7f38ca6 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -292,6 +292,7 @@ typedef struct BitmapAnd
 typedef struct BitmapOr
 {
 	Plan		plan;
+	bool		isshared;
 	List	   *bitmapplans;
 } BitmapOr;
 
@@ -420,6 +421,7 @@ typedef struct BitmapIndexScan
 {
 	Scan		scan;
 	Oid			indexid;		/* OID of index to scan */
+	bool		isshared;		/* Create shared bitmap if set */
 	List	   *indexqual;		/* list of index quals (OpExprs) */
 	List	   *indexqualorig;	/* the same in original form */
 } BitmapIndexScan;
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 53cad24..d22b039 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -53,7 +53,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index ebda308..3ec2d9d 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -56,6 +56,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
 extern int compute_parallel_worker(RelOptInfo *rel, BlockNumber heap_pages,
 						BlockNumber index_pages);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+										Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 0062fb8..60c78d1 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -787,6 +787,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#105Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#104)
Re: Parallel bitmap heap scan

On Mon, Mar 6, 2017 at 12:35 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Thu, Mar 2, 2017 at 6:52 PM, Robert Haas <robertmhaas@gmail.com> wrote:

0002 wasn't quite careful enough about the placement of #ifdef
USE_PREFETCH, but otherwise looks OK. Committed after changing that
and getting rid of the local variable prefetch_iterator, which seemed
to be adding rather than removing complexity after this refactoring.

0003 is rebased after this commit.

You've still got this:

+               if (DsaPointerIsValid(node->pstate->tbmiterator))
+                       tbm_free_shared_area(dsa, node->pstate->tbmiterator);
+
+               if (DsaPointerIsValid(node->pstate->prefetch_iterator))
+                       dsa_free(dsa, node->pstate->prefetch_iterator);

I'm trying to get to a point where both calls use
tbm_free_shared_area() - i.e. no peeking behind the abstraction layer.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#106Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#105)
Re: Parallel bitmap heap scan

On Tue, Mar 7, 2017 at 9:34 PM, Robert Haas <robertmhaas@gmail.com> wrote:

+               if (DsaPointerIsValid(node->pstate->tbmiterator))
+                       tbm_free_shared_area(dsa, node->pstate->tbmiterator);
+
+               if (DsaPointerIsValid(node->pstate->prefetch_iterator))
+                       dsa_free(dsa, node->pstate->prefetch_iterator);

As per latest code, both should be calling to tbm_free_shared_area
because tbm_free_shared_area is capable of handling that. My silly
mistake. I will fix it.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#107Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#102)
Re: Parallel bitmap heap scan

(On Tue, Feb 28, 2017 at 10:48 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

0001- same as previous with some changes for freeing the shared memory stuff.

+    if (--ptbase->refcount == 0)
+        dsa_free(dsa, istate->pagetable);
+
+    if (istate->spages)
+    {
+        ptpages = dsa_get_address(dsa, istate->spages);
+        if (--ptpages->refcount == 0)
+            dsa_free(dsa, istate->spages);
+    }
+    if (istate->schunks)
+    {
+        ptchunks = dsa_get_address(dsa, istate->schunks);
+        if (--ptchunks->refcount == 0)
+            dsa_free(dsa, istate->schunks);
+    }

This doesn't involve any locking, which I think will happen to work
with the current usage pattern but doesn't seem very robust in
general. I think you either need the refcounts to be protected by a
spinlock, or maybe better, use pg_atomic_uint32 for them. You want
something like if (pg_atomic_sub_fetch_u32(&refcount, 1) == 0) {
dsa_free(...) }

Otherwise, there's no guarantee it will get freed exactly once.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#108Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#106)
Re: Parallel bitmap heap scan

On Tue, Mar 7, 2017 at 11:09 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Tue, Mar 7, 2017 at 9:34 PM, Robert Haas <robertmhaas@gmail.com> wrote:

+               if (DsaPointerIsValid(node->pstate->tbmiterator))
+                       tbm_free_shared_area(dsa, node->pstate->tbmiterator);
+
+               if (DsaPointerIsValid(node->pstate->prefetch_iterator))
+                       dsa_free(dsa, node->pstate->prefetch_iterator);

As per latest code, both should be calling to tbm_free_shared_area
because tbm_free_shared_area is capable of handling that. My silly
mistake. I will fix it.

Thanks. I don't think I believe this rationale:

+                               /*
+                                * If one of the process has already
identified that we need
+                                * to do prefetch then let it perform
the prefetch and allow
+                                * others to proceed with the work in
hand.  Another option
+                                * could be that we allow all of them
to participate in
+                                * prefetching.  But, most of this
work done under mutex or
+                                * LWLock so ultimately we may end up
in prefetching
+                                * sequentially.
+                                */

I mean, IIUC, the call to PrefetchBuffer() is not done under any lock.
And that's the slow part. The tiny amount of time we spend updating
the prefetch information under the mutex should be insignificant
compared to the cost of actually reading the buffer. Unless I'm
missing something.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#109Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#108)
Re: Parallel bitmap heap scan

On Tue, Mar 7, 2017 at 9:44 PM, Robert Haas <robertmhaas@gmail.com> wrote:

I mean, IIUC, the call to PrefetchBuffer() is not done under any lock.
And that's the slow part. The tiny amount of time we spend updating
the prefetch information under the mutex should be insignificant
compared to the cost of actually reading the buffer. Unless I'm
missing something.

Okay, but IIUC, the PrefetchBuffer is an async call to load the buffer
if it's not already in shared buffer? so If instead of one process is
making multiple async calls to PrefetchBuffer, if we make it by
multiple processes will it be any faster? Or you are thinking that at
least we can make BufTableLookup call parallel because that is not an
async call.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#110Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#109)
Re: Parallel bitmap heap scan

On Tue, Mar 7, 2017 at 11:27 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Tue, Mar 7, 2017 at 9:44 PM, Robert Haas <robertmhaas@gmail.com> wrote:

I mean, IIUC, the call to PrefetchBuffer() is not done under any lock.
And that's the slow part. The tiny amount of time we spend updating
the prefetch information under the mutex should be insignificant
compared to the cost of actually reading the buffer. Unless I'm
missing something.

Okay, but IIUC, the PrefetchBuffer is an async call to load the buffer
if it's not already in shared buffer? so If instead of one process is
making multiple async calls to PrefetchBuffer, if we make it by
multiple processes will it be any faster? Or you are thinking that at
least we can make BufTableLookup call parallel because that is not an
async call.

It's not about speed. It's about not forgetting to prefetch. Suppose
that worker 1 becomes the prefetch worker but then doesn't return to
the Bitmap Heap Scan node for a long time because it's busy in some
other part of the plan tree. Now you just stop prefetching; that's
bad. You want prefetching to continue regardless of which workers are
busy doing what; as long as SOME worker is executing the parallel
bitmap heap scan, prefetching should continue as needed.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#111Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#110)
Re: Parallel bitmap heap scan

On Tue, Mar 7, 2017 at 10:07 PM, Robert Haas <robertmhaas@gmail.com> wrote:

It's not about speed. It's about not forgetting to prefetch. Suppose
that worker 1 becomes the prefetch worker but then doesn't return to
the Bitmap Heap Scan node for a long time because it's busy in some
other part of the plan tree. Now you just stop prefetching; that's
bad. You want prefetching to continue regardless of which workers are
busy doing what; as long as SOME worker is executing the parallel
bitmap heap scan, prefetching should continue as needed.

Right, I missed this part. I will fix this.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#112Dilip Kumar
dilipbalaut@gmail.com
In reply to: Dilip Kumar (#111)
2 attachment(s)
Re: Parallel bitmap heap scan

On Tue, Mar 7, 2017 at 10:10 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

It's not about speed. It's about not forgetting to prefetch. Suppose
that worker 1 becomes the prefetch worker but then doesn't return to
the Bitmap Heap Scan node for a long time because it's busy in some
other part of the plan tree. Now you just stop prefetching; that's
bad. You want prefetching to continue regardless of which workers are
busy doing what; as long as SOME worker is executing the parallel
bitmap heap scan, prefetching should continue as needed.

Right, I missed this part. I will fix this.

I have fixed this part, after doing that I realised if multiple
processes are prefetching then it may be possible that in boundary
cases (e.g. suppose prefetch_target is 3 and prefetch_pages is at 2)
there may be some extra prefetch but finally those prefetched blocks
will be used. Another, solution to this problem is that we can
increase the prefetch_pages in advance then call tbm_shared_iterate,
this will avoid extra prefetch. But I am not sure what will be best
here.

On Tue, Mar 7, 2017 at 9:41 PM, Robert Haas <robertmhaas@gmail.com> wrote:

+    if (--ptbase->refcount == 0)
+        dsa_free(dsa, istate->pagetable);
+
+    if (istate->spages)
+    {
+        ptpages = dsa_get_address(dsa, istate->spages);
+        if (--ptpages->refcount == 0)
+            dsa_free(dsa, istate->spages);
+    }
+    if (istate->schunks)
+    {
+        ptchunks = dsa_get_address(dsa, istate->schunks);
+        if (--ptchunks->refcount == 0)
+            dsa_free(dsa, istate->schunks);
+    }

This doesn't involve any locking, which I think will happen to work
with the current usage pattern but doesn't seem very robust in
general. I think you either need the refcounts to be protected by a
spinlock, or maybe better, use pg_atomic_uint32 for them. You want
something like if (pg_atomic_sub_fetch_u32(&refcount, 1) == 0) {
dsa_free(...) }

Otherwise, there's no guarantee it will get freed exactly once.

Fixed

On Tue, Mar 7, 2017 at 9:34 PM, Robert Haas <robertmhaas@gmail.com> wrote:

You've still got this:

+               if (DsaPointerIsValid(node->pstate->tbmiterator))
+                       tbm_free_shared_area(dsa, node->pstate->tbmiterator);
+
+               if (DsaPointerIsValid(node->pstate->prefetch_iterator))
+                       dsa_free(dsa, node->pstate->prefetch_iterator);

I'm trying to get to a point where both calls use
tbm_free_shared_area() - i.e. no peeking behind the abstraction layer.

Fixed

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0001-tidbitmap-support-shared-v8.patchapplication/octet-stream; name=0001-tidbitmap-support-shared-v8.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index fad5cb0..1d70fae 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1082,6 +1082,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel query dynamic shared memory allocation lock.</entry>
         </row>
         <row>
+         <entry><literal>tbm</></entry>
+         <entry>Waiting for TBM shared iterator lock.</entry>
+        </row>
+        <row>
          <entry morerows="9"><literal>Lock</></entry>
          <entry><literal>relation</></entry>
          <entry>Waiting to acquire a lock on a relation.</entry>
diff --git a/src/backend/access/gin/ginget.c b/src/backend/access/gin/ginget.c
index 60f005c..87cd9ea 100644
--- a/src/backend/access/gin/ginget.c
+++ b/src/backend/access/gin/ginget.c
@@ -120,7 +120,7 @@ collectMatchBitmap(GinBtreeData *btree, GinBtreeStack *stack,
 	Form_pg_attribute attr;
 
 	/* Initialize empty bitmap result */
-	scanEntry->matchBitmap = tbm_create(work_mem * 1024L);
+	scanEntry->matchBitmap = tbm_create(work_mem * 1024L, NULL);
 
 	/* Null query cannot partial-match anything */
 	if (scanEntry->isPartialMatch &&
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 4274e9a..94bb289 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,7 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L);
+		tbm = tbm_create(work_mem * 1024L, NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 3c6155b..1d280be 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,7 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L);
+				result = tbm_create(work_mem * 1024L, NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index 0885812..9dcef32 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -43,6 +43,8 @@
 #include "access/htup_details.h"
 #include "nodes/bitmapset.h"
 #include "nodes/tidbitmap.h"
+#include "storage/lwlock.h"
+#include "utils/dsa.h"
 
 /*
  * The maximum number of tuples per page is not large (typically 256 with
@@ -103,6 +105,15 @@ typedef struct PagetableEntry
 } PagetableEntry;
 
 /*
+ * Holds array of pagetable entries.
+ */
+typedef struct PTEntryArray
+{
+	pg_atomic_uint32	refcount;		/* no. of iterator attached */
+	PagetableEntry ptentry[FLEXIBLE_ARRAY_MEMBER];
+} PTEntryArray;
+
+/*
  * We want to avoid the overhead of creating the hashtable, which is
  * comparatively large, when not necessary. Particularly when we are using a
  * bitmap scan on the inside of a nestloop join: a bitmap may well live only
@@ -121,6 +132,16 @@ typedef enum
 } TBMStatus;
 
 /*
+ * Current iterating state of the TBM.
+ */
+typedef enum
+{
+	TBM_NOT_ITERATING,			/* not yet converted to page and chunk array */
+	TBM_ITERATING_PRIVATE,		/* converted to local page and chunk array */
+	TBM_ITERATING_SHARED		/* converted to shared page and chunk array */
+} TBMIteratingState;
+
+/*
  * Here is the representation for a whole TIDBitMap:
  */
 struct TIDBitmap
@@ -133,12 +154,17 @@ struct TIDBitmap
 	int			maxentries;		/* limit on same to meet maxbytes */
 	int			npages;			/* number of exact entries in pagetable */
 	int			nchunks;		/* number of lossy entries in pagetable */
-	bool		iterating;		/* tbm_begin_iterate called? */
+	TBMIteratingState iterating;	/* tbm_begin_iterate called? */
 	uint32		lossify_start;	/* offset to start lossifying hashtable at */
 	PagetableEntry entry1;		/* used when status == TBM_ONE_PAGE */
 	/* these are valid when iterating is true: */
 	PagetableEntry **spages;	/* sorted exact-page list, or NULL */
 	PagetableEntry **schunks;	/* sorted lossy-chunk list, or NULL */
+	dsa_pointer dsapagetable;	/* dsa_pointer to the element array */
+	dsa_pointer dsapagetableold;	/* dsa_pointer to the old element array */
+	dsa_pointer ptpages;		/* dsa_pointer to the page array */
+	dsa_pointer ptchunks;		/* dsa_pointer to the chunk array */
+	dsa_area   *dsa;			/* reference to per-query dsa area */
 };
 
 /*
@@ -156,6 +182,46 @@ struct TBMIterator
 	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
 };
 
+/*
+ * Holds the shared members of the iterator so that multiple processes
+ * can jointly iterate.
+ */
+typedef struct TBMSharedIteratorState
+{
+	int			nentries;		/* number of entries in pagetable */
+	int			maxentries;		/* limit on same to meet maxbytes */
+	int			npages;			/* number of exact entries in pagetable */
+	int			nchunks;		/* number of lossy entries in pagetable */
+	dsa_pointer pagetable;		/* dsa pointers to head of pagetable data */
+	dsa_pointer spages;			/* dsa pointer to page array */
+	dsa_pointer schunks;		/* dsa pointer to chunk array */
+	LWLock		lock;			/* lock to protect below members */
+	int			spageptr;		/* next spages index */
+	int			schunkptr;		/* next schunks index */
+	int			schunkbit;		/* next bit to check in current schunk */
+} TBMSharedIteratorState;
+
+/*
+ * pagetable iteration array.
+ */
+typedef struct PTIterationArray
+{
+	pg_atomic_uint32			refcount;	/* no. of iterator attached */
+	int			index[FLEXIBLE_ARRAY_MEMBER];	/* index array */
+} PTIterationArray;
+
+/*
+ * same as TBMIterator, but it is used for joint iteration, therefore this
+ * also holds a reference to the shared state.
+ */
+struct TBMSharedIterator
+{
+	TBMSharedIteratorState *state;		/* shared state */
+	PTEntryArray *ptbase;		/* pagetable element array */
+	PTIterationArray *ptpages;	/* sorted exact page index list */
+	PTIterationArray *ptchunks; /* sorted lossy page index list */
+	TBMIterateResult output;	/* MUST BE LAST (because variable-size) */
+};
 
 /* Local function prototypes */
 static void tbm_union_page(TIDBitmap *a, const PagetableEntry *bpage);
@@ -168,6 +234,8 @@ static bool tbm_page_is_lossy(const TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_mark_page_lossy(TIDBitmap *tbm, BlockNumber pageno);
 static void tbm_lossify(TIDBitmap *tbm);
 static int	tbm_comparator(const void *left, const void *right);
+static int tbm_shared_comparator(const void *left, const void *right,
+					  void *arg);
 
 /*
  * Simple inline murmur hash implementation for the exact width required, for
@@ -187,6 +255,7 @@ hash_blockno(BlockNumber b)
 }
 
 /* define hashtable mapping block numbers to PagetableEntry's */
+#define SH_USE_NONDEFAULT_ALLOCATOR
 #define SH_PREFIX pagetable
 #define SH_ELEMENT_TYPE PagetableEntry
 #define SH_KEY_TYPE BlockNumber
@@ -204,10 +273,12 @@ hash_blockno(BlockNumber b)
  *
  * The bitmap will live in the memory context that is CurrentMemoryContext
  * at the time of this call.  It will be limited to (approximately) maxbytes
- * total memory consumption.
+ * total memory consumption.  If the DSA passed to this function is not NULL
+ * then the memory for storing elements of the underlying page table will
+ * be allocated from the DSA.
  */
 TIDBitmap *
-tbm_create(long maxbytes)
+tbm_create(long maxbytes, dsa_area *dsa)
 {
 	TIDBitmap  *tbm;
 	long		nbuckets;
@@ -230,6 +301,7 @@ tbm_create(long maxbytes)
 	nbuckets = Max(nbuckets, 16);		/* sanity limit */
 	tbm->maxentries = (int) nbuckets;
 	tbm->lossify_start = 0;
+	tbm->dsa = dsa;
 
 	return tbm;
 }
@@ -244,7 +316,7 @@ tbm_create_pagetable(TIDBitmap *tbm)
 	Assert(tbm->status != TBM_HASH);
 	Assert(tbm->pagetable == NULL);
 
-	tbm->pagetable = pagetable_create(tbm->mcxt, 128, NULL);
+	tbm->pagetable = pagetable_create(tbm->mcxt, 128, tbm);
 
 	/* If entry1 is valid, push it into the hashtable */
 	if (tbm->status == TBM_ONE_PAGE)
@@ -281,6 +353,40 @@ tbm_free(TIDBitmap *tbm)
 }
 
 /*
+ * tbm_free_shared_area - free shared state
+ *
+ * Free shared iterator state, Also free shared pagetable and iterator arrays
+ * memory if they are not referred by any of the shared iterator i.e recount
+ * is becomes 0.
+ */
+void
+tbm_free_shared_area(dsa_area *dsa, dsa_pointer dp)
+{
+	TBMSharedIteratorState *istate = dsa_get_address(dsa, dp);
+	PTEntryArray *ptbase = dsa_get_address(dsa, istate->pagetable);
+	PTIterationArray *ptpages;
+	PTIterationArray *ptchunks;
+
+	if (pg_atomic_sub_fetch_u32(&ptbase->refcount, 1) == 0)
+		dsa_free(dsa, istate->pagetable);
+
+	if (istate->spages)
+	{
+		ptpages = dsa_get_address(dsa, istate->spages);
+		if (pg_atomic_sub_fetch_u32(&ptpages->refcount, 1) == 0)
+			dsa_free(dsa, istate->spages);
+	}
+	if (istate->schunks)
+	{
+		ptchunks = dsa_get_address(dsa, istate->schunks);
+		if (pg_atomic_sub_fetch_u32(&ptchunks->refcount, 1) == 0)
+			dsa_free(dsa, istate->schunks);
+	}
+
+	dsa_free(dsa, dp);
+}
+
+/*
  * tbm_add_tuples - add some tuple IDs to a TIDBitmap
  *
  * If recheck is true, then the recheck flag will be set in the
@@ -294,7 +400,7 @@ tbm_add_tuples(TIDBitmap *tbm, const ItemPointer tids, int ntids,
 	PagetableEntry *page = NULL;	/* only valid when currblk is valid */
 	int			i;
 
-	Assert(!tbm->iterating);
+	Assert(tbm->iterating == TBM_NOT_ITERATING);
 	for (i = 0; i < ntids; i++)
 	{
 		BlockNumber blk = ItemPointerGetBlockNumber(tids + i);
@@ -603,6 +709,8 @@ tbm_begin_iterate(TIDBitmap *tbm)
 {
 	TBMIterator *iterator;
 
+	Assert(tbm->iterating != TBM_ITERATING_SHARED);
+
 	/*
 	 * Create the TBMIterator struct, with enough trailing space to serve the
 	 * needs of the TBMIterateResult sub-struct.
@@ -624,7 +732,7 @@ tbm_begin_iterate(TIDBitmap *tbm)
 	 * attached to the bitmap not the iterator, so they can be used by more
 	 * than one iterator.
 	 */
-	if (tbm->status == TBM_HASH && !tbm->iterating)
+	if (tbm->status == TBM_HASH && tbm->iterating == TBM_NOT_ITERATING)
 	{
 		pagetable_iterator i;
 		PagetableEntry *page;
@@ -659,12 +767,211 @@ tbm_begin_iterate(TIDBitmap *tbm)
 				  tbm_comparator);
 	}
 
-	tbm->iterating = true;
+	tbm->iterating = TBM_ITERATING_PRIVATE;
 
 	return iterator;
 }
 
 /*
+ * tbm_prepare_shared_iterate - prepare shared iteration state for a TIDBitmap.
+ *
+ * The necessary shared state will be allocated from the DSA passed to
+ * tbm_create, so that multiple processes can attach to it and iterate jointly.
+ *
+ * This will convert the pagetable hash into page and chunk array of the index
+ * into pagetable array.
+ */
+dsa_pointer
+tbm_prepare_shared_iterate(TIDBitmap *tbm)
+{
+	dsa_pointer dp;
+	TBMSharedIteratorState *istate;
+	PTEntryArray *ptbase;
+	PTIterationArray *ptpages;
+	PTIterationArray *ptchunks;
+
+	Assert(tbm->dsa != NULL);
+	Assert(tbm->iterating != TBM_ITERATING_PRIVATE);
+
+	/*
+	 * Allocate TBMSharedIteratorState from DSA to hold the shared members and
+	 * lock, this will also be used by multiple worker for shared iterate.
+	 */
+	dp = dsa_allocate(tbm->dsa, sizeof(TBMSharedIteratorState));
+	istate = dsa_get_address(tbm->dsa, dp);
+
+	/*
+	 * If we're not already iterating, create and fill the sorted page lists.
+	 * (If we are, the sorted page lists are already stored in the TIDBitmap,
+	 * and we can just reuse them.)
+	 */
+	if (tbm->iterating == TBM_NOT_ITERATING)
+	{
+		pagetable_iterator i;
+		PagetableEntry *page;
+		int			idx;
+		int			npages;
+		int			nchunks;
+
+		/*
+		 * Allocate the page and chunk array memory from the DSA to share
+		 * across multiple processes.
+		 */
+		if (tbm->npages)
+		{
+			tbm->ptpages = dsa_allocate(tbm->dsa, sizeof(PTIterationArray) +
+										tbm->npages * sizeof(int));
+			ptpages = dsa_get_address(tbm->dsa, tbm->ptpages);
+			pg_atomic_init_u32(&ptpages->refcount, 0);
+		}
+		if (tbm->nchunks)
+		{
+			tbm->ptchunks = dsa_allocate(tbm->dsa, sizeof(PTIterationArray) +
+										 tbm->nchunks * sizeof(int));
+			ptchunks = dsa_get_address(tbm->dsa, tbm->ptchunks);
+			pg_atomic_init_u32(&ptchunks->refcount, 0);
+		}
+
+		/*
+		 * If TBM status is TBM_HASH then iterate over the pagetable and
+		 * convert it to page and chunk arrays.  But if it's in the
+		 * TBM_ONE_PAGE mode then directly allocate the space for one entry
+		 * from the DSA.
+		 */
+		npages = nchunks = 0;
+		if (tbm->status == TBM_HASH)
+		{
+			ptbase = dsa_get_address(tbm->dsa, tbm->dsapagetable);
+
+			pagetable_start_iterate(tbm->pagetable, &i);
+			while ((page = pagetable_iterate(tbm->pagetable, &i)) != NULL)
+			{
+				idx = page - ptbase->ptentry;
+				if (page->ischunk)
+					ptchunks->index[nchunks++] = idx;
+				else
+					ptpages->index[npages++] = idx;
+			}
+
+			Assert(npages == tbm->npages);
+			Assert(nchunks == tbm->nchunks);
+		}
+		else
+		{
+			/*
+			 * In one page mode allocate the space for one pagetable entry and
+			 * directly store its index i.e. 0 in page array
+			 */
+			tbm->dsapagetable = dsa_allocate(tbm->dsa, sizeof(PTEntryArray) +
+											 sizeof(PagetableEntry));
+			ptbase = dsa_get_address(tbm->dsa, tbm->dsapagetable);
+			ptpages->index[0] = 0;
+		}
+
+		pg_atomic_init_u32(&ptbase->refcount, 0);
+
+		if (npages > 1)
+			qsort_arg((void *) (ptpages->index), npages, sizeof(int),
+					  tbm_shared_comparator, (void *) ptbase->ptentry);
+		if (nchunks > 1)
+			qsort_arg((void *) (ptchunks->index), nchunks, sizeof(int),
+					  tbm_shared_comparator, (void *) ptbase->ptentry);
+	}
+
+	/*
+	 * Store the TBM members in the shared state so that we can share them
+	 * across multiple processes.
+	 */
+	istate->nentries = tbm->nentries;
+	istate->maxentries = tbm->maxentries;
+	istate->npages = tbm->npages;
+	istate->nchunks = tbm->nchunks;
+	istate->pagetable = tbm->dsapagetable;
+	istate->spages = tbm->ptpages;
+	istate->schunks = tbm->ptchunks;
+
+	ptbase = dsa_get_address(tbm->dsa, tbm->dsapagetable);
+	ptpages = dsa_get_address(tbm->dsa, tbm->ptpages);
+	ptchunks = dsa_get_address(tbm->dsa, tbm->ptchunks);
+
+	/*
+	 * For every shared iterator, referring to pagetable and iterator array,
+	 * increase the refcount by 1 so that while freeing the shared iterator
+	 * we don't free pagetable and iterator array until its refcount becomes 0.
+	 */
+	pg_atomic_add_fetch_u32(&ptbase->refcount, 1);
+	if (ptpages)
+		pg_atomic_add_fetch_u32(&ptpages->refcount, 1);
+	if (ptchunks)
+		pg_atomic_add_fetch_u32(&ptchunks->refcount, 1);
+
+	/* Initialize the iterator lock */
+	LWLockInitialize(&istate->lock, LWTRANCHE_TBM);
+
+	/* Initialize the shared iterator state */
+	istate->schunkbit = 0;
+	istate->schunkptr = 0;
+	istate->spageptr = 0;
+
+	tbm->iterating = TBM_ITERATING_SHARED;
+
+	return dp;
+}
+
+/*
+ * tbm_extract_page_tuple - extract the tuple offsets from a page
+ *
+ * The extracted offsets are stored into TBMIterateResult.
+ */
+static inline int
+tbm_extract_page_tuple(PagetableEntry *page, TBMIterateResult *output)
+{
+	int			wordnum;
+	int			ntuples = 0;
+
+	for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
+	{
+		bitmapword	w = page->words[wordnum];
+
+		if (w != 0)
+		{
+			int			off = wordnum * BITS_PER_BITMAPWORD + 1;
+
+			while (w != 0)
+			{
+				if (w & 1)
+					output->offsets[ntuples++] = (OffsetNumber) off;
+				off++;
+				w >>= 1;
+			}
+		}
+	}
+
+	return ntuples;
+}
+
+/*
+ *	tbm_advance_schunkbit - Advance the chunkbit
+ */
+static inline void
+tbm_advance_schunkbit(PagetableEntry *chunk, int *schunkbitp)
+{
+	int			schunkbit = *schunkbitp;
+
+	while (schunkbit < PAGES_PER_CHUNK)
+	{
+		int			wordnum = WORDNUM(schunkbit);
+		int			bitnum = BITNUM(schunkbit);
+
+		if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
+			break;
+		schunkbit++;
+	}
+
+	*schunkbitp = schunkbit;
+}
+
+/*
  * tbm_iterate - scan through next page of a TIDBitmap
  *
  * Returns a TBMIterateResult representing one page, or NULL if there are
@@ -682,7 +989,7 @@ tbm_iterate(TBMIterator *iterator)
 	TIDBitmap  *tbm = iterator->tbm;
 	TBMIterateResult *output = &(iterator->output);
 
-	Assert(tbm->iterating);
+	Assert(tbm->iterating == TBM_ITERATING_PRIVATE);
 
 	/*
 	 * If lossy chunk pages remain, make sure we've advanced schunkptr/
@@ -693,15 +1000,7 @@ tbm_iterate(TBMIterator *iterator)
 		PagetableEntry *chunk = tbm->schunks[iterator->schunkptr];
 		int			schunkbit = iterator->schunkbit;
 
-		while (schunkbit < PAGES_PER_CHUNK)
-		{
-			int			wordnum = WORDNUM(schunkbit);
-			int			bitnum = BITNUM(schunkbit);
-
-			if ((chunk->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
-				break;
-			schunkbit++;
-		}
+		tbm_advance_schunkbit(chunk, &schunkbit);
 		if (schunkbit < PAGES_PER_CHUNK)
 		{
 			iterator->schunkbit = schunkbit;
@@ -738,7 +1037,6 @@ tbm_iterate(TBMIterator *iterator)
 	{
 		PagetableEntry *page;
 		int			ntuples;
-		int			wordnum;
 
 		/* In ONE_PAGE state, we don't allocate an spages[] array */
 		if (tbm->status == TBM_ONE_PAGE)
@@ -747,24 +1045,7 @@ tbm_iterate(TBMIterator *iterator)
 			page = tbm->spages[iterator->spageptr];
 
 		/* scan bitmap to extract individual offset numbers */
-		ntuples = 0;
-		for (wordnum = 0; wordnum < WORDS_PER_PAGE; wordnum++)
-		{
-			bitmapword	w = page->words[wordnum];
-
-			if (w != 0)
-			{
-				int			off = wordnum * BITS_PER_BITMAPWORD + 1;
-
-				while (w != 0)
-				{
-					if (w & 1)
-						output->offsets[ntuples++] = (OffsetNumber) off;
-					off++;
-					w >>= 1;
-				}
-			}
-		}
+		ntuples = tbm_extract_page_tuple(page, output);
 		output->blockno = page->blockno;
 		output->ntuples = ntuples;
 		output->recheck = page->recheck;
@@ -777,6 +1058,94 @@ tbm_iterate(TBMIterator *iterator)
 }
 
 /*
+ *	tbm_shared_iterate - scan through next page of a TIDBitmap
+ *
+ *	As above, but this will iterate using an iterator which is shared
+ *	across multiple processes.  We need to acquire the iterator LWLock,
+ *	before accessing the shared members.
+ */
+TBMIterateResult *
+tbm_shared_iterate(TBMSharedIterator *iterator)
+{
+	TBMIterateResult *output = &iterator->output;
+	TBMSharedIteratorState *istate = iterator->state;
+	PagetableEntry *ptbase = iterator->ptbase->ptentry;
+	int		   *idxpages = iterator->ptpages->index;
+	int		   *idxchunks = iterator->ptchunks->index;
+
+	/* Acquire the LWLock before accessing the shared members */
+	LWLockAcquire(&istate->lock, LW_EXCLUSIVE);
+
+	/*
+	 * If lossy chunk pages remain, make sure we've advanced schunkptr/
+	 * schunkbit to the next set bit.
+	 */
+	while (istate->schunkptr < istate->nchunks)
+	{
+		PagetableEntry *chunk = &ptbase[idxchunks[istate->schunkptr]];
+		int			schunkbit = istate->schunkbit;
+
+		tbm_advance_schunkbit(chunk, &schunkbit);
+		if (schunkbit < PAGES_PER_CHUNK)
+		{
+			istate->schunkbit = schunkbit;
+			break;
+		}
+		/* advance to next chunk */
+		istate->schunkptr++;
+		istate->schunkbit = 0;
+	}
+
+	/*
+	 * If both chunk and per-page data remain, must output the numerically
+	 * earlier page.
+	 */
+	if (istate->schunkptr < istate->nchunks)
+	{
+		PagetableEntry *chunk = &ptbase[idxchunks[istate->schunkptr]];
+		PagetableEntry *page = &ptbase[idxpages[istate->spageptr]];
+		BlockNumber chunk_blockno;
+
+		chunk_blockno = chunk->blockno + istate->schunkbit;
+
+		if (istate->spageptr >= istate->npages ||
+			chunk_blockno < page->blockno)
+		{
+			/* Return a lossy page indicator from the chunk */
+			output->blockno = chunk_blockno;
+			output->ntuples = -1;
+			output->recheck = true;
+			istate->schunkbit++;
+
+			LWLockRelease(&istate->lock);
+			return output;
+		}
+	}
+
+	if (istate->spageptr < istate->npages)
+	{
+		PagetableEntry *page = &ptbase[idxpages[istate->spageptr]];
+		int			ntuples;
+
+		/* scan bitmap to extract individual offset numbers */
+		ntuples = tbm_extract_page_tuple(page, output);
+		output->blockno = page->blockno;
+		output->ntuples = ntuples;
+		output->recheck = page->recheck;
+		istate->spageptr++;
+
+		LWLockRelease(&istate->lock);
+
+		return output;
+	}
+
+	LWLockRelease(&istate->lock);
+
+	/* Nothing more in the bitmap */
+	return NULL;
+}
+
+/*
  * tbm_end_iterate - finish an iteration over a TIDBitmap
  *
  * Currently this is just a pfree, but it might do more someday.  (For
@@ -790,6 +1159,18 @@ tbm_end_iterate(TBMIterator *iterator)
 }
 
 /*
+ * tbm_end_shared_iterate - finish a shared iteration over a TIDBitmap
+ *
+ * This doesn't free any of the shared state associated with the iterator,
+ * just our backend-private state.
+ */
+void
+tbm_end_shared_iterate(TBMSharedIterator *iterator)
+{
+	pfree(iterator);
+}
+
+/*
  * tbm_find_pageentry - find a PagetableEntry for the pageno
  *
  * Returns NULL if there is no non-lossy entry for the pageno.
@@ -995,7 +1376,7 @@ tbm_lossify(TIDBitmap *tbm)
 	 * push nentries down to significantly less than maxentries, or else we'll
 	 * just end up doing this again very soon.  We shoot for maxentries/2.
 	 */
-	Assert(!tbm->iterating);
+	Assert(tbm->iterating == TBM_NOT_ITERATING);
 	Assert(tbm->status == TBM_HASH);
 
 	pagetable_start_iterate_at(tbm->pagetable, &i, tbm->lossify_start);
@@ -1061,3 +1442,105 @@ tbm_comparator(const void *left, const void *right)
 		return 1;
 	return 0;
 }
+
+/*
+ * As above, but this will get index into PagetableEntry array.  Therefore,
+ * it needs to get actual PagetableEntry using the index before comparing the
+ * blockno.
+ */
+static int
+tbm_shared_comparator(const void *left, const void *right, void *arg)
+{
+	PagetableEntry *base = (PagetableEntry *) arg;
+	PagetableEntry *lpage = &base[*(int *) left];
+	PagetableEntry *rpage = &base[*(int *) right];
+
+	if (lpage->blockno < rpage->blockno)
+		return -1;
+	else if (lpage->blockno > rpage->blockno)
+		return 1;
+	return 0;
+}
+
+/*
+ *	tbm_attach_shared_iterate
+ *
+ *	Allocate a backend-private iterator and attach the shared iterator state
+ *	to it so that multiple processed can iterate jointly.
+ *
+ *	We also converts the DSA pointers to local pointers and store them into
+ *	our private iterator.
+ */
+TBMSharedIterator *
+tbm_attach_shared_iterate(dsa_area *dsa, dsa_pointer dp)
+{
+	TBMSharedIterator *iterator;
+	TBMSharedIteratorState *istate;
+
+	/*
+	 * Create the TBMSharedIterator struct, with enough trailing space to
+	 * serve the needs of the TBMIterateResult sub-struct.
+	 */
+	iterator = (TBMSharedIterator *) palloc(sizeof(TBMSharedIterator) +
+								 MAX_TUPLES_PER_PAGE * sizeof(OffsetNumber));
+
+	istate = (TBMSharedIteratorState *) dsa_get_address(dsa, dp);
+
+	iterator->state = istate;
+
+	iterator->ptbase = dsa_get_address(dsa, istate->pagetable);
+
+	if (istate->npages)
+		iterator->ptpages = dsa_get_address(dsa, istate->spages);
+	if (istate->nchunks)
+		iterator->ptchunks = dsa_get_address(dsa, istate->schunks);
+
+	return iterator;
+}
+
+/*
+ * pagetable_allocate
+ *
+ * Callback function for allocating the memory for hashtable elements.
+ * Allocate memory for hashtable elements, using DSA if available.
+ */
+static inline void *
+pagetable_allocate(pagetable_hash *pagetable, Size size)
+{
+	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
+	PTEntryArray *ptbase;
+
+	if (tbm->dsa == NULL)
+		return MemoryContextAllocExtended(pagetable->ctx, size,
+										  MCXT_ALLOC_HUGE | MCXT_ALLOC_ZERO);
+
+	/*
+	 * Save the dsapagetable reference in dsapagetableold before allocating
+	 * new memory so that pagetable_free can free the old entry.
+	 */
+	tbm->dsapagetableold = tbm->dsapagetable;
+	tbm->dsapagetable = dsa_allocate0(tbm->dsa, sizeof(PTEntryArray) + size);
+
+	ptbase = dsa_get_address(tbm->dsa, tbm->dsapagetable);
+	return ptbase->ptentry;
+}
+
+/*
+ * pagetable_free
+ *
+ * Callback function for freeing hash table elements.
+ */
+static inline void
+pagetable_free(pagetable_hash *pagetable, void *pointer)
+{
+	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
+
+	/* pfree the input pointer if DSA is not available */
+	if (tbm->dsa == NULL)
+		pfree(pointer);
+	else if (DsaPointerIsValid(tbm->dsapagetableold))
+	{
+		dsa_free(tbm->dsa, tbm->dsapagetableold);
+		tbm->dsapagetableold = InvalidDsaPointer;
+	}
+}
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index ab81d94..3e13394 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -510,6 +510,7 @@ RegisterLWLockTranches(void)
 						  "predicate_lock_manager");
 	LWLockRegisterTranche(LWTRANCHE_PARALLEL_QUERY_DSA,
 						  "parallel_query_dsa");
+	LWLockRegisterTranche(LWTRANCHE_TBM, "tbm");
 
 	/* Register named tranches. */
 	for (i = 0; i < NamedLWLockTrancheRequests; i++)
diff --git a/src/include/nodes/tidbitmap.h b/src/include/nodes/tidbitmap.h
index 14992e0..87f4bb7 100644
--- a/src/include/nodes/tidbitmap.h
+++ b/src/include/nodes/tidbitmap.h
@@ -23,6 +23,7 @@
 #define TIDBITMAP_H
 
 #include "storage/itemptr.h"
+#include "utils/dsa.h"
 
 
 /*
@@ -33,6 +34,7 @@ typedef struct TIDBitmap TIDBitmap;
 
 /* Likewise, TBMIterator is private */
 typedef struct TBMIterator TBMIterator;
+typedef struct TBMSharedIterator TBMSharedIterator;
 
 /* Result structure for tbm_iterate */
 typedef struct
@@ -46,8 +48,9 @@ typedef struct
 
 /* function prototypes in nodes/tidbitmap.c */
 
-extern TIDBitmap *tbm_create(long maxbytes);
+extern TIDBitmap *tbm_create(long maxbytes, dsa_area *dsa);
 extern void tbm_free(TIDBitmap *tbm);
+extern void tbm_free_shared_area(dsa_area *dsa, dsa_pointer dp);
 
 extern void tbm_add_tuples(TIDBitmap *tbm,
 			   const ItemPointer tids, int ntids,
@@ -60,7 +63,12 @@ extern void tbm_intersect(TIDBitmap *a, const TIDBitmap *b);
 extern bool tbm_is_empty(const TIDBitmap *tbm);
 
 extern TBMIterator *tbm_begin_iterate(TIDBitmap *tbm);
+extern dsa_pointer tbm_prepare_shared_iterate(TIDBitmap *tbm);
 extern TBMIterateResult *tbm_iterate(TBMIterator *iterator);
+extern TBMIterateResult *tbm_shared_iterate(TBMSharedIterator *iterator);
 extern void tbm_end_iterate(TBMIterator *iterator);
+extern void tbm_end_shared_iterate(TBMSharedIterator *iterator);
+extern TBMSharedIterator *tbm_attach_shared_iterate(dsa_area *dsa,
+						  dsa_pointer dp);
 
 #endif   /* TIDBITMAP_H */
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 8bd93c3..0cd45bb 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -212,6 +212,7 @@ typedef enum BuiltinTrancheIds
 	LWTRANCHE_LOCK_MANAGER,
 	LWTRANCHE_PREDICATE_LOCK_MANAGER,
 	LWTRANCHE_PARALLEL_QUERY_DSA,
+	LWTRANCHE_TBM,
 	LWTRANCHE_FIRST_USER_DEFINED
 }	BuiltinTrancheIds;
 
0003-parallel-bitmap-heapscan-v8.patchapplication/octet-stream; name=0003-parallel-bitmap-heapscan-v8.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1d70fae..6a3481f 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1248,6 +1248,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapPopulate</></entry>
+         <entry>Waiting for the leader to populate the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index af25836..bffc971 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1754,6 +1754,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index de0e2ba..a1289e5 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
 
 #include "executor/execParallel.h"
 #include "executor/executor.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "executor/nodeCustom.h"
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
@@ -217,6 +218,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -277,6 +282,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -775,6 +785,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index c1aa9f1..0968347 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -58,6 +58,8 @@ static inline void BitmapAdjustPrefetchIterator(BitmapHeapScanState *node,
 static inline void BitmapAdjustPrefetchTarget(BitmapHeapScanState *node);
 static inline void BitmapPrefetch(BitmapHeapScanState *node,
 			   HeapScanDesc scan);
+static bool BitmapShouldInitializeSharedState(
+								  ParallelBitmapHeapState *pbminfo);
 
 
 /* ----------------------------------------------------------------
@@ -73,9 +75,12 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	HeapScanDesc scan;
 	TIDBitmap  *tbm;
 	TBMIterator *tbmiterator;
+	TBMSharedIterator *shared_tbmiterator;
 	TBMIterateResult *tbmres;
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelBitmapHeapState *pstate = node->pstate;
+	dsa_area   *dsa = node->ss.ps.state->es_query_dsa;
 
 	/*
 	 * extract necessary information from index scan node
@@ -84,7 +89,10 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	slot = node->ss.ss_ScanTupleSlot;
 	scan = node->ss.ss_currentScanDesc;
 	tbm = node->tbm;
-	tbmiterator = node->tbmiterator;
+	if (pstate == NULL)
+		tbmiterator = node->tbmiterator;
+	else
+		shared_tbmiterator = node->shared_tbmiterator;
 	tbmres = node->tbmres;
 
 	/*
@@ -99,25 +107,89 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 * node->prefetch_maximum.  This is to avoid doing a lot of prefetching in
 	 * a scan that stops after a few tuples because of a LIMIT.
 	 */
-	if (tbm == NULL)
+	if (!node->initialized)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		if (!pstate)
+		{
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
 
-		node->tbm = tbm;
-		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
-		node->tbmres = tbmres = NULL;
+			node->tbm = tbm;
+			node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+			node->tbmres = tbmres = NULL;
 
 #ifdef USE_PREFETCH
-		if (node->prefetch_maximum > 0)
-		{
-			node->prefetch_iterator = tbm_begin_iterate(tbm);
-			node->prefetch_pages = 0;
-			node->prefetch_target = -1;
+			if (node->prefetch_maximum > 0)
+			{
+				node->prefetch_iterator = tbm_begin_iterate(tbm);
+				node->prefetch_pages = 0;
+				node->prefetch_target = -1;
+			}
+#endif   /* USE_PREFETCH */
 		}
+		else
+		{
+			/*
+			 * The leader will immediately come out of the function, but
+			 * others will be blocked until leader populates the TBM and wakes
+			 * them up.
+			 */
+			if (BitmapShouldInitializeSharedState(pstate))
+			{
+				tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+				if (!tbm || !IsA(tbm, TIDBitmap))
+					elog(ERROR, "unrecognized result from subplan");
+
+				node->tbm = tbm;
+
+				/*
+				 * Prepare to iterate over the TBM. This will return the
+				 * dsa_pointer of the iterator state which will be used by
+				 * multiple processes to iterate jointly.
+				 */
+				pstate->tbmiterator = tbm_prepare_shared_iterate(tbm);
+#ifdef USE_PREFETCH
+				if (node->prefetch_maximum > 0)
+				{
+					pstate->prefetch_iterator =
+						tbm_prepare_shared_iterate(tbm);
+
+					/*
+					 * We don't need mutex here as we haven't yet woke up
+					 * others
+					 */
+					pstate->prefetch_pages = 0;
+					pstate->prefetch_target = -1;
+				}
+#endif
+
+				/*
+				 * By this time we have already populated the TBM and
+				 * initialized the shared iterators so set the state to
+				 * BM_FINISHED and wake up others.
+				 */
+				SpinLockAcquire(&pstate->mutex);
+				pstate->state = BM_FINISHED;
+				SpinLockRelease(&pstate->mutex);
+				ConditionVariableBroadcast(&pstate->cv);
+			}
+
+			/* Allocate a private iterator and attach the shared state to it */
+			node->shared_tbmiterator = shared_tbmiterator =
+				tbm_attach_shared_iterate(dsa, pstate->tbmiterator);
+			node->tbmres = tbmres = NULL;
+
+#ifdef USE_PREFETCH
+			if (node->prefetch_maximum > 0)
+			{
+				node->shared_prefetch_iterator =
+					tbm_attach_shared_iterate(dsa, pstate->prefetch_iterator);
+			}
 #endif   /* USE_PREFETCH */
+		}
+		node->initialized = true;
 	}
 
 	for (;;)
@@ -130,7 +202,14 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			/*
+			 * If we are in parallel mode perform shared iterate otherwise
+			 * local iterate.
+			 */
+			if (!pstate)
+				node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			else
+				node->tbmres = tbmres = tbm_shared_iterate(shared_tbmiterator);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -182,8 +261,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (!pstate)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else if (pstate->prefetch_target < node->prefetch_maximum)
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pstate->mutex);
+				if (pstate->prefetch_target < node->prefetch_maximum)
+					pstate->prefetch_target++;
+				SpinLockRelease(&pstate->mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -369,20 +462,53 @@ BitmapAdjustPrefetchIterator(BitmapHeapScanState *node,
 							 TBMIterateResult *tbmres)
 {
 #ifdef USE_PREFETCH
-	TBMIterator *prefetch_iterator = node->prefetch_iterator;
+	ParallelBitmapHeapState *pstate = node->pstate;
 
-	if (node->prefetch_pages > 0)
+	if (pstate == NULL)
 	{
-		/* The main iterator has closed the distance by one page */
-		node->prefetch_pages--;
+		TBMIterator *prefetch_iterator = node->prefetch_iterator;
+
+		if (node->prefetch_pages > 0)
+		{
+			/* The main iterator has closed the distance by one page */
+			node->prefetch_pages--;
+		}
+		else if (prefetch_iterator)
+		{
+			/* Do not let the prefetch iterator get behind the main one */
+			TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+
+			if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				elog(ERROR, "prefetch and main iterators are out of sync");
+		}
+		return;
 	}
-	else if (prefetch_iterator)
+
+	if (node->prefetch_maximum > 0)
 	{
-		/* Do not let the prefetch iterator get behind the main one */
-		TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+		TBMSharedIterator *prefetch_iterator = node->shared_prefetch_iterator;
 
-		if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
-			elog(ERROR, "prefetch and main iterators are out of sync");
+		SpinLockAcquire(&pstate->mutex);
+		if (pstate->prefetch_pages > 0)
+		{
+			node->prefetch_pages--;
+			SpinLockRelease(&pstate->mutex);
+		}
+		else
+		{
+			/* Release the mutex before iterating */
+			SpinLockRelease(&pstate->mutex);
+
+			/*
+			 * In case of shared mode, we can not ensure that the current
+			 * blockno of the main iterator and that of the prefetch iterator
+			 * are same.  It's possible that whatever blockno we are
+			 * prefetching will be processed by another process.  Therefore we
+			 * don't validate the blockno unlike we do in non-parallel case.
+			 */
+			if (prefetch_iterator)
+				tbm_shared_iterate(prefetch_iterator);
+		}
 	}
 #endif   /* USE_PREFETCH */
 }
@@ -399,14 +525,38 @@ static inline void
 BitmapAdjustPrefetchTarget(BitmapHeapScanState *node)
 {
 #ifdef USE_PREFETCH
-	if (node->prefetch_target >= node->prefetch_maximum)
-		 /* don't increase any further */ ;
-	else if (node->prefetch_target >= node->prefetch_maximum / 2)
-		node->prefetch_target = node->prefetch_maximum;
-	else if (node->prefetch_target > 0)
-		node->prefetch_target *= 2;
-	else
-		node->prefetch_target++;
+	ParallelBitmapHeapState *pstate = node->pstate;
+
+	if (pstate == NULL)
+	{
+		if (node->prefetch_target >= node->prefetch_maximum)
+			 /* don't increase any further */ ;
+		else if (node->prefetch_target >= node->prefetch_maximum / 2)
+			node->prefetch_target = node->prefetch_maximum;
+		else if (node->prefetch_target > 0)
+			node->prefetch_target *= 2;
+		else
+			node->prefetch_target++;
+		return;
+	}
+
+	/*
+	 * Check before acquiring the mutex so that we can avoid acquiring the
+	 * mutex if target has already reached to its max value.
+	 */
+	if (pstate->prefetch_target < node->prefetch_maximum)
+	{
+		SpinLockAcquire(&pstate->mutex);
+		if (pstate->prefetch_target >= node->prefetch_maximum)
+			 /* don't increase any further */ ;
+		else if (pstate->prefetch_target >= node->prefetch_maximum / 2)
+			pstate->prefetch_target = node->prefetch_maximum;
+		else if (pstate->prefetch_target > 0)
+			pstate->prefetch_target *= 2;
+		else
+			pstate->prefetch_target++;
+		SpinLockRelease(&pstate->mutex);
+	}
 #endif   /* USE_PREFETCH */
 }
 
@@ -417,23 +567,77 @@ static inline void
 BitmapPrefetch(BitmapHeapScanState *node, HeapScanDesc scan)
 {
 #ifdef USE_PREFETCH
-	TBMIterator *prefetch_iterator = node->prefetch_iterator;
+	ParallelBitmapHeapState *pstate = node->pstate;
 
-	if (prefetch_iterator)
+	if (pstate == NULL)
 	{
-		while (node->prefetch_pages < node->prefetch_target)
+		TBMIterator *prefetch_iterator = node->prefetch_iterator;
+
+		if (prefetch_iterator)
 		{
-			TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+			while (node->prefetch_pages < node->prefetch_target)
+			{
+				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+
+				if (tbmpre == NULL)
+				{
+					/* No more pages to prefetch */
+					tbm_end_iterate(prefetch_iterator);
+					node->prefetch_iterator = NULL;
+					break;
+				}
+				node->prefetch_pages++;
+				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+			}
+		}
+
+		return;
+	}
+
+	if (pstate->prefetch_pages < pstate->prefetch_target)
+	{
+		TBMSharedIterator *prefetch_iterator = node->shared_prefetch_iterator;
+
+		if (prefetch_iterator)
+		{
+			SpinLockAcquire(&pstate->mutex);
 
-			if (tbmpre == NULL)
+			/*
+			 * Recheck under the mutex, If some other process has already done
+			 * the enough prefetch then we need not to do anything.
+			 */
+			if (pstate->prefetch_pages >= pstate->prefetch_target)
 			{
-				/* No more pages to prefetch */
-				tbm_end_iterate(prefetch_iterator);
-				node->prefetch_iterator = NULL;
-				break;
+				SpinLockRelease(&pstate->mutex);
+				return;
 			}
-			node->prefetch_pages++;
-			PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+			SpinLockRelease(&pstate->mutex);
+
+			do
+			{
+				TBMIterateResult *tbmpre;
+
+				tbmpre = tbm_shared_iterate(prefetch_iterator);
+				if (tbmpre == NULL)
+				{
+					/* No more pages to prefetch */
+					tbm_end_shared_iterate(prefetch_iterator);
+					node->shared_prefetch_iterator = NULL;
+					break;
+				}
+
+				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+
+				SpinLockAcquire(&pstate->mutex);
+				pstate->prefetch_pages++;
+				if (pstate->prefetch_pages >= pstate->prefetch_target)
+				{
+					SpinLockRelease(&pstate->mutex);
+					break;
+				}
+				SpinLockRelease(&pstate->mutex);
+
+			} while (true);
 		}
 	}
 #endif   /* USE_PREFETCH */
@@ -488,12 +692,36 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->tbmiterator);
 	if (node->prefetch_iterator)
 		tbm_end_iterate(node->prefetch_iterator);
+	if (node->shared_tbmiterator)
+		tbm_end_shared_iterate(node->shared_tbmiterator);
+	if (node->shared_prefetch_iterator)
+		tbm_end_shared_iterate(node->shared_prefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
 	node->tbm = NULL;
 	node->tbmiterator = NULL;
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
+	node->initialized = false;
+	node->shared_tbmiterator = NULL;
+	node->shared_prefetch_iterator = NULL;
+
+	/* Reset parallel bitmap state, if present */
+	if (node->pstate)
+	{
+		dsa_area   *dsa = node->ss.ps.state->es_query_dsa;
+
+		node->pstate->state = BM_INITIAL;
+
+		if (DsaPointerIsValid(node->pstate->tbmiterator))
+			tbm_free_shared_area(dsa, node->pstate->tbmiterator);
+
+		if (DsaPointerIsValid(node->pstate->prefetch_iterator))
+			tbm_free_shared_area(dsa, node->pstate->prefetch_iterator);
+
+		node->pstate->tbmiterator = InvalidDsaPointer;
+		node->pstate->prefetch_iterator = InvalidDsaPointer;
+	}
 
 	ExecScanReScan(&node->ss);
 
@@ -546,6 +774,10 @@ ExecEndBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->prefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
+	if (node->shared_tbmiterator)
+		tbm_end_shared_iterate(node->shared_tbmiterator);
+	if (node->shared_prefetch_iterator)
+		tbm_end_shared_iterate(node->shared_prefetch_iterator);
 
 	/*
 	 * close heap scan
@@ -597,6 +829,10 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->pscan_len = 0;
+	scanstate->initialized = false;
+	scanstate->shared_tbmiterator = NULL;
+	scanstate->pstate = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -681,3 +917,126 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ *		BitmapShouldInitializeSharedState
+ *
+ *		The first process to come here and see the state to the BM_INITIAL
+ *		will become the leader for the parallel bitmap scan and will be
+ *		responsible for populating the TIDBitmap.  The other processes will
+ *		be blocked by the condition variable until the leader wakes them up.
+ * ---------------
+ */
+static bool
+BitmapShouldInitializeSharedState(ParallelBitmapHeapState *pstate)
+{
+	bool		needWait = false;
+	bool		leader = false;
+
+	while (1)
+	{
+		/*---------------
+		 * Check the current state
+		 * If state is
+		 * BM_INITIAL	  : We become the leader and set it to BM_INPROGRESS
+		 * BM_INPROGRESS : We need to wait till leader creates bitmap
+		 * BM_FINISHED	 : bitmap is ready so no need to wait
+		 *---------------
+		 */
+		SpinLockAcquire(&pstate->mutex);
+
+		if (pstate->state == BM_INITIAL)
+		{
+			pstate->state = BM_INPROGRESS;
+			leader = true;
+		}
+		else if (pstate->state == BM_INPROGRESS)
+			needWait = true;
+		else
+			needWait = false;
+
+		SpinLockRelease(&pstate->mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (leader || !needWait)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pstate->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return leader;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+
+	/* Estimate the size for sharing parallel Bitmap info. */
+	node->pscan_len = add_size(offsetof(ParallelBitmapHeapState,
+										phs_snapshot_data),
+							   EstimateSnapshotSpace(estate->es_snapshot));
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapHeapState *pstate;
+	EState	   *estate = node->ss.ps.state;
+
+	pstate = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	pstate->tbmiterator = 0;
+	pstate->prefetch_iterator = 0;
+
+	/* Initialize the mutex */
+	SpinLockInit(&pstate->mutex);
+	pstate->prefetch_pages = 0;
+	pstate->prefetch_target = 0;
+	pstate->state = BM_INITIAL;
+
+	ConditionVariableInit(&pstate->cv);
+	SerializeSnapshot(estate->es_snapshot, pstate->phs_snapshot_data);
+
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pstate);
+	node->pstate = pstate;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapHeapState *pstate;
+	Snapshot	snapshot;
+
+	pstate = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->pstate = pstate;
+
+	snapshot = RestoreSnapshot(pstate->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 94bb289..ce2f321 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,9 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L, NULL);
+		tbm = tbm_create(work_mem * 1024L,
+						 ((BitmapIndexScan *) node->ss.ps.plan)->isshared ?
+						 node->ss.ps.state->es_query_dsa : NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 1d280be..c0f2614 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,9 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L, NULL);
+				result = tbm_create(work_mem * 1024L,
+									((BitmapOr *) node->ps.plan)->isshared ?
+									node->ps.state->es_query_dsa : NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index bb2a8a3..ae08d2c 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -331,6 +331,7 @@ _copyBitmapOr(const BitmapOr *from)
 	/*
 	 * copy remainder of node
 	 */
+	COPY_SCALAR_FIELD(isshared);
 	COPY_NODE_FIELD(bitmapplans);
 
 	return newnode;
@@ -496,6 +497,7 @@ _copyBitmapIndexScan(const BitmapIndexScan *from)
 	 * copy remainder of node
 	 */
 	COPY_SCALAR_FIELD(indexid);
+	COPY_SCALAR_FIELD(isshared);
 	COPY_NODE_FIELD(indexqual);
 	COPY_NODE_FIELD(indexqualorig);
 
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b3802b4..0ec4e1b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -441,6 +441,7 @@ _outBitmapOr(StringInfo str, const BitmapOr *node)
 
 	_outPlanInfo(str, (const Plan *) node);
 
+	WRITE_BOOL_FIELD(isshared);
 	WRITE_NODE_FIELD(bitmapplans);
 }
 
@@ -520,6 +521,7 @@ _outBitmapIndexScan(StringInfo str, const BitmapIndexScan *node)
 	_outScanInfo(str, (const Scan *) node);
 
 	WRITE_OID_FIELD(indexid);
+	WRITE_BOOL_FIELD(isshared);
 	WRITE_NODE_FIELD(indexqual);
 	WRITE_NODE_FIELD(indexqualorig);
 }
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 05bf2e9..d0d68cc 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1605,6 +1605,7 @@ _readBitmapOr(void)
 
 	ReadCommonPlan(&local_node->plan);
 
+	READ_BOOL_FIELD(isshared);
 	READ_NODE_FIELD(bitmapplans);
 
 	READ_DONE();
@@ -1716,6 +1717,7 @@ _readBitmapIndexScan(void)
 	ReadCommonScan(&local_node->scan);
 
 	READ_OID_FIELD(indexid);
+	READ_BOOL_FIELD(isshared);
 	READ_NODE_FIELD(indexqual);
 	READ_NODE_FIELD(indexqualorig);
 
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 87a3faf..9240125 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2878,6 +2878,30 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 }
 
 /*
+ * create_partial_bitmap_paths
+ *	  Build partial bitmap heap path for the relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched, 0);
+
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+/*
  * Compute the number of parallel workers that should be used to scan a
  * relation.  We compute the parallel workers based on the size of the heap to
  * be scanned and the size of the index to be scanned, then choose a minimum
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index c138f57..b8cde32 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -860,6 +860,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -921,8 +922,21 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
+
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index d8e5b81..c2b72d4 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1617,6 +1621,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1659,6 +1668,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 1e953b4..eaf85da 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -125,6 +125,7 @@ static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
 						List *tlist, List *scan_clauses);
 static Plan *create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 					  List **qual, List **indexqual, List **indexECs);
+static void bitmap_subplan_mark_shared(Plan *plan);
 static TidScan *create_tidscan_plan(PlannerInfo *root, TidPath *best_path,
 					List *tlist, List *scan_clauses);
 static SubqueryScan *create_subqueryscan_plan(PlannerInfo *root,
@@ -2577,6 +2578,9 @@ create_bitmap_scan_plan(PlannerInfo *root,
 										   &bitmapqualorig, &indexquals,
 										   &indexECs);
 
+	if (best_path->path.parallel_aware)
+		bitmap_subplan_mark_shared(bitmapqualplan);
+
 	/*
 	 * The qpqual list must contain all restrictions not automatically handled
 	 * by the index, other than pseudoconstant clauses which will be handled
@@ -2712,6 +2716,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 			subplan = create_bitmap_subplan(root, (Path *) lfirst(l),
 											&subqual, &subindexqual,
 											&subindexEC);
+
 			subplans = lappend(subplans, subplan);
 			subquals = list_concat_unique(subquals, subqual);
 			subindexquals = list_concat_unique(subindexquals, subindexqual);
@@ -4700,6 +4705,24 @@ label_sort_with_costsize(PlannerInfo *root, Sort *plan, double limit_tuples)
 	plan->plan.parallel_aware = false;
 }
 
+/*
+ * bitmap_subplan_mark_shared
+ *	 Mark a shared flag in bitmap subplan so that it can create underlying
+ *	 bitmap in shared memory which is accecible by the multiple processes.
+ */
+static void
+bitmap_subplan_mark_shared(Plan *plan)
+{
+	if (IsA(plan, BitmapAnd))
+		bitmap_subplan_mark_shared(
+								linitial(((BitmapAnd *) plan)->bitmapplans));
+	else if (IsA(plan, BitmapOr))
+		((BitmapOr *) plan)->isshared = true;
+	else if (IsA(plan, BitmapIndexScan))
+		((BitmapIndexScan *) plan)->isshared = true;
+	else
+		elog(ERROR, "unrecognized node type: %d", nodeTag(plan));
+}
 
 /*****************************************************************************
  *
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3248296..6e70808 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1068,7 +1068,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1077,9 +1078,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3255,7 +3256,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 2fb9a8b..7cacb1e 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3395,6 +3395,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index a864f78..7e85510 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -179,6 +179,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 6332ea0..8698441 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1465,6 +1467,52 @@ typedef struct BitmapIndexScanState
 } BitmapIndexScanState;
 
 /* ----------------
+ *	 SharedBitmapState information
+ *
+ *		BM_INITIAL		TIDBitmap creation is not yet started, so first worker
+ *						to see this state will set the state to BM_INPROGRESS
+ *						and that process will be responsible for creating
+ *						TIDBitmap.
+ *		BM_INPROGRESS	TIDBitmap creation is already in progress, therefore
+ *						workers need to sleep until leader set the state to
+ *						BM_FINISHED and wake us up.
+ *		BM_FINISHED		TIDBitmap creation is done, so now all worker can
+ *						proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	BM_INITIAL,
+	BM_INPROGRESS,
+	BM_FINISHED
+} SharedBitmapState;
+
+/* ----------------
+ *	 ParallelBitmapHeapState information
+ *		tbmiterator				iterator for scanning current pages
+ *		prefetch_iterator		iterator for prefetching ahead of current page
+ *		mutex					mutual exclusion for the prefetching variable
+ *								and state
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state					current state of the TIDBitmap
+ *		cv						conditional wait variable
+ *		phs_snapshot_data		snapshot data shared to workers
+ * ----------------
+ */
+typedef struct ParallelBitmapHeapState
+{
+	dsa_pointer tbmiterator;
+	dsa_pointer prefetch_iterator;
+	slock_t		mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	SharedBitmapState state;
+	ConditionVariable cv;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+} ParallelBitmapHeapState;
+
+/* ----------------
  *	 BitmapHeapScanState information
  *
  *		bitmapqualorig	   execution state for bitmapqualorig expressions
@@ -1477,6 +1525,11 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		initialized		   is node is ready to iterate
+ *		shared_tbmiterator	   shared iterator
+ *		shared_prefetch_iterator shared iterator for prefetching
+ *		pstate			   shared state for parallel bitmap scan
  * ----------------
  */
 typedef struct BitmapHeapScanState
@@ -1492,6 +1545,11 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	bool		initialized;
+	TBMSharedIterator *shared_tbmiterator;
+	TBMSharedIterator *shared_prefetch_iterator;
+	ParallelBitmapHeapState *pstate;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f72f7a8..7f38ca6 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -292,6 +292,7 @@ typedef struct BitmapAnd
 typedef struct BitmapOr
 {
 	Plan		plan;
+	bool		isshared;
 	List	   *bitmapplans;
 } BitmapOr;
 
@@ -420,6 +421,7 @@ typedef struct BitmapIndexScan
 {
 	Scan		scan;
 	Oid			indexid;		/* OID of index to scan */
+	bool		isshared;		/* Create shared bitmap if set */
 	List	   *indexqual;		/* list of index quals (OpExprs) */
 	List	   *indexqualorig;	/* the same in original form */
 } BitmapIndexScan;
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 53cad24..d22b039 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -53,7 +53,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index bc0dcf4..247fd11 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -56,6 +56,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
 extern int compute_parallel_worker(RelOptInfo *rel, BlockNumber heap_pages,
 						BlockNumber index_pages);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+										Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 0062fb8..60c78d1 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -787,6 +787,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
#113Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#112)
Re: Parallel bitmap heap scan

On Wed, Mar 8, 2017 at 1:49 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Tue, Mar 7, 2017 at 10:10 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

It's not about speed. It's about not forgetting to prefetch. Suppose
that worker 1 becomes the prefetch worker but then doesn't return to
the Bitmap Heap Scan node for a long time because it's busy in some
other part of the plan tree. Now you just stop prefetching; that's
bad. You want prefetching to continue regardless of which workers are
busy doing what; as long as SOME worker is executing the parallel
bitmap heap scan, prefetching should continue as needed.

Right, I missed this part. I will fix this.

I have fixed this part, after doing that I realised if multiple
processes are prefetching then it may be possible that in boundary
cases (e.g. suppose prefetch_target is 3 and prefetch_pages is at 2)
there may be some extra prefetch but finally those prefetched blocks
will be used. Another, solution to this problem is that we can
increase the prefetch_pages in advance then call tbm_shared_iterate,
this will avoid extra prefetch. But I am not sure what will be best
here.

I don't think I understand exactly why this system might be prone to a
little bit of extra prefetching - can you explain further? I don't
think it will hurt anything as long as we are talking about a small
number of extra prefetches here and there.

Fixed

Committed 0001.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#114Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#113)
Re: Parallel bitmap heap scan

On Wed, Mar 8, 2017 at 6:42 PM, Robert Haas <robertmhaas@gmail.com> wrote:

I don't think I understand exactly why this system might be prone to a
little bit of extra prefetching - can you explain further?

Let me explain with an example, suppose there are 2 workers
prefetching jointly, lets assume
prefetch_target is 3, so for example when both the workers prefetch 1
page each then prefetch_page count will be 2, so both the workers will
go for next prefetch because prefetch_page is still less than prefetch
target, so again both the workers will do prefetch and totally they
will prefetch 4 pages.

I don't think it will hurt anything as long as we are talking about a small
number of extra prefetches here and there.

I completely agree with this point and I mentioned in the mail so that
it don't go unnoticed.

And, whatever extra prefetch we have done we will anyway use it unless
we stop the execution because of some limit clause.

Fixed

Committed 0001.

Thanks

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#115Robert Haas
robertmhaas@gmail.com
In reply to: Robert Haas (#113)
Re: Parallel bitmap heap scan

Reviewing 0003:

How about adding a regression test?

bitmap_subplan_mark_shared could use castNode(), which seems like it
would be better style. Maybe some other places, too.

+         <entry><literal>ParallelBitmapPopulate</></entry>
+         <entry>Waiting for the leader to populate the TidBitmap.</entry>
+        </row>
+        <row>

If you build the documentation, you'll find that this doesn't come out
right; you need to add 1 to the value of the nearest preceding
"morerows". (I fixed a similar issue with 0001 while committing.)

+        /*---------------
+         * Check the current state
+         * If state is
+         * BM_INITIAL     : We become the leader and set it to BM_INPROGRESS
+         * BM_INPROGRESS : We need to wait till leader creates bitmap
+         * BM_FINISHED   : bitmap is ready so no need to wait
+         *---------------

The formatting of this comment is slightly off - the comment for
BM_INITIAL isn't aligned the same as the others. But I would just
delete the whole comment, since more or less it recapitulates the
function header comment anyway.

I wonder if BitmapShouldInitializeSharedState couldn't be written a
little more compactly overall, like this:

{
SharedBitmapState state;

while (1)
{
SpinLockAcquire(&pstate->mutex);
state = pstate->state;
if (pstate->state == BM_INITIAL)
pstate->state = BM_INPROGRESS;
SpinLockRelease(&pstate->mutex);

/* If we are leader or leader has already created a TIDBITMAP */
if (state != BM_INPROGRESS)
break;

/* Sleep until leader finishes creating the bitmap */
ConditionVariableSleep(&pstate->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
}

ConditionVariableCancelSleep();

return (state == BM_INITIAL);
}

+                /*
+                 * By this time we have already populated the TBM and
+                 * initialized the shared iterators so set the state to
+                 * BM_FINISHED and wake up others.
+                 */
+                SpinLockAcquire(&pstate->mutex);
+                pstate->state = BM_FINISHED;
+                SpinLockRelease(&pstate->mutex);
+                ConditionVariableBroadcast(&pstate->cv);

I think it would be good to have a function for this, like
BitmapDoneInitializingSharedState(), and just call that function here.

+            SpinLockAcquire(&pstate->mutex);
+
+            /*
+             * Recheck under the mutex, If some other process has already done
+             * the enough prefetch then we need not to do anything.
+             */
+            if (pstate->prefetch_pages >= pstate->prefetch_target)
+                SpinLockRelease(&pstate->mutex);
+                return;
+            SpinLockRelease(&pstate->mutex);

I think it would be clearer to write this as:

SpinLockAcquire(&pstate->mutex);
do_prefetch = (pstate->prefetch_pages >= pstate->prefetch_target);
SpinLockRelease(&pstate->mutex);
if (!do_prefetch)
return;

Then it's more obvious what's going on with the spinlock. But
actually what I would do is also roll in the increment to prefetch
pages in there, so that you don't have to reacquire the spinlock after
calling PrefetchBuffer:

bool do_prefetch = false;
SpinLockAcquire(&pstate->mutex);
if (pstate->prefetch_pages < pstate->prefetch_target)
{
pstate->prefetch_pages++;
do_prefetch = true;
}
SpinLockRelease(&pstate->mutex);

That seems like it will reduce the amount of excess prefetching
considerably, and also simplify the code and cut the spinlock
acquisitions by 50%.

Overall I think this is in pretty good shape.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#116Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#115)
1 attachment(s)
Re: Parallel bitmap heap scan

On Wed, Mar 8, 2017 at 8:28 PM, Robert Haas <robertmhaas@gmail.com> wrote:

How about adding a regression test?

Added

bitmap_subplan_mark_shared could use castNode(), which seems like it
would be better style. Maybe some other places, too.

+         <entry><literal>ParallelBitmapPopulate</></entry>
+         <entry>Waiting for the leader to populate the TidBitmap.</entry>
+        </row>
+        <row>

If you build the documentation, you'll find that this doesn't come out
right; you need to add 1 to the value of the nearest preceding
"morerows". (I fixed a similar issue with 0001 while committing.)

Fixed

+        /*---------------
+         * Check the current state
+         * If state is
+         * BM_INITIAL     : We become the leader and set it to BM_INPROGRESS
+         * BM_INPROGRESS : We need to wait till leader creates bitmap
+         * BM_FINISHED   : bitmap is ready so no need to wait
+         *---------------

The formatting of this comment is slightly off - the comment for
BM_INITIAL isn't aligned the same as the others. But I would just
delete the whole comment, since more or less it recapitulates the
function header comment anyway.

Removed.

I wonder if BitmapShouldInitializeSharedState couldn't be written a
little more compactly overall, like this:

{
SharedBitmapState state;

while (1)
{
SpinLockAcquire(&pstate->mutex);
state = pstate->state;
if (pstate->state == BM_INITIAL)
pstate->state = BM_INPROGRESS;
SpinLockRelease(&pstate->mutex);

/* If we are leader or leader has already created a TIDBITMAP */
if (state != BM_INPROGRESS)
break;

/* Sleep until leader finishes creating the bitmap */
ConditionVariableSleep(&pstate->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
}

ConditionVariableCancelSleep();

return (state == BM_INITIAL);
}

This looks good, done this way

+                /*
+                 * By this time we have already populated the TBM and
+                 * initialized the shared iterators so set the state to
+                 * BM_FINISHED and wake up others.
+                 */
+                SpinLockAcquire(&pstate->mutex);
+                pstate->state = BM_FINISHED;
+                SpinLockRelease(&pstate->mutex);
+                ConditionVariableBroadcast(&pstate->cv);

I think it would be good to have a function for this, like
BitmapDoneInitializingSharedState(), and just call that function here.

Done

+            SpinLockAcquire(&pstate->mutex);
+
+            /*
+             * Recheck under the mutex, If some other process has already done
+             * the enough prefetch then we need not to do anything.
+             */
+            if (pstate->prefetch_pages >= pstate->prefetch_target)
+                SpinLockRelease(&pstate->mutex);
+                return;
+            SpinLockRelease(&pstate->mutex);

I think it would be clearer to write this as:

SpinLockAcquire(&pstate->mutex);
do_prefetch = (pstate->prefetch_pages >= pstate->prefetch_target);
SpinLockRelease(&pstate->mutex);
if (!do_prefetch)
return;

Then it's more obvious what's going on with the spinlock. But
actually what I would do is also roll in the increment to prefetch
pages in there, so that you don't have to reacquire the spinlock after
calling PrefetchBuffer:

bool do_prefetch = false;
SpinLockAcquire(&pstate->mutex);
if (pstate->prefetch_pages < pstate->prefetch_target)
{
pstate->prefetch_pages++;
do_prefetch = true;
}
SpinLockRelease(&pstate->mutex);

That seems like it will reduce the amount of excess prefetching
considerably, and also simplify the code and cut the spinlock
acquisitions by 50%.

Right, done that way

Overall I think this is in pretty good shape.

Thanks.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

0003-parallel-bitmap-heapscan-v9.patchapplication/octet-stream; name=0003-parallel-bitmap-heapscan-v9.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 27ed35f..4d03531 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1211,7 +1211,7 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting in an extension.</entry>
         </row>
         <row>
-         <entry morerows="10"><literal>IPC</></entry>
+         <entry morerows="11"><literal>IPC</></entry>
          <entry><literal>BgWorkerShutdown</></entry>
          <entry>Waiting for background worker to shut down.</entry>
         </row>
@@ -1248,6 +1248,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
          <entry>Waiting for parallel workers to finish computing.</entry>
         </row>
         <row>
+         <entry><literal>ParallelBitmapPopulate</></entry>
+         <entry>Waiting for the leader to populate the TidBitmap.</entry>
+        </row>
+        <row>
          <entry><literal>SafeSnapshot</></entry>
          <entry>Waiting for a snapshot for a <literal>READ ONLY DEFERRABLE</> transaction.</entry>
         </row>
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index af25836..bffc971 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1754,6 +1754,22 @@ retry:
 }
 
 /* ----------------
+ *		heap_update_snapshot
+ *
+ *		Update snapshot info in heap scan descriptor.
+ * ----------------
+ */
+void
+heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot)
+{
+	Assert(IsMVCCSnapshot(snapshot));
+
+	RegisterSnapshot(snapshot);
+	scan->rs_snapshot = snapshot;
+	scan->rs_temp_snap = true;
+}
+
+/* ----------------
  *		heap_getnext	- retrieve next tuple in scan
  *
  *		Fix to work with index relations.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index de0e2ba..a1289e5 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
 
 #include "executor/execParallel.h"
 #include "executor/executor.h"
+#include "executor/nodeBitmapHeapscan.h"
 #include "executor/nodeCustom.h"
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
@@ -217,6 +218,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecCustomScanEstimate((CustomScanState *) planstate,
 									   e->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
+									   e->pcxt);
+				break;
 			default:
 				break;
 		}
@@ -277,6 +282,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecCustomScanInitializeDSM((CustomScanState *) planstate,
 											d->pcxt);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
+											d->pcxt);
+				break;
+
 			default:
 				break;
 		}
@@ -775,6 +785,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 				ExecCustomScanInitializeWorker((CustomScanState *) planstate,
 											   toc);
 				break;
+			case T_BitmapHeapScanState:
+				ExecBitmapHeapInitializeWorker(
+									 (BitmapHeapScanState *) planstate, toc);
+				break;
 			default:
 				break;
 		}
diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c
index c1aa9f1..dfb61dd 100644
--- a/src/backend/executor/nodeBitmapHeapscan.c
+++ b/src/backend/executor/nodeBitmapHeapscan.c
@@ -53,11 +53,15 @@
 
 static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node);
 static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres);
+static inline void BitmapDoneInitializingSharedState(
+								  ParallelBitmapHeapState *pstate);
 static inline void BitmapAdjustPrefetchIterator(BitmapHeapScanState *node,
 							 TBMIterateResult *tbmres);
 static inline void BitmapAdjustPrefetchTarget(BitmapHeapScanState *node);
 static inline void BitmapPrefetch(BitmapHeapScanState *node,
 			   HeapScanDesc scan);
+static bool BitmapShouldInitializeSharedState(
+								  ParallelBitmapHeapState *pstate);
 
 
 /* ----------------------------------------------------------------
@@ -73,9 +77,12 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	HeapScanDesc scan;
 	TIDBitmap  *tbm;
 	TBMIterator *tbmiterator;
+	TBMSharedIterator *shared_tbmiterator;
 	TBMIterateResult *tbmres;
 	OffsetNumber targoffset;
 	TupleTableSlot *slot;
+	ParallelBitmapHeapState *pstate = node->pstate;
+	dsa_area   *dsa = node->ss.ps.state->es_query_dsa;
 
 	/*
 	 * extract necessary information from index scan node
@@ -84,7 +91,10 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	slot = node->ss.ss_ScanTupleSlot;
 	scan = node->ss.ss_currentScanDesc;
 	tbm = node->tbm;
-	tbmiterator = node->tbmiterator;
+	if (pstate == NULL)
+		tbmiterator = node->tbmiterator;
+	else
+		shared_tbmiterator = node->shared_tbmiterator;
 	tbmres = node->tbmres;
 
 	/*
@@ -99,25 +109,82 @@ BitmapHeapNext(BitmapHeapScanState *node)
 	 * node->prefetch_maximum.  This is to avoid doing a lot of prefetching in
 	 * a scan that stops after a few tuples because of a LIMIT.
 	 */
-	if (tbm == NULL)
+	if (!node->initialized)
 	{
-		tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+		if (!pstate)
+		{
+			tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
 
-		if (!tbm || !IsA(tbm, TIDBitmap))
-			elog(ERROR, "unrecognized result from subplan");
+			if (!tbm || !IsA(tbm, TIDBitmap))
+				elog(ERROR, "unrecognized result from subplan");
 
-		node->tbm = tbm;
-		node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
-		node->tbmres = tbmres = NULL;
+			node->tbm = tbm;
+			node->tbmiterator = tbmiterator = tbm_begin_iterate(tbm);
+			node->tbmres = tbmres = NULL;
 
 #ifdef USE_PREFETCH
-		if (node->prefetch_maximum > 0)
-		{
-			node->prefetch_iterator = tbm_begin_iterate(tbm);
-			node->prefetch_pages = 0;
-			node->prefetch_target = -1;
+			if (node->prefetch_maximum > 0)
+			{
+				node->prefetch_iterator = tbm_begin_iterate(tbm);
+				node->prefetch_pages = 0;
+				node->prefetch_target = -1;
+			}
+#endif   /* USE_PREFETCH */
 		}
+		else
+		{
+			/*
+			 * The leader will immediately come out of the function, but
+			 * others will be blocked until leader populates the TBM and wakes
+			 * them up.
+			 */
+			if (BitmapShouldInitializeSharedState(pstate))
+			{
+				tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node));
+				if (!tbm || !IsA(tbm, TIDBitmap))
+					elog(ERROR, "unrecognized result from subplan");
+
+				node->tbm = tbm;
+
+				/*
+				 * Prepare to iterate over the TBM. This will return the
+				 * dsa_pointer of the iterator state which will be used by
+				 * multiple processes to iterate jointly.
+				 */
+				pstate->tbmiterator = tbm_prepare_shared_iterate(tbm);
+#ifdef USE_PREFETCH
+				if (node->prefetch_maximum > 0)
+				{
+					pstate->prefetch_iterator =
+						tbm_prepare_shared_iterate(tbm);
+
+					/*
+					 * We don't need mutex here as we haven't yet woke up
+					 * others
+					 */
+					pstate->prefetch_pages = 0;
+					pstate->prefetch_target = -1;
+				}
+#endif
+
+				/* We have initialized the shared state so wake up others */
+				BitmapDoneInitializingSharedState(pstate);
+			}
+
+			/* Allocate a private iterator and attach the shared state to it */
+			node->shared_tbmiterator = shared_tbmiterator =
+				tbm_attach_shared_iterate(dsa, pstate->tbmiterator);
+			node->tbmres = tbmres = NULL;
+
+#ifdef USE_PREFETCH
+			if (node->prefetch_maximum > 0)
+			{
+				node->shared_prefetch_iterator =
+					tbm_attach_shared_iterate(dsa, pstate->prefetch_iterator);
+			}
 #endif   /* USE_PREFETCH */
+		}
+		node->initialized = true;
 	}
 
 	for (;;)
@@ -130,7 +197,14 @@ BitmapHeapNext(BitmapHeapScanState *node)
 		 */
 		if (tbmres == NULL)
 		{
-			node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			/*
+			 * If we are in parallel mode perform shared iterate otherwise
+			 * local iterate.
+			 */
+			if (!pstate)
+				node->tbmres = tbmres = tbm_iterate(tbmiterator);
+			else
+				node->tbmres = tbmres = tbm_shared_iterate(shared_tbmiterator);
 			if (tbmres == NULL)
 			{
 				/* no more entries in the bitmap */
@@ -182,8 +256,22 @@ BitmapHeapNext(BitmapHeapScanState *node)
 			 * Try to prefetch at least a few pages even before we get to the
 			 * second page if we don't stop reading after the first tuple.
 			 */
-			if (node->prefetch_target < node->prefetch_maximum)
-				node->prefetch_target++;
+			if (!pstate)
+			{
+				if (node->prefetch_target < node->prefetch_maximum)
+					node->prefetch_target++;
+			}
+			else if (pstate->prefetch_target < node->prefetch_maximum)
+			{
+				/*
+				 * If we are in parallel mode then grab prefetch_mutex before
+				 * updating prefetch target.
+				 */
+				SpinLockAcquire(&pstate->mutex);
+				if (pstate->prefetch_target < node->prefetch_maximum)
+					pstate->prefetch_target++;
+				SpinLockRelease(&pstate->mutex);
+			}
 #endif   /* USE_PREFETCH */
 		}
 
@@ -362,6 +450,21 @@ bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres)
 }
 
 /*
+ *	BitmapDoneInitializingSharedState - Shared state is initialized
+ *
+ *	By this time the leader has already populated the TBM and initialized the
+ *	shared state so wake up other processes.
+ */
+static inline void
+BitmapDoneInitializingSharedState(ParallelBitmapHeapState *pstate)
+{
+	SpinLockAcquire(&pstate->mutex);
+	pstate->state = BM_FINISHED;
+	SpinLockRelease(&pstate->mutex);
+	ConditionVariableBroadcast(&pstate->cv);
+}
+
+/*
  *	BitmapAdjustPrefetchIterator - Adjust the prefetch iterator
  */
 static inline void
@@ -369,20 +472,53 @@ BitmapAdjustPrefetchIterator(BitmapHeapScanState *node,
 							 TBMIterateResult *tbmres)
 {
 #ifdef USE_PREFETCH
-	TBMIterator *prefetch_iterator = node->prefetch_iterator;
+	ParallelBitmapHeapState *pstate = node->pstate;
 
-	if (node->prefetch_pages > 0)
+	if (pstate == NULL)
 	{
-		/* The main iterator has closed the distance by one page */
-		node->prefetch_pages--;
+		TBMIterator *prefetch_iterator = node->prefetch_iterator;
+
+		if (node->prefetch_pages > 0)
+		{
+			/* The main iterator has closed the distance by one page */
+			node->prefetch_pages--;
+		}
+		else if (prefetch_iterator)
+		{
+			/* Do not let the prefetch iterator get behind the main one */
+			TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+
+			if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
+				elog(ERROR, "prefetch and main iterators are out of sync");
+		}
+		return;
 	}
-	else if (prefetch_iterator)
+
+	if (node->prefetch_maximum > 0)
 	{
-		/* Do not let the prefetch iterator get behind the main one */
-		TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+		TBMSharedIterator *prefetch_iterator = node->shared_prefetch_iterator;
+
+		SpinLockAcquire(&pstate->mutex);
+		if (pstate->prefetch_pages > 0)
+		{
+			node->prefetch_pages--;
+			SpinLockRelease(&pstate->mutex);
+		}
+		else
+		{
+			/* Release the mutex before iterating */
+			SpinLockRelease(&pstate->mutex);
 
-		if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno)
-			elog(ERROR, "prefetch and main iterators are out of sync");
+			/*
+			 * In case of shared mode, we can not ensure that the current
+			 * blockno of the main iterator and that of the prefetch iterator
+			 * are same.  It's possible that whatever blockno we are
+			 * prefetching will be processed by another process.  Therefore we
+			 * don't validate the blockno unlike we do in non-parallel case.
+			 */
+			if (prefetch_iterator)
+				tbm_shared_iterate(prefetch_iterator);
+		}
 	}
 #endif   /* USE_PREFETCH */
 }
@@ -399,14 +535,38 @@ static inline void
 BitmapAdjustPrefetchTarget(BitmapHeapScanState *node)
 {
 #ifdef USE_PREFETCH
-	if (node->prefetch_target >= node->prefetch_maximum)
-		 /* don't increase any further */ ;
-	else if (node->prefetch_target >= node->prefetch_maximum / 2)
-		node->prefetch_target = node->prefetch_maximum;
-	else if (node->prefetch_target > 0)
-		node->prefetch_target *= 2;
-	else
-		node->prefetch_target++;
+	ParallelBitmapHeapState *pstate = node->pstate;
+
+	if (pstate == NULL)
+	{
+		if (node->prefetch_target >= node->prefetch_maximum)
+			 /* don't increase any further */ ;
+		else if (node->prefetch_target >= node->prefetch_maximum / 2)
+			node->prefetch_target = node->prefetch_maximum;
+		else if (node->prefetch_target > 0)
+			node->prefetch_target *= 2;
+		else
+			node->prefetch_target++;
+		return;
+	}
+
+	/*
+	 * Check before acquiring the mutex so that we can avoid acquiring the
+	 * mutex if target has already reached to its max value.
+	 */
+	if (pstate->prefetch_target < node->prefetch_maximum)
+	{
+		SpinLockAcquire(&pstate->mutex);
+		if (pstate->prefetch_target >= node->prefetch_maximum)
+			 /* don't increase any further */ ;
+		else if (pstate->prefetch_target >= node->prefetch_maximum / 2)
+			pstate->prefetch_target = node->prefetch_maximum;
+		else if (pstate->prefetch_target > 0)
+			pstate->prefetch_target *= 2;
+		else
+			pstate->prefetch_target++;
+		SpinLockRelease(&pstate->mutex);
+	}
 #endif   /* USE_PREFETCH */
 }
 
@@ -417,23 +577,70 @@ static inline void
 BitmapPrefetch(BitmapHeapScanState *node, HeapScanDesc scan)
 {
 #ifdef USE_PREFETCH
-	TBMIterator *prefetch_iterator = node->prefetch_iterator;
+	ParallelBitmapHeapState *pstate = node->pstate;
 
-	if (prefetch_iterator)
+	if (pstate == NULL)
 	{
-		while (node->prefetch_pages < node->prefetch_target)
+		TBMIterator *prefetch_iterator = node->prefetch_iterator;
+
+		if (prefetch_iterator)
 		{
-			TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+			while (node->prefetch_pages < node->prefetch_target)
+			{
+				TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator);
+
+				if (tbmpre == NULL)
+				{
+					/* No more pages to prefetch */
+					tbm_end_iterate(prefetch_iterator);
+					node->prefetch_iterator = NULL;
+					break;
+				}
+				node->prefetch_pages++;
+				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
+			}
+		}
+
+		return;
+	}
+
+	if (pstate->prefetch_pages < pstate->prefetch_target)
+	{
+		TBMSharedIterator *prefetch_iterator = node->shared_prefetch_iterator;
 
-			if (tbmpre == NULL)
+		if (prefetch_iterator)
+		{
+			while (1)
 			{
-				/* No more pages to prefetch */
-				tbm_end_iterate(prefetch_iterator);
-				node->prefetch_iterator = NULL;
-				break;
+				TBMIterateResult *tbmpre;
+				bool		do_prefetch = false;
+
+				/*
+				 * Recheck under the mutex, If some other process has already
+				 * done the enough prefetch then we need not to do anything.
+				 */
+				SpinLockAcquire(&pstate->mutex);
+				if (pstate->prefetch_pages < pstate->prefetch_target)
+				{
+					pstate->prefetch_pages++;
+					do_prefetch = true;
+				}
+				SpinLockRelease(&pstate->mutex);
+
+				if (!do_prefetch)
+					return;
+
+				tbmpre = tbm_shared_iterate(prefetch_iterator);
+				if (tbmpre == NULL)
+				{
+					/* No more pages to prefetch */
+					tbm_end_shared_iterate(prefetch_iterator);
+					node->shared_prefetch_iterator = NULL;
+					break;
+				}
+
+				PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 			}
-			node->prefetch_pages++;
-			PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno);
 		}
 	}
 #endif   /* USE_PREFETCH */
@@ -488,12 +695,36 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->tbmiterator);
 	if (node->prefetch_iterator)
 		tbm_end_iterate(node->prefetch_iterator);
+	if (node->shared_tbmiterator)
+		tbm_end_shared_iterate(node->shared_tbmiterator);
+	if (node->shared_prefetch_iterator)
+		tbm_end_shared_iterate(node->shared_prefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
 	node->tbm = NULL;
 	node->tbmiterator = NULL;
 	node->tbmres = NULL;
 	node->prefetch_iterator = NULL;
+	node->initialized = false;
+	node->shared_tbmiterator = NULL;
+	node->shared_prefetch_iterator = NULL;
+
+	/* Reset parallel bitmap state, if present */
+	if (node->pstate)
+	{
+		dsa_area   *dsa = node->ss.ps.state->es_query_dsa;
+
+		node->pstate->state = BM_INITIAL;
+
+		if (DsaPointerIsValid(node->pstate->tbmiterator))
+			tbm_free_shared_area(dsa, node->pstate->tbmiterator);
+
+		if (DsaPointerIsValid(node->pstate->prefetch_iterator))
+			tbm_free_shared_area(dsa, node->pstate->prefetch_iterator);
+
+		node->pstate->tbmiterator = InvalidDsaPointer;
+		node->pstate->prefetch_iterator = InvalidDsaPointer;
+	}
 
 	ExecScanReScan(&node->ss);
 
@@ -546,6 +777,10 @@ ExecEndBitmapHeapScan(BitmapHeapScanState *node)
 		tbm_end_iterate(node->prefetch_iterator);
 	if (node->tbm)
 		tbm_free(node->tbm);
+	if (node->shared_tbmiterator)
+		tbm_end_shared_iterate(node->shared_tbmiterator);
+	if (node->shared_prefetch_iterator)
+		tbm_end_shared_iterate(node->shared_prefetch_iterator);
 
 	/*
 	 * close heap scan
@@ -597,6 +832,10 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	scanstate->prefetch_target = 0;
 	/* may be updated below */
 	scanstate->prefetch_maximum = target_prefetch_pages;
+	scanstate->pscan_len = 0;
+	scanstate->initialized = false;
+	scanstate->shared_tbmiterator = NULL;
+	scanstate->pstate = NULL;
 
 	/*
 	 * Miscellaneous initialization
@@ -681,3 +920,109 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags)
 	 */
 	return scanstate;
 }
+
+/*----------------
+ *		BitmapShouldInitializeSharedState
+ *
+ *		The first process to come here and see the state to the BM_INITIAL
+ *		will become the leader for the parallel bitmap scan and will be
+ *		responsible for populating the TIDBitmap.  The other processes will
+ *		be blocked by the condition variable until the leader wakes them up.
+ * ---------------
+ */
+static bool
+BitmapShouldInitializeSharedState(ParallelBitmapHeapState *pstate)
+{
+	SharedBitmapState state;
+
+	while (1)
+	{
+		SpinLockAcquire(&pstate->mutex);
+		state = pstate->state;
+		if (pstate->state == BM_INITIAL)
+			pstate->state = BM_INPROGRESS;
+		SpinLockRelease(&pstate->mutex);
+
+		/* If we are leader or leader has already created a TIDBITMAP */
+		if (state != BM_INPROGRESS)
+			break;
+
+		/* Sleep until leader send wake up signal */
+		ConditionVariableSleep(&pstate->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN);
+	}
+
+	ConditionVariableCancelSleep();
+
+	return (state == BM_INITIAL);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapEstimate
+ *
+ *		estimates the space required to serialize bitmap scan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+
+	/* Estimate the size for sharing parallel Bitmap info. */
+	node->pscan_len = add_size(offsetof(ParallelBitmapHeapState,
+										phs_snapshot_data),
+							   EstimateSnapshotSpace(estate->es_snapshot));
+
+	shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeDSM
+ *
+ *		Set up a parallel bitmap heap scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt)
+{
+	ParallelBitmapHeapState *pstate;
+	EState	   *estate = node->ss.ps.state;
+
+	pstate = shm_toc_allocate(pcxt->toc, node->pscan_len);
+
+	pstate->tbmiterator = 0;
+	pstate->prefetch_iterator = 0;
+
+	/* Initialize the mutex */
+	SpinLockInit(&pstate->mutex);
+	pstate->prefetch_pages = 0;
+	pstate->prefetch_target = 0;
+	pstate->state = BM_INITIAL;
+
+	ConditionVariableInit(&pstate->cv);
+	SerializeSnapshot(estate->es_snapshot, pstate->phs_snapshot_data);
+
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pstate);
+	node->pstate = pstate;
+}
+
+/* ----------------------------------------------------------------
+ *		ExecBitmapHeapInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc)
+{
+	ParallelBitmapHeapState *pstate;
+	Snapshot	snapshot;
+
+	pstate = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->pstate = pstate;
+
+	snapshot = RestoreSnapshot(pstate->phs_snapshot_data);
+	heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot);
+}
diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c
index 94bb289..ce2f321 100644
--- a/src/backend/executor/nodeBitmapIndexscan.c
+++ b/src/backend/executor/nodeBitmapIndexscan.c
@@ -78,7 +78,9 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node)
 	else
 	{
 		/* XXX should we use less than work_mem for this? */
-		tbm = tbm_create(work_mem * 1024L, NULL);
+		tbm = tbm_create(work_mem * 1024L,
+						 ((BitmapIndexScan *) node->ss.ps.plan)->isshared ?
+						 node->ss.ps.state->es_query_dsa : NULL);
 	}
 
 	/*
diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c
index 1d280be..c0f2614 100644
--- a/src/backend/executor/nodeBitmapOr.c
+++ b/src/backend/executor/nodeBitmapOr.c
@@ -129,7 +129,9 @@ MultiExecBitmapOr(BitmapOrState *node)
 			if (result == NULL) /* first subplan */
 			{
 				/* XXX should we use less than work_mem for this? */
-				result = tbm_create(work_mem * 1024L, NULL);
+				result = tbm_create(work_mem * 1024L,
+									((BitmapOr *) node->ps.plan)->isshared ?
+									node->ps.state->es_query_dsa : NULL);
 			}
 
 			((BitmapIndexScanState *) subnode)->biss_result = result;
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index bb2a8a3..ae08d2c 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -331,6 +331,7 @@ _copyBitmapOr(const BitmapOr *from)
 	/*
 	 * copy remainder of node
 	 */
+	COPY_SCALAR_FIELD(isshared);
 	COPY_NODE_FIELD(bitmapplans);
 
 	return newnode;
@@ -496,6 +497,7 @@ _copyBitmapIndexScan(const BitmapIndexScan *from)
 	 * copy remainder of node
 	 */
 	COPY_SCALAR_FIELD(indexid);
+	COPY_SCALAR_FIELD(isshared);
 	COPY_NODE_FIELD(indexqual);
 	COPY_NODE_FIELD(indexqualorig);
 
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b3802b4..0ec4e1b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -441,6 +441,7 @@ _outBitmapOr(StringInfo str, const BitmapOr *node)
 
 	_outPlanInfo(str, (const Plan *) node);
 
+	WRITE_BOOL_FIELD(isshared);
 	WRITE_NODE_FIELD(bitmapplans);
 }
 
@@ -520,6 +521,7 @@ _outBitmapIndexScan(StringInfo str, const BitmapIndexScan *node)
 	_outScanInfo(str, (const Scan *) node);
 
 	WRITE_OID_FIELD(indexid);
+	WRITE_BOOL_FIELD(isshared);
 	WRITE_NODE_FIELD(indexqual);
 	WRITE_NODE_FIELD(indexqualorig);
 }
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 05bf2e9..d0d68cc 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1605,6 +1605,7 @@ _readBitmapOr(void)
 
 	ReadCommonPlan(&local_node->plan);
 
+	READ_BOOL_FIELD(isshared);
 	READ_NODE_FIELD(bitmapplans);
 
 	READ_DONE();
@@ -1716,6 +1717,7 @@ _readBitmapIndexScan(void)
 	ReadCommonScan(&local_node->scan);
 
 	READ_OID_FIELD(indexid);
+	READ_BOOL_FIELD(isshared);
 	READ_NODE_FIELD(indexqual);
 	READ_NODE_FIELD(indexqualorig);
 
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 87a3faf..9240125 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2878,6 +2878,30 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 }
 
 /*
+ * create_partial_bitmap_paths
+ *	  Build partial bitmap heap path for the relation
+ */
+void
+create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+							Path *bitmapqual)
+{
+	int			parallel_workers;
+	double		pages_fetched;
+
+	/* Compute heap pages for bitmap heap scan */
+	pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0,
+										 NULL, NULL);
+
+	parallel_workers = compute_parallel_worker(rel, pages_fetched, 0);
+
+	if (parallel_workers <= 0)
+		return;
+
+	add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel,
+					bitmapqual, rel->lateral_relids, 1.0, parallel_workers));
+}
+
+/*
  * Compute the number of parallel workers that should be used to scan a
  * relation.  We compute the parallel workers based on the size of the heap to
  * be scanned and the size of the index to be scanned, then choose a minimum
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index c138f57..b8cde32 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -860,6 +860,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 	QualCost	qpqual_cost;
 	Cost		cpu_per_tuple;
 	Cost		cost_per_page;
+	Cost		cpu_run_cost;
 	double		tuples_fetched;
 	double		pages_fetched;
 	double		spc_seq_page_cost,
@@ -921,8 +922,21 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+	cpu_run_cost = cpu_per_tuple * tuples_fetched;
+
+	/* Adjust costing for parallelism, if used. */
+	if (path->parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(path);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+		path->rows = clamp_row_est(path->rows / parallel_divisor);
+	}
+
+
+	run_cost += cpu_run_cost;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->pathtarget->cost.startup;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index d8e5b81..c2b72d4 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 
 		bitmapqual = choose_bitmap_and(root, rel, bitindexpaths);
 		bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-										rel->lateral_relids, 1.0);
+										rel->lateral_relids, 1.0, 0);
 		add_path(rel, (Path *) bpath);
+
+		/* create a partial bitmap heap path */
+		if (rel->consider_parallel && rel->lateral_relids == NULL)
+			create_partial_bitmap_paths(root, rel, bitmapqual);
 	}
 
 	/*
@@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel)
 			required_outer = get_bitmap_tree_required_outer(bitmapqual);
 			loop_count = get_loop_count(root, rel->relid, required_outer);
 			bpath = create_bitmap_heap_path(root, rel, bitmapqual,
-											required_outer, loop_count);
+											required_outer, loop_count, 0);
 			add_path(rel, (Path *) bpath);
 		}
 	}
@@ -1617,6 +1621,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = ipath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
 						  ipath,
@@ -1659,6 +1668,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths)
 	bpath.path.pathkeys = NIL;
 	bpath.bitmapqual = (Path *) &apath;
 
+	/*
+	 * Check the cost of temporary path without considering parallelism.
+	 * Parallel bitmap heap path will be considered at later stage.
+	 */
+	bpath.path.parallel_workers = 0;
+
 	/* Now we can do cost_bitmap_heap_scan */
 	cost_bitmap_heap_scan(&bpath.path, root, rel,
 						  bpath.path.param_info,
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 1e953b4..eaf85da 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -125,6 +125,7 @@ static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
 						List *tlist, List *scan_clauses);
 static Plan *create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 					  List **qual, List **indexqual, List **indexECs);
+static void bitmap_subplan_mark_shared(Plan *plan);
 static TidScan *create_tidscan_plan(PlannerInfo *root, TidPath *best_path,
 					List *tlist, List *scan_clauses);
 static SubqueryScan *create_subqueryscan_plan(PlannerInfo *root,
@@ -2577,6 +2578,9 @@ create_bitmap_scan_plan(PlannerInfo *root,
 										   &bitmapqualorig, &indexquals,
 										   &indexECs);
 
+	if (best_path->path.parallel_aware)
+		bitmap_subplan_mark_shared(bitmapqualplan);
+
 	/*
 	 * The qpqual list must contain all restrictions not automatically handled
 	 * by the index, other than pseudoconstant clauses which will be handled
@@ -2712,6 +2716,7 @@ create_bitmap_subplan(PlannerInfo *root, Path *bitmapqual,
 			subplan = create_bitmap_subplan(root, (Path *) lfirst(l),
 											&subqual, &subindexqual,
 											&subindexEC);
+
 			subplans = lappend(subplans, subplan);
 			subquals = list_concat_unique(subquals, subqual);
 			subindexquals = list_concat_unique(subindexquals, subindexqual);
@@ -4700,6 +4705,24 @@ label_sort_with_costsize(PlannerInfo *root, Sort *plan, double limit_tuples)
 	plan->plan.parallel_aware = false;
 }
 
+/*
+ * bitmap_subplan_mark_shared
+ *	 Mark a shared flag in bitmap subplan so that it can create underlying
+ *	 bitmap in shared memory which is accecible by the multiple processes.
+ */
+static void
+bitmap_subplan_mark_shared(Plan *plan)
+{
+	if (IsA(plan, BitmapAnd))
+		bitmap_subplan_mark_shared(
+								linitial(((BitmapAnd *) plan)->bitmapplans));
+	else if (IsA(plan, BitmapOr))
+		((BitmapOr *) plan)->isshared = true;
+	else if (IsA(plan, BitmapIndexScan))
+		((BitmapIndexScan *) plan)->isshared = true;
+	else
+		elog(ERROR, "unrecognized node type: %d", nodeTag(plan));
+}
 
 /*****************************************************************************
  *
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3248296..6e70808 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1068,7 +1068,8 @@ create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count)
+						double loop_count,
+						int parallel_degree)
 {
 	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
 
@@ -1077,9 +1078,9 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
-	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_aware = parallel_degree > 0 ? true : false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_degree;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
@@ -3255,7 +3256,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														rel,
 														bpath->bitmapqual,
 														required_outer,
-														loop_count);
+														loop_count, 0);
 			}
 		case T_SubqueryScan:
 			{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 2fb9a8b..7cacb1e 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3395,6 +3395,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_BITMAP_SCAN:
+			event_name = "ParallelBitmapScan";
+			break;
 		case WAIT_EVENT_SAFE_SNAPSHOT:
 			event_name = "SafeSnapshot";
 			break;
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index a864f78..7e85510 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -179,6 +179,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid,
 				   HeapTuple tup);
 
 extern void heap_sync(Relation relation);
+extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot);
 
 /* in heap/pruneheap.c */
 extern void heap_page_prune_opt(Relation relation, Buffer buffer);
diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h
index d7659b9..465c58e 100644
--- a/src/include/executor/nodeBitmapHeapscan.h
+++ b/src/include/executor/nodeBitmapHeapscan.h
@@ -15,10 +15,17 @@
 #define NODEBITMAPHEAPSCAN_H
 
 #include "nodes/execnodes.h"
+#include "access/parallel.h"
 
 extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags);
 extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node);
 extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node);
+extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node,
+					   ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node,
+							ParallelContext *pcxt);
+extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node,
+							   shm_toc *toc);
 
 #endif   /* NODEBITMAPHEAPSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 6332ea0..8698441 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -26,6 +26,8 @@
 #include "utils/sortsupport.h"
 #include "utils/tuplestore.h"
 #include "utils/tuplesort.h"
+#include "nodes/tidbitmap.h"
+#include "storage/condition_variable.h"
 
 
 /* ----------------
@@ -1465,6 +1467,52 @@ typedef struct BitmapIndexScanState
 } BitmapIndexScanState;
 
 /* ----------------
+ *	 SharedBitmapState information
+ *
+ *		BM_INITIAL		TIDBitmap creation is not yet started, so first worker
+ *						to see this state will set the state to BM_INPROGRESS
+ *						and that process will be responsible for creating
+ *						TIDBitmap.
+ *		BM_INPROGRESS	TIDBitmap creation is already in progress, therefore
+ *						workers need to sleep until leader set the state to
+ *						BM_FINISHED and wake us up.
+ *		BM_FINISHED		TIDBitmap creation is done, so now all worker can
+ *						proceed to iterate over TIDBitmap.
+ * ----------------
+ */
+typedef enum
+{
+	BM_INITIAL,
+	BM_INPROGRESS,
+	BM_FINISHED
+} SharedBitmapState;
+
+/* ----------------
+ *	 ParallelBitmapHeapState information
+ *		tbmiterator				iterator for scanning current pages
+ *		prefetch_iterator		iterator for prefetching ahead of current page
+ *		mutex					mutual exclusion for the prefetching variable
+ *								and state
+ *		prefetch_pages			# pages prefetch iterator is ahead of current
+ *		prefetch_target			current target prefetch distance
+ *		state					current state of the TIDBitmap
+ *		cv						conditional wait variable
+ *		phs_snapshot_data		snapshot data shared to workers
+ * ----------------
+ */
+typedef struct ParallelBitmapHeapState
+{
+	dsa_pointer tbmiterator;
+	dsa_pointer prefetch_iterator;
+	slock_t		mutex;
+	int			prefetch_pages;
+	int			prefetch_target;
+	SharedBitmapState state;
+	ConditionVariable cv;
+	char		phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+} ParallelBitmapHeapState;
+
+/* ----------------
  *	 BitmapHeapScanState information
  *
  *		bitmapqualorig	   execution state for bitmapqualorig expressions
@@ -1477,6 +1525,11 @@ typedef struct BitmapIndexScanState
  *		prefetch_pages	   # pages prefetch iterator is ahead of current
  *		prefetch_target    current target prefetch distance
  *		prefetch_maximum   maximum value for prefetch_target
+ *		pscan_len		   size of the shared memory for parallel bitmap
+ *		initialized		   is node is ready to iterate
+ *		shared_tbmiterator	   shared iterator
+ *		shared_prefetch_iterator shared iterator for prefetching
+ *		pstate			   shared state for parallel bitmap scan
  * ----------------
  */
 typedef struct BitmapHeapScanState
@@ -1492,6 +1545,11 @@ typedef struct BitmapHeapScanState
 	int			prefetch_pages;
 	int			prefetch_target;
 	int			prefetch_maximum;
+	Size		pscan_len;
+	bool		initialized;
+	TBMSharedIterator *shared_tbmiterator;
+	TBMSharedIterator *shared_prefetch_iterator;
+	ParallelBitmapHeapState *pstate;
 } BitmapHeapScanState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f72f7a8..7f38ca6 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -292,6 +292,7 @@ typedef struct BitmapAnd
 typedef struct BitmapOr
 {
 	Plan		plan;
+	bool		isshared;
 	List	   *bitmapplans;
 } BitmapOr;
 
@@ -420,6 +421,7 @@ typedef struct BitmapIndexScan
 {
 	Scan		scan;
 	Oid			indexid;		/* OID of index to scan */
+	bool		isshared;		/* Create shared bitmap if set */
 	List	   *indexqual;		/* list of index quals (OpExprs) */
 	List	   *indexqualorig;	/* the same in original form */
 } BitmapIndexScan;
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 53cad24..d22b039 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -53,7 +53,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						Relids required_outer,
-						double loop_count);
+						double loop_count,
+						int parallel_degree);
 extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index bc0dcf4..247fd11 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -56,6 +56,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
 extern int compute_parallel_worker(RelOptInfo *rel, BlockNumber heap_pages,
 						BlockNumber index_pages);
+extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
+										Path *bitmapqual);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 0062fb8..60c78d1 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -787,6 +787,7 @@ typedef enum
 	WAIT_EVENT_MQ_RECEIVE,
 	WAIT_EVENT_MQ_SEND,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_SAFE_SNAPSHOT,
 	WAIT_EVENT_SYNC_REP
 } WaitEventIPC;
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 75558d0..290b735 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -169,6 +169,25 @@ select  count(*) from tenk1 where thousand > 95;
 
 reset enable_seqscan;
 reset enable_bitmapscan;
+-- test parallel bitmap heap scan.
+set enable_seqscan to off;
+set enable_indexscan to off;
+explain (costs off)
+	select  count((unique1)) from tenk1 where hundred > 1;
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Finalize Aggregate
+   ->  Gather
+         Workers Planned: 4
+         ->  Partial Aggregate
+               ->  Parallel Bitmap Heap Scan on tenk1
+                     Recheck Cond: (hundred > 1)
+                     ->  Bitmap Index Scan on tenk1_hundred
+                           Index Cond: (hundred > 1)
+(8 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
 -- test parallel merge join path.
 set enable_hashjoin to off;
 set enable_nestloop to off;
diff --git a/src/test/regress/sql/select_parallel.sql b/src/test/regress/sql/select_parallel.sql
index ebdae7e..80412b9 100644
--- a/src/test/regress/sql/select_parallel.sql
+++ b/src/test/regress/sql/select_parallel.sql
@@ -64,6 +64,16 @@ select  count(*) from tenk1 where thousand > 95;
 reset enable_seqscan;
 reset enable_bitmapscan;
 
+-- test parallel bitmap heap scan.
+set enable_seqscan to off;
+set enable_indexscan to off;
+
+explain (costs off)
+	select  count((unique1)) from tenk1 where hundred > 1;
+
+reset enable_seqscan;
+reset enable_indexscan;
+
 -- test parallel merge join path.
 set enable_hashjoin to off;
 set enable_nestloop to off;
#117Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#116)
Re: Parallel bitmap heap scan

On Wed, Mar 8, 2017 at 11:20 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

Right, done that way

This didn't compile because you bobbled some code in
src/backend/nodes, but it was a trivial mistake so I fixed it.

Committed with that fix and a bunch of minor cosmetic changes.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#118Jeff Janes
jeff.janes@gmail.com
In reply to: Robert Haas (#117)
Re: Parallel bitmap heap scan

On Wed, Mar 8, 2017 at 9:08 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Mar 8, 2017 at 11:20 AM, Dilip Kumar <dilipbalaut@gmail.com>
wrote:

Right, done that way

This didn't compile because you bobbled some code in
src/backend/nodes, but it was a trivial mistake so I fixed it.

Committed with that fix and a bunch of minor cosmetic changes.

I'm getting some compiler warnings in gcc version 4.4.7 20120313 (Red Hat
4.4.7-17) (GCC):

nodeBitmapHeapscan.c: In function 'BitmapHeapNext':
nodeBitmapHeapscan.c:79: warning: 'tbmiterator' may be used uninitialized
in this function
nodeBitmapHeapscan.c:80: warning: 'shared_tbmiterator' may be used
uninitialized in this function

Cheers,

Jeff

#119Tom Lane
tgl@sss.pgh.pa.us
In reply to: Jeff Janes (#118)
Re: Parallel bitmap heap scan

Jeff Janes <jeff.janes@gmail.com> writes:

I'm getting some compiler warnings in gcc version 4.4.7 20120313 (Red Hat
4.4.7-17) (GCC):

Me too. Fix pushed.

regards, tom lane

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#120Robert Haas
robertmhaas@gmail.com
In reply to: Tom Lane (#119)
Re: Parallel bitmap heap scan

On Wed, Mar 8, 2017 at 12:45 PM, Tom Lane <tgl@sss.pgh.pa.us> wrote:

Jeff Janes <jeff.janes@gmail.com> writes:

I'm getting some compiler warnings in gcc version 4.4.7 20120313 (Red Hat
4.4.7-17) (GCC):

Me too. Fix pushed.

Thanks. Sorry for the hassle; my compiler isn't as picky about this
as I would like, and apparently Dilip's isn't either.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#121Tom Lane
tgl@sss.pgh.pa.us
In reply to: Robert Haas (#120)
Re: Parallel bitmap heap scan

Robert Haas <robertmhaas@gmail.com> writes:

On Wed, Mar 8, 2017 at 12:45 PM, Tom Lane <tgl@sss.pgh.pa.us> wrote:

Jeff Janes <jeff.janes@gmail.com> writes:

I'm getting some compiler warnings in gcc version 4.4.7 20120313 (Red Hat
4.4.7-17) (GCC):

Me too. Fix pushed.

Thanks. Sorry for the hassle; my compiler isn't as picky about this
as I would like, and apparently Dilip's isn't either.

Might be interesting to see whether -O level affects it. In principle,
whether you get the warning should depend on how much the compiler has
analyzed the logic flow ...

regards, tom lane

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#122Robert Haas
robertmhaas@gmail.com
In reply to: Tom Lane (#121)
Re: Parallel bitmap heap scan

On Wed, Mar 8, 2017 at 12:53 PM, Tom Lane <tgl@sss.pgh.pa.us> wrote:

Thanks. Sorry for the hassle; my compiler isn't as picky about this
as I would like, and apparently Dilip's isn't either.

Might be interesting to see whether -O level affects it. In principle,
whether you get the warning should depend on how much the compiler has
analyzed the logic flow ...

What I'm using is:

Configured with:
--prefix=/Applications/Xcode.app/Contents/Developer/usr
--with-gxx-include-dir=/usr/include/c++/4.2.1
Apple LLVM version 7.0.2 (clang-700.1.81)
Target: x86_64-apple-darwin14.5.0
Thread model: posix

While I haven't experimented with this too extensively, my general
impression is that this thing is extremely tolerant of uninitialized
variables. I just tried compiling nodeBitmapHeapscan.c with -Wall
-Werror and each of -O0, -O1, -O2, and -O3, and none of those produced
any warnings. I've been reluctant to go to the hassle of installing a
different compiler...

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#123Tom Lane
tgl@sss.pgh.pa.us
In reply to: Robert Haas (#122)
Re: Parallel bitmap heap scan

Robert Haas <robertmhaas@gmail.com> writes:

What I'm using is:

Configured with:
--prefix=/Applications/Xcode.app/Contents/Developer/usr
--with-gxx-include-dir=/usr/include/c++/4.2.1
Apple LLVM version 7.0.2 (clang-700.1.81)
Target: x86_64-apple-darwin14.5.0
Thread model: posix

Hm. I noticed that longfin didn't spit up on it either, despite having
-Werror turned on. That's a slightly newer version, but still Apple's
clang:

$ gcc -v
Configured with: --prefix=/Applications/Xcode.app/Contents/Developer/usr --with-gxx-include-dir=/usr/include/c++/4.2.1
Apple LLVM version 8.0.0 (clang-800.0.42.1)
Target: x86_64-apple-darwin16.4.0
Thread model: posix
InstalledDir: /Applications/Xcode.app/Contents/Developer/Toolchains/XcodeDefault.xctoolchain/usr/bin

regards, tom lane

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#124Robert Haas
robertmhaas@gmail.com
In reply to: Tom Lane (#123)
Re: Parallel bitmap heap scan

On Wed, Mar 8, 2017 at 1:18 PM, Tom Lane <tgl@sss.pgh.pa.us> wrote:

Robert Haas <robertmhaas@gmail.com> writes:

What I'm using is:

Configured with:
--prefix=/Applications/Xcode.app/Contents/Developer/usr
--with-gxx-include-dir=/usr/include/c++/4.2.1
Apple LLVM version 7.0.2 (clang-700.1.81)
Target: x86_64-apple-darwin14.5.0
Thread model: posix

Hm. I noticed that longfin didn't spit up on it either, despite having
-Werror turned on. That's a slightly newer version, but still Apple's
clang:

$ gcc -v
Configured with: --prefix=/Applications/Xcode.app/Contents/Developer/usr --with-gxx-include-dir=/usr/include/c++/4.2.1
Apple LLVM version 8.0.0 (clang-800.0.42.1)
Target: x86_64-apple-darwin16.4.0
Thread model: posix
InstalledDir: /Applications/Xcode.app/Contents/Developer/Toolchains/XcodeDefault.xctoolchain/usr/bin

Yeah. I think on my previous MacBook Pro, you could do this without
generating a warning:

int x;
printf("%d\n", x);

The compiler on this one detects that case, but that seems to be about
as far as it goes.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#125Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Robert Haas (#124)
Re: Parallel bitmap heap scan

On Wed, Mar 8, 2017 at 11:52 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Mar 8, 2017 at 1:18 PM, Tom Lane <tgl@sss.pgh.pa.us> wrote:

Robert Haas <robertmhaas@gmail.com> writes:

What I'm using is:

Configured with:
--prefix=/Applications/Xcode.app/Contents/Developer/usr
--with-gxx-include-dir=/usr/include/c++/4.2.1
Apple LLVM version 7.0.2 (clang-700.1.81)
Target: x86_64-apple-darwin14.5.0
Thread model: posix

Hm. I noticed that longfin didn't spit up on it either, despite having
-Werror turned on. That's a slightly newer version, but still Apple's
clang:

$ gcc -v
Configured with: --prefix=/Applications/Xcode.app/Contents/Developer/usr --with-gxx-include-dir=/usr/include/c++/4.2.1
Apple LLVM version 8.0.0 (clang-800.0.42.1)
Target: x86_64-apple-darwin16.4.0
Thread model: posix
InstalledDir: /Applications/Xcode.app/Contents/Developer/Toolchains/XcodeDefault.xctoolchain/usr/bin

Yeah. I think on my previous MacBook Pro, you could do this without
generating a warning:

int x;
printf("%d\n", x);

The compiler on this one detects that case, but that seems to be about
as far as it goes.

Recently, on testing TPC-H 300 scale factor I stumbled on to a error
for Q6, the test environment is as follows,
work_mem = 1GB,
shared_buffers = 10 GB,
Effective_cache_size = 10GB
random_page_cost = seq+page_cost =0.1

The error message is, ERROR: invalid DSA memory alloc request size 1610612740
In case required, stack trace is as follows,

#0 errfinish (dummy=0) at elog.c:415
#1 0x0000000010738820 in elog_finish (elevel=20, fmt=0x109115d0
"invalid DSA memory alloc request size %zu") at elog.c:1378
#2 0x0000000010778824 in dsa_allocate_extended (area=0x1001b53b138,
size=1610612740, flags=4) at dsa.c:676
#3 0x00000000103a3e60 in pagetable_allocate (pagetable=0x1001b52a590,
size=1610612736) at tidbitmap.c:1521
#4 0x00000000103a0488 in pagetable_grow (tb=0x1001b52a590,
newsize=33554432) at ../../../src/include/lib/simplehash.h:379
#5 0x00000000103a07b0 in pagetable_insert (tb=0x1001b52a590,
key=34962730, found=0x3fffc705ba48 "") at
../../../src/include/lib/simplehash.h:504
#6 0x00000000103a3354 in tbm_get_pageentry (tbm=0x1001b526fa0,
pageno=34962730) at tidbitmap.c:1235
#7 0x00000000103a1614 in tbm_add_tuples (tbm=0x1001b526fa0,
tids=0x1001b51d22a, ntids=1, recheck=0 '\000') at tidbitmap.c:425
#8 0x00000000100fdf24 in btgetbitmap (scan=0x1001b51c4a0,
tbm=0x1001b526fa0) at nbtree.c:460
#9 0x00000000100f2c48 in index_getbitmap (scan=0x1001b51c4a0,
bitmap=0x1001b526fa0) at indexam.c:726
#10 0x000000001033c7d8 in MultiExecBitmapIndexScan
(node=0x1001b51c180) at nodeBitmapIndexscan.c:91
#11 0x000000001031c0b4 in MultiExecProcNode (node=0x1001b51c180) at
execProcnode.c:611
#12 0x000000001033a8a8 in BitmapHeapNext (node=0x1001b5187a8) at
nodeBitmapHeapscan.c:143
#13 0x000000001032a094 in ExecScanFetch (node=0x1001b5187a8,
accessMtd=0x1033a6c8 <BitmapHeapNext>, recheckMtd=0x1033bab8
<BitmapHeapRecheck>) at execScan.c:95
#14 0x000000001032a194 in ExecScan (node=0x1001b5187a8,
accessMtd=0x1033a6c8 <BitmapHeapNext>, recheckMtd=0x1033bab8
<BitmapHeapRecheck>) at execScan.c:162
#15 0x000000001033bb88 in ExecBitmapHeapScan (node=0x1001b5187a8) at
nodeBitmapHeapscan.c:667

Please feel free to ask if any more information is required for this.
--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#126Dilip Kumar
dilipbalaut@gmail.com
In reply to: Rafia Sabih (#125)
1 attachment(s)
Re: Parallel bitmap heap scan

On Mon, Mar 27, 2017 at 12:53 PM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Recently, on testing TPC-H 300 scale factor I stumbled on to a error
for Q6, the test environment is as follows,
work_mem = 1GB,
shared_buffers = 10 GB,
Effective_cache_size = 10GB
random_page_cost = seq+page_cost =0.1

The error message is, ERROR: invalid DSA memory alloc request size 1610612740
In case required, stack trace is as follows,

Thanks for reporting. In pagetable_allocate, DSA_ALLOC_HUGE flag were
missing while allocating the memory from the DSA. I have also handled
the NULL pointer return from dsa_get_address.

Please verify with the attached patch.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

Attachments:

bitmap_hugepage_fix.patchapplication/octet-stream; name=bitmap_hugepage_fix.patchDownload
diff --git a/src/backend/nodes/tidbitmap.c b/src/backend/nodes/tidbitmap.c
index ae7a913..d7475ab 100644
--- a/src/backend/nodes/tidbitmap.c
+++ b/src/backend/nodes/tidbitmap.c
@@ -1522,7 +1522,7 @@ static inline void *
 pagetable_allocate(pagetable_hash *pagetable, Size size)
 {
 	TIDBitmap  *tbm = (TIDBitmap *) pagetable->private_data;
-	PTEntryArray *ptbase;
+	PTEntryArray *ptbase = NULL;
 
 	if (tbm->dsa == NULL)
 		return MemoryContextAllocExtended(pagetable->ctx, size,
@@ -1533,9 +1533,13 @@ pagetable_allocate(pagetable_hash *pagetable, Size size)
 	 * new memory so that pagetable_free can free the old entry.
 	 */
 	tbm->dsapagetableold = tbm->dsapagetable;
-	tbm->dsapagetable = dsa_allocate0(tbm->dsa, sizeof(PTEntryArray) + size);
-
+	tbm->dsapagetable = dsa_allocate_extended(tbm->dsa,
+											  sizeof(PTEntryArray) + size,
+											  DSA_ALLOC_HUGE | DSA_ALLOC_ZERO);
 	ptbase = dsa_get_address(tbm->dsa, tbm->dsapagetable);
+	if (ptbase == NULL)
+		return NULL;
+
 	return ptbase->ptentry;
 }
 
#127Robert Haas
robertmhaas@gmail.com
In reply to: Dilip Kumar (#126)
Re: Parallel bitmap heap scan

On Mon, Mar 27, 2017 at 5:02 AM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Mon, Mar 27, 2017 at 12:53 PM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Recently, on testing TPC-H 300 scale factor I stumbled on to a error
for Q6, the test environment is as follows,
work_mem = 1GB,
shared_buffers = 10 GB,
Effective_cache_size = 10GB
random_page_cost = seq+page_cost =0.1

The error message is, ERROR: invalid DSA memory alloc request size 1610612740
In case required, stack trace is as follows,

Thanks for reporting. In pagetable_allocate, DSA_ALLOC_HUGE flag were
missing while allocating the memory from the DSA. I have also handled
the NULL pointer return from dsa_get_address.

Please verify with the attached patch.

Failing to pass DSA_ALLOC_HUGE is an obvious oversight, but I think
the ptbase == NULL check shouldn't be needed, because we're not
passing DSA_ALLOC_NO_OOM. And that's good, because this is going to
be called from SH_CREATE, which isn't prepared for a NULL return
anyway.

Am I all wet?

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#128Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#127)
Re: Parallel bitmap heap scan

On Mon, Mar 27, 2017 at 5:58 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Failing to pass DSA_ALLOC_HUGE is an obvious oversight, but I think
the ptbase == NULL check shouldn't be needed, because we're not
passing DSA_ALLOC_NO_OOM. And that's good, because this is going to
be called from SH_CREATE, which isn't prepared for a NULL return
anyway.

Am I all wet?

Yes you are right that we are not passing DSA_ALLOC_NO_OOM, so
dsa_allocate_extended will return error in case if allocation failed.

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers