diff --git a/contrib/bloom/blutils.c b/contrib/bloom/blutils.c
index f5ee7bc..f79ecfc 100644
--- a/contrib/bloom/blutils.c
+++ b/contrib/bloom/blutils.c
@@ -119,6 +119,7 @@ blhandler(PG_FUNCTION_ARGS)
 	amroutine->amstorage = false;
 	amroutine->amclusterable = false;
 	amroutine->ampredlocks = false;
+	amroutine->amcanparallel = false;
 	amroutine->amkeytype = InvalidOid;
 
 	amroutine->ambuild = blbuild;
diff --git a/doc/src/sgml/indexam.sgml b/doc/src/sgml/indexam.sgml
index df71c06..4d17248 100644
--- a/doc/src/sgml/indexam.sgml
+++ b/doc/src/sgml/indexam.sgml
@@ -110,6 +110,8 @@ typedef struct IndexAmRoutine
     bool        amclusterable;
     /* does AM handle predicate locks? */
     bool        ampredlocks;
+    /* does AM support parallel scan? */
+    bool        amcanparallel;
     /* type of data stored in index, or InvalidOid if variable */
     Oid         amkeytype;
 
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index 5d4bb21..cfc3435 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -268,15 +268,28 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
   <title>Parallel Scans</title>
 
   <para>
-    Currently, the only type of scan which has been modified to work with
-    parallel query is a sequential scan.  Therefore, the driving table in
-    a parallel plan will always be scanned using a
-    <literal>Parallel Seq Scan</>.  The relation's blocks will be divided
-    among the cooperating processes.  Blocks are handed out one at a
-    time, so that access to the relation remains sequential.  Each process
-    will visit every tuple on the page assigned to it before requesting a new
-    page.
+    Currently, the type of scans that work with the parallel query are sequential
+    and index scans.
   </para>
+   
+  <para>
+    In <literal>Parallel Sequential Scans</>, the driving table in a parallel plan
+    will always be scanned using a <literal>Parallel Seq Scan</>.  The relation's
+    blocks will be divided among the cooperating processes.  Blocks are handed out
+    one at a time, so that access to the relation remains sequential.  Each process
+    will visit every tuple on the page assigned to it before requesting a new page.
+  </para>
+
+  <para>
+    In <literal>Parallel Index Scans</>, the driving table in a parallel plan will
+    always be scanned using a <literal>Parallel Index Scan</>.  Currently, the only
+    type of index which has been modified to work with the parallel query is
+    <literal>btree</>.  The parallelism is performed at the leaf level of <literal>btree</>.
+    The first backend (either master or worker backend) to start a scan will scan till
+    leaf and others will wait till it reaches the leaf level.  At leaf level, blocks are
+    handed out one at a time similar to <literal>Parallel Seq Scan</> till all the blocks
+    are finished or scan has reached the end point.
+   </para>
  </sect2>
 
  <sect2 id="parallel-joins">
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 826f3cc..b9ddcea 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -93,6 +93,7 @@ brinhandler(PG_FUNCTION_ARGS)
 	amroutine->amstorage = true;
 	amroutine->amclusterable = false;
 	amroutine->ampredlocks = false;
+	amroutine->amcanparallel = false;
 	amroutine->amkeytype = InvalidOid;
 
 	amroutine->ambuild = brinbuild;
diff --git a/src/backend/access/gin/ginutil.c b/src/backend/access/gin/ginutil.c
index a80039f..f6898ef 100644
--- a/src/backend/access/gin/ginutil.c
+++ b/src/backend/access/gin/ginutil.c
@@ -49,6 +49,7 @@ ginhandler(PG_FUNCTION_ARGS)
 	amroutine->amstorage = true;
 	amroutine->amclusterable = false;
 	amroutine->ampredlocks = false;
+	amroutine->amcanparallel = false;
 	amroutine->amkeytype = InvalidOid;
 
 	amroutine->ambuild = ginbuild;
diff --git a/src/backend/access/gist/gist.c b/src/backend/access/gist/gist.c
index 54f9dc4..d4978b8 100644
--- a/src/backend/access/gist/gist.c
+++ b/src/backend/access/gist/gist.c
@@ -70,6 +70,7 @@ gisthandler(PG_FUNCTION_ARGS)
 	amroutine->amstorage = true;
 	amroutine->amclusterable = true;
 	amroutine->ampredlocks = false;
+	amroutine->amcanparallel = false;
 	amroutine->amkeytype = InvalidOid;
 
 	amroutine->ambuild = gistbuild;
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index 17fd125..f3debe2 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -67,6 +67,7 @@ hashhandler(PG_FUNCTION_ARGS)
 	amroutine->amstorage = false;
 	amroutine->amclusterable = false;
 	amroutine->ampredlocks = false;
+	amroutine->amcanparallel = false;
 	amroutine->amkeytype = INT4OID;
 
 	amroutine->ambuild = hashbuild;
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index fad5f1b..28c9193 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -141,6 +141,7 @@ bthandler(PG_FUNCTION_ARGS)
 	amroutine->amstorage = false;
 	amroutine->amclusterable = true;
 	amroutine->ampredlocks = true;
+	amroutine->amcanparallel = true;
 	amroutine->amkeytype = InvalidOid;
 
 	amroutine->ambuild = btbuild;
diff --git a/src/backend/access/spgist/spgutils.c b/src/backend/access/spgist/spgutils.c
index 1729797..ae25adf 100644
--- a/src/backend/access/spgist/spgutils.c
+++ b/src/backend/access/spgist/spgutils.c
@@ -49,6 +49,7 @@ spghandler(PG_FUNCTION_ARGS)
 	amroutine->amstorage = false;
 	amroutine->amclusterable = false;
 	amroutine->ampredlocks = false;
+	amroutine->amcanparallel = false;
 	amroutine->amkeytype = InvalidOid;
 
 	amroutine->ambuild = spgbuild;
diff --git a/src/backend/executor/execAmi.c b/src/backend/executor/execAmi.c
index b52cfaa..32c0796 100644
--- a/src/backend/executor/execAmi.c
+++ b/src/backend/executor/execAmi.c
@@ -61,6 +61,7 @@
 
 static bool TargetListSupportsBackwardScan(List *targetlist);
 static bool IndexSupportsBackwardScan(Oid indexid);
+static bool GatherSupportsBackwardScan(Plan *node);
 
 
 /*
@@ -490,7 +491,7 @@ ExecSupportsBackwardScan(Plan *node)
 			return false;
 
 		case T_Gather:
-			return false;
+			return GatherSupportsBackwardScan(node);
 
 		case T_IndexScan:
 			return IndexSupportsBackwardScan(((IndexScan *) node)->indexid) &&
@@ -571,6 +572,25 @@ IndexSupportsBackwardScan(Oid indexid)
 }
 
 /*
+ * GatherSupportsBackwardScan - does a gather plan supports backward scan?
+ *
+ * Returns true if the outer plan node of gather supports backward scan.
+ * As of now, we can support backward scan, iff outer node of gather has
+ * index node.
+ */
+bool
+GatherSupportsBackwardScan(Plan *node)
+{
+	Plan	   *outer_node = outerPlan(node);
+
+	if (nodeTag(outer_node) == T_IndexScan)
+		return IndexSupportsBackwardScan(((IndexScan *) outer_node)->indexid) &&
+			TargetListSupportsBackwardScan(outer_node->targetlist);
+	else
+		return false;
+}
+
+/*
  * ExecMaterializesOutput - does a plan type materialize its output?
  *
  * Returns true if the plan node type is one that automatically materializes
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index e01fe6d..cb91cc3 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -28,6 +28,7 @@
 #include "executor/nodeCustom.h"
 #include "executor/nodeForeignscan.h"
 #include "executor/nodeSeqscan.h"
+#include "executor/nodeIndexscan.h"
 #include "executor/tqueue.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/planmain.h"
@@ -197,6 +198,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
 				ExecSeqScanEstimate((SeqScanState *) planstate,
 									e->pcxt);
 				break;
+			case T_IndexScanState:
+				ExecIndexScanEstimate((IndexScanState *) planstate,
+									  e->pcxt);
+				break;
 			case T_ForeignScanState:
 				ExecForeignScanEstimate((ForeignScanState *) planstate,
 										e->pcxt);
@@ -249,6 +254,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
 				ExecSeqScanInitializeDSM((SeqScanState *) planstate,
 										 d->pcxt);
 				break;
+			case T_IndexScanState:
+				ExecIndexScanInitializeDSM((IndexScanState *) planstate,
+										   d->pcxt);
+				break;
 			case T_ForeignScanState:
 				ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
 											 d->pcxt);
@@ -725,6 +734,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
 			case T_SeqScanState:
 				ExecSeqScanInitializeWorker((SeqScanState *) planstate, toc);
 				break;
+			case T_IndexScanState:
+				ExecIndexScanInitializeWorker((IndexScanState *) planstate, toc);
+				break;
 			case T_ForeignScanState:
 				ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
 												toc);
diff --git a/src/backend/executor/nodeIndexscan.c b/src/backend/executor/nodeIndexscan.c
index 97a6fac..2842cf9 100644
--- a/src/backend/executor/nodeIndexscan.c
+++ b/src/backend/executor/nodeIndexscan.c
@@ -22,6 +22,9 @@
  *		ExecEndIndexScan		releases all storage.
  *		ExecIndexMarkPos		marks scan position.
  *		ExecIndexRestrPos		restores scan position.
+ *		ExecIndexScanEstimate	estimates DSM space needed for parallel index scan
+ *		ExecIndexScanInitializeDSM initialize DSM for parallel indexscan
+ *		ExecIndexScanInitializeWorker attach to DSM info in parallel worker
  */
 #include "postgres.h"
 
@@ -515,6 +518,15 @@ ExecIndexScan(IndexScanState *node)
 void
 ExecReScanIndexScan(IndexScanState *node)
 {
+	bool		reset_parallel_scan = true;
+
+	/*
+	 * if we are here to just update the scan keys, then don't reset parallel
+	 * scan
+	 */
+	if (node->iss_NumRuntimeKeys != 0 && !node->iss_RuntimeKeysReady)
+		reset_parallel_scan = false;
+
 	/*
 	 * If we are doing runtime key calculations (ie, any of the index key
 	 * values weren't simple Consts), compute the new key values.  But first,
@@ -540,10 +552,16 @@ ExecReScanIndexScan(IndexScanState *node)
 			reorderqueue_pop(node);
 	}
 
-	/* reset index scan */
-	index_rescan(node->iss_ScanDesc,
-				 node->iss_ScanKeys, node->iss_NumScanKeys,
-				 node->iss_OrderByKeys, node->iss_NumOrderByKeys);
+	/* reset (parallel) index scan */
+	if (node->iss_ScanDesc)
+	{
+		index_rescan(node->iss_ScanDesc,
+					 node->iss_ScanKeys, node->iss_NumScanKeys,
+					 node->iss_OrderByKeys, node->iss_NumOrderByKeys);
+
+		if (reset_parallel_scan)
+			index_parallelrescan(node->iss_ScanDesc);
+	}
 	node->iss_ReachedEnd = false;
 
 	ExecScanReScan(&node->ss);
@@ -1018,22 +1036,29 @@ ExecInitIndexScan(IndexScan *node, EState *estate, int eflags)
 	}
 
 	/*
-	 * Initialize scan descriptor.
+	 * for parallel-aware node, we initialize the scan descriptor after
+	 * initializing the shared memory for parallel execution.
 	 */
-	indexstate->iss_ScanDesc = index_beginscan(currentRelation,
-											   indexstate->iss_RelationDesc,
-											   estate->es_snapshot,
-											   indexstate->iss_NumScanKeys,
+	if (!node->scan.plan.parallel_aware)
+	{
+		/*
+		 * Initialize scan descriptor.
+		 */
+		indexstate->iss_ScanDesc = index_beginscan(currentRelation,
+												indexstate->iss_RelationDesc,
+												   estate->es_snapshot,
+												 indexstate->iss_NumScanKeys,
 											 indexstate->iss_NumOrderByKeys);
 
-	/*
-	 * If no run-time keys to calculate, go ahead and pass the scankeys to the
-	 * index AM.
-	 */
-	if (indexstate->iss_NumRuntimeKeys == 0)
-		index_rescan(indexstate->iss_ScanDesc,
-					 indexstate->iss_ScanKeys, indexstate->iss_NumScanKeys,
+		/*
+		 * If no run-time keys to calculate, go ahead and pass the scankeys to
+		 * the index AM.
+		 */
+		if (indexstate->iss_NumRuntimeKeys == 0)
+			index_rescan(indexstate->iss_ScanDesc,
+					   indexstate->iss_ScanKeys, indexstate->iss_NumScanKeys,
 				indexstate->iss_OrderByKeys, indexstate->iss_NumOrderByKeys);
+	}
 
 	/*
 	 * all done.
@@ -1595,3 +1620,91 @@ ExecIndexBuildScanKeys(PlanState *planstate, Relation index,
 	else if (n_array_keys != 0)
 		elog(ERROR, "ScalarArrayOpExpr index qual found where not allowed");
 }
+
+/* ----------------------------------------------------------------
+ *						Parallel Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ *		ExecIndexScanEstimate
+ *
+ *		estimates the space required to serialize indexscan node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecIndexScanEstimate(IndexScanState *node,
+					  ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+
+	node->iss_PscanLen = index_parallelscan_estimate(node->iss_RelationDesc,
+													 estate->es_snapshot);
+	shm_toc_estimate_chunk(&pcxt->estimator, node->iss_PscanLen);
+	shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecIndexScanInitializeDSM
+ *
+ *		Set up a parallel index scan descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecIndexScanInitializeDSM(IndexScanState *node,
+						   ParallelContext *pcxt)
+{
+	EState	   *estate = node->ss.ps.state;
+	ParallelIndexScanDesc piscan;
+
+	piscan = shm_toc_allocate(pcxt->toc, node->iss_PscanLen);
+	index_parallelscan_initialize(node->ss.ss_currentRelation,
+								  node->iss_RelationDesc,
+								  estate->es_snapshot,
+								  piscan);
+	shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, piscan);
+	node->iss_ScanDesc =
+		index_beginscan_parallel(node->ss.ss_currentRelation,
+								 node->iss_RelationDesc,
+								 node->iss_NumScanKeys,
+								 node->iss_NumOrderByKeys,
+								 piscan);
+
+	/*
+	 * If no run-time keys to calculate, go ahead and pass the scankeys to the
+	 * index AM.
+	 */
+	if (node->iss_NumRuntimeKeys == 0)
+		index_rescan(node->iss_ScanDesc,
+					 node->iss_ScanKeys, node->iss_NumScanKeys,
+					 node->iss_OrderByKeys, node->iss_NumOrderByKeys);
+}
+
+/* ----------------------------------------------------------------
+ *		ExecIndexScanInitializeWorker
+ *
+ *		Copy relevant information from TOC into planstate.
+ * ----------------------------------------------------------------
+ */
+void
+ExecIndexScanInitializeWorker(IndexScanState *node, shm_toc *toc)
+{
+	ParallelIndexScanDesc piscan;
+
+	piscan = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id);
+	node->iss_ScanDesc =
+		index_beginscan_parallel(node->ss.ss_currentRelation,
+								 node->iss_RelationDesc,
+								 node->iss_NumScanKeys,
+								 node->iss_NumOrderByKeys,
+								 piscan);
+
+	/*
+	 * If no run-time keys to calculate, go ahead and pass the scankeys to the
+	 * index AM.
+	 */
+	if (node->iss_NumRuntimeKeys == 0)
+		index_rescan(node->iss_ScanDesc,
+					 node->iss_ScanKeys, node->iss_NumScanKeys,
+					 node->iss_OrderByKeys, node->iss_NumOrderByKeys);
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 7c017fe..b37ed7e 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -126,7 +126,6 @@ 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);
 
 
 /*
@@ -2872,7 +2871,7 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
  * relation.  "pages" is the number of pages from the relation that we
  * expect to scan.
  */
-static int
+int
 compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
 {
 	int			parallel_workers;
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 458f139..f4253d3 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -400,6 +400,7 @@ cost_index(IndexPath *path, PlannerInfo *root, double loop_count)
 	List	   *qpquals;
 	Cost		startup_cost = 0;
 	Cost		run_cost = 0;
+	Cost		cpu_run_cost = 0;
 	Cost		indexStartupCost;
 	Cost		indexTotalCost;
 	Selectivity indexSelectivity;
@@ -602,11 +603,24 @@ cost_index(IndexPath *path, PlannerInfo *root, double loop_count)
 	startup_cost += qpqual_cost.startup;
 	cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
 
-	run_cost += cpu_per_tuple * tuples_fetched;
+	cpu_run_cost += cpu_per_tuple * tuples_fetched;
 
 	/* tlist eval costs are paid per output row, not per tuple scanned */
 	startup_cost += path->path.pathtarget->cost.startup;
-	run_cost += path->path.pathtarget->cost.per_tuple * path->path.rows;
+	cpu_run_cost += path->path.pathtarget->cost.per_tuple * path->path.rows;
+
+	/* Adjust costing for parallelism, if used. */
+	if (path->path.parallel_workers > 0)
+	{
+		double		parallel_divisor = get_parallel_divisor(&path->path);
+
+		path->path.rows = clamp_row_est(path->path.rows / parallel_divisor);
+
+		/* The CPU cost is divided among all the workers. */
+		cpu_run_cost /= parallel_divisor;
+	}
+
+	run_cost += cpu_run_cost;
 
 	path->path.startup_cost = startup_cost;
 	path->path.total_cost = startup_cost + run_cost;
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 5283468..1905917 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -1042,8 +1042,43 @@ build_index_paths(PlannerInfo *root, RelOptInfo *rel,
 								  NoMovementScanDirection,
 								  index_only_scan,
 								  outer_relids,
-								  loop_count);
+								  loop_count,
+								  0);
 		result = lappend(result, ipath);
+
+		/*
+		 * If appropriate, consider parallel index scan.  We don't allow
+		 * parallel index scan for bitmap scans.
+		 */
+		if (index->amcanparallel &&
+			!index_only_scan &&
+			rel->consider_parallel &&
+			outer_relids == NULL &&
+			scantype != ST_BITMAPSCAN)
+		{
+			int			parallel_workers = 0;
+
+			parallel_workers = compute_parallel_worker(rel, index->pages);
+
+			if (parallel_workers > 0)
+			{
+				ipath = create_index_path(root, index,
+										  index_clauses,
+										  clause_columns,
+										  orderbyclauses,
+										  orderbyclausecols,
+										  useful_pathkeys,
+										  index_is_ordered ?
+										  ForwardScanDirection :
+										  NoMovementScanDirection,
+										  index_only_scan,
+										  outer_relids,
+										  loop_count,
+										  parallel_workers);
+
+				add_partial_path(rel, (Path *) ipath);
+			}
+		}
 	}
 
 	/*
@@ -1066,8 +1101,38 @@ build_index_paths(PlannerInfo *root, RelOptInfo *rel,
 									  BackwardScanDirection,
 									  index_only_scan,
 									  outer_relids,
-									  loop_count);
+									  loop_count,
+									  0);
 			result = lappend(result, ipath);
+
+			/* If appropriate, consider parallel index scan */
+			if (index->amcanparallel &&
+				!index_only_scan &&
+				rel->consider_parallel &&
+				outer_relids == NULL &&
+				scantype != ST_BITMAPSCAN)
+			{
+				int			parallel_workers = 0;
+
+				parallel_workers = compute_parallel_worker(rel, index->pages);
+
+				if (parallel_workers > 0)
+				{
+					ipath = create_index_path(root, index,
+											  index_clauses,
+											  clause_columns,
+											  NIL,
+											  NIL,
+											  useful_pathkeys,
+											  BackwardScanDirection,
+											  index_only_scan,
+											  outer_relids,
+											  loop_count,
+											  parallel_workers);
+
+					add_partial_path(rel, (Path *) ipath);
+				}
+			}
 		}
 	}
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 4b5902f..fd25749 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -5393,7 +5393,7 @@ plan_cluster_use_sort(Oid tableOid, Oid indexOid)
 	indexScanPath = create_index_path(root, indexInfo,
 									  NIL, NIL, NIL, NIL, NIL,
 									  ForwardScanDirection, false,
-									  NULL, 1.0);
+									  NULL, 1.0, 0);
 
 	return (seqScanAndSortPath.total_cost < indexScanPath->path.total_cost);
 }
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f440875..cdf5523 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -744,10 +744,8 @@ add_path_precheck(RelOptInfo *parent_rel,
  *	  As with add_path, we pfree paths that are found to be dominated by
  *	  another partial path; this requires that there be no other references to
  *	  such paths yet.  Hence, GatherPaths must not be created for a rel until
- *	  we're done creating all partial paths for it.  We do not currently build
- *	  partial indexscan paths, so there is no need for an exception for
- *	  IndexPaths here; for safety, we instead Assert that a path to be freed
- *	  isn't an IndexPath.
+ *	  we're done creating all partial paths for it.  As for add_path, we take
+ *	  an exception for IndexPaths here as well.
  */
 void
 add_partial_path(RelOptInfo *parent_rel, Path *new_path)
@@ -826,9 +824,12 @@ add_partial_path(RelOptInfo *parent_rel, Path *new_path)
 		{
 			parent_rel->partial_pathlist =
 				list_delete_cell(parent_rel->partial_pathlist, p1, p1_prev);
-			/* we should not see IndexPaths here, so always safe to delete */
-			Assert(!IsA(old_path, IndexPath));
-			pfree(old_path);
+
+			/*
+			 * Delete the data pointed-to by the deleted cell, if possible
+			 */
+			if (!IsA(old_path, IndexPath))
+				pfree(old_path);
 			/* p1_prev does not advance */
 		}
 		else
@@ -860,10 +861,9 @@ add_partial_path(RelOptInfo *parent_rel, Path *new_path)
 	}
 	else
 	{
-		/* we should not see IndexPaths here, so always safe to delete */
-		Assert(!IsA(new_path, IndexPath));
 		/* Reject and recycle the new path */
-		pfree(new_path);
+		if (!IsA(new_path, IndexPath))
+			pfree(new_path);
 	}
 }
 
@@ -1019,7 +1019,8 @@ create_index_path(PlannerInfo *root,
 				  ScanDirection indexscandir,
 				  bool indexonly,
 				  Relids required_outer,
-				  double loop_count)
+				  double loop_count,
+				  int parallel_workers)
 {
 	IndexPath  *pathnode = makeNode(IndexPath);
 	RelOptInfo *rel = index->rel;
@@ -1031,9 +1032,9 @@ create_index_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_workers > 0 ? true : false;;
 	pathnode->path.parallel_safe = rel->consider_parallel;
-	pathnode->path.parallel_workers = 0;
+	pathnode->path.parallel_workers = parallel_workers;
 	pathnode->path.pathkeys = pathkeys;
 
 	/* Convert clauses to indexquals the executor can handle */
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 7836e6b..4ed2705 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -241,6 +241,7 @@ get_relation_info(PlannerInfo *root, Oid relationObjectId, bool inhparent,
 			info->amoptionalkey = amroutine->amoptionalkey;
 			info->amsearcharray = amroutine->amsearcharray;
 			info->amsearchnulls = amroutine->amsearchnulls;
+			info->amcanparallel = amroutine->amcanparallel;
 			info->amhasgettuple = (amroutine->amgettuple != NULL);
 			info->amhasgetbitmap = (amroutine->amgetbitmap != NULL);
 			info->amcostestimate = amroutine->amcostestimate;
diff --git a/src/include/access/amapi.h b/src/include/access/amapi.h
index 18259ad..7fc8bd9 100644
--- a/src/include/access/amapi.h
+++ b/src/include/access/amapi.h
@@ -184,6 +184,8 @@ typedef struct IndexAmRoutine
 	bool		amclusterable;
 	/* does AM handle predicate locks? */
 	bool		ampredlocks;
+	/* does AM support parallel scan? */
+	bool		amcanparallel;
 	/* type of data stored in index, or InvalidOid if variable */
 	Oid			amkeytype;
 
diff --git a/src/include/executor/nodeIndexscan.h b/src/include/executor/nodeIndexscan.h
index 46d6f45..ea3f3a5 100644
--- a/src/include/executor/nodeIndexscan.h
+++ b/src/include/executor/nodeIndexscan.h
@@ -14,6 +14,7 @@
 #ifndef NODEINDEXSCAN_H
 #define NODEINDEXSCAN_H
 
+#include "access/parallel.h"
 #include "nodes/execnodes.h"
 
 extern IndexScanState *ExecInitIndexScan(IndexScan *node, EState *estate, int eflags);
@@ -22,6 +23,9 @@ extern void ExecEndIndexScan(IndexScanState *node);
 extern void ExecIndexMarkPos(IndexScanState *node);
 extern void ExecIndexRestrPos(IndexScanState *node);
 extern void ExecReScanIndexScan(IndexScanState *node);
+extern void ExecIndexScanEstimate(IndexScanState *node, ParallelContext *pcxt);
+extern void ExecIndexScanInitializeDSM(IndexScanState *node, ParallelContext *pcxt);
+extern void ExecIndexScanInitializeWorker(IndexScanState *node, shm_toc *toc);
 
 /*
  * These routines are exported to share code with nodeIndexonlyscan.c and
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 1da1e1f..b565f28 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1370,6 +1370,7 @@ typedef struct
  *		SortSupport		   for reordering ORDER BY exprs
  *		OrderByTypByVals   is the datatype of order by expression pass-by-value?
  *		OrderByTypLens	   typlens of the datatypes of order by expressions
+ *		pscan_len		   size of parallel index scan descriptor
  * ----------------
  */
 typedef struct IndexScanState
@@ -1396,6 +1397,9 @@ typedef struct IndexScanState
 	SortSupport iss_SortSupport;
 	bool	   *iss_OrderByTypByVals;
 	int16	   *iss_OrderByTypLens;
+
+	/* This is needed for parallel index scan */
+	Size		iss_PscanLen;
 } IndexScanState;
 
 /* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 643be54..f7ac6f6 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -629,6 +629,7 @@ typedef struct IndexOptInfo
 	bool		amsearchnulls;	/* can AM search for NULL/NOT NULL entries? */
 	bool		amhasgettuple;	/* does AM have amgettuple interface? */
 	bool		amhasgetbitmap; /* does AM have amgetbitmap interface? */
+	bool		amcanparallel;	/* does AM support parallel scan? */
 	/* Rather than include amapi.h here, we declare amcostestimate like this */
 	void		(*amcostestimate) ();	/* AM's cost estimator */
 } IndexOptInfo;
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 7b41317..c9884f2 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -47,7 +47,8 @@ extern IndexPath *create_index_path(PlannerInfo *root,
 				  ScanDirection indexscandir,
 				  bool indexonly,
 				  Relids required_outer,
-				  double loop_count);
+				  double loop_count,
+				  int parallel_workers);
 extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 81a9be7..fabf314 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,7 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern int	compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 18e21b7..140fb3c 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -99,6 +99,29 @@ explain (costs off)
    ->  Index Only Scan using tenk1_unique1 on tenk1
 (3 rows)
 
+-- test parallel index scans.
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+explain (costs off)
+	select  count((unique1)) from tenk1 where hundred > 1;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Finalize Aggregate
+   ->  Gather
+         Workers Planned: 4
+         ->  Partial Aggregate
+               ->  Parallel Index Scan using tenk1_hundred on tenk1
+                     Index Cond: (hundred > 1)
+(6 rows)
+
+select  count((unique1)) from tenk1 where hundred > 1;
+ count 
+-------
+  9800
+(1 row)
+
+reset enable_seqscan;
+reset enable_bitmapscan;
 set force_parallel_mode=1;
 explain (costs off)
   select stringu1::int2 from tenk1 where unique1 = 1;
diff --git a/src/test/regress/sql/select_parallel.sql b/src/test/regress/sql/select_parallel.sql
index 8b4090f..d7dfd28 100644
--- a/src/test/regress/sql/select_parallel.sql
+++ b/src/test/regress/sql/select_parallel.sql
@@ -39,6 +39,17 @@ explain (costs off)
 	select  sum(parallel_restricted(unique1)) from tenk1
 	group by(parallel_restricted(unique1));
 
+-- test parallel index scans.
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+
+explain (costs off)
+	select  count((unique1)) from tenk1 where hundred > 1;
+select  count((unique1)) from tenk1 where hundred > 1;
+
+reset enable_seqscan;
+reset enable_bitmapscan;
+
 set force_parallel_mode=1;
 
 explain (costs off)
