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 2015/10/27 09:24:57 UTC

[1/3] incubator-hawq git commit: HAWQ-36. Allow querying pg_locks

Repository: incubator-hawq
Updated Branches:
  refs/heads/master c35094a9a -> 7fc93c378


HAWQ-36. Allow querying pg_locks


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

Branch: refs/heads/master
Commit: 9d2fd0ccf4d3e16d949e9712da7e00744e83af9d
Parents: 2fc73a0
Author: Lirong Jian <ji...@gmail.com>
Authored: Tue Oct 27 10:53:53 2015 +0800
Committer: Lirong Jian <ji...@gmail.com>
Committed: Tue Oct 27 10:53:53 2015 +0800

----------------------------------------------------------------------
 src/backend/utils/adt/lockfuncs.c | 224 ---------------------------------
 1 file changed, 224 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/9d2fd0cc/src/backend/utils/adt/lockfuncs.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index d82679e..e73f9f7 100644
--- a/src/backend/utils/adt/lockfuncs.c
+++ b/src/backend/utils/adt/lockfuncs.c
@@ -128,123 +128,6 @@ pg_lock_status(PG_FUNCTION_ARGS)
 		mystatus->numSegLocks = 0;
 		mystatus->numsegresults = 0;
 		mystatus->segresults = NULL;
-
-		/*
-		 * Seeing the locks just from the masterDB isn't enough to know what is locked,
-		 * or if there is a deadlock.  That's because the segDBs also take locks.
-		 * Some locks show up only on the master, some only on the segDBs, and some on both.
-		 *
-		 * So, let's collect the lock information from all the segDBs.  Sure, this means
-		 * there are a lot more rows coming back from pg_locks than before, since most locks
-		 * on the segDBs happen across all the segDBs at the same time.  But not always,
-		 * so let's play it safe and get them all.
-		 */
-
-		if (Gp_role == GP_ROLE_DISPATCH)
-		{
-			struct pg_result **results = NULL;
-			StringInfoData buffer;
-			DispatchDataResult	result;
-			int i;
-
-			initStringInfo(&buffer);
-
-			/*
-			 * This query has to match the tupledesc we just made above.
-			 */
-
-			appendStringInfo(&buffer,
-					"SELECT * FROM  pg_lock_status() L "
-					 " (locktype text, database oid, relation oid, page int4, tuple int2,"
-					 " transactionid xid, classid oid, objid oid, objsubid int2,"
-					 " transaction xid, pid int4, mode text, granted boolean, "
-					 " mppSessionId int4, mppIsWriter boolean, gp_segment_id int4) ");
-
-			/*
-			 * Why dispatch something here, rather than do a UNION ALL in pg_locks view, and
-			 * a join to gp_dist_random('gp_id')?  There are several important reasons.
-			 *
-			 * The union all method is much slower, and requires taking locks on gp_id.
-			 * More importantly, applications such as pgAdmin do queries of this view that
-			 * involve a correlated subqueries joining to other catalog tables,
-			 * which works if we do it this way, but fails
-			 * if the view includes the union all.  That completely breaks the server status
-			 * display in pgAdmin.
-			 *
-			 * Why dispatch this way, rather than via SPI?  There are several advantages.
-			 * First, it's easy to get "writer gang is busy" errors if we use SPI.
-			 *
-			 * Second, this should be much faster, as it doesn't require setting up
-			 * the interconnect, and doesn't need to touch any actual data tables to be
-			 * able to get the gp_segment_id.
-			 *
-			 * The downside is we get n result sets, where n == number of segDBs.
-			 *
-			 * It would be better yet if we sent a plan tree rather than a text string,
-			 * so the segDBs don't need to parse it.  That would also avoid taking any relation locks
-			 * on the segDB to get this info (normally need to get an accessShareLock on pg_locks on the segDB
-			 * to make sure it doesn't go away during parsing).  But the only safe way I know to do this
-			 * is to hand-build the plan tree, and I'm to lazy to do it right now. It's just a matter of
-			 * building a function scan node, and filling it in with our result set info (from the tupledesc).
-			 *
-			 * One thing to note:  it's OK to join pg_locks with any catalog table or master-only table,
-			 * but joining to a distributed table will result in "writer gang busy: possible attempt to
-			 * execute volatile function in unsupported context" errors, because
-			 * the scan of the distributed table might already be running on the writer gang
-			 * when we want to dispatch this.
-			 *
-			 * This could be fixed by allocating a reader gang and dispatching to that, but the cost
-			 * of setting up a new gang is high, and I've never seen anyone need to join this to a
-			 * distributed table.
-			 *
-			 */
-
-			dispatch_statement_string(buffer.data, NULL, 0, NULL, &result, false);
-			results = result.result;
-
-			if (result.errbuf.len > 0)
-				ereport(ERROR, (errmsg("pg_lock internal error (gathered %d results from cmd '%s')", result.numresults, buffer.data),
-								errdetail("%s", result.errbuf.data)));
-
-			/*
-			 * I don't think resultCount can ever be zero if errbuf isn't set.
-			 * But check to be sure.
-			 */
-			if (result.numresults == 0)
-				elog(ERROR, "pg_locks didn't get back any data from the segDBs");
-
-			for (i = 0; i < result.numresults; i++)
-			{
-				/*
-				 * Any error here should have propagated into errbuf, so we shouldn't
-				 * ever see anything other that tuples_ok here.  But, check to be
-				 * sure.
-				 */
-				if (PQresultStatus(results[i]) != PGRES_TUPLES_OK)
-				{
-					elog(ERROR,"pg_locks: resultStatus not tuples_Ok");
-				}
-				else
-				{
-					/*
-					 * numSegLocks needs to be the total size we are returning to
-					 * the application. At the start of this loop, it has the count
-					 * for the masterDB locks.  Add each of the segDB lock counts.
-					 */
-					mystatus->numSegLocks += PQntuples(results[i]);
-				}
-			}
-
-			dispatch_free_result(&result);
-			mystatus->numsegresults = result.numresults;
-			/*
-			 * cdbdisp_dispatchRMCommand copies the result sets into our memory, which
-			 * will still exist on the subsequent calls.
-			 */
-			mystatus->segresults = results;
-
-			MemoryContextSwitchTo(oldcontext);
-		}
 	}
 
 	funcctx = SRF_PERCALL_SETUP();
@@ -436,113 +319,6 @@ pg_lock_status(PG_FUNCTION_ARGS)
 		SRF_RETURN_NEXT(funcctx, result);
 	}
 
-	/*
-	 * This loop only executes on the masterDB and only in dispatch mode, because that
-	 * is the only time we dispatched to the segDBs.
-	 */
-
-	while (mystatus->currIdx >= lockData->nelements && mystatus->currIdx < lockData->nelements + mystatus->numSegLocks)
-	{
-		HeapTuple	tuple;
-		Datum		result;
-		Datum		values[16];
-		bool		nulls[16];
-		int i;
-		int whichresultset = 0;
-		int whichelement = mystatus->currIdx - lockData->nelements;
-		int whichrow = whichelement;
-
-		Assert(Gp_role == GP_ROLE_DISPATCH);
-
-		/*
-		 * Because we have one result set per segDB (rather than one big result set with everything),
-		 * we need to figure out which result set we are on, and which row within that result set
-		 * we are returning.
-		 *
-		 * So, we walk through all the result sets and all the rows in each one, in order.
-		 */
-
-		while(whichrow >= PQntuples(mystatus->segresults[whichresultset]))
-		{
-			whichrow -= PQntuples(mystatus->segresults[whichresultset]);
-			whichresultset++;
-			if (whichresultset >= mystatus->numsegresults)
-				break;
-		}
-
-		/*
-		 * If this condition is true, we have already sent everything back,
-		 * and we just want to do the SRF_RETURN_DONE
-		 */
-		if (whichresultset >= mystatus->numsegresults)
-			break;
-
-		mystatus->currIdx++;
-
-		/*
-		 * Form tuple with appropriate data we got from the segDBs
-		 */
-		MemSet(values, 0, sizeof(values));
-		MemSet(nulls, false, sizeof(nulls));
-
-		/*
-		 * For each column, extract out the value (which comes out in text).
-		 * Convert it to the appropriate datatype to match our tupledesc,
-		 * and put that in values.
-		 * The columns look like this (from select statement earlier):
-		 *
-		 * "   (locktype text, database oid, relation oid, page int4, tuple int2,"
-		 *	"   transactionid xid, classid oid, objid oid, objsubid int2,"
-		 *	"    transaction xid, pid int4, mode text, granted boolean, "
-		 *	"    mppSessionId int4, mppIsWriter boolean, gp_segment_id int4) ,"
-		 */
-
-		values[0] = CStringGetTextDatum(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 0));
-		values[1] = ObjectIdGetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 1)));
-		values[2] = ObjectIdGetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 2)));
-		values[3] = UInt32GetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 3)));
-		values[4] = UInt16GetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 4)));
-
-		values[5] = TransactionIdGetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 5)));
-		values[6] = ObjectIdGetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 6)));
-		values[7] = ObjectIdGetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 7)));
-		values[8] = UInt16GetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 8)));
-
-		values[9] = TransactionIdGetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 9)));
-		values[10] = UInt32GetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow,10)));
-		values[11] = CStringGetTextDatum(PQgetvalue(mystatus->segresults[whichresultset], whichrow,11));
-		values[12] = BoolGetDatum(strncmp(PQgetvalue(mystatus->segresults[whichresultset], whichrow,12),"t",1)==0);
-		values[13] = Int32GetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow,13)));
-		values[14] = BoolGetDatum(strncmp(PQgetvalue(mystatus->segresults[whichresultset], whichrow,14),"t",1)==0);
-		values[15] = Int32GetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow,15)));
-
-		/*
-		 * Copy the null info over.  It should all match properly.
-		 */
-		for (i=0; i<16; i++)
-		{
-			nulls[i] = PQgetisnull(mystatus->segresults[whichresultset], whichrow, i);
-		}
-
-		tuple = heap_form_tuple(funcctx->tuple_desc, values, nulls);
-		result = HeapTupleGetDatum(tuple);
-		SRF_RETURN_NEXT(funcctx, result);
-	}
-
-	/*
-	 * if we dispatched to the segDBs, free up the memory holding the result sets.
-	 * Otherwise we might leak this memory each time we got called (does it automatically
-	 * get freed by the pool being deleted?  Probably, but this is safer).
-	 */
-	if (mystatus->segresults != NULL)
-	{
-		int i;
-		for (i = 0; i < mystatus->numsegresults; i++)
-			PQclear(mystatus->segresults[i]);
-
-		free(mystatus->segresults);
-	}
-
 	SRF_RETURN_DONE(funcctx);
 }
 


[3/3] incubator-hawq git commit: Merge branch 'HAWQ-36'

Posted by ji...@apache.org.
Merge branch 'HAWQ-36'


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

Branch: refs/heads/master
Commit: 7fc93c3783e11c5b6958c9efb138895b667af15b
Parents: 9d2fd0c e878bb1
Author: Lirong Jian <ji...@gmail.com>
Authored: Tue Oct 27 16:23:52 2015 +0800
Committer: Lirong Jian <ji...@gmail.com>
Committed: Tue Oct 27 16:23:52 2015 +0800

----------------------------------------------------------------------
 GNUmakefile.in                                          |  8 ++++----
 depends/libyarn/src/libyarnclient/LibYarnClientC.cpp    |  7 ++++++-
 .../resourcebroker/resourcebroker_LIBYARN_proc.c        | 12 ++++++------
 3 files changed, 16 insertions(+), 11 deletions(-)
----------------------------------------------------------------------



[2/3] incubator-hawq git commit: HAWQ-36. Allow querying pg_locks

Posted by ji...@apache.org.
HAWQ-36. Allow querying pg_locks


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

Branch: refs/heads/master
Commit: e878bb18a21630b638795d98e8c86066bb4bcc0e
Parents: c35094a
Author: Lirong Jian <ji...@gmail.com>
Authored: Tue Oct 27 10:53:53 2015 +0800
Committer: Lirong Jian <ji...@gmail.com>
Committed: Tue Oct 27 15:59:17 2015 +0800

----------------------------------------------------------------------
 src/backend/utils/adt/lockfuncs.c | 224 ---------------------------------
 1 file changed, 224 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/e878bb18/src/backend/utils/adt/lockfuncs.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index d82679e..e73f9f7 100644
--- a/src/backend/utils/adt/lockfuncs.c
+++ b/src/backend/utils/adt/lockfuncs.c
@@ -128,123 +128,6 @@ pg_lock_status(PG_FUNCTION_ARGS)
 		mystatus->numSegLocks = 0;
 		mystatus->numsegresults = 0;
 		mystatus->segresults = NULL;
-
-		/*
-		 * Seeing the locks just from the masterDB isn't enough to know what is locked,
-		 * or if there is a deadlock.  That's because the segDBs also take locks.
-		 * Some locks show up only on the master, some only on the segDBs, and some on both.
-		 *
-		 * So, let's collect the lock information from all the segDBs.  Sure, this means
-		 * there are a lot more rows coming back from pg_locks than before, since most locks
-		 * on the segDBs happen across all the segDBs at the same time.  But not always,
-		 * so let's play it safe and get them all.
-		 */
-
-		if (Gp_role == GP_ROLE_DISPATCH)
-		{
-			struct pg_result **results = NULL;
-			StringInfoData buffer;
-			DispatchDataResult	result;
-			int i;
-
-			initStringInfo(&buffer);
-
-			/*
-			 * This query has to match the tupledesc we just made above.
-			 */
-
-			appendStringInfo(&buffer,
-					"SELECT * FROM  pg_lock_status() L "
-					 " (locktype text, database oid, relation oid, page int4, tuple int2,"
-					 " transactionid xid, classid oid, objid oid, objsubid int2,"
-					 " transaction xid, pid int4, mode text, granted boolean, "
-					 " mppSessionId int4, mppIsWriter boolean, gp_segment_id int4) ");
-
-			/*
-			 * Why dispatch something here, rather than do a UNION ALL in pg_locks view, and
-			 * a join to gp_dist_random('gp_id')?  There are several important reasons.
-			 *
-			 * The union all method is much slower, and requires taking locks on gp_id.
-			 * More importantly, applications such as pgAdmin do queries of this view that
-			 * involve a correlated subqueries joining to other catalog tables,
-			 * which works if we do it this way, but fails
-			 * if the view includes the union all.  That completely breaks the server status
-			 * display in pgAdmin.
-			 *
-			 * Why dispatch this way, rather than via SPI?  There are several advantages.
-			 * First, it's easy to get "writer gang is busy" errors if we use SPI.
-			 *
-			 * Second, this should be much faster, as it doesn't require setting up
-			 * the interconnect, and doesn't need to touch any actual data tables to be
-			 * able to get the gp_segment_id.
-			 *
-			 * The downside is we get n result sets, where n == number of segDBs.
-			 *
-			 * It would be better yet if we sent a plan tree rather than a text string,
-			 * so the segDBs don't need to parse it.  That would also avoid taking any relation locks
-			 * on the segDB to get this info (normally need to get an accessShareLock on pg_locks on the segDB
-			 * to make sure it doesn't go away during parsing).  But the only safe way I know to do this
-			 * is to hand-build the plan tree, and I'm to lazy to do it right now. It's just a matter of
-			 * building a function scan node, and filling it in with our result set info (from the tupledesc).
-			 *
-			 * One thing to note:  it's OK to join pg_locks with any catalog table or master-only table,
-			 * but joining to a distributed table will result in "writer gang busy: possible attempt to
-			 * execute volatile function in unsupported context" errors, because
-			 * the scan of the distributed table might already be running on the writer gang
-			 * when we want to dispatch this.
-			 *
-			 * This could be fixed by allocating a reader gang and dispatching to that, but the cost
-			 * of setting up a new gang is high, and I've never seen anyone need to join this to a
-			 * distributed table.
-			 *
-			 */
-
-			dispatch_statement_string(buffer.data, NULL, 0, NULL, &result, false);
-			results = result.result;
-
-			if (result.errbuf.len > 0)
-				ereport(ERROR, (errmsg("pg_lock internal error (gathered %d results from cmd '%s')", result.numresults, buffer.data),
-								errdetail("%s", result.errbuf.data)));
-
-			/*
-			 * I don't think resultCount can ever be zero if errbuf isn't set.
-			 * But check to be sure.
-			 */
-			if (result.numresults == 0)
-				elog(ERROR, "pg_locks didn't get back any data from the segDBs");
-
-			for (i = 0; i < result.numresults; i++)
-			{
-				/*
-				 * Any error here should have propagated into errbuf, so we shouldn't
-				 * ever see anything other that tuples_ok here.  But, check to be
-				 * sure.
-				 */
-				if (PQresultStatus(results[i]) != PGRES_TUPLES_OK)
-				{
-					elog(ERROR,"pg_locks: resultStatus not tuples_Ok");
-				}
-				else
-				{
-					/*
-					 * numSegLocks needs to be the total size we are returning to
-					 * the application. At the start of this loop, it has the count
-					 * for the masterDB locks.  Add each of the segDB lock counts.
-					 */
-					mystatus->numSegLocks += PQntuples(results[i]);
-				}
-			}
-
-			dispatch_free_result(&result);
-			mystatus->numsegresults = result.numresults;
-			/*
-			 * cdbdisp_dispatchRMCommand copies the result sets into our memory, which
-			 * will still exist on the subsequent calls.
-			 */
-			mystatus->segresults = results;
-
-			MemoryContextSwitchTo(oldcontext);
-		}
 	}
 
 	funcctx = SRF_PERCALL_SETUP();
@@ -436,113 +319,6 @@ pg_lock_status(PG_FUNCTION_ARGS)
 		SRF_RETURN_NEXT(funcctx, result);
 	}
 
-	/*
-	 * This loop only executes on the masterDB and only in dispatch mode, because that
-	 * is the only time we dispatched to the segDBs.
-	 */
-
-	while (mystatus->currIdx >= lockData->nelements && mystatus->currIdx < lockData->nelements + mystatus->numSegLocks)
-	{
-		HeapTuple	tuple;
-		Datum		result;
-		Datum		values[16];
-		bool		nulls[16];
-		int i;
-		int whichresultset = 0;
-		int whichelement = mystatus->currIdx - lockData->nelements;
-		int whichrow = whichelement;
-
-		Assert(Gp_role == GP_ROLE_DISPATCH);
-
-		/*
-		 * Because we have one result set per segDB (rather than one big result set with everything),
-		 * we need to figure out which result set we are on, and which row within that result set
-		 * we are returning.
-		 *
-		 * So, we walk through all the result sets and all the rows in each one, in order.
-		 */
-
-		while(whichrow >= PQntuples(mystatus->segresults[whichresultset]))
-		{
-			whichrow -= PQntuples(mystatus->segresults[whichresultset]);
-			whichresultset++;
-			if (whichresultset >= mystatus->numsegresults)
-				break;
-		}
-
-		/*
-		 * If this condition is true, we have already sent everything back,
-		 * and we just want to do the SRF_RETURN_DONE
-		 */
-		if (whichresultset >= mystatus->numsegresults)
-			break;
-
-		mystatus->currIdx++;
-
-		/*
-		 * Form tuple with appropriate data we got from the segDBs
-		 */
-		MemSet(values, 0, sizeof(values));
-		MemSet(nulls, false, sizeof(nulls));
-
-		/*
-		 * For each column, extract out the value (which comes out in text).
-		 * Convert it to the appropriate datatype to match our tupledesc,
-		 * and put that in values.
-		 * The columns look like this (from select statement earlier):
-		 *
-		 * "   (locktype text, database oid, relation oid, page int4, tuple int2,"
-		 *	"   transactionid xid, classid oid, objid oid, objsubid int2,"
-		 *	"    transaction xid, pid int4, mode text, granted boolean, "
-		 *	"    mppSessionId int4, mppIsWriter boolean, gp_segment_id int4) ,"
-		 */
-
-		values[0] = CStringGetTextDatum(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 0));
-		values[1] = ObjectIdGetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 1)));
-		values[2] = ObjectIdGetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 2)));
-		values[3] = UInt32GetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 3)));
-		values[4] = UInt16GetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 4)));
-
-		values[5] = TransactionIdGetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 5)));
-		values[6] = ObjectIdGetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 6)));
-		values[7] = ObjectIdGetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 7)));
-		values[8] = UInt16GetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 8)));
-
-		values[9] = TransactionIdGetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow, 9)));
-		values[10] = UInt32GetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow,10)));
-		values[11] = CStringGetTextDatum(PQgetvalue(mystatus->segresults[whichresultset], whichrow,11));
-		values[12] = BoolGetDatum(strncmp(PQgetvalue(mystatus->segresults[whichresultset], whichrow,12),"t",1)==0);
-		values[13] = Int32GetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow,13)));
-		values[14] = BoolGetDatum(strncmp(PQgetvalue(mystatus->segresults[whichresultset], whichrow,14),"t",1)==0);
-		values[15] = Int32GetDatum(atoi(PQgetvalue(mystatus->segresults[whichresultset], whichrow,15)));
-
-		/*
-		 * Copy the null info over.  It should all match properly.
-		 */
-		for (i=0; i<16; i++)
-		{
-			nulls[i] = PQgetisnull(mystatus->segresults[whichresultset], whichrow, i);
-		}
-
-		tuple = heap_form_tuple(funcctx->tuple_desc, values, nulls);
-		result = HeapTupleGetDatum(tuple);
-		SRF_RETURN_NEXT(funcctx, result);
-	}
-
-	/*
-	 * if we dispatched to the segDBs, free up the memory holding the result sets.
-	 * Otherwise we might leak this memory each time we got called (does it automatically
-	 * get freed by the pool being deleted?  Probably, but this is safer).
-	 */
-	if (mystatus->segresults != NULL)
-	{
-		int i;
-		for (i = 0; i < mystatus->numsegresults; i++)
-			PQclear(mystatus->segresults[i]);
-
-		free(mystatus->segresults);
-	}
-
 	SRF_RETURN_DONE(funcctx);
 }