diff options
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; |