diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 379fc5c..b894454 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -947,9 +947,9 @@ ExplainNode(PlanState *planstate, List *ancestors,
 			{
 				Agg		   *agg = (Agg *) plan;
 
-				if (agg->finalizeAggs == false)
+				if (!DO_AGGMODE_FINALIZE(agg->aggmode))
 					operation = "Partial";
-				else if (agg->combineStates == true)
+				else if (DO_AGGMODE_COMBINE(agg->aggmode))
 					operation = "Finalize";
 
 				switch (agg->aggstrategy)
diff --git a/src/backend/executor/execQual.c b/src/backend/executor/execQual.c
index 01e04d3..5b9b11a 100644
--- a/src/backend/executor/execQual.c
+++ b/src/backend/executor/execQual.c
@@ -4519,7 +4519,7 @@ ExecInitExpr(Expr *node, PlanState *parent)
 					/* planner messed up */
 					elog(ERROR, "Aggref found in non-Agg plan node");
 				}
-				if (aggref->aggpartial == aggstate->finalizeAggs)
+				if (aggref->aggpartial == DO_AGGMODE_FINALIZE(aggstate->aggmode))
 				{
 					/* planner messed up */
 					if (aggref->aggpartial)
@@ -4528,7 +4528,7 @@ ExecInitExpr(Expr *node, PlanState *parent)
 						elog(ERROR, "non-partial Aggref found in non-finalize agg plan node");
 				}
 
-				if (aggref->aggcombine != aggstate->combineStates)
+				if (aggref->aggcombine != DO_AGGMODE_COMBINE(aggstate->aggmode))
 				{
 					/* planner messed up */
 					if (aggref->aggcombine)
diff --git a/src/backend/executor/nodeAgg.c b/src/backend/executor/nodeAgg.c
index a447964..8d845f7 100644
--- a/src/backend/executor/nodeAgg.c
+++ b/src/backend/executor/nodeAgg.c
@@ -1577,7 +1577,7 @@ finalize_aggregates(AggState *aggstate,
 												pergroupstate);
 		}
 
-		if (aggstate->finalizeAggs)
+		if (DO_AGGMODE_FINALIZE(aggstate->aggmode))
 			finalize_aggregate(aggstate, peragg, pergroupstate,
 							   &aggvalues[aggno], &aggnulls[aggno]);
 		else
@@ -2114,7 +2114,7 @@ agg_retrieve_direct(AggState *aggstate)
 				 */
 				for (;;)
 				{
-					if (!aggstate->combineStates)
+					if (!DO_AGGMODE_COMBINE(aggstate->aggmode))
 						advance_aggregates(aggstate, pergroup);
 					else
 						combine_aggregates(aggstate, pergroup);
@@ -2225,7 +2225,7 @@ agg_fill_hash_table(AggState *aggstate)
 		entry = lookup_hash_entry(aggstate, outerslot);
 
 		/* Advance the aggregates */
-		if (!aggstate->combineStates)
+		if (!DO_AGGMODE_COMBINE(aggstate->aggmode))
 			advance_aggregates(aggstate, entry->pergroup);
 		else
 			combine_aggregates(aggstate, entry->pergroup);
@@ -2360,9 +2360,7 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
 	aggstate->pertrans = NULL;
 	aggstate->curpertrans = NULL;
 	aggstate->agg_done = false;
-	aggstate->combineStates = node->combineStates;
-	aggstate->finalizeAggs = node->finalizeAggs;
-	aggstate->serialStates = node->serialStates;
+	aggstate->aggmode = node->aggmode;
 	aggstate->input_done = false;
 	aggstate->pergroup = NULL;
 	aggstate->grp_firstTuple = NULL;
@@ -2724,7 +2722,7 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
 		 * If this aggregation is performing state combines, then instead of
 		 * using the transition function, we'll use the combine function
 		 */
-		if (aggstate->combineStates)
+		if (DO_AGGMODE_COMBINE(aggstate->aggmode))
 		{
 			transfn_oid = aggform->aggcombinefn;
 
@@ -2736,7 +2734,7 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
 			transfn_oid = aggform->aggtransfn;
 
 		/* Final function only required if we're finalizing the aggregates */
-		if (aggstate->finalizeAggs)
+		if (DO_AGGMODE_FINALIZE(aggstate->aggmode))
 			peragg->finalfn_oid = finalfn_oid = aggform->aggfinalfn;
 		else
 			peragg->finalfn_oid = finalfn_oid = InvalidOid;
@@ -2745,30 +2743,38 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
 		deserialfn_oid = InvalidOid;
 
 		/*
-		 * Determine if we require serialization or deserialization of the
-		 * aggregate states. This is only required if the aggregate state is
-		 * internal.
+		 * Check if serial/deserialization is required. We only do this
+		 * when the transtype is INTERNAL.
 		 */
-		if (aggstate->serialStates && aggtranstype == INTERNALOID)
+		if (aggtranstype == INTERNALOID)
 		{
 			/*
-			 * The planner should only have generated an agg node with
-			 * serialStates if every aggregate with an INTERNAL state has
-			 * serialization/deserialization functions.  Verify that.
+			 * The planner should only have generated a serialize agg node if
+			 * every aggregate with an INTERNAL state has
+			 * serialization function.  Verify that.
 			 */
-			if (!OidIsValid(aggform->aggserialfn))
-				elog(ERROR, "serialfunc not set during serialStates aggregation step");
+			if (DO_AGGMODE_SERIALIZE(aggstate->aggmode))
+			{
+				if (!OidIsValid(aggform->aggserialfn))
+					elog(ERROR, "serialfunc not set during serialization aggregation");
 
-			if (!OidIsValid(aggform->aggdeserialfn))
-				elog(ERROR, "deserialfunc not set during serialStates aggregation step");
+				/* serialization only valid when not in finalize mode */
+				Assert(!DO_AGGMODE_FINALIZE(aggstate->aggmode));
 
-			/* serialization func only required when not finalizing aggs */
-			if (!aggstate->finalizeAggs)
 				serialfn_oid = aggform->aggserialfn;
+			}
+
+			/* Likewise for deserialization functions */
+			if (DO_AGGMODE_DESERIALIZE(aggstate->aggmode))
+			{
+				if (!OidIsValid(aggform->aggdeserialfn))
+					elog(ERROR, "deserialfunc not set during deserialization aggregation");
+
+				/* deserialization only valid when combining states */
+				Assert(DO_AGGMODE_COMBINE(aggstate->aggmode));
 
-			/* deserialization func only required when combining states */
-			if (aggstate->combineStates)
 				deserialfn_oid = aggform->aggdeserialfn;
+			}
 		}
 
 		/* Check that aggregate owner has permission to call component fns */
@@ -2972,7 +2978,7 @@ build_pertrans_for_aggref(AggStatePerTrans pertrans,
 	 * transfn and transfn_oid fields of pertrans refer to the combine
 	 * function rather than the transition function.
 	 */
-	if (aggstate->combineStates)
+	if (DO_AGGMODE_COMBINE(aggstate->aggmode))
 	{
 		Expr	   *combinefnexpr;
 
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 59add5b..f1319d7 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -870,9 +870,7 @@ _copyAgg(const Agg *from)
 	CopyPlanFields((const Plan *) from, (Plan *) newnode);
 
 	COPY_SCALAR_FIELD(aggstrategy);
-	COPY_SCALAR_FIELD(combineStates);
-	COPY_SCALAR_FIELD(finalizeAggs);
-	COPY_SCALAR_FIELD(serialStates);
+	COPY_SCALAR_FIELD(aggmode);
 	COPY_SCALAR_FIELD(numCols);
 	if (from->numCols > 0)
 	{
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b0a28f5..e81c69a 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -705,9 +705,7 @@ _outAgg(StringInfo str, const Agg *node)
 	_outPlanInfo(str, (const Plan *) node);
 
 	WRITE_ENUM_FIELD(aggstrategy, AggStrategy);
-	WRITE_BOOL_FIELD(combineStates);
-	WRITE_BOOL_FIELD(finalizeAggs);
-	WRITE_BOOL_FIELD(serialStates);
+	WRITE_ENUM_FIELD(aggmode, AggregateMode);
 	WRITE_INT_FIELD(numCols);
 
 	appendStringInfoString(str, " :grpColIdx");
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index b1f9e3e..72a8804 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1989,9 +1989,7 @@ _readAgg(void)
 	ReadCommonPlan(&local_node->plan);
 
 	READ_ENUM_FIELD(aggstrategy, AggStrategy);
-	READ_BOOL_FIELD(combineStates);
-	READ_BOOL_FIELD(finalizeAggs);
-	READ_BOOL_FIELD(serialStates);
+	READ_ENUM_FIELD(aggmode, AggregateMode);
 	READ_INT_FIELD(numCols);
 	READ_ATTRNUMBER_ARRAY(grpColIdx, local_node->numCols);
 	READ_OID_ARRAY(grpOperators, local_node->numCols);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index b2db6e8..315d4ea 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1304,9 +1304,7 @@ create_unique_plan(PlannerInfo *root, UniquePath *best_path, int flags)
 		plan = (Plan *) make_agg(build_path_tlist(root, &best_path->path),
 								 NIL,
 								 AGG_HASHED,
-								 false,
-								 true,
-								 false,
+								 AGGMODE_SIMPLE,
 								 numGroupCols,
 								 groupColIdx,
 								 groupOperators,
@@ -1610,9 +1608,7 @@ create_agg_plan(PlannerInfo *root, AggPath *best_path)
 
 	plan = make_agg(tlist, quals,
 					best_path->aggstrategy,
-					best_path->combineStates,
-					best_path->finalizeAggs,
-					best_path->serialStates,
+					best_path->aggmode,
 					list_length(best_path->groupClause),
 					extract_grouping_cols(best_path->groupClause,
 										  subplan->targetlist),
@@ -1765,9 +1761,7 @@ create_groupingsets_plan(PlannerInfo *root, GroupingSetsPath *best_path)
 			agg_plan = (Plan *) make_agg(NIL,
 										 NIL,
 										 AGG_SORTED,
-										 false,
-										 true,
-										 false,
+										 AGGMODE_SIMPLE,
 									   list_length((List *) linitial(gsets)),
 										 new_grpColIdx,
 										 extract_grouping_ops(groupClause),
@@ -1802,9 +1796,7 @@ create_groupingsets_plan(PlannerInfo *root, GroupingSetsPath *best_path)
 		plan = make_agg(build_path_tlist(root, &best_path->path),
 						best_path->qual,
 						(numGroupCols > 0) ? AGG_SORTED : AGG_PLAIN,
-						false,
-						true,
-						false,
+						AGGMODE_SIMPLE,
 						numGroupCols,
 						top_grpColIdx,
 						extract_grouping_ops(groupClause),
@@ -5653,9 +5645,8 @@ materialize_finished_plan(Plan *subplan)
 Agg *
 make_agg(List *tlist, List *qual,
 		 AggStrategy aggstrategy,
-		 bool combineStates, bool finalizeAggs, bool serialStates,
-		 int numGroupCols, AttrNumber *grpColIdx, Oid *grpOperators,
-		 List *groupingSets, List *chain,
+		 AggregateMode aggmode, int numGroupCols, AttrNumber *grpColIdx,
+		 Oid *grpOperators, List *groupingSets, List *chain,
 		 double dNumGroups, Plan *lefttree)
 {
 	Agg		   *node = makeNode(Agg);
@@ -5666,9 +5657,7 @@ make_agg(List *tlist, List *qual,
 	numGroups = (long) Min(dNumGroups, (double) LONG_MAX);
 
 	node->aggstrategy = aggstrategy;
-	node->combineStates = combineStates;
-	node->finalizeAggs = finalizeAggs;
-	node->serialStates = serialStates;
+	node->aggmode = aggmode;
 	node->numCols = numGroupCols;
 	node->grpColIdx = grpColIdx;
 	node->grpOperators = grpOperators;
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 2372311..5f4a4b8 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3390,10 +3390,10 @@ create_grouping_paths(PlannerInfo *root,
 	MemSet(&agg_costs, 0, sizeof(AggClauseCosts));
 	if (parse->hasAggs)
 	{
-		count_agg_clauses(root, (Node *) target->exprs, &agg_costs, true,
-						  false, false);
-		count_agg_clauses(root, parse->havingQual, &agg_costs, true, false,
-						  false);
+		count_agg_clauses(root, (Node *) target->exprs, &agg_costs,
+						  AGGMODE_SIMPLE);
+		count_agg_clauses(root, parse->havingQual, &agg_costs,
+						  AGGMODE_SIMPLE);
 	}
 
 	/*
@@ -3479,13 +3479,13 @@ create_grouping_paths(PlannerInfo *root,
 		{
 			/* partial phase */
 			count_agg_clauses(root, (Node *) partial_grouping_target->exprs,
-							  &agg_partial_costs, false, false, true);
+							  &agg_partial_costs, AGGMODE_PARTIAL_SERIAL);
 
 			/* final phase */
 			count_agg_clauses(root, (Node *) target->exprs, &agg_final_costs,
-							  true, true, true);
-			count_agg_clauses(root, parse->havingQual, &agg_final_costs, true,
-							  true, true);
+							  AGGMODE_FINAL_DESERIAL);
+			count_agg_clauses(root, parse->havingQual, &agg_final_costs,
+							  AGGMODE_FINAL_DESERIAL);
 		}
 
 		if (can_sort)
@@ -3525,9 +3525,7 @@ create_grouping_paths(PlannerInfo *root,
 														 NIL,
 														 &agg_partial_costs,
 														 dNumPartialGroups,
-														 false,
-														 false,
-														 true));
+														 AGGMODE_PARTIAL_SERIAL));
 					else
 						add_partial_path(grouped_rel, (Path *)
 										 create_group_path(root,
@@ -3567,9 +3565,7 @@ create_grouping_paths(PlannerInfo *root,
 												 NIL,
 												 &agg_partial_costs,
 												 dNumPartialGroups,
-												 false,
-												 false,
-												 true));
+												 AGGMODE_PARTIAL_SERIAL));
 			}
 		}
 	}
@@ -3632,9 +3628,7 @@ create_grouping_paths(PlannerInfo *root,
 											 (List *) parse->havingQual,
 											 &agg_costs,
 											 dNumGroups,
-											 false,
-											 true,
-											 false));
+											 AGGMODE_SIMPLE));
 				}
 				else if (parse->groupClause)
 				{
@@ -3699,9 +3693,7 @@ create_grouping_paths(PlannerInfo *root,
 										 (List *) parse->havingQual,
 										 &agg_final_costs,
 										 dNumGroups,
-										 true,
-										 true,
-										 true));
+										 AGGMODE_FINAL_DESERIAL));
 			else
 				add_path(grouped_rel, (Path *)
 						 create_group_path(root,
@@ -3742,9 +3734,7 @@ create_grouping_paths(PlannerInfo *root,
 									 (List *) parse->havingQual,
 									 &agg_costs,
 									 dNumGroups,
-									 false,
-									 true,
-									 false));
+									 AGGMODE_SIMPLE));
 		}
 
 		/*
@@ -3781,9 +3771,7 @@ create_grouping_paths(PlannerInfo *root,
 										 (List *) parse->havingQual,
 										 &agg_final_costs,
 										 dNumGroups,
-										 true,
-										 true,
-										 true));
+										 AGGMODE_FINAL_DESERIAL));
 			}
 		}
 	}
@@ -4125,9 +4113,7 @@ create_distinct_paths(PlannerInfo *root,
 								 NIL,
 								 NULL,
 								 numDistinctRows,
-								 false,
-								 true,
-								 false));
+								 AGGMODE_SIMPLE));
 	}
 
 	/* Give a helpful error if we failed to find any implementation */
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 17edc27..eb7f311 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -680,7 +680,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
 			{
 				Agg		   *aggplan = (Agg *) plan;
 
-				if (aggplan->combineStates)
+				if (DO_AGGMODE_COMBINE(aggplan->aggmode))
 					set_combineagg_references(root, plan, rtoffset);
 				else
 					set_upper_references(root, plan, rtoffset);
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 552b756..75b6f36 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -865,9 +865,7 @@ make_union_unique(SetOperationStmt *op, Path *path, List *tlist,
 										NIL,
 										NULL,
 										dNumGroups,
-										false,
-										true,
-										false);
+										AGGMODE_SIMPLE);
 	}
 	else
 	{
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 7138cad..22877d8 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -61,9 +61,7 @@ typedef struct
 {
 	PlannerInfo *root;
 	AggClauseCosts *costs;
-	bool		finalizeAggs;
-	bool		combineStates;
-	bool		serialStates;
+	AggregateMode aggmode;
 } count_agg_clauses_context;
 
 typedef struct
@@ -543,15 +541,13 @@ contain_agg_clause_walker(Node *node, void *context)
  */
 void
 count_agg_clauses(PlannerInfo *root, Node *clause, AggClauseCosts *costs,
-				  bool finalizeAggs, bool combineStates, bool serialStates)
+				  AggregateMode aggmode)
 {
 	count_agg_clauses_context context;
 
 	context.root = root;
 	context.costs = costs;
-	context.finalizeAggs = finalizeAggs;
-	context.combineStates = combineStates;
-	context.serialStates = serialStates;
+	context.aggmode = aggmode;
 	(void) count_agg_clauses_walker(clause, &context);
 }
 
@@ -628,24 +624,25 @@ count_agg_clauses_walker(Node *node, count_agg_clauses_context *context)
 		 * Add the appropriate component function execution costs to
 		 * appropriate totals.
 		 */
-		if (context->combineStates)
+		if (DO_AGGMODE_COMBINE(context->aggmode))
 		{
 			/* charge for combining previously aggregated states */
 			costs->transCost.per_tuple += get_func_cost(aggcombinefn) * cpu_operator_cost;
 
 			/* charge for deserialization, when appropriate */
-			if (context->serialStates && OidIsValid(aggdeserialfn))
+			if (DO_AGGMODE_DESERIALIZE(context->aggmode) &&
+				OidIsValid(aggdeserialfn))
 				costs->transCost.per_tuple += get_func_cost(aggdeserialfn) * cpu_operator_cost;
 		}
 		else
 			costs->transCost.per_tuple += get_func_cost(aggtransfn) * cpu_operator_cost;
 
-		if (context->finalizeAggs)
+		if (DO_AGGMODE_FINALIZE(context->aggmode))
 		{
 			if (OidIsValid(aggfinalfn))
 				costs->finalCost += get_func_cost(aggfinalfn) * cpu_operator_cost;
 		}
-		else if (context->serialStates)
+		else if (DO_AGGMODE_SERIALIZE(context->aggmode))
 		{
 			if (OidIsValid(aggserialfn))
 				costs->finalCost += get_func_cost(aggserialfn) * cpu_operator_cost;
@@ -655,7 +652,7 @@ count_agg_clauses_walker(Node *node, count_agg_clauses_context *context)
 		 * Some costs will already have been incurred by the initial aggregate
 		 * node, so we mustn't include these again.
 		 */
-		if (!context->combineStates)
+		if (!DO_AGGMODE_COMBINE(context->aggmode))
 		{
 			/* add the input expressions' cost to per-input-row costs */
 			cost_qual_eval_node(&argcosts, (Node *) aggref->args, context->root);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 8fd933f..2dc9678 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -2483,9 +2483,7 @@ create_agg_path(PlannerInfo *root,
 				List *qual,
 				const AggClauseCosts *aggcosts,
 				double numGroups,
-				bool combineStates,
-				bool finalizeAggs,
-				bool serialStates)
+				AggregateMode aggmode)
 {
 	AggPath    *pathnode = makeNode(AggPath);
 
@@ -2508,9 +2506,7 @@ create_agg_path(PlannerInfo *root,
 	pathnode->numGroups = numGroups;
 	pathnode->groupClause = groupClause;
 	pathnode->qual = qual;
-	pathnode->finalizeAggs = finalizeAggs;
-	pathnode->combineStates = combineStates;
-	pathnode->serialStates = serialStates;
+	pathnode->aggmode = aggmode;
 
 	cost_agg(&pathnode->path, root,
 			 aggstrategy, aggcosts,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 1ddf14a..4aba920 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1834,9 +1834,7 @@ typedef struct AggState
 	AggStatePerTrans curpertrans;		/* currently active trans state */
 	bool		input_done;		/* indicates end of input */
 	bool		agg_done;		/* indicates completion of Agg scan */
-	bool		combineStates;	/* input tuples contain transition states */
-	bool		finalizeAggs;	/* should we call the finalfn on agg states? */
-	bool		serialStates;	/* should agg states be (de)serialized? */
+	AggregateMode aggmode;	/* enum to control type of agg operation */
 	int			projected_set;	/* The last projected grouping set */
 	int			current_set;	/* The current grouping set being evaluated */
 	Bitmapset  *grouped_cols;	/* grouped cols in current projection */
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 72f53fd..5ecd51b 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -711,9 +711,7 @@ typedef struct Agg
 {
 	Plan		plan;
 	AggStrategy aggstrategy;	/* basic strategy, see nodes.h */
-	bool		combineStates;	/* input tuples contain transition states */
-	bool		finalizeAggs;	/* should we call the finalfn on agg states? */
-	bool		serialStates;	/* should agg states be (de)serialized? */
+	AggregateMode aggmode;		/* enum to control type of agg operation */
 	int			numCols;		/* number of grouping columns */
 	AttrNumber *grpColIdx;		/* their indexes in the target list */
 	Oid		   *grpOperators;	/* equality operators to compare with */
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 3de11f0..8d84898 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -301,6 +301,23 @@ typedef struct Aggref
 	int			location;		/* token location, or -1 if unknown */
 } Aggref;
 
+#define AGGOP_COMBINESTATES		0x1
+#define AGGOP_SERIALIZESTATES	0x2
+#define AGGOP_DESERIALIZESTATES	0x4
+#define AGGOP_FINALIZEAGGS		0x8
+
+typedef enum AggregateMode
+{
+  AGGMODE_SIMPLE = AGGOP_FINALIZEAGGS,
+  AGGMODE_PARTIAL_SERIAL = AGGOP_SERIALIZESTATES,
+  AGGMODE_FINAL_DESERIAL = AGGOP_COMBINESTATES | AGGOP_DESERIALIZESTATES | AGGOP_FINALIZEAGGS
+} AggregateMode;
+
+#define DO_AGGMODE_COMBINE(am)  (((am) & AGGOP_COMBINESTATES) != 0)
+#define DO_AGGMODE_SERIALIZE(am)  (((am) & AGGOP_SERIALIZESTATES) != 0)
+#define DO_AGGMODE_DESERIALIZE(am)  (((am) & AGGOP_DESERIALIZESTATES) != 0)
+#define DO_AGGMODE_FINALIZE(am)  (((am) & AGGOP_FINALIZEAGGS) != 0)
+
 /*
  * GroupingFunc
  *
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 9470df6..96bbe31 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1350,9 +1350,7 @@ typedef struct AggPath
 	double		numGroups;		/* estimated number of groups in input */
 	List	   *groupClause;	/* a list of SortGroupClause's */
 	List	   *qual;			/* quals (HAVING quals), if any */
-	bool		combineStates;	/* input is partially aggregated agg states */
-	bool		finalizeAggs;	/* should the executor call the finalfn? */
-	bool		serialStates;	/* should agg states be (de)serialized? */
+	AggregateMode aggmode;		/* enum to control type of agg operation */
 } AggPath;
 
 /*
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 53cf726..abec159 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -68,8 +68,7 @@ extern List *make_ands_implicit(Expr *clause);
 extern PartialAggType aggregates_allow_partial(Node *clause);
 extern bool contain_agg_clause(Node *clause);
 extern void count_agg_clauses(PlannerInfo *root, Node *clause,
-				  AggClauseCosts *costs, bool finalizeAggs,
-				  bool combineStates, bool serialStates);
+				  AggClauseCosts *costs, AggregateMode aggmode);
 
 extern bool contain_window_function(Node *clause);
 extern WindowFuncLists *find_window_functions(Node *clause, Index maxWinRef);
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 5de4c34..73bb27c 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -170,9 +170,7 @@ extern AggPath *create_agg_path(PlannerInfo *root,
 				List *qual,
 				const AggClauseCosts *aggcosts,
 				double numGroups,
-				bool combineStates,
-				bool finalizeAggs,
-				bool serialStates);
+				AggregateMode aggmode);
 extern GroupingSetsPath *create_groupingsets_path(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 Path *subpath,
diff --git a/src/include/optimizer/planmain.h b/src/include/optimizer/planmain.h
index c529085..834c614 100644
--- a/src/include/optimizer/planmain.h
+++ b/src/include/optimizer/planmain.h
@@ -59,9 +59,8 @@ extern bool is_projection_capable_plan(Plan *plan);
 /* External use of these functions is deprecated: */
 extern Sort *make_sort_from_sortclauses(List *sortcls, Plan *lefttree);
 extern Agg *make_agg(List *tlist, List *qual, AggStrategy aggstrategy,
-		 bool combineStates, bool finalizeAggs, bool serialStates,
-		 int numGroupCols, AttrNumber *grpColIdx, Oid *grpOperators,
-		 List *groupingSets, List *chain,
+		 AggregateMode aggmode, int numGroupCols, AttrNumber *grpColIdx,
+		 Oid *grpOperators, List *groupingSets, List *chain,
 		 double dNumGroups, Plan *lefttree);
 extern Limit *make_limit(Plan *lefttree, Node *limitOffset, Node *limitCount);
 
