pg12: version_compat.h, tuples, oids, misc

pull/2844/head
Philip Dubé 2019-08-08 21:55:52 +00:00
parent 9643ff580e
commit 018ad1c58e
22 changed files with 190 additions and 45 deletions

View File

@ -645,6 +645,8 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
relationDesc = RelationGetDescr(relation);
relationName = RelationGetRelationName(relation);
#if PG_VERSION_NUM < 120000
/* verify target relation does not use WITH (OIDS) PostgreSQL feature */
if (relationDesc->tdhasoid)
{
@ -653,6 +655,7 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
errdetail("Distributed relations must not specify the WITH "
"(OIDS) option in their definitions.")));
}
#endif
/* verify target relation does not use identity columns */
if (RelationUsesIdentityColumns(relationDesc))
@ -1196,7 +1199,11 @@ CopyLocalDataIntoShards(Oid distributedRelationId)
bool stopOnFailure = true;
EState *estate = NULL;
#if PG_VERSION_NUM >= 120000
TableScanDesc scan = NULL;
#else
HeapScanDesc scan = NULL;
#endif
HeapTuple tuple = NULL;
ExprContext *econtext = NULL;
MemoryContext oldContext = NULL;
@ -1230,7 +1237,7 @@ CopyLocalDataIntoShards(Oid distributedRelationId)
/* get the table columns */
tupleDescriptor = RelationGetDescr(distributedRelation);
slot = MakeSingleTupleTableSlot(tupleDescriptor);
slot = MakeSingleTupleTableSlotCompat(tupleDescriptor, &TTSOpsHeapTuple);
columnNameList = TupleDescColumnNameList(tupleDescriptor);
/* determine the partition column in the tuple descriptor */
@ -1256,14 +1263,22 @@ CopyLocalDataIntoShards(Oid distributedRelationId)
copyDest->rStartup(copyDest, 0, tupleDescriptor);
/* begin reading from local table */
#if PG_VERSION_NUM >= 120000
scan = table_beginscan(distributedRelation, GetActiveSnapshot(), 0, NULL);
#else
scan = heap_beginscan(distributedRelation, GetActiveSnapshot(), 0, NULL);
#endif
oldContext = MemoryContextSwitchTo(GetPerTupleMemoryContext(estate));
while ((tuple = heap_getnext(scan, ForwardScanDirection)) != NULL)
{
/* materialize tuple and send it to a shard */
#if PG_VERSION_NUM >= 120000
ExecStoreHeapTuple(tuple, slot, false);
#else
ExecStoreTuple(tuple, slot, InvalidBuffer, false);
#endif
copyDest->receiveSlot(slot, copyDest);
/* clear tuple memory */
@ -1293,7 +1308,11 @@ CopyLocalDataIntoShards(Oid distributedRelationId)
MemoryContextSwitchTo(oldContext);
/* finish reading from the local table */
#if PG_VERSION_NUM >= 120000
table_endscan(scan);
#else
heap_endscan(scan);
#endif
/* finish writing into the shards */
copyDest->rShutdown(copyDest);

View File

@ -463,7 +463,7 @@ CopyToExistingShards(CopyStmt *copyStatement, char *completionTag)
columnNulls = palloc0(columnCount * sizeof(bool));
/* set up a virtual tuple table slot */
tupleTableSlot = MakeSingleTupleTableSlot(tupleDescriptor);
tupleTableSlot = MakeSingleTupleTableSlotCompat(tupleDescriptor, &TTSOpsVirtual);
tupleTableSlot->tts_nvalid = columnCount;
tupleTableSlot->tts_values = columnValues;
tupleTableSlot->tts_isnull = columnNulls;
@ -561,8 +561,8 @@ CopyToExistingShards(CopyStmt *copyStatement, char *completionTag)
oldContext = MemoryContextSwitchTo(executorTupleContext);
/* parse a row from the input */
nextRowFound = NextCopyFrom(copyState, executorExpressionContext,
columnValues, columnNulls, NULL);
nextRowFound = NextCopyFromCompat(copyState, executorExpressionContext,
columnValues, columnNulls);
if (!nextRowFound)
{
@ -681,8 +681,8 @@ CopyToNewShards(CopyStmt *copyStatement, char *completionTag, Oid relationId)
oldContext = MemoryContextSwitchTo(executorTupleContext);
/* parse a row from the input */
nextRowFound = NextCopyFrom(copyState, executorExpressionContext,
columnValues, columnNulls, NULL);
nextRowFound = NextCopyFromCompat(copyState, executorExpressionContext,
columnValues, columnNulls);
if (!nextRowFound)
{

View File

@ -162,6 +162,11 @@ CitusBeginScan(CustomScanState *node, EState *estate, int eflags)
MarkCitusInitiatedCoordinatorBackend();
scanState = (CitusScanState *) node;
#if PG_VERSION_NUM >= 120000
ExecInitResultSlot(&scanState->customScanState.ss.ps, &TTSOpsMinimalTuple);
#endif
distributedPlan = scanState->distributedPlan;
if (distributedPlan->modLevel == ROW_MODIFY_READONLY ||
distributedPlan->insertSelectSubquery != NULL)

View File

@ -138,7 +138,9 @@ CitusExecutorRun(QueryDesc *queryDesc,
EState *estate = queryDesc->estate;
estate->es_processed = 0;
#if PG_VERSION_NUM < 120000
estate->es_lastoid = InvalidOid;
#endif
/* start and shutdown tuple receiver to simulate empty result */
dest->rStartup(queryDesc->dest, CMD_SELECT, queryDesc->tupDesc);
@ -351,8 +353,8 @@ ReadFileIntoTupleStore(char *fileName, char *copyFormat, TupleDesc tupleDescript
ResetPerTupleExprContext(executorState);
oldContext = MemoryContextSwitchTo(executorTupleContext);
nextRowFound = NextCopyFrom(copyState, executorExpressionContext,
columnValues, columnNulls, NULL);
nextRowFound = NextCopyFromCompat(copyState, executorExpressionContext,
columnValues, columnNulls);
if (!nextRowFound)
{
MemoryContextSwitchTo(oldContext);

View File

@ -706,7 +706,8 @@ SortTupleStore(CitusScanState *scanState)
/* iterate over all the sorted tuples, add them to original tuplestore */
while (true)
{
TupleTableSlot *newSlot = MakeSingleTupleTableSlot(tupleDescriptor);
TupleTableSlot *newSlot = MakeSingleTupleTableSlotCompat(tupleDescriptor,
&TTSOpsMinimalTuple);
bool found = tuplesort_gettupleslot(tuplesortstate, true, false, newSlot, NULL);
if (!found)

View File

@ -60,7 +60,6 @@
#include "utils/palloc.h"
#include "utils/relcache.h"
#include "utils/ruleutils.h"
#include "utils/tqual.h"
#include "utils/varlena.h"
@ -472,7 +471,6 @@ master_get_active_worker_nodes(PG_FUNCTION_ARGS)
MemoryContext oldContext = NULL;
List *workerNodeList = NIL;
TupleDesc tupleDescriptor = NULL;
bool hasOid = false;
/* create a function context for cross-call persistence */
functionContext = SRF_FIRSTCALL_INIT();
@ -490,7 +488,11 @@ master_get_active_worker_nodes(PG_FUNCTION_ARGS)
* This tuple descriptor must match the output parameters declared for
* the function in pg_proc.
*/
tupleDescriptor = CreateTemplateTupleDesc(WORKER_NODE_FIELDS, hasOid);
#if PG_VERSION_NUM < 120000
tupleDescriptor = CreateTemplateTupleDesc(WORKER_NODE_FIELDS, false);
#else
tupleDescriptor = CreateTemplateTupleDesc(WORKER_NODE_FIELDS);
#endif
TupleDescInitEntry(tupleDescriptor, (AttrNumber) 1, "node_name",
TEXTOID, -1, 0);
TupleDescInitEntry(tupleDescriptor, (AttrNumber) 2, "node_port",

View File

@ -203,7 +203,7 @@ UpdateRelationToShardNames(Node *node, List *relationShardList)
if (IsA(node, Query))
{
return query_tree_walker((Query *) node, UpdateRelationToShardNames,
relationShardList, QTW_EXAMINE_RTES);
relationShardList, QTW_EXAMINE_RTES_BEFORE);
}
if (!IsA(node, RangeTblEntry))

View File

@ -26,6 +26,7 @@
#include "distributed/query_pushdown_planning.h"
#include "distributed/recursive_planning.h"
#include "distributed/resource_lock.h"
#include "distributed/version_compat.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
#include "nodes/parsenodes.h"
@ -769,7 +770,7 @@ MultiTaskRouterSelectQuerySupported(Query *query)
Assert(subquery->commandType == CMD_SELECT);
/* pushing down rtes without relations yields (shardCount * expectedRows) */
if (subquery->rtable == NIL)
if (HasEmptyJoinTree(subquery))
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"Subqueries without relations are not allowed in "

View File

@ -35,6 +35,7 @@
#include "distributed/multi_physical_planner.h"
#include "distributed/pg_dist_partition.h"
#include "distributed/worker_protocol.h"
#include "distributed/version_compat.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
#include "nodes/print.h"
@ -53,7 +54,6 @@
#include "utils/lsyscache.h"
#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/tqual.h"
/* Config variable managed via guc.c */
@ -2966,7 +2966,11 @@ AggregateFunctionOid(const char *functionName, Oid inputType)
/* check if input type and found value type match */
if (procForm->proargtypes.values[0] == inputType)
{
#if PG_VERSION_NUM < 120000
functionOid = HeapTupleGetOid(heapTuple);
#else
functionOid = procForm->oid;
#endif
break;
}
}
@ -2996,8 +3000,9 @@ TypeOid(Oid schemaId, const char *typeName)
{
Oid typeOid;
typeOid = GetSysCacheOid2(TYPENAMENSP, PointerGetDatum(typeName),
ObjectIdGetDatum(schemaId));
typeOid = GetSysCacheOid2Compat(TYPENAMENSP, Anum_pg_type_oid, PointerGetDatum(
typeName),
ObjectIdGetDatum(schemaId));
return typeOid;
}

View File

@ -155,7 +155,7 @@ MultiLogicalPlanCreate(Query *originalQuery, Query *queryTree,
* FindNodeCheck finds a node for which the check function returns true.
*
* To call this function directly with an RTE, use:
* range_table_walker(rte, FindNodeCheck, check, QTW_EXAMINE_RTES)
* range_table_walker(rte, FindNodeCheck, check, QTW_EXAMINE_RTES_BEFORE)
*/
bool
FindNodeCheck(Node *node, bool (*check)(Node *))
@ -177,7 +177,8 @@ FindNodeCheck(Node *node, bool (*check)(Node *))
}
else if (IsA(node, Query))
{
return query_tree_walker((Query *) node, FindNodeCheck, check, QTW_EXAMINE_RTES);
return query_tree_walker((Query *) node, FindNodeCheck, check,
QTW_EXAMINE_RTES_BEFORE);
}
return expression_tree_walker(node, FindNodeCheck, check);
@ -385,7 +386,7 @@ AllTargetExpressionsAreColumnReferences(List *targetEntryList)
bool
FindNodeCheckInRangeTableList(List *rtable, bool (*check)(Node *))
{
return range_table_walker(rtable, FindNodeCheck, check, QTW_EXAMINE_RTES);
return range_table_walker(rtable, FindNodeCheck, check, QTW_EXAMINE_RTES_BEFORE);
}
@ -1997,7 +1998,8 @@ ExtractRangeTableRelationWalker(Node *node, List **rangeTableRelationList)
{
walkIsComplete = query_tree_walker((Query *) node,
ExtractRangeTableRelationWalker,
rangeTableRelationList, QTW_EXAMINE_RTES);
rangeTableRelationList,
QTW_EXAMINE_RTES_BEFORE);
}
else
{
@ -2045,7 +2047,7 @@ ExtractRangeTableEntryWalker(Node *node, List **rangeTableList)
walkIsComplete = query_tree_walker((Query *) node,
ExtractRangeTableEntryWalker,
rangeTableList,
QTW_EXAMINE_RTES);
QTW_EXAMINE_RTES_BEFORE);
}
else
{
@ -2053,7 +2055,7 @@ ExtractRangeTableEntryWalker(Node *node, List **rangeTableList)
walkIsComplete = range_table_walker(query->rtable,
ExtractRangeTableEntryWalker,
rangeTableList,
QTW_EXAMINE_RTES);
QTW_EXAMINE_RTES_BEFORE);
}
}
else

View File

@ -679,7 +679,11 @@ ModifyQuerySupported(Query *queryTree, Query *originalQuery, bool multiShardQuer
NULL, NULL);
}
}
else if (rangeTableEntry->rtekind == RTE_VALUES)
else if (rangeTableEntry->rtekind == RTE_VALUES
#if PG_VERSION_NUM >= 120000
|| rangeTableEntry->rtekind == RTE_RESULT
#endif
)
{
/* do nothing, this type is supported */
}

View File

@ -31,6 +31,7 @@
#include "distributed/pg_dist_partition.h"
#include "distributed/query_pushdown_planning.h"
#include "distributed/relation_restriction_equivalence.h"
#include "distributed/version_compat.h"
#include "nodes/nodeFuncs.h"
#if PG_VERSION_NUM >= 120000
#include "nodes/makefuncs.h"
@ -199,6 +200,32 @@ JoinTreeContainsSubquery(Query *query)
}
/*
* HasEmptyJoinTree returns whether the query selects from anything.
*/
bool
HasEmptyJoinTree(Query *query)
{
if (query->rtable == NIL)
{
return true;
}
#if PG_VERSION_NUM >= 120000
else if (list_length(query->rtable) == 1)
{
RangeTblEntry *rte = (RangeTblEntry *) linitial(query->rtable);
if (rte->rtekind == RTE_RESULT)
{
return true;
}
}
#endif
return false;
}
/*
* JoinTreeContainsSubqueryWalker returns true if the input joinTreeNode
* references to a subquery. Otherwise, recurses into the expression.
@ -656,7 +683,7 @@ FromClauseRecurringTupleType(Query *queryTree)
{
RecurringTuplesType recurType = RECURRING_TUPLES_INVALID;
if (queryTree->rtable == NIL)
if (HasEmptyJoinTree(queryTree))
{
return RECURRING_TUPLES_EMPTY_JOIN_TREE;
}
@ -822,7 +849,7 @@ DeferErrorIfCannotPushdownSubquery(Query *subqueryTree, bool outerMostQueryHasLi
return deferredError;
}
if (subqueryTree->rtable == NIL &&
if (HasEmptyJoinTree(subqueryTree) &&
contain_mutable_functions((Node *) subqueryTree->targetList))
{
preconditionsSatisfied = false;
@ -1014,7 +1041,11 @@ DeferErrorIfUnsupportedTableCombination(Query *queryTree)
* subquery, or immutable function.
*/
if (rangeTableEntry->rtekind == RTE_RELATION ||
rangeTableEntry->rtekind == RTE_SUBQUERY)
rangeTableEntry->rtekind == RTE_SUBQUERY
#if PG_VERSION_NUM >= 120000
|| rangeTableEntry->rtekind == RTE_RESULT
#endif
)
{
/* accepted */
}
@ -1337,7 +1368,7 @@ static bool
IsRecurringRangeTable(List *rangeTable, RecurringTuplesType *recurType)
{
return range_table_walker(rangeTable, HasRecurringTuples, recurType,
QTW_EXAMINE_RTES);
QTW_EXAMINE_RTES_BEFORE);
}
@ -1393,6 +1424,13 @@ HasRecurringTuples(Node *node, RecurringTuplesType *recurType)
*/
return true;
}
#if PG_VERSION_NUM >= 120000
else if (rangeTableEntry->rtekind == RTE_RESULT)
{
*recurType = RECURRING_TUPLES_EMPTY_JOIN_TREE;
return true;
}
#endif
return false;
}
@ -1400,7 +1438,7 @@ HasRecurringTuples(Node *node, RecurringTuplesType *recurType)
{
Query *query = (Query *) node;
if (query->rtable == NIL)
if (HasEmptyJoinTree(query))
{
*recurType = RECURRING_TUPLES_EMPTY_JOIN_TREE;
@ -1412,7 +1450,7 @@ HasRecurringTuples(Node *node, RecurringTuplesType *recurType)
}
return query_tree_walker((Query *) node, HasRecurringTuples,
recurType, QTW_EXAMINE_RTES);
recurType, QTW_EXAMINE_RTES_BEFORE);
}
return expression_tree_walker(node, HasRecurringTuples, recurType);

View File

@ -1222,7 +1222,8 @@ CteReferenceListWalker(Node *node, CteReferenceWalkerContext *context)
Query *query = (Query *) node;
context->level += 1;
query_tree_walker(query, CteReferenceListWalker, context, QTW_EXAMINE_RTES);
query_tree_walker(query, CteReferenceListWalker, context,
QTW_EXAMINE_RTES_BEFORE);
context->level -= 1;
return false;

View File

@ -13,6 +13,7 @@
#include "distributed/function_utils.h"
#include "distributed/multi_progress.h"
#include "distributed/version_compat.h"
#include "storage/dsm.h"
#include "utils/builtins.h"
@ -155,7 +156,8 @@ ProgressMonitorList(uint64 commandTypeMagicNumber, List **attachedDSMSegments)
getProgressInfoFunctionOid,
commandTypeDatum);
tupleTableSlot = MakeSingleTupleTableSlot(progressResultSet->setDesc);
tupleTableSlot = MakeSingleTupleTableSlotCompat(progressResultSet->setDesc,
&TTSOpsMinimalTuple);
/* iterate over tuples in tuple store, and send them to destination */
for (;;)

View File

@ -756,7 +756,7 @@ AppendShardIdToName(char **name, uint64 shardId)
if (neededBytes < 0)
{
ereport(ERROR, (errcode(ERRCODE_OUT_OF_MEMORY),
errmsg("out of memory: %s", strerror(errno))));
errmsg("out of memory: %m")));
}
else if (neededBytes >= NAMEDATALEN)
{

View File

@ -309,10 +309,13 @@ GetRangeTblKind(RangeTblEntry *rte)
case RTE_JOIN:
case RTE_VALUES:
case RTE_CTE:
{
rteKind = (CitusRTEKind) rte->rtekind;
break;
}
#if PG_VERSION_NUM >= 120000
case RTE_RESULT:
#endif
{
rteKind = (CitusRTEKind) rte->rtekind;
break;
}
case RTE_FUNCTION:
{

View File

@ -45,9 +45,15 @@ CitusSetTag(Node *node, int tag)
nodeTypeName *local_node = (nodeTypeName *) CitusSetTag((Node *) node, T_##nodeTypeName)
/* And a few guys need only the pg_strtok support fields */
#if PG_VERSION_NUM >= 120000
#define READ_TEMP_LOCALS() \
char *token; \
const char *token; \
int length
#else
#define READ_TEMP_LOCALS() \
char *token; \
int length
#endif
/* ... but most need both */
#define READ_LOCALS(nodeTypeName) \

View File

@ -128,7 +128,11 @@ get_extension_schema(Oid ext_oid)
rel = heap_open(ExtensionRelationId, AccessShareLock);
ScanKeyInit(&entry[0],
#if PG_VERSION_NUM >= 120000
Anum_pg_extension_oid,
#else
ObjectIdAttributeNumber,
#endif
BTEqualStrategyNumber, F_OIDEQ,
ObjectIdGetDatum(ext_oid));

View File

@ -13,7 +13,6 @@
#include "citus_version.h"
#include "fmgr.h"
#include "utils/uuid.h"
#include "utils/backend_random.h"
bool EnableStatisticsCollection = true; /* send basic usage statistics to Citus */
@ -600,11 +599,11 @@ citus_server_id(PG_FUNCTION_ARGS)
uint8 *buf = (uint8 *) palloc(UUID_LEN);
/*
* If pg_backend_random() fails, fall-back to using random(). In previous
* versions of postgres we don't have pg_backend_random(), so use it by
* If pg_strong_random() fails, fall-back to using random(). In previous
* versions of postgres we don't have pg_strong_random(), so use it by
* default in that case.
*/
if (!pg_backend_random((char *) buf, UUID_LEN))
if (!pg_strong_random((char *) buf, UUID_LEN))
{
int bufIdx = 0;
for (bufIdx = 0; bufIdx < UUID_LEN; bufIdx++)

View File

@ -17,6 +17,10 @@
#include "funcapi.h"
#include "miscadmin.h"
#if PG_VERSION_NUM >= 120000
#include "access/genam.h"
#include "access/table.h"
#endif
#include "access/htup_details.h"
#include "access/xact.h"
#include "catalog/dependency.h"
@ -35,7 +39,6 @@
#include "utils/builtins.h"
#include "utils/snapmgr.h"
#include "utils/syscache.h"
#include "utils/tqual.h"
/* Local functions forward declarations */
@ -263,7 +266,11 @@ Datum
worker_cleanup_job_schema_cache(PG_FUNCTION_ARGS)
{
Relation pgNamespace = NULL;
#if PG_VERSION_NUM >= 120000
TableScanDesc scanDescriptor = NULL;
#else
HeapScanDesc scanDescriptor = NULL;
#endif
ScanKey scanKey = NULL;
int scanKeyCount = 0;
HeapTuple heapTuple = NULL;
@ -271,7 +278,11 @@ worker_cleanup_job_schema_cache(PG_FUNCTION_ARGS)
CheckCitusVersion(ERROR);
pgNamespace = heap_open(NamespaceRelationId, AccessExclusiveLock);
#if PG_VERSION_NUM >= 120000
scanDescriptor = table_beginscan_catalog(pgNamespace, scanKeyCount, scanKey);
#else
scanDescriptor = heap_beginscan_catalog(pgNamespace, scanKeyCount, scanKey);
#endif
heapTuple = heap_getnext(scanDescriptor, ForwardScanDirection);
while (HeapTupleIsValid(heapTuple))
@ -362,7 +373,8 @@ RemoveJobSchema(StringInfo schemaName)
Datum schemaNameDatum = CStringGetDatum(schemaName->data);
Oid schemaId = InvalidOid;
schemaId = GetSysCacheOid(NAMESPACENAME, schemaNameDatum, 0, 0, 0);
schemaId = GetSysCacheOid1Compat(NAMESPACENAME, Anum_pg_namespace_oid,
schemaNameDatum);
if (OidIsValid(schemaId))
{
ObjectAddress schemaObject = { 0, 0, 0 };

View File

@ -24,6 +24,7 @@ extern bool SubqueryPushdown;
extern bool ShouldUseSubqueryPushDown(Query *originalQuery, Query *rewrittenQuery);
extern bool JoinTreeContainsSubquery(Query *query);
extern bool HasEmptyJoinTree(Query *query);
extern bool WhereClauseContainsSubquery(Query *query);
extern bool SafeToPushdownWindowFunction(Query *query, StringInfo *errorDetail);
extern MultiNode * SubqueryMultiNodeTree(Query *originalQuery,

View File

@ -16,7 +16,11 @@
#include "catalog/namespace.h"
#include "nodes/parsenodes.h"
#if (PG_VERSION_NUM >= 100000 && PG_VERSION_NUM < 110000)
#if (PG_VERSION_NUM >= 120000)
#include "optimizer/optimizer.h"
#endif
#if (PG_VERSION_NUM < 110000)
#include "access/hash.h"
#include "storage/fd.h"
@ -240,5 +244,39 @@ RangeVarGetRelidInternal(const RangeVar *relation, LOCKMODE lockmode, uint32 fla
#endif
#if PG_VERSION_NUM >= 120000
#define MakeSingleTupleTableSlotCompat MakeSingleTupleTableSlot
#define AllocSetContextCreateExtended AllocSetContextCreateInternal
#define NextCopyFromCompat NextCopyFrom
#define ArrayRef SubscriptingRef
#define T_ArrayRef T_SubscriptingRef
#define or_clause is_orclause
#define GetSysCacheOid1Compat GetSysCacheOid1
#define GetSysCacheOid2Compat GetSysCacheOid2
#define GetSysCacheOid3Compat GetSysCacheOid3
#define GetSysCacheOid4Compat GetSysCacheOid4
#else /* pre PG12 */
#define QTW_EXAMINE_RTES_BEFORE QTW_EXAMINE_RTES
#define MakeSingleTupleTableSlotCompat(tupleDesc, tts_opts) \
MakeSingleTupleTableSlot(tupleDesc)
#define NextCopyFromCompat(cstate, econtext, values, nulls) \
NextCopyFrom(cstate, econtext, values, nulls, NULL)
/*
* In PG12 GetSysCacheOid requires an oid column,
* whereas beforehand the oid column was implicit with WITH OIDS
*/
#define GetSysCacheOid1Compat(cacheId, oidcol, key1) \
GetSysCacheOid1(cacheId, key1)
#define GetSysCacheOid2Compat(cacheId, oidcol, key1, key2) \
GetSysCacheOid2(cacheId, key1, key2)
#define GetSysCacheOid3Compat(cacheId, oidcol, key1, key2, key3) \
GetSysCacheOid3(cacheId, key1, key2, key3)
#define GetSysCacheOid4Compat(cacheId, oidcol, key1, key2, key3, key4) \
GetSysCacheOid4(cacheId, key1, key2, key3, key4)
#endif /* PG12 */
#endif /* VERSION_COMPAT_H */