summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-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;