You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hawq.apache.org by ji...@apache.org on 2016/01/15 10:10:24 UTC

[1/3] incubator-hawq git commit: HAWQ-258. Remove fast sequence

Repository: incubator-hawq
Updated Branches:
  refs/heads/master 875067d63 -> ae38cfbd6


http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/optimizer/util/pathnode.c
----------------------------------------------------------------------
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index bbb766c..2eca7ff 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -254,9 +254,6 @@ pathnode_walk_kids(Path            *path,
             case T_BitmapHeapScan:
                     v = pathnode_walk_node(((BitmapHeapPath *)path)->bitmapqual, walker, context);
                     break;
-			case T_BitmapAppendOnlyScan:
-					v = pathnode_walk_node(((BitmapAppendOnlyPath *)path)->bitmapqual, walker, context);
-					break;
             case T_BitmapAnd:
                     v = pathnode_walk_list(((BitmapAndPath *)path)->bitmapquals, walker, context);
                     break;
@@ -1373,73 +1370,6 @@ create_bitmap_heap_path(PlannerInfo *root,
 }
 
 /*
- * create_bitmap_appendonly_path
- *	  Creates a path node for a bitmap Append-Only row table scan.
- *
- * 'bitmapqual' is a tree of IndexPath, BitmapAndPath, and BitmapOrPath nodes.
- *
- * If this is a join inner indexscan path, 'outer_rel' is the outer relation,
- * and all the component IndexPaths should have been costed accordingly.
- *
- * NOTE: This is mostly the same as the create_bitmap_heap_path routine.
- */
-BitmapAppendOnlyPath *
-create_bitmap_appendonly_path(PlannerInfo *root,
-							  RelOptInfo *rel,
-							  Path *bitmapqual,
-							  RelOptInfo *outer_rel,
-							  bool isAORow)
-{
-	BitmapAppendOnlyPath *pathnode = makeNode(BitmapAppendOnlyPath);
-
-	pathnode->path.pathtype = T_BitmapAppendOnlyScan;
-	pathnode->path.parent = rel;
-	pathnode->path.pathkeys = NIL;		/* always unordered */
-
-    /* Distribution is same as the base table. */
-    pathnode->path.locus = cdbpathlocus_from_baserel(root, rel);
-    pathnode->path.motionHazard = false;
-    pathnode->path.rescannable = true;
-
-	pathnode->bitmapqual = bitmapqual;
-	pathnode->isjoininner = (outer_rel != NULL);
-	pathnode->isAORow = isAORow;
-
-	if (pathnode->isjoininner)
-	{
-		/*
-		 * We must compute the estimated number of output rows for the
-		 * indexscan.  This is less than rel->rows because of the additional
-		 * selectivity of the join clauses.  We make use of the selectivity
-		 * estimated for the bitmap to do this; this isn't really quite right
-		 * since there may be restriction conditions not included in the
-		 * bitmap ...
-		 */
-		Cost		indexTotalCost;
-		Selectivity indexSelectivity;
-
-		cost_bitmap_tree_node(bitmapqual, &indexTotalCost, &indexSelectivity);
-		pathnode->rows = rel->tuples * indexSelectivity;
-		if (pathnode->rows > rel->rows)
-			pathnode->rows = rel->rows;
-		/* Like costsize.c, force estimate to be at least one row */
-		pathnode->rows = clamp_row_est(pathnode->rows);
-	}
-	else
-	{
-		/*
-		 * The number of rows is the same as the parent rel's estimate, since
-		 * this isn't a join inner indexscan.
-		 */
-		pathnode->rows = rel->rows;
-	}
-
-	cost_bitmap_appendonly_scan(&pathnode->path, root, rel, bitmapqual, outer_rel);
-
-	return pathnode;
-}
-
-/*
  * create_bitmap_and_path
  *	  Creates a path node representing a BitmapAnd.
  */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/optimizer/util/walkers.c
----------------------------------------------------------------------
diff --git a/src/backend/optimizer/util/walkers.c b/src/backend/optimizer/util/walkers.c
index 90d58bc..6f1ee51 100644
--- a/src/backend/optimizer/util/walkers.c
+++ b/src/backend/optimizer/util/walkers.c
@@ -928,7 +928,6 @@ plan_tree_walker(Node *node,
 		case T_DynamicTableScan:
 		case T_ParquetScan:
 		case T_BitmapHeapScan:
-		case T_BitmapAppendOnlyScan:
 		case T_BitmapTableScan:
 		case T_FunctionScan:
 		case T_TableFunctionScan:

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/tcop/utility.c
----------------------------------------------------------------------
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index ae42d6c..372d664 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -23,7 +23,6 @@
 #include "catalog/namespace.h"
 #include "catalog/toasting.h"
 #include "catalog/aoseg.h"
-#include "catalog/aoblkdir.h"
 #include "commands/alter.h"
 #include "commands/async.h"
 #include "commands/cluster.h"
@@ -944,13 +943,6 @@ ProcessUtility(Node *parsetree,
 							((CreateStmt *) parsetree)->oidInfo.aosegIndexOid,
 							&(((CreateStmt *) parsetree)->oidInfo.aosegComptypeOid),
 							((CreateStmt *) parsetree)->is_part_child);
-
-					if (((CreateStmt *)parsetree)->buildAoBlkdir)
-						AlterTableCreateAoBlkdirTableWithOid(relOid,
-							((CreateStmt *) parsetree)->oidInfo.aoblkdirOid,
-							((CreateStmt *) parsetree)->oidInfo.aoblkdirIndexOid,
-							&(((CreateStmt *) parsetree)->oidInfo.aoblkdirComptypeOid),
-							((CreateStmt *)parsetree)->is_part_child);
 				}
 				CommandCounterIncrement();
 				/*

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/utils/misc/guc.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index cc3ece5..b75dcb0 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -100,7 +100,6 @@
 #include "cdb/cdbvars.h"
 #include "cdb/cdbdisp.h"
 #include "cdb/dispatcher.h"
-#include "cdb/cdbappendonlyblockdirectory.h"
 #include "cdb/cdbquerycontextdispatching.h"
 #include "cdb/memquota.h"
 #include "utils/vmem_tracker.h"
@@ -5971,27 +5970,6 @@ static struct config_int ConfigureNamesInt[] =
 	},
 
 	{
-		{"gp_blockdirectory_entry_min_range", PGC_USERSET, GP_ARRAY_TUNING,
-		 gettext_noop("Minimal range in bytes one block directory entry covers."),
-		 gettext_noop("Used to reduce the size of a block directory."),
-		 GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE | GUC_GPDB_ADDOPT
-		},
-		&gp_blockdirectory_entry_min_range,
-		0, 0, INT_MAX, NULL, NULL
-	},
-
-	{
-		{"gp_blockdirectory_minipage_size", PGC_USERSET, GP_ARRAY_TUNING,
-		 gettext_noop("Number of entries one row in a block directory table contains."),
-		 gettext_noop("Use smaller value in non-bulk load cases."),
-		 GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE | GUC_GPDB_ADDOPT
-		},
-		&gp_blockdirectory_minipage_size,
-		NUM_MINIPAGE_ENTRIES, 1, NUM_MINIPAGE_ENTRIES, NULL, NULL
-	},
-
-
-	{
 		{"gp_segworker_relative_priority", PGC_POSTMASTER, RESOURCES_MGM,
 		gettext_noop("Priority for the segworkers relative to the postmaster's priority."),
 		NULL,

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/include/catalog/aoblkdir.h
----------------------------------------------------------------------
diff --git a/src/include/catalog/aoblkdir.h b/src/include/catalog/aoblkdir.h
deleted file mode 100644
index 84356e5..0000000
--- a/src/include/catalog/aoblkdir.h
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*-------------------------------------------------------------------------
- *
- * aoblkdir.h
- *
- *   This file contains some definitions to support creation of aoblkdir tables.
- *
- *-------------------------------------------------------------------------
- */
-#ifndef AOBLKDIR_H
-#define AOBLKDIR_H
-
-/*
- * Macros to the attribute number for each attribute
- * in the block directory relation.
- */
-#define Natts_pg_aoblkdir              4
-#define Anum_pg_aoblkdir_segno         1
-#define Anum_pg_aoblkdir_columngroupno 2
-#define Anum_pg_aoblkdir_firstrownum   3
-#define Anum_pg_aoblkdir_minipage      4
-
-extern void AlterTableCreateAoBlkdirTable(Oid relOid);
-extern void AlterTableCreateAoBlkdirTableWithOid(
-	Oid relOid, Oid newOid, Oid newIndexOid,
-	Oid * comptypeOid, bool is_part_child);
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/include/catalog/calico.pl
----------------------------------------------------------------------
diff --git a/src/include/catalog/calico.pl b/src/include/catalog/calico.pl
index 61bc137..28612b0 100755
--- a/src/include/catalog/calico.pl
+++ b/src/include/catalog/calico.pl
@@ -3519,8 +3519,6 @@ sub more_header
 #include "catalog/gp_segment_config.h"
 #include "catalog/gp_san_config.h"
 
-#include "catalog/gp_fastsequence.h"
-
 #include "catalog/gp_master_mirroring.h"
 #include "catalog/gp_persistent.h"
 #include "catalog/gp_global_sequence.h"
@@ -5075,7 +5073,6 @@ sub get_builtin_object_check_case_str()
 {
 	my $str = "";
 	my @clist = ("GpPolicyRelationId, FormData_gp_policy *, localoid",
-				 "FastSequenceRelationId, Form_gp_fastsequence, objid",
 				 "AggregateRelationId, Form_pg_aggregate, aggfnoid",
 				 "AccessMethodOperatorRelationId, Form_pg_amop, amopclaid",
 				 "AccessMethodProcedureRelationId, Form_pg_amproc, amopclaid",

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/include/catalog/gp_fastsequence.h
----------------------------------------------------------------------
diff --git a/src/include/catalog/gp_fastsequence.h b/src/include/catalog/gp_fastsequence.h
deleted file mode 100644
index a4733cd..0000000
--- a/src/include/catalog/gp_fastsequence.h
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/*-------------------------------------------------------------------------
- *
- * gp_fastsequence.h
- *    a table maintaining a light-weight fast sequence number for a unique
- *    object.
- *
- * $Id: $
- * $Change: $
- * $DateTime: $
- * $Author: $
- *-------------------------------------------------------------------------
- */
-#ifndef GP_FASTSEQUENCE_H
-#define GP_FASTSEQUENCE_H
-
-#include "catalog/genbki.h"
-#include "storage/itemptr.h"
-
-/*
- * gp_fastsequence definition
- */
-
-/* TIDYCAT_BEGINFAKEDEF
-
-   CREATE TABLE gp_fastsequence
-   with (camelcase=FastSequence, oid=false, relid=5043, reltype_oid=6453, content=SEGMENT_LOCAL)
-   (
-   objid   oid,
-   objmod  bigint,
-   last_sequence bigint
-   contentid	integer
-   );
-
-   create unique index on gp_fastsequence(objid, objmod, contentid) with (indexid=6067);
-
-   alter table gp_fastsequence add fk objid on pg_class(oid);
-
-   TIDYCAT_ENDFAKEDEF
-*/
-#define FastSequenceRelationId 5043
-
-CATALOG(gp_fastsequence,5043) BKI_WITHOUT_OIDS
-{
-	Oid				objid;				/* object oid */
-	int8			objmod;				/* object modifier */
-	int8			last_sequence;      /* the last sequence number used by the object */
-	int4			contentid;			/* content id */
-} FormData_gp_fastsequence;
-
-
-/* ----------------
-*		Form_gp_fastsequence corresponds to a pointer to a tuple with
-*		the format of gp_fastsequence relation.
-* ----------------
-*/
-typedef FormData_gp_fastsequence *Form_gp_fastsequence;
-
-#define Natts_gp_fastsequence				4
-#define Anum_gp_fastsequence_objid			1
-#define Anum_gp_fastsequence_objmod         2
-#define Anum_gp_fastsequence_last_sequence	3
-#define Anum_gp_fastsequence_contentid		4
-
-#define NUM_FAST_SEQUENCES					 100
-
-/* No initial content */
-
-/*
- * Insert a new light-weight fast sequence entry for a given object.
- *
- * The tid for the new entry is returned.
- */
-extern void InsertFastSequenceEntry(Oid objid, int64 objmod, int64 lastSequence,
-		ItemPointer tid);
-
-/*
- * GetFastSequences
- *
- * Get a list of consecutive sequence numbers. The starting sequence
- * number is the maximal value between 'lastsequence' + 1 and minSequence.
- * The length of the list is given.
- *
- * If there is not such an entry for objid in the table, create
- * one here.
- *
- * The existing entry for objid in the table is updated with a new
- * lastsequence value.
- *
- * The tuple id value for this entry is copied out to 'tid'.
- */
-extern int64 GetFastSequences(Oid objid, int64 objmod,
-							  int64 minSequence, int64 numSequences,
-							  ItemPointer tid);
-
-/*
- * GetFastSequencesByTid
- *
- * Same as GetFastSequences, except that the tuple tid is given, and the tuple id
- * is not valid.
- */
-extern int64 GetFastSequencesByTid(ItemPointer tid,
-								   int64 minSequence,
-								   int64 numSequences);
-
-/*
- * RemoveFastSequenceEntry
- *
- * Remove all entries associated with the given object id.
- *
- * If the given objid is an invalid OID, this function simply
- * returns.
- *
- * If the given valid objid does not have an entry in
- * gp_fastsequence, this function errors out.
- */
-extern void RemoveFastSequenceEntry(Oid objid);
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/include/catalog/indexing.h
----------------------------------------------------------------------
diff --git a/src/include/catalog/indexing.h b/src/include/catalog/indexing.h
index 73a69d1..f0d33e7 100644
--- a/src/include/catalog/indexing.h
+++ b/src/include/catalog/indexing.h
@@ -209,9 +209,6 @@ DECLARE_UNIQUE_INDEX(gp_policy_localoid_index, 6103, on gp_distribution_policy u
 DECLARE_UNIQUE_INDEX(pg_appendonly_relid_index, 5007, on pg_appendonly using btree(relid oid_ops));
 #define AppendOnlyRelidIndexId  5007
 
-DECLARE_UNIQUE_INDEX(gp_fastsequence_objid_objmod_contentid_index, 6067, on gp_fastsequence using btree(objid oid_ops, objmod  int8_ops, contentid int4_ops));
-#define FastSequenceObjidObjmodContentidIndexId 6067
-
 DECLARE_UNIQUE_INDEX(pg_appendonly_alter_column_relid_index, 5031, on pg_appendonly_alter_column using btree(relid oid_ops, changenum int4_ops));
 #define AppendOnlyAlterColumnRelidIndexId  5031
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/include/catalog/pg_tidycat.h
----------------------------------------------------------------------
diff --git a/src/include/catalog/pg_tidycat.h b/src/include/catalog/pg_tidycat.h
index 9f969c8..7d96d0c 100644
--- a/src/include/catalog/pg_tidycat.h
+++ b/src/include/catalog/pg_tidycat.h
@@ -27,7 +27,6 @@
 
 
 #include "catalog/gp_configuration.h"
-#include "catalog/gp_fastsequence.h"
 #include "catalog/gp_global_sequence.h"
 #include "catalog/gp_id.h"
 #include "catalog/gp_master_mirroring.h"

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/include/catalog/pg_type.h
----------------------------------------------------------------------
diff --git a/src/include/catalog/pg_type.h b/src/include/catalog/pg_type.h
index c356f14..4960bdb 100644
--- a/src/include/catalog/pg_type.h
+++ b/src/include/catalog/pg_type.h
@@ -721,9 +721,6 @@ DATA(insert OID = 6444 ( gp_san_configuration	    PGNSP PGUID -1 f c t \054 5035
 /* relation id: 5039 - gp_fault_strategy 20101104 */
 DATA(insert OID = 6443 ( gp_fault_strategy	    PGNSP PGUID -1 f c t \054 5039 0 record_in record_out record_recv record_send - d x f 0 -1 0 _null_ _null_ ));	
 #define GP_FAULT_STRATEGY_RELTYPE_OID 6443
-/* relation id: 5043 - gp_fastsequence 20101104 */
-DATA(insert OID = 6453 ( gp_fastsequence	    PGNSP PGUID -1 f c t \054 5043 0 record_in record_out record_recv record_send - d x f 0 -1 0 _null_ _null_ ));	
-#define GP_FASTSEQUENCE_RELTYPE_OID 6453
 /* relation id: 5096 - gp_global_sequence 20101104 */
 DATA(insert OID = 6995 ( gp_global_sequence	    PGNSP PGUID -1 f c t \054 5096 0 record_in record_out record_recv record_send - d x f 0 -1 0 _null_ _null_ ));	
 #define GP_GLOBAL_SEQUENCE_RELTYPE_OID 6995

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/include/cdb/cdbappendonlyam.h
----------------------------------------------------------------------
diff --git a/src/include/cdb/cdbappendonlyam.h b/src/include/cdb/cdbappendonlyam.h
index 8b984f3..5a6129a 100644
--- a/src/include/cdb/cdbappendonlyam.h
+++ b/src/include/cdb/cdbappendonlyam.h
@@ -56,7 +56,6 @@
 #include "cdb/cdbappendonlystoragelayer.h"
 #include "cdb/cdbappendonlystorageread.h"
 #include "cdb/cdbappendonlystoragewrite.h"
-#include "cdb/cdbappendonlyblockdirectory.h"
 
 #include "cdb/cdbquerycontextdispatching.h"
 
@@ -85,14 +84,11 @@ typedef struct AppendOnlyInsertDescData
 	float8			insertCount;
 	float8			varblockCount;
 	int64           rowCount; /* total row count before insert */
-	int64           numSequences; /* total number of available sequences */
-	int64           lastSequence; /* last used sequence */
 	BlockNumber		cur_segno;
 	AppendOnlyEntry *aoEntry;
 	FileSegInfo     *fsInfo;
 	VarBlockMaker	varBlockMaker;
 	int64			bufferCount;
-	int64			blockFirstRowNum;
 	bool			shouldCompress;
 	bool			usingChecksum;
 	bool			useNoToast;
@@ -122,10 +118,6 @@ typedef struct AppendOnlyInsertDescData
 
 	uint8			*nonCompressedData;
 
-	/* The block directory for the appendonly relation. */
-	AppendOnlyBlockDirectory blockDirectory;
-
-
 	QueryContextDispatchingSendBack sendback;
 
 } AppendOnlyInsertDescData;
@@ -214,15 +206,6 @@ typedef struct AppendOnlyScanDescData
 
 	AppendOnlyEntry		*aoEntry;
 	
-	/*
-	 * The block directory info.
-	 *
-	 * For AO tables that are upgraded from pre-3.4 release, the block directory 
-	 * built during the first index creation.
-	 */
-	bool buildBlockDirectory; /* Indicate whether to build block directory while scanning */
-	AppendOnlyBlockDirectory *blockDirectory;
-
 	List *splits;
 
 	bool toCloseFile;
@@ -230,90 +213,6 @@ typedef struct AppendOnlyScanDescData
 
 typedef AppendOnlyScanDescData *AppendOnlyScanDesc;
 
-/*
- * Statistics on the latest fetch.
- */
-typedef struct AppendOnlyFetchDetail
-{
-	int64		rangeFileOffset;
-	int64		rangeFirstRowNum;
-	int64		rangeAfterFileOffset;
-	int64		rangeLastRowNum;
-					/*
-					 * The range covered by the Block Directory.
-					 */
-	
-	int64		skipBlockCount;
-					/*
-					 * Number of blocks skipped since the previous block processed in
-					 * the range.
-					 */
-	
-	int64		blockFileOffset;
-	int32		blockOverallLen;
-	int64		blockFirstRowNum;
-	int64		blockLastRowNum;
-	bool		isCompressed;
-	bool		isLargeContent;
-					/*
-					 * The last block processed.
-					 */
-
-} AppendOnlyFetchDetail;
-
-
-/*
- * Used for fetch individual tuples from specified by TID of append only relations 
- * using the AO Block Directory, BufferedRead and VarBlocks
- */
-typedef struct AppendOnlyFetchDescData
-{
-	Relation		relation;
-	Snapshot		appendOnlyMetaDataSnapshot;
-
-	MemoryContext	initContext;
-
-	AppendOnlyStorageAttributes	storageAttributes;
-	AppendOnlyStorageRead		storageRead;
-
-	char						*title;
-				/*
-				 * A phrase that better describes the purpose of the this open.
-				 *
-				 * We manage the storage for this.
-				 */
-
-
-	int				totalSegfiles;
-	FileSegInfo 	**segmentFileInfo;
-
-	AppendOnlyEntry *aoEntry;
-
-	char			*segmentFileName;
-	int				segmentFileNameMaxLen;
-
-	int32			usableBlockSize;
-
-	AppendOnlyBlockDirectory	blockDirectory;
-
-	AppendOnlyExecutorReadBlock executorReadBlock;
-
-	CurrentSegmentFile currentSegmentFile;
-	
-	int64		scanNextFileOffset;
-	int64		scanNextRowNum;
-
-	int64		scanAfterFileOffset;
-	int64		scanLastRowNum;
-
-	CurrentBlock currentBlock;
-
-	int64	skipBlockCount;
-
-}	AppendOnlyFetchDescData;
-
-typedef AppendOnlyFetchDescData *AppendOnlyFetchDesc;
-
 /* ----------------
  *		function prototypes for appendonly access method
  * ----------------
@@ -328,17 +227,6 @@ extern void appendonly_endscan(AppendOnlyScanDesc scan);
 extern MemTuple appendonly_getnext(AppendOnlyScanDesc scan, 
 									ScanDirection direction,
 									TupleTableSlot *slot);
-extern AppendOnlyFetchDesc appendonly_fetch_init(
-	Relation 	relation,
-	Snapshot 	appendOnlyMetaDataSnapshot);
-extern bool appendonly_fetch(
-	AppendOnlyFetchDesc aoFetchDesc,
-	AOTupleId *aoTid,
-	TupleTableSlot *slot);
-extern void appendonly_fetch_detail(
-	AppendOnlyFetchDesc aoFetchDesc, 
-	AppendOnlyFetchDetail *aoFetchDetail);
-extern void appendonly_fetch_finish(AppendOnlyFetchDesc aoFetchDesc);
 extern AppendOnlyInsertDesc appendonly_insert_init(Relation rel, ResultRelSegFileInfo *segfileinfo);
 extern void appendonly_insert(
 		AppendOnlyInsertDesc aoInsertDesc, 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/include/cdb/cdbappendonlyblockdirectory.h
----------------------------------------------------------------------
diff --git a/src/include/cdb/cdbappendonlyblockdirectory.h b/src/include/cdb/cdbappendonlyblockdirectory.h
deleted file mode 100644
index c85488c..0000000
--- a/src/include/cdb/cdbappendonlyblockdirectory.h
+++ /dev/null
@@ -1,210 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/*------------------------------------------------------------------------------
- *
- * cdbappendonlyblockdirectory.h
- *
- * $Id: $
- * $Change: $
- * $DateTime: $
- * $Author: $
- *------------------------------------------------------------------------------
- */
-#ifndef APPENDONLYBLOCKDIRECTORY_H
-#define APPENDONLYBLOCKDIRECTORY_H
-
-#include "access/aosegfiles.h"
-#include "access/appendonlytid.h"
-#include "access/skey.h"
-
-extern int gp_blockdirectory_entry_min_range;
-extern int gp_blockdirectory_minipage_size;
-
-typedef struct AppendOnlyBlockDirectoryEntry
-{
-	/*
-	 * The range of blocks covered by the Block Directory entry.
-	 */
-	struct range
-	{
-		int64		fileOffset;
-		int64		firstRowNum;
-
-		int64		afterFileOffset;
-		int64		lastRowNum;
-	} range;
-
-} AppendOnlyBlockDirectoryEntry;
-
-/*
- * The entry in the minipage.
- */
-typedef struct MinipageEntry
-{
-	int64 firstRowNum;
-	int64 fileOffset;
-	int64 rowCount;
-} MinipageEntry;
-
-/*
- * Define a varlena type for a minipage.
- */
-typedef struct Minipage
-{
-	/* Total length. Must be the first. */
-	int32 _len;
-	int32 version;
-	uint32 nEntry;
-	
-	/* Varlena array */
-	MinipageEntry entry[1];
-} Minipage;
-
-/*
- * Define the relevant info for a minipage for each
- * column group.
- */
-typedef struct MinipagePerColumnGroup
-{
-	Minipage *minipage;
-	uint32 numMinipageEntries;
-	ItemPointerData tupleTid;
-} MinipagePerColumnGroup;
-
-/*
- * I don't know the ideal value here. But let us put approximate
- * 8 minipages per heap page.
- */
-#define NUM_MINIPAGE_ENTRIES (((MaxTupleSize)/8 - sizeof(HeapTupleHeaderData) - 64 * 3)\
-							  / sizeof(MinipageEntry))
-
-/*
- * Define a structure for the append-only relation block directory.
- */
-typedef struct AppendOnlyBlockDirectory
-{
-	Relation aoRel;
-	Snapshot appendOnlyMetaDataSnapshot;
-	Relation blkdirRel;
-	Relation blkdirIdx;
-	int numColumnGroups;
-	
-	MemoryContext memoryContext;
-	
-	int				totalSegfiles;
-	FileSegInfo 	**segmentFileInfo;
-
-	/*
-	 * Current segment file number.
-	 */
-	int currentSegmentFileNum;
-	FileSegInfo *currentSegmentFileInfo;
-
-	/*
-	 * Last minipage that contains an array of MinipageEntries.
-	 */
-	MinipagePerColumnGroup *minipages;
-
-	/*
-	 * Some temporary space to help form tuples to be inserted into
-	 * the block directory, and to help the index scan.
-	 */
-	Datum *values;
-	bool *nulls;
-	int numScanKeys;
-	ScanKey scanKeys;
-	StrategyNumber *strategyNumbers;
-
-}	AppendOnlyBlockDirectory;
-
-
-typedef struct CurrentBlock
-{
-	AppendOnlyBlockDirectoryEntry blockDirectoryEntry;
-
-	bool have;
-
-	int64 fileOffset;
-	
-	int32 overallBlockLen;
-	
-	int64 firstRowNum;
-	int64 lastRowNum;
-	
-	bool isCompressed;
-	bool isLargeContent;
-	
-	bool		gotContents;
-} CurrentBlock;
-
-typedef struct CurrentSegmentFile
-{
-	bool isOpen;
-	
-	int num;
-	
-	int64 logicalEof;
-} CurrentSegmentFile;
-
-extern void AppendOnlyBlockDirectoryEntry_GetBeginRange(
-	AppendOnlyBlockDirectoryEntry	*directoryEntry,
-	int64							*fileOffset,
-	int64							*firstRowNum);
-extern void AppendOnlyBlockDirectoryEntry_GetEndRange(
-	AppendOnlyBlockDirectoryEntry	*directoryEntry,
-	int64							*afterFileOffset,
-	int64							*lastRowNum);
-extern bool AppendOnlyBlockDirectoryEntry_RangeHasRow(
-	AppendOnlyBlockDirectoryEntry	*directoryEntry,
-	int64							checkRowNum);
-extern bool AppendOnlyBlockDirectory_GetEntry(
-	AppendOnlyBlockDirectory		*blockDirectory,
-	AOTupleId 						*aoTupleId,
-	int                             columnGroupNo,
-	AppendOnlyBlockDirectoryEntry	*directoryEntry);
-extern void AppendOnlyBlockDirectory_Init_forInsert(
-	AppendOnlyBlockDirectory *blockDirectory,
-	AppendOnlyEntry *aoEntry,
-	Snapshot appendOnlyMetaDataSnapshot,
-	FileSegInfo *segmentFileInfo,
-	int64 lastSequence,
-	Relation aoRel,
-	int segno,
-	int numColumnGroups);
-extern void AppendOnlyBlockDirectory_Init_forSearch(
-	AppendOnlyBlockDirectory *blockDirectory,
-	AppendOnlyEntry *aoEntry,
-	Snapshot appendOnlyMetaDataSnapshot,
-	FileSegInfo **segmentFileInfo,
-	int totalSegfiles,
-	Relation aoRel,
-	int numColumnGroups);
-extern bool AppendOnlyBlockDirectory_InsertEntry(
-	AppendOnlyBlockDirectory *blockDirectory,
-	int columnGroupNo,
-	int64 firstRowNum,
-	int64 fileOffset,
-	int64 rowCount);
-extern void AppendOnlyBlockDirectory_End_forInsert(
-	AppendOnlyBlockDirectory *blockDirectory);
-extern void AppendOnlyBlockDirectory_End_forSearch(
-	AppendOnlyBlockDirectory *blockDirectory);
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/include/cdb/cdbappendonlystoragewrite.h
----------------------------------------------------------------------
diff --git a/src/include/cdb/cdbappendonlystoragewrite.h b/src/include/cdb/cdbappendonlystoragewrite.h
index 7c2edd5..cc8e111 100644
--- a/src/include/cdb/cdbappendonlystoragewrite.h
+++ b/src/include/cdb/cdbappendonlystoragewrite.h
@@ -120,16 +120,6 @@ typedef struct AppendOnlyStorageWrite
 			 * The number of blocks written since the beginning of the segment file.
 			 */
 
-	bool	isFirstRowNumSet;
-			/*
-			 * When true, the first row number for the next block has
-			 * been explicitly set and will be stored in the Append-Only
-			 * Storage Header.
-			 */
-
-	int64	firstRowNum;
-			/* Explicitly set first row number for the next block. */
-
 	int64   lastWriteBeginPosition;
 			/* The beginning of the write buffer for the last write. */
 
@@ -510,16 +500,6 @@ extern void AppendOnlyStorageWrite_Content(
  * new overhead length.
  */
 
-
-/*
- * Set the first row value for the next Append-Only Storage
- * Block to be written.  Only applies to the next block.
- */
-extern void AppendOnlyStorageWrite_SetFirstRowNum(
-	AppendOnlyStorageWrite		*storageWrite,
-	int64						firstRowNum);
-
-
 extern char *AppendOnlyStorageWrite_ContextStr(
 	AppendOnlyStorageWrite		*storageWrite);
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/include/cdb/cdbquerycontextdispatching.h
----------------------------------------------------------------------
diff --git a/src/include/cdb/cdbquerycontextdispatching.h b/src/include/cdb/cdbquerycontextdispatching.h
index 0760925..815f21f 100644
--- a/src/include/cdb/cdbquerycontextdispatching.h
+++ b/src/include/cdb/cdbquerycontextdispatching.h
@@ -90,8 +90,6 @@ struct QueryContextDispatchingSendBackData
 	int32 numfiles;
 	int64 *eof;
 	int64 *uncompressed_eof;
-
-	int64 nextFastSequence;
 };
 typedef struct QueryContextDispatchingSendBackData * QueryContextDispatchingSendBack;
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/include/executor/executor.h
----------------------------------------------------------------------
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index f1ce5aa..a6ff148 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -366,22 +366,6 @@ extern void BitmapHeapScanEnd(ScanState *scanState);
 extern void BitmapHeapScanReScan(ScanState *scanState);
 
 /*
- * prototypes from functions in execBitmapAOScan.c
- */
-extern TupleTableSlot *BitmapAOScanNext(ScanState *scanState);
-extern void BitmapAOScanBegin(ScanState *scanState);
-extern void BitmapAOScanEnd(ScanState *scanState);
-extern void BitmapAOScanReScan(ScanState *scanState);
-
-/*
- * prototypes from functions in execBitmapParquetScan.c
- */
-extern TupleTableSlot *BitmapParquetScanNext(ScanState *scanState);
-extern void BitmapParquetScanBegin(ScanState *scanState);
-extern void BitmapParquetScanEnd(ScanState *scanState);
-extern void BitmapParquetScanReScan(ScanState *scanState);
-
-/*
  * prototypes from functions in execBitmapTableScan.c
  */
 extern void initGpmonPktForBitmapTableScan(Plan *planNode, gpmon_packet_t *gpmon_pkt, EState *estate);

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/include/executor/nodeBitmapAppendOnlyscan.h
----------------------------------------------------------------------
diff --git a/src/include/executor/nodeBitmapAppendOnlyscan.h b/src/include/executor/nodeBitmapAppendOnlyscan.h
deleted file mode 100755
index 76a10ff..0000000
--- a/src/include/executor/nodeBitmapAppendOnlyscan.h
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/*-------------------------------------------------------------------------
- *
- * nodeBitmapAppendOnlyscan.h
- *
- *
- * Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group
- * Portions Copyright (c) 1994, Regents of the University of California
- * Portions Copyright (c) 2008-2009, Greenplum Inc.
- *
- *
- *-------------------------------------------------------------------------
- */
-#ifndef NODEBITMAPAPPENDONLYSCAN_H
-#define NODEBITMAPAPPENDONLYSCAN_H
-
-#include "nodes/execnodes.h"
-
-extern int	ExecCountSlotsBitmapAppendOnlyScan(BitmapAppendOnlyScan *node);
-extern BitmapAppendOnlyScanState *ExecInitBitmapAppendOnlyScan(BitmapAppendOnlyScan *node, EState *estate, int eflags);
-extern TupleTableSlot *ExecBitmapAppendOnlyScan(BitmapAppendOnlyScanState *node);
-extern void ExecEndBitmapAppendOnlyScan(BitmapAppendOnlyScanState *node);
-extern void ExecBitmapAppendOnlyReScan(BitmapAppendOnlyScanState *node, ExprContext *exprCtxt);
-extern void ExecEagerFreeBitmapAppendOnlyScan(BitmapAppendOnlyScanState *node);
-
-enum
-{
-	GPMON_BITMAPAPPENDONLYSCAN_PAGE = GPMON_QEXEC_M_NODE_START, 
-    	GPMON_BITMAPAPPENDONLYSCAN_RESCAN,
-	GPMON_BITMAPAPPENDONLYSCAN_TOTAL
-};
-
-static inline gpmon_packet_t * GpmonPktFromBitmapAppendOnlyScanState(BitmapAppendOnlyScanState *node)
-{
-	return &node->ss.ps.gpmon_pkt;
-}
-
-#endif   /* NODEBITMAPAPPENDONLYSCAN_H */
-

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/include/nodes/nodes.h
----------------------------------------------------------------------
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index a31be24..a01aaa6 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -90,7 +90,6 @@ typedef enum NodeTag
 	T_DynamicIndexScan,
 	T_BitmapIndexScan,
 	T_BitmapHeapScan,
-	T_BitmapAppendOnlyScan,
 	T_BitmapTableScan,
 	T_TidScan,
 	T_SubqueryScan,
@@ -148,7 +147,6 @@ typedef enum NodeTag
 	T_DynamicIndexScanState,
 	T_BitmapIndexScanState,
 	T_BitmapHeapScanState,
-	T_BitmapAppendOnlyScanState,
 	T_BitmapTableScanState,
 	T_TidScanState,
 	T_SubqueryScanState,

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/include/optimizer/pathnode.h
----------------------------------------------------------------------
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index fce497e..2684291 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -77,11 +77,6 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,
 						RelOptInfo *outer_rel);
-extern BitmapAppendOnlyPath *create_bitmap_appendonly_path(PlannerInfo *root,
-														   RelOptInfo *rel,
-														   Path *bitmapqual,
-														   RelOptInfo *outer_rel,
-														   bool isAORow);
 extern BitmapTableScanPath *create_bitmap_table_scan_path(PlannerInfo *root,
 						RelOptInfo *rel,
 						Path *bitmapqual,

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/test/regress/expected/insert.out
----------------------------------------------------------------------
diff --git a/src/test/regress/expected/insert.out b/src/test/regress/expected/insert.out
index b29534a..d254596 100644
--- a/src/test/regress/expected/insert.out
+++ b/src/test/regress/expected/insert.out
@@ -88,8 +88,3 @@ NOTICE:  Table doesn't have 'DISTRIBUTED BY' clause -- Using column named 'a' as
 HINT:  The 'DISTRIBUTED BY' clause determines the distribution of data. Make sure column(s) chosen are the optimal data distribution key to minimize skew.
 INSERT INTO mpp14758 select * from generate_series(1,10);
 ABORT;
-SELECT * from gp_fastsequence where objid not in (select oid from pg_class);
- objid | objmod | last_sequence | contentid 
--------+--------+---------------+-----------
-(0 rows)
-

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/test/regress/expected/insert_optimizer.out
----------------------------------------------------------------------
diff --git a/src/test/regress/expected/insert_optimizer.out b/src/test/regress/expected/insert_optimizer.out
index 48bd94a..58121e5 100755
--- a/src/test/regress/expected/insert_optimizer.out
+++ b/src/test/regress/expected/insert_optimizer.out
@@ -90,8 +90,3 @@ NOTICE:  Table doesn't have 'DISTRIBUTED BY' clause -- Using column named 'a' as
 HINT:  The 'DISTRIBUTED BY' clause determines the distribution of data. Make sure column(s) chosen are the optimal data distribution key to minimize skew.
 INSERT INTO mpp14758 select * from generate_series(1,10);
 ABORT;
-SELECT * from gp_fastsequence where objid not in (select oid from pg_class);
- objid | objmod | last_sequence | contentid 
--------+--------+---------------+-----------
-(0 rows)
-

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/test/regress/sql/insert.sql
----------------------------------------------------------------------
diff --git a/src/test/regress/sql/insert.sql b/src/test/regress/sql/insert.sql
index 97d627e..7e40a20 100644
--- a/src/test/regress/sql/insert.sql
+++ b/src/test/regress/sql/insert.sql
@@ -71,4 +71,3 @@ BEGIN;
 CREATE TABLE mpp14758(a int) with (appendonly=true);
 INSERT INTO mpp14758 select * from generate_series(1,10);
 ABORT;
-SELECT * from gp_fastsequence where objid not in (select oid from pg_class);

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/test/unit/mock/mock_info.json
----------------------------------------------------------------------
diff --git a/src/test/unit/mock/mock_info.json b/src/test/unit/mock/mock_info.json
index 92669e6..90275e0 100644
--- a/src/test/unit/mock/mock_info.json
+++ b/src/test/unit/mock/mock_info.json
@@ -2833,28 +2833,6 @@
             "parameter": [], 
             "return": "Oid"
         }, 
-        "GetFastSequences": {
-            "filename": "src/backend/catalog/gp_fastsequence.c", 
-            "header filename": "src/include/catalog/gp_fastsequence.h", 
-            "parameter": [
-                "objid", 
-                "objmod", 
-                "minSequence", 
-                "numSequences", 
-                "tid"
-            ], 
-            "return": "int64"
-        }, 
-        "GetFastSequencesByTid": {
-            "filename": "src/backend/catalog/gp_fastsequence.c", 
-            "header filename": "src/include/catalog/gp_fastsequence.h", 
-            "parameter": [
-                "tid", 
-                "minSequence", 
-                "numSequences"
-            ], 
-            "return": "int64"
-        }, 
         "GetFreeIndexPage": {
             "filename": "src/backend/storage/freespace/freespace.c", 
             "header filename": "src/include/storage/freespace.h", 
@@ -3504,17 +3482,6 @@
             ], 
             "return": "void"
         }, 
-        "InsertFastSequenceEntry": {
-            "filename": "src/backend/catalog/gp_fastsequence.c", 
-            "header filename": "src/include/catalog/gp_fastsequence.h", 
-            "parameter": [
-                "objid", 
-                "objmod", 
-                "lastSequence", 
-                "tid"
-            ], 
-            "return": "void"
-        }, 
         "IsAbortInProgress": {
             "filename": "src/backend/access/transam/xact.c", 
             "header filename": "src/include/access/xact.h", 
@@ -7053,14 +7020,6 @@
             ], 
             "return": "void"
         }, 
-        "RemoveFastSequenceEntry": {
-            "filename": "src/backend/catalog/gp_fastsequence.c", 
-            "header filename": "src/include/catalog/gp_fastsequence.h", 
-            "parameter": [
-                "objid"
-            ], 
-            "return": "void"
-        }, 
         "RemoveRelationAttributeEncoding": {
             "filename": "src/backend/catalog/pg_attribute_encoding.c", 
             "header filename": "src/include/catalog/pg_attribute_encoding.h", 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/tools/bin/gppylib/data/2.0.json
----------------------------------------------------------------------
diff --git a/tools/bin/gppylib/data/2.0.json b/tools/bin/gppylib/data/2.0.json
index bd69df2..af8bede 100644
--- a/tools/bin/gppylib/data/2.0.json
+++ b/tools/bin/gppylib/data/2.0.json
@@ -280,99 +280,6 @@
          "text" : "with (camelcase=GpPolicy, oid=false, relid=5002, content=MASTER_ONLY)"
       }
    },
-   "gp_fastsequence" : {
-      "CamelCaseRelationId" : "FastSequenceRelationId",
-      "UppercaseReltypeOid" : "GP_FASTSEQUENCE_RELTYPE_OID",
-      "colh" : {
-         "contentid" : "int4",
-         "last_sequence" : "bigint",
-         "objid" : "Oid",
-         "objmod" : "bigint"
-      },
-      "cols" : [
-         {
-            "colname" : "objid",
-            "ctype" : "Oid",
-            "precomment" : "\n**TK_BLANK_LINE**",
-            "sqltype" : "oid"
-         },
-         {
-            "colname" : "objmod",
-            "ctype" : "bigint",
-            "sqltype" : "bigint"
-         },
-         {
-            "colname" : "last_sequence",
-            "ctype" : "bigint",
-            "sqltype" : "bigint"
-         },
-         {
-            "colname" : "contentid",
-            "ctype" : "int4",
-            "sqltype" : "integer"
-         }
-      ],
-      "filename" : "gp_fastsequence.h",
-      "fk_list" : [
-         {
-            "fkcols" : [
-               "objid"
-            ],
-            "pkcols" : [
-               "oid"
-            ],
-            "pktable" : "pg_class",
-            "type" : "scalar"
-         }
-      ],
-      "foreign_keys" : [
-         [
-            [
-               "objid"
-            ],
-            "pg_class",
-            [
-               "oid"
-            ]
-         ]
-      ],
-      "indexes" : [
-         {
-            "CamelCaseIndexId" : "FastSequenceObjidObjmodContentidIndexId",
-            "cols" : [
-               [
-                  "objid",
-                  "oid_ops"
-               ],
-               [
-                  "objmod",
-                  "bigint_ops"
-               ],
-               [
-                  "contentid",
-                  "int4_ops"
-               ]
-            ],
-            "indexid" : "6067",
-            "unique" : "1",
-            "with" : {
-               "indexid" : "6067"
-            }
-         }
-      ],
-      "relid_comment_tag" : "/* relation id: 5043 - gp_fastsequence */\n",
-      "tabdef_text" : "\n   CREATE TABLE gp_fastsequence\n   with (camelcase=FastSequence, oid=false, relid=5043, reltype_oid=6453, content=SEGMENT_LOCAL)\n   (\n   objid   oid,\n   objmod  bigint,\n   last_sequence bigint\n   contentid\tinteger\n   )",
-      "with" : {
-         "bootstrap" : 0,
-         "camelcase" : "FastSequence",
-         "content" : "SEGMENT_LOCAL",
-         "oid" : "",
-         "relid" : "5043",
-         "reltype_oid" : "6453",
-         "shared" : 0,
-         "text" : "with (camelcase=FastSequence, oid=false, relid=5043, reltype_oid=6453, content=SEGMENT_LOCAL)"
-      }
-   },
    "gp_fault_strategy" : {
       "CamelCaseRelationId" : "GpFaultStrategyRelationId",
       "UppercaseReltypeOid" : "GP_FAULT_STRATEGY_RELTYPE_OID",



[3/3] incubator-hawq git commit: HAWQ-258. Remove fast sequence

Posted by ji...@apache.org.
HAWQ-258. Remove fast sequence


Project: http://git-wip-us.apache.org/repos/asf/incubator-hawq/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hawq/commit/ae38cfbd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hawq/tree/ae38cfbd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hawq/diff/ae38cfbd

Branch: refs/heads/master
Commit: ae38cfbd675ef7c60149d595657d896add5ce864
Parents: 875067d
Author: Lirong Jian <ji...@gmail.com>
Authored: Tue Jan 12 16:26:41 2016 +0800
Committer: Lirong Jian <ji...@gmail.com>
Committed: Fri Jan 15 16:49:07 2016 +0800

----------------------------------------------------------------------
 src/backend/access/appendonly/Makefile          |    2 +-
 src/backend/access/appendonly/aosegfiles.c      |    7 -
 src/backend/access/appendonly/appendonlyam.c    |  790 +------------
 .../appendonly/appendonlyblockdirectory.c       | 1113 ------------------
 src/backend/access/external/hd_work_mgr.c       |    2 -
 src/backend/access/heap/heapam.c                |   37 -
 src/backend/access/index/caql.files             |    1 -
 src/backend/access/index/caqlfilemap.json       |    3 -
 src/backend/access/nbtree/nbtinsert.c           |   55 -
 src/backend/access/parquet/parquetsegfiles.c    |    5 -
 src/backend/bootstrap/bootparse.y               |    5 -
 src/backend/catalog/Makefile                    |    4 +-
 src/backend/catalog/aoblkdir.c                  |  264 -----
 src/backend/catalog/caql/Makefile               |    1 -
 src/backend/catalog/core/catcoregen.py          |    1 -
 src/backend/catalog/gp_fastsequence.c           |  416 -------
 src/backend/catalog/heap.c                      |    1 -
 src/backend/catalog/index.c                     |   40 +-
 src/backend/catalog/pg_appendonly.c             |    4 -
 src/backend/cdb/cdbappendonlystoragewrite.c     |   68 +-
 src/backend/cdb/cdbllize.c                      |    1 -
 src/backend/cdb/cdbpath.c                       |    1 -
 src/backend/cdb/cdbpersistentrelfile.c          |   47 -
 src/backend/cdb/cdbplan.c                       |   15 -
 src/backend/cdb/cdbquerycontextdispatching.c    |   53 -
 src/backend/cdb/cdbtargeteddispatch.c           |    2 -
 src/backend/commands/analyze.c                  |    1 -
 src/backend/commands/cluster.c                  |    7 -
 src/backend/commands/explain.c                  |   17 -
 src/backend/executor/Makefile                   |    4 +-
 src/backend/executor/execAmi.c                  |   12 +-
 src/backend/executor/execBitmapAOScan.c         |  280 -----
 src/backend/executor/execBitmapParquetScan.c    |   69 --
 src/backend/executor/execBitmapTableScan.c      |   14 -
 src/backend/executor/execMain.c                 |    1 -
 src/backend/executor/execProcnode.c             |   27 +-
 src/backend/executor/execUtils.c                |    2 -
 src/backend/executor/nodeBitmapAppendOnlyscan.c |  612 ----------
 src/backend/gp_libpq_fe/fe-protocol3.c          |    6 -
 src/backend/nodes/copyfuncs.c                   |   25 -
 src/backend/nodes/outfast.c                     |   14 -
 src/backend/nodes/outfuncs.c                    |    3 -
 src/backend/nodes/print.c                       |    5 +-
 src/backend/nodes/readfast.c                    |   16 -
 src/backend/optimizer/path/indxpath.c           |    3 -
 src/backend/optimizer/plan/createplan.c         |  154 ---
 src/backend/optimizer/plan/planpartition.c      |    1 -
 src/backend/optimizer/plan/setrefs.c            |   35 +-
 src/backend/optimizer/plan/subselect.c          |    5 -
 src/backend/optimizer/util/pathnode.c           |   70 --
 src/backend/optimizer/util/walkers.c            |    1 -
 src/backend/tcop/utility.c                      |    8 -
 src/backend/utils/misc/guc.c                    |   22 -
 src/include/catalog/aoblkdir.h                  |   45 -
 src/include/catalog/calico.pl                   |    3 -
 src/include/catalog/gp_fastsequence.h           |  137 ---
 src/include/catalog/indexing.h                  |    3 -
 src/include/catalog/pg_tidycat.h                |    1 -
 src/include/catalog/pg_type.h                   |    3 -
 src/include/cdb/cdbappendonlyam.h               |  112 --
 src/include/cdb/cdbappendonlyblockdirectory.h   |  210 ----
 src/include/cdb/cdbappendonlystoragewrite.h     |   20 -
 src/include/cdb/cdbquerycontextdispatching.h    |    2 -
 src/include/executor/executor.h                 |   16 -
 src/include/executor/nodeBitmapAppendOnlyscan.h |   56 -
 src/include/nodes/nodes.h                       |    2 -
 src/include/optimizer/pathnode.h                |    5 -
 src/test/regress/expected/insert.out            |    5 -
 src/test/regress/expected/insert_optimizer.out  |    5 -
 src/test/regress/sql/insert.sql                 |    1 -
 src/test/unit/mock/mock_info.json               |   41 -
 tools/bin/gppylib/data/2.0.json                 |   93 --
 72 files changed, 36 insertions(+), 5076 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/access/appendonly/Makefile
----------------------------------------------------------------------
diff --git a/src/backend/access/appendonly/Makefile b/src/backend/access/appendonly/Makefile
index 2e6282a..805acca 100755
--- a/src/backend/access/appendonly/Makefile
+++ b/src/backend/access/appendonly/Makefile
@@ -29,7 +29,7 @@ include $(top_builddir)/src/Makefile.global
 override CPPFLAGS := -I$(top_srcdir)/src/backend/gp_libpq_fe $(CPPFLAGS)
 override CPPFLAGS := -I$(libpq_srcdir) $(CPPFLAGS)
 
-OBJS = appendonlyam.o aosegfiles.o aomd.o appendonlywriter.o appendonlytid.o appendonlyblockdirectory.o
+OBJS = appendonlyam.o aosegfiles.o aomd.o appendonlywriter.o appendonlytid.o
 
 include $(top_srcdir)/src/backend/common.mk
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/access/appendonly/aosegfiles.c
----------------------------------------------------------------------
diff --git a/src/backend/access/appendonly/aosegfiles.c b/src/backend/access/appendonly/aosegfiles.c
index da160ce..6da6a21 100644
--- a/src/backend/access/appendonly/aosegfiles.c
+++ b/src/backend/access/appendonly/aosegfiles.c
@@ -40,7 +40,6 @@
 #include "catalog/dependency.h"
 #include "catalog/indexing.h"
 #include "catalog/namespace.h"
-#include "catalog/gp_fastsequence.h"
 #include "catalog/aoseg.h"
 #include "cdb/cdbvars.h"
 #include "executor/spi.h"
@@ -93,15 +92,9 @@ InsertInitialSegnoEntry(AppendOnlyEntry *aoEntry, int segno)
 	int			natts = 0;
 	bool	   *nulls;
 	Datum	   *values;
-	ItemPointerData tid;
 
 	Assert(aoEntry != NULL);
 
-	InsertFastSequenceEntry(aoEntry->segrelid,
-							(int64)segno,
-							0,
-							&tid);
-
 	if (segno == 0)
 	{
 		return;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/access/appendonly/appendonlyam.c
----------------------------------------------------------------------
diff --git a/src/backend/access/appendonly/appendonlyam.c b/src/backend/access/appendonly/appendonlyam.c
index 68031f4..a7e88d2 100644
--- a/src/backend/access/appendonly/appendonlyam.c
+++ b/src/backend/access/appendonly/appendonlyam.c
@@ -76,7 +76,6 @@
 #include "catalog/pg_appendonly.h"
 #include "catalog/pg_attribute_encoding.h"
 #include "catalog/namespace.h"
-#include "catalog/gp_fastsequence.h"
 #include "cdb/cdbvars.h"
 #include "cdb/cdbappendonlyam.h"
 #include "pgstat.h"
@@ -306,46 +305,6 @@ SetNextFileSegForRead(AppendOnlyScanDesc scan)
 		 */
 		if(end_of_split > 0)
 		{
-			/* Initialize the block directory for inserts if needed. */
-			if (scan->buildBlockDirectory)
-			{
-				ItemPointerData tid;
-
-                /*
-				 * if building the block directory, we need to make sure the
-				 * sequence starts higher than our highest tuple's rownum.  In
-				 * the case of upgraded blocks, the highest tuple will
-				 * have tupCount as its row num for non-upgrade cases, which
-				 * use the sequence, it will be enough to start off the end
-				 * of the sequence; note that this is not ideal -- if we are at
-				 * least curSegInfo->tupcount + 1 then we don't even need to
-				 * update the sequence value.
-                 */
-                int64 firstSequence =
-                    GetFastSequences(scan->aoEntry->segrelid,
-                                     segno,
-                                     1,
-                                     NUM_FAST_SEQUENCES,
-                                     &tid);
-
-
-				AppendOnlyBlockDirectory_Init_forInsert(scan->blockDirectory,
-														scan->aoEntry,
-														scan->appendOnlyMetaDataSnapshot,
-														NULL,
-														0, /* lastSequence */
-														scan->aos_rd,
-														segno, /* segno */
-														1 /* columnGroupNo */);
-
-				Assert(!"need contentid here");
-				InsertFastSequenceEntry(scan->aoEntry->segrelid,
-										segno,
-										firstSequence,
-										/*TODO, need change in hawq*/
-										&tid);
-			}
-
 			finished_all_splits = false;
 			break;
 		}
@@ -448,16 +407,14 @@ errcontext_appendonly_insert_block(AppendOnlyInsertDesc aoInsertDesc)
 	char	*relationName = NameStr(aoInsertDesc->aoi_rel->rd_rel->relname);
 	int		segmentFileNum = aoInsertDesc->cur_segno;
 	int64	headerOffsetInFile = AppendOnlyStorageWrite_CurrentPosition(&aoInsertDesc->storageWrite);
-	int64	blockFirstRowNum = aoInsertDesc->blockFirstRowNum;
 	int64	bufferCount = aoInsertDesc->bufferCount;
 
 	errcontext(
 		 "Append-Only table '%s', segment file #%d, block header offset in file = " INT64_FORMAT ", "
-		 "block first row number " INT64_FORMAT ", bufferCount " INT64_FORMAT ")",
+		 ", bufferCount " INT64_FORMAT ")",
 		 relationName,
 		 segmentFileNum,
 		 headerOffsetInFile,
-		 blockFirstRowNum,
 		 bufferCount);
 
 	return 0;
@@ -595,8 +552,6 @@ CloseWritableFileSeg(AppendOnlyInsertDesc aoInsertDesc)
 	aoInsertDesc->sendback->eof[0] = fileLen;
 	aoInsertDesc->sendback->uncompressed_eof[0] = fileLen_uncompressed;
 
-	aoInsertDesc->sendback->nextFastSequence = aoInsertDesc->lastSequence + aoInsertDesc->numSequences - 1;
-
 	/*
 	 * Update the AO segment info table with our new eof
 	 */
@@ -1222,12 +1177,6 @@ LABEL_START_GETNEXTBLOCK:
 									&scan->executorReadBlock,
 									true))
 	{
-		if (scan->buildBlockDirectory)
-		{
-			Assert(scan->blockDirectory != NULL);
-			AppendOnlyBlockDirectory_End_forInsert(scan->blockDirectory);
-		}
-
 		/* done reading the file */
 		if(scan->toCloseFile){
 			CloseScannedFileSeg(scan);
@@ -1237,28 +1186,18 @@ LABEL_START_GETNEXTBLOCK:
 		return false;
 	}
 
-	if (scan->buildBlockDirectory)
-	{
-		Assert(scan->blockDirectory != NULL);
-		AppendOnlyBlockDirectory_InsertEntry(
-			scan->blockDirectory, 0,
-			scan->executorReadBlock.blockFirstRowNum,
-			scan->executorReadBlock.headerOffsetInFile,
-			scan->executorReadBlock.rowCount);
-	}
-
-    //skip invalid small content blocks
-    if(!scan->executorReadBlock.isLarge 
-            && scan->executorReadBlock.executorBlockKind == AoExecutorBlockKind_SingleRow
-            && scan->executorReadBlock.rowCount==0)
-    {
-        //skip current block
-        AppendOnlyStorageRead_SkipCurrentBlock(&scan->storageRead, true);
-        goto LABEL_START_GETNEXTBLOCK;
-    }else{
-        AppendOnlyExecutorReadBlock_GetContents(
-									&scan->executorReadBlock);
-    }
+  //skip invalid small content blocks
+  if(!scan->executorReadBlock.isLarge
+          && scan->executorReadBlock.executorBlockKind == AoExecutorBlockKind_SingleRow
+          && scan->executorReadBlock.rowCount==0)
+  {
+      //skip current block
+      AppendOnlyStorageRead_SkipCurrentBlock(&scan->storageRead, true);
+      goto LABEL_START_GETNEXTBLOCK;
+  }else{
+      AppendOnlyExecutorReadBlock_GetContents(
+                &scan->executorReadBlock);
+  }
 	return true;
 }
 
@@ -1349,11 +1288,6 @@ setupNextWriteBlock(AppendOnlyInsertDesc aoInsertDesc)
 	 */
 	AppendOnlyStorageWrite_PadOutForSplit(&aoInsertDesc->storageWrite, aoInsertDesc->usableBlockSize);
 
-	/* Set the firstRowNum for the block */
-	aoInsertDesc->blockFirstRowNum = aoInsertDesc->lastSequence + 1;
-	AppendOnlyStorageWrite_SetFirstRowNum(&aoInsertDesc->storageWrite,
-										  aoInsertDesc->blockFirstRowNum);
-
 	if(!aoInsertDesc->shouldCompress)
 	{
 		aoInsertDesc->nonCompressedData =
@@ -1484,14 +1418,6 @@ finishWriteBlock(AppendOnlyInsertDesc aoInsertDesc)
 							itemCount);
 	}
 
-	/* Insert an entry to the block directory */
-	AppendOnlyBlockDirectory_InsertEntry(
-		&aoInsertDesc->blockDirectory,
-		0,
-		aoInsertDesc->blockFirstRowNum,
-		AppendOnlyStorageWrite_LastWriteBeginPosition(&aoInsertDesc->storageWrite),
-		itemCount);
-
 	Assert(aoInsertDesc->nonCompressedData == NULL);
 	Assert(!AppendOnlyStorageWrite_IsBufferAllocated(&aoInsertDesc->storageWrite));
 }
@@ -1613,9 +1539,6 @@ appendonly_beginscan(Relation relation, Snapshot appendOnlyMetaDataSnapshot, int
 	//pgstat_initstats(relation);
 	initscan(scan, key);
 
-	scan->buildBlockDirectory = false;
-	scan->blockDirectory = NULL;
-
 	return scan;
 }
 
@@ -1701,623 +1624,6 @@ appendonly_getnext(AppendOnlyScanDesc scan, ScanDirection direction, TupleTableS
 	return tup;
 }
 
-static void
-closeFetchSegmentFile(
-	AppendOnlyFetchDesc aoFetchDesc)
-{
-	Assert(aoFetchDesc->currentSegmentFile.isOpen);
-
-	AppendOnlyStorageRead_CloseFile(&aoFetchDesc->storageRead);
-
-	aoFetchDesc->currentSegmentFile.isOpen = false;
-}
-
-static bool
-openFetchSegmentFile(
-	AppendOnlyFetchDesc aoFetchDesc,
-	int					openSegmentFileNum)
-{
-	int		i;
-
-	FileSegInfo	*fsInfo;
-	int			segmentFileNum;
-	int64		logicalEof;
-	int32		fileSegNo;
-
-	Assert(!aoFetchDesc->currentSegmentFile.isOpen);
-
-	i = 0;
-	while (true)
-	{
-		if (i >= aoFetchDesc->totalSegfiles)
-			return false;	// Segment file not visible in catalog information.
-
-		fsInfo = aoFetchDesc->segmentFileInfo[i];
-		segmentFileNum = fsInfo->segno;
-		if (openSegmentFileNum == segmentFileNum)
-		{
-			logicalEof = (int64)fsInfo->eof;
-			break;
-		}
-		i++;
-	}
-
-	/*
-	 * Don't try to open a segment file when its EOF is 0, since the file may not
-	 * exist. See MPP-8280.
-	 */
-	if (logicalEof == 0)
-		return false;
-
-	MakeAOSegmentFileName(
-			aoFetchDesc->relation,
-			openSegmentFileNum, -1,
-			&fileSegNo,
-			aoFetchDesc->segmentFileName);
-	Assert(strlen(aoFetchDesc->segmentFileName) + 1 <=
-		   aoFetchDesc->segmentFileNameMaxLen);
-
-	// UNDONE: Appropriate to use Try here?
-	if (!AppendOnlyStorageRead_TryOpenFile(
-						&aoFetchDesc->storageRead,
-						aoFetchDesc->segmentFileName,
-						logicalEof,
-						-1))
-		return false;
-
-	aoFetchDesc->currentSegmentFile.num = openSegmentFileNum;
-	aoFetchDesc->currentSegmentFile.logicalEof = logicalEof;
-
-	aoFetchDesc->currentSegmentFile.isOpen = true;
-
-	return true;
-}
-
-static bool
-fetchNextBlock(
-	AppendOnlyFetchDesc aoFetchDesc)
-{
-	AppendOnlyExecutorReadBlock *executorReadBlock =
-										&aoFetchDesc->executorReadBlock;
-
-	/*
-	 * Try to read next block.
-	 */
-	if (!AppendOnlyExecutorReadBlock_GetBlockInfo(
-			&aoFetchDesc->storageRead,
-			&aoFetchDesc->executorReadBlock,
-			true))
-		return false;	// Hit end of range.
-
-	/*
-	 * Unpack information into member variables.
-	 */
-	aoFetchDesc->currentBlock.have = true;
-	aoFetchDesc->currentBlock.fileOffset =
-				executorReadBlock->headerOffsetInFile;
-	aoFetchDesc->currentBlock.overallBlockLen =
-				AppendOnlyStorageRead_OverallBlockLen(
-										&aoFetchDesc->storageRead);
-	aoFetchDesc->currentBlock.firstRowNum =
-				executorReadBlock->blockFirstRowNum;
-	aoFetchDesc->currentBlock.lastRowNum =
-				executorReadBlock->blockFirstRowNum +
-				executorReadBlock->rowCount - 1;
-
-	aoFetchDesc->currentBlock.isCompressed =
-				executorReadBlock->isCompressed;
-	aoFetchDesc->currentBlock.isLargeContent =
-				executorReadBlock->isLarge;
-
-	aoFetchDesc->currentBlock.gotContents = false;
-
-	return true;
-}
-
-static bool
-fetchFromCurrentBlock(
-	AppendOnlyFetchDesc aoFetchDesc,
-	int64				rowNum,
-	TupleTableSlot 		*slot)
-{
-	Assert(aoFetchDesc->currentBlock.have);
-	Assert(rowNum >= aoFetchDesc->currentBlock.firstRowNum);
-	Assert(rowNum <= aoFetchDesc->currentBlock.lastRowNum);
-
-	if (!aoFetchDesc->currentBlock.gotContents)
-	{
-		/*
-		 * Do decompression if necessary and get contents.
-		 */
-		AppendOnlyExecutorReadBlock_GetContents(
-						&aoFetchDesc->executorReadBlock);
-
-		aoFetchDesc->currentBlock.gotContents = true;
-	}
-
-	return AppendOnlyExecutorReadBlock_FetchTuple(
-							&aoFetchDesc->executorReadBlock,
-							rowNum,
-							/* nkeys */ 0,
-							/* key */ NULL,
-							slot);
-}
-
-static void
-positionFirstBlockOfRange(
-	AppendOnlyFetchDesc aoFetchDesc)
-{
-	AppendOnlyBlockDirectoryEntry_GetBeginRange(
-				&aoFetchDesc->currentBlock.blockDirectoryEntry,
-				&aoFetchDesc->scanNextFileOffset,
-				&aoFetchDesc->scanNextRowNum);
-}
-
-static void
-positionLimitToEndOfRange(
-	AppendOnlyFetchDesc aoFetchDesc)
-{
-	AppendOnlyBlockDirectoryEntry_GetEndRange(
-				&aoFetchDesc->currentBlock.blockDirectoryEntry,
-				&aoFetchDesc->scanAfterFileOffset,
-				&aoFetchDesc->scanLastRowNum);
-}
-
-
-static void
-positionSkipCurrentBlock(
-	AppendOnlyFetchDesc aoFetchDesc)
-{
-	aoFetchDesc->scanNextFileOffset =
-		aoFetchDesc->currentBlock.fileOffset +
-		aoFetchDesc->currentBlock.overallBlockLen;
-
-	aoFetchDesc->scanNextRowNum = aoFetchDesc->currentBlock.lastRowNum + 1;
-}
-
-/*
- * Scan through blocks to find row.
- *
- * If row is not represented in any of the blocks covered by the Block Directory, then the row
- * falls into a row gap.  The row must have been aborted or deleted and reclaimed.
- */
-static bool
-scanToFetchTuple(
-	AppendOnlyFetchDesc aoFetchDesc,
-	int64				rowNum,
-	TupleTableSlot 		*slot)
-{
-	if (aoFetchDesc->scanNextFileOffset >=
-		aoFetchDesc->scanAfterFileOffset)
-		return false;	// No more blocks requested for range.
-
-	if (aoFetchDesc->currentSegmentFile.logicalEof ==
-		aoFetchDesc->scanNextFileOffset)
-		return false;	// No more blocks in this file.
-
-	if (aoFetchDesc->currentSegmentFile.logicalEof <
-		aoFetchDesc->scanNextFileOffset)
-		return false;	// UNDONE: Why does our next scan position go beyond logical EOF?
-
-	/*
-	 * Temporarily restrict our reading to just the range.
-	 */
-	AppendOnlyStorageRead_SetTemporaryRange(
-		&aoFetchDesc->storageRead,
-		aoFetchDesc->scanNextFileOffset,
-		aoFetchDesc->scanAfterFileOffset);
-	AppendOnlyExecutionReadBlock_SetSegmentFileNum(
-		&aoFetchDesc->executorReadBlock,
-		aoFetchDesc->currentSegmentFile.num);
-	AppendOnlyExecutionReadBlock_SetPositionInfo(
-		&aoFetchDesc->executorReadBlock,
-		aoFetchDesc->scanNextRowNum);
-
-	aoFetchDesc->skipBlockCount = 0;
-	while (true)
-	{
-		/*
-		 * Fetch block starting at scanNextFileOffset.
-		 */
-		if (!fetchNextBlock(aoFetchDesc))
-			return false;	// No more blocks.
-
-		/*
-		 * Examine new current block header information.
-		 */
-		if (rowNum < aoFetchDesc->currentBlock.firstRowNum)
-		{
-			/*
-			 * Since we have read a new block, the temporary
-			 * range for the read needs to be adjusted
-			 * accordingly. Otherwise, the underlying bufferedRead
-			 * may stop reading more data because of the
-			 * previously-set smaller temporary range.
-			 */
-			int64 beginFileOffset = aoFetchDesc->currentBlock.fileOffset;
-			int64 afterFileOffset = aoFetchDesc->currentBlock.fileOffset +
-				aoFetchDesc->currentBlock.overallBlockLen;
-
-			AppendOnlyStorageRead_SetTemporaryRange(
-				&aoFetchDesc->storageRead,
-				beginFileOffset,
-				afterFileOffset);
-
-			return false;	// Row fell in gap between blocks.
-		}
-
-		if (rowNum <= aoFetchDesc->currentBlock.lastRowNum)
-			return fetchFromCurrentBlock(aoFetchDesc, rowNum, slot);
-
-		/*
-		 * Update information to get next block.
-		 */
-		Assert(!aoFetchDesc->currentBlock.gotContents);
-		
-		/* MPP-17061: reach the end of range covered by block directory entry */
-		if ((aoFetchDesc->currentBlock.fileOffset +
-			aoFetchDesc->currentBlock.overallBlockLen) >=
-			aoFetchDesc->scanAfterFileOffset)
-		{
-			return false;
-		}
-			
-		AppendOnlyExecutionReadBlock_FinishedScanBlock(
-									&aoFetchDesc->executorReadBlock);
-
-		AppendOnlyStorageRead_SkipCurrentBlock(
-									&aoFetchDesc->storageRead,true);
-		aoFetchDesc->skipBlockCount++;
-	}
-}
-
-
-AppendOnlyFetchDesc
-appendonly_fetch_init(
-	Relation 	relation,
-	Snapshot 	appendOnlyMetaDataSnapshot)
-{
-	AppendOnlyFetchDesc	aoFetchDesc;
-	AppendOnlyEntry		*aoentry;
-
-	AppendOnlyStorageAttributes *attr;
-
-	ValidateAppendOnlyMetaDataSnapshot(&appendOnlyMetaDataSnapshot);
-	PGFunction *fns = NULL;
-
-	StringInfoData titleBuf;
-
-	/*
-	 * increment relation ref count while scanning relation
-	 *
-	 * This is just to make really sure the relcache entry won't go away while
-	 * the scan has a pointer to it.  Caller should be holding the rel open
-	 * anyway, so this is redundant in all normal scenarios...
-	 */
-	RelationIncrementReferenceCount(relation);
-
-	/*
-	 * allocate scan descriptor
-	 */
-	aoFetchDesc = (AppendOnlyFetchDesc) palloc0(sizeof(AppendOnlyFetchDescData));
-
-	aoFetchDesc->relation = relation;
-	aoFetchDesc->appendOnlyMetaDataSnapshot = appendOnlyMetaDataSnapshot;
-
-	aoFetchDesc->initContext = CurrentMemoryContext;
-
-	aoFetchDesc->segmentFileNameMaxLen = AOSegmentFilePathNameLen(relation) + 1;
-	aoFetchDesc->segmentFileName =
-						(char*)palloc(aoFetchDesc->segmentFileNameMaxLen);
-	aoFetchDesc->segmentFileName[0] = '\0';
-
-	initStringInfo(&titleBuf);
-	appendStringInfo(&titleBuf, "Fetch of Append-Only Row-Oriented relation '%s'",
-					 RelationGetRelationName(relation));
-	aoFetchDesc->title = titleBuf.data;
-
-	/*
-	 * Get the pg_appendonly information for this table
-	 */
-	aoentry = GetAppendOnlyEntry(RelationGetRelid(relation), appendOnlyMetaDataSnapshot);
-
-	aoFetchDesc->aoEntry = aoentry;
-
-	/*
-	 * Fill in Append-Only Storage layer attributes.
-	 */
-	attr = &aoFetchDesc->storageAttributes;
-
-	/*
-	 * These attributes describe the AppendOnly format to be scanned.
-	 */
-  if (aoentry->compresstype == NULL || pg_strcasecmp(aoentry->compresstype, "none") == 0)
-		attr->compress = false;
-	else
-		attr->compress = true;
-	if (aoentry->compresstype != NULL)
-		attr->compressType = aoentry->compresstype;
-	else
-		attr->compressType = "none";
-	attr->compressLevel = aoentry->compresslevel;
-	attr->checksum			= aoentry->checksum;
-	attr->safeFSWriteSize	= aoentry->safefswritesize;
-	attr->splitsize = aoentry->splitsize;
-	attr->version			= aoentry->version;
-
-	AORelationVersion_CheckValid(attr->version);
-
-	aoFetchDesc->usableBlockSize = aoentry->blocksize;
-				/* AppendOnlyStorage_GetUsableBlockSize(aoentry->blocksize); */
-
-	/*
-	 * Get information about all the file segments we need to scan
-	 * Currently, fetch operation is disabled. So we just set the
-	 * segmentFileInfo NULL.
-	 */
-	aoFetchDesc->segmentFileInfo = NULL;
-	/*
-						GetAllFileSegInfo(
-									relation,
-									aoentry,
-									appendOnlyMetaDataSnapshot,
-									false,
-									&aoFetchDesc->totalSegfiles);
-	*/
-	AppendOnlyStorageRead_Init(
-						&aoFetchDesc->storageRead,
-						aoFetchDesc->initContext,
-						aoFetchDesc->usableBlockSize,
-						NameStr(aoFetchDesc->relation->rd_rel->relname),
-						aoFetchDesc->title,
-						&aoFetchDesc->storageAttributes);
-
-
-	fns = RelationGetRelationCompressionFuncs(relation);
-	aoFetchDesc->storageRead.compression_functions = fns;
-
-	if (fns)
-	{
-		PGFunction cons = fns[COMPRESSION_CONSTRUCTOR];
-		CompressionState *cs;
-		StorageAttributes sa;
-
-		sa.comptype = aoentry->compresstype;
-		sa.complevel = aoentry->compresslevel;
-		sa.blocksize = aoentry->blocksize;
-
-
-		cs = callCompressionConstructor(cons, RelationGetDescr(relation),
-										&sa,
-										false /* decompress */);
-		aoFetchDesc->storageRead.compressionState = cs;
-	}
-
-	AppendOnlyExecutorReadBlock_Init(
-						&aoFetchDesc->executorReadBlock,
-						aoFetchDesc->relation,
-						aoFetchDesc->initContext,
-						&aoFetchDesc->storageRead,
-						aoFetchDesc->usableBlockSize);
-
-	AppendOnlyBlockDirectory_Init_forSearch(
-						&aoFetchDesc->blockDirectory,
-						aoentry,
-						appendOnlyMetaDataSnapshot,
-						aoFetchDesc->segmentFileInfo,
-						aoFetchDesc->totalSegfiles,
-						aoFetchDesc->relation,
-						1);
-
-	return aoFetchDesc;
-
-}
-
-/*
- * appendonly_fetch -- fetch the tuple for a given tid.
- *
- * If the 'slot' is not NULL, the fetched tuple will be assigned to the slot.
- *
- * Return true if such a tuple is found. Otherwise, return false.
- */
-bool
-appendonly_fetch(
-	AppendOnlyFetchDesc 	aoFetchDesc,
-	AOTupleId 				*aoTupleId,
-	TupleTableSlot 			*slot)
-{
-	int		segmentFileNum = AOTupleIdGet_segmentFileNum(aoTupleId);
-	int64 	rowNum = AOTupleIdGet_rowNum(aoTupleId);
-
-	/*
-	 * Do we have a current block?  If it has the requested tuple,
-	 * that would be a great performance optimization.
-	 */
-	if (aoFetchDesc->currentBlock.have)
-	{
-		if (segmentFileNum == aoFetchDesc->currentSegmentFile.num &&
-			segmentFileNum == aoFetchDesc->blockDirectory.currentSegmentFileNum)
-		{
-			if (rowNum >= aoFetchDesc->currentBlock.firstRowNum &&
-				rowNum <= aoFetchDesc->currentBlock.lastRowNum)
-				return fetchFromCurrentBlock(aoFetchDesc, rowNum, slot);
-
-			/*
-			 * Otherwize, if the current Block Directory entry covers the request tuples,
-			 * lets use its information as another performance optimization.
-			 */
-			if (AppendOnlyBlockDirectoryEntry_RangeHasRow(
-							&aoFetchDesc->currentBlock.blockDirectoryEntry,
-							rowNum))
-			{
-				/*
-				 * The tuple is covered by the current Block Directory entry, but is it
-				 * before or after our current block?
-				 */
-				if (rowNum < aoFetchDesc->currentBlock.firstRowNum)
-				{
-					/*
-					 * XXX This could happen when an insert is cancelled. In that case, we
-					 * fetched the next block that has a higher firstRowNum when we
-					 * try to find the first cancelled row. So for the second or any
-					 * cancelled row, we enter here, and re-read the previous block.
-					 * This seems inefficient.
-					 *
-					 * We may be able to fix this by adding an entry to the block
-					 * directory for those cancelled inserts.
-					 */
-
-					/*
-					 * Set scan range to prior blocks.
-					 */
-					positionFirstBlockOfRange(aoFetchDesc);
-
-					// Set limit to before current block.
-					aoFetchDesc->scanAfterFileOffset =
-									aoFetchDesc->currentBlock.fileOffset;
-
-					aoFetchDesc->scanLastRowNum =
-									aoFetchDesc->currentBlock.firstRowNum - 1;
-				}
-				else
-				{
-					/*
-					 * Set scan range to following blocks.
-					 */
-					positionSkipCurrentBlock(aoFetchDesc);
-
-					positionLimitToEndOfRange(aoFetchDesc);
-				}
-
-				if (scanToFetchTuple(aoFetchDesc, rowNum, slot))
-					return true;
-
-				if (slot != NULL)
-					ExecClearTuple(slot);
-				return false;	// Segment file not in aoseg table..
-			}
-		}
-	}
-
-//	resetCurrentBlockInfo(aoFetchDesc);
-
-	/*
-	 * Open or switch open, if necessary.
-	 */
-	if (aoFetchDesc->currentSegmentFile.isOpen &&
-		segmentFileNum != aoFetchDesc->currentSegmentFile.num)
-	{
-#ifdef USE_ASSERT_CHECKING
-		if (segmentFileNum < aoFetchDesc->currentSegmentFile.num)
-			ereport(WARNING,
-					(errmsg("Append-only fetch requires scan prior segment file: "
-							"segmentFileNum %d, rowNum " INT64_FORMAT
-							", currentSegmentFileNum %d",
-							segmentFileNum, rowNum, aoFetchDesc->currentSegmentFile.num)));
-#endif
-		closeFetchSegmentFile(aoFetchDesc);
-
-		Assert(!aoFetchDesc->currentSegmentFile.isOpen);
-	}
-
-	if (!aoFetchDesc->currentSegmentFile.isOpen)
-	{
-		if (!openFetchSegmentFile(
-					aoFetchDesc,
-					segmentFileNum))
-		{
-			if (slot != NULL)
-				ExecClearTuple(slot);
-			return false;	// Segment file not in aoseg table..
-							// Must be aborted or deleted and reclaimed.
-		}
-	}
-
-	/*
-	 * Need to get the Block Directory entry that covers the TID.
-	 */
-	if (!AppendOnlyBlockDirectory_GetEntry(
-									&aoFetchDesc->blockDirectory,
-									aoTupleId,
-									0,
-									&aoFetchDesc->currentBlock.blockDirectoryEntry))
-	{
-		if (slot != NULL)
-		{
-			ExecClearTuple(slot);
-		}
-		return false;	/* Row not represented in Block Directory. */
-						/* Must be aborted or deleted and reclaimed. */
-	}
-
-	/*
-	 * Set scan range covered by new Block Directory entry.
-	 */
-	positionFirstBlockOfRange(aoFetchDesc);
-
-	positionLimitToEndOfRange(aoFetchDesc);
-
-	if (scanToFetchTuple(aoFetchDesc, rowNum, slot))
-		return true;
-
-	if (slot != NULL)
-		ExecClearTuple(slot);
-	return false;	// Segment file not in aoseg table..
-}
-
-void
-appendonly_fetch_detail(
-	AppendOnlyFetchDesc 		aoFetchDesc,
-	AppendOnlyFetchDetail 		*aoFetchDetail)
-{
-	aoFetchDetail->rangeFileOffset =
-			aoFetchDesc->currentBlock.blockDirectoryEntry.range.fileOffset;
-	aoFetchDetail->rangeFirstRowNum =
-			aoFetchDesc->currentBlock.blockDirectoryEntry.range.firstRowNum;
-	aoFetchDetail->rangeAfterFileOffset =
-			aoFetchDesc->currentBlock.blockDirectoryEntry.range.afterFileOffset;
-	aoFetchDetail->rangeLastRowNum =
-			aoFetchDesc->currentBlock.blockDirectoryEntry.range.lastRowNum;
-
-	aoFetchDetail->skipBlockCount = aoFetchDesc->skipBlockCount;
-
-	aoFetchDetail->blockFileOffset = aoFetchDesc->currentBlock.fileOffset;
-	aoFetchDetail->blockOverallLen = aoFetchDesc->currentBlock.overallBlockLen;
-	aoFetchDetail->blockFirstRowNum = aoFetchDesc->currentBlock.firstRowNum;
-	aoFetchDetail->blockLastRowNum = aoFetchDesc->currentBlock.lastRowNum;
-	aoFetchDetail->isCompressed = aoFetchDesc->currentBlock.isCompressed;
-	aoFetchDetail->isLargeContent = aoFetchDesc->currentBlock.isLargeContent;
-}
-
-void
-appendonly_fetch_finish(AppendOnlyFetchDesc aoFetchDesc)
-{
-	RelationDecrementReferenceCount(aoFetchDesc->relation);
-
-	AppendOnlyStorageRead_CloseFile(&aoFetchDesc->storageRead);
-
-	AppendOnlyStorageRead_FinishSession(&aoFetchDesc->storageRead);
-
-	AppendOnlyExecutorReadBlock_Finish(&aoFetchDesc->executorReadBlock);
-
-	AppendOnlyBlockDirectory_End_forSearch(&aoFetchDesc->blockDirectory);
-
-	if (aoFetchDesc->segmentFileInfo)
-	{
-		FreeAllSegFileInfo(aoFetchDesc->segmentFileInfo, aoFetchDesc->totalSegfiles);
-		pfree(aoFetchDesc->segmentFileInfo);
-		aoFetchDesc->segmentFileInfo = NULL;
-	}
-
-	pfree(aoFetchDesc->aoEntry);
-	aoFetchDesc->aoEntry = NULL;
-
-	pfree(aoFetchDesc->segmentFileName);
-	aoFetchDesc->segmentFileName = NULL;
-
-	pfree(aoFetchDesc->title);
-}
-
 /*
  * appendonly_insert_init
  *
@@ -2334,7 +1640,6 @@ appendonly_insert_init(Relation rel, ResultRelSegFileInfo *segfileinfo)
 	AppendOnlyInsertDesc 	aoInsertDesc;
 	AppendOnlyEntry				*aoentry;
 	int 							maxtupsize;
-	int64 						firstSequence = 0;
 	PGFunction 				*fns;
 	int 					desiredOverflowBytes = 0;
 	size_t 					(*desiredCompressionSize)(size_t input);
@@ -2366,7 +1671,6 @@ appendonly_insert_init(Relation rel, ResultRelSegFileInfo *segfileinfo)
 	aoInsertDesc->appendFilePathName[0] = '\0';
 
 	aoInsertDesc->bufferCount = 0;
-	aoInsertDesc->blockFirstRowNum = 0;
 	aoInsertDesc->insertCount = 0;
 	aoInsertDesc->varblockCount = 0;
 	aoInsertDesc->rowCount = 0;
@@ -2472,13 +1776,6 @@ appendonly_insert_init(Relation rel, ResultRelSegFileInfo *segfileinfo)
 		     (aoentry->compresstype ? aoentry->compresstype : "<none>"),
 		     attr->compressLevel);
 
-	/*
-	 * Temporarily set the firstRowNum for the block so that we can
-	 * calculate the correct header length.
-	 */
-	AppendOnlyStorageWrite_SetFirstRowNum(&aoInsertDesc->storageWrite,
-										  1);
-
 	aoInsertDesc->completeHeaderLen =
 					AppendOnlyStorageWrite_CompleteHeaderLen(
 										&aoInsertDesc->storageWrite,
@@ -2509,31 +1806,8 @@ appendonly_insert_init(Relation rel, ResultRelSegFileInfo *segfileinfo)
 	Assert(!ItemPointerIsValid(&aoInsertDesc->fsInfo->sequence_tid));
 	Assert(aoInsertDesc->fsInfo->segno == segfileinfo->segno);
 
-	/*
-	firstSequence =
-		GetFastSequences(aoInsertDesc->aoEntry->segrelid,
-						 segfileinfo->segno,
-						 aoInsertDesc->rowCount + 1,
-						 NUM_FAST_SEQUENCES,
-						 &aoInsertDesc->fsInfo->sequence_tid);
-						 */
-	firstSequence = aoInsertDesc->rowCount + 1;
-	aoInsertDesc->numSequences = NUM_FAST_SEQUENCES;
-
-	/* Set last_sequence value */
-	Assert(firstSequence > aoInsertDesc->rowCount);
-	aoInsertDesc->lastSequence = firstSequence - 1;
-
 	setupNextWriteBlock(aoInsertDesc);
 
-	/* Initialize the block directory. */
-	AppendOnlyBlockDirectory_Init_forInsert(
-		&(aoInsertDesc->blockDirectory), 
-		aoentry, 
-		aoInsertDesc->appendOnlyMetaDataSnapshot,		// CONCERN: Safe to assume all block directory entries for segment are "covered" by same exclusive lock.
-		aoInsertDesc->fsInfo, aoInsertDesc->lastSequence,
-		rel, segfileinfo->segno, 1);
-
 	return aoInsertDesc;
 }
 
@@ -2766,48 +2040,12 @@ appendonly_insert_init(Relation rel, ResultRelSegFileInfo *segfileinfo)
 	}
 
 	aoInsertDesc->insertCount++;
-	aoInsertDesc->lastSequence++;
-	if (aoInsertDesc->numSequences > 0)
-		(aoInsertDesc->numSequences)--;
-
-	Assert(aoInsertDesc->numSequences >= 0);
-
 	pgstat_count_heap_insert(relation);
 
 	*tupleOid = MemTupleGetOid(tup, aoInsertDesc->mt_bind);
 
 	AOTupleIdInit_Init(aoTupleId);
 	AOTupleIdInit_segmentFileNum(aoTupleId, aoInsertDesc->cur_segno);
-	AOTupleIdInit_rowNum(aoTupleId, aoInsertDesc->lastSequence);
-
-	/*
-	 * If the allocated fast sequence numbers are used up, we request for
-	 * a next list of fast sequence numbers.
-	 */
-	if (aoInsertDesc->numSequences == 0)
-	{
-		int64 firstSequence;
-
-		/*
-		 * in hawq, catalog are in memory heap table,
-		 * ItemPointer of tuple is invalid.
-		 */
-		if (Gp_role == GP_ROLE_EXECUTE)
-		{
-		    /*
-			firstSequence = GetFastSequences(aoInsertDesc->aoEntry->segrelid,
-					aoInsertDesc->cur_segno, aoInsertDesc->lastSequence + 1,
-					NUM_FAST_SEQUENCES, &aoInsertDesc->fsInfo->sequence_tid);
-					*/
-		    firstSequence = aoInsertDesc->lastSequence + 1;
-		} else {
-			firstSequence = GetFastSequencesByTid(
-					&aoInsertDesc->fsInfo->sequence_tid,
-					aoInsertDesc->lastSequence + 1, NUM_FAST_SEQUENCES);
-		}
-		Assert(firstSequence == aoInsertDesc->lastSequence + 1);
-		aoInsertDesc->numSequences = NUM_FAST_SEQUENCES;
-	}
 
 	if (Debug_appendonly_print_insert_tuple)
 	{
@@ -2840,8 +2078,6 @@ appendonly_insert_finish(AppendOnlyInsertDesc aoInsertDesc)
 
 	CloseWritableFileSeg(aoInsertDesc);
 
-	AppendOnlyBlockDirectory_End_forInsert(&(aoInsertDesc->blockDirectory));
-
 	AppendOnlyStorageWrite_FinishSession(&aoInsertDesc->storageWrite);
 
 	pfree(aoInsertDesc->aoEntry);

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/access/appendonly/appendonlyblockdirectory.c
----------------------------------------------------------------------
diff --git a/src/backend/access/appendonly/appendonlyblockdirectory.c b/src/backend/access/appendonly/appendonlyblockdirectory.c
deleted file mode 100644
index df5b45d..0000000
--- a/src/backend/access/appendonly/appendonlyblockdirectory.c
+++ /dev/null
@@ -1,1113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/*-----------------------------------------------------------------------------
- *
- * appendonlyblockdirectory
- *    maintain the block directory to blocks in an append-only relation
- * files.
- *
- *-----------------------------------------------------------
- */
-#include "postgres.h"
-
-#include "cdb/cdbappendonlyblockdirectory.h"
-#include "catalog/aoblkdir.h"
-#include "access/heapam.h"
-#include "access/genam.h"
-#include "catalog/indexing.h"
-#include "parser/parse_oper.h"
-#include "utils/lsyscache.h"
-#include "utils/memutils.h"
-#include "utils/guc.h"
-#include "cdb/cdbappendonlyam.h"
-
-int gp_blockdirectory_entry_min_range = 0;
-int gp_blockdirectory_minipage_size = NUM_MINIPAGE_ENTRIES;
-
-static inline uint32 minipage_size(uint32 nEntry)
-{
-	return offsetof(Minipage, entry) +
-		sizeof(MinipageEntry) * nEntry;
-}
-
-static void load_last_minipage(
-	AppendOnlyBlockDirectory *blockDirectory,
-	int64 lastSequence,
-	int columnGroupNo);
-static void init_scankeys(
-	TupleDesc tupleDesc,
-	int nkeys, ScanKey scanKeys,
-	StrategyNumber *strategyNumbers);
-static int find_minipage_entry(
-	Minipage *minipage,
-	uint32 numEntries,
-	int64 rowNum);
-static void extract_minipage(
-	AppendOnlyBlockDirectory *blockDirectory,
-	HeapTuple tuple,
-	TupleDesc tupleDesc,
-	int columnGroupNo);
-static void write_minipage(AppendOnlyBlockDirectory *blockDirectory,
-						   int columnGroupNo);
-
-void 
-AppendOnlyBlockDirectoryEntry_GetBeginRange(
-	AppendOnlyBlockDirectoryEntry	*directoryEntry,
-	int64							*fileOffset,
-	int64							*firstRowNum)
-{
-	*fileOffset = directoryEntry->range.fileOffset;
-	*firstRowNum = directoryEntry->range.firstRowNum;
-}
-
-void 
-AppendOnlyBlockDirectoryEntry_GetEndRange(
-	AppendOnlyBlockDirectoryEntry	*directoryEntry,
-	int64							*afterFileOffset,
-	int64							*lastRowNum)
-{
-	*afterFileOffset = directoryEntry->range.afterFileOffset;
-	*lastRowNum = directoryEntry->range.lastRowNum;
-}
-
-bool
-AppendOnlyBlockDirectoryEntry_RangeHasRow(
-	AppendOnlyBlockDirectoryEntry	*directoryEntry,
-	int64							checkRowNum)
-{
-	return (checkRowNum >= directoryEntry->range.firstRowNum &&
-		    checkRowNum <= directoryEntry->range.lastRowNum);
-}
-
-/*
- * init_internal
- *
- * Initialize the block directory structure.
- */
-static void
-init_internal(AppendOnlyBlockDirectory *blockDirectory)
-{
-	MemoryContext oldcxt;
-	int numScanKeys;
-	TupleDesc heapTupleDesc;
-	TupleDesc idxTupleDesc;
-	int groupNo;
-	
-	Assert(blockDirectory->blkdirRel != NULL);
-	Assert(blockDirectory->blkdirIdx != NULL);
-	
-	blockDirectory->memoryContext =
-		AllocSetContextCreate(CurrentMemoryContext,
-							  "BlockDirectoryContext",
-							  ALLOCSET_DEFAULT_MINSIZE,
-							  ALLOCSET_DEFAULT_INITSIZE,
-							  ALLOCSET_DEFAULT_MAXSIZE);
-
-	oldcxt = MemoryContextSwitchTo(blockDirectory->memoryContext);
-
-	heapTupleDesc = RelationGetDescr(blockDirectory->blkdirRel);
-	blockDirectory->values = palloc0(sizeof(Datum) * heapTupleDesc->natts);
-	blockDirectory->nulls = palloc0(sizeof(bool) * heapTupleDesc->natts);
-	blockDirectory->numScanKeys = 3;
-	numScanKeys = blockDirectory->numScanKeys;
-	blockDirectory->scanKeys = palloc0(numScanKeys * sizeof(ScanKeyData));
-
-	blockDirectory->strategyNumbers = palloc0(numScanKeys * sizeof(StrategyNumber));
-	blockDirectory->strategyNumbers[0] = BTEqualStrategyNumber;
-	blockDirectory->strategyNumbers[1] = BTEqualStrategyNumber;
-	blockDirectory->strategyNumbers[2] = BTLessEqualStrategyNumber;
-
-	idxTupleDesc = RelationGetDescr(blockDirectory->blkdirIdx);
-
-	init_scankeys(idxTupleDesc, numScanKeys,
-				  blockDirectory->scanKeys,
-				  blockDirectory->strategyNumbers);
-
-	/* Initialize the last minipage */
-	blockDirectory->minipages =
-		palloc0(sizeof(MinipagePerColumnGroup) * blockDirectory->numColumnGroups);
-	for (groupNo = 0; groupNo < blockDirectory->numColumnGroups; groupNo++)
-	{
-		MinipagePerColumnGroup *minipageInfo = &blockDirectory->minipages[groupNo];
-		
-		minipageInfo->minipage =
-			palloc0(minipage_size(NUM_MINIPAGE_ENTRIES));
-		minipageInfo->numMinipageEntries = 0;
-	}
-	
-
-	MemoryContextSwitchTo(oldcxt);
-}
-
-/*
- * AppendOnlyBlockDirectory_Init_forSearch
- *
- * Initialize the block directory to handle the lookup.
- *
- * If the block directory relation for this appendonly relation
- * does not exist before calling this function, set blkdirRel
- * and blkdirIdx to NULL, and return.
- */
-void
-AppendOnlyBlockDirectory_Init_forSearch(
-	AppendOnlyBlockDirectory *blockDirectory,
-	AppendOnlyEntry *aoEntry,
-	Snapshot appendOnlyMetaDataSnapshot,
-	FileSegInfo **segmentFileInfo,
-	int totalSegfiles,
-	Relation aoRel,
-	int numColumnGroups)
-{	
-	Assert(aoEntry != NULL);
-
-	blockDirectory->aoRel = aoRel;
-
-	if (!OidIsValid(aoEntry->blkdirrelid))
-	{
-		Assert(!OidIsValid(aoEntry->blkdiridxid));
-		blockDirectory->blkdirRel = NULL;
-		blockDirectory->blkdirIdx = NULL;
-
-		return;
-	}
-
-	if (Debug_appendonly_print_blockdirectory)
-		ereport(LOG,
-				(errmsg("Append-only block directory init for search: "
-						"(totalSegfiles, numColumnGroups)="
-						"(%d, %d)",
-						totalSegfiles, numColumnGroups)));
-
-	blockDirectory->segmentFileInfo = segmentFileInfo;
-	blockDirectory->totalSegfiles = totalSegfiles;
-	blockDirectory->aoRel = aoRel;
-	blockDirectory->appendOnlyMetaDataSnapshot = appendOnlyMetaDataSnapshot;
-	blockDirectory->numColumnGroups = numColumnGroups;
-	blockDirectory->currentSegmentFileNum = -1;
-
-	Assert(OidIsValid(aoEntry->blkdirrelid));
-
-	blockDirectory->blkdirRel =
-		heap_open(aoEntry->blkdirrelid, AccessShareLock);
-
-	Assert(OidIsValid(aoEntry->blkdiridxid));
-
-	blockDirectory->blkdirIdx =
-		index_open(aoEntry->blkdiridxid, AccessShareLock);
-
-	init_internal(blockDirectory);
-}
-
-/*
- * AppendOnlyBlockDirectory_Init_forInsert
- *
- * Initialize the block directory to handle the inserts.
- *
- * If the block directory relation for this appendonly relation
- * does not exist before calling this function, set blkdirRel
- * and blkdirIdx to NULL, and return.
- */
-void
-AppendOnlyBlockDirectory_Init_forInsert(
-	AppendOnlyBlockDirectory *blockDirectory,
-	AppendOnlyEntry *aoEntry,
-	Snapshot appendOnlyMetaDataSnapshot,
-	FileSegInfo *segmentFileInfo,
-	int64 lastSequence,
-	Relation aoRel,
-	int segno,
-	int numColumnGroups)
-{
-	int groupNo;
-
-	Assert(aoEntry != NULL);
-
-	blockDirectory->aoRel = aoRel;
-	blockDirectory->appendOnlyMetaDataSnapshot = appendOnlyMetaDataSnapshot;
-
-	if (!OidIsValid(aoEntry->blkdirrelid))
-	{
-		Assert(!OidIsValid(aoEntry->blkdiridxid));
-		blockDirectory->blkdirRel = NULL;
-		blockDirectory->blkdirIdx = NULL;
-
-		return;
-	}
-
-	blockDirectory->segmentFileInfo = NULL;
-	blockDirectory->totalSegfiles = -1;
-	blockDirectory->currentSegmentFileInfo = segmentFileInfo;
-	
-	blockDirectory->currentSegmentFileNum = segno;
-	blockDirectory->numColumnGroups = numColumnGroups;
-
-	Assert(OidIsValid(aoEntry->blkdirrelid));
-
-	blockDirectory->blkdirRel =
-		heap_open(aoEntry->blkdirrelid, RowExclusiveLock);
-
-	Assert(OidIsValid(aoEntry->blkdiridxid));
-
-	blockDirectory->blkdirIdx =
-		index_open(aoEntry->blkdiridxid, RowExclusiveLock);
-
-	init_internal(blockDirectory);
-
-	if (Debug_appendonly_print_blockdirectory)
-		ereport(LOG,
-				(errmsg("Append-only block directory init for insert: "
-						"(segno, numColumnGroups, isAOCol, lastSequence)="
-						"(%d, %d, " INT64_FORMAT ")",
-						segno, numColumnGroups, lastSequence)));
-
-	/*
-	 * Load the last minipages from the block directory relation.
-	 */
-	for(groupNo = 0; groupNo < blockDirectory->numColumnGroups; groupNo++)
-	{
-		load_last_minipage(blockDirectory, lastSequence, groupNo);
-	}
-}
-
-static bool
-set_directoryentry_range(
-	AppendOnlyBlockDirectory *blockDirectory,
-	int columnGroupNo,
-	int entry_no,
-	AppendOnlyBlockDirectoryEntry *directoryEntry)
-{
-	MinipagePerColumnGroup *minipageInfo =
-		&blockDirectory->minipages[columnGroupNo];
-	FileSegInfo *fsInfo;
-	MinipageEntry *entry;
-	MinipageEntry *next_entry = NULL;
-	
-	Assert(entry_no >= 0 && ((uint32)entry_no) < minipageInfo->numMinipageEntries);
-	
-	fsInfo = blockDirectory->currentSegmentFileInfo;
-	Assert(fsInfo != NULL);
-
-	entry = &(minipageInfo->minipage->entry[entry_no]);
-	if (((uint32)entry_no) < minipageInfo->numMinipageEntries - 1)
-	{
-		next_entry = &(minipageInfo->minipage->entry[entry_no + 1]);
-	}
-	
-	directoryEntry->range.fileOffset = entry->fileOffset;
-	directoryEntry->range.firstRowNum = entry->firstRowNum;
-	if (next_entry != NULL)
-	{
-		directoryEntry->range.afterFileOffset = next_entry->fileOffset;
-	}
-	else
-	{
-		directoryEntry->range.afterFileOffset = fsInfo->eof;
-	}
-
-	directoryEntry->range.lastRowNum = entry->firstRowNum + entry->rowCount - 1;
-	if (next_entry == NULL && gp_blockdirectory_entry_min_range != 0)
-	{
-		directoryEntry->range.lastRowNum = (~(((int64)1) << 63)); /* set to the maximal value */
-	}
-
-	/*
-	 * When crashes during inserts, or cancellation during inserts,
-	 * the block directory may contain out-of-date entries. We
-	 * check for the end of file here. If the requested directory entry
-	 * is after the end of file, return false.
-	 */
-	if (directoryEntry->range.fileOffset > fsInfo->eof)
-		return false;
-
-	if ((directoryEntry->range.afterFileOffset > fsInfo->eof))
-	{
-		directoryEntry->range.afterFileOffset = fsInfo->eof;
-	}
-	
-	if (Debug_appendonly_print_blockdirectory)
-		ereport(LOG,
-				(errmsg("Append-only block directory find entry: "
-						"(columnGroupNo, firstRowNum, fileOffset, lastRowNum, afterFileOffset) = "
-						"(%d, " INT64_FORMAT ", " INT64_FORMAT ", " INT64_FORMAT ", " INT64_FORMAT ")",
-						columnGroupNo, directoryEntry->range.firstRowNum,
-						directoryEntry->range.fileOffset, directoryEntry->range.lastRowNum,
-						directoryEntry->range.afterFileOffset)));
-
-	return true;
-}
-
-/*
- * AppendOnlyBlockDirectory_GetEntry
- *
- * Find a directory entry for the given AOTupleId in the block directory.
- * If such an entry is found, return true. Otherwise, return false.
- *
- * The range for directoryEntry is assigned accordingly in this function.
- *
- * The block directory for the appendonly table should exist before calling
- * this function.
- */
-bool
-AppendOnlyBlockDirectory_GetEntry(
-	AppendOnlyBlockDirectory		*blockDirectory,
-	AOTupleId 						*aoTupleId,
-	int                             columnGroupNo,
-	AppendOnlyBlockDirectoryEntry	*directoryEntry)
-{
-	int segmentFileNum = AOTupleIdGet_segmentFileNum(aoTupleId);
-	int64 rowNum = AOTupleIdGet_rowNum(aoTupleId);
-	int		i;
-	Relation blkdirRel = blockDirectory->blkdirRel;
-	Relation blkdirIdx = blockDirectory->blkdirIdx;
-	int numScanKeys = blockDirectory->numScanKeys;
-	ScanKey scanKeys = blockDirectory->scanKeys;
-	
-	TupleDesc heapTupleDesc;
-	TupleDesc idxTupleDesc;
-	FileSegInfo *fsInfo = NULL;
-	IndexScanDesc idxScanDesc;
-	HeapTuple tuple = NULL;
-	MinipagePerColumnGroup *minipageInfo =
-		&blockDirectory->minipages[columnGroupNo];
-	int entry_no = -1;
-	int tmpGroupNo;
-	
-	if (blkdirRel == NULL || blkdirIdx == NULL)
-	{
-		Assert(RelationIsValid(blockDirectory->aoRel));
-		
-		ereport(ERROR,
-				(errcode(ERRCODE_GP_INTERNAL_ERROR),
-				 errmsg("Block directory for append-only relation '%s' does not exist",
-						RelationGetRelationName(blockDirectory->aoRel))));
-		return false;
-	}
-
-	if (Debug_appendonly_print_blockdirectory)
-		ereport(LOG,
-				(errmsg("Append-only block directory get entry: "
-						"(columnGroupNo, segmentFileNum, rowNum) = "
-						"(%d, %d, " INT64_FORMAT ")",
-						columnGroupNo, segmentFileNum, rowNum)));
-
-	/*
-	 * If the segment file number is the same as
-	 * blockDirectory->currentSegmentFileNum, the in-memory minipage
-	 * may contain such an entry. We search the in-memory minipage
-	 * first. If such an entry can not be found, we search for the
-	 * appropriate minipage by using the block directory btree index.
-	 */
-	if (segmentFileNum == blockDirectory->currentSegmentFileNum &&
-		minipageInfo->numMinipageEntries > 0)
-	{
-		Assert(blockDirectory->currentSegmentFileInfo != NULL);
-		
-		/*
-		 * Check if the existing minipage contains the requested
-		 * rowNum. If so, just get it.
-		 */
-		entry_no = find_minipage_entry(minipageInfo->minipage,
-									   minipageInfo->numMinipageEntries,
-									   rowNum);
-		if (entry_no != -1)
-		{
-			return set_directoryentry_range(blockDirectory,
-									 columnGroupNo,
-									 entry_no,
-									 directoryEntry);
-
-		}
-
-		/*
-		 * The given rowNum may point to a tuple that does not exist
-		 * in the AO table any more, either because of cancellation of
-		 * an insert, or due to crashes during an insert. If this is
-		 * the case, rowNum is smaller than the highest entry in
-		 * the in-memory minipage entry.
-		 */
-		else
-		{
-			MinipageEntry *entry =
-				&minipageInfo->minipage->entry[minipageInfo->numMinipageEntries - 1];
-			
-			if (rowNum < entry->firstRowNum + entry->rowCount - 1)
-				return false;
-		}
-	}
-
-	for (i = 0; i < blockDirectory->totalSegfiles; i++)
-	{
-		fsInfo = blockDirectory->segmentFileInfo[i];
-		
-		if (segmentFileNum == fsInfo->segno)
-			break;
-	}
-
-	Assert(fsInfo != NULL);
-
-	/*
-	 * Search the btree index to find the minipage that contains
-	 * the rowNum. We find the minipages for all column groups, since
-	 * currently we will need to access all columns at the same time.
-	 */
-	heapTupleDesc = RelationGetDescr(blkdirRel);
-	idxTupleDesc = RelationGetDescr(blkdirIdx);
-
-	Assert(numScanKeys == 3);
-
-	for (tmpGroupNo = 0; tmpGroupNo < blockDirectory->numColumnGroups; tmpGroupNo++)
-	{
-		/* Setup the scan keys for the scan. */
-		Assert(scanKeys != NULL);
-		scanKeys[0].sk_argument = Int32GetDatum(segmentFileNum);
-		scanKeys[1].sk_argument = Int32GetDatum(tmpGroupNo);
-		scanKeys[2].sk_argument = Int64GetDatum(rowNum);
-		
-		idxScanDesc = index_beginscan(blkdirRel, blkdirIdx,
-									  blockDirectory->appendOnlyMetaDataSnapshot,
-									  numScanKeys, scanKeys);
-	
-		tuple = index_getnext(idxScanDesc, BackwardScanDirection);
-
-		if (tuple != NULL)
-		{
-			/*
-			 * MPP-17061: we need to update currentSegmentFileNum
-			 * & currentSegmentFileInfo at the same time when we 
-			 * load the minipage for the block directory entry we
-			 * found, otherwise we would risk having inconsistency
-			 * between currentSegmentFileNum/currentSegmentFileInfo
-			 * and minipage contents, which would cause wrong block
-			 * header offset being returned in following block 
-			 * directory entry look up.
-			 */
-			blockDirectory->currentSegmentFileNum = segmentFileNum;
-			blockDirectory->currentSegmentFileInfo = fsInfo;
-			MinipagePerColumnGroup *minipageInfo;
-			
-			minipageInfo = &blockDirectory->minipages[tmpGroupNo];
-			extract_minipage(blockDirectory,
-							 tuple,
-							 heapTupleDesc,
-							 tmpGroupNo);
-		}
-		else
-		{
-			/* MPP-17061: index look up failed, row is invisible */
-			index_endscan(idxScanDesc);
-			return false;
-		}	
-
-		index_endscan(idxScanDesc);
-	}
-	
-	{
-		MinipagePerColumnGroup *minipageInfo;
-		minipageInfo = &blockDirectory->minipages[columnGroupNo];
-
-		/*
-		 * Perform a binary search over the minipage to find
-		 * the entry about the AO block.
-		 */
-		entry_no = find_minipage_entry(minipageInfo->minipage,
-									   minipageInfo->numMinipageEntries,
-									   rowNum);
-
-		/* If there are no entries, return false. */
-		if (entry_no == -1 && minipageInfo->numMinipageEntries == 0)
-			return false;
-
-		if (entry_no == -1)
-		{
-			/*
-			 * Since the last few blocks may not be logged in the block
-			 * directory, we always use the last entry.
-			 */
-			entry_no = minipageInfo->numMinipageEntries - 1;
-		}
-		return set_directoryentry_range(blockDirectory,
-										columnGroupNo,
-										entry_no,
-										directoryEntry);
-	}
-	
-	return false;
-}
-
-/*
- * AppendOnlyBlockDirectory_InsertEntry
- *
- * Insert an entry to the block directory. This entry is appended to the 
- * in-memory minipage. If the minipage is full, it is written to the block
- * directory relation on disk. After that, the new entry is added to the
- * new in-memory minipage.
- *
- * To reduce the size of a block directory, this function ignores new entries
- * when the range between the offset value of the latest existing entry and
- * the offset of the new entry is smaller than gp_blockdirectory_entry_min_range
- * (if it is set). Otherwise, the latest existing entry is updated with new
- * rowCount value, and the given new entry is appended to the in-memory minipage.
- *
- * If the block directory for the appendonly relation does not exist,
- * this function simply returns.
- *
- * If rowCount is 0, simple return false.
- */
-bool
-AppendOnlyBlockDirectory_InsertEntry(
-	AppendOnlyBlockDirectory *blockDirectory,
-	int columnGroupNo,
-	int64 firstRowNum,
-	int64 fileOffset,
-	int64 rowCount)
-{
-	MinipageEntry *entry = NULL;
-	MinipagePerColumnGroup *minipageInfo =
-		&blockDirectory->minipages[columnGroupNo];
-	int lastEntryNo;
-
-	if (rowCount == 0)
-		return false;
-	
-	if (blockDirectory->blkdirRel == NULL ||
-		blockDirectory->blkdirIdx == NULL)
-		return false;
-
-	Assert(minipageInfo->numMinipageEntries <= (uint32)NUM_MINIPAGE_ENTRIES);
-
-	lastEntryNo = minipageInfo->numMinipageEntries - 1;
-	if (lastEntryNo >= 0)
-	{
-		entry = &(minipageInfo->minipage->entry[lastEntryNo]);
-
-		Assert(entry->firstRowNum < firstRowNum);
-		Assert(entry->fileOffset < fileOffset);
-		
-		if (gp_blockdirectory_entry_min_range > 0 &&
-			fileOffset - entry->fileOffset < gp_blockdirectory_entry_min_range)
-			return true;
-		
-		/* Update the rowCount in the latest entry */
-		Assert(entry->rowCount <= firstRowNum - entry->firstRowNum);
-
-		if (Debug_appendonly_print_blockdirectory)
-			ereport(LOG,
-					(errmsg("Append-only block directory update entry: "
-							"(firstRowNum, columnGroupNo, fileOffset, rowCount) = (" INT64_FORMAT
-							", %d, " INT64_FORMAT ", " INT64_FORMAT ") at index %d to "
-							"(firstRowNum, columnGroupNo, fileOffset, rowCount) = (" INT64_FORMAT
-							", %d, " INT64_FORMAT ", " INT64_FORMAT ")",
-							entry->firstRowNum, columnGroupNo, entry->fileOffset, entry->rowCount,
-							minipageInfo->numMinipageEntries - 1,
-							entry->firstRowNum, columnGroupNo, entry->fileOffset,
-							firstRowNum - entry->firstRowNum)));
-		
-		entry->rowCount = firstRowNum - entry->firstRowNum;
-	}
-	
-	if (minipageInfo->numMinipageEntries >= (uint32)gp_blockdirectory_minipage_size)
-	{
-		write_minipage(blockDirectory, columnGroupNo);
-
-		/* Set tupleTid to invalid */
-		ItemPointerSetInvalid(&minipageInfo->tupleTid);
-
-		/*
-		 * Clear out the entries.
-		 */
-		MemSet(minipageInfo->minipage->entry, 0,
-			   minipageInfo->numMinipageEntries * sizeof(MinipageEntry));
-		minipageInfo->numMinipageEntries = 0;
-	}
-	
-	Assert(minipageInfo->numMinipageEntries < (uint32)gp_blockdirectory_minipage_size);
-
-	entry = &(minipageInfo->minipage->entry[minipageInfo->numMinipageEntries]);
-	entry->firstRowNum = firstRowNum;
-	entry->fileOffset = fileOffset;
-	entry->rowCount = rowCount;
-	
-	minipageInfo->numMinipageEntries++;
-	
-	if(Debug_appendonly_print_blockdirectory)
-	{
-		ereport(LOG,
-				(errmsg("Append-only block directory insert entry: "
-						"(firstRowNum, columnGroupNo, fileOffset, rowCount) = (" INT64_FORMAT
-						", %d, " INT64_FORMAT ", " INT64_FORMAT ") at index %d",
-						entry->firstRowNum, columnGroupNo, entry->fileOffset, entry->rowCount,
-						minipageInfo->numMinipageEntries - 1)));
-	}
-
-	return true;
-}
-
-/*
- * init_scankeys
- *
- * Initialize the scan keys.
- */
-static void
-init_scankeys(TupleDesc tupleDesc,
-			  int nkeys, ScanKey scanKeys,
-			  StrategyNumber *strategyNumbers)
-{
-	int keyNo;
-
-	Assert(nkeys <= tupleDesc->natts);
-	
-	for (keyNo = 0; keyNo < nkeys; keyNo ++)
-	{
-		ScanKey scanKey = (ScanKey)(((char *)scanKeys) + 
-									keyNo * sizeof(ScanKeyData));
-		RegProcedure opfuncid;
-		StrategyNumber strategyNumber = strategyNumbers[keyNo];
-
-		Assert(strategyNumber <= BTMaxStrategyNumber &&
-			   strategyNumber != InvalidStrategy);
-		
-		if (strategyNumber == BTEqualStrategyNumber)
-		{
-			opfuncid = equality_oper_funcid(tupleDesc->attrs[keyNo]->atttypid);
-			ScanKeyEntryInitialize(scanKey,
-								   0, /* sk_flag */
-								   keyNo + 1, /* attribute number to scan */
-								   BTEqualStrategyNumber, /* strategy */
-								   InvalidOid, /* strategy subtype */
-								   opfuncid, /* reg proc to use */
-								   0 /* constant */
-				);
-		}
-		else
-		{
-			Oid gtOid, leOid;
-			
-			gtOid = reverse_ordering_oper_opid(tupleDesc->attrs[keyNo]->atttypid);
-			leOid = get_negator(gtOid);
-			opfuncid = get_opcode(leOid);
-			
-			ScanKeyEntryInitialize(scanKey,
-								   0, /* sk_flag */
-								   keyNo + 1, /* attribute number to scan */
-								   strategyNumber, /* strategy */
-								   InvalidOid, /* strategy subtype */
-								   opfuncid, /* reg proc to use */
-								   0 /* constant */
-				);
-		}
-	}
-}
-
-/*
- * copy_out_minipage
- *
- * Copy out the minipage content from a deformed tuple.
- */
-static inline void
-copy_out_minipage(MinipagePerColumnGroup *minipageInfo,
-				  Datum minipage_value,
-				  bool minipage_isnull)
-{
-	struct varlena *value;
-	struct varlena *detoast_value;
-
-	Assert(!minipage_isnull);
-
-	value = (struct varlena *)
-		DatumGetPointer(minipage_value);
-	detoast_value = pg_detoast_datum(value);
-	Assert( VARSIZE(detoast_value) <= minipage_size(NUM_MINIPAGE_ENTRIES));
-
-	memcpy(minipageInfo->minipage, detoast_value, VARSIZE(detoast_value));
-	if (detoast_value != value)
-		pfree(detoast_value);
-	
-	Assert(minipageInfo->minipage->nEntry <= NUM_MINIPAGE_ENTRIES);
-	
-	minipageInfo->numMinipageEntries = minipageInfo->minipage->nEntry;
-}
-
-
-/*
- * extract_minipage
- *
- * Extract the minipage info from the given tuple. The tupleTid
- * is also set here.
- */
-static void
-extract_minipage(AppendOnlyBlockDirectory *blockDirectory,
-				 HeapTuple tuple,
-				 TupleDesc tupleDesc,
-				 int columnGroupNo)
-{
-	Datum *values = blockDirectory->values;
-	bool *nulls = blockDirectory->nulls;
-	MinipagePerColumnGroup *minipageInfo =
-		&blockDirectory->minipages[columnGroupNo];
-	FileSegInfo *fsInfo = blockDirectory->currentSegmentFileInfo;
-	int64 eof;
-	int start, end, mid=0;
-	bool found = false;
-	
-	heap_deform_tuple(tuple, tupleDesc, values, nulls);
-
-	Assert(blockDirectory->currentSegmentFileNum ==
-		   DatumGetInt32(values[Anum_pg_aoblkdir_segno - 1]));
-
-	/*
-	 * Copy out the minipage
-	 */
-	copy_out_minipage(minipageInfo,
-					  values[Anum_pg_aoblkdir_minipage - 1],
-					  nulls[Anum_pg_aoblkdir_minipage - 1]);
-
-	ItemPointerCopy(&tuple->t_self, &minipageInfo->tupleTid);
-	
-	/*
-	 * When crashes during inserts, or cancellation during inserts,
-	 * there are out-of-date minipage entries in the block directory.
-	 * We reset those entries here.
-	 */
-	Assert(fsInfo != NULL);
-	eof = fsInfo->eof;
-
-	start = 0;
-	end = minipageInfo->numMinipageEntries - 1;
-	while (start <= end)
-	{
-		mid = (end - start + 1) / 2 + start;
-		if (minipageInfo->minipage->entry[mid].fileOffset > eof)
-			end = mid - 1;
-		else if (minipageInfo->minipage->entry[mid].fileOffset < eof)
-			start = mid + 1;
-		else
-		{
-			found = true;
-			break;
-		}
-	}
-
-	minipageInfo->numMinipageEntries = 0;
-	if (found)
-		minipageInfo->numMinipageEntries = mid;
-	else if (start > 0)
-	{
-		minipageInfo->numMinipageEntries = start;
-		Assert(minipageInfo->minipage->entry[start - 1].fileOffset < eof);
-	}
-}
-
-/*
- * load_last_minipage
- *
- * Search through the block directory btree to find the last row that
- * contains the last minipage.
- */
-static void
-load_last_minipage(AppendOnlyBlockDirectory *blockDirectory,
-				   int64 lastSequence,
-				   int columnGroupNo)
-{
-	Relation blkdirRel = blockDirectory->blkdirRel;
-	Relation blkdirIdx = blockDirectory->blkdirIdx;
-	TupleDesc idxTupleDesc;
-	TupleDesc heapTupleDesc;
-	IndexScanDesc idxScanDesc;
-	HeapTuple tuple = NULL;
-	MemoryContext oldcxt;
-	int numScanKeys = blockDirectory->numScanKeys;
-	ScanKey scanKeys = blockDirectory->scanKeys;
-	
-#ifdef USE_ASSERT_CHECKING
-	StrategyNumber *strategyNumbers = blockDirectory->strategyNumbers;
-#endif /* USE_ASSERT_CHECKING */
-	
-	Assert(blockDirectory->aoRel != NULL);
-	Assert(blockDirectory->blkdirRel != NULL);
-	Assert(blockDirectory->blkdirIdx != NULL);
-
-	oldcxt = MemoryContextSwitchTo(blockDirectory->memoryContext);
-	
-	heapTupleDesc = RelationGetDescr(blkdirRel);
-	idxTupleDesc = RelationGetDescr(blkdirIdx);
-
-	Assert(numScanKeys == 3);
-	Assert(blockDirectory->currentSegmentFileInfo != NULL);
-
-	/* Setup the scan keys for the scan. */
-	Assert(scanKeys != NULL);
-	Assert(strategyNumbers != NULL);
-	if (lastSequence == 0)
-		lastSequence = 1;
-	
-	scanKeys[0].sk_argument =
-		Int32GetDatum(blockDirectory->currentSegmentFileNum);
-	scanKeys[1].sk_argument = Int32GetDatum(columnGroupNo);
-	scanKeys[2].sk_argument = Int64GetDatum(lastSequence);
-
-	/*
-	 * Search the btree to find the entry in the block directory
-	 * that contains the last minipage.
-	 */
-	idxScanDesc = index_beginscan(blkdirRel, blkdirIdx,
-								  blockDirectory->appendOnlyMetaDataSnapshot,
-								  numScanKeys, scanKeys);
-	
-	tuple = index_getnext(idxScanDesc, BackwardScanDirection);
-	if (tuple != NULL)
-	{
-		extract_minipage(blockDirectory,
-						 tuple,
-						 heapTupleDesc,
-						 columnGroupNo);
-	}
-	
-	index_endscan(idxScanDesc);
-
-	MemoryContextSwitchTo(oldcxt);
-
-	if (Debug_appendonly_print_blockdirectory)
-		ereport(LOG,
-				(errmsg("Append-only block directory load last minipage: "
-						"(columnGroupNo, lastSequence, nEntries) = (%d, " INT64_FORMAT ", %u)",
-						columnGroupNo, lastSequence,
-						blockDirectory->minipages[columnGroupNo].numMinipageEntries)));
-	
-}
-
-/*
- * find_minipage_entry
- *
- * Find the minipage entry that covers the given rowNum.
- * If such an entry does not exists, -1 is returned. Otherwise
- * the index to such an entry in the minipage array is returned.
- */
-static int
-find_minipage_entry(Minipage *minipage,
-					uint32 numEntries,
-					int64 rowNum)
-{
-	int start_no, end_no;
-	int entry_no;
-	MinipageEntry *entry;
-	
-	start_no = 0;
-	end_no = numEntries - 1;
-	while (start_no <= end_no)
-	{
-		entry_no = start_no + (end_no - start_no + 1) / 2;
-		Assert(entry_no >= start_no && entry_no <= end_no);
-		
-		entry = &(minipage->entry[entry_no]);
-		
-		Assert(entry->firstRowNum > 0);
-		Assert(entry->rowCount > 0);
-		
-		if (entry->firstRowNum <= rowNum &&
-			entry->firstRowNum + entry->rowCount > rowNum)
-			break;
-		else if (entry->firstRowNum > rowNum)
-		{
-			end_no = entry_no - 1;
-		}
-		else
-		{
-			start_no = entry_no + 1;
-		}
-	}
-	
-	if (start_no <= end_no)
-		return entry_no;
-	else
-		return -1;
-}
-
-/*
- * write_minipage
- *
- * Write the in-memory minipage to the block directory relation.
- */
-static void
-write_minipage(AppendOnlyBlockDirectory *blockDirectory,
-			   int columnGroupNo)
-{
-	HeapTuple tuple;
-	MemoryContext oldcxt;
-	Datum *values = blockDirectory->values;
-	bool *nulls = blockDirectory->nulls;
-	Relation blkdirRel = blockDirectory->blkdirRel;
-	TupleDesc heapTupleDesc = RelationGetDescr(blkdirRel);
-	MinipagePerColumnGroup *minipageInfo =
-		&blockDirectory->minipages[columnGroupNo];
-	
-	Assert(minipageInfo->numMinipageEntries > 0);
-
-	oldcxt = MemoryContextSwitchTo(blockDirectory->memoryContext);
-	
-	Assert(blkdirRel != NULL);
-	
-	values[Anum_pg_aoblkdir_segno - 1] =
-		Int32GetDatum(blockDirectory->currentSegmentFileNum);
-	nulls[Anum_pg_aoblkdir_segno - 1] = false;
-
-	values[Anum_pg_aoblkdir_columngroupno - 1] =
-		Int32GetDatum(columnGroupNo);
-	nulls[Anum_pg_aoblkdir_columngroupno - 1] = false;
-
-	values[Anum_pg_aoblkdir_firstrownum - 1] =
-		Int64GetDatum(minipageInfo->minipage->entry[0].firstRowNum);
-	nulls[Anum_pg_aoblkdir_firstrownum - 1] = false;
-
-	SET_VARSIZE(minipageInfo->minipage,
-				minipage_size(minipageInfo->numMinipageEntries));
-	minipageInfo->minipage->nEntry = minipageInfo->numMinipageEntries;
-	values[Anum_pg_aoblkdir_minipage - 1] =
-		PointerGetDatum(minipageInfo->minipage);
-	nulls[Anum_pg_aoblkdir_minipage - 1] = false;
-	
-	tuple = heaptuple_form_to(heapTupleDesc,
-							  values,
-							  nulls,
-							  NULL,
-							  NULL);
-	
-	/*
-	 * Write out the minipage to the block directory relation.
-	 * If this minipage is already in the relation, we update
-	 * the row. Otherwise, a new row is inserted.
-	 */
-	if (ItemPointerIsValid(&minipageInfo->tupleTid))
-	{
-		if (Debug_appendonly_print_blockdirectory)
-			ereport(LOG,
-					(errmsg("Append-only block directory update a minipage: "
-							"(segno, columnGroupNo, nEntries, firstRowNum) = "
-							"(%d, %d, %u, " INT64_FORMAT ")",
-							blockDirectory->currentSegmentFileNum,
-							columnGroupNo, minipageInfo->numMinipageEntries,
-							minipageInfo->minipage->entry[0].firstRowNum)));
-
-		simple_heap_update(blkdirRel, &minipageInfo->tupleTid, tuple);
-	}
-	else
-	{
-		if (Debug_appendonly_print_blockdirectory)
-			ereport(LOG,
-					(errmsg("Append-only block directory insert a minipage: "
-							"(segno, columnGroupNo, nEntries, firstRowNum) = "
-							"(%d, %d, %u, " INT64_FORMAT ")",
-							blockDirectory->currentSegmentFileNum,
-							columnGroupNo, minipageInfo->numMinipageEntries,
-							minipageInfo->minipage->entry[0].firstRowNum)));
-
-		simple_heap_insert(blkdirRel, tuple);
-	}
-	
-	CatalogUpdateIndexes(blkdirRel, tuple);
-	
-	heap_freetuple(tuple);
-	
-	MemoryContextSwitchTo(oldcxt);
-}
-
-
-
-void
-AppendOnlyBlockDirectory_End_forInsert(
-	AppendOnlyBlockDirectory *blockDirectory)
-{
-	int groupNo;
-
-	if (blockDirectory->blkdirRel == NULL ||
-		blockDirectory->blkdirIdx == NULL)
-		return;
-	
-	for (groupNo = 0; groupNo < blockDirectory->numColumnGroups; groupNo++)
-	{
-		MinipagePerColumnGroup *minipageInfo =
-			&blockDirectory->minipages[groupNo];
-		
-		if (minipageInfo->numMinipageEntries > 0)
-		{
-			write_minipage(blockDirectory, groupNo);
-			if (Debug_appendonly_print_blockdirectory)
-				ereport(LOG,
-						(errmsg("Append-only block directory end of insert write minipage: "
-								"(columnGroupNo, nEntries) = (%d, %u)",
-								groupNo, minipageInfo->numMinipageEntries)));
-		}
-		
-		pfree(minipageInfo->minipage);
-	}
-
-	if (Debug_appendonly_print_blockdirectory)
-		ereport(LOG,
-				(errmsg("Append-only block directory end for insert: "
-						"(segno, numColumnGroups)="
-						"(%d, %d)",
-						blockDirectory->currentSegmentFileNum,
-						blockDirectory->numColumnGroups)));
-
-	pfree(blockDirectory->values);
-	pfree(blockDirectory->nulls);
-	pfree(blockDirectory->minipages);
-	pfree(blockDirectory->scanKeys);
-	pfree(blockDirectory->strategyNumbers);
-	
-	index_close(blockDirectory->blkdirIdx, RowExclusiveLock);
-	heap_close(blockDirectory->blkdirRel, RowExclusiveLock);
-	
-	MemoryContextDelete(blockDirectory->memoryContext);
-}
-
-void
-AppendOnlyBlockDirectory_End_forSearch(
-	AppendOnlyBlockDirectory *blockDirectory)
-{
-	int groupNo;
-	
-	if (blockDirectory->blkdirRel == NULL ||
-		blockDirectory->blkdirIdx == NULL)
-		return;
-
-	for (groupNo = 0; groupNo < blockDirectory->numColumnGroups; groupNo++)
-	{
-		pfree(blockDirectory->minipages[groupNo].minipage);
-	}
-
-	if (Debug_appendonly_print_blockdirectory)
-		ereport(LOG,
-				(errmsg("Append-only block directory end for search: "
-						"(totalSegfiles, numColumnGroups )="
-						"(%d, %d)",
-						blockDirectory->totalSegfiles,
-						blockDirectory->numColumnGroups)));
-
-	pfree(blockDirectory->values);
-	pfree(blockDirectory->nulls);
-	pfree(blockDirectory->minipages);
-	pfree(blockDirectory->scanKeys);
-	pfree(blockDirectory->strategyNumbers);
-	
-	index_close(blockDirectory->blkdirIdx, AccessShareLock);
-	heap_close(blockDirectory->blkdirRel, AccessShareLock);
-
-	MemoryContextDelete(blockDirectory->memoryContext);
-}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/access/external/hd_work_mgr.c
----------------------------------------------------------------------
diff --git a/src/backend/access/external/hd_work_mgr.c b/src/backend/access/external/hd_work_mgr.c
index b5bf6b5..c7c3e20 100644
--- a/src/backend/access/external/hd_work_mgr.c
+++ b/src/backend/access/external/hd_work_mgr.c
@@ -242,8 +242,6 @@ static void assign_pxf_port_to_fragments(int remote_rest_port, List *fragments)
 PxfFragmentStatsElem *get_pxf_fragments_statistics(char *uri, Relation rel)
 {
 	ClientContext client_context; /* holds the communication info */
-	char *analyzer = NULL;
-	char *profile = NULL;
 	PxfInputData inputData = {0};
 	PxfFragmentStatsElem *result = NULL;
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/access/heap/heapam.c
----------------------------------------------------------------------
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index b4d4e5c..78427e9 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -50,7 +50,6 @@
 #include "catalog/catalog.h"
 #include "catalog/catquery.h"
 #include "catalog/gp_policy.h"
-#include "catalog/gp_fastsequence.h"
 #include "catalog/namespace.h"
 #include "miscadmin.h"
 #include "pgstat.h"
@@ -4888,42 +4887,6 @@ heap_xlog_insert(XLogRecPtr lsn, XLogRecord *record)
 	HeapTupleHeaderSetCmin(htup, FirstCommandId);
 	htup->t_ctid = xlrec->target.tid;
 
-	/* MPP-16881: if GPDB crashed when CTAS for AO/CO is running, we should "frozen"
-	 * delete left-over entries in gp_fastsequence catalog table when replaying XLOG
-	 * during Pass3 crash recovery.
-	 */
-	if (xlrec->target.node.relNode == FastSequenceRelationId)
-	{
-		Form_gp_fastsequence redo_entry = 
-							(Form_gp_fastsequence) ((char *) htup + htup->t_hoff);
-		HASH_SEQ_STATUS iterateStatus;
-		hash_seq_init(&iterateStatus, pass2RecoveryHashShmem->hash);
-	
-		while (true)
-		{
-			Pass2RecoveryHashEntry_s *entry = 
-									(Pass2RecoveryHashEntry_s *)
-									hash_seq_search(&iterateStatus);
-			
-			if (entry == NULL)
-				break;
-			
-			if (entry->objid == redo_entry->objid)
-			{
-				htup->t_infomask &= 0;
-				htup->t_infomask |= HEAP_XMIN_INVALID;
-				HeapTupleHeaderSetXmin(htup, 0);
-				HeapTupleHeaderSetXmax(htup, FrozenTransactionId);
-				hash_seq_term(&iterateStatus); 
-				if (Debug_persistent_print)
-					elog(LOG, "frozen deleting gp_fastsequence entry"
-						 "for aborted AO insert transaction on objid %d",
-						 redo_entry->objid);
-				break;
-			}
-		}
-	}
-
 	offnum = PageAddItem(page, (Item) htup, newlen, offnum,
 						 LP_USED | OverwritePageMode);
 	if (offnum == InvalidOffsetNumber)

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/access/index/caql.files
----------------------------------------------------------------------
diff --git a/src/backend/access/index/caql.files b/src/backend/access/index/caql.files
index 3edb42c..412aa24 100644
--- a/src/backend/access/index/caql.files
+++ b/src/backend/access/index/caql.files
@@ -4,7 +4,6 @@
 ../../../..//src/backend/catalog/caql/caqlanalyze.c
 ../../../..//src/backend/catalog/caql/test/caqlanalyze_test.c
 ../../../..//src/backend/catalog/dependency.c
-../../../..//src/backend/catalog/gp_fastsequence.c
 ../../../..//src/backend/catalog/heap.c
 ../../../..//src/backend/catalog/index.c
 ../../../..//src/backend/catalog/namespace.c

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/access/index/caqlfilemap.json
----------------------------------------------------------------------
diff --git a/src/backend/access/index/caqlfilemap.json b/src/backend/access/index/caqlfilemap.json
index 8ff325c..7e9e089 100644
--- a/src/backend/access/index/caqlfilemap.json
+++ b/src/backend/access/index/caqlfilemap.json
@@ -152,9 +152,6 @@
    "functions.c" : [
       "pg_proc"
    ],
-   "gp_fastsequence.c" : [
-      "gp_fastsequence"
-   ],
    "heap.c" : [
       "pg_attrdef",
       "pg_attribute",

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/access/nbtree/nbtinsert.c
----------------------------------------------------------------------
diff --git a/src/backend/access/nbtree/nbtinsert.c b/src/backend/access/nbtree/nbtinsert.c
index 24f7bd9..92ea375 100644
--- a/src/backend/access/nbtree/nbtinsert.c
+++ b/src/backend/access/nbtree/nbtinsert.c
@@ -162,51 +162,6 @@ top:
 }
 
 /*
- * _bt_ao_check_unique -- Check for violation of unique index constraint
- *  for AO tables.
- *
- * Returns InvalidTransactionId if there is no conflict, else an xact ID
- * we must wait for to see if it commits a conflicting tuple.	If an actual
- * conflict is detected, no return --- just ereport().
- */
-static TransactionId
-_bt_ao_check_unique(Relation rel, Relation aoRel, ItemPointer tid)
-{
-	TransactionId xwait = InvalidTransactionId;
-	
-	Assert(RelationIsAoRows(aoRel));
-	
-	if (RelationIsAoRows(aoRel))
-	{
-		AppendOnlyFetchDesc aoFetchDesc =
-			appendonly_fetch_init(aoRel, SnapshotDirty);
-
-		if (appendonly_fetch(aoFetchDesc, (AOTupleId*)tid, NULL))
-		{
-			xwait =
-				(TransactionIdIsValid(SnapshotDirty->xmin)) ?
-				SnapshotDirty->xmin : SnapshotDirty->xmax;
-
-			/*
-			 * If this tuple is not being updated by other transaction,
-			 * then we have a definite conflict. Ereport here.
-			 */
-			if (!TransactionIdIsValid(xwait))
-				ereport(ERROR,
-						(errcode(ERRCODE_UNIQUE_VIOLATION),
-						 errmsg("duplicate key violates unique constraint \"%s\"",
-								RelationGetRelationName(rel)),
-						 errOmitLocation(true)));
-		}
-	
-		appendonly_fetch_finish(aoFetchDesc);
-		pfree(aoFetchDesc);
-	}
-	
-	return xwait;
-}
-
-/*
  *	_bt_check_unique() -- Check for violation of unique index constraint
  *
  * Returns InvalidTransactionId if there is no conflict, else an xact ID
@@ -288,16 +243,6 @@ _bt_check_unique(Relation rel, IndexTuple itup, Relation heapRel,
 				 * If the parent relation is an AO/CO table, we have to find out
 				 * if this tuple is actually in the table.
 				 */
-				if (RelationIsAoRows(heapRel))
-				{
-					TransactionId xwait =
-						_bt_ao_check_unique(rel, heapRel, &curitup->t_tid);
-
-					if (TransactionIdIsValid(xwait))
-						return xwait;
-				}
-					
-				else
 				{
 					htup.t_self = curitup->t_tid;
 					if (heap_fetch(heapRel, SnapshotDirty, &htup, &hbuffer,

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/access/parquet/parquetsegfiles.c
----------------------------------------------------------------------
diff --git a/src/backend/access/parquet/parquetsegfiles.c b/src/backend/access/parquet/parquetsegfiles.c
index cf8d140..3404a9e 100644
--- a/src/backend/access/parquet/parquetsegfiles.c
+++ b/src/backend/access/parquet/parquetsegfiles.c
@@ -35,7 +35,6 @@
 #include "catalog/dependency.h"
 #include "catalog/indexing.h"
 #include "catalog/namespace.h"
-#include "catalog/gp_fastsequence.h"
 #include "catalog/pg_appendonly.h"
 #include "cdb/cdbvars.h"
 #include "executor/spi.h"
@@ -191,13 +190,9 @@ void InsertInitialParquetSegnoEntry(AppendOnlyEntry *aoEntry, int segno) {
 	int natts = 0;
 	bool *nulls;
 	Datum *values;
-	ItemPointerData tid;
 
 	Assert(aoEntry != NULL);
 
-	InsertFastSequenceEntry(aoEntry->segrelid, (int64) segno, 0,
-			&tid);
-
 	if (segno == 0)
 	{
 		return;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/bootstrap/bootparse.y
----------------------------------------------------------------------
diff --git a/src/backend/bootstrap/bootparse.y b/src/backend/bootstrap/bootparse.y
index 4e9ff00..74f8471 100755
--- a/src/backend/bootstrap/bootparse.y
+++ b/src/backend/bootstrap/bootparse.y
@@ -47,7 +47,6 @@
 #include "catalog/gp_configuration.h"
 #include "catalog/gp_persistent.h"
 #include "catalog/gp_global_sequence.h"
-#include "catalog/gp_fastsequence.h"
 #include "catalog/gp_san_config.h"
 #include "catalog/gp_segment_config.h"
 #include "catalog/gp_verification_history.h"
@@ -317,10 +316,6 @@ Boot_CreateStmt:
 							case GpFaultStrategyRelationId:
 								typid = GP_FAULT_STRATEGY_RELTYPE_OID;
 								break;
-/* relation id: 5043 - gp_fastsequence 20101104 */
-							case FastSequenceRelationId:
-								typid = GP_FASTSEQUENCE_RELTYPE_OID;
-								break;
 /* relation id: 5096 - gp_global_sequence 20101104 */
 							case GpGlobalSequenceRelationId:
 								typid = GP_GLOBAL_SEQUENCE_RELTYPE_OID;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/catalog/Makefile
----------------------------------------------------------------------
diff --git a/src/backend/catalog/Makefile b/src/backend/catalog/Makefile
index 54d26cb..be3e5da 100644
--- a/src/backend/catalog/Makefile
+++ b/src/backend/catalog/Makefile
@@ -19,7 +19,7 @@ OBJS = catalog.o dependency.o heap.o index.o indexing.o namespace.o aclchk.o \
        pg_aggregate.o pg_appendonly.o pg_constraint.o pg_conversion.o pg_depend.o \
        pg_exttable.o pg_extprotocol.o pg_filesystem.o pg_largeobject.o pg_namespace.o \
        pg_operator.o pg_proc.o pg_proc_callback.o pg_shdepend.o \
-       pg_type.o toasting.o aoseg.o aoblkdir.o gp_fastsequence.o \
+       pg_type.o toasting.o aoseg.o \
 	pg_attribute_encoding.o pg_compression.o $(QUICKLZ_COMPRESSION)
 
 SUBDIRS = caql core hcatalog
@@ -113,7 +113,7 @@ POSTGRES_BKI_SRCS := $(addprefix $(top_srcdir)/src/include/catalog/,\
 	gp_verification_history.h \
 	pg_window.h \
 	pg_exttable.h pg_appendonly.h pg_appendonly_alter_column.h gp_master_mirroring.h aoseg.h \
-	gp_fastsequence.h pg_extprotocol.h \
+	pg_extprotocol.h \
 	pg_partition.h pg_partition_rule.h pg_filespace.h pg_filespace_entry.h \
 	gp_global_sequence.h gp_persistent.h \
 	$(TIDYCAT_BKI_SRCS) \

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/catalog/aoblkdir.c
----------------------------------------------------------------------
diff --git a/src/backend/catalog/aoblkdir.c b/src/backend/catalog/aoblkdir.c
deleted file mode 100644
index 23863e0..0000000
--- a/src/backend/catalog/aoblkdir.c
+++ /dev/null
@@ -1,264 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*-------------------------------------------------------------------------
- *
- * aoblkdir.c
- *   This file contains routines to support creation of append-only block
- *   directory tables. This file is identical in functionality to aoseg.c
- *   that exists in the same directory.
- *
- *-------------------------------------------------------------------------
- */
-#include "postgres.h"
-
-#include "access/heapam.h"
-#include "access/xact.h"
-#include "catalog/dependency.h"
-#include "catalog/heap.h"
-#include "catalog/index.h"
-#include "catalog/indexing.h"
-#include "catalog/pg_namespace.h"
-#include "catalog/pg_opclass.h"
-#include "catalog/pg_type.h"
-#include "commands/tablespace.h"
-#include "miscadmin.h"
-#include "nodes/makefuncs.h"
-#include "utils/builtins.h"
-#include "utils/syscache.h"
-#include "catalog/aoblkdir.h"
-
-
-static bool
-create_aoblkdir_table(Relation rel, Oid aoblkdirOid,
-					  Oid aoblkdirIndexOid, Oid *comptypeOid);
-
-/*
- * AlterTableCreateAoBlkdirTable
- *    If the table needs an AO block directory table, and doesn't already
- *    have one, then create an aoblkdir table.
- * 
- * We expect the caller to have verified that the relation is AO table and have
- * already done any necessary permission checks.  Callers expect this function
- * to end with CommandCounterIncrement if it makes any changes.
- */
-void
-AlterTableCreateAoBlkdirTable(Oid relOid)
-{
-	Relation rel;
-	
-	/*
-	 * We've well and truly locked the table if we need to, so don't now.
-	 * This is useful for avoiding lock table overflows with partitioning.
-	 */
-	rel = heap_open(relOid, NoLock);
-
-	/* create_aoblkdir_table does all the work */
-	(void) create_aoblkdir_table(rel, InvalidOid, InvalidOid, NULL);
-
-	heap_close(rel, NoLock);
-}
-
-void
-AlterTableCreateAoBlkdirTableWithOid(Oid relOid, Oid newOid, Oid newIndexOid,
-									 Oid * comptypeOid, bool is_part_child)
-{
-	Relation	rel;
-
-	/*
-	 * Grab an exclusive lock on the target table, which we will NOT release
-	 * until end of transaction.  (This is probably redundant in all present
-	 * uses...)
-	 */
-	if (is_part_child)
-		rel = heap_open(relOid, NoLock);
-	else
-		rel = heap_open(relOid, AccessExclusiveLock);
-
-	/* create_aoblkdir_table does all the work */
-	(void) create_aoblkdir_table(rel, newOid, newIndexOid, comptypeOid);
-
-	heap_close(rel, NoLock);
-}
-
-/*
- * create_aoblkdir_table
- *
- * rel is already opened and exclusive-locked.
- * comptypeOid is InvalidOid.
- */
-static bool
-create_aoblkdir_table(Relation rel, Oid aoblkdirOid,
-					  Oid aoblkdirIndexOid, Oid *comptypeOid)
-{
-	Oid relOid = RelationGetRelid(rel);
-	Oid	aoblkdir_relid;
-	Oid	aoblkdir_idxid;
-	bool shared_relation = rel->rd_rel->relisshared;
-	char aoblkdir_relname[NAMEDATALEN];
-	char aoblkdir_idxname[NAMEDATALEN];
-	TupleDesc	tupdesc;
-	IndexInfo  *indexInfo;
-	Oid			classObjectId[3];
-	ObjectAddress baseobject;
-	ObjectAddress aoblkdirobject;
-	Oid			tablespaceOid = ChooseTablespaceForLimitedObject(rel->rd_rel->reltablespace);
-
-	if (!RelationIsAoRows(rel))
-		return false;
-	
-	/*
-	 * We cannot allow creating a block directory for a shared relation
-	 * after initdb (because there's no way to let other databases know
-	 * this block directory.
-	 */
-	if (shared_relation && !IsBootstrapProcessingMode())
-		ereport(ERROR,
-				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-				 errmsg("shared tables cannot have block directory after initdb")));
-
-	GetAppendOnlyEntryAuxOids(relOid, SnapshotNow, NULL,NULL, &aoblkdir_relid, &aoblkdir_idxid);
-
-	/*
-	 * Does it have a block directory?
-	 */
-	if (aoblkdir_relid != InvalidOid)
-	{
-		return false;
-	}
-
-	snprintf(aoblkdir_relname, sizeof(aoblkdir_relname),
-			 "pg_aoblkdir_%u", relOid);
-	snprintf(aoblkdir_idxname, sizeof(aoblkdir_idxname),
-			 "pg_aoblkdir_%u_index", relOid);
-	
-	/* Create a tuple descriptor */
-	tupdesc = CreateTemplateTupleDesc(4, false);
-	TupleDescInitEntry(tupdesc, (AttrNumber) 1,
-					   "segno",
-					   INT4OID,
-					   -1, 0);
-	TupleDescInitEntry(tupdesc, (AttrNumber) 2,
-					   "columngroup_no",
-					   INT4OID,
-					   -1, 0);
-	TupleDescInitEntry(tupdesc, (AttrNumber) 3,
-					   "first_row_no",
-					   INT8OID,
-					   -1, 0);
-	TupleDescInitEntry(tupdesc, (AttrNumber) 4,
-					   "minipage",
-					   VARBITOID,
-					   -1, 0);
-	/*
-	 * We don't want any toast columns here.
-	 */
-	tupdesc->attrs[0]->attstorage = 'p';
-	tupdesc->attrs[1]->attstorage = 'p';
-	tupdesc->attrs[2]->attstorage = 'p';
-	tupdesc->attrs[2]->attstorage = 'p';
-
-	/*
-	 * We place aoblkdir relation in the pg_aoseg namespace
-	 * even if its master relation is a temp table. There cannot be
-	 * any naming collision, and the aoblkdir relation will be
-	 * destroyed when its master is, so there is no need to handle
-	 * the aoblkdir relation as temp.
-	 */
-	aoblkdir_relid = heap_create_with_catalog(aoblkdir_relname,
-											  PG_AOSEGMENT_NAMESPACE,
-											  tablespaceOid,
-											  aoblkdirOid,
-											  rel->rd_rel->relowner,
-											  tupdesc,
-											  /* relam */ InvalidOid,
-											  RELKIND_AOBLOCKDIR,
-											  RELSTORAGE_HEAP,
-											  shared_relation,
-											  true,
-											  /* bufferPoolBulkLoad */ false,
-											  0,
-											  ONCOMMIT_NOOP,
-											  NULL, /* GP Policy */
-											  (Datum) 0,
-											  true,
-											  comptypeOid,
-						 					  /* persistentTid */ NULL,
-						 					  /* persistentSerialNum */ NULL);
-	
-	/* Make this table visible, else index creation will fail */
-	CommandCounterIncrement();
-	
-	/*
-	 * Create index on segno, first_row_no.
-	 */
-	indexInfo = makeNode(IndexInfo);
-	indexInfo->ii_NumIndexAttrs = 3;
-	indexInfo->ii_KeyAttrNumbers[0] = 1;
-	indexInfo->ii_KeyAttrNumbers[1] = 2;
-	indexInfo->ii_KeyAttrNumbers[2] = 3;
-	indexInfo->ii_Expressions = NIL;
-	indexInfo->ii_ExpressionsState = NIL;
-	indexInfo->ii_Predicate = NIL;
-	indexInfo->ii_PredicateState = NIL;
-	indexInfo->ii_Unique = false;
-	indexInfo->ii_Concurrent = false;
-	
-	classObjectId[0] = INT4_BTREE_OPS_OID;
-	classObjectId[1] = INT4_BTREE_OPS_OID;
-	classObjectId[2] = INT8_BTREE_OPS_OID;
-
-	aoblkdir_idxid = index_create(aoblkdirOid, aoblkdir_idxname, aoblkdirIndexOid,
-								  indexInfo,
-								  BTREE_AM_OID,
-								  tablespaceOid,
-								  classObjectId, (Datum) 0,
-								  true, false, (Oid *) NULL, true, false, false, NULL);
-	
-	/* Unlock target table -- no one can see it */
-	UnlockRelationOid(aoblkdirOid, ShareLock);
-	/* Unlock the index -- no one can see it anyway */
-	UnlockRelationOid(aoblkdirIndexOid, AccessExclusiveLock);
-
-	/*
-	 * Store the aoblkdir table's OID in the parent relation's pg_appendonly row.
-	 */
-	UpdateAppendOnlyEntryAuxOids(relOid, InvalidOid, InvalidOid,
-								 aoblkdir_relid, aoblkdir_idxid);
-
-	/*
-	 * Register dependency from the aoseg table to the master, so that the
-	 * aoseg table will be deleted if the master is.
-	 */
-	baseobject.classId = RelationRelationId;
-	baseobject.objectId = relOid;
-	baseobject.objectSubId = 0;
-	aoblkdirobject.classId = RelationRelationId;
-	aoblkdirobject.objectId = aoblkdirOid;
-	aoblkdirobject.objectSubId = 0;
-
-	recordDependencyOn(&aoblkdirobject, &baseobject, DEPENDENCY_INTERNAL);
-
-	/*
-	 * Make changes visible
-	 */
-	CommandCounterIncrement();
-
-	return true;
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/catalog/caql/Makefile
----------------------------------------------------------------------
diff --git a/src/backend/catalog/caql/Makefile b/src/backend/catalog/caql/Makefile
index 77bcb5c..67c1876 100644
--- a/src/backend/catalog/caql/Makefile
+++ b/src/backend/catalog/caql/Makefile
@@ -37,7 +37,6 @@ CAQL_CQL_SRCS := $(addprefix $(top_srcdir)/src/backend/,\
 	access/heap/heapam.c \
 	catalog/aclchk.c \
 	catalog/dependency.c \
-	catalog/gp_fastsequence.c \
 	catalog/heap.c \
 	catalog/index.c \
 	catalog/namespace.c \

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/catalog/core/catcoregen.py
----------------------------------------------------------------------
diff --git a/src/backend/catalog/core/catcoregen.py b/src/backend/catalog/core/catcoregen.py
index 1c725fe..16c0f31 100644
--- a/src/backend/catalog/core/catcoregen.py
+++ b/src/backend/catalog/core/catcoregen.py
@@ -420,7 +420,6 @@ CatCoreTableTemplate = """
 #include "catalog/catcore.h"
 #include "catalog/catalog.h"
 #include "catalog/gp_configuration.h"
-#include "catalog/gp_fastsequence.h"
 #include "catalog/gp_id.h"
 #include "catalog/gp_master_mirroring.h"
 #include "catalog/gp_policy.h"



[2/3] incubator-hawq git commit: HAWQ-258. Remove fast sequence

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/catalog/gp_fastsequence.c
----------------------------------------------------------------------
diff --git a/src/backend/catalog/gp_fastsequence.c b/src/backend/catalog/gp_fastsequence.c
deleted file mode 100644
index 2284aa7..0000000
--- a/src/backend/catalog/gp_fastsequence.c
+++ /dev/null
@@ -1,416 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*-------------------------------------------------------------------------
- *
- * gp_fastsequence.c
- *    routines to maintain a light-weight sequence table.
- *
- *-------------------------------------------------------------------------
- */
-#include "postgres.h"
-
-#include "catalog/catquery.h"
-#include "catalog/gp_fastsequence.h"
-#include "catalog/indexing.h"
-#include "utils/relcache.h"
-#include "utils/fmgroids.h"
-#include "access/genam.h"
-#include "access/htup.h"
-#include "access/heapam.h"
-#include "cdb/cdbvars.h"
-
-static void update_fastsequence(
-	Relation gp_fastsequence_rel,
-	HeapTuple oldTuple,
-	TupleDesc tupleDesc,
-	Oid objid,
-	int64 objmod,
-	int64 newLastSequence,
-	ItemPointer tid);
-
-/*
- * InsertFastSequenceEntry
- *
- * Insert a new fast sequence entry for a given object. If the given
- * object already exists in the table, this function replaces the old
- * entry with a fresh initial value.
- *
- * The tid for the new entry is returned.
- */
-void
-InsertFastSequenceEntry(Oid objid, int64 objmod, int64 lastSequence,
-		ItemPointer tid)
-{
-	Relation gp_fastsequence_rel;
-	TupleDesc tupleDesc;
-	int natts = 0;
-	Datum *values;
-	bool *nulls;
-	HeapTuple tuple = NULL;
-	cqContext	 cqc;
-	
-	/*
-	 * Open and lock the gp_fastsequence catalog table.
-	 */
-	gp_fastsequence_rel = heap_open(FastSequenceRelationId, RowExclusiveLock);
-	tupleDesc = RelationGetDescr(gp_fastsequence_rel);
-	
-	tuple = caql_getfirst(
-			caql_addrel(cqclr(&cqc), gp_fastsequence_rel),
-			cql("SELECT * FROM gp_fastsequence "
-				" WHERE objid = :1 "
-				" AND objmod = :2 "
-				" AND contentid = :3 "
-				" FOR UPDATE ",
-				ObjectIdGetDatum(objid),
-				Int64GetDatum(objmod),
-				Int32GetDatum(-1)));
-
-	if (tuple == NULL)
-	{
-		natts = tupleDesc->natts;
-		values = palloc0(sizeof(Datum) * natts);
-		nulls = palloc0(sizeof(bool) * natts);
-	
-		values[Anum_gp_fastsequence_objid - 1] = ObjectIdGetDatum(objid);
-		values[Anum_gp_fastsequence_objmod - 1] = Int64GetDatum(objmod);
-		values[Anum_gp_fastsequence_last_sequence - 1] = Int64GetDatum(lastSequence);
-		values[Anum_gp_fastsequence_contentid - 1] = Int32GetDatum(-1);
-
-		tuple = heaptuple_form_to(tupleDesc, values, nulls, NULL, NULL);
-		frozen_heap_insert(gp_fastsequence_rel, tuple);
-		/*
-		 * in hawq, index only exists on master
-		 */
-		if (Gp_role != GP_ROLE_EXECUTE)
-			CatalogUpdateIndexes(gp_fastsequence_rel, tuple);
-	
-		ItemPointerCopy(&tuple->t_self, tid);
-
-		heap_freetuple(tuple);
-		pfree(values);
-		pfree(nulls);
-	}
-	else
-	{
-		update_fastsequence(gp_fastsequence_rel,
-							tuple,
-							tupleDesc,
-							objid,
-							objmod,
-							lastSequence,
-							tid);
-	}
-
-	/*
-	 * Since the tid for this row may be used later in this transaction, 
-	 * we keep the lock until the end of the transaction.
-	 */
-	heap_close(gp_fastsequence_rel, NoLock);
-}
-
-/*
- * update_fastsequnece -- update the fast sequence number for (objid, objmod).
- *
- * If such an entry exists in the table, it is provided in oldTuple. This tuple
- * is updated with the new value. Otherwise, a new tuple is inserted into the
- * table.
- *
- * The tuple id value for the entry is copied out to 'tid'.
- *
- * NOTE: in hawq, we do not add new tuple in QE, it should be dispatched
- * from master, and only update is allowed on QE.
- */
-static void
-update_fastsequence(Relation gp_fastsequence_rel,
-					HeapTuple oldTuple,
-					TupleDesc tupleDesc,
-					Oid objid,
-					int64 objmod,
-					int64 newLastSequence,
-					ItemPointer tid)
-{
-	Datum *values;
-	bool *nulls;
-	HeapTuple newTuple;
-
-	values = palloc0(sizeof(Datum) * tupleDesc->natts);
-	nulls = palloc0(sizeof(bool) * tupleDesc->natts);
-
-	/*
-	 * If such a tuple does not exist, insert a new one.
-	 */
-	if (oldTuple == NULL)
-	{
-		Assert(Gp_role != GP_ROLE_EXECUTE);
-
-		values[Anum_gp_fastsequence_objid - 1] = ObjectIdGetDatum(objid);
-		values[Anum_gp_fastsequence_objmod - 1] = Int64GetDatum(objmod);
-		values[Anum_gp_fastsequence_last_sequence - 1] = Int64GetDatum(newLastSequence);
-		values[Anum_gp_fastsequence_contentid - 1] = Int32GetDatum(-1);
-
-		newTuple = heaptuple_form_to(tupleDesc, values, nulls, NULL, NULL);
-		frozen_heap_insert(gp_fastsequence_rel, newTuple);
-
-		CatalogUpdateIndexes(gp_fastsequence_rel, newTuple);
-
-		ItemPointerCopy(&newTuple->t_self, tid);
-
-		heap_freetuple(newTuple);
-	}
-
-	else
-	{
-#ifdef USE_ASSERT_CHECKING
-		Oid oldObjid;
-		int64 oldObjmod;
-		bool isNull;
-		
-		oldObjid = heap_getattr(oldTuple, Anum_gp_fastsequence_objid, tupleDesc, &isNull);
-		Assert(!isNull);
-		oldObjmod = heap_getattr(oldTuple, Anum_gp_fastsequence_objmod, tupleDesc, &isNull);
-		Assert(!isNull);
-		Assert(oldObjid == objid && oldObjmod == objmod);
-#endif
-
-		values[Anum_gp_fastsequence_objid - 1] = ObjectIdGetDatum(objid);
-		values[Anum_gp_fastsequence_objmod - 1] = Int64GetDatum(objmod);
-		values[Anum_gp_fastsequence_last_sequence - 1] = Int64GetDatum(newLastSequence);
-		values[Anum_gp_fastsequence_contentid - 1] = Int32GetDatum(-1);
-
-		newTuple = heap_form_tuple(tupleDesc, values, nulls);
-		newTuple->t_data->t_ctid = oldTuple->t_data->t_ctid;
-		newTuple->t_self = oldTuple->t_self;
-		if (tupleDesc->tdhasoid)
-			HeapTupleSetOid(newTuple, HeapTupleGetOid(oldTuple));
-		
-		if (Gp_role != GP_ROLE_EXECUTE)
-		{
-			heap_inplace_update(gp_fastsequence_rel, newTuple);
-		}
-		else
-		{
-			InMemHeapRelation inmemrel = OidGetInMemHeapRelation(
-					gp_fastsequence_rel->rd_id, INMEM_HEAP_MAPPING);
-			if (NULL == inmemrel)
-			{
-				elog(ERROR, "cannot find in-memory table: %s",
-						RelationGetRelationName(gp_fastsequence_rel));
-			}
-
-			InMemHeap_Update(inmemrel, &oldTuple->t_self, newTuple);
-		}
-
-		ItemPointerCopy(&newTuple->t_self, tid);
-
-		heap_freetuple(newTuple);
-	}
-	
-	pfree(values);
-	pfree(nulls);
-}
-
-/*
- * GetFastSequences
- *
- * Get a list of consecutive sequence numbers. The starting sequence
- * number is the maximal value between 'lastsequence' + 1 and minSequence.
- * The length of the list is given.
- *
- * If there is not such an entry for objid in the table, create
- * one here.
- *
- * The existing entry for objid in the table is updated with a new
- * lastsequence value.
- *
- * The tuple id value for this entry is copied out to 'tid'.
- */
-int64 GetFastSequences(Oid objid, int64 objmod,
-					   int64 minSequence, int64 numSequences,
-					   ItemPointer tid)
-{
-	Relation gp_fastsequence_rel;
-	TupleDesc tupleDesc;
-	HeapTuple tuple;
-	int64 firstSequence = minSequence;
-	Datum lastSequenceDatum;
-	int64 newLastSequence;
-
-	Assert(tid != NULL);
-	Assert(Gp_role != GP_ROLE_DISPATCH);
-	
-	gp_fastsequence_rel = heap_open(FastSequenceRelationId, RowExclusiveLock);
-	tupleDesc = RelationGetDescr(gp_fastsequence_rel);
-	
-	cqContext *pcqCtx = caql_beginscan(
-					NULL,
-					cql("SELECT * FROM gp_fastsequence "
-						" WHERE objid = :1 "
-						" AND objmod = :2 "
-						" FOR UPDATE ",
-						ObjectIdGetDatum(objid),
-						Int64GetDatum(objmod)));
-
-	tuple = caql_getnext(pcqCtx);
-
-	if (tuple == NULL)
-	{
-		elog(ERROR, "gp_fastsequence should be dispatched to QE, "
-				"objid = %u, objmod = "INT64_FORMAT", minseq = "INT64_FORMAT", numseq = "INT64_FORMAT".",
-				objid, objmod, minSequence, numSequences);
-
-		newLastSequence = firstSequence + numSequences - 1;
-	}
-	else
-	{
-		bool isNull;
-
-		lastSequenceDatum = heap_getattr(tuple, Anum_gp_fastsequence_last_sequence,
-										tupleDesc, &isNull);
-		
-		if (isNull)
-			ereport(ERROR,
-					(errcode(ERRCODE_UNDEFINED_OBJECT),
-					 errmsg("got an invalid lastsequence number: NULL")));
-		
-		if (DatumGetInt64(lastSequenceDatum) + 1 > firstSequence)
-			firstSequence = DatumGetInt64(lastSequenceDatum) + 1;
-		newLastSequence = firstSequence + numSequences - 1;
-	}
-	
-	update_fastsequence(gp_fastsequence_rel, tuple, tupleDesc,
-						objid, objmod, newLastSequence, tid);
-		
-
-	caql_endscan(pcqCtx);
-
-	/*
-	 * Since the tid for this row may be used later in this transaction, 
-	 * we keep the lock until the end of the transaction.
-	 */
-	heap_close(gp_fastsequence_rel, NoLock);
-
-	return firstSequence;
-}
-
-/*
- * GetFastSequencesByTid
- *
- * Same as GetFastSequences, except that the tuple tid is given, and the tuple id
- * is not valid.
- */
-int64
-GetFastSequencesByTid(ItemPointer tid,
-					  int64 minSequence,
-					  int64 numSequences)
-{
-	Relation gp_fastsequence_rel;
-	TupleDesc tupleDesc;
-	HeapTupleData tuple;
-	Buffer userbuf;
-	bool found = false;
-	Datum lastSequenceDatum;
-	int64 newLastSequence;
-	int64 firstSequence = minSequence;
-	bool isNull;
-	Oid objidDatum;
-	int64 objmodDatum;
-
-	gp_fastsequence_rel = heap_open(FastSequenceRelationId, RowExclusiveLock);
-	tupleDesc = RelationGetDescr(gp_fastsequence_rel);
-
-	Assert(ItemPointerIsValid(tid));
-
-	ItemPointerCopy(tid, &tuple.t_self);
-
-	found = heap_fetch(gp_fastsequence_rel, SnapshotNow, &tuple,
-					   &userbuf, false, NULL);
-	Assert(found);
-	
-	lastSequenceDatum = heap_getattr(&tuple, Anum_gp_fastsequence_last_sequence,
-									 gp_fastsequence_rel->rd_att, &isNull);
-	if (isNull)
-		ereport(ERROR,
-				(errcode(ERRCODE_UNDEFINED_OBJECT),
-				 errmsg("got an invalid lastsequence number: NULL")));
-	
-	objidDatum = heap_getattr(&tuple, Anum_gp_fastsequence_objid,
-							  gp_fastsequence_rel->rd_att, &isNull);
-	if (isNull)
-		ereport(ERROR,
-				(errcode(ERRCODE_UNDEFINED_OBJECT),
-				 errmsg("got an invalid objid: NULL")));
-	
-	objmodDatum = heap_getattr(&tuple, Anum_gp_fastsequence_objmod,
-							   gp_fastsequence_rel->rd_att, &isNull);
-	if (isNull)
-		ereport(ERROR,
-				(errcode(ERRCODE_UNDEFINED_OBJECT),
-				 errmsg("got an invalid objmod: NULL")));
-	
-	if (DatumGetInt64(lastSequenceDatum) + 1 > minSequence)
-		firstSequence = DatumGetInt64(lastSequenceDatum) + 1;
-	newLastSequence = firstSequence + numSequences - 1;
-	
-	update_fastsequence(gp_fastsequence_rel,
-						&tuple,
-						tupleDesc,
-						DatumGetObjectId(objidDatum),
-						DatumGetInt64(objmodDatum),
-						newLastSequence,
-						tid);
-	
-	ReleaseBuffer(userbuf);
-	
-	/*
-	 * Since the tid for this row may be used later in this transaction, 
-	 * we keep the lock until the end of the transaction.
-	 */
-	heap_close(gp_fastsequence_rel, NoLock);
-
-	return firstSequence;
-}
-
-/*
- * RemoveFastSequenceEntry
- *
- * Remove all entries associated with the given object id.
- *
- * If the given objid is an invalid OID, this function simply
- * returns.
- *
- * It is okay for the given valid objid to have no entries in
- * gp_fastsequence.
- */
-void
-RemoveFastSequenceEntry(Oid objid)
-{
-	int numDel;
-	
-	if (!OidIsValid(objid))
-		return;
-
-	numDel = caql_getcount(
-			NULL,
-			cql("DELETE FROM gp_fastsequence "
-				" WHERE objid = :1 ",
-				ObjectIdGetDatum(objid)));
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/catalog/heap.c
----------------------------------------------------------------------
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 477eb67..c61a2e1 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -78,7 +78,6 @@
 #include "catalog/pg_statistic.h"
 #include "catalog/pg_tablespace.h"
 #include "catalog/pg_type.h"
-#include "catalog/gp_fastsequence.h"
 #include "cdb/cdbappendonlyam.h"
 #include "cdb/cdbpartition.h"
 #include "cdb/cdbanalyze.h"

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/catalog/index.c
----------------------------------------------------------------------
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 29dc07a..9d55a87 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -62,7 +62,6 @@
 #include "catalog/pg_opclass.h"
 #include "catalog/pg_tablespace.h"
 #include "catalog/pg_type.h"
-#include "catalog/aoblkdir.h"
 #include "commands/tablecmds.h"
 #include "executor/executor.h"
 #include "miscadmin.h"
@@ -1822,7 +1821,6 @@ IndexBuildAppendOnlyRowScan(Relation parentRelation,
 	double reltuples = 0;
 	Datum		values[INDEX_MAX_KEYS];
 	bool		isnull[INDEX_MAX_KEYS];
-	AppendOnlyBlockDirectory *blockDirectory = NULL;
 	
 	Assert(estate->es_per_tuple_exprcontext != NULL);
 	econtext = estate->es_per_tuple_exprcontext;
@@ -1836,39 +1834,6 @@ IndexBuildAppendOnlyRowScan(Relation parentRelation,
 								  snapshot,
 								  0,
 								  NULL);
-
-	if (!OidIsValid(aoscan->aoEntry->blkdirrelid) ||
-		!OidIsValid(aoscan->aoEntry->blkdiridxid))
-	{
-		IndexInfoOpaque *opaque;
-
-		if (indexInfo->ii_Concurrent)
-			ereport(ERROR,
-					(errcode(ERRCODE_GP_COMMAND_ERROR),
-					 errmsg("Cannot create index concurrently. Create an index non-concurrently "
-					        "before creating an index concurrently in an appendonly table.")));
-		
-		/* Obtain the oids from IndexInfo. */
-		Assert(indexInfo->opaque != NULL);
-
-		opaque = (IndexInfoOpaque *)indexInfo->opaque;
-		
-		Assert(OidIsValid(opaque->blkdirRelOid) && OidIsValid(opaque->blkdirIdxOid));
-		AlterTableCreateAoBlkdirTableWithOid(RelationGetRelid(parentRelation),
-											 opaque->blkdirRelOid,
-											 opaque->blkdirIdxOid,
-											 &opaque->blkdirComptypeOid,
-											 false);
-
-		/* Update blkdirrelid, blkdiridxid in aoEntry with new values */
-		aoscan->aoEntry->blkdirrelid = opaque->blkdirRelOid;
-		aoscan->aoEntry->blkdiridxid = opaque->blkdirIdxOid;
-		
-		aoscan->buildBlockDirectory = true;
-		aoscan->blockDirectory =
-			(AppendOnlyBlockDirectory *)palloc0(sizeof(AppendOnlyBlockDirectory));
-		blockDirectory = aoscan->blockDirectory;
-	}
 	
 	while (appendonly_getnext(aoscan, ForwardScanDirection, slot) != NULL)
 	{
@@ -1908,10 +1873,7 @@ IndexBuildAppendOnlyRowScan(Relation parentRelation,
 	}
 	
 	appendonly_endscan(aoscan);
-	
-	if (blockDirectory != NULL)
-		pfree(blockDirectory);
-	
+
 	return reltuples;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/catalog/pg_appendonly.c
----------------------------------------------------------------------
diff --git a/src/backend/catalog/pg_appendonly.c b/src/backend/catalog/pg_appendonly.c
index 02280e8..d7638b4 100644
--- a/src/backend/catalog/pg_appendonly.c
+++ b/src/backend/catalog/pg_appendonly.c
@@ -33,7 +33,6 @@
 #include "catalog/pg_appendonly.h"
 #include "catalog/pg_type.h"
 #include "catalog/pg_proc.h"
-#include "catalog/gp_fastsequence.h"
 #include "access/genam.h"
 #include "access/heapam.h"
 #include "catalog/catquery.h"
@@ -705,9 +704,6 @@ RemoveAppendonlyEntry(Oid relid)
 		Assert(OidIsValid(aosegrelid));
 	}
 
-	/* Piggyback here to remove gp_fastsequence entries */
-	RemoveFastSequenceEntry(aosegrelid);
-
 	/*
 	 * Delete the appendonly table entry from the catalog (pg_appendonly).
 	 */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/cdb/cdbappendonlystoragewrite.c
----------------------------------------------------------------------
diff --git a/src/backend/cdb/cdbappendonlystoragewrite.c b/src/backend/cdb/cdbappendonlystoragewrite.c
index c740115..18288b5 100644
--- a/src/backend/cdb/cdbappendonlystoragewrite.c
+++ b/src/backend/cdb/cdbappendonlystoragewrite.c
@@ -696,9 +696,6 @@ int32 AppendOnlyStorageWrite_CompleteHeaderLen(
 		completeHeaderLen += (AoHeader_LongSize - AoHeader_RegularSize);
 	}
 
-	if (storageWrite->isFirstRowNumSet)
-		completeHeaderLen += sizeof(int64);
-
 	return completeHeaderLen;
 }
 
@@ -718,9 +715,6 @@ static int32 AppendOnlyStorageWrite_LargeContentHeaderLen(
 
 	completeHeaderLen = storageWrite->regularHeaderLen;
 
-	if (storageWrite->isFirstRowNumSet)
-		completeHeaderLen += sizeof(int64);
-
 	// UNDONE: Right alignment?
 
 	return completeHeaderLen;
@@ -1300,9 +1294,9 @@ AppendOnlyStorageWrite_CompressAppend(
 			AppendOnlyStorageFormat_MakeSmallContentHeader(
 										header,
 										storageWrite->storageAttributes.checksum,
-										storageWrite->isFirstRowNumSet,
+										false,
 										storageWrite->storageAttributes.version,
-										storageWrite->firstRowNum,
+										1,
 										executorBlockKind,
 										itemCount,
 										sourceLen,
@@ -1316,9 +1310,9 @@ AppendOnlyStorageWrite_CompressAppend(
 			AppendOnlyStorageFormat_MakeBulkDenseContentHeader(
 										header,
 										storageWrite->storageAttributes.checksum,
-										storageWrite->isFirstRowNumSet,
+										false,
 										storageWrite->storageAttributes.version,
-										storageWrite->firstRowNum,
+										1,
 										executorBlockKind,
 										itemCount,
 										sourceLen,
@@ -1382,9 +1376,9 @@ AppendOnlyStorageWrite_CompressAppend(
 		AppendOnlyStorageFormat_MakeSmallContentHeader(
 								header,
 								storageWrite->storageAttributes.checksum,
-								storageWrite->isFirstRowNumSet,
+								false,
 								storageWrite->storageAttributes.version,
-								storageWrite->firstRowNum,
+								1,
 								executorBlockKind,
 								itemCount,
 								sourceLen,
@@ -1457,7 +1451,7 @@ void AppendOnlyStorageWrite_FinishBuffer(
 			 contentLen,
 			 storageWrite->maxBufferLen,
 			 storageWrite->currentCompleteHeaderLen,
-			 (storageWrite->isFirstRowNumSet ? "true" : "false"));
+			 (false ? "true" : "false"));
 
 
 	headerOffsetInFile = BufferedAppendCurrentBufferPosition(&storageWrite->bufferedAppend);
@@ -1489,9 +1483,9 @@ void AppendOnlyStorageWrite_FinishBuffer(
 			AppendOnlyStorageFormat_MakeSmallContentHeader(
 									nonCompressedHeader,
 									storageWrite->storageAttributes.checksum,
-									storageWrite->isFirstRowNumSet,
+									false,
 									storageWrite->storageAttributes.version,
-									storageWrite->firstRowNum,
+									1,
 									executorBlockKind,
 									rowCount,
 									contentLen,
@@ -1505,9 +1499,9 @@ void AppendOnlyStorageWrite_FinishBuffer(
 			AppendOnlyStorageFormat_MakeNonBulkDenseContentHeader(
 										nonCompressedHeader,
 										storageWrite->storageAttributes.checksum,
-										storageWrite->isFirstRowNumSet,
+										false,
 										storageWrite->storageAttributes.version,
-										storageWrite->firstRowNum,
+										1,
 										executorBlockKind,
 										rowCount,
 										contentLen);
@@ -1621,7 +1615,6 @@ void AppendOnlyStorageWrite_FinishBuffer(
 
 	Assert(storageWrite->currentCompleteHeaderLen == 0);
 	storageWrite->currentBuffer = NULL;
-	storageWrite->isFirstRowNumSet = false;
 }
 
 /*
@@ -1644,12 +1637,6 @@ void AppendOnlyStorageWrite_CancelLastBuffer(
 	}
 
 	storageWrite->currentCompleteHeaderLen = 0;
-
-	/*
-	 * Since we don't know if AppendOnlyStorageWrite_Content will be called next or
-	 * the writer is doing something else, let's turn off the firstRowNum flag.
-	 */
-	storageWrite->isFirstRowNumSet = false;
 }
 
 // -----------------------------------------------------------------------------
@@ -1818,9 +1805,9 @@ void AppendOnlyStorageWrite_Content(
 		AppendOnlyStorageFormat_MakeLargeContentHeader(
 								largeContentHeader,
 								storageWrite->storageAttributes.checksum,
-								storageWrite->isFirstRowNumSet,
+								false,
 								storageWrite->storageAttributes.version,
-								storageWrite->firstRowNum,
+								1,
 								executorBlockKind,
 								rowCount,
 								contentLen);
@@ -1833,11 +1820,6 @@ void AppendOnlyStorageWrite_Content(
 		// Declare it finished.
 		storageWrite->currentCompleteHeaderLen = 0;
 
-		/*
-		 * Now write the fragments as type Block.
-		 */
-		storageWrite->isFirstRowNumSet = false;	// Not written with fragments.
-
 		smallContentHeaderLen = 
 				AppendOnlyStorageWrite_CompleteHeaderLen(
 													storageWrite,
@@ -1927,30 +1909,6 @@ void AppendOnlyStorageWrite_Content(
 		}
 	}
 
-	storageWrite->isFirstRowNumSet = false;
-
 	// Verify we have no buffer allocated.
 	Assert(storageWrite->currentCompleteHeaderLen == 0);
 }
-
-// -----------------------------------------------------------------------------
-// Optional: Set First Row Number
-// -----------------------------------------------------------------------------
-
-/*
- * Set the first row value for the next Append-Only Storage
- * Block to be written.  Only applies to the next block.
- */
-void AppendOnlyStorageWrite_SetFirstRowNum(
-	AppendOnlyStorageWrite		*storageWrite,
-	int64						firstRowNum)
-{
-
-	Assert(storageWrite != NULL);
-	Assert(storageWrite->isActive);
-
-	// UNDONE: Range check firstRowNum
-
-	storageWrite->isFirstRowNumSet = true;
-	storageWrite->firstRowNum = firstRowNum;
-}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/cdb/cdbllize.c
----------------------------------------------------------------------
diff --git a/src/backend/cdb/cdbllize.c b/src/backend/cdb/cdbllize.c
index 393e6bb..4aa8795 100644
--- a/src/backend/cdb/cdbllize.c
+++ b/src/backend/cdb/cdbllize.c
@@ -1328,7 +1328,6 @@ motion_sanity_walker(Node *node, sanity_result_t *result)
 		case T_IndexScan:
 		case T_BitmapIndexScan:
 		case T_BitmapHeapScan:
-		case T_BitmapAppendOnlyScan:
 		case T_BitmapTableScan:
 		case T_TidScan:
 		case T_SubqueryScan:

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/cdb/cdbpath.c
----------------------------------------------------------------------
diff --git a/src/backend/cdb/cdbpath.c b/src/backend/cdb/cdbpath.c
index fc25f4c..8c38ee7 100644
--- a/src/backend/cdb/cdbpath.c
+++ b/src/backend/cdb/cdbpath.c
@@ -1392,7 +1392,6 @@ cdbpath_dedup_fixup_walker(Path *path, void *context)
 		case T_ParquetScan:
 		case T_IndexScan:
 		case T_BitmapHeapScan:
-		case T_BitmapAppendOnlyScan:
 		case T_BitmapTableScan:
 		case T_TidScan:
 		case T_SubqueryScan:

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/cdb/cdbpersistentrelfile.c
----------------------------------------------------------------------
diff --git a/src/backend/cdb/cdbpersistentrelfile.c b/src/backend/cdb/cdbpersistentrelfile.c
index e7c2d63..2b8e3e8 100644
--- a/src/backend/cdb/cdbpersistentrelfile.c
+++ b/src/backend/cdb/cdbpersistentrelfile.c
@@ -36,7 +36,6 @@
 #include "catalog/pg_tablespace.h"
 #include "catalog/pg_database.h"
 #include "catalog/gp_persistent.h"
-#include "catalog/gp_fastsequence.h"
 #include "cdb/cdbsharedoidsearch.h"
 #include "access/persistentfilesysobjname.h"
 #include "cdb/cdbdirectopen.h"
@@ -713,52 +712,6 @@ PersistentFileSysObjStateChangeResult PersistentRelfile_MarkAbortingCreate(
 		return false;	// The initdb process will load the persistent table once we out of bootstrap mode.
 	}
 
-	/* MPP-16543: When inserting tuples into AO table, row numbers will be
-	 * generated from gp_fastsequence catalog table, as part of the design,
-	 * these sequence numbers are not reusable, even if the AO insert 
-	 * transaction is aborted. The entry in gp_fastsequence was inserted
-	 * using frozen_heap_insert, which means it's always visible. 
-
-	 * Aborted AO insert transaction will cause inconsistency between 
-	 * gp_fastsequence and pg_class, the solution is to introduce "frozen 
-	 * delete" - inplace update tuple's MVCC header to make it invisible.
-	 */
-
-	Relation gp_fastsequence_rel = heap_open(FastSequenceRelationId, RowExclusiveLock);
-	HeapTuple   tup;
-	SysScanDesc scan;
-	ScanKeyData skey; 
-	ScanKeyInit(&skey,
-				Anum_gp_fastsequence_objid,
-				BTEqualStrategyNumber,
-				F_OIDEQ,
-				relFileNode->relNode);
-
-	scan = systable_beginscan(gp_fastsequence_rel,
-							  InvalidOid,
-							  false,
-							  SnapshotNow,
-							  1,
-							  &skey);
-	while (HeapTupleIsValid(tup = systable_getnext(scan)))
-	{
-		Form_gp_fastsequence found = (Form_gp_fastsequence) GETSTRUCT(tup);
-		if (found->objid == relFileNode->relNode) 
-		{	
-			if (Debug_persistent_print)
-			{
-			elog(LOG, "frozen deleting gp_fastsequence entry for aborted AO insert transaction on relation %s", relpath(*relFileNode));
-			}
-
-			frozen_heap_inplace_delete(gp_fastsequence_rel, tup);
-		}
-	}						
-	systable_endscan(scan);
-	heap_close(gp_fastsequence_rel, RowExclusiveLock);
-	
-
-	
-
 	PersistentRelfile_VerifyInitScan();
 
 	// Do this check after skipping out if in bootstrap mode.

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/cdb/cdbplan.c
----------------------------------------------------------------------
diff --git a/src/backend/cdb/cdbplan.c b/src/backend/cdb/cdbplan.c
index 68d65f0..31152c8 100644
--- a/src/backend/cdb/cdbplan.c
+++ b/src/backend/cdb/cdbplan.c
@@ -415,22 +415,7 @@ plan_tree_mutator(Node *node,
 				return (Node *) newbmheapscan;
 			}
 			break;
-		
-
-		case T_BitmapAppendOnlyScan:
-			{
-				BitmapAppendOnlyScan  *bmappendonlyscan = (BitmapAppendOnlyScan *) node;
-				BitmapAppendOnlyScan  *newbmappendonlyscan;
-
-				FLATCOPY(newbmappendonlyscan, bmappendonlyscan, BitmapAppendOnlyScan);
-				SCANMUTATE(newbmappendonlyscan, bmappendonlyscan);
 				
-				MUTATE(newbmappendonlyscan->bitmapqualorig, bmappendonlyscan->bitmapqualorig, List *);
-	
-				return (Node *) newbmappendonlyscan;
-			}
-			break;
-		
 		case T_BitmapTableScan:
 			{
 				BitmapTableScan  *bmtablescan = (BitmapTableScan *) node;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/cdb/cdbquerycontextdispatching.c
----------------------------------------------------------------------
diff --git a/src/backend/cdb/cdbquerycontextdispatching.c b/src/backend/cdb/cdbquerycontextdispatching.c
index b8607bd..ca4d6b2 100644
--- a/src/backend/cdb/cdbquerycontextdispatching.c
+++ b/src/backend/cdb/cdbquerycontextdispatching.c
@@ -36,8 +36,6 @@
 #include "catalog/aoseg.h"
 #include "catalog/catalog.h"
 #include "catalog/catquery.h"
-#include "catalog/gp_fastsequence.h"
-#include "catalog/gp_fastsequence.h"
 #include "catalog/pg_amop.h"
 #include "catalog/pg_amproc.h"
 #include "catalog/pg_aggregate.h"
@@ -55,7 +53,6 @@
 #include "catalog/pg_operator.h"
 #include "catalog/pg_proc.h"
 #include "catalog/pg_tablespace.h"
-#include "catalog/gp_fastsequence.h"
 #include "catalog/toasting.h"
 #include "cdb/cdbdispatchedtablespaceinfo.h"
 #include "cdb/cdbfilesystemcredential.h"
@@ -1385,49 +1382,6 @@ static void
 prepareDispatchedCatalogFastSequence(QueryContextInfo *cxt, Oid relid,
 									 List *segnos)
 {
-    QueryContextDispatchingHashKey hkey;
-    QueryContextDispatchingHashEntry *hentry = NULL;
-    bool found;
-
-    SysScanDesc scanDesc;
-    HeapTuple tuple;
-    Datum contentid;
-    ScanKeyData scanKeys[1];
-    Relation fast_seq_rel;
-
-    hkey.objid = relid;
-    hkey.type = RelationType;
-    hentry = hash_search(cxt->htab, &hkey, HASH_FIND, &found);
-    Assert(found);
-    Assert(hentry);
-
-    fast_seq_rel = heap_open(FastSequenceRelationId, AccessShareLock);
-
-    ScanKeyInit(&scanKeys[0], Anum_gp_fastsequence_objid, BTEqualStrategyNumber,
-            F_OIDEQ, ObjectIdGetDatum(hentry->aoseg_relid));
-
-    scanDesc = systable_beginscan(fast_seq_rel, InvalidOid, FALSE,
-            SnapshotNow, 1, scanKeys);
-
-    while (HeapTupleIsValid(tuple = systable_getnext(scanDesc)))
-    {
-    		int i;
-    		Datum segno;
-        contentid = heap_getattr(tuple, Anum_gp_fastsequence_contentid,
-                RelationGetDescr(fast_seq_rel), NULL);
-        segno = heap_getattr(tuple, Anum_gp_fastsequence_objmod,
-        			RelationGetDescr(fast_seq_rel), NULL);
-        i = list_find_int(segnos, DatumGetInt32(segno));
-        if (i != -1)
-        {
-        		AddTupleToContextInfo(cxt, FastSequenceRelationId, "gp_fastsequence",
-        				tuple, DatumGetInt32(contentid));
-        }
-    }
-
-    systable_endscan(scanDesc);
-
-    heap_close(fast_seq_rel, AccessShareLock);
 }
 
 /*
@@ -2815,9 +2769,6 @@ UpdateCatalogModifiedOnSegments(QueryContextDispatchingSendBack sendback)
 				sendback->eof[0], sendback->uncompressed_eof[0], sendback->insertCount);
 	}
 
-	ItemPointerData tid;
-	InsertFastSequenceEntry(aoEntry->segrelid, sendback->segno,
-			sendback->nextFastSequence, &tid);
 	heap_close(rel, AccessShareLock);
 
 	/*
@@ -2882,10 +2833,6 @@ AddSendbackChangedCatalogContent(StringInfo buf,
 		pq_sendint64(buf, sendback->uncompressed_eof[i]);
 	}
 
-	/*
-	 * 10, send next fast sequence.
-	 */
-	pq_sendint64(buf, sendback->nextFastSequence);
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/cdb/cdbtargeteddispatch.c
----------------------------------------------------------------------
diff --git a/src/backend/cdb/cdbtargeteddispatch.c b/src/backend/cdb/cdbtargeteddispatch.c
index 720008d..4dfa6a7 100644
--- a/src/backend/cdb/cdbtargeteddispatch.c
+++ b/src/backend/cdb/cdbtargeteddispatch.c
@@ -166,7 +166,6 @@ GetContentIdsFromPlanForSingleRelation(List *rtable, Plan *plan, int rangeTableI
 	InitDirectDispatchCalculationInfo(&result);
 
 	if ( nodeTag((Node*)plan) == T_BitmapHeapScan ||
-		 nodeTag((Node*)plan) == T_BitmapAppendOnlyScan ||
 		 nodeTag((Node*)plan) == T_BitmapTableScan)
 	{
 		/* do not assert for bitmap heap scan --> it can have a child which is an index scan */
@@ -471,7 +470,6 @@ AssignContentIdsToPlanData_Walker(Node *node, void *context)
 				/* no change to dispatchInfo --> just iterate children */
 				break;
 			case T_BitmapHeapScan:
-			case T_BitmapAppendOnlyScan:
 			case T_BitmapTableScan:
 				/* no change to dispatchInfo --> just iterate children */
 				break;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/commands/analyze.c
----------------------------------------------------------------------
diff --git a/src/backend/commands/analyze.c b/src/backend/commands/analyze.c
index 1d25cb0..d0e2523 100644
--- a/src/backend/commands/analyze.c
+++ b/src/backend/commands/analyze.c
@@ -918,7 +918,6 @@ static void analyzeRelation(Relation relation, List *lAttributeNames, bool rooto
 	List	*indexOidList = NIL;
 	ListCell	*lc = NULL;
 	StringInfoData location;
-	StringInfoData err_msg;
 	
 	initStringInfo(&location);
 	relationOid		= RelationGetRelid(relation);

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/commands/cluster.c
----------------------------------------------------------------------
diff --git a/src/backend/commands/cluster.c b/src/backend/commands/cluster.c
index ab6b87e..9372efe 100644
--- a/src/backend/commands/cluster.c
+++ b/src/backend/commands/cluster.c
@@ -50,7 +50,6 @@
 #include "catalog/pg_type.h"
 #include "catalog/toasting.h"
 #include "catalog/aoseg.h"
-#include "catalog/aoblkdir.h"
 #include "catalog/pg_tablespace.h"
 #include "commands/cluster.h"
 #include "commands/tablecmds.h"
@@ -857,12 +856,6 @@ make_new_heap(Oid OIDOldHeap, const char *NewName, Oid NewTableSpace,
 	AlterTableCreateAoSegTableWithOid(OIDNewHeap, aOid, aiOid,
 									  aosegComptypeOid, is_part);
 
-    if ( createAoBlockDirectory )
-    {
-	    AlterTableCreateAoBlkdirTableWithOid(OIDNewHeap, blkdirOid, blkdirIdxOid,
-										 aoblkdirComptypeOid, is_part);
-    }
-
 	cloneAttributeEncoding(OIDOldHeap,
 						   OIDNewHeap,
 						   RelationGetNumberOfAttributes(OldHeap));

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/commands/explain.c
----------------------------------------------------------------------
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 876b7ad..51afde2 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -1167,12 +1167,6 @@ explain_outNode(StringInfo str,
 		case T_BitmapHeapScan:
 			pname = "Bitmap Heap Scan";
 			break;
-		case T_BitmapAppendOnlyScan:
-			if (((BitmapAppendOnlyScan *)plan)->isAORow)
-				pname = "Bitmap Append-Only Row-Oriented Scan";
-			else
-				pname = "Bitmap Append-Only Column-Oriented Scan";
-			break;
 		case T_BitmapTableScan:
 			pname = "Bitmap Table Scan";
 			break;
@@ -1355,7 +1349,6 @@ explain_outNode(StringInfo str,
 		case T_DynamicTableScan:
 		case T_DynamicIndexScan:
 		case T_BitmapHeapScan:
-		case T_BitmapAppendOnlyScan:
 		case T_BitmapTableScan:
 		case T_TidScan:
 			if (((Scan *) plan)->scanrelid > 0)
@@ -1538,7 +1531,6 @@ explain_outNode(StringInfo str,
 						   str, indent, es);
 			break;
 		case T_BitmapHeapScan:
-		case T_BitmapAppendOnlyScan:
 		case T_BitmapTableScan:
 			/* XXX do we want to show this in production? */
 			if (nodeTag(plan) == T_BitmapHeapScan)
@@ -1549,14 +1541,6 @@ explain_outNode(StringInfo str,
 							   plan, outer_plan,
 							   str, indent, es);
 			}
-			else if (nodeTag(plan) == T_BitmapAppendOnlyScan)
-			{
-				show_scan_qual(((BitmapAppendOnlyScan *) plan)->bitmapqualorig,
-							   "Recheck Cond",
-							   ((Scan *) plan)->scanrelid,
-							   plan, outer_plan,
-							   str, indent, es);
-			}
 			else if (nodeTag(plan) == T_BitmapTableScan)
 			{
 				show_scan_qual(((BitmapTableScan *) plan)->bitmapqualorig,
@@ -1901,7 +1885,6 @@ explain_outNode(StringInfo str,
 		explain_outNode(str, outerPlan(plan),
 						outerPlanState(planstate),
 						(IsA(plan, BitmapHeapScan) |
-						 IsA(plan, BitmapAppendOnlyScan) |
 						 IsA(plan, BitmapTableScan)) ? outer_plan : NULL,
 						indent + 3, es,isSequential);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/executor/Makefile
----------------------------------------------------------------------
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index d2bf879..2589e6a 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -17,13 +17,13 @@ override CPPFLAGS := -I$(top_srcdir)/src/backend/gp_libpq_fe $(CPPFLAGS)
 OBJS = execAmi.o execGrouping.o execHHashagg.o execJunk.o execMain.o \
        execProcnode.o execQual.o execScan.o execTuples.o execGpmon.o \
        execUtils.o execWorkfile.o execHeapScan.o execAOScan.o execParquetScan.o\
-       execBitmapTableScan.o execBitmapHeapScan.o execBitmapAOScan.o execBitmapParquetScan.o execDynamicScan.o \
+       execBitmapTableScan.o execBitmapHeapScan.o execDynamicScan.o \
        execIndexscan.o \
        functions.o \
        instrument.o \
        nodeAppend.o nodeAgg.o \
        nodeBitmapAnd.o nodeBitmapOr.o \
-       nodeBitmapAppendOnlyscan.o nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeBitmapTableScan.o \
+       nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeBitmapTableScan.o \
        nodeExternalscan.o \
        nodeFunctionscan.o \
        nodeHash.o nodeHashjoin.o \

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/executor/execAmi.c
----------------------------------------------------------------------
diff --git a/src/backend/executor/execAmi.c b/src/backend/executor/execAmi.c
index 541ca8c..e51991a 100644
--- a/src/backend/executor/execAmi.c
+++ b/src/backend/executor/execAmi.c
@@ -46,7 +46,6 @@
 #include "executor/nodeTidscan.h"
 #include "executor/nodeUnique.h"
 #include "executor/nodeValuesscan.h"
-#include "executor/nodeBitmapAppendOnlyscan.h"
 #include "executor/nodeWindow.h"
 #include "executor/nodeShareInputScan.h"
 
@@ -188,10 +187,6 @@ ExecReScan(PlanState *node, ExprContext *exprCtxt)
 			ExecValuesReScan((ValuesScanState *) node, exprCtxt);
 			break;
 
-		case T_BitmapAppendOnlyScanState:
-			ExecBitmapAppendOnlyReScan((BitmapAppendOnlyScanState *) node, exprCtxt);
-			break;
-
 		case T_NestLoopState:
 			ExecReScanNestLoop((NestLoopState *) node, exprCtxt);
 			break;
@@ -640,11 +635,7 @@ ExecEagerFree(PlanState *node)
 		case T_BitmapHeapScanState:
 			ExecEagerFreeBitmapHeapScan((BitmapHeapScanState *)node);
 			break;
-			
-		case T_BitmapAppendOnlyScanState:
-			ExecEagerFreeBitmapAppendOnlyScan((BitmapAppendOnlyScanState *)node);
-			break;
-			
+
 		case T_BitmapTableScanState:
 			ExecEagerFreeBitmapTableScan((BitmapTableScanState *)node);
 			break;
@@ -788,7 +779,6 @@ ExecEagerFreeChildNodes(PlanState *node, bool subplanDone)
 		case T_ExternalScanState:
 		case T_IndexScanState:
 		case T_BitmapHeapScanState:
-		case T_BitmapAppendOnlyScanState:
 		case T_FunctionScanState:
 		case T_MaterialState:
 		case T_SortState:

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/executor/execBitmapAOScan.c
----------------------------------------------------------------------
diff --git a/src/backend/executor/execBitmapAOScan.c b/src/backend/executor/execBitmapAOScan.c
deleted file mode 100644
index 94ef8f4..0000000
--- a/src/backend/executor/execBitmapAOScan.c
+++ /dev/null
@@ -1,280 +0,0 @@
-/*
- * execBitmapAOScan.c
- *   Support routines for scanning AO and AOCO tables using bitmaps.
- *
- * Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group
- * Portions Copyright (c) 1994, Regents of the University of California
- *
- */
-#include "postgres.h"
-
-#include "access/heapam.h"
-#include "executor/execdebug.h"
-#include "executor/nodeBitmapAppendOnlyscan.h"
-#include "cdb/cdbappendonlyam.h"
-#include "pgstat.h"
-#include "utils/memutils.h"
-#include "miscadmin.h"
-#include "parser/parsetree.h"
-#include "cdb/cdbvars.h" /* gp_select_invisible */
-#include "nodes/tidbitmap.h"
-
-typedef struct
-{
-	int			tupleIndex;
-	int			nTuples;
-} AOIteratorState;
-
-/*
- * Prepares for a new AO scan.
- */
-void
-BitmapAOScanBegin(ScanState *scanState)
-{
-	BitmapTableScanState *node = (BitmapTableScanState *)(scanState);
-	Relation currentRelation = node->ss.ss_currentRelation;
-	EState *estate = node->ss.ps.state;
-
-	if (scanState->tableType == TableTypeAppendOnly)
-	{
-		node->scanDesc =
-			appendonly_fetch_init(currentRelation,
-								  estate->es_snapshot);
-	}
-	else if (scanState->tableType == TableTypeParquet)
-	{
-	  Assert(!"BitmapScan for Parquet is not supported yet");
-      /*
-       * Obtain the projection.
-       */
-      Assert(currentRelation->rd_att != NULL);
-
-      bool *proj = (bool *)palloc0(sizeof(bool) * currentRelation->rd_att->natts);
-
-      GetNeededColumnsForScan((Node *) node->ss.ps.plan->targetlist, proj, currentRelation->rd_att->natts);
-      GetNeededColumnsForScan((Node *) node->ss.ps.plan->qual, proj, currentRelation->rd_att->natts);
-
-      int colno = 0;
-
-      /* Check if any column is projected */
-      for(colno = 0; colno < currentRelation->rd_att->natts; colno++)
-      {
-        if(proj[colno])
-        {
-          break;
-        }
-      }
-
-      /*
-       * At least project one column. Since the tids stored in the index may not have
-       * a corresponding tuple any more (because of previous crashes, for example), we
-       * need to read the tuple to make sure.
-       */
-      if(colno == currentRelation->rd_att->natts)
-      {
-        proj[0] = true;
-      }
-
-      /*
-        node->scanDesc =
-            parquet_fetch_init(currentRelation, estate->es_snapshot, proj);
-       */
-	}
-	else
-	{
-		Assert(!"Invalid table type");
-	}
-
-
-	/*
-	 * AO doesn't need rechecking every tuple once it resolves
-	 * from the bitmap page, except when it deals with lossy
-	 * bitmap, which is handled via scanState->isLossyBitmapPage.
-	 */
-	node->recheckTuples = false;
-}
-
-/*
- * Cleans up after the scanning is done.
- */
-void
-BitmapAOScanEnd(ScanState *scanState)
-{
-	BitmapTableScanState *node = (BitmapTableScanState *)scanState;
-	Assert(node->ss.scan_state == SCAN_SCAN);
-
-	if (scanState->tableType == TableTypeAppendOnly)
-	{
-		appendonly_fetch_finish((AppendOnlyFetchDesc)node->scanDesc);
-	}
-	else if (scanState->tableType == TableTypeParquet)
-	{
-	  Assert(!"BitmapScan for Parquet is not supported yet");
-	  /*
-		pfree(((AOCSFetchDesc)node->scanDesc)->proj);
-		aocs_fetch_finish(node->scanDesc);
-		*/
-	}
-	else
-	{
-		Assert(!"Invalid table type");
-	}
-	pfree(node->scanDesc);
-	node->scanDesc = NULL;
-
-	if (node->iterator)
-	{
-		pfree(node->iterator);
-		node->iterator = NULL;
-	}
-}
-
-/*
- * Returns the next matching tuple.
- */
-TupleTableSlot *
-BitmapAOScanNext(ScanState *scanState)
-{
-	BitmapTableScanState *node = (BitmapTableScanState *)scanState;
-
-	TupleTableSlot *slot = node->ss.ss_ScanTupleSlot;
-
-	TBMIterateResult *tbmres = (TBMIterateResult *)node->tbmres;
-
-	/* Make sure we never cross 15-bit offset number [MPP-24326] */
-	Assert(tbmres->ntuples <= INT16_MAX + 1);
-
-	OffsetNumber psuedoHeapOffset;
-	ItemPointerData psudeoHeapTid;
-	AOTupleId aoTid;
-
-	Assert(tbmres != NULL && tbmres->ntuples != 0);
-	Assert(node->needNewBitmapPage == false);
-
-	AOIteratorState *iterator = (AOIteratorState *)node->iterator;
-	for (;;)
-	{
-		CHECK_FOR_INTERRUPTS();
-
-		if (iterator == NULL)
-		{
-			iterator = palloc0(sizeof(AOIteratorState));
-
-			if (node->isLossyBitmapPage)
-			{
-				/* Iterate over the first 2^15 tuples [MPP-24326] */
-				iterator->nTuples = INT16_MAX + 1;
-			}
-			else
-			{
-				iterator->nTuples = tbmres->ntuples;
-			}
-			/* Start from the beginning of the page */
-			iterator->tupleIndex = 0;
-
-			node->iterator = iterator;
-		}
-		else
-		{
-			/*
-			 * Continuing in previously obtained page; advance tupleIndex
-			 */
-			iterator->tupleIndex++;
-		}
-
-		/*
-		 * Out of range?  If so, nothing more to look at on this page
-		 */
-		if (iterator->tupleIndex < 0 || iterator->tupleIndex >= iterator->nTuples)
-		{
-			pfree(iterator);
-
-			node->iterator = NULL;
-
-			node->needNewBitmapPage = true;
-
-			return ExecClearTuple(slot);
-		}
-
-		/*
-		 * Must account for lossy page info...
-		 */
-		if (node->isLossyBitmapPage)
-		{
-			/* We are iterating through all items. */
-			psuedoHeapOffset = iterator->tupleIndex;
-		}
-		else
-		{
-			Assert(iterator->tupleIndex <= tbmres->ntuples);
-			psuedoHeapOffset = tbmres->offsets[iterator->tupleIndex];
-
-			/*
-			 * Ensure that the reserved 16-th bit is always ON for offsets from
-			 * lossless bitmap pages [MPP-24326].
-			 */
-			Assert(((uint16)(psuedoHeapOffset & 0x8000)) > 0);
-		}
-
-		/*
-		 * Okay to fetch the tuple
-		 */
-		ItemPointerSet(
-				&psudeoHeapTid,
-				tbmres->blockno,
-				psuedoHeapOffset);
-
-		tbm_convert_appendonly_tid_out(&psudeoHeapTid, &aoTid);
-
-		if (scanState->tableType == TableTypeAppendOnly)
-		{
-			appendonly_fetch((AppendOnlyFetchDesc)node->scanDesc, &aoTid, slot);
-		}
-		else if (scanState->tableType == TableTypeParquet)
-		{
-		  Assert(!"BitmapScan for Parquet is not supported yet");
-		  /*
-			Assert(scanState->tableType == TableTypeAOCS);
-			aocs_fetch((AOCSFetchDesc)node->scanDesc, &aoTid, slot);
-			*/
-		}
-
-      	if (TupIsNull(slot))
-      	{
-			continue;
-      	}
-
-		Assert(ItemPointerIsValid(slot_get_ctid(slot)));
-
-		pgstat_count_heap_fetch(node->ss.ss_currentRelation);
-
-		if (!BitmapTableScanRecheckTuple(node, slot))
-		{
-			ExecClearTuple(slot);
-			continue;
-		}
-
-		return slot;
-	}
-
-	/*
-	 * We should never reach here as the termination is handled
-	 * from nodeBitmapTableScan.
-	 */
-	Assert(false);
-	return NULL;
-}
-
-/*
- * Prepares for a re-scan.
- */
-void
-BitmapAOScanReScan(ScanState *scanState)
-{
-	/*
-	 * As per the existing implementation from nodeBitmapAppendOnlyScan.c
-	 * for rescanning of AO, we don't have anything specific
-	 * to do here (the refactored BitmapTableScan takes care of everything).
-	 */
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/executor/execBitmapParquetScan.c
----------------------------------------------------------------------
diff --git a/src/backend/executor/execBitmapParquetScan.c b/src/backend/executor/execBitmapParquetScan.c
deleted file mode 100644
index ec960b6..0000000
--- a/src/backend/executor/execBitmapParquetScan.c
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * execBitmapParquetScan.c
- *   Support routines for scanning parquet tables using bitmaps.
- *
- */
-
-#include "postgres.h"
-#include "nodes/execnodes.h"
-#include "executor/tuptable.h"
-
-void BitmapParquetScanBegin(ScanState *scanState);
-void BitmapParquetScanEnd(ScanState *scanState);
-TupleTableSlot* BitmapParquetScanNext(ScanState *scanState);
-void BitmapParquetScanReScan(ScanState *scanState);
-
-/*
- * Prepares for a new parquet scan.
- */
-void
-BitmapParquetScanBegin(ScanState *scanState)
-{
-	Insist(!"Bitmap index scan on parquet table is not supported");
-}
-
-/*
- * Cleans up after the scanning is done.
- */
-void
-BitmapParquetScanEnd(ScanState *scanState)
-{
-	Insist(!"Bitmap index scan on parquet table is not supported");
-}
-
-/*
- * Returns the next matching tuple.
- */
-TupleTableSlot *
-BitmapParquetScanNext(ScanState *scanState)
-{
-	Insist(!"Bitmap index scan on parquet table is not supported");
-}
-
-/*
- * Prepares for a re-scan.
- */
-void
-BitmapParquetScanReScan(ScanState *scanState)
-{
-	Insist(!"Bitmap index scan on parquet table is not supported");
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/executor/execBitmapTableScan.c
----------------------------------------------------------------------
diff --git a/src/backend/executor/execBitmapTableScan.c b/src/backend/executor/execBitmapTableScan.c
index aa4a491..066624e 100644
--- a/src/backend/executor/execBitmapTableScan.c
+++ b/src/backend/executor/execBitmapTableScan.c
@@ -62,20 +62,6 @@ getBitmapTableScanMethod(TableType tableType)
 		{
 			&BitmapHeapScanNext, &BitmapHeapScanBegin, &BitmapHeapScanEnd,
 			&BitmapHeapScanReScan, &MarkRestrNotAllowed, &MarkRestrNotAllowed
-		},
-		{
-			&BitmapAOScanNext, &BitmapAOScanBegin, &BitmapAOScanEnd,
-			&BitmapAOScanReScan, &MarkRestrNotAllowed, &MarkRestrNotAllowed
-		},
-		{
-			/* The same set of methods serve both AO and AOCO scans */
-			&BitmapAOScanNext, &BitmapAOScanBegin, &BitmapAOScanEnd,
-			&BitmapAOScanReScan, &MarkRestrNotAllowed, &MarkRestrNotAllowed
-		},
-		{
-			/* The same set of methods serve both AO and AOCO scans */
-			&BitmapParquetScanNext, &BitmapParquetScanBegin, &BitmapParquetScanEnd,
-			&BitmapParquetScanReScan, &MarkRestrNotAllowed, &MarkRestrNotAllowed
 		}
 	};
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/executor/execMain.c
----------------------------------------------------------------------
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 444d9a4..1c215b0 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -65,7 +65,6 @@
 #include "catalog/namespace.h"
 #include "catalog/toasting.h"
 #include "catalog/aoseg.h"
-#include "catalog/aoblkdir.h"
 #include "catalog/catalog.h"
 #include "catalog/pg_attribute_encoding.h"
 #include "catalog/pg_type.h"

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/executor/execProcnode.c
----------------------------------------------------------------------
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index d3ad168..0cc8716 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -109,7 +109,6 @@
 #include "executor/nodeBitmapIndexscan.h"
 #include "executor/nodeBitmapTableScan.h"
 #include "executor/nodeBitmapOr.h"
-#include "executor/nodeBitmapAppendOnlyscan.h"
 #include "executor/nodeExternalscan.h"
 #include "executor/nodeTableScan.h"
 #include "executor/nodeDML.h"
@@ -265,8 +264,7 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
 	 */
 	if (force_bitmap_table_scan)
 	{
-		if (IsA(node, BitmapHeapScan) ||
-				IsA(node, BitmapAppendOnlyScan))
+		if (IsA(node, BitmapHeapScan))
 		{
 			node->type = T_BitmapTableScan;
 		}
@@ -417,17 +415,6 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
 			END_MEMORY_ACCOUNT();
 			break;
 
-		case T_BitmapAppendOnlyScan:
-			curMemoryAccount = CREATE_EXECUTOR_MEMORY_ACCOUNT(isAlienPlanNode, node, BitmapAppendOnlyScan);
-
-			START_MEMORY_ACCOUNT(curMemoryAccount);
-			{
-			result = (PlanState *) ExecInitBitmapAppendOnlyScan((BitmapAppendOnlyScan*) node,
-														        estate, eflags);
-			}
-			END_MEMORY_ACCOUNT();
-			break;
-
 		case T_BitmapTableScan:
 			curMemoryAccount = CREATE_EXECUTOR_MEMORY_ACCOUNT(isAlienPlanNode, node, BitmapTableScan);
 
@@ -842,7 +829,6 @@ ExecProcNode(PlanState *node)
 		&&Exec_Jmp_DynamicIndexScan,
 		&&Exec_Jmp_BitmapIndexScan,
 		&&Exec_Jmp_BitmapHeapScan,
-		&&Exec_Jmp_BitmapAppendOnlyScan,
 		&&Exec_Jmp_BitmapTableScan,
 		&&Exec_Jmp_TidScan,
 		&&Exec_Jmp_SubqueryScan,
@@ -945,10 +931,6 @@ Exec_Jmp_BitmapHeapScan:
 	result = ExecBitmapHeapScan((BitmapHeapScanState *) node);
 	goto Exec_Jmp_Done;
 
-Exec_Jmp_BitmapAppendOnlyScan:
-	result = ExecBitmapAppendOnlyScan((BitmapAppendOnlyScanState *) node);
-	goto Exec_Jmp_Done;
-
 Exec_Jmp_BitmapTableScan:
 	result = ExecBitmapTableScan((BitmapTableScanState *) node);
 	goto Exec_Jmp_Done;
@@ -1369,9 +1351,6 @@ ExecCountSlotsNode(Plan *node)
 
 		case T_BitmapHeapScan:
 			return ExecCountSlotsBitmapHeapScan((BitmapHeapScan *) node);
-
-		case T_BitmapAppendOnlyScan:
-			return ExecCountSlotsBitmapAppendOnlyScan((BitmapAppendOnlyScan*) node);
 			
 		case T_BitmapTableScan:
 			return ExecCountSlotsBitmapTableScan((BitmapTableScan *) node);
@@ -1649,10 +1628,6 @@ ExecEndNode(PlanState *node)
 			ExecEndBitmapHeapScan((BitmapHeapScanState *) node);
 			break;
 
-		case T_BitmapAppendOnlyScanState:
-			ExecEndBitmapAppendOnlyScan((BitmapAppendOnlyScanState *) node);
-			break;
-
 		case T_BitmapTableScanState:
 			ExecEndBitmapTableScan((BitmapTableScanState *) node);
 			break;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/executor/execUtils.c
----------------------------------------------------------------------
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index 399584f..ffd756d 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -2341,7 +2341,6 @@ void (*initGpmonPktFuncs[])(Plan *planNode, gpmon_packet_t *gpmon_pkt, EState *e
 	&initGpmonPktForDynamicIndexScan, /* T_DynamicIndexScan */
 	&initGpmonPktForBitmapIndexScan, /* T_BitmapIndexScan */
 	&initGpmonPktForBitmapHeapScan, /* T_BitmapHeapScan */
-	&initGpmonPktForBitmapAppendOnlyScan, /* T_BitmapAppendOnlyScan */
 	&initGpmonPktForBitmapTableScan, /* T_BitmapTableScan */
 	&initGpmonPktForTidScan, /* T_TidScan */
 	&initGpmonPktForSubqueryScan, /* T_SubqueryScan */
@@ -2480,7 +2479,6 @@ sendInitGpmonPkts(Plan *node, EState *estate)
 
 		case T_Result:
 		case T_BitmapHeapScan:
-		case T_BitmapAppendOnlyScan:
 		case T_BitmapTableScan:
 		case T_ShareInputScan:
 		case T_Material:

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/executor/nodeBitmapAppendOnlyscan.c
----------------------------------------------------------------------
diff --git a/src/backend/executor/nodeBitmapAppendOnlyscan.c b/src/backend/executor/nodeBitmapAppendOnlyscan.c
deleted file mode 100755
index 77316cf..0000000
--- a/src/backend/executor/nodeBitmapAppendOnlyscan.c
+++ /dev/null
@@ -1,612 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/*-------------------------------------------------------------------------
- *
- * nodeBitmapAppendOnlyscan.c
- *	  Routines to support bitmapped scan from Append-Only relations
- *
- * This is a modified copy of nodeBitmapHeapscan.c converted to Append-Only.
- *
- *
- * Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group
- * Portions Copyright (c) 1994, Regents of the University of California
- * Portions Copyright (c) 2008-2009, Greenplum Inc.
- *
- *-------------------------------------------------------------------------
- */
-/*
- * INTERFACE ROUTINES
- *		ExecBitmapAppendOnlyScan		scan from an AO relation using bitmap info
- *		ExecBitmapAppendOnlyNext		workhorse for above
- *		ExecInitBitmapAppendOnlyScan	creates and initializes state info.
- *		ExecBitmapAppendOnlyReScan	prepares to rescan the plan.
- *		ExecEndBitmapAppendOnlyScan	releases all storage.
- */
-#include "postgres.h"
-
-#include "access/heapam.h"
-#include "executor/execdebug.h"
-#include "executor/nodeBitmapAppendOnlyscan.h"
-#include "cdb/cdbappendonlyam.h"
-#include "pgstat.h"
-#include "utils/memutils.h"
-#include "miscadmin.h"
-#include "parser/parsetree.h"
-#include "cdb/cdbvars.h" /* gp_select_invisible */
-#include "nodes/tidbitmap.h"
-
-static TupleTableSlot *BitmapAppendOnlyScanNext(BitmapAppendOnlyScanState *node);
-
-/*
- * Initialize the fetch descriptor for the BitmapAppendOnlyScanState if
- * it is not initialized.
- */
-static void
-initFetchDesc(BitmapAppendOnlyScanState *scanstate)
-{
-	BitmapAppendOnlyScan *node = (BitmapAppendOnlyScan *)(scanstate->ss.ps.plan);
-	Relation currentRelation = scanstate->ss.ss_currentRelation;
-	EState *estate = scanstate->ss.ps.state;
-
-	if (node->isAORow)
-	{
-		if (scanstate->baos_currentAOFetchDesc == NULL)
-		{
-			scanstate->baos_currentAOFetchDesc = 
-				appendonly_fetch_init(currentRelation,
-									  estate->es_snapshot);
-		}
-	}
-	
-}
-
-/*
- * Free fetch descriptor.
- */
-static inline void
-freeFetchDesc(BitmapAppendOnlyScanState *scanstate)
-{
-	if (scanstate->baos_currentAOFetchDesc != NULL)
-	{
-		Assert(((BitmapAppendOnlyScan *)(scanstate->ss.ps.plan))->isAORow);
-		appendonly_fetch_finish(scanstate->baos_currentAOFetchDesc);
-		pfree(scanstate->baos_currentAOFetchDesc);
-		scanstate->baos_currentAOFetchDesc = NULL;
-	}
-
-}
-
-/*
- * Initialize the state relevant to bitmaps.
- */
-static inline void
-initBitmapState(BitmapAppendOnlyScanState *scanstate)
-{
-	if (scanstate->baos_tbmres == NULL)
-	{
-		scanstate->baos_tbmres =
-			palloc(sizeof(TBMIterateResult) +
-					MAX_TUPLES_PER_PAGE * sizeof(OffsetNumber));
-
-		/* initialize result header */
-		MemSetAligned(scanstate->baos_tbmres, 0, sizeof(TBMIterateResult));
-	}
-}
-
-/*
- * Free the state relevant to bitmaps
- */
-static inline void
-freeBitmapState(BitmapAppendOnlyScanState *scanstate)
-{
-	if (scanstate->baos_tbm != NULL)
-	{
-		if(IsA(scanstate->baos_tbm, HashBitmap))
-			tbm_free((HashBitmap *)scanstate->baos_tbm);
-		else
-            tbm_bitmap_free(scanstate->baos_tbm);
-
-		scanstate->baos_tbm = NULL;
-	}
-	if (scanstate->baos_tbmres != NULL)
-	{
-		pfree(scanstate->baos_tbmres);
-		scanstate->baos_tbmres = NULL;
-	}
-}
-
-/* ----------------------------------------------------------------
- *		BitmapAppendOnlyNext
- *
- *		Retrieve next tuple from the BitmapAppendOnlyScan node's currentRelation
- * ----------------------------------------------------------------
- */
-static TupleTableSlot *
-BitmapAppendOnlyScanNext(BitmapAppendOnlyScanState *node)
-{
-	EState	   *estate;
-	ExprContext *econtext;
-	AppendOnlyFetchDesc aoFetchDesc;
-	Index		scanrelid;
-	Node  		*tbm;
-	TBMIterateResult *tbmres;
-	OffsetNumber psuedoHeapOffset;
-	ItemPointerData psudeoHeapTid;
-	AOTupleId aoTid;
-	TupleTableSlot *slot;
-
-	/*
-	 * extract necessary information from index scan node
-	 */
-	estate = node->ss.ps.state;
-	econtext = node->ss.ps.ps_ExprContext;
-	slot = node->ss.ss_ScanTupleSlot;
-
-	initBitmapState(node);
-	initFetchDesc(node);
-
-	aoFetchDesc = node->baos_currentAOFetchDesc;
-	scanrelid = ((BitmapAppendOnlyScan *) node->ss.ps.plan)->scan.scanrelid;
-	tbm = node->baos_tbm;
-	tbmres = (TBMIterateResult *) node->baos_tbmres;
-	Assert(tbmres != NULL);
-
-	/*
-	 * Check if we are evaluating PlanQual for tuple of this relation.
-	 * Additional checking is not good, but no other way for now. We could
-	 * introduce new nodes for this case and handle IndexScan --> NewNode
-	 * switching in Init/ReScan plan...
-	 */
-	if (estate->es_evTuple != NULL &&
-		estate->es_evTuple[scanrelid - 1] != NULL)
-	{
-		if (estate->es_evTupleNull[scanrelid - 1])
-		{
-			freeFetchDesc(node);
-			freeBitmapState(node);
-			
-			return ExecClearTuple(slot);
-		}
-
-		ExecStoreGenericTuple(estate->es_evTuple[scanrelid - 1],
-					   slot, false);
-
-		/* Does the tuple meet the original qual conditions? */
-		econtext->ecxt_scantuple = slot;
-
-		ResetExprContext(econtext);
-
-		if (!ExecQual(node->baos_bitmapqualorig, econtext, false))
-		{
-			ExecEagerFreeBitmapAppendOnlyScan(node);
-
-			ExecClearTuple(slot);		/* would not be returned by scan */
-		}
-
-		/* Flag for the next call that no more tuples */
-		estate->es_evTupleNull[scanrelid - 1] = true;
-
-		if (!TupIsNull(slot))
-		{
-			Gpmon_M_Incr_Rows_Out(GpmonPktFromBitmapAppendOnlyScanState(node));
-			CheckSendPlanStateGpmonPkt(&node->ss.ps);
-		}
-		return slot;
-	}
-
-	/*
-	 * If we haven't yet performed the underlying index scan, or
-	 * we have used up the bitmaps from the previous scan, do the next scan,
-	 * and prepare the bitmap to be iterated over.
- 	 */
-	if (tbm == NULL)
-	{
-		tbm = (Node *) MultiExecProcNode(outerPlanState(node));
-
-		if (tbm != NULL && (!(IsA(tbm, HashBitmap) ||
-							  IsA(tbm, StreamBitmap))))
-			elog(ERROR, "unrecognized result from subplan");
-
-		/* When a HashBitmap is returned, set the returning bitmaps
-		 * in the subplan to NULL, so that the subplan nodes do not
-		 * mistakenly try to release the space during the rescan.
-		 */
-		if (tbm != NULL && IsA(tbm, HashBitmap))
-			tbm_reset_bitmaps(outerPlanState(node));
-
-		node->baos_tbm = tbm;
-	}
-
-	if (tbm == NULL)
-	{
-		ExecEagerFreeBitmapAppendOnlyScan(node);
-
-		return ExecClearTuple(slot);
-	}
-
-	Assert(tbm != NULL);
-	Assert(tbmres != NULL);
-
-	for (;;)
-	{
-		CHECK_FOR_INTERRUPTS();
-
-		if (!node->baos_gotpage)
-		{
-			/*
-			 * Obtain the next psuedo-heap-page-info with item bit-map.  Later, we'll
-			 * convert the (psuedo) heap block number and item number to an
-			 * Append-Only TID.
-			 */
-			if (!tbm_iterate(tbm, tbmres))
-			{
-				/* no more entries in the bitmap */
-				break;
-			}
-
-			/* If tbmres contains no tuples, continue. */
-			if (tbmres->ntuples == 0)
-				continue;
-
-			Gpmon_M_Incr(GpmonPktFromBitmapAppendOnlyScanState(node), GPMON_BITMAPAPPENDONLYSCAN_PAGE);
-			CheckSendPlanStateGpmonPkt(&node->ss.ps);
-
-		 	node->baos_gotpage = true;
-
-			/*
-		 	* Set cindex to first slot to examine
-		 	*/
-			node->baos_cindex = 0;
-
-			node->baos_lossy = (tbmres->ntuples < 0);
-			if (!node->baos_lossy)
-				node->baos_ntuples = tbmres->ntuples;
-			else
-				node->baos_ntuples = MAX_TUPLES_PER_PAGE;
-				
-		}
-		else
-		{
-			/*
-			 * Continuing in previously obtained page; advance cindex
-			 */
-			node->baos_cindex++;
-		}
-
-		/*
-		 * Out of range?  If so, nothing more to look at on this page
-		 */
-		if (node->baos_cindex < 0 || node->baos_cindex >= node->baos_ntuples)
-		{
-		 	node->baos_gotpage = false;
-			continue;
-		}
-
-		/*
-		 * Must account for lossy page info...
-		 */
-		if (node->baos_lossy)
-			psuedoHeapOffset = node->baos_cindex;	// We are iterating through all items.
-		else
-		{
-			Assert(node->baos_cindex <= tbmres->ntuples);
-			psuedoHeapOffset = tbmres->offsets[node->baos_cindex];
-		}
-
-		/*
-		 * Okay to fetch the tuple
-		 */
-		ItemPointerSet(
-				&psudeoHeapTid, 
-				tbmres->blockno, 
-				psuedoHeapOffset);
-
-		tbm_convert_appendonly_tid_out(&psudeoHeapTid, &aoTid);
-
-		if (aoFetchDesc != NULL)
-		{
-			appendonly_fetch(aoFetchDesc, &aoTid, slot);
-		}
-		
-      	if (TupIsNull(slot))
-			continue;
-
-		pgstat_count_heap_fetch(node->ss.ss_currentRelation);
-
-		/*
-		 * If we are using lossy info, we have to recheck the qual
-		 * conditions at every tuple.
-		 */
-		if (node->baos_lossy)
-		{
-			econtext->ecxt_scantuple = slot;
-			ResetExprContext(econtext);
-
-			if (!ExecQual(node->baos_bitmapqualorig, econtext, false))
-			{
-				/* Fails recheck, so drop it and loop back for another */
-				ExecClearTuple(slot);
-				continue;
-			}
-		}
-
-		/* OK to return this tuple */
-      	if (!TupIsNull(slot))
-		{
-			Gpmon_M_Incr_Rows_Out(GpmonPktFromBitmapAppendOnlyScanState(node));
-			CheckSendPlanStateGpmonPkt(&node->ss.ps);
-		}
-
-		return slot;
-	}
-
-	/*
-	 * if we get here it means we are at the end of the scan..
-	 */
-	ExecEagerFreeBitmapAppendOnlyScan(node);
-
-	return ExecClearTuple(slot);
-}
-
-/* ----------------------------------------------------------------
- *		ExecBitmapAppendOnlyScan(node)
- * ----------------------------------------------------------------
- */
-TupleTableSlot *
-ExecBitmapAppendOnlyScan(BitmapAppendOnlyScanState *node)
-{
-	/*
-	 * use BitmapAppendOnlyNext as access method
-	 */
-	return ExecScan(&node->ss, (ExecScanAccessMtd) BitmapAppendOnlyScanNext);
-}
-
-/* ----------------------------------------------------------------
- *		ExecBitmapAppendOnlyReScan(node)
- * ----------------------------------------------------------------
- */
-void
-ExecBitmapAppendOnlyReScan(BitmapAppendOnlyScanState *node, ExprContext *exprCtxt)
-{
-	EState	   *estate;
-	Index		scanrelid;
-
-	estate = node->ss.ps.state;
-	scanrelid = ((BitmapAppendOnlyScan *) node->ss.ps.plan)->scan.scanrelid;
-
-	/* node->aofs.ps.ps_TupFromTlist = false; */
-
-	/*
-	 * If we are being passed an outer tuple, link it into the "regular"
-	 * per-tuple econtext for possible qual eval.
-	 */
-	if (exprCtxt != NULL)
-	{
-		ExprContext *stdecontext;
-
-		stdecontext = node->ss.ps.ps_ExprContext;
-		stdecontext->ecxt_outertuple = exprCtxt->ecxt_outertuple;
-	}
-
-	/* If this is re-scanning of PlanQual ... */
-	if (estate->es_evTuple != NULL &&
-		estate->es_evTuple[scanrelid - 1] != NULL)
-	{
-		estate->es_evTupleNull[scanrelid - 1] = false;
-	}
-
-	/*
-	 * NOTE: The appendonly_fetch routine can fetch randomly, so no need to reset it.
-	 */
-
-	freeBitmapState(node);
-	tbm_reset_bitmaps(outerPlanState(node));
-
-	/*
-	 * Always rescan the input immediately, to ensure we can pass down any
-	 * outer tuple that might be used in index quals.
-	 */
-	Gpmon_M_Incr(GpmonPktFromBitmapAppendOnlyScanState(node), GPMON_BITMAPAPPENDONLYSCAN_RESCAN);
-	CheckSendPlanStateGpmonPkt(&node->ss.ps);
-
-	ExecReScan(outerPlanState(node), exprCtxt);
-}
-
-/* ----------------------------------------------------------------
- *		ExecEndBitmapAppendOnlyScan
- * ----------------------------------------------------------------
- */
-void
-ExecEndBitmapAppendOnlyScan(BitmapAppendOnlyScanState *node)
-{
-	Relation	relation;
-
-	/*
-	 * extract information from the node
-	 */
-	relation = node->ss.ss_currentRelation;
-
-	/*
-	 * Free the exprcontext
-	 */
-	ExecFreeExprContext(&node->ss.ps);
-
-	/*
-	 * clear out tuple table slots
-	 */
-	ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
-	ExecClearTuple(node->ss.ss_ScanTupleSlot);
-
-	/*
-	 * close down subplans
-	 */
-	ExecEndNode(outerPlanState(node));
-
-	ExecEagerFreeBitmapAppendOnlyScan(node);
-	
-	/*
-	 * close the heap relation.
-	 */
-	ExecCloseScanRelation(relation);
-
-	node->baos_gotpage = false;
-	node->baos_lossy = false;
-	node->baos_cindex = 0;
-	node->baos_ntuples = 0;
-
-	EndPlanStateGpmonPkt(&node->ss.ps);
-}
-
-/* ----------------------------------------------------------------
- *		ExecInitBitmapAppendOnlyScan
- *
- *		Initializes the scan's state information.
- * ----------------------------------------------------------------
- */
-BitmapAppendOnlyScanState *
-ExecInitBitmapAppendOnlyScan(BitmapAppendOnlyScan *node, EState *estate, int eflags)
-{
-	BitmapAppendOnlyScanState *scanstate;
-	Relation	currentRelation;
-
-	/* check for unsupported flags */
-	Assert(!(eflags & (EXEC_FLAG_BACKWARD | EXEC_FLAG_MARK)));
-
-	Assert(IsA(node, BitmapAppendOnlyScan));
-
-	/*
-	 * Assert caller didn't ask for an unsafe snapshot --- see comments at
-	 * head of file.
-	 *
-	 * MPP-4703: the MVCC-snapshot restriction is required for correct results.
-	 * our test-mode may deliberately return incorrect results, but that's OK.
-	 */
-	Assert(IsMVCCSnapshot(estate->es_snapshot) || gp_select_invisible);
-
-	/*
-	 * create state structure
-	 */
-	scanstate = makeNode(BitmapAppendOnlyScanState);
-	scanstate->ss.ps.plan = (Plan *) node;
-	scanstate->ss.ps.state = estate;
-
-	scanstate->baos_tbm = NULL;
-	scanstate->baos_tbmres = NULL;
-	scanstate->baos_gotpage = false;
-	scanstate->baos_lossy = false;
-	scanstate->baos_cindex = 0;
-	scanstate->baos_ntuples = 0;
-
-	/*
-	 * Miscellaneous initialization
-	 *
-	 * create expression context for node
-	 */
-	ExecAssignExprContext(estate, &scanstate->ss.ps);
-
-	/* scanstate->aofs.ps.ps_TupFromTlist = false;*/
-
-	/*
-	 * initialize child expressions
-	 */
-	scanstate->ss.ps.targetlist = (List *)
-		ExecInitExpr((Expr *) node->scan.plan.targetlist,
-					 (PlanState *) scanstate);
-	scanstate->ss.ps.qual = (List *)
-		ExecInitExpr((Expr *) node->scan.plan.qual,
-					 (PlanState *) scanstate);
-	scanstate->baos_bitmapqualorig = (List *)
-		ExecInitExpr((Expr *) node->bitmapqualorig,
-					 (PlanState *) scanstate);
-
-#define BITMAPAPPENDONLYSCAN_NSLOTS 2
-
-	/*
-	 * tuple table initialization
-	 */
-	ExecInitResultTupleSlot(estate, &scanstate->ss.ps);
-	ExecInitScanTupleSlot(estate, &scanstate->ss);
-
-	/*
-	 * open the base relation and acquire appropriate lock on it.
-	 */
-	currentRelation = ExecOpenScanRelation(estate, node->scan.scanrelid);
-
-	scanstate->ss.ss_currentRelation = currentRelation;
-
-	/*
-	 * get the scan type from the relation descriptor.
-	 */
-	ExecAssignScanType(&scanstate->ss, RelationGetDescr(currentRelation));
-
-	/*
-	 * Initialize result tuple type and projection info.
-	 */
-	ExecAssignResultTypeFromTL(&scanstate->ss.ps);
-	ExecAssignScanProjectionInfo(&scanstate->ss);
-
-	scanstate->baos_currentAOFetchDesc = NULL;
-	
-	/*
-	 * initialize child nodes
-	 *
-	 * We do this last because the child nodes will open indexscans on our
-	 * relation's indexes, and we want to be sure we have acquired a lock on
-	 * the relation first.
-	 */
-	outerPlanState(scanstate) = ExecInitNode(outerPlan(node), estate, eflags);
-
-	initGpmonPktForBitmapAppendOnlyScan((Plan *)node, &scanstate->ss.ps.gpmon_pkt, estate);
-
-	/*
-	 * all done.
-	 */
-	return scanstate;
-}
-
-int
-ExecCountSlotsBitmapAppendOnlyScan(BitmapAppendOnlyScan *node)
-{
-	return ExecCountSlotsNode(outerPlan((Plan *) node)) +
-		ExecCountSlotsNode(innerPlan((Plan *) node)) + BITMAPAPPENDONLYSCAN_NSLOTS;
-}
-
-void
-initGpmonPktForBitmapAppendOnlyScan(Plan *planNode, gpmon_packet_t *gpmon_pkt, EState *estate)
-{
-	Assert(planNode != NULL && gpmon_pkt != NULL && IsA(planNode, BitmapAppendOnlyScan));
-
-	{
-		RangeTblEntry *rte = rt_fetch(((BitmapAppendOnlyScan *)planNode)->scan.scanrelid,
-									  estate->es_range_table);
-		char schema_rel_name[SCAN_REL_NAME_BUF_SIZE] = {0};
-		
-		Assert(GPMON_BITMAPAPPENDONLYSCAN_TOTAL <= (int)GPMON_QEXEC_M_COUNT);
-		InitPlanNodeGpmonPkt(planNode, gpmon_pkt, estate, PMNT_BitmapAppendOnlyScan,
-							 (int64)planNode->plan_rows,
-							 GetScanRelNameGpmon(rte->relid, schema_rel_name));
-	}
-}
-
-void
-ExecEagerFreeBitmapAppendOnlyScan(BitmapAppendOnlyScanState *node)
-{
-	freeFetchDesc(node);
-	freeBitmapState(node);
-}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/gp_libpq_fe/fe-protocol3.c
----------------------------------------------------------------------
diff --git a/src/backend/gp_libpq_fe/fe-protocol3.c b/src/backend/gp_libpq_fe/fe-protocol3.c
index ac4eeb9..4143d49 100644
--- a/src/backend/gp_libpq_fe/fe-protocol3.c
+++ b/src/backend/gp_libpq_fe/fe-protocol3.c
@@ -610,12 +610,6 @@ pqParseInput3(PGconn *conn)
 							if (pqGetInt64(&(sendback[i].uncompressed_eof[j]), conn))
 										return;
 						}
-
-						/*
-						 * 10, get fast sequence.
-						 */
-						if (pqGetInt64(&(sendback[i].nextFastSequence), conn))
-								return;
 					}
 				}
 					break;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/nodes/copyfuncs.c
----------------------------------------------------------------------
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 94527b4..8f48008 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -587,28 +587,6 @@ _copyBitmapHeapScan(BitmapHeapScan *from)
 }
 
 /*
- * _copyBitmapAppendOnlyScan
- */
-static BitmapAppendOnlyScan *
-_copyBitmapAppendOnlyScan(BitmapAppendOnlyScan *from)
-{
-	BitmapAppendOnlyScan *newnode = makeNode(BitmapAppendOnlyScan);
-
-	/*
-	 * copy node superclass fields
-	 */
-	CopyScanFields((Scan *) from, (Scan *) newnode);
-
-	/*
-	 * copy remainder of node
-	 */
-	COPY_NODE_FIELD(bitmapqualorig);
-	COPY_SCALAR_FIELD(isAORow);
-
-	return newnode;
-}
-
-/*
  * _copyBitmapTableScan
  */
 static BitmapTableScan *
@@ -4449,9 +4427,6 @@ copyObject(void *from)
 		case T_BitmapHeapScan:
 			retval = _copyBitmapHeapScan(from);
 			break;
-		case T_BitmapAppendOnlyScan:
-			retval = _copyBitmapAppendOnlyScan(from);
-			break;
 		case T_BitmapTableScan:
 			retval = _copyBitmapTableScan(from);
 			break;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/nodes/outfast.c
----------------------------------------------------------------------
diff --git a/src/backend/nodes/outfast.c b/src/backend/nodes/outfast.c
index 4600c88..ac943c9 100644
--- a/src/backend/nodes/outfast.c
+++ b/src/backend/nodes/outfast.c
@@ -689,17 +689,6 @@ _outBitmapHeapScan(StringInfo str, BitmapHeapScan *node)
 }
 
 static void
-_outBitmapAppendOnlyScan(StringInfo str, BitmapAppendOnlyScan *node)
-{
-	WRITE_NODE_TYPE("BITMAPAPPENDONLYSCAN");
-
-	_outScanInfo(str, (Scan *) node);
-
-	WRITE_LIST_FIELD(bitmapqualorig);
-	WRITE_BOOL_FIELD(isAORow);
-}
-
-static void
 _outBitmapTableScan(StringInfo str, BitmapTableScan *node)
 {
 	WRITE_NODE_TYPE("BITMAPTABLESCAN");
@@ -3997,9 +3986,6 @@ _outNode(StringInfo str, void *obj)
 			case T_BitmapHeapScan:
 				_outBitmapHeapScan(str, obj);
 				break;
-			case T_BitmapAppendOnlyScan:
-				_outBitmapAppendOnlyScan(str, obj);
-				break;
 			case T_BitmapTableScan:
 				_outBitmapTableScan(str, obj);
 				break;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/nodes/outfuncs.c
----------------------------------------------------------------------
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 2b6aad2..0f35610 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -4176,9 +4176,6 @@ _outNode(StringInfo str, void *obj)
 			case T_BitmapHeapScan:
 				_outBitmapHeapScan(str, obj);
 				break;
-			case T_BitmapAppendOnlyScan:
-				_outBitmapAppendOnlyScan(str, obj);
-				break;
 			case T_BitmapTableScan:
 				_outBitmapTableScan(str, obj);
 				break;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/nodes/print.c
----------------------------------------------------------------------
diff --git a/src/backend/nodes/print.c b/src/backend/nodes/print.c
index 75e015f..03367b0 100644
--- a/src/backend/nodes/print.c
+++ b/src/backend/nodes/print.c
@@ -529,8 +529,6 @@ char * plannode_type(Plan *p)
 			return "FUNCTIONSCAN";
 		case T_ValuesScan:
 			return "VALUESSCAN";
-		case T_BitmapAppendOnlyScan:
-			return "BITMAPAPPENDONLYSCAN";
 		case T_BitmapTableScan:
 			return "BITMAPTABLESCAN";
 		case T_Join:
@@ -588,8 +586,7 @@ print_plan_recursive(struct Plan *p, struct Query *parsetree, int indentLevel, c
 		   p->plan_rows, p->plan_width);
 	if (IsA(p, Scan) ||
 		IsA(p, SeqScan) ||
-		IsA(p, BitmapHeapScan) ||
-		IsA(p, BitmapAppendOnlyScan))
+		IsA(p, BitmapHeapScan))
 	{
 		RangeTblEntry *rte;
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/nodes/readfast.c
----------------------------------------------------------------------
diff --git a/src/backend/nodes/readfast.c b/src/backend/nodes/readfast.c
index 38caa38..cfdcc06 100644
--- a/src/backend/nodes/readfast.c
+++ b/src/backend/nodes/readfast.c
@@ -3195,19 +3195,6 @@ _readBitmapHeapScan(const char ** str)
 	READ_DONE();
 }
 
-static BitmapAppendOnlyScan *
-_readBitmapAppendOnlyScan(const char ** str)
-{
-	READ_LOCALS(BitmapAppendOnlyScan);
-
-	readScanInfo(str, (Scan *)local_node);
-
-	READ_NODE_FIELD(bitmapqualorig);
-	READ_BOOL_FIELD(isAORow);
-
-	READ_DONE();
-}
-
 static BitmapTableScan *
 _readBitmapTableScan(const char ** str)
 {
@@ -4268,9 +4255,6 @@ readNodeBinary(const char ** str)
 			case T_BitmapHeapScan:
 				return_value = _readBitmapHeapScan(str);
 				break;
-			case T_BitmapAppendOnlyScan:
-				return_value = _readBitmapAppendOnlyScan(str);
-				break;
 			case T_BitmapTableScan:
 				return_value = _readBitmapTableScan(str);
 				break;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/optimizer/path/indxpath.c
----------------------------------------------------------------------
diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c
index 25d2114..e8baea8 100644
--- a/src/backend/optimizer/path/indxpath.c
+++ b/src/backend/optimizer/path/indxpath.c
@@ -157,9 +157,6 @@ create_bitmap_scan_path(char relstorage,
 		case RELSTORAGE_HEAP:
 			path = (Path *)create_bitmap_heap_path(root, rel, bitmapqual, outer_rel);
 			break;
-		case RELSTORAGE_AOROWS:
-			path = (Path *)create_bitmap_appendonly_path(root, rel, bitmapqual, outer_rel, true);
-			break;
 		default:
 			elog(ERROR, "unrecognized relstorage type %d for using bitmap scan path",
 				 relstorage);

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/optimizer/plan/createplan.c
----------------------------------------------------------------------
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 09da978..ba6069d 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -119,9 +119,6 @@ static TableFunctionScan *create_tablefunction_plan(CreatePlanContext *ctx,
 													List *scan_clauses);
 static ValuesScan *create_valuesscan_plan(CreatePlanContext *ctx, Path *best_path,
 					   List *tlist, List *scan_clauses);
-static BitmapAppendOnlyScan *create_bitmap_appendonly_scan_plan(CreatePlanContext *ctx,
-						BitmapAppendOnlyPath *best_path,
-						List *tlist, List *scan_clauses);
 static Plan *create_nestloop_plan(CreatePlanContext *ctx, NestPath *best_path,
 					 Plan *outer_plan, Plan *inner_plan);
 static MergeJoin *create_mergejoin_plan(CreatePlanContext *ctx, MergePath *best_path,
@@ -167,12 +164,6 @@ static BitmapHeapScan *make_bitmap_heapscan(List *qptlist,
 					 Plan *lefttree,
 					 List *bitmapqualorig,
 					 Index scanrelid);
-static BitmapAppendOnlyScan *make_bitmap_appendonlyscan(List *qptlist,
-														List *qpqual,
-														Plan *lefttree,
-														List *bitmapqualorig,
-														Index scanrelid,
-														bool isAORow);
 static TableFunctionScan* make_tablefunction(List *tlist,
 											 List *scan_quals,
 											 Plan *subplan,
@@ -250,7 +241,6 @@ create_subplan(CreatePlanContext *ctx, Path *best_path)
 		case T_AppendOnlyScan:
 		case T_ParquetScan:
 		case T_BitmapHeapScan:
-		case T_BitmapAppendOnlyScan:
 		case T_BitmapTableScan:
 		case T_TidScan:
 		case T_SubqueryScan:
@@ -383,13 +373,6 @@ create_scan_plan(CreatePlanContext *ctx, Path *best_path)
 													scan_clauses);
 			break;
 
-		case T_BitmapAppendOnlyScan:
-			plan = (Plan *) create_bitmap_appendonly_scan_plan(ctx,
-												    (BitmapAppendOnlyPath *) best_path,
-													tlist,
-													scan_clauses);
-			break;
-
 		case T_TidScan:
 			plan = (Plan *) create_tidscan_plan(ctx,
 												(TidPath *) best_path,
@@ -565,7 +548,6 @@ disuse_physical_tlist(Plan *plan, Path *path)
 		case T_ExternalScan:
 		case T_IndexScan:
 		case T_BitmapHeapScan:
-		case T_BitmapAppendOnlyScan:
 		case T_BitmapTableScan:
 		case T_TidScan:
 		case T_SubqueryScan:
@@ -2196,118 +2178,6 @@ create_bitmap_scan_plan(CreatePlanContext *ctx,
 }
 
 /*
- * create_bitmap_appendonly_scan_plan
- *
- * NOTE: Copy of create_bitmap_scan_plan routine.
- */
-static BitmapAppendOnlyScan *
-create_bitmap_appendonly_scan_plan(CreatePlanContext *ctx,
-						BitmapAppendOnlyPath *best_path,
-						List *tlist,
-						List *scan_clauses)
-{
-	Index		baserelid = best_path->path.parent->relid;
-	Plan	   *bitmapqualplan;
-	List	   *bitmapqualorig = NULL;
-	List	   *indexquals = NULL;
-	List	   *qpqual;
-	ListCell   *l;
-	BitmapAppendOnlyScan *scan_plan;
-
-	/* it should be a base rel... */
-	Assert(baserelid > 0);
-	Assert(best_path->path.parent->rtekind == RTE_RELATION);
-
-	/* Process the bitmapqual tree into a Plan tree and qual lists */
-	bitmapqualplan = create_bitmap_subplan(ctx, best_path->bitmapqual,
-										   &bitmapqualorig, &indexquals);
-
-	/* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
-	scan_clauses = extract_actual_clauses(scan_clauses, false);
-
-	/*
-	 * If this is a innerjoin scan, the indexclauses will contain join clauses
-	 * that are not present in scan_clauses (since the passed-in value is just
-	 * the rel's baserestrictinfo list).  We must add these clauses to
-	 * scan_clauses to ensure they get checked.  In most cases we will remove
-	 * the join clauses again below, but if a join clause contains a special
-	 * operator, we need to make sure it gets into the scan_clauses.
-	 */
-	if (best_path->isjoininner)
-	{
-		scan_clauses = list_concat_unique(scan_clauses, bitmapqualorig);
-	}
-
-	/*
-	 * The qpqual list must contain all restrictions not automatically handled
-	 * by the index.  All the predicates in the indexquals will be checked
-	 * (either by the index itself, or by nodeBitmapHeapscan.c), but if there
-	 * are any "special" or lossy operators involved then they must be added
-	 * to qpqual.  The upshot is that qpqual must contain scan_clauses minus
-	 * whatever appears in indexquals.
-	 *
-	 * In normal cases simple equal() checks will be enough to spot duplicate
-	 * clauses, so we try that first.  In some situations (particularly with
-	 * OR'd index conditions) we may have scan_clauses that are not equal to,
-	 * but are logically implied by, the index quals; so we also try a
-	 * predicate_implied_by() check to see if we can discard quals that way.
-	 * (predicate_implied_by assumes its first input contains only immutable
-	 * functions, so we have to check that.)
-	 *
-	 * Unlike create_indexscan_plan(), we need take no special thought here
-	 * for partial index predicates; this is because the predicate conditions
-	 * are already listed in bitmapqualorig and indexquals.  Bitmap scans have
-	 * to do it that way because predicate conditions need to be rechecked if
-	 * the scan becomes lossy.
-	 */
-	qpqual = NIL;
-	foreach(l, scan_clauses)
-	{
-		Node	   *clause = (Node *) lfirst(l);
-
-		if (list_member(indexquals, clause))
-			continue;
-		if (!contain_mutable_functions(clause))
-		{
-			List	   *clausel = list_make1(clause);
-
-			if (predicate_implied_by(clausel, indexquals))
-				continue;
-		}
-		qpqual = lappend(qpqual, clause);
-	}
-
-	/* Sort clauses into best execution order */
-	qpqual = order_qual_clauses(ctx->root, qpqual);
-
-	/*
-	 * When dealing with special or lossy operators, we will at this point
-	 * have duplicate clauses in qpqual and bitmapqualorig.  We may as well
-	 * drop 'em from bitmapqualorig, since there's no point in making the
-	 * tests twice.
-	 */
-	bitmapqualorig = list_difference_ptr(bitmapqualorig, qpqual);
-
-	/*
-	 * Copy the finished bitmapqualorig to make sure we have an independent
-	 * copy --- needed in case there are subplans in the index quals
-	 */
-	bitmapqualorig = copyObject(bitmapqualorig);
-
-	/* Finally ready to build the plan node */
-	scan_plan = make_bitmap_appendonlyscan(tlist,
-										   qpqual,
-										   bitmapqualplan,
-										   bitmapqualorig,
-										   baserelid,
-										   best_path->isAORow);
-
-	copy_path_costsize(ctx->root, &scan_plan->scan.plan, &best_path->path);
-
-	return scan_plan;
-}
-
-/*
  * Given a bitmapqual tree, generate the Plan tree that implements it
  *
  * As byproducts, we also return in *qual and *indexqual the qual lists
@@ -3750,30 +3620,6 @@ make_bitmap_heapscan(List *qptlist,
 	return node;
 }
 
-static BitmapAppendOnlyScan *
-make_bitmap_appendonlyscan(List *qptlist,
-						   List *qpqual,
-						   Plan *lefttree,
-						   List *bitmapqualorig,
-						   Index scanrelid,
-						   bool isAORow)
-{
-	BitmapAppendOnlyScan *node = makeNode(BitmapAppendOnlyScan);
-	Plan	   *plan = &node->scan.plan;
-
-	/* cost should be inserted by caller */
-	plan->targetlist = qptlist;
-	plan->qual = qpqual;
-	plan->lefttree = lefttree;
-	plan->righttree = NULL;
-	node->scan.scanrelid = scanrelid;
-
-	node->bitmapqualorig = bitmapqualorig;
-	node->isAORow = isAORow;
-
-	return node;
-}
-
 static TidScan *
 make_tidscan(List *qptlist,
 			 List *qpqual,

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/optimizer/plan/planpartition.c
----------------------------------------------------------------------
diff --git a/src/backend/optimizer/plan/planpartition.c b/src/backend/optimizer/plan/planpartition.c
index 9940de6..bf92b28 100644
--- a/src/backend/optimizer/plan/planpartition.c
+++ b/src/backend/optimizer/plan/planpartition.c
@@ -781,7 +781,6 @@ AdjustVarnoWalker(Node *node, AdjustVarnoContext *ctx)
 		case T_IndexScan:
 		case T_BitmapIndexScan:
 		case T_BitmapHeapScan:
-		case T_BitmapAppendOnlyScan:
 		case T_BitmapTableScan:
 		case T_TidScan:
 		case T_FunctionScan:

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/optimizer/plan/setrefs.c
----------------------------------------------------------------------
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 461e39e..afce5a5 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -568,30 +568,6 @@ set_plan_refs(PlannerGlobal *glob, Plan *plan, const int rtoffset)
 			fix_scan_list(glob, splan->bitmapqualorig, rtoffset);
 		}
 			break;
-		case T_BitmapAppendOnlyScan:
-		{
-			BitmapAppendOnlyScan *splan = (BitmapAppendOnlyScan *) plan;
-			
-			if (cdb_expr_requires_full_eval((Node *)plan->targetlist))
-				return  cdb_insert_result_node(glob, plan, rtoffset);
-			
-			splan->scan.scanrelid += rtoffset;
-
-#ifdef USE_ASSERT_CHECKING
-			RangeTblEntry *rte = rt_fetch(splan->scan.scanrelid, glob->finalrtable);
-			char relstorage = get_rel_relstorage(rte->relid);
-			Assert(relstorage == RELSTORAGE_AOROWS ||
-				   relstorage == RELSTORAGE_PARQUET);
-#endif
-
-			splan->scan.plan.targetlist =
-			fix_scan_list(glob, splan->scan.plan.targetlist, rtoffset);
-			splan->scan.plan.qual =
-			fix_scan_list(glob, splan->scan.plan.qual, rtoffset);
-			splan->bitmapqualorig =
-			fix_scan_list(glob, splan->bitmapqualorig, rtoffset);
-		}
-			break;
 		case T_BitmapTableScan:
 		{
 			BitmapTableScan *splan = (BitmapTableScan *) plan;
@@ -1473,8 +1449,7 @@ set_inner_join_references(PlannerGlobal *glob, Plan *inner_plan,
 			Assert(inner_plan->qual == NIL);
 		}
 	}
-	else if (IsA(inner_plan, BitmapHeapScan) ||
-			 IsA(inner_plan, BitmapAppendOnlyScan))
+	else if (IsA(inner_plan, BitmapHeapScan))
 	{
 		/*
 		 * The inner side is a bitmap scan plan.  Fix the top node, and
@@ -1491,14 +1466,6 @@ set_inner_join_references(PlannerGlobal *glob, Plan *inner_plan,
 			innerrel = innerscan->scan.scanrelid;
 			bitmapqualorig_p = &(innerscan->bitmapqualorig);
 		}
-		else
-		{
-			Assert(IsA(inner_plan, BitmapAppendOnlyScan));
-			
-			BitmapAppendOnlyScan *innerscan = (BitmapAppendOnlyScan *) inner_plan;
-			innerrel = innerscan->scan.scanrelid;
-			bitmapqualorig_p = &(innerscan->bitmapqualorig);
-		}
 		
 		/* only refs to outer vars get changed in the inner qual */
 		if (NumRelids((Node *) (*bitmapqualorig_p)) > 1)

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/ae38cfbd/src/backend/optimizer/plan/subselect.c
----------------------------------------------------------------------
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index 9e3da25..0289c59 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -1193,11 +1193,6 @@ finalize_plan(PlannerInfo *root, Plan *plan, List *rtable,
 							  &context);
 			break;
 
-		case T_BitmapAppendOnlyScan:
-			finalize_primnode((Node *) ((BitmapAppendOnlyScan *) plan)->bitmapqualorig,
-							  &context);
-			break;
-
 		case T_BitmapTableScan:
 			finalize_primnode((Node *) ((BitmapTableScan *) plan)->bitmapqualorig,
 							  &context);