summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorMichael Paquier2012-07-23 02:11:31 +0000
committerMichael Paquier2012-07-23 02:11:31 +0000
commit38b2b79f21859016e2b23d1c1203cd1d64ae833d (patch)
tree712ca428d1124494b2e666768aade53a8b08a85c
parentd8fc6215ef228112ec49f1d3e8b919856c5266c1 (diff)
Command Id communication protocol between Postgres-XC nodes
This commit adds a feature to allow Postgres-XC nodes to communicate a Command Id to remote nodes. Remote nodes can also send back a Command Id to the Coordinator commanding the transaction. This has as consequences to solve numerous issues with data view on cursors as well as solving an old issue XC had with INSERT SELECT when INSERT is done on a child after scanning the parent. This also allows to correctly increment the command ID on Coordinator in the case of triggers and/or constraints being fired on remote nodes. In order to allow nodes to communicate command ID, and a new message type 'M' is added. Note: this patch is also fixing some whitespaces in xact.c. Patch from Abbas Butt, I just added some simplifications, comments, and finalized the packing.
-rw-r--r--src/backend/access/transam/xact.c238
-rw-r--r--src/backend/commands/portalcmds.c11
-rw-r--r--src/backend/nodes/copyfuncs.c2
-rw-r--r--src/backend/nodes/equalfuncs.c4
-rw-r--r--src/backend/nodes/outfuncs.c1
-rw-r--r--src/backend/optimizer/plan/createplan.c2
-rw-r--r--src/backend/parser/analyze.c8
-rw-r--r--src/backend/pgxc/pool/execRemote.c79
-rw-r--r--src/backend/pgxc/pool/pgxcnode.c35
-rw-r--r--src/backend/tcop/postgres.c13
-rw-r--r--src/backend/tcop/pquery.c34
-rw-r--r--src/backend/utils/time/snapmgr.c26
-rw-r--r--src/include/access/xact.h6
-rw-r--r--src/include/nodes/parsenodes.h2
-rw-r--r--src/include/pgxc/execRemote.h2
-rw-r--r--src/include/pgxc/pgxcnode.h1
-rw-r--r--src/include/pgxc/planner.h2
-rw-r--r--src/test/regress/expected/combocid_1.out6
-rw-r--r--src/test/regress/expected/plpgsql_1.out26
-rw-r--r--src/test/regress/expected/portals_1.out3
-rw-r--r--src/test/regress/expected/select_views_2.out924
-rw-r--r--src/test/regress/expected/xc_misc.out187
-rw-r--r--src/test/regress/sql/plpgsql.sql8
-rw-r--r--src/test/regress/sql/xc_misc.sql110
24 files changed, 1642 insertions, 88 deletions
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index cc5dc6016a..47da554a2b 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -29,6 +29,8 @@
#include "pgxc/execRemote.h"
/* PGXC_DATANODE */
#include "postmaster/autovacuum.h"
+#include "libpq/pqformat.h"
+#include "libpq/libpq.h"
#endif
#include "access/multixact.h"
#include "access/subtrans.h"
@@ -148,7 +150,7 @@ typedef struct TransactionStateData
GlobalTransactionId auxilliaryTransactionId;
bool isLocalParameterUsed; /* Check if a local parameter is active
* in transaction block (SET LOCAL, DEFERRED) */
-#else
+#else
TransactionId transactionId; /* my XID, or Invalid if none */
#endif
SubTransactionId subTransactionId; /* my subxact ID */
@@ -183,7 +185,7 @@ static TransactionStateData TopTransactionStateData = {
0, /* prepared global transaction id */
0, /* commit prepared global transaction id */
false, /* isLocalParameterUsed */
-#else
+#else
0, /* transaction id */
#endif
0, /* subtransaction id */
@@ -223,6 +225,21 @@ static SubTransactionId currentSubTransactionId;
static CommandId currentCommandId;
static bool currentCommandIdUsed;
+#ifdef PGXC
+/*
+ * Parameters for communication control of Command ID between Postgres-XC nodes.
+ * isCommandIdReceived is used to determine of a command ID has been received by a remote
+ * node from a Coordinator.
+ * sendCommandId is used to determine if a Postgres-XC node needs to communicate its command ID.
+ * This is possible for both remote nodes and Coordinators connected to applications.
+ * receivedCommandId is the command ID received on Coordinator from remote node or on remote node
+ * from Coordinator.
+ */
+static bool isCommandIdReceived;
+static bool sendCommandId;
+static CommandId receivedCommandId;
+#endif
+
/*
* xactStartTimestamp is the value of transaction_timestamp().
* stmtStartTimestamp is the value of statement_timestamp().
@@ -644,7 +661,7 @@ GetTopGlobalTransactionId()
return s->topGlobalTransansactionId;
}
-GlobalTransactionId
+GlobalTransactionId
GetAuxilliaryTransactionId()
{
TransactionState s = CurrentTransactionState;
@@ -689,6 +706,32 @@ GetCurrentSubTransactionId(void)
CommandId
GetCurrentCommandId(bool used)
{
+#ifdef PGXC
+ /* If coordinator has sent a command id, remote node should use it */
+ if (IsConnFromCoord() && isCommandIdReceived)
+ {
+ /*
+ * Indicate to successive calls of this function that the sent command id has
+ * already been used.
+ */
+ isCommandIdReceived = false;
+ currentCommandId = GetReceivedCommandId();
+ }
+ else if (IS_PGXC_COORDINATOR && !IsConnFromCoord())
+ {
+ /*
+ * If command id reported by remote node is greater that the current
+ * command id, the coordinator needs to use it. This is required because
+ * a remote node can increase the command id sent by the coordinator
+ * e.g. in case a trigger fires at the remote node and inserts some rows
+ * The coordinator should now send the next command id knowing
+ * the largest command id either current or received from remote node.
+ */
+ if (GetReceivedCommandId() > currentCommandId)
+ currentCommandId = GetReceivedCommandId();
+ }
+#endif
+
/* this is global to a transaction, not subtransaction-local */
if (used)
currentCommandIdUsed = true;
@@ -920,6 +963,17 @@ CommandCounterIncrement(void)
/* Propagate new command ID into static snapshots */
SnapshotSetCommandId(currentCommandId);
+#ifdef PGXC
+ /*
+ * Remote node should report local command id changes only if
+ * required by the Coordinator. The requirement of the
+ * Coordinator is inferred from the fact that Coordinator
+ * has itself sent the command id to the remote nodes.
+ */
+ if (IsConnFromCoord() && IsSendCommandId())
+ ReportCommandIdChange(currentCommandId);
+#endif
+
/*
* Make any catalog changes done by the just-completed command visible
* in the local syscache. We obviously don't need to do this after a
@@ -1913,7 +1967,7 @@ StartTransaction(void)
*/
if (DefaultXactIsoLevel == XACT_SERIALIZABLE)
DefaultXactIsoLevel = XACT_REPEATABLE_READ;
-#endif
+#endif
XactIsoLevel = DefaultXactIsoLevel;
forceSyncCommit = false;
MyXactAccessedTempRel = false;
@@ -1926,7 +1980,18 @@ StartTransaction(void)
currentSubTransactionId = TopSubTransactionId;
currentCommandId = FirstCommandId;
currentCommandIdUsed = false;
-
+#ifdef PGXC
+ /*
+ * Parameters related to global command ID control for transaction.
+ * Send the 1st command ID.
+ */
+ isCommandIdReceived = false;
+ if (IsConnFromCoord())
+ {
+ SetReceivedCommandId(FirstCommandId);
+ SetSendCommandId(false);
+ }
+#endif
/*
* initialize reported xid accounting
*/
@@ -2027,9 +2092,9 @@ CommitTransaction(void)
TransStateAsString(s->state));
Assert(s->parent == NULL);
-#ifdef PGXC
+#ifdef PGXC
/*
- * If we are a Coordinator and currently serving the client,
+ * If we are a Coordinator and currently serving the client,
* we must run a 2PC if more than one nodes are involved in this
* transaction. We first prepare on the remote nodes and if everything goes
* right, we commit locally and then commit on the remote nodes. We must
@@ -2108,7 +2173,7 @@ CommitTransaction(void)
s->auxilliaryTransactionId = InvalidGlobalTransactionId;
}
}
-#endif
+#endif
/*
* Do pre-commit processing that involves calling user-defined code, such
@@ -2165,7 +2230,7 @@ CommitTransaction(void)
*/
PreCommit_Notify();
-#ifdef PGXC
+#ifdef PGXC
if (IS_PGXC_COORDINATOR && !IsConnFromCoord())
{
/*
@@ -2332,6 +2397,17 @@ CommitTransaction(void)
#ifdef PGXC
s->isLocalParameterUsed = false;
ForgetTransactionLocalNode();
+
+ /*
+ * Set the command ID of Coordinator to be sent to the remote nodes
+ * as the 1st one.
+ * For remote nodes, enforce the command ID sending flag to false to avoid
+ * sending any command ID by default as now transaction is done.
+ */
+ if (IS_PGXC_COORDINATOR && !IsConnFromCoord())
+ SetReceivedCommandId(FirstCommandId);
+ else
+ SetSendCommandId(false);
#endif
/*
@@ -2366,7 +2442,7 @@ AtEOXact_GlobalTxn(bool commit)
if (IS_PGXC_COORDINATOR && !IsConnFromCoord())
{
- if (commit)
+ if (commit)
{
if (GlobalTransactionIdIsValid(s->auxilliaryTransactionId) &&
GlobalTransactionIdIsValid(s->topGlobalTransansactionId))
@@ -2379,7 +2455,7 @@ AtEOXact_GlobalTxn(bool commit)
}
else
{
- /*
+ /*
* XXX Why don't we have a single API to abort both the GXIDs
* together ?
*/
@@ -2401,7 +2477,7 @@ AtEOXact_GlobalTxn(bool commit)
RollbackTranGTM(s->topGlobalTransansactionId);
}
}
-
+
s->topGlobalTransansactionId = InvalidGlobalTransactionId;
s->auxilliaryTransactionId = InvalidGlobalTransactionId;
@@ -2429,7 +2505,7 @@ PrepareTransaction(void)
#ifdef PGXC
bool isImplicit = !(s->blockState == TBLOCK_PREPARE);
char *nodestring;
-#endif
+#endif
ShowTransactionState("PrepareTransaction");
@@ -2702,6 +2778,17 @@ PrepareTransaction(void)
ForgetTransactionLocalNode();
}
SetNextTransactionId(InvalidTransactionId);
+
+ /*
+ * Set the command ID of Coordinator to be sent to the remote nodes
+ * as the 1st one.
+ * For remote nodes, enforce the command ID sending flag to false to avoid
+ * sending any command ID by default as now transaction is done.
+ */
+ if (IS_PGXC_COORDINATOR && !IsConnFromCoord())
+ SetReceivedCommandId(FirstCommandId);
+ else
+ SetSendCommandId(false);
#endif
}
@@ -2715,7 +2802,7 @@ AbortTransaction(void)
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
-#ifdef PGXC
+#ifdef PGXC
/*
* Save the current top transaction ID. We need this to close the
* transaction at the GTM at thr end
@@ -2863,7 +2950,7 @@ AbortTransaction(void)
pgstat_report_xact_timestamp(0);
}
-#ifdef PGXC
+#ifdef PGXC
ForgetTransactionLocalNode();
#endif
/*
@@ -2871,7 +2958,7 @@ AbortTransaction(void)
*/
RESUME_INTERRUPTS();
-#ifdef PGXC
+#ifdef PGXC
AtEOXact_GlobalTxn(false);
AtEOXact_Remote();
#endif
@@ -2920,6 +3007,18 @@ CleanupTransaction(void)
* default
*/
s->state = TRANS_DEFAULT;
+#ifdef PGXC
+ /*
+ * Set the command ID of Coordinator to be sent to the remote nodes
+ * as the 1st one.
+ * For remote nodes, enforce the command ID sending flag to false to avoid
+ * sending any command ID by default as now transaction is done.
+ */
+ if (IS_PGXC_COORDINATOR && !IsConnFromCoord())
+ SetReceivedCommandId(FirstCommandId);
+ else
+ SetSendCommandId(false);
+#endif
}
/*
@@ -3715,6 +3814,17 @@ BeginTransactionBlock(void)
BlockStateAsString(s->blockState));
break;
}
+
+#ifdef PGXC
+ /*
+ * Set command Id sending flag only for a local Coordinator when transaction begins,
+ * For a remote node this flag is set to true only if a command ID has been received
+ * from a Coordinator. This may not be always the case depending on the queries being
+ * run and how command Ids are generated on remote nodes.
+ */
+ if (IS_PGXC_COORDINATOR && !IsConnFromCoord())
+ SetSendCommandId(true);
+#endif
}
/*
@@ -3765,6 +3875,11 @@ PrepareTransactionBlock(char *gid)
}
}
+#ifdef PGXC
+ /* Reset command ID sending flag */
+ SetSendCommandId(false);
+#endif
+
return result;
}
@@ -3884,6 +3999,11 @@ EndTransactionBlock(void)
break;
}
+#ifdef PGXC
+ /* Reset command Id sending flag */
+ SetSendCommandId(false);
+#endif
+
return result;
}
@@ -3975,6 +4095,11 @@ UserAbortTransactionBlock(void)
BlockStateAsString(s->blockState));
break;
}
+
+#ifdef PGXC
+ /* Reset Command Id sending flag */
+ SetSendCommandId(false);
+#endif
}
/*
@@ -5576,4 +5701,85 @@ IsXidImplicit(const char *xid)
return false;
return true;
}
+
+/*
+ * SaveReceivedCommandId
+ * Save a received command ID from another node for future use.
+ */
+void
+SaveReceivedCommandId(CommandId cid)
+{
+ /* Set the new command ID */
+ SetReceivedCommandId(cid);
+
+ /*
+ * Change command ID information status to report any changes in remote ID
+ * for a remote node. A new command ID has also been received.
+ */
+ if (IsConnFromCoord())
+ {
+ SetSendCommandId(true);
+ isCommandIdReceived = true;
+ }
+}
+
+/*
+ * SetReceivedCommandId
+ * Set the command Id received from other nodes
+ */
+void
+SetReceivedCommandId(CommandId cid)
+{
+ receivedCommandId = cid;
+}
+
+/*
+ * GetReceivedCommandId
+ * Get the command id received from other nodes
+ */
+CommandId
+GetReceivedCommandId(void)
+{
+ return receivedCommandId;
+}
+
+
+/*
+ * ReportCommandIdChange
+ * ReportCommandIdChange reports a change in current command id at remote node
+ * to the Coordinator. This is required because a remote node can increment command
+ * Id in case of triggers or constraints.
+ */
+void
+ReportCommandIdChange(CommandId cid)
+{
+ StringInfoData buf;
+
+ /* Send command Id change to Coordinator */
+ pq_beginmessage(&buf, 'M');
+ pq_sendint(&buf, cid, 4);
+ pq_endmessage(&buf);
+ pq_flush();
+}
+
+/*
+ * IsSendCommandId
+ * Get status of command ID sending. If set at true, command ID needs to be communicated
+ * to other nodes.
+ */
+bool
+IsSendCommandId(void)
+{
+ return sendCommandId;
+}
+
+/*
+ * SetSendCommandId
+ * Change status of command ID sending.
+ */
+void
+SetSendCommandId(bool status)
+{
+ sendCommandId = status;
+}
#endif
diff --git a/src/backend/commands/portalcmds.c b/src/backend/commands/portalcmds.c
index 89086aa371..67d9aefe06 100644
--- a/src/backend/commands/portalcmds.c
+++ b/src/backend/commands/portalcmds.c
@@ -31,6 +31,9 @@
#include "utils/memutils.h"
#include "utils/snapmgr.h"
+#ifdef PGXC
+#include "pgxc/pgxc.h"
+#endif
/*
* PerformCursorOpen
@@ -73,6 +76,14 @@ PerformCursorOpen(PlannedStmt *stmt, ParamListInfo params,
*/
portal = CreatePortal(cstmt->portalname, false, false);
+#ifdef PGXC
+ /*
+ * Consume the command id of the command creating the cursor
+ */
+ if (IS_PGXC_COORDINATOR&& !IsConnFromCoord())
+ GetCurrentCommandId(true);
+#endif
+
oldContext = MemoryContextSwitchTo(PortalGetHeapMemory(portal));
stmt = copyObject(stmt);
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index e4be2c2898..c7d43cedbb 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -1036,6 +1036,7 @@ _copyRemoteQuery(RemoteQuery *from)
COPY_STRING_FIELD(outer_statement);
COPY_STRING_FIELD(join_condition);
COPY_SCALAR_FIELD(has_row_marks);
+ COPY_SCALAR_FIELD(has_ins_child_sel_parent);
return newnode;
}
@@ -2541,6 +2542,7 @@ _copyQuery(Query *from)
COPY_NODE_FIELD(constraintDeps);
#ifdef PGXC
COPY_STRING_FIELD(sql_statement);
+ COPY_SCALAR_FIELD(is_ins_child_sel_parent);
#endif
return newnode;
diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c
index 8ed895980d..26bef21dea 100644
--- a/src/backend/nodes/equalfuncs.c
+++ b/src/backend/nodes/equalfuncs.c
@@ -928,6 +928,10 @@ _equalQuery(Query *a, Query *b)
COMPARE_NODE_FIELD(setOperations);
COMPARE_NODE_FIELD(constraintDeps);
+#ifdef PGXC
+ COMPARE_SCALAR_FIELD(is_ins_child_sel_parent);
+#endif
+
return true;
}
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 9d4970d675..4d1894d137 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -478,6 +478,7 @@ _outRemoteQuery(StringInfo str, RemoteQuery *node)
WRITE_ENUM_FIELD(exec_type, RemoteQueryExecType);
WRITE_BOOL_FIELD(is_temp);
WRITE_BOOL_FIELD(has_row_marks);
+ WRITE_BOOL_FIELD(has_ins_child_sel_parent);
}
static void
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index bce367c5b4..1512c2a1d0 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -2789,6 +2789,8 @@ create_remotequery_plan(PlannerInfo *root, Path *best_path,
/* PGXCTODO - get better estimates */
scan_plan->scan.plan.plan_rows = 1000;
+ scan_plan->has_ins_child_sel_parent = root->parse->is_ins_child_sel_parent;
+
return (Plan *)scan_plan;
}
#endif
diff --git a/src/backend/parser/analyze.c b/src/backend/parser/analyze.c
index 7f39bc7d72..e496b2f6d3 100644
--- a/src/backend/parser/analyze.c
+++ b/src/backend/parser/analyze.c
@@ -60,6 +60,7 @@
#include "pgxc/poolmgr.h"
#include "catalog/pgxc_node.h"
#include "pgxc/xc_maintenance_mode.h"
+#include "access/xact.h"
#endif
#include "utils/rel.h"
@@ -531,9 +532,10 @@ transformInsertStmt(ParseState *pstate, InsertStmt *stmt)
#ifdef PGXC
target_rte = rt_fetch(qry->resultRelation, pstate->p_rtable);
if (is_relation_child(target_rte, selectQuery->rtable))
- ereport(ERROR,
- (errcode(ERRCODE_STATEMENT_TOO_COMPLEX),
- (errmsg("INSERT SELECT is not supported when inserts are done in a child by selecting from its parent"))));
+ {
+ qry->is_ins_child_sel_parent = true;
+ SetSendCommandId(true);
+ }
#endif
rtr = makeNode(RangeTblRef);
/* assume new rte is at end */
diff --git a/src/backend/pgxc/pool/execRemote.c b/src/backend/pgxc/pool/execRemote.c
index ce556c699b..9284d9c99f 100644
--- a/src/backend/pgxc/pool/execRemote.c
+++ b/src/backend/pgxc/pool/execRemote.c
@@ -22,6 +22,7 @@
#include "access/sysattr.h"
#include "access/transam.h"
#include "access/xact.h"
+#include "access/relscan.h"
#include "catalog/pg_type.h"
#include "catalog/pgxc_node.h"
#include "commands/prepare.h"
@@ -293,6 +294,7 @@ CreateResponseCombiner(int node_count, CombineType combine_type)
combiner->tapenodes = NULL;
combiner->initAggregates = true;
combiner->copy_file = NULL;
+ combiner->rqs_cmd_id = FirstCommandId;
return combiner;
}
@@ -778,6 +780,27 @@ HandleCmdComplete(CmdType commandType, CombineTag *combine,
}
/*
+ * HandleDatanodeCommandId ('M') message from a Datanode connection
+ */
+static void
+HandleDatanodeCommandId(RemoteQueryState *combiner, char *msg_body, size_t len)
+{
+ uint32 n32;
+ CommandId cid;
+
+ Assert(msg_body != NULL);
+ Assert(len >= 2);
+
+ /* Get the command Id */
+ memcpy(&n32, &msg_body[0], 4);
+ cid = ntohl(n32);
+
+ /* If received command Id is higher than current one, set it to a new value */
+ if (cid > GetReceivedCommandId())
+ SetReceivedCommandId(cid);
+}
+
+/*
* Examine the specified combiner state and determine if command was completed
* successfully
*/
@@ -1320,15 +1343,12 @@ handle_response(PGXCNodeHandle * conn, RemoteQueryState *combiner)
#endif
return result;
}
-
-#ifdef PGXC
+ case 'M': /* Command Id */
+ HandleDatanodeCommandId(combiner, msg, msg_len);
+ break;
case 'b':
- {
- conn->state = DN_CONNECTION_STATE_IDLE;
- return RESPONSE_BARRIER_OK;
- }
-#endif
-
+ conn->state = DN_CONNECTION_STATE_IDLE;
+ return RESPONSE_BARRIER_OK;
case 'I': /* EmptyQuery */
default:
/* sync lost? */
@@ -1573,6 +1593,8 @@ pgxc_node_remote_prepare(char *prepareGID)
if ((write_conn_count == 0) || (prepareGID == NULL))
return false;
+ SetSendCommandId(false);
+
/* Save the prepareGID in the global state information */
sprintf(remoteXactState.prepareGID, "%s", prepareGID);
@@ -1732,6 +1754,8 @@ pgxc_node_remote_commit(void)
if (read_conn_count + write_conn_count == 0)
return;
+ SetSendCommandId(false);
+
/*
* Barrier:
*
@@ -1936,6 +1960,7 @@ pgxc_node_remote_abort(void)
int new_conn_count = 0;
RemoteQueryState *combiner = NULL;
+ SetSendCommandId(false);
/* Send COMMIT/ROLLBACK PREPARED TRANSACTION to the remote nodes */
for (i = 0; i < write_conn_count + read_conn_count; i++)
@@ -2529,6 +2554,12 @@ ExecInitRemoteQuery(RemoteQuery *node, EState *estate, int eflags)
ExecAssignResultTypeFromTL(&remotestate->ss.ps);
ExecAssignScanProjectionInfo(&remotestate->ss);
+ if (node->has_ins_child_sel_parent)
+ {
+ /* Save command id of the insert-select query */
+ remotestate->rqs_cmd_id = GetCurrentCommandId(false);
+ }
+
return remotestate;
}
@@ -2696,10 +2727,41 @@ pgxc_start_command_on_connection(PGXCNodeHandle *connection,
RemoteQueryState *remotestate,
Snapshot snapshot)
{
+ CommandId cid;
RemoteQuery *step = (RemoteQuery *) remotestate->ss.ps.plan;
if (connection->state == DN_CONNECTION_STATE_QUERY)
BufferConnection(connection);
+ /*
+ * Scan descriptor would be valid and would contain a valid snapshot
+ * in cases when we need to send out of order command id to data node
+ * e.g. in case of a fetch
+ */
+
+ if (remotestate->cursor != NULL &&
+ remotestate->cursor[0] != '\0' &&
+ remotestate->ss.ss_currentScanDesc != NULL &&
+ remotestate->ss.ss_currentScanDesc->rs_snapshot != NULL)
+ cid = remotestate->ss.ss_currentScanDesc->rs_snapshot->curcid;
+ else
+ {
+ /*
+ * An insert into a child by selecting form its parent gets translated
+ * into a multi-statement transaction in which first we select from parent
+ * and then insert into child, then select form child and insert into child.
+ * The select from child should not see the just inserted rows.
+ * The command id of the select from child is therefore set to
+ * the command id of the insert-select query saved earlier.
+ */
+ if (step->exec_nodes->accesstype == RELATION_ACCESS_READ && step->has_ins_child_sel_parent)
+ cid = remotestate->rqs_cmd_id;
+ else
+ cid = GetCurrentCommandId(false);
+ }
+
+ if (pgxc_node_send_cmd_id(connection, cid) < 0 )
+ return false;
+
if (snapshot && pgxc_node_send_snapshot(connection, snapshot))
return false;
if (step->statement || step->cursor || step->param_types)
@@ -2903,7 +2965,6 @@ do_query(RemoteQueryState *node)
if (step->cursor)
{
- node->cursor = step->cursor;
node->cursor_count = regular_conn_count;
node->cursor_connections = (PGXCNodeHandle **) palloc(regular_conn_count * sizeof(PGXCNodeHandle *));
memcpy(node->cursor_connections, connections, regular_conn_count * sizeof(PGXCNodeHandle *));
diff --git a/src/backend/pgxc/pool/pgxcnode.c b/src/backend/pgxc/pool/pgxcnode.c
index 5b4cf2e289..99afba4e4c 100644
--- a/src/backend/pgxc/pool/pgxcnode.c
+++ b/src/backend/pgxc/pool/pgxcnode.c
@@ -49,6 +49,7 @@
#include "utils/formatting.h"
#include "../interfaces/libpq/libpq-fe.h"
+#define CMD_ID_MSG_LEN 8
/* Number of connections held */
static int datanode_count = 0;
@@ -1596,6 +1597,40 @@ pgxc_node_send_gxid(PGXCNodeHandle *handle, GlobalTransactionId gxid)
return 0;
}
+/*
+ * Send the Command ID down to the PGXC node
+ */
+int
+pgxc_node_send_cmd_id(PGXCNodeHandle *handle, CommandId cid)
+{
+ int msglen = CMD_ID_MSG_LEN;
+ int i32;
+
+ /* No need to send command ID if its sending flag is not enabled */
+ if (!IsSendCommandId())
+ return 0;
+
+ /* Invalid connection state, return error */
+ if (handle->state != DN_CONNECTION_STATE_IDLE)
+ return EOF;
+
+ /* msgType + msgLen */
+ if (ensure_out_buffer_capacity(handle->outEnd + 1 + msglen, handle) != 0)
+ {
+ add_error_message(handle, "out of memory");
+ return EOF;
+ }
+
+ handle->outBuffer[handle->outEnd++] = 'M';
+ msglen = htonl(msglen);
+ memcpy(handle->outBuffer + handle->outEnd, &msglen, 4);
+ handle->outEnd += 4;
+ i32 = htonl(cid);
+ memcpy(handle->outBuffer + handle->outEnd, &i32, 4);
+ handle->outEnd += 4;
+
+ return 0;
+}
/*
* Send the snapshot down to the PGXC node
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index c845e6115b..b1f90c1da8 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -460,6 +460,7 @@ SocketBackend(StringInfo inBuf)
errmsg("invalid frontend message type %d", qtype)));
break;
#ifdef PGXC /* PGXC_DATANODE */
+ case 'M': /* Command ID */
case 'g': /* GXID */
case 's': /* Snapshot */
case 't': /* Timestamp */
@@ -731,7 +732,7 @@ pg_rewrite_query(Query *query)
#ifdef PGXC
else if ((query->commandType == CMD_SELECT) && (query->intoClause != NULL))
{
- /*
+ /*
* CREATE TABLE AS SELECT and SELECT INTO are rewritten so that the
* target table is created first. The SELECT query is then transformed
* into an INSERT INTO statement
@@ -4075,7 +4076,7 @@ PostgresMain(int argc, char *argv[], const char *username)
#ifdef PGXC
/*
* Helps us catch any problems where we did not send down a snapshot
- * when it was expected. However if any deferred trigger is supposed
+ * when it was expected. However if any deferred trigger is supposed
* to be fired at commit time we need to preserve the snapshot sent previously
*/
if ((IS_PGXC_DATANODE || IsConnFromCoord()) && !IsAnyAfterTriggerDeferred())
@@ -4366,6 +4367,14 @@ PostgresMain(int argc, char *argv[], const char *username)
*/
break;
#ifdef PGXC
+ case 'M': /* Command ID */
+ {
+ CommandId cid = (CommandId) pq_getmsgint(&input_message, 4);
+ elog(DEBUG1, "Received cmd id %u", cid);
+ SaveReceivedCommandId(cid);
+ }
+ break;
+
case 'g': /* gxid */
{
/* Set the GXID we were passed down */
diff --git a/src/backend/tcop/pquery.c b/src/backend/tcop/pquery.c
index 2b094dd653..1e91960a85 100644
--- a/src/backend/tcop/pquery.c
+++ b/src/backend/tcop/pquery.c
@@ -25,6 +25,7 @@
#include "pgxc/pgxc.h"
#include "pgxc/planner.h"
#include "pgxc/execRemote.h"
+#include "access/relscan.h"
#endif
#include "tcop/pquery.h"
#include "tcop/tcopprot.h"
@@ -970,6 +971,34 @@ PortalRunSelect(Portal portal,
else
{
PushActiveSnapshot(queryDesc->snapshot);
+
+#ifdef PGXC
+ if (portal->name != NULL &&
+ portal->name[0] != '\0' &&
+ IsA(queryDesc->planstate, RemoteQueryState))
+ {
+ /*
+ * The snapshot in the query descriptor contains the
+ * command id of the command creating the cursor. We copy
+ * that snapshot in RemoteQueryState so that the do_query
+ * function knows while sending the select (resulting from
+ * a fetch) to the corresponding remote node with the command
+ * id of the command that created the cursor.
+ */
+ HeapScanDesc scan;
+ RemoteQueryState *rqs = (RemoteQueryState *)queryDesc->planstate;
+
+ /* Allocate and initialize scan descriptor */
+ scan = (HeapScanDesc) palloc0(sizeof(HeapScanDescData));
+ /* Copy snap shot into the scan descriptor */
+ scan->rs_snapshot = queryDesc->snapshot;
+ /* Copy scan descriptor in remote query state */
+ rqs->ss.ss_currentScanDesc = scan;
+
+ rqs->cursor = pstrdup(portal->name);
+ }
+#endif
+
ExecutorRun(queryDesc, direction, count);
nprocessed = queryDesc->estate->es_processed;
PopActiveSnapshot();
@@ -1324,7 +1353,7 @@ PortalRunMulti(Portal portal, bool isTopLevel,
/* it's special for INSERT */
if (IS_PGXC_COORDINATOR &&
pstmt->commandType == CMD_INSERT)
- HandleCmdComplete(pstmt->commandType, &combine,
+ HandleCmdComplete(pstmt->commandType, &combine,
completionTag, strlen(completionTag));
#endif
}
@@ -1405,7 +1434,7 @@ PortalRunMulti(Portal portal, bool isTopLevel,
* e.g. an INSERT that does an UPDATE instead should not print "0 1" if
* one row was updated. See QueryRewrite(), step 3, for details.
*/
-
+
#ifdef PGXC
if (IS_PGXC_COORDINATOR && combine.data[0] != '\0')
strcpy(completionTag, combine.data);
@@ -1729,4 +1758,3 @@ DoPortalRewind(Portal portal)
portal->portalPos = 0;
portal->posOverflow = false;
}
-
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 740c487b2c..9d31e741b7 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -154,7 +154,33 @@ GetTransactionSnapshot(void)
}
if (IsolationUsesXactSnapshot())
+ {
+#ifdef PGXC
+ /*
+ * Consider this test case taken from portals.sql
+ *
+ * CREATE TABLE cursor (a int, b int) distribute by replication;
+ * INSERT INTO cursor VALUES (10);
+ * BEGIN;
+ * SET TRANSACTION ISOLATION LEVEL SERIALIZABLE;
+ * DECLARE c1 NO SCROLL CURSOR FOR SELECT * FROM cursor FOR UPDATE;
+ * INSERT INTO cursor VALUES (2);
+ * FETCH ALL FROM c1;
+ * would result in
+ * ERROR: attempted to lock invisible tuple
+ * because FETCH would be sent as a select to the remote nodes
+ * with command id 0, whereas the command id would be 2
+ * in the current snapshot.
+ * (1 sent by Coordinator due to declare cursor &
+ * 2 because of the insert inside the transaction)
+ * The command id should therefore be updated in the
+ * current snapshot.
+ */
+ if (IS_PGXC_DATANODE)
+ SnapshotSetCommandId(GetCurrentCommandId(false));
+#endif
return CurrentSnapshot;
+ }
CurrentSnapshot = GetSnapshotData(&CurrentSnapshotData);
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 63221d2d5f..15512c25c0 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -285,6 +285,12 @@ extern void RegisterTransactionLocalNode(bool write);
extern bool IsTransactionLocalNode(bool write);
extern void ForgetTransactionLocalNode(void);
extern bool IsXidImplicit(const char *xid);
+extern void SaveReceivedCommandId(CommandId cid);
+extern void SetReceivedCommandId(CommandId cid);
+extern CommandId GetReceivedCommandId(void);
+extern void ReportCommandIdChange(CommandId cid);
+extern bool IsSendCommandId(void);
+extern void SetSendCommandId(bool status);
#endif
extern int xactGetCommittedChildren(TransactionId **ptr);
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 270108f7e7..e8f2317c1b 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -162,6 +162,8 @@ typedef struct Query
* should Datanode finalise the aggregates? */
bool is_local; /* enforce query execution on local node
* this is used by EXECUTE DIRECT especially. */
+ bool is_ins_child_sel_parent;/* true if the query is such an INSERT SELECT that
+ * inserts into a child by selecting from its parent */
#endif
} Query;
diff --git a/src/include/pgxc/execRemote.h b/src/include/pgxc/execRemote.h
index 9afccf5c8e..32a88ecca4 100644
--- a/src/include/pgxc/execRemote.h
+++ b/src/include/pgxc/execRemote.h
@@ -121,7 +121,7 @@ typedef struct RemoteQueryState
int eflags; /* capability flags to pass to tuplestore */
bool eof_underlying; /* reached end of underlying plan? */
Tuplestorestate *tuplestorestate;
-
+ CommandId rqs_cmd_id; /* Cmd id to use in some special cases */
} RemoteQueryState;
/* Multinode Executor */
diff --git a/src/include/pgxc/pgxcnode.h b/src/include/pgxc/pgxcnode.h
index f98504afd3..87046106cf 100644
--- a/src/include/pgxc/pgxcnode.h
+++ b/src/include/pgxc/pgxcnode.h
@@ -172,6 +172,7 @@ extern int pgxc_node_send_query_extended(PGXCNodeHandle *handle, const char *que
int paramlen, char *params,
bool send_describe, int fetch_size);
extern int pgxc_node_send_gxid(PGXCNodeHandle * handle, GlobalTransactionId gxid);
+extern int pgxc_node_send_cmd_id(PGXCNodeHandle *handle, CommandId cid);
extern int pgxc_node_send_snapshot(PGXCNodeHandle * handle, Snapshot snapshot);
extern int pgxc_node_send_timestamp(PGXCNodeHandle * handle, TimestampTz timestamp);
diff --git a/src/include/pgxc/planner.h b/src/include/pgxc/planner.h
index 20054810bd..d70c51b853 100644
--- a/src/include/pgxc/planner.h
+++ b/src/include/pgxc/planner.h
@@ -110,6 +110,8 @@ typedef struct
char *outer_statement;
char *join_condition;
bool has_row_marks; /* Did SELECT had FOR UPDATE/SHARE? */
+ bool has_ins_child_sel_parent; /* This node is part of an INSERT SELECT that
+ * inserts into child by selecting from its parent */
} RemoteQuery;
/*
diff --git a/src/test/regress/expected/combocid_1.out b/src/test/regress/expected/combocid_1.out
index 2b57c7d9f3..efcac048e9 100644
--- a/src/test/regress/expected/combocid_1.out
+++ b/src/test/regress/expected/combocid_1.out
@@ -21,8 +21,8 @@ INSERT INTO combocidtest VALUES (2);
SELECT ctid,cmin,* FROM combocidtest ORDER BY ctid;
ctid | cmin | foobar
-------+------+--------
- (0,1) | 0 | 1
- (0,2) | 1 | 2
+ (0,1) | 10 | 1
+ (0,2) | 11 | 2
(2 rows)
SAVEPOINT s1;
@@ -77,7 +77,7 @@ INSERT INTO combocidtest VALUES (444);
SELECT ctid,cmin,* FROM combocidtest ORDER BY ctid;
ctid | cmin | foobar
-------+------+--------
- (0,4) | 0 | 444
+ (0,4) | 10 | 444
(1 row)
SAVEPOINT s1;
diff --git a/src/test/regress/expected/plpgsql_1.out b/src/test/regress/expected/plpgsql_1.out
index 70faf91e9c..e21396e6d8 100644
--- a/src/test/regress/expected/plpgsql_1.out
+++ b/src/test/regress/expected/plpgsql_1.out
@@ -3014,7 +3014,7 @@ declare
c refcursor;
x integer;
begin
- open c scroll for execute 'select f1 from int4_tbl';
+ open c scroll for execute 'select f1 from int4_tbl order by 1';
fetch last from c into x;
while found loop
return next x;
@@ -3023,12 +3023,12 @@ begin
close c;
end;
$$ language plpgsql;
-select * from sc_test() order by 1;
- sc_test
-------------
- 0
- 123456
- 2147483647
+select * from sc_test();
+ sc_test
+-------------
+ 2147483647
+ 0
+ -2147483647
(3 rows)
create or replace function sc_test() returns setof integer as $$
@@ -3036,7 +3036,7 @@ declare
c refcursor;
x integer;
begin
- open c scroll for execute 'select f1 from int4_tbl';
+ open c scroll for execute 'select f1 from int4_tbl order by 1';
fetch last from c into x;
while found loop
return next x;
@@ -3046,11 +3046,11 @@ begin
close c;
end;
$$ language plpgsql;
-select * from sc_test() order by 1;
- sc_test
--------------
- -2147483647
- 2147483647
+select * from sc_test();
+ sc_test
+------------
+ 2147483647
+ -123456
(2 rows)
create or replace function sc_test() returns setof integer as $$
diff --git a/src/test/regress/expected/portals_1.out b/src/test/regress/expected/portals_1.out
index 576ea6f534..f28bb80d86 100644
--- a/src/test/regress/expected/portals_1.out
+++ b/src/test/regress/expected/portals_1.out
@@ -1184,8 +1184,7 @@ UPDATE cursor SET a = 2;
FETCH ALL FROM c1;
a | b
---+---
- 2 |
-(1 row)
+(0 rows)
COMMIT;
DROP TABLE cursor;
diff --git a/src/test/regress/expected/select_views_2.out b/src/test/regress/expected/select_views_2.out
index b3f54f821d..6aa3fb5075 100644
--- a/src/test/regress/expected/select_views_2.out
+++ b/src/test/regress/expected/select_views_2.out
@@ -40,7 +40,6 @@ SELECT * FROM street ORDER BY name,cname,thepath::text;
Avenue 140th | [(-122.1656,37.003),(-122.1691,37.988)] | Berkeley
Avenue 140th | [(-122.1656,37.003),(-122.1691,37.988)] | Oakland
Avenue D | [(-122.298,37.848),(-122.3024,37.849)] | Berkeley
- B St | [(-122.1749,37.451),(-122.1743,37.443)] | Oakland
Bancroft Ave | [(-122.15714,37.4242),(-122.156,37.409)] | Oakland
Bancroft Ave | [(-122.1643,37.523),(-122.1631,37.508),(-122.1621,37.493)] | Oakland
Birch St | [(-122.1617,37.425),(-122.1614,37.417)] | Oakland
@@ -49,11 +48,11 @@ SELECT * FROM street ORDER BY name,cname,thepath::text;
Bridgepointe Dr | [(-122.0514,37.305),(-122.0509,37.299)] | Oakland
Broadmore Ave | [(-122.095,37.522),(-122.0936,37.497)] | Oakland
Broadway | [(-122.2409,37.586),(-122.2395,37.601)] | Berkeley
+ B St | [(-122.1749,37.451),(-122.1743,37.443)] | Oakland
Buckingham Blvd | [(-122.2231,37.59),(-122.2214,37.606)] | Berkeley
Butterfield Dr | [(-122.0838,37.002),(-122.0834,37.987)] | Berkeley
Butterfield Dr | [(-122.0838,37.002),(-122.0834,37.987)] | Oakland
Butterfield Dr | [(-122.0838,37.002),(-122.0834,37.987)] | Oakland
- C St | [(-122.1768,37.46),(-122.1749,37.435)] | Oakland
Calaveras Creek | [(-121.8203,37.035),(-121.8207,37.931)] | Oakland
Calaveras Creek | [(-121.8203,37.035),(-121.8207,37.931)] | Oakland
California St | [(-122.2032,37.005),(-122.2016,37.996)] | Berkeley
@@ -87,9 +86,9 @@ SELECT * FROM street ORDER BY name,cname,thepath::text;
Creston Road | [(-122.2639,37.002),(-122.2613,37.986),(-122.2602,37.978),(-122.2598,37.973)] | Lafayette
Crow Canyon Creek | [(-122.043,37.905),(-122.0368,37.71)] | Berkeley
Crystaline Dr | [(-121.925856,37),(-121.925869,37.00527)] | Oakland
+ C St | [(-122.1768,37.46),(-122.1749,37.435)] | Oakland
Cull Canyon Road | [(-122.0536,37.435),(-122.0499,37.315)] | Oakland
Cull Creek | [(-122.0624,37.875),(-122.0582,37.527)] | Berkeley
- D St | [(-122.1811,37.505),(-122.1805,37.497)] | Oakland
Decoto Road | [(-122.0159,37.006),(-122.016,37.002),(-122.0164,37.993)] | Berkeley
Decoto Road | [(-122.0159,37.006),(-122.016,37.002),(-122.0164,37.993)] | Oakland
Decoto Road | [(-122.0159,37.006),(-122.016,37.002),(-122.0164,37.993)] | Oakland
@@ -100,13 +99,14 @@ SELECT * FROM street ORDER BY name,cname,thepath::text;
Driftwood Dr | [(-122.0109,37.482),(-122.0113,37.477)] | Oakland
Driscoll Road | [(-121.9482,37.403),(-121.948451,37.39995)] | Oakland
Driscoll Road | [(-121.9482,37.403),(-121.948451,37.39995)] | Oakland
- E St | [(-122.1832,37.505),(-122.1826,37.498),(-122.182,37.49)] | Oakland
+ D St | [(-122.1811,37.505),(-122.1805,37.497)] | Oakland
Eden Ave | [(-122.1143,37.505),(-122.1142,37.491)] | Oakland
Eden Creek | [(-122.022037,37.00675),(-122.0221,37.998)] | Berkeley
Eden Creek | [(-122.022037,37.00675),(-122.0221,37.998)] | Oakland
Eden Creek | [(-122.022037,37.00675),(-122.0221,37.998)] | Oakland
Edgewater Dr | [(-122.201,37.379),(-122.2042,37.41)] | Lafayette
Enos Way | [(-121.7677,37.896),(-121.7673,37.91)] | Oakland
+ E St | [(-122.1832,37.505),(-122.1826,37.498),(-122.182,37.49)] | Oakland
Euclid Ave | [(-122.2671,37.009),(-122.2666,37.987)] | Berkeley
Euclid Ave | [(-122.2671,37.009),(-122.2666,37.987)] | Lafayette
Fairview Ave | [(-121.999,37.428),(-121.9863,37.351)] | Oakland
@@ -136,6 +136,8 @@ SELECT * FROM street ORDER BY name,cname,thepath::text;
I- 580 | [(-122.018,37.019),(-122.0009,37.032),(-121.9787,37.983),(-121.958,37.984),(-121.9571,37.986)] | Oakland
I- 580 | [(-122.1108,37.023),(-122.1101,37.02),(-122.108103,37.00764),(-122.108,37.007),(-122.1069,37.998),(-122.1064,37.994),(-122.1053,37.982),(-122.1048,37.977),(-122.1032,37.958),(-122.1026,37.953),(-122.1013,37.938),(-122.0989,37.911),(-122.0984,37.91),(-122.098,37.908)] | Oakland
I- 580 | [(-122.1543,37.703),(-122.1535,37.694),(-122.1512,37.655),(-122.1475,37.603),(-122.1468,37.583),(-122.1472,37.569),(-122.149044,37.54874),(-122.1493,37.546),(-122.1501,37.532),(-122.1506,37.509),(-122.1495,37.482),(-122.1487,37.467),(-122.1477,37.447),(-122.1414,37.383),(-122.1404,37.376),(-122.1398,37.372),(-122.139,37.356),(-122.1388,37.353),(-122.1385,37.34),(-122.1382,37.33),(-122.1378,37.316)] | Oakland
+ I- 580/I-680 Ramp | ((-121.9207,37.988),(-121.9192,37.016)) | Oakland
+ I- 580/I-680 Ramp | ((-121.9207,37.988),(-121.9192,37.016)) | Oakland
I- 580 Ramp | [(-122.093241,37.90351),(-122.09364,37.89634),(-122.093788,37.89212)] | Berkeley
I- 580 Ramp | [(-122.0934,37.896),(-122.09257,37.89961),(-122.0911,37.906)] | Berkeley
I- 580 Ramp | [(-122.0941,37.897),(-122.0943,37.902)] | Berkeley
@@ -152,12 +154,10 @@ SELECT * FROM street ORDER BY name,cname,thepath::text;
I- 580 Ramp | [(-121.9368,37.986),(-121.936483,37.98832),(-121.9353,37.997),(-121.93504,37.00035),(-121.9346,37.006),(-121.933764,37.00031),(-121.9333,37.997),(-121.9322,37.989)] | Oakland
I- 580 Ramp | [(-122.1086,37.003),(-122.1068,37.993),(-122.1066,37.992),(-122.1053,37.982)] | Oakland
I- 580 Ramp | [(-122.1414,37.383),(-122.1407,37.376),(-122.1403,37.372),(-122.139,37.356)] | Oakland
- I- 580/I-680 Ramp | ((-121.9207,37.988),(-121.9192,37.016)) | Oakland
- I- 580/I-680 Ramp | ((-121.9207,37.988),(-121.9192,37.016)) | Oakland
- I- 680 | ((-121.939,37.15),(-121.9387,37.145),(-121.9373,37.125),(-121.934242,37.07643),(-121.933886,37.0709),(-121.9337,37.068),(-121.933122,37.06139),(-121.932736,37.05698),(-121.93222,37.05108),(-121.931844,37.04678),(-121.930113,37.027),(-121.926829,37),(-121.9265,37.998),(-121.9217,37.96),(-121.9203,37.949),(-121.9184,37.934)) | Oakland
- I- 680 | ((-121.939,37.15),(-121.9387,37.145),(-121.9373,37.125),(-121.934242,37.07643),(-121.933886,37.0709),(-121.9337,37.068),(-121.933122,37.06139),(-121.932736,37.05698),(-121.93222,37.05108),(-121.931844,37.04678),(-121.930113,37.027),(-121.926829,37),(-121.9265,37.998),(-121.9217,37.96),(-121.9203,37.949),(-121.9184,37.934)) | Oakland
I- 680 | [(-121.9101,37.715),(-121.911269,37.74682),(-121.9119,37.764),(-121.9124,37.776),(-121.9174,37.905),(-121.9194,37.957),(-121.9207,37.988)] | Oakland
I- 680 | [(-121.9184,37.934),(-121.917,37.913),(-121.9122,37.83),(-121.9052,37.702)] | Oakland
+ I- 680 | ((-121.939,37.15),(-121.9387,37.145),(-121.9373,37.125),(-121.934242,37.07643),(-121.933886,37.0709),(-121.9337,37.068),(-121.933122,37.06139),(-121.932736,37.05698),(-121.93222,37.05108),(-121.931844,37.04678),(-121.930113,37.027),(-121.926829,37),(-121.9265,37.998),(-121.9217,37.96),(-121.9203,37.949),(-121.9184,37.934)) | Oakland
+ I- 680 | ((-121.939,37.15),(-121.9387,37.145),(-121.9373,37.125),(-121.934242,37.07643),(-121.933886,37.0709),(-121.9337,37.068),(-121.933122,37.06139),(-121.932736,37.05698),(-121.93222,37.05108),(-121.931844,37.04678),(-121.930113,37.027),(-121.926829,37),(-121.9265,37.998),(-121.9217,37.96),(-121.9203,37.949),(-121.9184,37.934)) | Oakland
I- 680 Ramp | [(-121.8833,37.376),(-121.8833,37.392),(-121.883,37.4),(-121.8835,37.402),(-121.8852,37.422)] | Oakland
I- 680 Ramp | [(-121.8833,37.376),(-121.8833,37.392),(-121.883,37.4),(-121.8835,37.402),(-121.8852,37.422)] | Oakland
I- 680 Ramp | [(-121.92,37.438),(-121.9218,37.424),(-121.9238,37.408),(-121.9252,37.392)] | Oakland
@@ -173,9 +173,9 @@ SELECT * FROM street ORDER BY name,cname,thepath::text;
I- 880 | [(-122.1755,37.185),(-122.1747,37.178),(-122.1742,37.173),(-122.1692,37.126),(-122.167792,37.11594),(-122.16757,37.11435),(-122.1671,37.111),(-122.1655,37.1),(-122.165169,37.09811),(-122.1641,37.092),(-122.1596,37.061),(-122.158381,37.05275),(-122.155991,37.03657),(-122.1531,37.017),(-122.1478,37.98),(-122.1407,37.932),(-122.1394,37.924),(-122.1389,37.92),(-122.1376,37.91)] | Berkeley
I- 880 | [(-122.2214,37.711),(-122.2202,37.699),(-122.2199,37.695),(-122.219,37.682),(-122.2184,37.672),(-122.2173,37.652),(-122.2159,37.638),(-122.2144,37.616),(-122.2138,37.612),(-122.2135,37.609),(-122.212,37.592),(-122.2116,37.586),(-122.2111,37.581)] | Berkeley
I- 880 | [(-122.2707,37.975),(-122.2693,37.972),(-122.2681,37.966),(-122.267,37.962),(-122.2659,37.957),(-122.2648,37.952),(-122.2636,37.946),(-122.2625,37.935),(-122.2617,37.927),(-122.2607,37.921),(-122.2593,37.916),(-122.258,37.911),(-122.2536,37.898),(-122.2432,37.858),(-122.2408,37.845),(-122.2386,37.827),(-122.2374,37.811)] | Berkeley
+ I- 880 | [(-121.948,37.933),(-121.9471,37.925),(-121.9467,37.923),(-121.946,37.918),(-121.9452,37.912),(-121.937,37.852)] | Oakland
I- 880 | ((-121.9669,37.075),(-121.9663,37.071),(-121.9656,37.065),(-121.9618,37.037),(-121.95689,37),(-121.948,37.933)) | Oakland
I- 880 | ((-121.9669,37.075),(-121.9663,37.071),(-121.9656,37.065),(-121.9618,37.037),(-121.95689,37),(-121.948,37.933)) | Oakland
- I- 880 | [(-121.948,37.933),(-121.9471,37.925),(-121.9467,37.923),(-121.946,37.918),(-121.9452,37.912),(-121.937,37.852)] | Oakland
I- 880 | [(-122.0219,37.466),(-122.0205,37.447),(-122.020331,37.44447),(-122.020008,37.43962),(-122.0195,37.432),(-122.0193,37.429),(-122.0164,37.393),(-122.010219,37.34771),(-122.0041,37.313)] | Oakland
I- 880 | [(-122.0375,37.632),(-122.0359,37.619),(-122.0358,37.616),(-122.034514,37.60409),(-122.031876,37.57965),(-122.031193,37.57332),(-122.03016,37.56375),(-122.02943,37.55698),(-122.028689,37.54929),(-122.027833,37.53908),(-122.025979,37.51698),(-122.0238,37.491)] | Oakland
I- 880 | [(-122.0612,37.003),(-122.0604,37.991),(-122.0596,37.982),(-122.0585,37.967),(-122.0583,37.961),(-122.0553,37.918),(-122.053635,37.89475),(-122.050759,37.8546),(-122.05,37.844),(-122.0485,37.817),(-122.0483,37.813),(-122.0482,37.811)] | Oakland
@@ -211,12 +211,12 @@ SELECT * FROM street ORDER BY name,cname,thepath::text;
Kaiser Dr | [(-122.067163,37.47821),(-122.060402,37.51961)] | Oakland
Keeler Ave | [(-122.2578,37.906),(-122.2579,37.899)] | Berkeley
Kildare Road | [(-122.0968,37.016),(-122.0959,37)] | Oakland
- La Playa Dr | [(-122.1039,37.545),(-122.101,37.493)] | Oakland
Laguna Ave | [(-122.2099,37.989),(-122.2089,37)] | Berkeley
Laguna Ave | [(-122.2099,37.989),(-122.2089,37)] | Lafayette
Lakehurst Cir | [(-122.284729,37.89025),(-122.286096,37.90364)] | Berkeley
Lakeshore Ave | [(-122.2586,37.99),(-122.2556,37.006)] | Berkeley
Lakeshore Ave | [(-122.2586,37.99),(-122.2556,37.006)] | Lafayette
+ La Playa Dr | [(-122.1039,37.545),(-122.101,37.493)] | Oakland
Las Positas Road | [(-121.764488,37.99199),(-121.75569,37.02022)] | Oakland
Las Positas Road | [(-121.764488,37.99199),(-121.75569,37.02022)] | Oakland
Linden St | [(-122.2867,37.998),(-122.2864,37.008)] | Berkeley
@@ -328,10 +328,10 @@ SELECT * FROM street ORDER BY name,cname,thepath::text;
Warm Springs Blvd | [(-121.933956,37),(-121.9343,37.97)] | Oakland
Welch Creek Road | [(-121.7695,37.386),(-121.7737,37.413)] | Oakland
Welch Creek Road | [(-121.7695,37.386),(-121.7737,37.413)] | Oakland
- West Loop Road | [(-122.0576,37.604),(-122.0602,37.586)] | Berkeley
Western Pacific Railroad Spur | [(-122.0394,37.018),(-122.0394,37.961)] | Berkeley
Western Pacific Railroad Spur | [(-122.0394,37.018),(-122.0394,37.961)] | Oakland
Western Pacific Railroad Spur | [(-122.0394,37.018),(-122.0394,37.961)] | Oakland
+ West Loop Road | [(-122.0576,37.604),(-122.0602,37.586)] | Berkeley
Whitlock Creek | [(-121.74683,37.91276),(-121.733107,37)] | Oakland
Whitlock Creek | [(-121.74683,37.91276),(-121.733107,37)] | Oakland
Willimet Way | [(-122.0964,37.517),(-122.0949,37.493)] | Oakland
@@ -341,9 +341,905 @@ SELECT * FROM street ORDER BY name,cname,thepath::text;
(333 rows)
SELECT name, #thepath FROM iexit ORDER BY 1, 2;
- name | ?column?
-------+----------
-(0 rows)
+ name | ?column?
+------------------------------------+----------
+ I- 580 | 2
+ I- 580 | 2
+ I- 580 | 2
+ I- 580 | 2
+ I- 580 | 2
+ I- 580 | 2
+ I- 580 | 2
+ I- 580 | 2
+ I- 580 | 2
+ I- 580 | 2
+ I- 580 | 2
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 3
+ I- 580 | 4
+ I- 580 | 4
+ I- 580 | 4
+ I- 580 | 4
+ I- 580 | 5
+ I- 580 | 5
+ I- 580 | 5
+ I- 580 | 5
+ I- 580 | 5
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 6
+ I- 580 | 7
+ I- 580 | 7
+ I- 580 | 7
+ I- 580 | 7
+ I- 580 | 7
+ I- 580 | 7
+ I- 580 | 7
+ I- 580 | 8
+ I- 580 | 8
+ I- 580 | 8
+ I- 580 | 8
+ I- 580 | 8
+ I- 580 | 8
+ I- 580 | 8
+ I- 580 | 8
+ I- 580 | 8
+ I- 580 | 9
+ I- 580 | 9
+ I- 580 | 9
+ I- 580 | 9
+ I- 580 | 9
+ I- 580 | 12
+ I- 580 | 12
+ I- 580 | 12
+ I- 580 | 12
+ I- 580 | 12
+ I- 580 | 12
+ I- 580 | 12
+ I- 580 | 12
+ I- 580 | 12
+ I- 580 | 12
+ I- 580 | 13
+ I- 580 | 13
+ I- 580 | 13
+ I- 580 | 13
+ I- 580 | 13
+ I- 580 | 13
+ I- 580 | 14
+ I- 580 | 14
+ I- 580 | 14
+ I- 580 | 14
+ I- 580 | 14
+ I- 580 | 14
+ I- 580 | 14
+ I- 580 | 14
+ I- 580 | 18
+ I- 580 | 18
+ I- 580 | 18
+ I- 580 | 18
+ I- 580 | 18
+ I- 580 | 18
+ I- 580 | 21
+ I- 580 | 21
+ I- 580 | 21
+ I- 580 | 21
+ I- 580 | 21
+ I- 580 | 21
+ I- 580 | 21
+ I- 580 | 21
+ I- 580 | 21
+ I- 580 | 21
+ I- 580 | 22
+ I- 580 | 22
+ I- 580/I-680 Ramp | 2
+ I- 580/I-680 Ramp | 2
+ I- 580/I-680 Ramp | 2
+ I- 580/I-680 Ramp | 2
+ I- 580/I-680 Ramp | 2
+ I- 580/I-680 Ramp | 2
+ I- 580/I-680 Ramp | 4
+ I- 580/I-680 Ramp | 4
+ I- 580/I-680 Ramp | 4
+ I- 580/I-680 Ramp | 4
+ I- 580/I-680 Ramp | 5
+ I- 580/I-680 Ramp | 6
+ I- 580/I-680 Ramp | 6
+ I- 580/I-680 Ramp | 6
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 2
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 3
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 4
+ I- 580 Ramp | 5
+ I- 580 Ramp | 5
+ I- 580 Ramp | 5
+ I- 580 Ramp | 5
+ I- 580 Ramp | 5
+ I- 580 Ramp | 5
+ I- 580 Ramp | 5
+ I- 580 Ramp | 5
+ I- 580 Ramp | 5
+ I- 580 Ramp | 5
+ I- 580 Ramp | 5
+ I- 580 Ramp | 5
+ I- 580 Ramp | 5
+ I- 580 Ramp | 5
+ I- 580 Ramp | 6
+ I- 580 Ramp | 6
+ I- 580 Ramp | 6
+ I- 580 Ramp | 7
+ I- 580 Ramp | 8
+ I- 580 Ramp | 8
+ I- 580 Ramp | 8
+ I- 580 Ramp | 8
+ I- 580 Ramp | 8
+ I- 580 Ramp | 8
+ I- 680 | 2
+ I- 680 | 2
+ I- 680 | 2
+ I- 680 | 2
+ I- 680 | 2
+ I- 680 | 2
+ I- 680 | 2
+ I- 680 | 3
+ I- 680 | 3
+ I- 680 | 3
+ I- 680 | 4
+ I- 680 | 4
+ I- 680 | 4
+ I- 680 | 5
+ I- 680 | 5
+ I- 680 | 5
+ I- 680 | 7
+ I- 680 | 7
+ I- 680 | 7
+ I- 680 | 7
+ I- 680 | 8
+ I- 680 | 8
+ I- 680 | 8
+ I- 680 | 8
+ I- 680 | 10
+ I- 680 | 10
+ I- 680 | 10
+ I- 680 | 10
+ I- 680 | 10
+ I- 680 | 10
+ I- 680 | 10
+ I- 680 | 16
+ I- 680 | 16
+ I- 680 | 16
+ I- 680 | 16
+ I- 680 | 16
+ I- 680 | 16
+ I- 680 | 16
+ I- 680 | 16
+ I- 680 Ramp | 2
+ I- 680 Ramp | 2
+ I- 680 Ramp | 2
+ I- 680 Ramp | 2
+ I- 680 Ramp | 2
+ I- 680 Ramp | 2
+ I- 680 Ramp | 2
+ I- 680 Ramp | 2
+ I- 680 Ramp | 2
+ I- 680 Ramp | 2
+ I- 680 Ramp | 2
+ I- 680 Ramp | 2
+ I- 680 Ramp | 2
+ I- 680 Ramp | 2
+ I- 680 Ramp | 2
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 3
+ I- 680 Ramp | 4
+ I- 680 Ramp | 4
+ I- 680 Ramp | 4
+ I- 680 Ramp | 5
+ I- 680 Ramp | 5
+ I- 680 Ramp | 5
+ I- 680 Ramp | 5
+ I- 680 Ramp | 5
+ I- 680 Ramp | 5
+ I- 680 Ramp | 6
+ I- 680 Ramp | 6
+ I- 680 Ramp | 6
+ I- 680 Ramp | 6
+ I- 680 Ramp | 7
+ I- 680 Ramp | 7
+ I- 680 Ramp | 7
+ I- 680 Ramp | 7
+ I- 680 Ramp | 8
+ I- 680 Ramp | 8
+ I- 680 Ramp | 8
+ I- 680 Ramp | 8
+ I- 80 | 2
+ I- 80 | 2
+ I- 80 | 2
+ I- 80 | 2
+ I- 80 | 2
+ I- 80 | 2
+ I- 80 | 2
+ I- 80 | 2
+ I- 80 | 2
+ I- 80 | 2
+ I- 80 | 2
+ I- 80 | 2
+ I- 80 | 2
+ I- 80 | 2
+ I- 80 | 3
+ I- 80 | 3
+ I- 80 | 3
+ I- 80 | 4
+ I- 80 | 4
+ I- 80 | 4
+ I- 80 | 4
+ I- 80 | 4
+ I- 80 | 5
+ I- 80 | 5
+ I- 80 | 5
+ I- 80 | 5
+ I- 80 | 5
+ I- 80 | 5
+ I- 80 | 5
+ I- 80 | 5
+ I- 80 | 5
+ I- 80 | 11
+ I- 80 | 11
+ I- 80 | 11
+ I- 80 | 11
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 2
+ I- 80 Ramp | 3
+ I- 80 Ramp | 3
+ I- 80 Ramp | 3
+ I- 80 Ramp | 3
+ I- 80 Ramp | 3
+ I- 80 Ramp | 3
+ I- 80 Ramp | 3
+ I- 80 Ramp | 3
+ I- 80 Ramp | 3
+ I- 80 Ramp | 4
+ I- 80 Ramp | 4
+ I- 80 Ramp | 4
+ I- 80 Ramp | 4
+ I- 80 Ramp | 5
+ I- 80 Ramp | 5
+ I- 80 Ramp | 5
+ I- 80 Ramp | 5
+ I- 80 Ramp | 5
+ I- 80 Ramp | 5
+ I- 80 Ramp | 5
+ I- 80 Ramp | 7
+ I- 80 Ramp | 7
+ I- 80 Ramp | 7
+ I- 80 Ramp | 7
+ I- 880 | 2
+ I- 880 | 2
+ I- 880 | 2
+ I- 880 | 2
+ I- 880 | 2
+ I- 880 | 5
+ I- 880 | 5
+ I- 880 | 5
+ I- 880 | 5
+ I- 880 | 5
+ I- 880 | 5
+ I- 880 | 6
+ I- 880 | 6
+ I- 880 | 6
+ I- 880 | 6
+ I- 880 | 6
+ I- 880 | 6
+ I- 880 | 6
+ I- 880 | 6
+ I- 880 | 6
+ I- 880 | 6
+ I- 880 | 6
+ I- 880 | 6
+ I- 880 | 6
+ I- 880 | 6
+ I- 880 | 7
+ I- 880 | 7
+ I- 880 | 7
+ I- 880 | 7
+ I- 880 | 7
+ I- 880 | 7
+ I- 880 | 7
+ I- 880 | 9
+ I- 880 | 9
+ I- 880 | 9
+ I- 880 | 9
+ I- 880 | 9
+ I- 880 | 9
+ I- 880 | 9
+ I- 880 | 10
+ I- 880 | 10
+ I- 880 | 10
+ I- 880 | 10
+ I- 880 | 10
+ I- 880 | 10
+ I- 880 | 10
+ I- 880 | 10
+ I- 880 | 10
+ I- 880 | 10
+ I- 880 | 10
+ I- 880 | 10
+ I- 880 | 12
+ I- 880 | 12
+ I- 880 | 12
+ I- 880 | 12
+ I- 880 | 12
+ I- 880 | 12
+ I- 880 | 12
+ I- 880 | 12
+ I- 880 | 12
+ I- 880 | 12
+ I- 880 | 12
+ I- 880 | 13
+ I- 880 | 13
+ I- 880 | 13
+ I- 880 | 13
+ I- 880 | 13
+ I- 880 | 13
+ I- 880 | 13
+ I- 880 | 13
+ I- 880 | 13
+ I- 880 | 13
+ I- 880 | 13
+ I- 880 | 13
+ I- 880 | 14
+ I- 880 | 14
+ I- 880 | 14
+ I- 880 | 14
+ I- 880 | 14
+ I- 880 | 14
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 17
+ I- 880 | 19
+ I- 880 | 19
+ I- 880 | 19
+ I- 880 | 19
+ I- 880 | 19
+ I- 880 | 19
+ I- 880 | 19
+ I- 880 | 19
+ I- 880 | 19
+ I- 880 | 19
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 2
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 3
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 4
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 5
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 6
+ I- 880 Ramp | 8
+ I- 880 Ramp | 8
+ I- 880 Ramp | 8
+ I- 980 | 2
+ I- 980 | 2
+ I- 980 | 2
+ I- 980 | 2
+ I- 980 | 2
+ I- 980 | 2
+ I- 980 | 2
+ I- 980 | 2
+ I- 980 | 3
+ I- 980 | 3
+ I- 980 | 3
+ I- 980 | 3
+ I- 980 | 3
+ I- 980 | 3
+ I- 980 | 3
+ I- 980 | 3
+ I- 980 | 3
+ I- 980 | 4
+ I- 980 | 4
+ I- 980 | 5
+ I- 980 | 5
+ I- 980 | 7
+ I- 980 | 7
+ I- 980 | 7
+ I- 980 | 7
+ I- 980 | 12
+ I- 980 Ramp | 3
+ I- 980 Ramp | 3
+ I- 980 Ramp | 3
+ I- 980 Ramp | 7
+(896 rows)
SELECT * FROM toyemp WHERE name = 'sharon';
name | age | location | annualsal
diff --git a/src/test/regress/expected/xc_misc.out b/src/test/regress/expected/xc_misc.out
index 9f11ff59b0..0354b122c1 100644
--- a/src/test/regress/expected/xc_misc.out
+++ b/src/test/regress/expected/xc_misc.out
@@ -95,7 +95,7 @@ drop function test_fun_2();
drop table tab4;
drop table tab5;
drop table my_tab1;
--- Test to make sure that the block of
+-- Test to make sure that the
-- INSERT SELECT in case of inserts into a child by selecting from
-- a parent works fine
create table t_11 ( a int, b int);
@@ -104,21 +104,17 @@ insert into t_11 values(1,2),(3,4);
insert into t_22 select * from t_11; -- should pass
CREATE TABLE c_11 () INHERITS (t_11);
insert into c_11 select * from t_22; -- should pass
-insert into c_11 select * from t_11; -- should fail
-ERROR: INSERT SELECT is not supported when inserts are done in a child by selecting from its parent
-insert into c_11 (select * from t_11 union all select * from t_22); -- should fail
-ERROR: INSERT SELECT is not supported when inserts are done in a child by selecting from its parent
-insert into c_11 (select * from t_11,t_22); -- should fail
-ERROR: INSERT SELECT is not supported when inserts are done in a child by selecting from its parent
+insert into c_11 select * from t_11; -- should insert 2
+insert into c_11 (select * from t_11 union all select * from t_22);
+insert into c_11 (select t_11.a, t_22.b from t_11,t_22);
insert into c_11 (select * from t_22 where a in (select a from t_11)); -- should pass
-insert into c_11 (select * from t_11 where a in (select a from t_22)); -- should fail
-ERROR: INSERT SELECT is not supported when inserts are done in a child by selecting from its parent
+insert into c_11 (select * from t_11 where a in (select a from t_22));
insert into t_11 select * from c_11; -- should pass
-- test to make sure count from a parent table works fine
select count(*) from t_11;
count
-------
- 10
+ 222
(1 row)
CREATE TABLE grand_parent (code int, population float, altitude int);
@@ -128,8 +124,7 @@ INSERT INTO my_parent VALUES (1, 2.1, 73);
CREATE TABLE child_11 () INHERITS (my_parent);
CREATE TABLE grand_child () INHERITS (child_11);
INSERT INTO child_11 SELECT * FROM grand_parent; -- should pass
-INSERT INTO child_11 SELECT * FROM my_parent; -- should fail
-ERROR: INSERT SELECT is not supported when inserts are done in a child by selecting from its parent
+INSERT INTO child_11 SELECT * FROM my_parent;
INSERT INTO grand_child SELECT * FROM my_parent; -- should pass
INSERT INTO grand_child SELECT * FROM grand_parent; -- should pass
drop table grand_child;
@@ -139,3 +134,171 @@ drop table grand_parent;
drop table c_11;
drop table t_22;
drop table t_11;
+---------------------------------
+-- Ensure that command ids are sent to data nodes and are reported back to coordinator
+---------------------------------
+create table my_tbl( f1 int);
+begin;
+ insert into my_tbl values(100),(101),(102),(103),(104),(105);
+end;
+select cmin, cmax, * from my_tbl order by f1; -- command id should be in sequence and increasing
+ cmin | cmax | f1
+------+------+-----
+ 0 | 0 | 100
+ 1 | 1 | 101
+ 2 | 2 | 102
+ 3 | 3 | 103
+ 4 | 4 | 104
+ 5 | 5 | 105
+(6 rows)
+
+---------------------------------
+-- Ensure that command id is consumed by declare cursor
+---------------------------------
+begin;
+ DECLARE c1 CURSOR FOR SELECT * FROM my_tbl;
+ INSERT INTO my_tbl VALUES (200);
+ select cmin, cmax,* from my_tbl where f1 = 200; -- should give 1 as command id of row containing 200
+ cmin | cmax | f1
+------+------+-----
+ 1 | 1 | 200
+(1 row)
+
+end;
+---------------------------------
+-- insert into child by seleting from parent
+---------------------------------
+create table tt_11 ( a int, b int);
+insert into tt_11 values(1,2),(3,4);
+CREATE TABLE cc_11 () INHERITS (tt_11);
+insert into cc_11 select * from tt_11;
+select * from cc_11 order by a; -- should insert 2 rows
+ a | b
+---+---
+ 1 | 2
+ 3 | 4
+(2 rows)
+
+begin;
+ insert into cc_11 values(5,6);
+ insert into cc_11 select * from tt_11; -- should insert the row (5,6)
+end;
+select * from cc_11 order by a;
+ a | b
+---+---
+ 1 | 2
+ 1 | 2
+ 1 | 2
+ 3 | 4
+ 3 | 4
+ 3 | 4
+ 5 | 6
+ 5 | 6
+(8 rows)
+
+---------------------------------
+create table tt_33 ( a int, b int);
+insert into tt_33 values(1,2),(3,4);
+CREATE TABLE cc_33 () INHERITS (tt_33);
+insert into cc_33 select * from tt_33;
+begin;
+ insert into cc_33 values(5,6);
+ insert into cc_33 select * from tt_33; -- should insert row (5,6)
+ insert into cc_33 values(7,8);
+ select * from cc_33 order by a;
+ a | b
+---+---
+ 1 | 2
+ 1 | 2
+ 1 | 2
+ 3 | 4
+ 3 | 4
+ 3 | 4
+ 5 | 6
+ 5 | 6
+ 7 | 8
+(9 rows)
+
+ insert into cc_33 select * from tt_33; -- should insert row (7,8)
+end;
+select * from cc_33 order by a;
+ a | b
+---+---
+ 1 | 2
+ 1 | 2
+ 1 | 2
+ 1 | 2
+ 1 | 2
+ 1 | 2
+ 1 | 2
+ 3 | 4
+ 3 | 4
+ 3 | 4
+ 3 | 4
+ 3 | 4
+ 3 | 4
+ 3 | 4
+ 5 | 6
+ 5 | 6
+ 5 | 6
+ 5 | 6
+ 7 | 8
+ 7 | 8
+(20 rows)
+
+---------------------------------
+-- Ensure that rows inserted into the table after declaring the cursor do not show up in fetch
+---------------------------------
+CREATE TABLE tt_22 (a int, b int) distribute by replication;
+INSERT INTO tt_22 VALUES (10);
+BEGIN;
+SET TRANSACTION ISOLATION LEVEL SERIALIZABLE;
+DECLARE c1 NO SCROLL CURSOR FOR SELECT * FROM tt_22 ORDER BY a FOR UPDATE;
+INSERT INTO tt_22 VALUES (2);
+FETCH ALL FROM c1; -- should not show the row (2)
+ a | b
+----+---
+ 10 |
+(1 row)
+
+END;
+BEGIN;
+SET TRANSACTION ISOLATION LEVEL SERIALIZABLE;
+DECLARE c1 NO SCROLL CURSOR FOR SELECT * FROM tt_22 ORDER BY a FOR UPDATE;
+INSERT INTO tt_22 VALUES (3);
+FETCH ALL FROM c1; -- should not show the row (3)
+ a | b
+----+---
+ 2 |
+ 10 |
+(2 rows)
+
+DECLARE c2 NO SCROLL CURSOR FOR SELECT * FROM tt_22 ORDER BY a FOR UPDATE;
+INSERT INTO tt_22 VALUES (4);
+FETCH ALL FROM c2; -- should not show the row (4)
+ a | b
+----+---
+ 2 |
+ 3 |
+ 10 |
+(3 rows)
+
+DECLARE c3 NO SCROLL CURSOR FOR SELECT * FROM tt_22 ORDER BY a FOR UPDATE;
+INSERT INTO tt_22 VALUES (5);
+FETCH ALL FROM c3; -- should not show the row (5)
+ a | b
+----+---
+ 2 |
+ 3 |
+ 4 |
+ 10 |
+(4 rows)
+
+END;
+DROP TABLE tt_22;
+-----------------------------------
+drop table my_tbl;
+drop table cc_33;
+drop table tt_33;
+drop table cc_11;
+drop table tt_11;
diff --git a/src/test/regress/sql/plpgsql.sql b/src/test/regress/sql/plpgsql.sql
index a892e63638..38ff423cab 100644
--- a/src/test/regress/sql/plpgsql.sql
+++ b/src/test/regress/sql/plpgsql.sql
@@ -2529,7 +2529,7 @@ declare
c refcursor;
x integer;
begin
- open c scroll for execute 'select f1 from int4_tbl';
+ open c scroll for execute 'select f1 from int4_tbl order by 1';
fetch last from c into x;
while found loop
return next x;
@@ -2539,14 +2539,14 @@ begin
end;
$$ language plpgsql;
-select * from sc_test() order by 1;
+select * from sc_test();
create or replace function sc_test() returns setof integer as $$
declare
c refcursor;
x integer;
begin
- open c scroll for execute 'select f1 from int4_tbl';
+ open c scroll for execute 'select f1 from int4_tbl order by 1';
fetch last from c into x;
while found loop
return next x;
@@ -2557,7 +2557,7 @@ begin
end;
$$ language plpgsql;
-select * from sc_test() order by 1;
+select * from sc_test();
create or replace function sc_test() returns setof integer as $$
declare
diff --git a/src/test/regress/sql/xc_misc.sql b/src/test/regress/sql/xc_misc.sql
index 94abc30b84..3f92e8001e 100644
--- a/src/test/regress/sql/xc_misc.sql
+++ b/src/test/regress/sql/xc_misc.sql
@@ -82,7 +82,7 @@ drop table tab4;
drop table tab5;
drop table my_tab1;
--- Test to make sure that the block of
+-- Test to make sure that the
-- INSERT SELECT in case of inserts into a child by selecting from
-- a parent works fine
@@ -93,11 +93,11 @@ insert into t_22 select * from t_11; -- should pass
CREATE TABLE c_11 () INHERITS (t_11);
insert into c_11 select * from t_22; -- should pass
-insert into c_11 select * from t_11; -- should fail
-insert into c_11 (select * from t_11 union all select * from t_22); -- should fail
-insert into c_11 (select * from t_11,t_22); -- should fail
+insert into c_11 select * from t_11; -- should insert 2
+insert into c_11 (select * from t_11 union all select * from t_22);
+insert into c_11 (select t_11.a, t_22.b from t_11,t_22);
insert into c_11 (select * from t_22 where a in (select a from t_11)); -- should pass
-insert into c_11 (select * from t_11 where a in (select a from t_22)); -- should fail
+insert into c_11 (select * from t_11 where a in (select a from t_22));
insert into t_11 select * from c_11; -- should pass
-- test to make sure count from a parent table works fine
@@ -111,7 +111,7 @@ CREATE TABLE child_11 () INHERITS (my_parent);
CREATE TABLE grand_child () INHERITS (child_11);
INSERT INTO child_11 SELECT * FROM grand_parent; -- should pass
-INSERT INTO child_11 SELECT * FROM my_parent; -- should fail
+INSERT INTO child_11 SELECT * FROM my_parent;
INSERT INTO grand_child SELECT * FROM my_parent; -- should pass
INSERT INTO grand_child SELECT * FROM grand_parent; -- should pass
@@ -122,3 +122,101 @@ drop table grand_parent;
drop table c_11;
drop table t_22;
drop table t_11;
+
+---------------------------------
+-- Ensure that command ids are sent to data nodes and are reported back to coordinator
+---------------------------------
+create table my_tbl( f1 int);
+
+begin;
+ insert into my_tbl values(100),(101),(102),(103),(104),(105);
+end;
+
+select cmin, cmax, * from my_tbl order by f1; -- command id should be in sequence and increasing
+
+---------------------------------
+-- Ensure that command id is consumed by declare cursor
+---------------------------------
+begin;
+ DECLARE c1 CURSOR FOR SELECT * FROM my_tbl;
+ INSERT INTO my_tbl VALUES (200);
+ select cmin, cmax,* from my_tbl where f1 = 200; -- should give 1 as command id of row containing 200
+end;
+
+---------------------------------
+-- insert into child by seleting from parent
+---------------------------------
+create table tt_11 ( a int, b int);
+insert into tt_11 values(1,2),(3,4);
+
+CREATE TABLE cc_11 () INHERITS (tt_11);
+insert into cc_11 select * from tt_11;
+
+select * from cc_11 order by a; -- should insert 2 rows
+
+begin;
+ insert into cc_11 values(5,6);
+ insert into cc_11 select * from tt_11; -- should insert the row (5,6)
+end;
+
+select * from cc_11 order by a;
+
+---------------------------------
+
+create table tt_33 ( a int, b int);
+insert into tt_33 values(1,2),(3,4);
+
+CREATE TABLE cc_33 () INHERITS (tt_33);
+insert into cc_33 select * from tt_33;
+
+begin;
+ insert into cc_33 values(5,6);
+ insert into cc_33 select * from tt_33; -- should insert row (5,6)
+ insert into cc_33 values(7,8);
+ select * from cc_33 order by a;
+ insert into cc_33 select * from tt_33; -- should insert row (7,8)
+end;
+
+select * from cc_33 order by a;
+
+---------------------------------
+-- Ensure that rows inserted into the table after declaring the cursor do not show up in fetch
+---------------------------------
+CREATE TABLE tt_22 (a int, b int) distribute by replication;
+
+INSERT INTO tt_22 VALUES (10);
+
+BEGIN;
+SET TRANSACTION ISOLATION LEVEL SERIALIZABLE;
+DECLARE c1 NO SCROLL CURSOR FOR SELECT * FROM tt_22 ORDER BY a FOR UPDATE;
+INSERT INTO tt_22 VALUES (2);
+FETCH ALL FROM c1; -- should not show the row (2)
+END;
+
+BEGIN;
+SET TRANSACTION ISOLATION LEVEL SERIALIZABLE;
+DECLARE c1 NO SCROLL CURSOR FOR SELECT * FROM tt_22 ORDER BY a FOR UPDATE;
+INSERT INTO tt_22 VALUES (3);
+FETCH ALL FROM c1; -- should not show the row (3)
+
+DECLARE c2 NO SCROLL CURSOR FOR SELECT * FROM tt_22 ORDER BY a FOR UPDATE;
+INSERT INTO tt_22 VALUES (4);
+FETCH ALL FROM c2; -- should not show the row (4)
+
+DECLARE c3 NO SCROLL CURSOR FOR SELECT * FROM tt_22 ORDER BY a FOR UPDATE;
+INSERT INTO tt_22 VALUES (5);
+FETCH ALL FROM c3; -- should not show the row (5)
+
+END;
+
+DROP TABLE tt_22;
+
+-----------------------------------
+
+drop table my_tbl;
+
+drop table cc_33;
+drop table tt_33;
+
+drop table cc_11;
+drop table tt_11;