table.
Display additional information regarding the plan. Specifically, include
the output column list for each node in the plan tree, schema-qualify
table and function names, always label variables in expressions with
- their range table alias, and always print the name of each trigger for
- which statistics are displayed. This parameter defaults to
+ their range table alias, always print the name of each trigger for
+ which statistics are displayed, and print FDW-specific information for
+ each ForeignScan node in the plan tree. This parameter defaults to
<literal>FALSE</literal>.
</para>
</listitem>
<term><replaceable class="PARAMETER">name</replaceable></term>
<listitem>
<para>
- The name (optionally schema-qualified) of an existing table to
+ The name (optionally schema-qualified) of an existing table
+ or a existing foreign table to
lock. If <literal>ONLY</> is specified, only that table is
locked. If <literal>ONLY</> is not specified, the table and all
its descendant tables (if any) are locked.
or <literal>FOR SHARE</literal> requires
<literal>UPDATE</literal> privilege as well (for at least one column
of each table so selected).
+ So you cannot lock contents of a foreign table because only SELECT
+ privilege can be granted on foreign tables.
</para>
</refsect1>
case T_WorkTableScan:
pname = sname = "WorkTable Scan";
break;
+ case T_ForeignScan:
+ pname = sname = "Foreign Scan";
+ break;
case T_Material:
pname = sname = "Materialize";
break;
case T_ValuesScan:
case T_CteScan:
case T_WorkTableScan:
+ case T_ForeignScan:
ExplainScanTarget((Scan *) plan, es);
break;
case T_BitmapIndexScan:
case T_ValuesScan:
case T_CteScan:
case T_WorkTableScan:
+ case T_ForeignScan:
case T_SubqueryScan:
show_scan_qual(plan->qual, "Filter", planstate, ancestors, es);
break;
break;
}
+ /* Show FDW specific information, if any */
+ if (es->verbose && IsA(plan, ForeignScan))
+ {
+ ForeignScan *scan = (ForeignScan *) plan;
+ if (scan->fplan->explainInfo)
+ ExplainPropertyText("FDW-Info", scan->fplan->explainInfo, es);
+ }
+
/* Show buffer usage */
if (es->buffers)
{
case T_IndexScan:
case T_BitmapHeapScan:
case T_TidScan:
+ case T_ForeignScan:
/* Assert it's on a real relation */
Assert(rte->rtekind == RTE_RELATION);
objectname = get_rel_name(rte->relid);
nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
- nodeWindowAgg.o tstoreReceiver.o spi.o
+ nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
include $(top_srcdir)/src/backend/common.mk
#include "executor/nodeBitmapOr.h"
#include "executor/nodeCtescan.h"
#include "executor/nodeFunctionscan.h"
+#include "executor/nodeForeignscan.h"
#include "executor/nodeGroup.h"
#include "executor/nodeGroup.h"
#include "executor/nodeHash.h"
ExecReScanWorkTableScan((WorkTableScanState *) node);
break;
+ case T_ForeignScanState:
+ ExecForeignReScan((ForeignScanState *) node);
+ break;
+
case T_NestLoopState:
ExecReScanNestLoop((NestLoopState *) node);
break;
break;
}
break;
+ case RELKIND_FOREIGN_TABLE:
+ ereport(ERROR,
+ (errcode(ERRCODE_WRONG_OBJECT_TYPE),
+ errmsg("cannot change foreign table \"%s\"",
+ RelationGetRelationName(resultRelationDesc))));
+ break;
default:
ereport(ERROR,
(errcode(ERRCODE_WRONG_OBJECT_TYPE),
#include "executor/nodeBitmapOr.h"
#include "executor/nodeCtescan.h"
#include "executor/nodeFunctionscan.h"
+#include "executor/nodeForeignscan.h"
#include "executor/nodeGroup.h"
#include "executor/nodeHash.h"
#include "executor/nodeHashjoin.h"
estate, eflags);
break;
+ case T_ForeignScan:
+ result = (PlanState *) ExecInitForeignScan((ForeignScan *) node,
+ estate, eflags);
+ break;
+
/*
* join nodes
*/
result = ExecWorkTableScan((WorkTableScanState *) node);
break;
+ case T_ForeignScanState:
+ result = ExecForeignScan((ForeignScanState *) node);
+ break;
+
/*
* join nodes
*/
ExecEndWorkTableScan((WorkTableScanState *) node);
break;
+ case T_ForeignScanState:
+ ExecEndForeignScan((ForeignScanState *) node);
+ break;
+
/*
* join nodes
*/
--- /dev/null
+/*-------------------------------------------------------------------------
+ *
+ * nodeForeignscan.c
+ * Support routines for sequential scans of foreign tables.
+ *
+ * Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeForeignscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecForeignScan sequentially scans a foreign table.
+ * ExecForeignNext retrieve next tuple in sequential order.
+ * ExecInitForeignScan creates and initializes a seqscan node.
+ * ExecEndForeignScan releases any storage allocated.
+ * ExecForeignReScan rescans the foreign table
+ * ExecForeignMarkPos marks scan position
+ * ExecForeignRestrPos restores scan position
+ */
+#include "postgres.h"
+
+#include "executor/executor.h"
+#include "executor/nodeForeignscan.h"
+#include "foreign/foreign.h"
+#include "miscadmin.h"
+
+static TupleTableSlot *ForeignNext(ForeignScanState *node);
+static bool ForeignRecheck(ForeignScanState *node, TupleTableSlot *slot);
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ForeignNext
+ *
+ * This is a workhorse for ExecForeignScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+ForeignNext(ForeignScanState *node)
+{
+ TupleTableSlot *slot = node->ss.ss_ScanTupleSlot;
+
+ Assert(node->ss.ps.state->es_direction == ForwardScanDirection);
+
+ /* tupleslot will be filled by Iterate. */
+ if (node->routine->Iterate == NULL)
+ ereport(ERROR,
+ (errmsg("foreign-data wrapper must support Iterate to scan foreign table")));
+ node->routine->Iterate(node->fstate, slot);
+
+ /* Set tableoid if the tuple was valid. */
+ if (HeapTupleIsValid(slot->tts_tuple))
+ {
+ /*
+ * If the foreign-data wrapper returned a MinimalTuple, materialize the
+ * tuple to store system attributes.
+ */
+ if (!TTS_HAS_PHYSICAL_TUPLE(slot))
+ ExecMaterializeSlot(slot);
+
+ /* overwrite only tableoid of the tuple */
+ slot->tts_tuple->t_tableOid =
+ RelationGetRelid(node->ss.ss_currentRelation);
+ }
+
+ return slot;
+}
+
+/*
+ * ForeignRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+ForeignRecheck(ForeignScanState *node, TupleTableSlot *slot)
+{
+ /* ForeignScan never use keys in ForeignNext. */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * ExecForeignScan(node)
+ *
+ * Scans the relation sequentially and returns the next qualifying
+ * tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecForeignScan(ForeignScanState *node)
+{
+ return ExecScan((ScanState *) node,
+ (ExecScanAccessMtd) ForeignNext,
+ (ExecScanRecheckMtd) ForeignRecheck);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecInitForeignScan
+ * ----------------------------------------------------------------
+ */
+ForeignScanState *
+ExecInitForeignScan(ForeignScan *node, EState *estate, int eflags)
+{
+ ForeignScanState *scanstate;
+ Relation currentRelation;
+ FdwRoutine *routine;
+
+ /*
+ * foreign scan has no child node.
+ * but not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ scanstate = makeNode(ForeignScanState);
+ scanstate->ss.ps.plan = (Plan *) node;
+ scanstate->ss.ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &scanstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ scanstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) scanstate);
+ scanstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) scanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &scanstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &scanstate->ss);
+
+ /*
+ * initialize scan relation. get the relation object id from the
+ * relid'th entry in the range table, open that relation and acquire
+ * appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate, node->scan.scanrelid);
+ scanstate->ss.ss_currentRelation = currentRelation;
+ ExecAssignScanType(&scanstate->ss, RelationGetDescr(currentRelation));
+ scanstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&scanstate->ss.ps);
+ ExecAssignScanProjectionInfo(&scanstate->ss);
+
+ /* cache the routine for the table in ForeignScanState */
+ routine = GetFdwRoutineByRelId(RelationGetRelid(currentRelation));
+ scanstate->routine = routine;
+
+ /*
+ * If this execution was not for EXPLAIN w/o ANALYZE flag, initiate the
+ * foreign scan.
+ */
+ if (!(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ {
+ ForeignScan *scan = (ForeignScan *) scanstate->ss.ps.plan;
+ if (routine->BeginScan != NULL)
+ scanstate->fstate = routine->BeginScan(scan->fplan,
+ estate->es_param_list_info);
+ }
+
+ return scanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndForeignScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndForeignScan(ForeignScanState *node)
+{
+ Relation relation;
+
+ /* close the scan */
+ if (node->routine->EndScan != NULL)
+ node->routine->EndScan(node->fstate);
+
+ /* get information from node */
+ relation = node->ss.ss_currentRelation;
+
+ /* Free the exprcontext */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /* clean out the tuple table */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /* close the relation. */
+ ExecCloseScanRelation(relation);
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecForeignReScan
+ *
+ * Rescans the relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecForeignReScan(ForeignScanState *node)
+{
+ if (node->routine->ReScan != NULL)
+ node->routine->ReScan(node->fstate);
+
+ ExecScanReScan((ScanState *) node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecForeignMarkPos(node)
+ *
+ * Marks scan position.
+ * ----------------------------------------------------------------
+ */
+void
+ExecForeignMarkPos(ForeignScanState *node)
+{
+ elog(ERROR, "ForeignScan does not support mark/restore");
+}
+
+/* ----------------------------------------------------------------
+ * ExecForeignRestrPos
+ *
+ * Restores scan position.
+ * ----------------------------------------------------------------
+ */
+void
+ExecForeignRestrPos(ForeignScanState *node)
+{
+ elog(ERROR, "ForeignScan does not support mark/restore");
+}
return ft;
}
+/*
+ * GetFdwRoutine - look up the handler of the foreign-data wrapper by OID and
+ * retrieve FdwRoutine.
+ */
+FdwRoutine *
+GetFdwRoutine(Oid fdwhandler)
+{
+ FmgrInfo flinfo;
+ FunctionCallInfoData fcinfo;
+ Datum result;
+ FdwRoutine *routine;
+
+ if (fdwhandler == InvalidOid)
+ elog(ERROR, "foreign-data wrapper has no handler");
+
+ fmgr_info(fdwhandler, &flinfo);
+ InitFunctionCallInfoData(fcinfo, &flinfo, 0, NULL, NULL);
+ result = FunctionCallInvoke(&fcinfo);
+
+ if (fcinfo.isnull ||
+ (routine = (FdwRoutine *) DatumGetPointer(result)) == NULL)
+ {
+ elog(ERROR, "function %u returned NULL", flinfo.fn_oid);
+ routine = NULL; /* keep compiler quiet */
+ }
+
+ return routine;
+}
+
+/*
+ * GetFdwRoutineByRelId - look up the handler of the foreign-data wrapper by
+ * OID of the foreign table and retrieve FdwRoutine.
+ */
+FdwRoutine *
+GetFdwRoutineByRelId(Oid relid)
+{
+ HeapTuple tp;
+ Form_pg_foreign_data_wrapper fdwform;
+ Form_pg_foreign_server serverform;
+ Form_pg_foreign_table tableform;
+ Oid serverid;
+ Oid fdwid;
+ Oid fdwhandler;
+
+ /* Get function OID for the foreign table. */
+ tp = SearchSysCache1(FOREIGNTABLEREL, ObjectIdGetDatum(relid));
+ if (!HeapTupleIsValid(tp))
+ elog(ERROR, "cache lookup failed for foreign table %u", relid);
+ tableform = (Form_pg_foreign_table) GETSTRUCT(tp);
+ serverid = tableform->ftserver;
+ ReleaseSysCache(tp);
+
+ tp = SearchSysCache1(FOREIGNSERVEROID, ObjectIdGetDatum(serverid));
+ if (!HeapTupleIsValid(tp))
+ elog(ERROR, "cache lookup failed for foreign server %u", serverid);
+ serverform = (Form_pg_foreign_server) GETSTRUCT(tp);
+ fdwid = serverform->srvfdw;
+ ReleaseSysCache(tp);
+
+ tp = SearchSysCache1(FOREIGNDATAWRAPPEROID, ObjectIdGetDatum(fdwid));
+ if (!HeapTupleIsValid(tp))
+ elog(ERROR, "cache lookup failed for foreign-data wrapper %u", fdwid);
+ fdwform = (Form_pg_foreign_data_wrapper) GETSTRUCT(tp);
+ fdwhandler = fdwform->fdwhandler;
+ ReleaseSysCache(tp);
+
+ return GetFdwRoutine(fdwhandler);
+}
+
/*
* Determine the relation is a foreign table.
*/
return newnode;
}
+/*
+ * _copyForeignScan
+ */
+static ForeignScan *
+_copyForeignScan(ForeignScan *from)
+{
+ ForeignScan *newnode = makeNode(ForeignScan);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((Scan *) from, (Scan *) newnode);
+ COPY_NODE_FIELD(fplan);
+
+ return newnode;
+}
+
+/*
+ * _copyFdwPlan
+ */
+static FdwPlan *
+_copyFdwPlan(FdwPlan *from)
+{
+ FdwPlan *newnode = makeNode(FdwPlan);
+
+ /*
+ * copy node superclass fields
+ */
+ COPY_STRING_FIELD(explainInfo);
+ COPY_SCALAR_FIELD(startup_cost);
+ COPY_SCALAR_FIELD(total_cost);
+ COPY_NODE_FIELD(private);
+
+ return newnode;
+}
+
/*
* CopyJoinFields
*
case T_WorkTableScan:
retval = _copyWorkTableScan(from);
break;
+ case T_ForeignScan:
+ retval = _copyForeignScan(from);
+ break;
+ case T_FdwPlan:
+ retval = _copyFdwPlan(from);
+ break;
case T_Join:
retval = _copyJoin(from);
break;
WRITE_INT_FIELD(wtParam);
}
+static void
+_outForeignScan(StringInfo str, ForeignScan *node)
+{
+ WRITE_NODE_TYPE("FOREIGNSCAN");
+
+ _outScanInfo(str, (Scan *) node);
+}
+
+static void
+_outFdwPlan(StringInfo str, FdwPlan *node)
+{
+ WRITE_NODE_TYPE("FDWPLAN");
+
+ WRITE_STRING_FIELD(explainInfo);
+ WRITE_FLOAT_FIELD(startup_cost, "%.2f");
+ WRITE_FLOAT_FIELD(total_cost, "%.2f");
+ WRITE_NODE_FIELD(private);
+}
+
static void
_outJoin(StringInfo str, Join *node)
{
WRITE_NODE_FIELD(tidquals);
}
+static void
+_outForeignPath(StringInfo str, ForeignPath *node)
+{
+ WRITE_NODE_TYPE("FOREIGNPATH");
+
+ _outPathInfo(str, (Path *) node);
+}
+
static void
_outAppendPath(StringInfo str, AppendPath *node)
{
case T_WorkTableScan:
_outWorkTableScan(str, obj);
break;
+ case T_ForeignScan:
+ _outForeignScan(str, obj);
+ break;
+ case T_FdwPlan:
+ _outFdwPlan(str, obj);
+ break;
case T_Join:
_outJoin(str, obj);
break;
case T_TidPath:
_outTidPath(str, obj);
break;
+ case T_ForeignPath:
+ _outForeignPath(str, obj);
+ break;
case T_AppendPath:
_outAppendPath(str, obj);
break;
#include <math.h>
+#include "foreign/foreign.h"
#include "nodes/nodeFuncs.h"
#ifdef OPTIMIZER_DEBUG
#include "nodes/print.h"
* least one dimension of cost or sortedness.
*/
- /* Consider sequential scan */
- add_path(rel, create_seqscan_path(root, rel));
+ if (IsForeignTable(rte->relid))
+ {
+ /* only foreign scan path is applyable to foreign table */
+ add_path(rel, create_foreignscan_path(root, rel));
+ }
+ else
+ {
+ /* Consider sequential scan */
+ add_path(rel, create_seqscan_path(root, rel));
- /* Consider index scans */
- create_index_paths(root, rel);
+ /* Consider index scans */
+ create_index_paths(root, rel);
- /* Consider TID scans */
- create_tidscan_paths(root, rel);
+ /* Consider TID scans */
+ create_tidscan_paths(root, rel);
+ }
/* Now find the cheapest of the paths for this rel */
set_cheapest(rel);
case T_TidPath:
ptype = "TidScan";
break;
+ case T_ForeignPath:
+ ptype = "ForeignScan";
+ break;
case T_AppendPath:
ptype = "Append";
break;
#include <math.h>
#include "access/skey.h"
+#include "catalog/pg_class.h"
#include "miscadmin.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
List *tlist, List *scan_clauses);
static WorkTableScan *create_worktablescan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static ForeignScan *create_foreignscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses);
static NestLoop *create_nestloop_plan(PlannerInfo *root, NestPath *best_path,
Plan *outer_plan, Plan *inner_plan);
static MergeJoin *create_mergejoin_plan(PlannerInfo *root, MergePath *best_path,
Index scanrelid, int ctePlanId, int cteParam);
static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
Index scanrelid, int wtParam);
+static ForeignScan *make_foreignscan(List *qptlist, RangeTblEntry *rte,
+ List *qpqual, Index scanrelid, FdwPlan *fplan);
static BitmapAnd *make_bitmap_and(List *bitmapplans);
static BitmapOr *make_bitmap_or(List *bitmapplans);
static NestLoop *make_nestloop(List *tlist,
case T_ValuesScan:
case T_CteScan:
case T_WorkTableScan:
+ case T_ForeignScan:
plan = create_scan_plan(root, best_path);
break;
case T_HashJoin:
scan_clauses);
break;
+ case T_ForeignScan:
+ plan = (Plan *) create_foreignscan_plan(root,
+ best_path,
+ tlist,
+ scan_clauses);
+ break;
+
default:
elog(ERROR, "unrecognized node type: %d",
(int) best_path->pathtype);
case T_ValuesScan:
case T_CteScan:
case T_WorkTableScan:
+ case T_ForeignScan:
plan->targetlist = build_relation_tlist(path->parent);
break;
default:
return scan_plan;
}
+/*
+ * create_foreignscan_plan
+ * Returns a foreignscan plan for the base relation scanned by 'best_path'
+ * with restriction clauses 'scan_clauses' and targetlist 'tlist'.
+ */
+static ForeignScan *
+create_foreignscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses)
+{
+ ForeignPath *fpath = (ForeignPath *) best_path;
+ ForeignScan *scan_plan;
+ Index scan_relid = best_path->parent->relid;
+ RangeTblEntry *rte;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->parent->rtekind == RTE_RELATION);
+ rte = planner_rt_fetch(scan_relid, root);
+ Assert(rte->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ scan_plan = make_foreignscan(tlist,
+ rte,
+ scan_clauses,
+ scan_relid,
+ fpath->fplan);
+
+ copy_path_costsize(&scan_plan->scan.plan, best_path);
+
+ return scan_plan;
+}
+
/*****************************************************************************
*
return node;
}
+static ForeignScan *
+make_foreignscan(List *qptlist,
+ RangeTblEntry *rte,
+ List *qpqual,
+ Index scanrelid,
+ FdwPlan *fplan)
+{
+ ForeignScan *node = makeNode(ForeignScan);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->fplan = fplan;
+
+ return node;
+}
+
Append *
make_append(List *appendplans, List *tlist)
{
fix_scan_list(glob, splan->scan.plan.qual, rtoffset);
}
break;
+ case T_ForeignScan:
+ {
+ ForeignScan *splan = (ForeignScan *) plan;
+
+ splan->scan.scanrelid += rtoffset;
+ splan->scan.plan.targetlist =
+ fix_scan_list(glob, splan->scan.plan.targetlist, rtoffset);
+ splan->scan.plan.qual =
+ fix_scan_list(glob, splan->scan.plan.qual, rtoffset);
+ }
+ break;
case T_NestLoop:
case T_MergeJoin:
case T_HashJoin:
context.paramids = bms_add_members(context.paramids, scan_params);
break;
+ case T_ForeignScan:
+ context.paramids = bms_add_members(context.paramids, scan_params);
+ break;
+
case T_ModifyTable:
{
ModifyTable *mtplan = (ModifyTable *) plan;
continue;
}
+ /*
+ * SELECT FOR UPDATE/SHARE is not allowd to foreign tables because
+ * they are read-only.
+ */
+ if (newrelation->rd_rel->relkind == RELKIND_FOREIGN_TABLE &&
+ lockmode != AccessShareLock)
+ ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("SELECT FOR UPDATE/SHARE is not allowed with foreign tables")));
+
/*
* Build an RTE for the child, and attach to query's rangetable list.
* We copy most fields of the parent's RTE, but replace relation OID,
#include <math.h>
#include "catalog/pg_operator.h"
+#include "foreign/foreign.h"
#include "miscadmin.h"
#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
return pathnode;
}
+/*
+ * create_foreignscan_path
+ * Creates a path corresponding to a scan of a foreign table,
+ * returning the pathnode.
+ */
+Path *
+create_foreignscan_path(PlannerInfo *root, RelOptInfo *rel)
+{
+ RangeTblEntry *rte;
+ FdwRoutine *routine;
+ ForeignPath *pathnode = makeNode(ForeignPath);
+
+ pathnode->path.pathtype = T_ForeignScan;
+ pathnode->path.parent = rel;
+ pathnode->path.pathkeys = NIL; /* result is always unordered */
+
+ rte = planner_rt_fetch(rel->relid, root);
+ routine = GetFdwRoutineByRelId(rte->relid);
+ pathnode->fplan = routine->PlanRelScan(rte->relid, root, rel);
+
+ /* use costs estimated by FDW */
+ pathnode->path.startup_cost = pathnode->fplan->startup_cost;
+ pathnode->path.total_cost = pathnode->fplan->total_cost;
+
+ return (Path *) pathnode;
+}
+
/*
* create_nestloop_path
* Creates a pathnode corresponding to a nestloop join between two
*pages = 1;
*tuples = 1;
break;
+ case RELKIND_FOREIGN_TABLE:
+ /* foreign tables has no storage, trust statistics */
+ *pages = rel->rd_rel->relpages;
+ *tuples = rel->rd_rel->reltuples;
+ break;
default:
/* else it has no disk storage; probably shouldn't get here? */
*pages = 0;
*
* We also support building a "physical" tlist for subqueries, functions,
* values lists, and CTEs, since the same optimization can occur in
- * SubqueryScan, FunctionScan, ValuesScan, CteScan, and WorkTableScan nodes.
+ * SubqueryScan, FunctionScan, ValuesScan, CteScan, WorkTableScan and
+ * ForeignScan nodes.
*/
List *
build_physical_tlist(PlannerInfo *root, RelOptInfo *rel)
--- /dev/null
+/*-------------------------------------------------------------------------
+ *
+ * nodeForeignscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeForeignscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEFOREIGNSCAN_H
+#define NODEFOREIGNSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern ForeignScanState *ExecInitForeignScan(ForeignScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecForeignScan(ForeignScanState *node);
+extern void ExecEndForeignScan(ForeignScanState *node);
+extern void ExecForeignMarkPos(ForeignScanState *node);
+extern void ExecForeignRestrPos(ForeignScanState *node);
+extern void ExecForeignReScan(ForeignScanState *node);
+
+#endif /* NODEFOREIGNSCAN_H */
--- /dev/null
+/*-------------------------------------------------------------------------
+ *
+ * fdwapi.h
+ * API for foreign-data wrappers
+ *
+ * Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group
+ *
+ * src/include/foreign/fdwapi.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef FDWAPI_H
+#define FDWAPI_H
+
+#include "executor/tuptable.h"
+#include "nodes/pg_list.h"
+#include "nodes/relation.h"
+
+/*
+ * When a plan is going to be cached, the plan node is copied into another
+ * context with copyObject. It means that FdwPlan, a part of ForeignScan plan
+ * node, and its contents must have copyObject support too.
+ */
+struct FdwPlan
+{
+ NodeTag type;
+
+ /*
+ * Free-form text shown in EXPLAIN. The SQL to be sent to the remote
+ * server is typically shown here.
+ */
+ char *explainInfo;
+
+ /*
+ * Cost estimation info. The startup_cost should include the cost of
+ * connecting to the remote host and sending over the query, as well as
+ * the cost of starting up the query so that it returns the first result
+ * row.
+ */
+ double startup_cost;
+ double total_cost;
+#ifdef HOOK_ESTIMATE_REL_SIZE
+ double rows;
+ int width;
+#endif
+
+ /*
+ * FDW-private data. FDW must guarantee that every elements in this list
+ * have copyObject support. If FDW needs to store arbitrary data such as
+ * non-Node structure, Const of bytea would be able to use as a container.
+ */
+ List *private;
+};
+typedef struct FdwPlan FdwPlan;
+
+struct FdwExecutionState
+{
+ /* FDW-private data */
+ void *private;
+};
+typedef struct FdwExecutionState FdwExecutionState;
+
+/*
+ * Common interface routines of FDW, inspired by the FDW API in the SQL/MED
+ * standard, but adapted to the PostgreSQL world.
+ *
+ * A foreign-data wrapper implements these routines. At a minimum, it must
+ * implement BeginScan, Iterate and EndScan, and either PlanNative or
+ * PlanRelScan.
+ *
+ * The PlanXXX functions return an FdwPlan struct that can later be executed
+ * with BeginScan. The implementation should fill in the cost estimates in
+ * FdwPlan, as well as a tuple descriptor that describes the result set.
+ */
+struct FdwRoutine
+{
+#ifdef IN_THE_FUTURE
+ /*
+ * Plan a query of arbitrary native SQL (or other query language supported
+ * by the foreign server). This is used for SQL/MED passthrough mode, or
+ * e.g contrib/dblink.
+ */
+ FdwPlan *(*PlanNative)(Oid serverid, char *query);
+
+ /*
+ * Plan a whole subquery. This is used for example to execute an aggregate
+ * query remotely without pulling all the rows to the local server.
+ *
+ * The implementation can return NULL if it cannot satisfy the whole
+ * subquery, in which case the planner will break down the query into
+ * smaller parts and call PlanRelScan for the foreign tables involved.
+ *
+ * The implementation must be careful to only accept queries it fully
+ * understands! For example, if it ignores windowClauses, and returns
+ * a non-NULL results for a query that contains one, the windowClause
+ * would be lost and the query would return incorrect results.
+ */
+ FdwPlan *(*PlanQuery)(PlannerInfo *root, Query query);
+#endif
+
+ /*
+ * Plan a scan on a foreign table. 'foreigntableid' identifies the foreign
+ * table, and 'attnos' is an integer list of attribute numbers for the
+ * columns to be returned. Note that 'attnos' can also be an empty list,
+ * typically for "SELECT COUNT(*) FROM foreigntable" style queries where
+ * we just need to know how many rows there are. The number and type of
+ * attributes in the tuple descriptor in the returned FdwPlan must match
+ * the attributes specified in attnos, or an error will be thrown.
+ *
+ * 'root' and 'baserel' contain context information that the
+ * implementation can use to restrict the rows that are fetched.
+ * baserel->baserestrictinfo is particularly interseting, as it contains
+ * quals (WHERE clauses) that can be used to filter the rows in the remote
+ * server. 'root' and 'baserel' can be safely ignored, the planner will
+ * re-check the quals on every fetched row anyway.
+ */
+ FdwPlan *(*PlanRelScan)(Oid foreigntableid, PlannerInfo *root,
+ RelOptInfo *baserel);
+
+ /*
+ * Begin execution of a foreign scan. This function is called when an
+ * actual scan is needed, so EXPLAIN without ANALYZE option doesn't call
+ * BeginScan().
+ */
+ FdwExecutionState *(*BeginScan)(FdwPlan *plan, ParamListInfo params);
+
+ /*
+ * Fetch the next record and store it into slot.
+ */
+ void (*Iterate)(FdwExecutionState *state, TupleTableSlot *slot);
+
+ /*
+ * Reset the read pointer to the head of the scan.
+ * This function will be called when the new outer tuple was acquired in a
+ * nested loop.
+ */
+ void (*ReScan)(FdwExecutionState *state);
+
+ /*
+ * End the foreign scan and do clean up.
+ */
+ void (*EndScan)(FdwExecutionState *state);
+};
+typedef struct FdwRoutine FdwRoutine;
+
+#endif /* FDWAPI_H */
+
#define FOREIGN_H
#include "executor/tuptable.h"
+#include "foreign/fdwapi.h"
#include "utils/relcache.h"
bool missing_ok);
extern Oid GetForeignDataWrapperOidByName(const char *name, bool missing_ok);
extern ForeignTable *GetForeignTable(Oid relid);
+extern FdwRoutine *GetFdwRoutine(Oid fdwhandler);
+extern FdwRoutine *GetFdwRoutineByRelId(Oid relid);
extern bool IsForeignTable(Oid relid);
extern Oid GetFdwValidator(Oid relid);
extern List *GetGenericOptionsPerColumn(Oid relid, int2 attnum);
#include "access/genam.h"
#include "access/heapam.h"
#include "access/skey.h"
+#include "foreign/fdwapi.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "nodes/relation.h"
#include "nodes/tidbitmap.h"
#include "utils/hsearch.h"
#include "utils/rel.h"
RecursiveUnionState *rustate;
} WorkTableScanState;
+/* ----------------
+ * ForeignScanState information
+ *
+ * ForeignScan nodes are used to scan the foreign table managed by
+ * a foreign server.
+ * ----------------
+ */
+typedef struct ForeignScanState
+{
+ ScanState ss; /* its first field is NodeTag */
+ FdwRoutine *routine;
+ FdwExecutionState *fstate; /* private data for each data wrapper */
+} ForeignScanState;
+
/* ----------------------------------------------------------------
* Join State Information
* ----------------------------------------------------------------
T_ValuesScan,
T_CteScan,
T_WorkTableScan,
+ T_ForeignScan,
+ T_FdwPlan,
T_Join,
T_NestLoop,
T_MergeJoin,
T_ValuesScanState,
T_CteScanState,
T_WorkTableScanState,
+ T_ForeignScanState,
T_JoinState,
T_NestLoopState,
T_MergeJoinState,
T_MergePath,
T_HashPath,
T_TidPath,
+ T_ForeignPath,
T_AppendPath,
T_MergeAppendPath,
T_ResultPath,
#define PLANNODES_H
#include "access/sdir.h"
+#include "foreign/fdwapi.h"
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
#include "storage/itemptr.h"
int wtParam; /* ID of Param representing work table */
} WorkTableScan;
+/* ----------------
+ * ForeignScan node
+ * ----------------
+ */
+typedef struct ForeignScan
+{
+ Scan scan;
+ FdwPlan *fplan;
+} ForeignScan;
+
/*
* ==========
List *tidquals; /* qual(s) involving CTID = something */
} TidPath;
+/*
+ * ForeignPath represents a scan on a foreign table
+ */
+typedef struct ForeignPath
+{
+ Path path;
+ struct FdwPlan *fplan;
+} ForeignPath;
+
/*
* AppendPath represents an Append plan, ie, successive execution of
* several member plans.
extern Path *create_valuesscan_path(PlannerInfo *root, RelOptInfo *rel);
extern Path *create_ctescan_path(PlannerInfo *root, RelOptInfo *rel);
extern Path *create_worktablescan_path(PlannerInfo *root, RelOptInfo *rel);
+extern Path *create_foreignscan_path(PlannerInfo *root, RelOptInfo *rel);
extern NestPath *create_nestloop_path(PlannerInfo *root,
RelOptInfo *joinrel,
CREATE INDEX id_ft1_c2 ON ft1 (c2); -- ERROR
ERROR: "ft1" is not a table
+SELECT * FROM ft1; -- ERROR
+ERROR: foreign-data wrapper has no handler
+EXPLAIN (VERBOSE) SELECT * FROM ft1; -- ERROR
+ERROR: foreign-data wrapper has no handler
-- ALTER FOREIGN TABLE
COMMENT ON FOREIGN TABLE ft1 IS 'foreign table';
COMMENT ON FOREIGN TABLE ft1 IS NULL;
\d+ ft2
\det+
CREATE INDEX id_ft1_c2 ON ft1 (c2); -- ERROR
+SELECT * FROM ft1; -- ERROR
+EXPLAIN (VERBOSE) SELECT * FROM ft1; -- ERROR
-- ALTER FOREIGN TABLE
COMMENT ON FOREIGN TABLE ft1 IS 'foreign table';